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 @@ -32,16 +32,17 @@
import java.util.concurrent.atomic.AtomicReference;
import java.util.function.Function;
import java.util.function.Predicate;
import org.apache.iceberg.TableMetadata.MetadataLogEntry;
import org.apache.iceberg.encryption.EncryptionManager;
import org.apache.iceberg.exceptions.AlreadyExistsException;
import org.apache.iceberg.exceptions.CommitFailedException;
import org.apache.iceberg.exceptions.NoSuchTableException;
import org.apache.iceberg.io.FileIO;
import org.apache.iceberg.io.LocationProvider;
import org.apache.iceberg.io.OutputFile;
import org.apache.iceberg.io.*;
import org.apache.iceberg.relocated.com.google.common.base.Objects;
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.Sets;
import org.apache.iceberg.util.FileIOUtil;
import org.apache.iceberg.util.PropertyUtil;
import org.apache.iceberg.util.Tasks;
import org.slf4j.Logger;
Expand Down Expand Up @@ -411,17 +412,12 @@ private void deleteRemovedMetadataFiles(TableMetadata base, TableMetadata metada
TableProperties.METADATA_DELETE_AFTER_COMMIT_ENABLED_DEFAULT);

if (deleteAfterCommit) {
Set<TableMetadata.MetadataLogEntry> removedPreviousMetadataFiles =
Sets.newHashSet(base.previousFiles());
Set<MetadataLogEntry> removedPreviousMetadataFiles = Sets.newHashSet(base.previousFiles());
removedPreviousMetadataFiles.removeAll(metadata.previousFiles());
Tasks.foreach(removedPreviousMetadataFiles)
.noRetry()
.suppressFailureWhenFinished()
.onFailure(
(previousMetadataFile, exc) ->
LOG.warn(
"Delete failed for previous metadata file: {}", previousMetadataFile, exc))
.run(previousMetadataFile -> io().deleteFile(previousMetadataFile.file()));
Iterable<String> deletedFiles =
Iterables.transform(removedPreviousMetadataFiles, MetadataLogEntry::file);

FileIOUtil.bulkDelete(io(), deletedFiles).name("previous metadata file").execute();
}
}
}
52 changes: 19 additions & 33 deletions core/src/main/java/org/apache/iceberg/BaseTransaction.java
Original file line number Diff line number Diff line change
Expand Up @@ -41,8 +41,10 @@
import org.apache.iceberg.io.LocationProvider;
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.collect.Iterables;
import org.apache.iceberg.relocated.com.google.common.collect.Lists;
import org.apache.iceberg.relocated.com.google.common.collect.Sets;
import org.apache.iceberg.util.FileIOUtil;
import org.apache.iceberg.util.PropertyUtil;
import org.apache.iceberg.util.Tasks;
import org.slf4j.Logger;
Expand All @@ -62,12 +64,12 @@ enum TransactionType {
private final TableOperations ops;
private final TransactionTable transactionTable;
private final TableOperations transactionOps;
private final List<PendingUpdate> updates;
private final List<PendingUpdate<?>> updates;
private final Set<Long> intermediateSnapshotIds;
private final Set<String> deletedFiles =
Sets.newHashSet(); // keep track of files deleted in the most recent commit
private final Consumer<String> enqueueDelete = deletedFiles::add;
private TransactionType type;
private final TransactionType type;
private TableMetadata base;
private TableMetadata current;
private boolean hasLastOpCommitted;
Expand Down Expand Up @@ -287,20 +289,16 @@ private void commitCreateTransaction() {
.run(
update -> {
if (update instanceof SnapshotProducer) {
((SnapshotProducer) update).cleanAll();
((SnapshotProducer<?>) update).cleanAll();
}
});

throw e;

} finally {
// create table never needs to retry because the table has no previous state. because retries
// are not a
// concern, it is safe to delete all of the deleted files from individual operations
Tasks.foreach(deletedFiles)
.suppressFailureWhenFinished()
.onFailure((file, exc) -> LOG.warn("Failed to delete uncommitted file: {}", file, exc))
.run(ops.io()::deleteFile);
// are not a concern, it is safe to delete all the deleted files from individual operations
FileIOUtil.bulkDelete(ops().io(), deletedFiles).name("uncommitted file").execute();
}
}

