-
Notifications
You must be signed in to change notification settings - Fork 3k
Bulk delete #6682
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Bulk delete #6682
Changes from all commits
f9cdfe5
cf89f8c
40f4b2a
5480073
275ee77
01db68d
c253c51
c6b3cb0
cb28cba
0f032ca
66913e6
e81edc0
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change | ||||||||||||||
|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|
|
|
@@ -18,27 +18,44 @@ | |||||||||||||||
| */ | ||||||||||||||||
| package org.apache.iceberg.hadoop; | ||||||||||||||||
|
|
||||||||||||||||
| import java.io.FileNotFoundException; | ||||||||||||||||
| import java.io.IOException; | ||||||||||||||||
| import java.io.UncheckedIOException; | ||||||||||||||||
| import java.util.Iterator; | ||||||||||||||||
| import java.util.Map; | ||||||||||||||||
| import java.util.concurrent.ExecutorService; | ||||||||||||||||
| import java.util.concurrent.atomic.AtomicInteger; | ||||||||||||||||
| import java.util.function.Function; | ||||||||||||||||
| import org.apache.hadoop.conf.Configuration; | ||||||||||||||||
| import org.apache.hadoop.fs.FileSystem; | ||||||||||||||||
| import org.apache.hadoop.fs.Path; | ||||||||||||||||
| import org.apache.hadoop.fs.RemoteIterator; | ||||||||||||||||
| import org.apache.iceberg.exceptions.RuntimeIOException; | ||||||||||||||||
| import org.apache.iceberg.io.BulkDeletionFailureException; | ||||||||||||||||
| import org.apache.iceberg.io.FileIO; | ||||||||||||||||
| import org.apache.iceberg.io.FileInfo; | ||||||||||||||||
| import org.apache.iceberg.io.InputFile; | ||||||||||||||||
| import org.apache.iceberg.io.OutputFile; | ||||||||||||||||
| import org.apache.iceberg.io.SupportsBulkOperations; | ||||||||||||||||
| import org.apache.iceberg.io.SupportsPrefixOperations; | ||||||||||||||||
| import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; | ||||||||||||||||
| import org.apache.iceberg.relocated.com.google.common.collect.Streams; | ||||||||||||||||
| import org.apache.iceberg.util.SerializableMap; | ||||||||||||||||
| import org.apache.iceberg.util.SerializableSupplier; | ||||||||||||||||
| import org.apache.iceberg.util.Tasks; | ||||||||||||||||
| import org.apache.iceberg.util.ThreadPools; | ||||||||||||||||
| import org.slf4j.Logger; | ||||||||||||||||
| import org.slf4j.LoggerFactory; | ||||||||||||||||
|
|
||||||||||||||||
| public class HadoopFileIO implements FileIO, HadoopConfigurable, SupportsPrefixOperations { | ||||||||||||||||
| public class HadoopFileIO | ||||||||||||||||
| implements FileIO, HadoopConfigurable, SupportsPrefixOperations, SupportsBulkOperations { | ||||||||||||||||
|
|
||||||||||||||||
| private static final Logger LOG = LoggerFactory.getLogger(HadoopFileIO.class); | ||||||||||||||||
| private static final String DELETE_FILE_PARALLELISM = "iceberg.hadoop.delete-file-parallelism"; | ||||||||||||||||
| private static final String DELETE_FILE_POOL_NAME = "iceberg-hadoopfileio-delete"; | ||||||||||||||||
| private static final int DELETE_RETRY_ATTEMPTS = 3; | ||||||||||||||||
| private static final int DEFAULT_DELETE_CORE_MULTIPLE = 4; | ||||||||||||||||
| private static volatile ExecutorService executorService; | ||||||||||||||||
|
|
||||||||||||||||
| private SerializableSupplier<Configuration> hadoopConf; | ||||||||||||||||
| private SerializableMap<String, String> properties = SerializableMap.copyOf(ImmutableMap.of()); | ||||||||||||||||
|
|
@@ -149,6 +166,43 @@ public void deletePrefix(String prefix) { | |||||||||||||||
| } | ||||||||||||||||
| } | ||||||||||||||||
|
|
||||||||||||||||
| @Override | ||||||||||||||||
| public void deleteFiles(Iterable<String> pathsToDelete) throws BulkDeletionFailureException { | ||||||||||||||||
| AtomicInteger failureCount = new AtomicInteger(0); | ||||||||||||||||
| Tasks.foreach(pathsToDelete) | ||||||||||||||||
| .executeWith(executorService()) | ||||||||||||||||
| .retry(DELETE_RETRY_ATTEMPTS) | ||||||||||||||||
| .stopRetryOn(FileNotFoundException.class) | ||||||||||||||||
aokolnychyi marked this conversation as resolved.
Show resolved
Hide resolved
|
||||||||||||||||
| .suppressFailureWhenFinished() | ||||||||||||||||
| .onFailure( | ||||||||||||||||
| (f, e) -> { | ||||||||||||||||
| LOG.error("Failure during bulk delete on file: {} ", f, e); | ||||||||||||||||
| failureCount.incrementAndGet(); | ||||||||||||||||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This is going to increment the count on each failed attempt and won't be accurate. We could count the number of successfully deleted files instead and then use
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Ah I thought it was once per element
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I'm not sure we want to go over the iterable more than once ... let me think about this
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I double checked this, it only fires off when all retries are exhausted so it is correct as is. scala> def testFailure() = {
var failureCount =0
Tasks.foreach("value")
.retry(3)
.onFailure((y, x: Throwable) => failureCount += 1)
.suppressFailureWhenFinished()
.run(x => throw new Exception("ohNO"))
failureCount
}
scala> testFailure()
23/03/01 10:16:22 WARN Tasks: Retrying task after failure: ohNO
java.lang.Exception: ohNO
...
23/03/01 10:16:23 WARN Tasks: Retrying task after failure: ohNO
java.lang.Exception: ohNO
...
23/03/01 10:16:25 WARN Tasks: Retrying task after failure: ohNO
java.lang.Exception: ohNO
...
res21: Int = 1
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. iceberg/core/src/main/java/org/apache/iceberg/util/Tasks.java Lines 219 to 225 in 715c9b9
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. You are right, we overlooked it while reviewing another PR. I like it more. I'll update |
||||||||||||||||
| }) | ||||||||||||||||
| .run(this::deleteFile); | ||||||||||||||||
|
|
||||||||||||||||
| if (failureCount.get() != 0) { | ||||||||||||||||
| throw new BulkDeletionFailureException(failureCount.get()); | ||||||||||||||||
| } | ||||||||||||||||
| } | ||||||||||||||||
|
|
||||||||||||||||
| private int deleteThreads() { | ||||||||||||||||
| int defaultValue = Runtime.getRuntime().availableProcessors() * DEFAULT_DELETE_CORE_MULTIPLE; | ||||||||||||||||
| return conf().getInt(DELETE_FILE_PARALLELISM, defaultValue); | ||||||||||||||||
| } | ||||||||||||||||
|
|
||||||||||||||||
| private ExecutorService executorService() { | ||||||||||||||||
| if (executorService == null) { | ||||||||||||||||
| synchronized (HadoopFileIO.class) { | ||||||||||||||||
| if (executorService == null) { | ||||||||||||||||
| executorService = ThreadPools.newWorkerPool(DELETE_FILE_POOL_NAME, deleteThreads()); | ||||||||||||||||
| } | ||||||||||||||||
| } | ||||||||||||||||
| } | ||||||||||||||||
|
|
||||||||||||||||
| return executorService; | ||||||||||||||||
| } | ||||||||||||||||
|
|
||||||||||||||||
| /** | ||||||||||||||||
| * This class is a simple adaptor to allow for using Hadoop's RemoteIterator as an Iterator. | ||||||||||||||||
| * | ||||||||||||||||
|
|
||||||||||||||||
Uh oh!
There was an error while loading. Please reload this page.