singhpk234 commented on code in PR #5379:
URL: https://github.com/apache/iceberg/pull/5379#discussion_r942011801


##########
aws/src/main/java/org/apache/iceberg/aws/s3/S3FileIO.java:
##########
@@ -244,26 +260,33 @@ private void tagFileToDelete(String path, Set<Tag> 
deleteTags) throws S3Exceptio
     client().putObjectTagging(putObjectTaggingRequest);
   }
 
-  private List<String> deleteObjectsInBucket(String bucket, Collection<String> 
objects) {
-    if (!objects.isEmpty()) {
-      List<ObjectIdentifier> objectIds =
-          objects.stream()
-              .map(objectKey -> 
ObjectIdentifier.builder().key(objectKey).build())
-              .collect(Collectors.toList());
-      DeleteObjectsRequest deleteObjectsRequest =
-          DeleteObjectsRequest.builder()
-              .bucket(bucket)
-              .delete(Delete.builder().objects(objectIds).build())
-              .build();
-      DeleteObjectsResponse response = 
client().deleteObjects(deleteObjectsRequest);
-      if (response.hasErrors()) {
-        return response.errors().stream()
-            .map(error -> String.format("s3://%s/%s", bucket, error.key()))
+  private List<String> deleteBatch(String bucket, Collection<String> 
keysToDelete) {
+    List<ObjectIdentifier> objectIds =
+        keysToDelete.stream()
+            .map(key -> ObjectIdentifier.builder().key(key).build())
             .collect(Collectors.toList());
+    DeleteObjectsRequest request =
+        DeleteObjectsRequest.builder()
+            .bucket(bucket)
+            .delete(Delete.builder().objects(objectIds).build())
+            .build();
+    List<String> failures = Lists.newArrayList();
+    try {
+      DeleteObjectsResponse response = client().deleteObjects(request);
+      if (response.hasErrors()) {
+        failures.addAll(
+            response.errors().stream()
+                .map(error -> String.format("s3://%s/%s", request.bucket(), 
error.key()))
+                .collect(Collectors.toList()));
       }
+    } catch (Exception e) {

Review Comment:
   [doubt] Any reason we are catching a generic exception here ?



##########
aws/src/main/java/org/apache/iceberg/aws/s3/S3FileIO.java:
##########
@@ -182,41 +184,55 @@ public void deleteFiles(Iterable<String> paths) throws 
BulkDeletionFailureExcept
           .run(path -> tagFileToDelete(path, awsProperties.s3DeleteTags()));
     }
 
-    if (!awsProperties.isS3DeleteEnabled()) {
-      return;
-    }
+    if (awsProperties.isS3DeleteEnabled()) {
+      SetMultimap<String, String> bucketToObjects =
+          Multimaps.newSetMultimap(Maps.newHashMap(), Sets::newHashSet);
+      List<Future<List<String>>> deletionTasks = Lists.newArrayList();
+      for (String path : paths) {
+        S3URI location = new S3URI(path, 
awsProperties.s3BucketToAccessPointMapping());
+        String bucket = location.bucket();
+        String objectKey = location.key();
+        bucketToObjects.get(bucket).add(objectKey);
+        if (bucketToObjects.get(bucket).size() == 
awsProperties.s3FileIoDeleteBatchSize()) {
+          Set<String> keys = Sets.newHashSet(bucketToObjects.get(bucket));
+          Future<List<String>> deletionTask =
+              executorService().submit(() -> deleteBatch(bucket, keys));
+          deletionTasks.add(deletionTask);
+          bucketToObjects.removeAll(bucket);
+        }
+      }
 
-    SetMultimap<String, String> bucketToObjects =
-        Multimaps.newSetMultimap(Maps.newHashMap(), Sets::newHashSet);
-    int numberOfFailedDeletions = 0;
-    for (String path : paths) {
-      S3URI location = new S3URI(path, 
awsProperties.s3BucketToAccessPointMapping());
-      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);
-        numberOfFailedDeletions += failedDeletionsForBatch.size();
-        failedDeletionsForBatch.forEach(
-            failedPath -> LOG.warn("Failed to delete object at path {}", 
failedPath));
-        bucketToObjects.removeAll(bucket);
+      // Delete the remainder
+      for (Map.Entry<String, Collection<String>> bucketToObjectsEntry :
+          bucketToObjects.asMap().entrySet()) {
+        Future<List<String>> deletionTask =
+            executorService()
+                .submit(
+                    () ->
+                        deleteBatch(
+                            bucketToObjectsEntry.getKey(), 
bucketToObjectsEntry.getValue()));
+        deletionTasks.add(deletionTask);
       }
-      bucketToObjects.get(bucket).add(objectKey);
-    }
 
-    // Delete the remainder
-    for (Map.Entry<String, Collection<String>> bucketToObjectsEntry :
-        bucketToObjects.asMap().entrySet()) {
-      final String bucket = bucketToObjectsEntry.getKey();
-      final Collection<String> objects = bucketToObjectsEntry.getValue();
-      List<String> failedDeletions = deleteObjectsInBucket(bucket, objects);
-      failedDeletions.forEach(
-          failedPath -> LOG.warn("Failed to delete object at path {}", 
failedPath));
-      numberOfFailedDeletions += failedDeletions.size();
-    }
+      int totalFailedDeletions = 0;
+
+      for (Future<List<String>> deletionTask : deletionTasks) {
+        try {
+          List<String> failedDeletions = deletionTask.get();
+          failedDeletions.forEach(
+              failedPath -> LOG.warn("Failed to delete object at path {}", 
failedPath));
+          totalFailedDeletions += failedDeletions.size();
+        } catch (ExecutionException e) {
+          LOG.warn("Caught unexpected exception during batch deletion: ", 
e.getCause());
+        } catch (InterruptedException e) {
+          Thread.currentThread().interrupt();
+          throw new RuntimeException("Interrupted when waiting for deletions 
to complete", e);

Review Comment:
   [question] if we are going to throw an exception here, should we cancel the 
remaining futures just in-case they haven't completed, Thoughts ? 



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