Skip to content
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@
import java.util.Map;
import java.util.concurrent.ExecutorService;
import java.util.function.Consumer;
import org.apache.iceberg.io.SupportsBulkOperations;
import org.apache.iceberg.relocated.com.google.common.base.Preconditions;

/**
Expand Down Expand Up @@ -59,7 +60,7 @@ public interface DeleteOrphanFiles extends Action<DeleteOrphanFiles, DeleteOrpha
/**
* Passes an alternative delete implementation that will be used for orphan files.
*
* <p>This method allows users to customize the delete func. For example, one may set a custom
* <p>This method allows users to customize the delete function. For example, one may set a custom
* delete func and collect all orphan files into a set instead of physically removing them.
*
* <p>If not set, defaults to using the table's {@link org.apache.iceberg.io.FileIO io}
Expand All @@ -71,12 +72,14 @@ public interface DeleteOrphanFiles extends Action<DeleteOrphanFiles, DeleteOrpha
DeleteOrphanFiles deleteWith(Consumer<String> deleteFunc);

/**
* Passes an alternative executor service that will be used for removing orphaned files.
*
* <p>If this method is not called, orphaned manifests and data files will still be deleted in the
* current thread.
*
* <p>
* Passes an alternative executor service that will be used for removing orphaned files. This
* service will only be used if a custom delete function is provided by {@link
* #deleteWith(Consumer)} or if the FileIO does not {@link SupportsBulkOperations support bulk
* deletes}. Otherwise, parallelism should be controlled by the IO specific {@link
* SupportsBulkOperations#deleteFiles(Iterable) deleteFiles} method.
*
* <p>If this method is not called and bulk deletes are not supported, orphaned manifests and data
* files will still be deleted in the current thread.
*
* @param executorService the service to use
* @return this for method chaining
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@
import java.util.concurrent.ExecutorService;
import java.util.function.Consumer;
import org.apache.iceberg.io.FileIO;
import org.apache.iceberg.io.SupportsBulkOperations;

/**
* An action that deletes all files referenced by a table metadata file.
Expand All @@ -44,9 +45,11 @@ public interface DeleteReachableFiles
DeleteReachableFiles deleteWith(Consumer<String> deleteFunc);

/**
* Passes an alternative executor service that will be used for files removal.
*
* <p>If this method is not called, files will be deleted in the current thread.
* Passes an alternative executor service that will be used for files removal. This service will
* only be used if a custom delete function is provided by {@link #deleteWith(Consumer)} or if the
* FileIO does not {@link SupportsBulkOperations support bulk deletes}. Otherwise, parallelism
* should be controlled by the IO specific {@link SupportsBulkOperations#deleteFiles(Iterable)
* deleteFiles} method.
*
* @param executorService the service to use
* @return this for method chaining
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@
import java.util.concurrent.ExecutorService;
import java.util.function.Consumer;
import org.apache.iceberg.Snapshot;
import org.apache.iceberg.io.SupportsBulkOperations;

/**
* An action that expires snapshots in a table.
Expand Down Expand Up @@ -80,11 +81,14 @@ public interface ExpireSnapshots extends Action<ExpireSnapshots, ExpireSnapshots
ExpireSnapshots deleteWith(Consumer<String> deleteFunc);

/**
* Passes an alternative executor service that will be used for manifests, data and delete files
* deletion.
* Passes an alternative executor service that will be used for files removal. This service will
* only be used if a custom delete function is provided by {@link #deleteWith(Consumer)} or if the
* FileIO does not {@link SupportsBulkOperations support bulk deletes}. Otherwise, parallelism
* should be controlled by the IO specific {@link SupportsBulkOperations#deleteFiles(Iterable)
* deleteFiles} method.
*
* <p>If this method is not called, unnecessary manifests and content files will still be deleted
* in the current thread.
* <p>If this method is not called and bulk deletes are not supported, unnecessary manifests and
* content files will still be deleted in the current thread.
*
* <p>Identical to {@link org.apache.iceberg.ExpireSnapshots#executeDeleteWith(ExecutorService)}
*
Expand Down
56 changes: 55 additions & 1 deletion core/src/main/java/org/apache/iceberg/hadoop/HadoopFileIO.java
Original file line number Diff line number Diff line change
Expand Up @@ -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());
Expand Down Expand Up @@ -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)
.suppressFailureWhenFinished()
.onFailure(
(f, e) -> {
LOG.error("Failure during bulk delete on file: {} ", f, e);
failureCount.incrementAndGet();
Copy link
Contributor

Choose a reason for hiding this comment

The 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 Iterables.size(pathsToDelete) to find how many we were supposed to delete.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Ah I thought it was once per element

Copy link
Member Author

Choose a reason for hiding this comment

The 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

Copy link
Member Author

Choose a reason for hiding this comment

The 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

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

runTaskWithRetry(task, item);
succeeded.add(item);
} catch (Exception e) {
exceptions.add(e);
if (onFailure != null) {
tryRunOnFailure(item, e);
Code in question (RunWithRetry) does all retries before hitting "onFailure"

Copy link
Contributor

Choose a reason for hiding this comment

The 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 SparkCleanupUtil to follow this patter as well.

})
.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.
*
Expand Down
30 changes: 28 additions & 2 deletions core/src/test/java/org/apache/iceberg/hadoop/HadoopFileIOTest.java
Original file line number Diff line number Diff line change
Expand Up @@ -27,10 +27,12 @@
import java.util.List;
import java.util.Random;
import java.util.UUID;
import java.util.stream.Collectors;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.iceberg.TestHelpers;
import org.apache.iceberg.io.BulkDeletionFailureException;
import org.apache.iceberg.io.FileIO;
import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
import org.apache.iceberg.relocated.com.google.common.collect.Lists;
Expand Down Expand Up @@ -120,6 +122,26 @@ public void testDeletePrefix() {
() -> hadoopFileIO.listPrefix(parent.toUri().toString()).iterator());
}

@Test
public void testDeleteFiles() {
Path parent = new Path(tempDir.toURI());
List<Path> filesCreated = createRandomFiles(parent, 10);
hadoopFileIO.deleteFiles(
filesCreated.stream().map(Path::toString).collect(Collectors.toList()));
filesCreated.forEach(
file -> Assert.assertFalse(hadoopFileIO.newInputFile(file.toString()).exists()));
}

@Test
public void testDeleteFilesErrorHandling() {
List<String> filesCreated =
random.ints(2).mapToObj(x -> "fakefsnotreal://file-" + x).collect(Collectors.toList());
Assert.assertThrows(
"Should throw a BulkDeletionFailure Exceptions when files can't be deleted",
BulkDeletionFailureException.class,
() -> hadoopFileIO.deleteFiles(filesCreated));
}

@Test
public void testHadoopFileIOKryoSerialization() throws IOException {
FileIO testHadoopFileIO = new HadoopFileIO();
Expand All @@ -142,17 +164,21 @@ public void testHadoopFileIOJavaSerialization() throws IOException, ClassNotFoun
Assert.assertEquals(testHadoopFileIO.properties(), roundTripSerializedFileIO.properties());
}

private void createRandomFiles(Path parent, int count) {
private List<Path> createRandomFiles(Path parent, int count) {
List<Path> paths = Lists.newArrayList();
random
.ints(count)
.parallel()
.forEach(
i -> {
try {
fs.createNewFile(new Path(parent, "file-" + i));
Path path = new Path(parent, "file-" + i);
paths.add(path);
fs.createNewFile(path);
} catch (IOException e) {
throw new UncheckedIOException(e);
}
});
return paths;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@
import static org.apache.spark.sql.functions.col;
import static org.apache.spark.sql.functions.lit;

import java.util.Collection;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
Expand All @@ -46,14 +47,19 @@
import org.apache.iceberg.TableMetadata;
import org.apache.iceberg.exceptions.NotFoundException;
import org.apache.iceberg.exceptions.ValidationException;
import org.apache.iceberg.io.BulkDeletionFailureException;
import org.apache.iceberg.io.CloseableIterator;
import org.apache.iceberg.io.ClosingIterator;
import org.apache.iceberg.io.FileIO;
import org.apache.iceberg.io.SupportsBulkOperations;
import org.apache.iceberg.relocated.com.google.common.base.Joiner;
import org.apache.iceberg.relocated.com.google.common.base.Splitter;
import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
import org.apache.iceberg.relocated.com.google.common.collect.Iterators;
import org.apache.iceberg.relocated.com.google.common.collect.ListMultimap;
import org.apache.iceberg.relocated.com.google.common.collect.Lists;
import org.apache.iceberg.relocated.com.google.common.collect.Maps;
import org.apache.iceberg.relocated.com.google.common.collect.Multimaps;
import org.apache.iceberg.spark.JobGroupInfo;
import org.apache.iceberg.spark.JobGroupUtils;
import org.apache.iceberg.spark.SparkTableUtil;
Expand Down Expand Up @@ -85,6 +91,7 @@ abstract class BaseSparkAction<ThisT> {
private static final Logger LOG = LoggerFactory.getLogger(BaseSparkAction.class);
private static final AtomicInteger JOB_COUNTER = new AtomicInteger();
private static final int DELETE_NUM_RETRIES = 3;
private static final int DELETE_GROUP_SIZE = 100000;

private final SparkSession spark;
private final JavaSparkContext sparkContext;
Expand Down Expand Up @@ -253,6 +260,37 @@ protected DeleteSummary deleteFiles(
return summary;
}

protected DeleteSummary deleteFiles(SupportsBulkOperations io, Iterator<FileInfo> files) {
DeleteSummary summary = new DeleteSummary();
Iterator<List<FileInfo>> fileGroups = Iterators.partition(files, DELETE_GROUP_SIZE);

Tasks.foreach(fileGroups)
.suppressFailureWhenFinished()
.run(fileGroup -> deleteFileGroup(fileGroup, io, summary));

return summary;
}

private static void deleteFileGroup(
List<FileInfo> fileGroup, SupportsBulkOperations io, DeleteSummary summary) {

ListMultimap<String, FileInfo> filesByType = Multimaps.index(fileGroup, FileInfo::getType);
ListMultimap<String, String> pathsByType =
Multimaps.transformValues(filesByType, FileInfo::getPath);

for (Map.Entry<String, Collection<String>> entry : pathsByType.asMap().entrySet()) {
String type = entry.getKey();
Collection<String> paths = entry.getValue();
int failures = 0;
try {
io.deleteFiles(paths);
} catch (BulkDeletionFailureException e) {
failures = e.numberFailedObjects();
}
summary.deletedFiles(type, paths.size() - failures);
}
}

static class DeleteSummary {
private final AtomicLong dataFilesCount = new AtomicLong(0L);
private final AtomicLong positionDeleteFilesCount = new AtomicLong(0L);
Expand All @@ -261,6 +299,30 @@ static class DeleteSummary {
private final AtomicLong manifestListsCount = new AtomicLong(0L);
private final AtomicLong otherFilesCount = new AtomicLong(0L);

public void deletedFiles(String type, int numFiles) {
if (FileContent.DATA.name().equalsIgnoreCase(type)) {
dataFilesCount.addAndGet(numFiles);

} else if (FileContent.POSITION_DELETES.name().equalsIgnoreCase(type)) {
positionDeleteFilesCount.addAndGet(numFiles);

} else if (FileContent.EQUALITY_DELETES.name().equalsIgnoreCase(type)) {
equalityDeleteFilesCount.addAndGet(numFiles);

} else if (MANIFEST.equalsIgnoreCase(type)) {
manifestsCount.addAndGet(numFiles);

} else if (MANIFEST_LIST.equalsIgnoreCase(type)) {
manifestListsCount.addAndGet(numFiles);

} else if (OTHERS.equalsIgnoreCase(type)) {
otherFilesCount.addAndGet(numFiles);

} else {
throw new ValidationException("Illegal file type: %s", type);
}
}

public void deletedFile(String path, String type) {
if (FileContent.DATA.name().equalsIgnoreCase(type)) {
dataFilesCount.incrementAndGet();
Expand Down
Loading