diff --git a/api/src/main/java/org/apache/iceberg/actions/DeleteOrphanFiles.java b/api/src/main/java/org/apache/iceberg/actions/DeleteOrphanFiles.java index 0e00eb67b217..4e8f80fa833f 100644 --- a/api/src/main/java/org/apache/iceberg/actions/DeleteOrphanFiles.java +++ b/api/src/main/java/org/apache/iceberg/actions/DeleteOrphanFiles.java @@ -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; /** @@ -59,7 +60,7 @@ public interface DeleteOrphanFiles extends ActionThis method allows users to customize the delete func. For example, one may set a custom + *

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

If not set, defaults to using the table's {@link org.apache.iceberg.io.FileIO io} @@ -71,12 +72,14 @@ public interface DeleteOrphanFiles extends Action deleteFunc); /** - * Passes an alternative executor service that will be used for removing orphaned files. - * - *

If this method is not called, orphaned manifests and data files will still be deleted in the - * current thread. - * - *

+ * 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. + * + *

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 diff --git a/api/src/main/java/org/apache/iceberg/actions/DeleteReachableFiles.java b/api/src/main/java/org/apache/iceberg/actions/DeleteReachableFiles.java index aa15ded71450..d4ebb5a2007c 100644 --- a/api/src/main/java/org/apache/iceberg/actions/DeleteReachableFiles.java +++ b/api/src/main/java/org/apache/iceberg/actions/DeleteReachableFiles.java @@ -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. @@ -44,9 +45,11 @@ public interface DeleteReachableFiles DeleteReachableFiles deleteWith(Consumer deleteFunc); /** - * Passes an alternative executor service that will be used for files removal. - * - *

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 diff --git a/api/src/main/java/org/apache/iceberg/actions/ExpireSnapshots.java b/api/src/main/java/org/apache/iceberg/actions/ExpireSnapshots.java index 2b1924c4d720..0e02f4bec964 100644 --- a/api/src/main/java/org/apache/iceberg/actions/ExpireSnapshots.java +++ b/api/src/main/java/org/apache/iceberg/actions/ExpireSnapshots.java @@ -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. @@ -80,11 +81,14 @@ public interface ExpireSnapshots extends Action 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. * - *

If this method is not called, unnecessary manifests and content files will still be deleted - * in the current thread. + *

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

Identical to {@link org.apache.iceberg.ExpireSnapshots#executeDeleteWith(ExecutorService)} * diff --git a/core/src/main/java/org/apache/iceberg/hadoop/HadoopFileIO.java b/core/src/main/java/org/apache/iceberg/hadoop/HadoopFileIO.java index 8f34994d6374..04ead0bd6791 100644 --- a/core/src/main/java/org/apache/iceberg/hadoop/HadoopFileIO.java +++ b/core/src/main/java/org/apache/iceberg/hadoop/HadoopFileIO.java @@ -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 hadoopConf; private SerializableMap properties = SerializableMap.copyOf(ImmutableMap.of()); @@ -149,6 +166,43 @@ public void deletePrefix(String prefix) { } } + @Override + public void deleteFiles(Iterable 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(); + }) + .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. * diff --git a/core/src/test/java/org/apache/iceberg/hadoop/HadoopFileIOTest.java b/core/src/test/java/org/apache/iceberg/hadoop/HadoopFileIOTest.java index 2cf375592f7d..7920eae54be5 100644 --- a/core/src/test/java/org/apache/iceberg/hadoop/HadoopFileIOTest.java +++ b/core/src/test/java/org/apache/iceberg/hadoop/HadoopFileIOTest.java @@ -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; @@ -120,6 +122,26 @@ public void testDeletePrefix() { () -> hadoopFileIO.listPrefix(parent.toUri().toString()).iterator()); } + @Test + public void testDeleteFiles() { + Path parent = new Path(tempDir.toURI()); + List 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 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(); @@ -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 createRandomFiles(Path parent, int count) { + List 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; } } diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/BaseSparkAction.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/BaseSparkAction.java index cdd80040fa9e..1b285e8caca8 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/BaseSparkAction.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/BaseSparkAction.java @@ -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; @@ -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; @@ -85,6 +91,7 @@ abstract class BaseSparkAction { 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; @@ -253,6 +260,37 @@ protected DeleteSummary deleteFiles( return summary; } + protected DeleteSummary deleteFiles(SupportsBulkOperations io, Iterator files) { + DeleteSummary summary = new DeleteSummary(); + Iterator> fileGroups = Iterators.partition(files, DELETE_GROUP_SIZE); + + Tasks.foreach(fileGroups) + .suppressFailureWhenFinished() + .run(fileGroup -> deleteFileGroup(fileGroup, io, summary)); + + return summary; + } + + private static void deleteFileGroup( + List fileGroup, SupportsBulkOperations io, DeleteSummary summary) { + + ListMultimap filesByType = Multimaps.index(fileGroup, FileInfo::getType); + ListMultimap pathsByType = + Multimaps.transformValues(filesByType, FileInfo::getPath); + + for (Map.Entry> entry : pathsByType.asMap().entrySet()) { + String type = entry.getKey(); + Collection 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); @@ -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(); diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/DeleteOrphanFilesSparkAction.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/DeleteOrphanFilesSparkAction.java index 1abd2107ed7f..ea73403c2e60 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/DeleteOrphanFilesSparkAction.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/DeleteOrphanFilesSparkAction.java @@ -48,6 +48,8 @@ import org.apache.iceberg.actions.DeleteOrphanFiles; import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.hadoop.HiddenPathFilter; +import org.apache.iceberg.io.BulkDeletionFailureException; +import org.apache.iceberg.io.SupportsBulkOperations; import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.base.Strings; @@ -111,21 +113,13 @@ public class DeleteOrphanFilesSparkAction extends BaseSparkAction defaultDelete = - new Consumer() { - @Override - public void accept(String file) { - table.io().deleteFile(file); - } - }; - private Map equalSchemes = flattenMap(EQUAL_SCHEMES_DEFAULT); private Map equalAuthorities = Collections.emptyMap(); private PrefixMismatchMode prefixMismatchMode = PrefixMismatchMode.ERROR; private String location = null; private long olderThanTimestamp = System.currentTimeMillis() - TimeUnit.DAYS.toMillis(3); private Dataset compareToFileList; - private Consumer deleteFunc = defaultDelete; + private Consumer deleteFunc = null; private ExecutorService deleteExecutorService = null; DeleteOrphanFilesSparkAction(SparkSession spark, Table table) { @@ -239,6 +233,16 @@ private String jobDesc() { return String.format("Deleting orphan files (%s) from %s", optionsAsString, table.name()); } + private void deleteFiles(SupportsBulkOperations io, List paths) { + try { + io.deleteFiles(paths); + LOG.info("Deleted {} files using bulk deletes", paths.size()); + } catch (BulkDeletionFailureException e) { + int deletedFilesCount = paths.size() - e.numberFailedObjects(); + LOG.warn("Deleted only {} of {} files using bulk deletes", deletedFilesCount, paths.size()); + } + } + private DeleteOrphanFiles.Result doExecute() { Dataset actualFileIdentDS = actualFileIdentDS(); Dataset validFileIdentDS = validFileIdentDS(); @@ -246,12 +250,27 @@ private DeleteOrphanFiles.Result doExecute() { List orphanFiles = findOrphanFiles(spark(), actualFileIdentDS, validFileIdentDS, prefixMismatchMode); - Tasks.foreach(orphanFiles) - .noRetry() - .executeWith(deleteExecutorService) - .suppressFailureWhenFinished() - .onFailure((file, exc) -> LOG.warn("Failed to delete file: {}", file, exc)) - .run(deleteFunc::accept); + if (deleteFunc == null && table.io() instanceof SupportsBulkOperations) { + deleteFiles((SupportsBulkOperations) table.io(), orphanFiles); + } else { + + Tasks.Builder deleteTasks = + Tasks.foreach(orphanFiles) + .noRetry() + .executeWith(deleteExecutorService) + .suppressFailureWhenFinished() + .onFailure((file, exc) -> LOG.warn("Failed to delete file: {}", file, exc)); + + if (deleteFunc == null) { + LOG.info( + "Table IO {} does not support bulk operations. Using non-bulk deletes.", + table.io().getClass().getName()); + deleteTasks.run(table.io()::deleteFile); + } else { + LOG.info("Custom delete function provided. Using non-bulk deletes"); + deleteTasks.run(deleteFunc::accept); + } + } return new BaseDeleteOrphanFilesActionResult(orphanFiles); } diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/DeleteReachableFilesSparkAction.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/DeleteReachableFilesSparkAction.java index 0f01afa287bb..cdc60a659d7c 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/DeleteReachableFilesSparkAction.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/DeleteReachableFilesSparkAction.java @@ -32,6 +32,7 @@ import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.hadoop.HadoopFileIO; import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.SupportsBulkOperations; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.spark.JobGroupInfo; import org.apache.iceberg.util.PropertyUtil; @@ -54,15 +55,8 @@ public class DeleteReachableFilesSparkAction private static final Logger LOG = LoggerFactory.getLogger(DeleteReachableFilesSparkAction.class); private final String metadataFileLocation; - private final Consumer defaultDelete = - new Consumer() { - @Override - public void accept(String file) { - io.deleteFile(file); - } - }; - - private Consumer deleteFunc = defaultDelete; + + private Consumer deleteFunc = null; private ExecutorService deleteExecutorService = null; private FileIO io = new HadoopFileIO(spark().sessionState().newHadoopConf()); @@ -132,7 +126,22 @@ private Dataset reachableFileDS(TableMetadata metadata) { } private DeleteReachableFiles.Result deleteFiles(Iterator files) { - DeleteSummary summary = deleteFiles(deleteExecutorService, deleteFunc, files); + DeleteSummary summary; + if (deleteFunc == null && io instanceof SupportsBulkOperations) { + summary = deleteFiles((SupportsBulkOperations) io, files); + } else { + + if (deleteFunc == null) { + LOG.info( + "Table IO {} does not support bulk operations. Using non-bulk deletes.", + io.getClass().getName()); + summary = deleteFiles(deleteExecutorService, io::deleteFile, files); + } else { + LOG.info("Custom delete function provided. Using non-bulk deletes"); + summary = deleteFiles(deleteExecutorService, deleteFunc, files); + } + } + LOG.info("Deleted {} total files", summary.totalFilesCount()); return new BaseDeleteReachableFilesActionResult( diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/ExpireSnapshotsSparkAction.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/ExpireSnapshotsSparkAction.java index d9af48c221f1..95e153a9a5a6 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/ExpireSnapshotsSparkAction.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/ExpireSnapshotsSparkAction.java @@ -35,6 +35,7 @@ import org.apache.iceberg.actions.BaseExpireSnapshotsActionResult; import org.apache.iceberg.actions.ExpireSnapshots; import org.apache.iceberg.exceptions.ValidationException; +import org.apache.iceberg.io.SupportsBulkOperations; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Sets; @@ -74,18 +75,11 @@ public class ExpireSnapshotsSparkAction extends BaseSparkAction defaultDelete = - new Consumer() { - @Override - public void accept(String file) { - ops.io().deleteFile(file); - } - }; private final Set expiredSnapshotIds = Sets.newHashSet(); private Long expireOlderThanValue = null; private Integer retainLastValue = null; - private Consumer deleteFunc = defaultDelete; + private Consumer deleteFunc = null; private ExecutorService deleteExecutorService = null; private Dataset expiredFileDS = null; @@ -265,7 +259,22 @@ private Set findExpiredSnapshotIds( } private ExpireSnapshots.Result deleteFiles(Iterator files) { - DeleteSummary summary = deleteFiles(deleteExecutorService, deleteFunc, files); + DeleteSummary summary; + if (deleteFunc == null && table.io() instanceof SupportsBulkOperations) { + summary = deleteFiles((SupportsBulkOperations) table.io(), files); + } else { + + if (deleteFunc == null) { + LOG.info( + "Table IO {} does not support bulk operations. Using non-bulk deletes.", + table.io().getClass().getName()); + summary = deleteFiles(deleteExecutorService, table.io()::deleteFile, files); + } else { + LOG.info("Custom delete function provided. Using non-bulk deletes"); + summary = deleteFiles(deleteExecutorService, deleteFunc, files); + } + } + LOG.info("Deleted {} total files", summary.totalFilesCount()); return new BaseExpireSnapshotsActionResult( diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/ExpireSnapshotsProcedure.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/ExpireSnapshotsProcedure.java index aff4b44f94fa..8d979ea05460 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/ExpireSnapshotsProcedure.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/ExpireSnapshotsProcedure.java @@ -20,6 +20,7 @@ import org.apache.iceberg.Table; import org.apache.iceberg.actions.ExpireSnapshots; +import org.apache.iceberg.io.SupportsBulkOperations; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.spark.actions.ExpireSnapshotsSparkAction; import org.apache.iceberg.spark.actions.SparkActions; @@ -33,6 +34,8 @@ import org.apache.spark.sql.types.Metadata; import org.apache.spark.sql.types.StructField; import org.apache.spark.sql.types.StructType; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * A procedure that expires snapshots in a table. @@ -41,6 +44,8 @@ */ public class ExpireSnapshotsProcedure extends BaseProcedure { + private static final Logger LOG = LoggerFactory.getLogger(ExpireSnapshotsProcedure.class); + private static final ProcedureParameter[] PARAMETERS = new ProcedureParameter[] { ProcedureParameter.required("table", DataTypes.StringType), @@ -114,7 +119,17 @@ public InternalRow[] call(InternalRow args) { } if (maxConcurrentDeletes != null) { - action.executeDeleteWith(executorService(maxConcurrentDeletes, "expire-snapshots")); + if (table.io() instanceof SupportsBulkOperations) { + LOG.warn( + "max_concurrent_deletes only works with FileIOs that do not support bulk deletes. This" + + "table is currently using {} which supports bulk deletes so the parameter will be ignored. " + + "See that IO's documentation to learn how to adjust parallelism for that particular " + + "IO's bulk delete.", + table.io().getClass().getName()); + } else { + + action.executeDeleteWith(executorService(maxConcurrentDeletes, "expire-snapshots")); + } } if (streamResult != null) { diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/RemoveOrphanFilesProcedure.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/RemoveOrphanFilesProcedure.java index f49f37c02ea5..6e66ea2629b8 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/RemoveOrphanFilesProcedure.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/RemoveOrphanFilesProcedure.java @@ -23,6 +23,7 @@ import org.apache.iceberg.Table; import org.apache.iceberg.actions.DeleteOrphanFiles; import org.apache.iceberg.actions.DeleteOrphanFiles.PrefixMismatchMode; +import org.apache.iceberg.io.SupportsBulkOperations; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.relocated.com.google.common.collect.Maps; @@ -39,6 +40,8 @@ import org.apache.spark.sql.types.StructField; import org.apache.spark.sql.types.StructType; import org.apache.spark.unsafe.types.UTF8String; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import scala.runtime.BoxedUnit; /** @@ -47,6 +50,7 @@ * @see SparkActions#deleteOrphanFiles(Table) */ public class RemoveOrphanFilesProcedure extends BaseProcedure { + private static final Logger LOG = LoggerFactory.getLogger(RemoveOrphanFilesProcedure.class); private static final ProcedureParameter[] PARAMETERS = new ProcedureParameter[] { @@ -154,7 +158,17 @@ public InternalRow[] call(InternalRow args) { } if (maxConcurrentDeletes != null) { - action.executeDeleteWith(executorService(maxConcurrentDeletes, "remove-orphans")); + if (table.io() instanceof SupportsBulkOperations) { + LOG.warn( + "max_concurrent_deletes only works with FileIOs that do not support bulk deletes. This" + + "table is currently using {} which supports bulk deletes so the parameter will be ignored. " + + "See that IO's documentation to learn how to adjust parallelism for that particular " + + "IO's bulk delete.", + table.io().getClass().getName()); + } else { + + action.executeDeleteWith(executorService(maxConcurrentDeletes, "remove-orphans")); + } } if (fileListView != null) {