-
Notifications
You must be signed in to change notification settings - Fork 3.4k
Improve parallelism in remove_orphan_files procedure #26326
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
Changes from all commits
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 |
|---|---|---|
|
|
@@ -25,6 +25,7 @@ | |
| import com.google.common.collect.Lists; | ||
| import com.google.common.collect.Sets; | ||
| import com.google.common.collect.Streams; | ||
| import io.airlift.concurrent.MoreFutures; | ||
| import io.airlift.json.JsonCodec; | ||
| import io.airlift.log.Logger; | ||
| import io.airlift.slice.Slice; | ||
|
|
@@ -217,6 +218,7 @@ | |
| import java.util.concurrent.ExecutionException; | ||
| import java.util.concurrent.Executor; | ||
| import java.util.concurrent.ExecutorService; | ||
| import java.util.concurrent.Future; | ||
| import java.util.concurrent.atomic.AtomicInteger; | ||
| import java.util.concurrent.atomic.AtomicReference; | ||
| import java.util.function.Consumer; | ||
|
|
@@ -457,6 +459,7 @@ public class IcebergMetadata | |
| private final ExecutorService icebergScanExecutor; | ||
| private final Executor metadataFetchingExecutor; | ||
| private final ExecutorService icebergPlanningExecutor; | ||
| private final ExecutorService icebergFileDeleteExecutor; | ||
| private final Map<IcebergTableHandle, AtomicReference<TableStatistics>> tableStatisticsCache = new ConcurrentHashMap<>(); | ||
|
|
||
| private Transaction transaction; | ||
|
|
@@ -474,7 +477,8 @@ public IcebergMetadata( | |
| Predicate<String> allowedExtraProperties, | ||
| ExecutorService icebergScanExecutor, | ||
| Executor metadataFetchingExecutor, | ||
| ExecutorService icebergPlanningExecutor) | ||
| ExecutorService icebergPlanningExecutor, | ||
| ExecutorService icebergFileDeleteExecutor) | ||
| { | ||
| this.typeManager = requireNonNull(typeManager, "typeManager is null"); | ||
| this.trinoCatalogHandle = requireNonNull(trinoCatalogHandle, "trinoCatalogHandle is null"); | ||
|
|
@@ -488,6 +492,7 @@ public IcebergMetadata( | |
| this.icebergScanExecutor = requireNonNull(icebergScanExecutor, "icebergScanExecutor is null"); | ||
| this.metadataFetchingExecutor = requireNonNull(metadataFetchingExecutor, "metadataFetchingExecutor is null"); | ||
| this.icebergPlanningExecutor = requireNonNull(icebergPlanningExecutor, "icebergPlanningExecutor is null"); | ||
| this.icebergFileDeleteExecutor = requireNonNull(icebergFileDeleteExecutor, "icebergFileDeleteExecutor is null"); | ||
| } | ||
|
|
||
| @Override | ||
|
|
@@ -2280,7 +2285,7 @@ private void removeOrphanFiles(Table table, ConnectorSession session, SchemaTabl | |
|
|
||
| validMetadataFileNames.add(fileName(manifest.path())); | ||
| try (ManifestReader<? extends ContentFile<?>> manifestReader = readerForManifest(table, manifest)) { | ||
| for (ContentFile<?> contentFile : manifestReader) { | ||
| for (ContentFile<?> contentFile : manifestReader.select(ImmutableList.of("file_path"))) { | ||
| validDataFileNames.add(fileName(contentFile.location())); | ||
| } | ||
| } | ||
|
|
@@ -2347,18 +2352,19 @@ private static ManifestReader<? extends ContentFile<?>> readerForManifest(Table | |
|
|
||
| private void scanAndDeleteInvalidFiles(Table table, ConnectorSession session, SchemaTableName schemaTableName, Instant expiration, Set<String> validFiles, String subfolder, Map<String, String> fileIoProperties) | ||
| { | ||
| List<Future<?>> deleteFutures = new ArrayList<>(); | ||
| try { | ||
| List<Location> filesToDelete = new ArrayList<>(); | ||
| List<Location> filesToDelete = new ArrayList<>(DELETE_BATCH_SIZE); | ||
| TrinoFileSystem fileSystem = fileSystemFactory.create(session.getIdentity(), fileIoProperties); | ||
| FileIterator allFiles = fileSystem.listFiles(Location.of(table.location()).appendPath(subfolder)); | ||
| while (allFiles.hasNext()) { | ||
| FileEntry entry = allFiles.next(); | ||
| if (entry.lastModified().isBefore(expiration) && !validFiles.contains(entry.location().fileName())) { | ||
| filesToDelete.add(entry.location()); | ||
| if (filesToDelete.size() >= DELETE_BATCH_SIZE) { | ||
| log.debug("Deleting files while removing orphan files for table %s [%s]", schemaTableName, filesToDelete); | ||
| fileSystem.deleteFiles(filesToDelete); | ||
| filesToDelete.clear(); | ||
| List<Location> finalFilesToDelete = filesToDelete; | ||
| deleteFutures.add(icebergFileDeleteExecutor.submit(() -> deleteFiles(finalFilesToDelete, schemaTableName, fileSystem))); | ||
|
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. It is worth seeing whether in production use cases the object store provider will choose to return Is the filesystem layer being able to cope with backpressure?
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. Overall parallelism is bounded by |
||
| filesToDelete = new ArrayList<>(DELETE_BATCH_SIZE); | ||
| } | ||
| } | ||
| else { | ||
|
|
@@ -2369,6 +2375,22 @@ private void scanAndDeleteInvalidFiles(Table table, ConnectorSession session, Sc | |
| log.debug("Deleting files while removing orphan files for table %s %s", schemaTableName, filesToDelete); | ||
| fileSystem.deleteFiles(filesToDelete); | ||
| } | ||
|
|
||
| deleteFutures.forEach(MoreFutures::getFutureValue); | ||
raunaqmorarka marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| } | ||
| catch (IOException | UncheckedIOException e) { | ||
| throw new TrinoException(ICEBERG_FILESYSTEM_ERROR, "Failed accessing data for table: " + schemaTableName, e); | ||
| } | ||
| finally { | ||
| deleteFutures.forEach(future -> future.cancel(true)); | ||
raunaqmorarka marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| } | ||
| } | ||
|
|
||
| private void deleteFiles(List<Location> files, SchemaTableName schemaTableName, TrinoFileSystem fileSystem) | ||
| { | ||
| log.debug("Deleting files while removing orphan files for table %s [%s]", schemaTableName, files); | ||
| try { | ||
| fileSystem.deleteFiles(files); | ||
| } | ||
| catch (IOException | UncheckedIOException e) { | ||
| throw new TrinoException(ICEBERG_FILESYSTEM_ERROR, "Failed accessing data for table: " + schemaTableName, e); | ||
|
|
||
Uh oh!
There was an error while loading. Please reload this page.