Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,8 @@
import java.util.concurrent.ExecutorService;
import java.util.concurrent.atomic.AtomicLong;
import java.util.function.Consumer;
import java.util.function.Function;
import java.util.stream.Collectors;
import org.apache.iceberg.FileContent;
import org.apache.iceberg.HasTableOperations;
import org.apache.iceberg.Table;
Expand All @@ -36,6 +38,7 @@
import org.apache.iceberg.actions.ExpireSnapshots;
import org.apache.iceberg.exceptions.NotFoundException;
import org.apache.iceberg.exceptions.ValidationException;
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.Preconditions;
import org.apache.iceberg.relocated.com.google.common.collect.Lists;
Expand Down Expand Up @@ -83,10 +86,20 @@ public void accept(String file) {
}
};

private final Consumer<Iterable<String>> bulkDelete =
new Consumer<Iterable<String>>() {
@Override
public void accept(Iterable<String> files) {
SupportsBulkOperations bulkOperations = (SupportsBulkOperations) ops.io();
bulkOperations.deleteFiles(files);
}
};

private final Set<Long> expiredSnapshotIds = Sets.newHashSet();
private Long expireOlderThanValue = null;
private Integer retainLastValue = null;
private Consumer<String> deleteFunc = defaultDelete;
private Consumer<Iterable<String>> bulkDeleteFunc = bulkDelete;
private ExecutorService deleteExecutorService = null;
private Dataset<Row> expiredFiles = null;

Expand Down Expand Up @@ -139,6 +152,14 @@ public ExpireSnapshotsSparkAction deleteWith(Consumer<String> newDeleteFunc) {
return this;
}

public ExpireSnapshotsSparkAction bulkDeleteWith(Consumer<Iterable<String>> newBulkDeleteFunc) {
Preconditions.checkArgument(
ops.io() instanceof SupportsBulkOperations,
"FileIO %s does not support bulk deletion",
table.io().getClass().getName());
Copy link
Contributor Author

@dramaticlly dramaticlly Aug 2, 2022

Choose a reason for hiding this comment

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

I shall probably move this table.io() to ops.io() for consistency, but given check take 1+hr to complete, I will handle it together with other review feedback

this.bulkDeleteFunc = newBulkDeleteFunc;
return this;
}
/**
* Expires snapshots and commits the changes to the table, returning a Dataset of files to delete.
*
Expand Down Expand Up @@ -215,13 +236,71 @@ private String jobDesc() {
private ExpireSnapshots.Result doExecute() {
boolean streamResults =
PropertyUtil.propertyAsBoolean(options(), STREAM_RESULTS, STREAM_RESULTS_DEFAULT);
if (streamResults) {
return deleteFiles(expire().toLocalIterator());
boolean supportBulkDeletion = ops.io() instanceof SupportsBulkOperations;

final Dataset<Row> expiredRows = expire();
final Iterator<Row> expiredIterator =
streamResults ? expiredRows.toLocalIterator() : expiredRows.collectAsList().iterator();

if (supportBulkDeletion) {
return deleteFilesInBulk(expiredRows.collectAsList());
} else {
return deleteFiles(expire().collectAsList().iterator());
return deleteFiles(expiredIterator);
}
}

/**
* Bulk deletes files passed to it based on their type. It now delegates task management to fileIO
*
* @param batch a list of Spark Rows of the structure (path: String, type: String)
* @return Statistics on which files were deleted
*/
private ExpireSnapshots.Result deleteFilesInBulk(List<Row> batch) {
AtomicLong dataFileCount = new AtomicLong(0L);
AtomicLong posDeleteFileCount = new AtomicLong(0L);
AtomicLong eqDeleteFileCount = new AtomicLong(0L);
AtomicLong manifestCount = new AtomicLong(0L);
AtomicLong manifestListCount = new AtomicLong(0L);
Function<Row, String> fileName = r -> r.getString(0);

List<String> fileNamed = batch.stream().map(fileName).collect(Collectors.toList());
bulkDeleteFunc.accept(fileNamed);
long dataFileFound =
batch.stream()
.filter(r -> FileContent.DATA.name().equalsIgnoreCase(r.getString(1)))
.count();
dataFileCount.addAndGet(dataFileFound);
long posDeleteFileFound =
batch.stream()
.filter(r -> FileContent.POSITION_DELETES.name().equalsIgnoreCase(r.getString(1)))
.count();
posDeleteFileCount.addAndGet(posDeleteFileFound);
long eqDeleteFileFound =
batch.stream()
.filter(r -> FileContent.EQUALITY_DELETES.name().equalsIgnoreCase(r.getString(1)))
.count();
eqDeleteFileCount.addAndGet(eqDeleteFileFound);
long manifestFound =
batch.stream().filter(r -> MANIFEST.equalsIgnoreCase(r.getString(1))).count();
manifestCount.addAndGet(manifestFound);
long manifestListFound =
batch.stream().filter(r -> MANIFEST_LIST.equalsIgnoreCase(r.getString(1))).count();
manifestListCount.addAndGet(manifestListFound);

long contentFileCount =
dataFileCount.get() + posDeleteFileCount.get() + eqDeleteFileCount.get();
LOG.info(
"Deleted {} total files in bulk",
contentFileCount + manifestCount.get() + manifestListCount.get());

return new BaseExpireSnapshotsActionResult(
dataFileCount.get(),
posDeleteFileCount.get(),
eqDeleteFileCount.get(),
manifestCount.get(),
manifestListCount.get());
}

private Dataset<Row> buildValidFileDF(TableMetadata metadata) {
Table staticTable = newStaticTable(metadata, table.io());
return buildValidContentFileWithTypeDF(staticTable)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1182,6 +1182,21 @@ public void testExpireAction() {
action.expire());
}

@Test
public void testExpireBulkDeleteWithThrowOnUnsupportedFileIO() {
table.newAppend().appendFile(FILE_A).commit();
Set<String> bulkDeletedFiles = Sets.newHashSet();

AssertHelpers.assertThrows(
"Should complain about unsupported fileIO",
IllegalArgumentException.class,
"does not support bulk deletion",
() ->
SparkActions.get()
.expireSnapshots(table)
.bulkDeleteWith(files -> files.forEach(bulkDeletedFiles::add)));
}

@Test
public void testUseLocalIterator() {
table.newFastAppend().appendFile(FILE_A).commit();
Expand Down