danielcweeks commented on a change in pull request #4052:
URL: https://github.com/apache/iceberg/pull/4052#discussion_r818180293



##########
File path: aws/src/main/java/org/apache/iceberg/aws/s3/S3FileIO.java
##########
@@ -100,6 +115,67 @@ public void deleteFile(String path) {
     client().deleteObject(deleteRequest);
   }
 
+  /**
+   * Deletes the given paths in a batched manner.
+   * <p>
+   * The paths are grouped by bucket, and deletion is triggered when we either 
reach the configured batch size
+   * or have a final remainder batch for each bucket.
+   *
+   * @param paths paths to delete
+   */
+  @Override
+  public void deleteFiles(Iterable<String> paths) {
+    SetMultimap<String, String> bucketToObjects = 
Multimaps.newSetMultimap(Maps.newHashMap(), Sets::newHashSet);
+    List<String> failedDeletions = Lists.newArrayList();
+    for (String path : paths) {
+      S3URI location = new S3URI(path);
+      String bucket = location.bucket();
+      String objectKey = location.key();
+      Set<String> objectsInBucket = bucketToObjects.get(bucket);
+      if (objectsInBucket.size() == awsProperties.s3FileIoDeleteBatchSize()) {
+        List<String> failedDeletionsForBatch = deleteObjectsInBucket(bucket, 
objectsInBucket);

Review comment:
       I feel like we should not rely on failure/exceptions and keep delete 
"best effort".  The reason for this is that the cleanup being done using FileIO 
delete (whether it's for failed/speculative tasks or other maintenance type 
clean) up is not a requirement for iceberg to function and will lead to 
spurious errors and inconsistency (as we see the normal delete path doesn't 
report errors in multiple implementations).
   
   Even in a delete failure case, the required action is to run the maintenance 
procedures to perform orphan file cleanup, so we end up propagating errors that 
could result in job failure due to issues that would normally not affect a job.
   
   I think it's fine to log failed deletes for debugging purposes, but we 
shouldn't rely on the deletes.
   
   Alternatively, we could keep this bulk functionality out of the FileIO 
implementation and add a "feature" interface like `SupportsBulkDelete` that can 
be used for S3 specifically.  Not all implementations of FileIO have a concept 
of bulk delete (S3 is the only one I'm aware of).  That would allow us to have 
a stricter contract around deletes for cleanup procedures.
   
   I'd actually prefer the latter as it keeps FileIO cleaner and we can 
separate contract behaviors without introducing inconsistency.  Actions may 
need additional feature not available (e.g. listing) that we don't want to add 
to the FileIO contract directly.
   
   




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