Skip to content

Reduce memory usage of remove_orphan_files procedure #25847

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

Open
wants to merge 1 commit into
base: master
Choose a base branch
from
Open
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 @@ -149,6 +149,7 @@
import org.apache.iceberg.ManifestFile;
import org.apache.iceberg.ManifestFiles;
import org.apache.iceberg.ManifestReader;
import org.apache.iceberg.ManifestUtils;
import org.apache.iceberg.MetadataColumns;
import org.apache.iceberg.PartitionField;
import org.apache.iceberg.PartitionSpec;
Expand Down Expand Up @@ -206,10 +207,12 @@
import java.util.HashSet;
import java.util.Iterator;
import java.util.LinkedHashMap;
import java.util.LinkedList;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.OptionalLong;
import java.util.Queue;
import java.util.Set;
import java.util.concurrent.Callable;
import java.util.concurrent.ConcurrentHashMap;
Expand Down Expand Up @@ -238,6 +241,7 @@
import static com.google.common.collect.ImmutableSet.toImmutableSet;
import static com.google.common.collect.Iterables.getLast;
import static com.google.common.collect.Iterables.getOnlyElement;
import static com.google.common.collect.Lists.newArrayList;
import static com.google.common.collect.Maps.transformValues;
import static com.google.common.collect.Sets.difference;
import static io.trino.filesystem.Locations.isS3Tables;
Expand Down Expand Up @@ -2251,12 +2255,13 @@ private void removeOrphanFiles(Table table, ConnectorSession session, SchemaTabl
ImmutableSet.Builder<String> validMetadataFileNames = ImmutableSet.builder();
ImmutableSet.Builder<String> validDataFileNames = ImmutableSet.builder();

for (Snapshot snapshot : table.snapshots()) {
if (snapshot.manifestListLocation() != null) {
validMetadataFileNames.add(fileName(snapshot.manifestListLocation()));
ArrayList<String> manifestListLocations = newArrayList(Iterables.transform(table.snapshots(), Snapshot::manifestListLocation));
for (String manifestListLocation : manifestListLocations) {
if (manifestListLocation != null) {
validMetadataFileNames.add(fileName(manifestListLocation));
}

for (ManifestFile manifest : loadAllManifestsFromSnapshot(table, snapshot)) {
for (ManifestFile manifest : loadAllManifestsFromManifestList(table, manifestListLocation)) {
if (!processedManifestFilePaths.add(manifest.path())) {
// Already read this manifest
continue;
Expand Down Expand Up @@ -3460,6 +3465,21 @@ private static List<ManifestFile> loadAllManifestsFromSnapshot(Table icebergTabl
}
}

/**
* Use instead of loadAllManifestsFromSnapshot when loading manifests from multiple distinct snapshots
* Each BaseSnapshot object caches manifest files separately, so loading manifests from multiple distinct snapshots
* results in O(num_snapshots^2) copies of the same manifest file metadata in memory
*/
private static List<ManifestFile> loadAllManifestsFromManifestList(Table icebergTable, String manifestListLocation)
Copy link
Member

Choose a reason for hiding this comment

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

Can this move to ManifestUtils ?

Copy link
Contributor Author

@grantatspothero grantatspothero May 22, 2025

Choose a reason for hiding this comment

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

Note the org.apache.iceberg package.

The intention of this class was to utilize the package private method ManifestLists.read.

We want the behavior of BaseSnapshot.allManifests() without caching, but iceberg does not support that. My workaround works fine for v2 iceberg tables, but I'm concerned it might not work with v1 iceberg tables. See:
https://github.com/apache/iceberg/blob/main/core/src/main/java/org/apache/iceberg/BaseSnapshot.java#L174-L186

Copy link
Member

Choose a reason for hiding this comment

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

Is it possible to restrict the fix to V2 iceberg tables while we look for a solution for V1 tables in https://apache-iceberg.slack.com/archives/C03LG1D563F/p1747921709647279 ?

{
try {
return ManifestUtils.read(icebergTable.io(), manifestListLocation);
}
catch (NotFoundException | UncheckedIOException e) {
throw new TrinoException(ICEBERG_INVALID_METADATA, "Error accessing manifest file for table %s".formatted(icebergTable.name()), e);
}
}

private static Set<Integer> identityPartitionColumnsInAllSpecs(Table table)
{
// Extract identity partition column source ids common to ALL specs
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,29 @@
/*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.iceberg;

import org.apache.iceberg.io.FileIO;

import java.util.List;

public class ManifestUtils
{
private ManifestUtils() {}

public static List<ManifestFile> read(FileIO fileIO, String manifestListLocation)
{
// Avoid using snapshot.allManifests() which can easily leak memory
return ManifestLists.read(fileIO.newInputFile(manifestListLocation));
}
}
Loading