You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by GitBox <gi...@apache.org> on 2022/08/10 17:22:34 UTC

[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #5379: AWS: Use executor service by default when performing batch deletion of files

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


##########
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));

Review Comment:
   nit: I think you can use `path` instead of `failedPath` to stay on one line



##########
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:
   This is why I usually recommend using `Tasks` for any concurrent code. The current implementation seems reasonable and starts execution as soon as there is a batch so we can keep it.
   
   Just as a reference, one could use `BinPacking` and `Tasks` for this type of work in the future .



##########
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()));

Review Comment:
   nit: Would defining extra vars help readability?
   
   ```
   String bucket = bucketToObjectsEntry.getKey();
   Collection<String> keys = bucketToObjectsEntry.getValue();
   Future<List<String>> deletionTask =
       executorService().submit(() -> deleteBatch(bucket, keys));
   ```



-- 
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-unsubscribe@iceberg.apache.org

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


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