Expand Down Expand Up @@ -329,7 +327,7 @@ private void commitReplaceTransaction(boolean orCreate) {
}
}

// because this is a replace table, it will always completely replace the table
// because this is a replacement table, it will always completely replace the table
// metadata. even if it was just updated.
if (base != underlyingOps.current()) {
this.base = underlyingOps.current(); // just refreshed
Expand All @@ -348,20 +346,17 @@ private void commitReplaceTransaction(boolean orCreate) {
.run(
update -> {
if (update instanceof SnapshotProducer) {
((SnapshotProducer) update).cleanAll();
((SnapshotProducer<?>) update).cleanAll();
}
});

throw e;

} finally {
// replace table never needs to retry because the table state is completely replaced. because
// retries are not
// a concern, it is safe to delete all of the deleted files from individual operations
Tasks.foreach(deletedFiles)
.suppressFailureWhenFinished()
.onFailure((file, exc) -> LOG.warn("Failed to delete uncommitted file: {}", file, exc))
.run(ops.io()::deleteFile);
// retries are not a concern, it is safe to delete all the deleted files from individual
// operations
FileIOUtil.bulkDelete(ops().io(), deletedFiles).name("uncommitted file").execute();
}
}

Expand Down Expand Up @@ -421,16 +416,10 @@ private void commitSimpleTransaction() {
// is retried.
Set<String> committedFiles = committedFiles(ops, intermediateSnapshotIds);
if (committedFiles != null) {
// delete all of the files that were deleted in the most recent set of operation commits
Tasks.foreach(deletedFiles)
.suppressFailureWhenFinished()
.onFailure((file, exc) -> LOG.warn("Failed to delete uncommitted file: {}", file, exc))
.run(
path -> {
if (!committedFiles.contains(path)) {
ops.io().deleteFile(path);
}
});
// delete all the files that were deleted in the most recent set of operation commits
Iterable<String> uncommittedFiles =
Iterables.filter(deletedFiles, path -> !committedFiles.contains(path));
FileIOUtil.bulkDelete(ops().io(), uncommittedFiles).name("uncommitted file").execute();
} else {
LOG.warn("Failed to load metadata for a committed snapshot, skipping clean-up");
}
Expand All @@ -447,23 +436,20 @@ private void cleanUpOnCommitFailure() {
.run(
update -> {
if (update instanceof SnapshotProducer) {
((SnapshotProducer) update).cleanAll();
((SnapshotProducer<?>) update).cleanAll();
}
});

// delete all files that were cleaned up
Tasks.foreach(deletedFiles)
.suppressFailureWhenFinished()
.onFailure((file, exc) -> LOG.warn("Failed to delete uncommitted file: {}", file, exc))
.run(ops.io()::deleteFile);
FileIOUtil.bulkDelete(ops().io(), deletedFiles).name("uncommitted file").execute();
}

private void applyUpdates(TableOperations underlyingOps) {
if (base != underlyingOps.refresh()) {
// use refreshed the metadata
this.base = underlyingOps.current();
this.current = underlyingOps.current();
for (PendingUpdate update : updates) {
for (PendingUpdate<?> update : updates) {
// re-commit each update in the chain to apply it and update current
try {
update.commit();
Expand Down
101 changes: 44 additions & 57 deletions core/src/main/java/org/apache/iceberg/CatalogUtil.java
Original file line number Diff line number Diff line change
Expand Up @@ -22,14 +22,14 @@
import static org.apache.iceberg.TableProperties.GC_ENABLED_DEFAULT;

import java.io.IOException;
import java.io.UncheckedIOException;
import java.util.Locale;
import java.util.Map;
import java.util.Set;
import org.apache.iceberg.catalog.Catalog;
import org.apache.iceberg.common.DynClasses;
import org.apache.iceberg.common.DynConstructors;
import org.apache.iceberg.common.DynMethods;
import org.apache.iceberg.exceptions.RuntimeIOException;
import org.apache.iceberg.exceptions.ValidationException;
import org.apache.iceberg.hadoop.Configurable;
import org.apache.iceberg.io.FileIO;
Expand All @@ -38,8 +38,8 @@
import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
import org.apache.iceberg.relocated.com.google.common.collect.MapMaker;
import org.apache.iceberg.relocated.com.google.common.collect.Sets;
import org.apache.iceberg.util.FileIOUtil;
import org.apache.iceberg.util.PropertyUtil;
import org.apache.iceberg.util.Tasks;
import org.apache.iceberg.util.ThreadPools;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
Expand Down Expand Up @@ -92,7 +92,7 @@ public static void dropTableData(FileIO io, TableMetadata metadata) {

LOG.info("Manifests to delete: {}", Joiner.on(", ").join(manifestsToDelete));

// run all of the deletes
// run all the deletes

boolean gcEnabled =
PropertyUtil.propertyAsBoolean(metadata.properties(), GC_ENABLED, GC_ENABLED_DEFAULT);
Expand All @@ -102,37 +102,26 @@ public static void dropTableData(FileIO io, TableMetadata metadata) {
deleteFiles(io, manifestsToDelete);
}

Tasks.foreach(Iterables.transform(manifestsToDelete, ManifestFile::path))
FileIOUtil.bulkDelete(io, Iterables.transform(manifestsToDelete, ManifestFile::path))
.name("manifest")
.executeWith(ThreadPools.getWorkerPool())
.noRetry()
.suppressFailureWhenFinished()
.onFailure((manifest, exc) -> LOG.warn("Delete failed for manifest: {}", manifest, exc))
.run(io::deleteFile);
.execute();

Tasks.foreach(manifestListsToDelete)
FileIOUtil.bulkDelete(io, manifestListsToDelete)
.name("manifest list")
.executeWith(ThreadPools.getWorkerPool())
.noRetry()
.suppressFailureWhenFinished()
.onFailure((list, exc) -> LOG.warn("Delete failed for manifest list: {}", list, exc))
.run(io::deleteFile);
.execute();

Tasks.foreach(
Iterables.transform(metadata.previousFiles(), TableMetadata.MetadataLogEntry::file))
FileIOUtil.bulkDelete(
io, Iterables.transform(metadata.previousFiles(), TableMetadata.MetadataLogEntry::file))
.name("previous metadata file")
.executeWith(ThreadPools.getWorkerPool())
.noRetry()
.suppressFailureWhenFinished()
.onFailure(
(metadataFile, exc) ->
LOG.warn("Delete failed for previous metadata file: {}", metadataFile, exc))
.run(io::deleteFile);

Tasks.foreach(metadata.metadataFileLocation())
.noRetry()
.suppressFailureWhenFinished()
.onFailure(
(metadataFile, exc) ->
LOG.warn("Delete failed for metadata file: {}", metadataFile, exc))
.run(io::deleteFile);
.execute();

FileIOUtil.bulkDelete(io, metadata.metadataFileLocation())
.name("metadata file")
.executeWith(ThreadPools.getWorkerPool())
.execute();
}

@SuppressWarnings("DangerousStringInternUsage")
Expand All @@ -141,35 +130,33 @@ private static void deleteFiles(FileIO io, Set<ManifestFile> allManifests) {
Map<String, Boolean> deletedFiles =
new MapMaker().concurrencyLevel(ThreadPools.WORKER_THREAD_POOL_SIZE).weakKeys().makeMap();

Tasks.foreach(allManifests)
.noRetry()
.suppressFailureWhenFinished()
.executeWith(ThreadPools.getWorkerPool())
.onFailure(
(item, exc) ->
LOG.warn("Failed to get deleted files: this may cause orphaned data files", exc))
.run(
manifest -> {
try (ManifestReader<?> reader = ManifestFiles.open(manifest, io)) {
for (ManifestEntry<?> entry : reader.entries()) {
// intern the file path because the weak key map uses identity (==) instead of
// equals
String path = entry.file().path().toString().intern();
Boolean alreadyDeleted = deletedFiles.putIfAbsent(path, true);
if (alreadyDeleted == null || !alreadyDeleted) {
try {
io.deleteFile(path);
} catch (RuntimeException e) {
// this may happen if the map of deleted files gets cleaned up by gc
LOG.warn("Delete failed for data file: {}", path, e);
}
Iterable<String> removedFiles =
Iterables.concat(
Iterables.transform(
allManifests,
manifest -> {
try (ManifestReader<?> reader = ManifestFiles.open(manifest, io)) {
Iterable<String> paths =
// intern the file path because the weak key map uses identity (==)
// instead of equals
Iterables.transform(
reader.entries(), entry -> entry.file().path().toString().intern());
return Iterables.filter(
paths,
path -> {
Boolean alreadyDeleted = deletedFiles.putIfAbsent(path, true);
return alreadyDeleted == null || !alreadyDeleted;
});
} catch (IOException e) {
throw new UncheckedIOException(
"Failed to read manifest file: " + manifest.path(), e);
}
}
} catch (IOException e) {
throw new RuntimeIOException(
e, "Failed to read manifest file: %s", manifest.path());
}
});
}));

FileIOUtil.bulkDelete(io, removedFiles)
.name("data file")
.executeWith(ThreadPools.getWorkerPool())
.execute();
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -39,17 +39,13 @@
import org.apache.iceberg.io.FileIO;
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.collect.Iterables;
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.relocated.com.google.common.collect.Sets;
import org.apache.iceberg.relocated.com.google.common.collect.Streams;
import org.apache.iceberg.util.PropertyUtil;
import org.apache.iceberg.util.StructLikeWrapper;
import org.apache.iceberg.util.TableScanUtil;
import org.apache.iceberg.util.Tasks;
import org.apache.iceberg.util.*;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

Expand Down Expand Up @@ -293,11 +289,7 @@ private void replaceDataFiles(
throw e;
} catch (Exception e) {
LOG.warn("Failed to commit rewrite, cleaning up rewritten files", e);
Tasks.foreach(Iterables.transform(addedDataFiles, f -> f.path().toString()))
.noRetry()
.suppressFailureWhenFinished()
.onFailure((location, exc) -> LOG.warn("Failed to delete: {}", location, exc))
.run(fileIO::deleteFile);
FileIOUtil.bulkDeleteFiles(fileIO, addedDataFiles).execute();
throw e;
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -35,7 +35,7 @@
import org.apache.iceberg.relocated.com.google.common.collect.Queues;
import org.apache.iceberg.relocated.com.google.common.collect.Sets;
import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
import org.apache.iceberg.util.Tasks;
import org.apache.iceberg.util.FileIOUtil;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

Expand Down Expand Up @@ -97,12 +97,7 @@ public void commitFileGroups(Set<RewriteFileGroup> fileGroups) {
public void abortFileGroup(RewriteFileGroup fileGroup) {
Preconditions.checkState(
fileGroup.addedFiles() != null, "Cannot abort a fileGroup that was not rewritten");

Tasks.foreach(fileGroup.addedFiles())
.noRetry()
.suppressFailureWhenFinished()
.onFailure((dataFile, exc) -> LOG.warn("Failed to delete: {}", dataFile.path(), exc))
.run(dataFile -> table.io().deleteFile(dataFile.path().toString()));
FileIOUtil.bulkDeleteFiles(table.io(), fileGroup.addedFiles()).execute();
}

public void commitOrClean(Set<RewriteFileGroup> rewriteGroups) {
Expand Down
Loading