danielcweeks commented on code in PR #10233:
URL: https://github.com/apache/iceberg/pull/10233#discussion_r2025719727


##########
core/src/main/java/org/apache/iceberg/hadoop/HadoopFileIO.java:
##########
@@ -173,10 +207,49 @@ public void deletePrefix(String prefix) {
     }
   }
 
+  /**
+   * Delete files.
+   *
+   * <p>If the Hadoop Bulk Delete API is available, this API is used through 
{@link
+   * #hadoopBulkDelete(Iterable)}. Otherwise, each file is deleted 
individually in a thread pool.
+   *
+   * @param pathsToDelete The paths to delete
+   * @throws BulkDeletionFailureException failure to delete one or more files.
+   */
   @Override
-  public void deleteFiles(Iterable<String> pathsToDelete) throws 
BulkDeletionFailureException {
+  public void deleteFiles(final Iterable<String> pathsToDelete)
+      throws BulkDeletionFailureException {
+    Iterable<String> targetPaths = pathsToDelete;
+    if (HADOOP_BULK_DELETE.get()) {
+      // hadoop bulk delete.
+      try {
+        final List<Map.Entry<Path, String>> pathsNotDeleted = 
hadoopBulkDelete(targetPaths);
+        if (pathsNotDeleted.isEmpty()) {
+          // the normal outcome: everything was deleted.
+          return;
+        }
+        // the bad outcome: one or more files were not deleted.
+        // those files will be handed down to the classic per-file deletion 
loop for retrying.
+        // Transient failures may be recovered from, while unrecoverable 
failures
+        // will have errors reported below.
+        targetPaths =
+            pathsNotDeleted.stream()
+                .map(
+                    entry -> {
+                      LOG.info("Failed to delete {} cause: {}", 
entry.getKey(), entry.getValue());
+                      return entry.getKey().toString();
+                    })
+                .collect(Collectors.toList());
+      } catch (RuntimeException e) {
+        // Something went wrong.
+        // log then fall through to the classic delete
+        LOG.warn("Failed to use bulk delete -falling back to single delete 
calls", e);
+      }
+    }

Review Comment:
   I don't think we need this whole fallback logic.  If bulk delete is 
available, we shouldn't assume that it might not work.  We're not trying to 
make strong guarantees about deletes, so we should just rely on the bulk to do 
the work and we can either log or throw `BulkDeletionFailureException` like 
other implementations.



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

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

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


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

Reply via email to