From 76fc60e0a66242f51a101fc3c7f85b5950e98950 Mon Sep 17 00:00:00 2001 From: vaultah <4944562+vaultah@users.noreply.github.com> Date: Fri, 1 Aug 2025 14:47:25 +0100 Subject: [PATCH 01/18] Update sizes of rewritten manifests in manifest lists --- .../apache/iceberg/RewriteTablePathUtil.java | 188 +++++++++++++- .../actions/RewriteTablePathSparkAction.java | 242 ++++++++++++------ .../actions/TestRewriteTablePathsAction.java | 184 ++++++++++++- .../actions/RewriteTablePathSparkAction.java | 242 ++++++++++++------ .../actions/TestRewriteTablePathsAction.java | 184 ++++++++++++- .../actions/RewriteTablePathSparkAction.java | 242 ++++++++++++------ .../actions/TestRewriteTablePathsAction.java | 184 ++++++++++++- 7 files changed, 1228 insertions(+), 238 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/RewriteTablePathUtil.java b/core/src/main/java/org/apache/iceberg/RewriteTablePathUtil.java index f63222c1854a..e5876f8a18aa 100644 --- a/core/src/main/java/org/apache/iceberg/RewriteTablePathUtil.java +++ b/core/src/main/java/org/apache/iceberg/RewriteTablePathUtil.java @@ -87,6 +87,24 @@ public Set> copyPlan() { } } + public static class RewrittenFileInfo implements Serializable { + private final String newPath; + private final long newSize; + + public RewrittenFileInfo(String newPath, long newSize) { + this.newPath = newPath; + this.newSize = newSize; + } + + public String getNewPath() { + return newPath; + } + + public long getNewSize() { + return newSize; + } + } + /** * Create a new table metadata object, replacing path references * @@ -226,7 +244,9 @@ private static List updatePathInSnapshots( * @param outputPath location to write the manifest list * @return a copy plan for manifest files whose metadata were contained in the rewritten manifest * list + * @deprecated since 1.10.0, will be removed in 1.11.0 */ + @Deprecated public static RewriteResult rewriteManifestList( Snapshot snapshot, FileIO io, @@ -274,6 +294,65 @@ public static RewriteResult rewriteManifestList( } } + /** + * Rewrite a manifest list representing a snapshot, replacing path references. + * @param snapshot snapshot represented by the manifest list + * @param io file io + * @param tableMetadata metadata of table + * @param rewrittenManifests information about rewritten manifest files + * @param sourcePrefix source prefix that will be replaced + * @param targetPrefix target prefix that will replace it + * @param stagingDir staging directory + * @param outputPath location to write the manifest list + */ + public static void rewriteManifestList( + Snapshot snapshot, + FileIO io, + TableMetadata tableMetadata, + Map rewrittenManifests, + String sourcePrefix, + String targetPrefix, + String stagingDir, + String outputPath) { + OutputFile outputFile = io.newOutputFile(outputPath); + + List manifestFiles = manifestFilesInSnapshot(io, snapshot); + manifestFiles.forEach( + mf -> + Preconditions.checkArgument( + mf.path().startsWith(sourcePrefix), + "Encountered manifest file %s not under the source prefix %s", + mf.path(), + sourcePrefix)); + + try (FileAppender writer = + ManifestLists.write( + tableMetadata.formatVersion(), + outputFile, + snapshot.snapshotId(), + snapshot.parentId(), + snapshot.sequenceNumber(), + snapshot.firstRowId())) { + + for (ManifestFile file : manifestFiles) { + ManifestFile newFile = file.copy(); + + if (rewrittenManifests.containsKey(file.path())) { + String rewrittenPath = rewrittenManifests.get(file.path()).getNewPath(); + long rewrittenSize = rewrittenManifests.get(file.path()).getNewSize(); + ((StructLike) newFile).set(0, rewrittenPath); + ((StructLike) newFile).set(1, rewrittenSize); + } else { + ((StructLike) newFile).set(0, newPath(newFile.path(), sourcePrefix, targetPrefix)); + } + writer.add(newFile); + } + } catch (IOException e) { + throw new UncheckedIOException( + "Failed to rewrite the manifest list file " + snapshot.manifestListLocation(), e); + } + } + private static List manifestFilesInSnapshot(FileIO io, Snapshot snapshot) { String path = snapshot.manifestListLocation(); List manifestFiles = Lists.newLinkedList(); @@ -333,7 +412,9 @@ public static RewriteResult rewriteDataManifest( * @param sourcePrefix source prefix that will be replaced * @param targetPrefix target prefix that will replace it * @return a copy plan of content files in the manifest that was rewritten + * @deprecated since 1.10.0, will be removed in 1.11.0 */ + @Deprecated public static RewriteResult rewriteDataManifest( ManifestFile manifestFile, Set snapshotIds, @@ -357,6 +438,55 @@ public static RewriteResult rewriteDataManifest( } } + + /** + * Rewrite a data manifest, replacing path references. + * + * @param manifestFile source manifest file to rewrite + * @param snapshotIds snapshot ids for filtering returned data manifest entries + * @param outputFile output file to rewrite manifest file to + * @param io file io + * @param format format of the manifest file + * @param specsById map of partition specs by id + * @param sourcePrefix source prefix that will be replaced + * @param targetPrefix target prefix that will replace it + * @return rewritten manifest file and a copy plan for the referenced content files + */ + public static Pair> rewriteDataManifestWithResult( + ManifestFile manifestFile, + Set snapshotIds, + OutputFile outputFile, + FileIO io, + int format, + Map specsById, + String sourcePrefix, + String targetPrefix) + throws IOException { + PartitionSpec spec = specsById.get(manifestFile.partitionSpecId()); + ManifestWriter writer = + ManifestFiles.write(format, spec, outputFile, manifestFile.snapshotId()); + RewriteResult rewriteResult = null; + + try (ManifestWriter dataManifestWriter = writer; + ManifestReader reader = + ManifestFiles.read(manifestFile, io, specsById) + .select(Arrays.asList("*"))) { + rewriteResult = + StreamSupport.stream(reader.entries().spliterator(), false) + .map( + entry -> + writeDataFileEntry( + entry, + snapshotIds, + spec, + sourcePrefix, + targetPrefix, + writer)) + .reduce(new RewriteResult<>(), RewriteResult::append); + } + return Pair.of(writer.toManifestFile(), rewriteResult); + } + /** * Rewrite a delete manifest, replacing path references. * @@ -411,8 +541,11 @@ public static RewriteResult rewriteDeleteManifest( * @param targetPrefix target prefix that will replace it * @param stagingLocation staging location for rewritten files (referred delete file will be * rewritten here) - * @return a copy plan of content files in the manifest that was rewritten + * @return rewritten manifest file and a copy plan of content files + * in the manifest that was rewritten + * @deprecated since 1.10.0, will be removed in 1.11.0 */ + @Deprecated public static RewriteResult rewriteDeleteManifest( ManifestFile manifestFile, Set snapshotIds, @@ -445,6 +578,59 @@ public static RewriteResult rewriteDeleteManifest( } } + /** + * Rewrite a delete manifest, replacing path references. + * + * @param manifestFile source delete manifest to rewrite + * @param snapshotIds snapshot ids for filtering returned delete manifest entries + * @param outputFile output file to rewrite manifest file to + * @param io file io + * @param format format of the manifest file + * @param specsById map of partition specs by id + * @param sourcePrefix source prefix that will be replaced + * @param targetPrefix target prefix that will replace it + * @param stagingLocation staging location for rewritten files (referred delete file will be + * rewritten here) + * @return rewritten manifest file and a copy plan for the referenced content files + */ + public static Pair> rewriteDeleteManifestWithResult( + ManifestFile manifestFile, + Set snapshotIds, + OutputFile outputFile, + FileIO io, + int format, + Map specsById, + String sourcePrefix, + String targetPrefix, + String stagingLocation) + throws IOException { + PartitionSpec spec = specsById.get(manifestFile.partitionSpecId()); + ManifestWriter writer = + ManifestFiles.writeDeleteManifest(format, spec, outputFile, manifestFile.snapshotId()); + RewriteResult rewriteResult = null; + + try (ManifestWriter deleteManifestWriter = writer; + ManifestReader reader = + ManifestFiles.readDeleteManifest(manifestFile, io, specsById) + .select(Arrays.asList("*"))) { + rewriteResult = + StreamSupport.stream(reader.entries().spliterator(), false) + .map( + entry -> + writeDeleteFileEntry( + entry, + snapshotIds, + spec, + sourcePrefix, + targetPrefix, + stagingLocation, + writer)) + .reduce(new RewriteResult<>(), RewriteResult::append); + } + + return Pair.of(writer.toManifestFile(), rewriteResult); + } + private static RewriteResult writeDataFileEntry( ManifestEntry entry, Set snapshotIds, diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteTablePathSparkAction.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteTablePathSparkAction.java index fbeabfe63e0e..21abf32cf0c9 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteTablePathSparkAction.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteTablePathSparkAction.java @@ -22,11 +22,13 @@ import java.io.IOException; import java.io.OutputStreamWriter; import java.nio.charset.StandardCharsets; +import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Set; import java.util.UUID; import java.util.stream.Collectors; +import java.util.stream.Stream; import org.apache.iceberg.ContentFile; import org.apache.iceberg.DataFile; import org.apache.iceberg.DeleteFile; @@ -37,6 +39,7 @@ import org.apache.iceberg.RewriteTablePathUtil; import org.apache.iceberg.RewriteTablePathUtil.PositionDeleteReaderWriter; import org.apache.iceberg.RewriteTablePathUtil.RewriteResult; +import org.apache.iceberg.RewriteTablePathUtil.RewrittenFileInfo; import org.apache.iceberg.Schema; import org.apache.iceberg.Snapshot; import org.apache.iceberg.StaticTableOperations; @@ -57,6 +60,7 @@ import org.apache.iceberg.data.parquet.GenericParquetReaders; import org.apache.iceberg.data.parquet.GenericParquetWriter; import org.apache.iceberg.deletes.PositionDeleteWriter; +import org.apache.iceberg.exceptions.NotFoundException; import org.apache.iceberg.exceptions.RuntimeIOException; import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.io.DeleteSchemaUtil; @@ -68,13 +72,13 @@ 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.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.spark.JobGroupInfo; import org.apache.iceberg.spark.source.SerializableTableWithSize; import org.apache.iceberg.util.Pair; import org.apache.spark.api.java.function.ForeachFunction; import org.apache.spark.api.java.function.MapFunction; -import org.apache.spark.api.java.function.ReduceFunction; import org.apache.spark.broadcast.Broadcast; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Encoder; @@ -84,6 +88,7 @@ import org.apache.spark.sql.functions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import scala.Tuple3; public class RewriteTablePathSparkAction extends BaseSparkAction implements RewriteTablePath { @@ -282,23 +287,25 @@ private String rebuildMetadata() { RewriteResult rewriteVersionResult = rewriteVersionFiles(endMetadata); Set deltaSnapshots = deltaSnapshots(startMetadata, rewriteVersionResult.toRewrite()); - Set manifestsToRewrite = manifestsToRewrite(deltaSnapshots, startMetadata); - Set validSnapshots = - Sets.difference(snapshotSet(endMetadata), snapshotSet(startMetadata)); + // rebuild manifest files + Set manifestsToRewrite = manifestsToRewrite(deltaSnapshots, startMetadata); + ManifestsRewriteResult rewriteManifestResult = + rewriteManifests(deltaSnapshots, endMetadata, manifestsToRewrite); // rebuild manifest-list files + Set validSnapshots = + Sets.difference(snapshotSet(endMetadata), snapshotSet(startMetadata)); RewriteResult rewriteManifestListResult = validSnapshots.stream() - .map(snapshot -> rewriteManifestList(snapshot, endMetadata, manifestsToRewrite)) + .map( + snapshot -> + rewriteManifestList( + snapshot, endMetadata, rewriteManifestResult.getRewrittenManifests())) .reduce(new RewriteResult<>(), RewriteResult::append); - // rebuild manifest files - RewriteContentFileResult rewriteManifestResult = - rewriteManifests(deltaSnapshots, endMetadata, rewriteManifestListResult.toRewrite()); - // rebuild position delete files Set deleteFiles = - rewriteManifestResult.toRewrite().stream() + rewriteManifestResult.getContentFileResult().toRewrite().stream() .filter(e -> e instanceof DeleteFile) .map(e -> (DeleteFile) e) .collect(Collectors.toSet()); @@ -307,8 +314,7 @@ private String rebuildMetadata() { Set> copyPlan = Sets.newHashSet(); copyPlan.addAll(rewriteVersionResult.copyPlan()); copyPlan.addAll(rewriteManifestListResult.copyPlan()); - copyPlan.addAll(rewriteManifestResult.copyPlan()); - + copyPlan.addAll(rewriteManifestResult.getContentFileResult().copyPlan()); return saveFileList(copyPlan); } @@ -415,27 +421,26 @@ private Set> statsFileCopyPlan( * @param snapshot snapshot represented by the manifest list * @param tableMetadata metadata of table * @param manifestsToRewrite filter of manifests to rewrite. - * @return a result including a copy plan for the manifests contained in the manifest list, as - * well as for the manifest list itself + * @return a result including a copy plan for the manifest list itself */ private RewriteResult rewriteManifestList( - Snapshot snapshot, TableMetadata tableMetadata, Set manifestsToRewrite) { + Snapshot snapshot, + TableMetadata tableMetadata, + Map rewrittenManifests) { RewriteResult result = new RewriteResult<>(); String path = snapshot.manifestListLocation(); String outputPath = RewriteTablePathUtil.stagingPath(path, stagingDir); - RewriteResult rewriteResult = - RewriteTablePathUtil.rewriteManifestList( - snapshot, - table.io(), - tableMetadata, - manifestsToRewrite, - sourcePrefix, - targetPrefix, - stagingDir, - outputPath); - - result.append(rewriteResult); + RewriteTablePathUtil.rewriteManifestList( + snapshot, + table.io(), + tableMetadata, + rewrittenManifests, + sourcePrefix, + targetPrefix, + stagingDir, + outputPath); + // add the manifest list copy plan itself to the result result .copyPlan() @@ -443,26 +448,58 @@ private RewriteResult rewriteManifestList( return result; } - private Set manifestsToRewrite( + private Set manifestsToRewrite( Set deltaSnapshots, TableMetadata startMetadata) { try { Table endStaticTable = newStaticTable(endVersionName, table.io()); - Dataset lastVersionFiles = manifestDS(endStaticTable).select("path"); + Dataset allManifestsDF = manifestDS(endStaticTable).select("path"); + Set expectedManifestPaths; + if (startMetadata == null) { - return Sets.newHashSet(lastVersionFiles.distinct().as(Encoders.STRING()).collectAsList()); + expectedManifestPaths = + Sets.newHashSet(allManifestsDF.distinct().as(Encoders.STRING()).collectAsList()); } else { Set deltaSnapshotIds = deltaSnapshots.stream().map(Snapshot::snapshotId).collect(Collectors.toSet()); - return Sets.newHashSet( - lastVersionFiles - .distinct() - .filter( - functions - .column(ManifestFile.SNAPSHOT_ID.name()) - .isInCollection(deltaSnapshotIds)) - .as(Encoders.STRING()) - .collectAsList()); + expectedManifestPaths = + Sets.newHashSet( + allManifestsDF + .distinct() + .filter( + functions + .column(ManifestFile.SNAPSHOT_ID.name()) + .isInCollection(deltaSnapshotIds)) + .as(Encoders.STRING()) + .collectAsList()); } + + Set foundManifests = + deltaSnapshots.stream() + .flatMap( + s -> { + try { + return s.allManifests(table.io()).stream(); + } catch (NotFoundException e) { + LOG.warn( + "Skipping snapshot {} as its manifest list is missing (likely expired).", + s.snapshotId(), + e); + return Stream.empty(); + } + }) + .collect(Collectors.toSet()); + + Set foundManifestPaths = + foundManifests.stream().map(ManifestFile::path).collect(Collectors.toSet()); + Set missingPaths = Sets.difference(expectedManifestPaths, foundManifestPaths); + Preconditions.checkState( + missingPaths.isEmpty(), + "Could not find all expected manifests. Missing files: %s", + String.join(", ", missingPaths)); + + return foundManifests.stream() + .filter(m -> expectedManifestPaths.contains(m.path())) + .collect(Collectors.toSet()); } catch (Exception e) { throw new UnsupportedOperationException( "Unable to build the manifest files dataframe. The end version in use may contain invalid snapshots. " @@ -491,48 +528,94 @@ public RewriteContentFileResult appendDeleteFile(RewriteResult r1) { } } - /** Rewrite manifest files in a distributed manner and return rewritten data files path pairs. */ - private RewriteContentFileResult rewriteManifests( + public static class ManifestsRewriteResult { + private final RewriteContentFileResult contentFileResult; + private final Map rewrittenManifests; + + ManifestsRewriteResult( + RewriteContentFileResult contentFileResult, + Map rewrittenManifests) { + this.contentFileResult = contentFileResult; + this.rewrittenManifests = rewrittenManifests; + } + + public RewriteContentFileResult getContentFileResult() { + return contentFileResult; + } + + public Map getRewrittenManifests() { + return rewrittenManifests; + } + } + + /** Rewrite manifest files in a distributed manner and return the resulting manifests + * and content files selected for rewriting. + */ + private ManifestsRewriteResult rewriteManifests( Set deltaSnapshots, TableMetadata tableMetadata, Set toRewrite) { if (toRewrite.isEmpty()) { - return new RewriteContentFileResult(); + return new ManifestsRewriteResult(new RewriteContentFileResult(), Maps.newHashMap()); } Encoder manifestFileEncoder = Encoders.javaSerialization(ManifestFile.class); + Encoder contentResultEncoder = + Encoders.javaSerialization(RewriteContentFileResult.class); + Encoder> tupleEncoder = + Encoders.tuple(Encoders.STRING(), manifestFileEncoder, contentResultEncoder); + Dataset manifestDS = spark().createDataset(Lists.newArrayList(toRewrite), manifestFileEncoder); Set deltaSnapshotIds = deltaSnapshots.stream().map(Snapshot::snapshotId).collect(Collectors.toSet()); - return manifestDS - .repartition(toRewrite.size()) - .map( - toManifests( - tableBroadcast(), - sparkContext().broadcast(deltaSnapshotIds), - stagingDir, - tableMetadata.formatVersion(), - sourcePrefix, - targetPrefix), - Encoders.bean(RewriteContentFileResult.class)) - // duplicates are expected here as the same data file can have different statuses - // (e.g. added and deleted) - .reduce((ReduceFunction) RewriteContentFileResult::append); - } - - private static MapFunction toManifests( - Broadcast table, - Broadcast> deltaSnapshotIds, - String stagingLocation, - int format, - String sourcePrefix, - String targetPrefix) { + RewriteContentFileResult finalContentResult = new RewriteContentFileResult(); + Iterator> resultIterator = + manifestDS + .repartition(toRewrite.size()) + .map( + toManifests( + tableBroadcast(), + sparkContext().broadcast(deltaSnapshotIds), + stagingDir, + tableMetadata.formatVersion(), + sourcePrefix, + targetPrefix), + tupleEncoder) + .toLocalIterator(); + + Map rewrittenManifests = Maps.newHashMap(); + + while (resultIterator.hasNext()) { + Tuple3 resultTuple = resultIterator.next(); + String originalManifestPath = resultTuple._1(); + ManifestFile rewrittenManifest = resultTuple._2(); + RewriteContentFileResult contentFileResult = resultTuple._3(); + String targetManifestPath = + RewriteTablePathUtil.newPath(originalManifestPath, sourcePrefix, targetPrefix); + + finalContentResult.append(contentFileResult); + finalContentResult.copyPlan().add(Pair.of(rewrittenManifest.path(), targetManifestPath)); + rewrittenManifests.put( + originalManifestPath, + new RewrittenFileInfo(targetManifestPath, rewrittenManifest.length())); + } + + return new ManifestsRewriteResult(finalContentResult, rewrittenManifests); + } + + private static MapFunction> + toManifests( + Broadcast
table, + Broadcast> deltaSnapshotIds, + String stagingLocation, + int format, + String sourcePrefix, + String targetPrefix) { return manifestFile -> { - RewriteContentFileResult result = new RewriteContentFileResult(); switch (manifestFile.content()) { case DATA: - result.appendDataFile( + Pair> dataFileResult = writeDataManifest( manifestFile, table, @@ -540,10 +623,13 @@ private static MapFunction toManifests( stagingLocation, format, sourcePrefix, - targetPrefix)); - break; + targetPrefix); + return Tuple3.apply( + manifestFile.path(), + dataFileResult.first(), + new RewriteContentFileResult().appendDataFile(dataFileResult.second())); case DELETES: - result.appendDeleteFile( + Pair> deleteFileResult = writeDeleteManifest( manifestFile, table, @@ -551,17 +637,19 @@ private static MapFunction toManifests( stagingLocation, format, sourcePrefix, - targetPrefix)); - break; + targetPrefix); + return Tuple3.apply( + manifestFile.path(), + deleteFileResult.first(), + new RewriteContentFileResult().appendDeleteFile(deleteFileResult.second())); default: throw new UnsupportedOperationException( "Unsupported manifest type: " + manifestFile.content()); } - return result; }; } - private static RewriteResult writeDataManifest( + private static Pair> writeDataManifest( ManifestFile manifestFile, Broadcast
table, Broadcast> snapshotIds, @@ -575,7 +663,7 @@ private static RewriteResult writeDataManifest( OutputFile outputFile = io.newOutputFile(stagingPath); Map specsById = table.getValue().specs(); Set deltaSnapshotIds = snapshotIds.value(); - return RewriteTablePathUtil.rewriteDataManifest( + return RewriteTablePathUtil.rewriteDataManifestWithResult( manifestFile, deltaSnapshotIds, outputFile, @@ -589,7 +677,7 @@ private static RewriteResult writeDataManifest( } } - private static RewriteResult writeDeleteManifest( + private static Pair> writeDeleteManifest( ManifestFile manifestFile, Broadcast
table, Broadcast> snapshotIds, @@ -603,7 +691,7 @@ private static RewriteResult writeDeleteManifest( OutputFile outputFile = io.newOutputFile(stagingPath); Map specsById = table.getValue().specs(); Set deltaSnapshotIds = snapshotIds.value(); - return RewriteTablePathUtil.rewriteDeleteManifest( + return RewriteTablePathUtil.rewriteDeleteManifestWithResult( manifestFile, deltaSnapshotIds, outputFile, diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteTablePathsAction.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteTablePathsAction.java index 705e0074c79f..c737d551821e 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteTablePathsAction.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteTablePathsAction.java @@ -39,6 +39,8 @@ import org.apache.iceberg.DeleteFile; import org.apache.iceberg.HasTableOperations; import org.apache.iceberg.ImmutableGenericPartitionStatisticsFile; +import org.apache.iceberg.ManifestContent; +import org.apache.iceberg.ManifestFile; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; import org.apache.iceberg.Snapshot; @@ -343,7 +345,7 @@ public void testManifestRewriteAndIncrementalCopy() throws Exception { int addedManifest = Iterables.size(rewriteManifestResult.addedManifests()); // only move version v4, which is the version generated by rewrite manifest - RewriteTablePath.Result postReweiteResult = + RewriteTablePath.Result postRewriteResult = actions() .rewriteTablePath(table) .rewriteLocationPrefix(table.location(), targetTableLocation()) @@ -352,7 +354,7 @@ public void testManifestRewriteAndIncrementalCopy() throws Exception { .execute(); // no data files need to move - checkFileNum(1, 1, addedManifest, 3, postReweiteResult); + checkFileNum(1, 1, addedManifest, 3, postRewriteResult); } @Test @@ -1122,6 +1124,184 @@ public void testKryoDeserializeBroadcastValues() { assertThat(tableBroadcast.getValue().uuid()).isEqualTo(table.uuid()); } + @Test + public void testFullRewriteUpdatesAllManifestLengthsInManifestList( + @TempDir Path rootTargetLocation) throws Exception { + String location = newTableLocation(); + Table sourceTable = createTableWithSnapshots(location, 10); + + addAnyPositionDelete( + sourceTable, removePrefix(sourceTable.location() + "/data/deeply/nested/deletes.parquet")); + + Map manifestSizesBeforeRewrite = + sourceTable.currentSnapshot().allManifests(sourceTable.io()).stream() + .collect(Collectors.toMap(m -> fileName(m.path()), m -> m.length())); + + // Rewrite table metadata to a location that's much longer than the original in order + // to make manifests larger + String targetLocation = toAbsolute(rootTargetLocation) + generateLongNestedPath(25); + RewriteTablePath.Result result = + actions() + .rewriteTablePath(sourceTable) + .rewriteLocationPrefix(newTableLocation(), targetLocation) + .execute(); + + // 1 + 11 JSON metadata files, 11 snapshots, 11 manifests, 10 data files, 1 delete file + checkFileNum(12, 11, 11, 45, result); + copyTableFiles(result); + + Table targetTable = TABLES.load(targetLocation); + + // We have rewritten all manifests. Make sure all sizes were correctly updated in manifest lists + assertThat(targetTable.currentSnapshot().allManifests(targetTable.io())) + .allSatisfy( + manifest -> { + String manifestName = fileName(manifest.path()); + assertThat(manifest.length()) + .isNotEqualTo(manifestSizesBeforeRewrite.get(manifestName)); + }) + .allSatisfy( + manifest -> { + assertThat(targetTable.io().newInputFile(manifest.path()).getLength()) + .isEqualTo(manifest.length()); + }); + } + + @Test + public void testKeepsExistingDataManifestLengthInManifestList(@TempDir Path rootTargetLocation) + throws Exception { + String location = newTableLocation(); + Table sourceTable = createTableWithSnapshots(location, 10); + + Map manifestSizesBeforeRewrite = + sourceTable.currentSnapshot().allManifests(sourceTable.io()).stream() + .collect(Collectors.toMap(m -> fileName(m.path()), m -> m.length())); + + // Rewrite just the latest table version to a location that's much longer than + // the original in order to make manifests larger + String targetLocation = toAbsolute(rootTargetLocation) + generateLongNestedPath(25); + RewriteTablePath.Result result = + actions() + .rewriteTablePath(sourceTable) + .rewriteLocationPrefix(newTableLocation(), targetLocation) + .startVersion("v10.metadata.json") + .execute(); + + // 1 metadata JSON file, 1 snapshot, 1 manifest, 4 data files + checkFileNum(1, 1, 1, 4, result); + copyTableFiles(result); + + Table targetTable = TABLES.load(targetLocation); + Snapshot lastSnapshot = targetTable.currentSnapshot(); + + List allManifests = lastSnapshot.allManifests(targetTable.io()); + ManifestFile rewrittenManifest = + Iterables.getOnlyElement( + allManifests.stream() + .filter(manifest -> manifest.snapshotId() == lastSnapshot.snapshotId()) + .collect(Collectors.toList())); + + // We have rewritten exactly one (data) manifest. Check that its size was correctly + // updated in the manifest list and sizes of all other manifests were left unchanged + assertThat(allManifests) + .filteredOn(manifest -> manifest != rewrittenManifest) + .allSatisfy( + manifest -> { + String manifestName = fileName(manifest.path()); + assertThat(manifest.length()).isEqualTo(manifestSizesBeforeRewrite.get(manifestName)); + }); + assertThat(rewrittenManifest.content()).isEqualTo(ManifestContent.DATA); + assertThat(rewrittenManifest.length()) + .isNotEqualTo(manifestSizesBeforeRewrite.get(fileName(rewrittenManifest.path()))) + .isEqualTo(targetTable.io().newInputFile(rewrittenManifest.path()).getLength()); + } + + @Test + public void testKeepsExistingDeleteManifestLengthInManifestList(@TempDir Path rootTargetLocation) + throws Exception { + String location = newTableLocation(); + Table sourceTable = createTableWithSnapshots(location, 5); + + // Add two more snapshots with just position deletes + addAnyPositionDelete( + sourceTable, + removePrefix(sourceTable.location() + "/data/deeply/nested/deletes-1.parquet")); + addAnyPositionDelete( + sourceTable, + removePrefix(sourceTable.location() + "/data/deeply/nested/deletes-2.parquet")); + + Map manifestSizesBeforeRewrite = + sourceTable.currentSnapshot().allManifests(sourceTable.io()).stream() + .collect(Collectors.toMap(m -> fileName(m.path()), m -> m.length())); + + // Rewrite just the latest table version to a location that's much longer than + // the original in order to make manifests larger + String targetLocation = toAbsolute(rootTargetLocation) + generateLongNestedPath(25); + RewriteTablePath.Result result = + actions() + .rewriteTablePath(sourceTable) + .rewriteLocationPrefix(newTableLocation(), targetLocation) + .startVersion("v7.metadata.json") + .execute(); + + // 1 metadata JSON file, 1 snapshot, 1 manifest, 1 delete file + checkFileNum(1, 1, 1, 4, result); + copyTableFiles(result); + + Table targetTable = TABLES.load(targetLocation); + Snapshot lastSnapshot = targetTable.currentSnapshot(); + + List allManifests = lastSnapshot.allManifests(targetTable.io()); + ManifestFile rewrittenManifest = + Iterables.getOnlyElement( + allManifests.stream() + .filter(manifest -> manifest.snapshotId() == lastSnapshot.snapshotId()) + .collect(Collectors.toList())); + + // We have rewritten exactly one (delete) manifest. Check that its size was correctly + // updated in the manifest list and sizes of all other manifests were left unchanged + assertThat(allManifests) + .filteredOn(manifest -> manifest != rewrittenManifest) + .allSatisfy( + manifest -> { + String manifestName = fileName(manifest.path()); + assertThat(manifest.length()).isEqualTo(manifestSizesBeforeRewrite.get(manifestName)); + }); + assertThat(rewrittenManifest.content()).isEqualTo(ManifestContent.DELETES); + assertThat(rewrittenManifest.length()) + .isNotEqualTo(manifestSizesBeforeRewrite.get(fileName(rewrittenManifest.path()))) + .isEqualTo(targetTable.io().newInputFile(rewrittenManifest.path()).getLength()); + } + + protected static String generateLongNestedPath(int depth) { + StringBuilder pathBuilder = new StringBuilder(); + for (int i = 1; i <= depth; i++) { + pathBuilder.append(String.format("/%03d", i)); + } + pathBuilder.append("/"); + return pathBuilder.toString(); + } + + protected void addAnyPositionDelete(Table table, String path) throws Exception { + DataFile dataFile = + StreamSupport.stream(table.snapshots().spliterator(), false) + .flatMap( + snapshot -> + StreamSupport.stream(snapshot.addedDataFiles(table.io()).spliterator(), false)) + .findAny() + .get(); + + List> deletes = Lists.newArrayList(Pair.of(dataFile.location(), 0L)); + File file = new File(path); + + DeleteFile positionDeletes = + FileHelpers.writeDeleteFile( + table, table.io().newOutputFile(file.toURI().toString()), deletes) + .first(); + + table.newRowDelta().addDeletes(positionDeletes).commit(); + } + protected void checkFileNum( int versionFileCount, int manifestListCount, diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteTablePathSparkAction.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteTablePathSparkAction.java index fbeabfe63e0e..21abf32cf0c9 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteTablePathSparkAction.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteTablePathSparkAction.java @@ -22,11 +22,13 @@ import java.io.IOException; import java.io.OutputStreamWriter; import java.nio.charset.StandardCharsets; +import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Set; import java.util.UUID; import java.util.stream.Collectors; +import java.util.stream.Stream; import org.apache.iceberg.ContentFile; import org.apache.iceberg.DataFile; import org.apache.iceberg.DeleteFile; @@ -37,6 +39,7 @@ import org.apache.iceberg.RewriteTablePathUtil; import org.apache.iceberg.RewriteTablePathUtil.PositionDeleteReaderWriter; import org.apache.iceberg.RewriteTablePathUtil.RewriteResult; +import org.apache.iceberg.RewriteTablePathUtil.RewrittenFileInfo; import org.apache.iceberg.Schema; import org.apache.iceberg.Snapshot; import org.apache.iceberg.StaticTableOperations; @@ -57,6 +60,7 @@ import org.apache.iceberg.data.parquet.GenericParquetReaders; import org.apache.iceberg.data.parquet.GenericParquetWriter; import org.apache.iceberg.deletes.PositionDeleteWriter; +import org.apache.iceberg.exceptions.NotFoundException; import org.apache.iceberg.exceptions.RuntimeIOException; import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.io.DeleteSchemaUtil; @@ -68,13 +72,13 @@ 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.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.spark.JobGroupInfo; import org.apache.iceberg.spark.source.SerializableTableWithSize; import org.apache.iceberg.util.Pair; import org.apache.spark.api.java.function.ForeachFunction; import org.apache.spark.api.java.function.MapFunction; -import org.apache.spark.api.java.function.ReduceFunction; import org.apache.spark.broadcast.Broadcast; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Encoder; @@ -84,6 +88,7 @@ import org.apache.spark.sql.functions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import scala.Tuple3; public class RewriteTablePathSparkAction extends BaseSparkAction implements RewriteTablePath { @@ -282,23 +287,25 @@ private String rebuildMetadata() { RewriteResult rewriteVersionResult = rewriteVersionFiles(endMetadata); Set deltaSnapshots = deltaSnapshots(startMetadata, rewriteVersionResult.toRewrite()); - Set manifestsToRewrite = manifestsToRewrite(deltaSnapshots, startMetadata); - Set validSnapshots = - Sets.difference(snapshotSet(endMetadata), snapshotSet(startMetadata)); + // rebuild manifest files + Set manifestsToRewrite = manifestsToRewrite(deltaSnapshots, startMetadata); + ManifestsRewriteResult rewriteManifestResult = + rewriteManifests(deltaSnapshots, endMetadata, manifestsToRewrite); // rebuild manifest-list files + Set validSnapshots = + Sets.difference(snapshotSet(endMetadata), snapshotSet(startMetadata)); RewriteResult rewriteManifestListResult = validSnapshots.stream() - .map(snapshot -> rewriteManifestList(snapshot, endMetadata, manifestsToRewrite)) + .map( + snapshot -> + rewriteManifestList( + snapshot, endMetadata, rewriteManifestResult.getRewrittenManifests())) .reduce(new RewriteResult<>(), RewriteResult::append); - // rebuild manifest files - RewriteContentFileResult rewriteManifestResult = - rewriteManifests(deltaSnapshots, endMetadata, rewriteManifestListResult.toRewrite()); - // rebuild position delete files Set deleteFiles = - rewriteManifestResult.toRewrite().stream() + rewriteManifestResult.getContentFileResult().toRewrite().stream() .filter(e -> e instanceof DeleteFile) .map(e -> (DeleteFile) e) .collect(Collectors.toSet()); @@ -307,8 +314,7 @@ private String rebuildMetadata() { Set> copyPlan = Sets.newHashSet(); copyPlan.addAll(rewriteVersionResult.copyPlan()); copyPlan.addAll(rewriteManifestListResult.copyPlan()); - copyPlan.addAll(rewriteManifestResult.copyPlan()); - + copyPlan.addAll(rewriteManifestResult.getContentFileResult().copyPlan()); return saveFileList(copyPlan); } @@ -415,27 +421,26 @@ private Set> statsFileCopyPlan( * @param snapshot snapshot represented by the manifest list * @param tableMetadata metadata of table * @param manifestsToRewrite filter of manifests to rewrite. - * @return a result including a copy plan for the manifests contained in the manifest list, as - * well as for the manifest list itself + * @return a result including a copy plan for the manifest list itself */ private RewriteResult rewriteManifestList( - Snapshot snapshot, TableMetadata tableMetadata, Set manifestsToRewrite) { + Snapshot snapshot, + TableMetadata tableMetadata, + Map rewrittenManifests) { RewriteResult result = new RewriteResult<>(); String path = snapshot.manifestListLocation(); String outputPath = RewriteTablePathUtil.stagingPath(path, stagingDir); - RewriteResult rewriteResult = - RewriteTablePathUtil.rewriteManifestList( - snapshot, - table.io(), - tableMetadata, - manifestsToRewrite, - sourcePrefix, - targetPrefix, - stagingDir, - outputPath); - - result.append(rewriteResult); + RewriteTablePathUtil.rewriteManifestList( + snapshot, + table.io(), + tableMetadata, + rewrittenManifests, + sourcePrefix, + targetPrefix, + stagingDir, + outputPath); + // add the manifest list copy plan itself to the result result .copyPlan() @@ -443,26 +448,58 @@ private RewriteResult rewriteManifestList( return result; } - private Set manifestsToRewrite( + private Set manifestsToRewrite( Set deltaSnapshots, TableMetadata startMetadata) { try { Table endStaticTable = newStaticTable(endVersionName, table.io()); - Dataset lastVersionFiles = manifestDS(endStaticTable).select("path"); + Dataset allManifestsDF = manifestDS(endStaticTable).select("path"); + Set expectedManifestPaths; + if (startMetadata == null) { - return Sets.newHashSet(lastVersionFiles.distinct().as(Encoders.STRING()).collectAsList()); + expectedManifestPaths = + Sets.newHashSet(allManifestsDF.distinct().as(Encoders.STRING()).collectAsList()); } else { Set deltaSnapshotIds = deltaSnapshots.stream().map(Snapshot::snapshotId).collect(Collectors.toSet()); - return Sets.newHashSet( - lastVersionFiles - .distinct() - .filter( - functions - .column(ManifestFile.SNAPSHOT_ID.name()) - .isInCollection(deltaSnapshotIds)) - .as(Encoders.STRING()) - .collectAsList()); + expectedManifestPaths = + Sets.newHashSet( + allManifestsDF + .distinct() + .filter( + functions + .column(ManifestFile.SNAPSHOT_ID.name()) + .isInCollection(deltaSnapshotIds)) + .as(Encoders.STRING()) + .collectAsList()); } + + Set foundManifests = + deltaSnapshots.stream() + .flatMap( + s -> { + try { + return s.allManifests(table.io()).stream(); + } catch (NotFoundException e) { + LOG.warn( + "Skipping snapshot {} as its manifest list is missing (likely expired).", + s.snapshotId(), + e); + return Stream.empty(); + } + }) + .collect(Collectors.toSet()); + + Set foundManifestPaths = + foundManifests.stream().map(ManifestFile::path).collect(Collectors.toSet()); + Set missingPaths = Sets.difference(expectedManifestPaths, foundManifestPaths); + Preconditions.checkState( + missingPaths.isEmpty(), + "Could not find all expected manifests. Missing files: %s", + String.join(", ", missingPaths)); + + return foundManifests.stream() + .filter(m -> expectedManifestPaths.contains(m.path())) + .collect(Collectors.toSet()); } catch (Exception e) { throw new UnsupportedOperationException( "Unable to build the manifest files dataframe. The end version in use may contain invalid snapshots. " @@ -491,48 +528,94 @@ public RewriteContentFileResult appendDeleteFile(RewriteResult r1) { } } - /** Rewrite manifest files in a distributed manner and return rewritten data files path pairs. */ - private RewriteContentFileResult rewriteManifests( + public static class ManifestsRewriteResult { + private final RewriteContentFileResult contentFileResult; + private final Map rewrittenManifests; + + ManifestsRewriteResult( + RewriteContentFileResult contentFileResult, + Map rewrittenManifests) { + this.contentFileResult = contentFileResult; + this.rewrittenManifests = rewrittenManifests; + } + + public RewriteContentFileResult getContentFileResult() { + return contentFileResult; + } + + public Map getRewrittenManifests() { + return rewrittenManifests; + } + } + + /** Rewrite manifest files in a distributed manner and return the resulting manifests + * and content files selected for rewriting. + */ + private ManifestsRewriteResult rewriteManifests( Set deltaSnapshots, TableMetadata tableMetadata, Set toRewrite) { if (toRewrite.isEmpty()) { - return new RewriteContentFileResult(); + return new ManifestsRewriteResult(new RewriteContentFileResult(), Maps.newHashMap()); } Encoder manifestFileEncoder = Encoders.javaSerialization(ManifestFile.class); + Encoder contentResultEncoder = + Encoders.javaSerialization(RewriteContentFileResult.class); + Encoder> tupleEncoder = + Encoders.tuple(Encoders.STRING(), manifestFileEncoder, contentResultEncoder); + Dataset manifestDS = spark().createDataset(Lists.newArrayList(toRewrite), manifestFileEncoder); Set deltaSnapshotIds = deltaSnapshots.stream().map(Snapshot::snapshotId).collect(Collectors.toSet()); - return manifestDS - .repartition(toRewrite.size()) - .map( - toManifests( - tableBroadcast(), - sparkContext().broadcast(deltaSnapshotIds), - stagingDir, - tableMetadata.formatVersion(), - sourcePrefix, - targetPrefix), - Encoders.bean(RewriteContentFileResult.class)) - // duplicates are expected here as the same data file can have different statuses - // (e.g. added and deleted) - .reduce((ReduceFunction) RewriteContentFileResult::append); - } - - private static MapFunction toManifests( - Broadcast
table, - Broadcast> deltaSnapshotIds, - String stagingLocation, - int format, - String sourcePrefix, - String targetPrefix) { + RewriteContentFileResult finalContentResult = new RewriteContentFileResult(); + Iterator> resultIterator = + manifestDS + .repartition(toRewrite.size()) + .map( + toManifests( + tableBroadcast(), + sparkContext().broadcast(deltaSnapshotIds), + stagingDir, + tableMetadata.formatVersion(), + sourcePrefix, + targetPrefix), + tupleEncoder) + .toLocalIterator(); + + Map rewrittenManifests = Maps.newHashMap(); + + while (resultIterator.hasNext()) { + Tuple3 resultTuple = resultIterator.next(); + String originalManifestPath = resultTuple._1(); + ManifestFile rewrittenManifest = resultTuple._2(); + RewriteContentFileResult contentFileResult = resultTuple._3(); + String targetManifestPath = + RewriteTablePathUtil.newPath(originalManifestPath, sourcePrefix, targetPrefix); + + finalContentResult.append(contentFileResult); + finalContentResult.copyPlan().add(Pair.of(rewrittenManifest.path(), targetManifestPath)); + rewrittenManifests.put( + originalManifestPath, + new RewrittenFileInfo(targetManifestPath, rewrittenManifest.length())); + } + + return new ManifestsRewriteResult(finalContentResult, rewrittenManifests); + } + + private static MapFunction> + toManifests( + Broadcast
table, + Broadcast> deltaSnapshotIds, + String stagingLocation, + int format, + String sourcePrefix, + String targetPrefix) { return manifestFile -> { - RewriteContentFileResult result = new RewriteContentFileResult(); switch (manifestFile.content()) { case DATA: - result.appendDataFile( + Pair> dataFileResult = writeDataManifest( manifestFile, table, @@ -540,10 +623,13 @@ private static MapFunction toManifests( stagingLocation, format, sourcePrefix, - targetPrefix)); - break; + targetPrefix); + return Tuple3.apply( + manifestFile.path(), + dataFileResult.first(), + new RewriteContentFileResult().appendDataFile(dataFileResult.second())); case DELETES: - result.appendDeleteFile( + Pair> deleteFileResult = writeDeleteManifest( manifestFile, table, @@ -551,17 +637,19 @@ private static MapFunction toManifests( stagingLocation, format, sourcePrefix, - targetPrefix)); - break; + targetPrefix); + return Tuple3.apply( + manifestFile.path(), + deleteFileResult.first(), + new RewriteContentFileResult().appendDeleteFile(deleteFileResult.second())); default: throw new UnsupportedOperationException( "Unsupported manifest type: " + manifestFile.content()); } - return result; }; } - private static RewriteResult writeDataManifest( + private static Pair> writeDataManifest( ManifestFile manifestFile, Broadcast
table, Broadcast> snapshotIds, @@ -575,7 +663,7 @@ private static RewriteResult writeDataManifest( OutputFile outputFile = io.newOutputFile(stagingPath); Map specsById = table.getValue().specs(); Set deltaSnapshotIds = snapshotIds.value(); - return RewriteTablePathUtil.rewriteDataManifest( + return RewriteTablePathUtil.rewriteDataManifestWithResult( manifestFile, deltaSnapshotIds, outputFile, @@ -589,7 +677,7 @@ private static RewriteResult writeDataManifest( } } - private static RewriteResult writeDeleteManifest( + private static Pair> writeDeleteManifest( ManifestFile manifestFile, Broadcast
table, Broadcast> snapshotIds, @@ -603,7 +691,7 @@ private static RewriteResult writeDeleteManifest( OutputFile outputFile = io.newOutputFile(stagingPath); Map specsById = table.getValue().specs(); Set deltaSnapshotIds = snapshotIds.value(); - return RewriteTablePathUtil.rewriteDeleteManifest( + return RewriteTablePathUtil.rewriteDeleteManifestWithResult( manifestFile, deltaSnapshotIds, outputFile, diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteTablePathsAction.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteTablePathsAction.java index 705e0074c79f..c737d551821e 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteTablePathsAction.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteTablePathsAction.java @@ -39,6 +39,8 @@ import org.apache.iceberg.DeleteFile; import org.apache.iceberg.HasTableOperations; import org.apache.iceberg.ImmutableGenericPartitionStatisticsFile; +import org.apache.iceberg.ManifestContent; +import org.apache.iceberg.ManifestFile; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; import org.apache.iceberg.Snapshot; @@ -343,7 +345,7 @@ public void testManifestRewriteAndIncrementalCopy() throws Exception { int addedManifest = Iterables.size(rewriteManifestResult.addedManifests()); // only move version v4, which is the version generated by rewrite manifest - RewriteTablePath.Result postReweiteResult = + RewriteTablePath.Result postRewriteResult = actions() .rewriteTablePath(table) .rewriteLocationPrefix(table.location(), targetTableLocation()) @@ -352,7 +354,7 @@ public void testManifestRewriteAndIncrementalCopy() throws Exception { .execute(); // no data files need to move - checkFileNum(1, 1, addedManifest, 3, postReweiteResult); + checkFileNum(1, 1, addedManifest, 3, postRewriteResult); } @Test @@ -1122,6 +1124,184 @@ public void testKryoDeserializeBroadcastValues() { assertThat(tableBroadcast.getValue().uuid()).isEqualTo(table.uuid()); } + @Test + public void testFullRewriteUpdatesAllManifestLengthsInManifestList( + @TempDir Path rootTargetLocation) throws Exception { + String location = newTableLocation(); + Table sourceTable = createTableWithSnapshots(location, 10); + + addAnyPositionDelete( + sourceTable, removePrefix(sourceTable.location() + "/data/deeply/nested/deletes.parquet")); + + Map manifestSizesBeforeRewrite = + sourceTable.currentSnapshot().allManifests(sourceTable.io()).stream() + .collect(Collectors.toMap(m -> fileName(m.path()), m -> m.length())); + + // Rewrite table metadata to a location that's much longer than the original in order + // to make manifests larger + String targetLocation = toAbsolute(rootTargetLocation) + generateLongNestedPath(25); + RewriteTablePath.Result result = + actions() + .rewriteTablePath(sourceTable) + .rewriteLocationPrefix(newTableLocation(), targetLocation) + .execute(); + + // 1 + 11 JSON metadata files, 11 snapshots, 11 manifests, 10 data files, 1 delete file + checkFileNum(12, 11, 11, 45, result); + copyTableFiles(result); + + Table targetTable = TABLES.load(targetLocation); + + // We have rewritten all manifests. Make sure all sizes were correctly updated in manifest lists + assertThat(targetTable.currentSnapshot().allManifests(targetTable.io())) + .allSatisfy( + manifest -> { + String manifestName = fileName(manifest.path()); + assertThat(manifest.length()) + .isNotEqualTo(manifestSizesBeforeRewrite.get(manifestName)); + }) + .allSatisfy( + manifest -> { + assertThat(targetTable.io().newInputFile(manifest.path()).getLength()) + .isEqualTo(manifest.length()); + }); + } + + @Test + public void testKeepsExistingDataManifestLengthInManifestList(@TempDir Path rootTargetLocation) + throws Exception { + String location = newTableLocation(); + Table sourceTable = createTableWithSnapshots(location, 10); + + Map manifestSizesBeforeRewrite = + sourceTable.currentSnapshot().allManifests(sourceTable.io()).stream() + .collect(Collectors.toMap(m -> fileName(m.path()), m -> m.length())); + + // Rewrite just the latest table version to a location that's much longer than + // the original in order to make manifests larger + String targetLocation = toAbsolute(rootTargetLocation) + generateLongNestedPath(25); + RewriteTablePath.Result result = + actions() + .rewriteTablePath(sourceTable) + .rewriteLocationPrefix(newTableLocation(), targetLocation) + .startVersion("v10.metadata.json") + .execute(); + + // 1 metadata JSON file, 1 snapshot, 1 manifest, 4 data files + checkFileNum(1, 1, 1, 4, result); + copyTableFiles(result); + + Table targetTable = TABLES.load(targetLocation); + Snapshot lastSnapshot = targetTable.currentSnapshot(); + + List allManifests = lastSnapshot.allManifests(targetTable.io()); + ManifestFile rewrittenManifest = + Iterables.getOnlyElement( + allManifests.stream() + .filter(manifest -> manifest.snapshotId() == lastSnapshot.snapshotId()) + .collect(Collectors.toList())); + + // We have rewritten exactly one (data) manifest. Check that its size was correctly + // updated in the manifest list and sizes of all other manifests were left unchanged + assertThat(allManifests) + .filteredOn(manifest -> manifest != rewrittenManifest) + .allSatisfy( + manifest -> { + String manifestName = fileName(manifest.path()); + assertThat(manifest.length()).isEqualTo(manifestSizesBeforeRewrite.get(manifestName)); + }); + assertThat(rewrittenManifest.content()).isEqualTo(ManifestContent.DATA); + assertThat(rewrittenManifest.length()) + .isNotEqualTo(manifestSizesBeforeRewrite.get(fileName(rewrittenManifest.path()))) + .isEqualTo(targetTable.io().newInputFile(rewrittenManifest.path()).getLength()); + } + + @Test + public void testKeepsExistingDeleteManifestLengthInManifestList(@TempDir Path rootTargetLocation) + throws Exception { + String location = newTableLocation(); + Table sourceTable = createTableWithSnapshots(location, 5); + + // Add two more snapshots with just position deletes + addAnyPositionDelete( + sourceTable, + removePrefix(sourceTable.location() + "/data/deeply/nested/deletes-1.parquet")); + addAnyPositionDelete( + sourceTable, + removePrefix(sourceTable.location() + "/data/deeply/nested/deletes-2.parquet")); + + Map manifestSizesBeforeRewrite = + sourceTable.currentSnapshot().allManifests(sourceTable.io()).stream() + .collect(Collectors.toMap(m -> fileName(m.path()), m -> m.length())); + + // Rewrite just the latest table version to a location that's much longer than + // the original in order to make manifests larger + String targetLocation = toAbsolute(rootTargetLocation) + generateLongNestedPath(25); + RewriteTablePath.Result result = + actions() + .rewriteTablePath(sourceTable) + .rewriteLocationPrefix(newTableLocation(), targetLocation) + .startVersion("v7.metadata.json") + .execute(); + + // 1 metadata JSON file, 1 snapshot, 1 manifest, 1 delete file + checkFileNum(1, 1, 1, 4, result); + copyTableFiles(result); + + Table targetTable = TABLES.load(targetLocation); + Snapshot lastSnapshot = targetTable.currentSnapshot(); + + List allManifests = lastSnapshot.allManifests(targetTable.io()); + ManifestFile rewrittenManifest = + Iterables.getOnlyElement( + allManifests.stream() + .filter(manifest -> manifest.snapshotId() == lastSnapshot.snapshotId()) + .collect(Collectors.toList())); + + // We have rewritten exactly one (delete) manifest. Check that its size was correctly + // updated in the manifest list and sizes of all other manifests were left unchanged + assertThat(allManifests) + .filteredOn(manifest -> manifest != rewrittenManifest) + .allSatisfy( + manifest -> { + String manifestName = fileName(manifest.path()); + assertThat(manifest.length()).isEqualTo(manifestSizesBeforeRewrite.get(manifestName)); + }); + assertThat(rewrittenManifest.content()).isEqualTo(ManifestContent.DELETES); + assertThat(rewrittenManifest.length()) + .isNotEqualTo(manifestSizesBeforeRewrite.get(fileName(rewrittenManifest.path()))) + .isEqualTo(targetTable.io().newInputFile(rewrittenManifest.path()).getLength()); + } + + protected static String generateLongNestedPath(int depth) { + StringBuilder pathBuilder = new StringBuilder(); + for (int i = 1; i <= depth; i++) { + pathBuilder.append(String.format("/%03d", i)); + } + pathBuilder.append("/"); + return pathBuilder.toString(); + } + + protected void addAnyPositionDelete(Table table, String path) throws Exception { + DataFile dataFile = + StreamSupport.stream(table.snapshots().spliterator(), false) + .flatMap( + snapshot -> + StreamSupport.stream(snapshot.addedDataFiles(table.io()).spliterator(), false)) + .findAny() + .get(); + + List> deletes = Lists.newArrayList(Pair.of(dataFile.location(), 0L)); + File file = new File(path); + + DeleteFile positionDeletes = + FileHelpers.writeDeleteFile( + table, table.io().newOutputFile(file.toURI().toString()), deletes) + .first(); + + table.newRowDelta().addDeletes(positionDeletes).commit(); + } + protected void checkFileNum( int versionFileCount, int manifestListCount, diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteTablePathSparkAction.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteTablePathSparkAction.java index fbeabfe63e0e..21abf32cf0c9 100644 --- a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteTablePathSparkAction.java +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteTablePathSparkAction.java @@ -22,11 +22,13 @@ import java.io.IOException; import java.io.OutputStreamWriter; import java.nio.charset.StandardCharsets; +import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Set; import java.util.UUID; import java.util.stream.Collectors; +import java.util.stream.Stream; import org.apache.iceberg.ContentFile; import org.apache.iceberg.DataFile; import org.apache.iceberg.DeleteFile; @@ -37,6 +39,7 @@ import org.apache.iceberg.RewriteTablePathUtil; import org.apache.iceberg.RewriteTablePathUtil.PositionDeleteReaderWriter; import org.apache.iceberg.RewriteTablePathUtil.RewriteResult; +import org.apache.iceberg.RewriteTablePathUtil.RewrittenFileInfo; import org.apache.iceberg.Schema; import org.apache.iceberg.Snapshot; import org.apache.iceberg.StaticTableOperations; @@ -57,6 +60,7 @@ import org.apache.iceberg.data.parquet.GenericParquetReaders; import org.apache.iceberg.data.parquet.GenericParquetWriter; import org.apache.iceberg.deletes.PositionDeleteWriter; +import org.apache.iceberg.exceptions.NotFoundException; import org.apache.iceberg.exceptions.RuntimeIOException; import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.io.DeleteSchemaUtil; @@ -68,13 +72,13 @@ 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.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.spark.JobGroupInfo; import org.apache.iceberg.spark.source.SerializableTableWithSize; import org.apache.iceberg.util.Pair; import org.apache.spark.api.java.function.ForeachFunction; import org.apache.spark.api.java.function.MapFunction; -import org.apache.spark.api.java.function.ReduceFunction; import org.apache.spark.broadcast.Broadcast; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Encoder; @@ -84,6 +88,7 @@ import org.apache.spark.sql.functions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import scala.Tuple3; public class RewriteTablePathSparkAction extends BaseSparkAction implements RewriteTablePath { @@ -282,23 +287,25 @@ private String rebuildMetadata() { RewriteResult rewriteVersionResult = rewriteVersionFiles(endMetadata); Set deltaSnapshots = deltaSnapshots(startMetadata, rewriteVersionResult.toRewrite()); - Set manifestsToRewrite = manifestsToRewrite(deltaSnapshots, startMetadata); - Set validSnapshots = - Sets.difference(snapshotSet(endMetadata), snapshotSet(startMetadata)); + // rebuild manifest files + Set manifestsToRewrite = manifestsToRewrite(deltaSnapshots, startMetadata); + ManifestsRewriteResult rewriteManifestResult = + rewriteManifests(deltaSnapshots, endMetadata, manifestsToRewrite); // rebuild manifest-list files + Set validSnapshots = + Sets.difference(snapshotSet(endMetadata), snapshotSet(startMetadata)); RewriteResult rewriteManifestListResult = validSnapshots.stream() - .map(snapshot -> rewriteManifestList(snapshot, endMetadata, manifestsToRewrite)) + .map( + snapshot -> + rewriteManifestList( + snapshot, endMetadata, rewriteManifestResult.getRewrittenManifests())) .reduce(new RewriteResult<>(), RewriteResult::append); - // rebuild manifest files - RewriteContentFileResult rewriteManifestResult = - rewriteManifests(deltaSnapshots, endMetadata, rewriteManifestListResult.toRewrite()); - // rebuild position delete files Set deleteFiles = - rewriteManifestResult.toRewrite().stream() + rewriteManifestResult.getContentFileResult().toRewrite().stream() .filter(e -> e instanceof DeleteFile) .map(e -> (DeleteFile) e) .collect(Collectors.toSet()); @@ -307,8 +314,7 @@ private String rebuildMetadata() { Set> copyPlan = Sets.newHashSet(); copyPlan.addAll(rewriteVersionResult.copyPlan()); copyPlan.addAll(rewriteManifestListResult.copyPlan()); - copyPlan.addAll(rewriteManifestResult.copyPlan()); - + copyPlan.addAll(rewriteManifestResult.getContentFileResult().copyPlan()); return saveFileList(copyPlan); } @@ -415,27 +421,26 @@ private Set> statsFileCopyPlan( * @param snapshot snapshot represented by the manifest list * @param tableMetadata metadata of table * @param manifestsToRewrite filter of manifests to rewrite. - * @return a result including a copy plan for the manifests contained in the manifest list, as - * well as for the manifest list itself + * @return a result including a copy plan for the manifest list itself */ private RewriteResult rewriteManifestList( - Snapshot snapshot, TableMetadata tableMetadata, Set manifestsToRewrite) { + Snapshot snapshot, + TableMetadata tableMetadata, + Map rewrittenManifests) { RewriteResult result = new RewriteResult<>(); String path = snapshot.manifestListLocation(); String outputPath = RewriteTablePathUtil.stagingPath(path, stagingDir); - RewriteResult rewriteResult = - RewriteTablePathUtil.rewriteManifestList( - snapshot, - table.io(), - tableMetadata, - manifestsToRewrite, - sourcePrefix, - targetPrefix, - stagingDir, - outputPath); - - result.append(rewriteResult); + RewriteTablePathUtil.rewriteManifestList( + snapshot, + table.io(), + tableMetadata, + rewrittenManifests, + sourcePrefix, + targetPrefix, + stagingDir, + outputPath); + // add the manifest list copy plan itself to the result result .copyPlan() @@ -443,26 +448,58 @@ private RewriteResult rewriteManifestList( return result; } - private Set manifestsToRewrite( + private Set manifestsToRewrite( Set deltaSnapshots, TableMetadata startMetadata) { try { Table endStaticTable = newStaticTable(endVersionName, table.io()); - Dataset lastVersionFiles = manifestDS(endStaticTable).select("path"); + Dataset allManifestsDF = manifestDS(endStaticTable).select("path"); + Set expectedManifestPaths; + if (startMetadata == null) { - return Sets.newHashSet(lastVersionFiles.distinct().as(Encoders.STRING()).collectAsList()); + expectedManifestPaths = + Sets.newHashSet(allManifestsDF.distinct().as(Encoders.STRING()).collectAsList()); } else { Set deltaSnapshotIds = deltaSnapshots.stream().map(Snapshot::snapshotId).collect(Collectors.toSet()); - return Sets.newHashSet( - lastVersionFiles - .distinct() - .filter( - functions - .column(ManifestFile.SNAPSHOT_ID.name()) - .isInCollection(deltaSnapshotIds)) - .as(Encoders.STRING()) - .collectAsList()); + expectedManifestPaths = + Sets.newHashSet( + allManifestsDF + .distinct() + .filter( + functions + .column(ManifestFile.SNAPSHOT_ID.name()) + .isInCollection(deltaSnapshotIds)) + .as(Encoders.STRING()) + .collectAsList()); } + + Set foundManifests = + deltaSnapshots.stream() + .flatMap( + s -> { + try { + return s.allManifests(table.io()).stream(); + } catch (NotFoundException e) { + LOG.warn( + "Skipping snapshot {} as its manifest list is missing (likely expired).", + s.snapshotId(), + e); + return Stream.empty(); + } + }) + .collect(Collectors.toSet()); + + Set foundManifestPaths = + foundManifests.stream().map(ManifestFile::path).collect(Collectors.toSet()); + Set missingPaths = Sets.difference(expectedManifestPaths, foundManifestPaths); + Preconditions.checkState( + missingPaths.isEmpty(), + "Could not find all expected manifests. Missing files: %s", + String.join(", ", missingPaths)); + + return foundManifests.stream() + .filter(m -> expectedManifestPaths.contains(m.path())) + .collect(Collectors.toSet()); } catch (Exception e) { throw new UnsupportedOperationException( "Unable to build the manifest files dataframe. The end version in use may contain invalid snapshots. " @@ -491,48 +528,94 @@ public RewriteContentFileResult appendDeleteFile(RewriteResult r1) { } } - /** Rewrite manifest files in a distributed manner and return rewritten data files path pairs. */ - private RewriteContentFileResult rewriteManifests( + public static class ManifestsRewriteResult { + private final RewriteContentFileResult contentFileResult; + private final Map rewrittenManifests; + + ManifestsRewriteResult( + RewriteContentFileResult contentFileResult, + Map rewrittenManifests) { + this.contentFileResult = contentFileResult; + this.rewrittenManifests = rewrittenManifests; + } + + public RewriteContentFileResult getContentFileResult() { + return contentFileResult; + } + + public Map getRewrittenManifests() { + return rewrittenManifests; + } + } + + /** Rewrite manifest files in a distributed manner and return the resulting manifests + * and content files selected for rewriting. + */ + private ManifestsRewriteResult rewriteManifests( Set deltaSnapshots, TableMetadata tableMetadata, Set toRewrite) { if (toRewrite.isEmpty()) { - return new RewriteContentFileResult(); + return new ManifestsRewriteResult(new RewriteContentFileResult(), Maps.newHashMap()); } Encoder manifestFileEncoder = Encoders.javaSerialization(ManifestFile.class); + Encoder contentResultEncoder = + Encoders.javaSerialization(RewriteContentFileResult.class); + Encoder> tupleEncoder = + Encoders.tuple(Encoders.STRING(), manifestFileEncoder, contentResultEncoder); + Dataset manifestDS = spark().createDataset(Lists.newArrayList(toRewrite), manifestFileEncoder); Set deltaSnapshotIds = deltaSnapshots.stream().map(Snapshot::snapshotId).collect(Collectors.toSet()); - return manifestDS - .repartition(toRewrite.size()) - .map( - toManifests( - tableBroadcast(), - sparkContext().broadcast(deltaSnapshotIds), - stagingDir, - tableMetadata.formatVersion(), - sourcePrefix, - targetPrefix), - Encoders.bean(RewriteContentFileResult.class)) - // duplicates are expected here as the same data file can have different statuses - // (e.g. added and deleted) - .reduce((ReduceFunction) RewriteContentFileResult::append); - } - - private static MapFunction toManifests( - Broadcast
table, - Broadcast> deltaSnapshotIds, - String stagingLocation, - int format, - String sourcePrefix, - String targetPrefix) { + RewriteContentFileResult finalContentResult = new RewriteContentFileResult(); + Iterator> resultIterator = + manifestDS + .repartition(toRewrite.size()) + .map( + toManifests( + tableBroadcast(), + sparkContext().broadcast(deltaSnapshotIds), + stagingDir, + tableMetadata.formatVersion(), + sourcePrefix, + targetPrefix), + tupleEncoder) + .toLocalIterator(); + + Map rewrittenManifests = Maps.newHashMap(); + + while (resultIterator.hasNext()) { + Tuple3 resultTuple = resultIterator.next(); + String originalManifestPath = resultTuple._1(); + ManifestFile rewrittenManifest = resultTuple._2(); + RewriteContentFileResult contentFileResult = resultTuple._3(); + String targetManifestPath = + RewriteTablePathUtil.newPath(originalManifestPath, sourcePrefix, targetPrefix); + + finalContentResult.append(contentFileResult); + finalContentResult.copyPlan().add(Pair.of(rewrittenManifest.path(), targetManifestPath)); + rewrittenManifests.put( + originalManifestPath, + new RewrittenFileInfo(targetManifestPath, rewrittenManifest.length())); + } + + return new ManifestsRewriteResult(finalContentResult, rewrittenManifests); + } + + private static MapFunction> + toManifests( + Broadcast
table, + Broadcast> deltaSnapshotIds, + String stagingLocation, + int format, + String sourcePrefix, + String targetPrefix) { return manifestFile -> { - RewriteContentFileResult result = new RewriteContentFileResult(); switch (manifestFile.content()) { case DATA: - result.appendDataFile( + Pair> dataFileResult = writeDataManifest( manifestFile, table, @@ -540,10 +623,13 @@ private static MapFunction toManifests( stagingLocation, format, sourcePrefix, - targetPrefix)); - break; + targetPrefix); + return Tuple3.apply( + manifestFile.path(), + dataFileResult.first(), + new RewriteContentFileResult().appendDataFile(dataFileResult.second())); case DELETES: - result.appendDeleteFile( + Pair> deleteFileResult = writeDeleteManifest( manifestFile, table, @@ -551,17 +637,19 @@ private static MapFunction toManifests( stagingLocation, format, sourcePrefix, - targetPrefix)); - break; + targetPrefix); + return Tuple3.apply( + manifestFile.path(), + deleteFileResult.first(), + new RewriteContentFileResult().appendDeleteFile(deleteFileResult.second())); default: throw new UnsupportedOperationException( "Unsupported manifest type: " + manifestFile.content()); } - return result; }; } - private static RewriteResult writeDataManifest( + private static Pair> writeDataManifest( ManifestFile manifestFile, Broadcast
table, Broadcast> snapshotIds, @@ -575,7 +663,7 @@ private static RewriteResult writeDataManifest( OutputFile outputFile = io.newOutputFile(stagingPath); Map specsById = table.getValue().specs(); Set deltaSnapshotIds = snapshotIds.value(); - return RewriteTablePathUtil.rewriteDataManifest( + return RewriteTablePathUtil.rewriteDataManifestWithResult( manifestFile, deltaSnapshotIds, outputFile, @@ -589,7 +677,7 @@ private static RewriteResult writeDataManifest( } } - private static RewriteResult writeDeleteManifest( + private static Pair> writeDeleteManifest( ManifestFile manifestFile, Broadcast
table, Broadcast> snapshotIds, @@ -603,7 +691,7 @@ private static RewriteResult writeDeleteManifest( OutputFile outputFile = io.newOutputFile(stagingPath); Map specsById = table.getValue().specs(); Set deltaSnapshotIds = snapshotIds.value(); - return RewriteTablePathUtil.rewriteDeleteManifest( + return RewriteTablePathUtil.rewriteDeleteManifestWithResult( manifestFile, deltaSnapshotIds, outputFile, diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteTablePathsAction.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteTablePathsAction.java index 705e0074c79f..c737d551821e 100644 --- a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteTablePathsAction.java +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteTablePathsAction.java @@ -39,6 +39,8 @@ import org.apache.iceberg.DeleteFile; import org.apache.iceberg.HasTableOperations; import org.apache.iceberg.ImmutableGenericPartitionStatisticsFile; +import org.apache.iceberg.ManifestContent; +import org.apache.iceberg.ManifestFile; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; import org.apache.iceberg.Snapshot; @@ -343,7 +345,7 @@ public void testManifestRewriteAndIncrementalCopy() throws Exception { int addedManifest = Iterables.size(rewriteManifestResult.addedManifests()); // only move version v4, which is the version generated by rewrite manifest - RewriteTablePath.Result postReweiteResult = + RewriteTablePath.Result postRewriteResult = actions() .rewriteTablePath(table) .rewriteLocationPrefix(table.location(), targetTableLocation()) @@ -352,7 +354,7 @@ public void testManifestRewriteAndIncrementalCopy() throws Exception { .execute(); // no data files need to move - checkFileNum(1, 1, addedManifest, 3, postReweiteResult); + checkFileNum(1, 1, addedManifest, 3, postRewriteResult); } @Test @@ -1122,6 +1124,184 @@ public void testKryoDeserializeBroadcastValues() { assertThat(tableBroadcast.getValue().uuid()).isEqualTo(table.uuid()); } + @Test + public void testFullRewriteUpdatesAllManifestLengthsInManifestList( + @TempDir Path rootTargetLocation) throws Exception { + String location = newTableLocation(); + Table sourceTable = createTableWithSnapshots(location, 10); + + addAnyPositionDelete( + sourceTable, removePrefix(sourceTable.location() + "/data/deeply/nested/deletes.parquet")); + + Map manifestSizesBeforeRewrite = + sourceTable.currentSnapshot().allManifests(sourceTable.io()).stream() + .collect(Collectors.toMap(m -> fileName(m.path()), m -> m.length())); + + // Rewrite table metadata to a location that's much longer than the original in order + // to make manifests larger + String targetLocation = toAbsolute(rootTargetLocation) + generateLongNestedPath(25); + RewriteTablePath.Result result = + actions() + .rewriteTablePath(sourceTable) + .rewriteLocationPrefix(newTableLocation(), targetLocation) + .execute(); + + // 1 + 11 JSON metadata files, 11 snapshots, 11 manifests, 10 data files, 1 delete file + checkFileNum(12, 11, 11, 45, result); + copyTableFiles(result); + + Table targetTable = TABLES.load(targetLocation); + + // We have rewritten all manifests. Make sure all sizes were correctly updated in manifest lists + assertThat(targetTable.currentSnapshot().allManifests(targetTable.io())) + .allSatisfy( + manifest -> { + String manifestName = fileName(manifest.path()); + assertThat(manifest.length()) + .isNotEqualTo(manifestSizesBeforeRewrite.get(manifestName)); + }) + .allSatisfy( + manifest -> { + assertThat(targetTable.io().newInputFile(manifest.path()).getLength()) + .isEqualTo(manifest.length()); + }); + } + + @Test + public void testKeepsExistingDataManifestLengthInManifestList(@TempDir Path rootTargetLocation) + throws Exception { + String location = newTableLocation(); + Table sourceTable = createTableWithSnapshots(location, 10); + + Map manifestSizesBeforeRewrite = + sourceTable.currentSnapshot().allManifests(sourceTable.io()).stream() + .collect(Collectors.toMap(m -> fileName(m.path()), m -> m.length())); + + // Rewrite just the latest table version to a location that's much longer than + // the original in order to make manifests larger + String targetLocation = toAbsolute(rootTargetLocation) + generateLongNestedPath(25); + RewriteTablePath.Result result = + actions() + .rewriteTablePath(sourceTable) + .rewriteLocationPrefix(newTableLocation(), targetLocation) + .startVersion("v10.metadata.json") + .execute(); + + // 1 metadata JSON file, 1 snapshot, 1 manifest, 4 data files + checkFileNum(1, 1, 1, 4, result); + copyTableFiles(result); + + Table targetTable = TABLES.load(targetLocation); + Snapshot lastSnapshot = targetTable.currentSnapshot(); + + List allManifests = lastSnapshot.allManifests(targetTable.io()); + ManifestFile rewrittenManifest = + Iterables.getOnlyElement( + allManifests.stream() + .filter(manifest -> manifest.snapshotId() == lastSnapshot.snapshotId()) + .collect(Collectors.toList())); + + // We have rewritten exactly one (data) manifest. Check that its size was correctly + // updated in the manifest list and sizes of all other manifests were left unchanged + assertThat(allManifests) + .filteredOn(manifest -> manifest != rewrittenManifest) + .allSatisfy( + manifest -> { + String manifestName = fileName(manifest.path()); + assertThat(manifest.length()).isEqualTo(manifestSizesBeforeRewrite.get(manifestName)); + }); + assertThat(rewrittenManifest.content()).isEqualTo(ManifestContent.DATA); + assertThat(rewrittenManifest.length()) + .isNotEqualTo(manifestSizesBeforeRewrite.get(fileName(rewrittenManifest.path()))) + .isEqualTo(targetTable.io().newInputFile(rewrittenManifest.path()).getLength()); + } + + @Test + public void testKeepsExistingDeleteManifestLengthInManifestList(@TempDir Path rootTargetLocation) + throws Exception { + String location = newTableLocation(); + Table sourceTable = createTableWithSnapshots(location, 5); + + // Add two more snapshots with just position deletes + addAnyPositionDelete( + sourceTable, + removePrefix(sourceTable.location() + "/data/deeply/nested/deletes-1.parquet")); + addAnyPositionDelete( + sourceTable, + removePrefix(sourceTable.location() + "/data/deeply/nested/deletes-2.parquet")); + + Map manifestSizesBeforeRewrite = + sourceTable.currentSnapshot().allManifests(sourceTable.io()).stream() + .collect(Collectors.toMap(m -> fileName(m.path()), m -> m.length())); + + // Rewrite just the latest table version to a location that's much longer than + // the original in order to make manifests larger + String targetLocation = toAbsolute(rootTargetLocation) + generateLongNestedPath(25); + RewriteTablePath.Result result = + actions() + .rewriteTablePath(sourceTable) + .rewriteLocationPrefix(newTableLocation(), targetLocation) + .startVersion("v7.metadata.json") + .execute(); + + // 1 metadata JSON file, 1 snapshot, 1 manifest, 1 delete file + checkFileNum(1, 1, 1, 4, result); + copyTableFiles(result); + + Table targetTable = TABLES.load(targetLocation); + Snapshot lastSnapshot = targetTable.currentSnapshot(); + + List allManifests = lastSnapshot.allManifests(targetTable.io()); + ManifestFile rewrittenManifest = + Iterables.getOnlyElement( + allManifests.stream() + .filter(manifest -> manifest.snapshotId() == lastSnapshot.snapshotId()) + .collect(Collectors.toList())); + + // We have rewritten exactly one (delete) manifest. Check that its size was correctly + // updated in the manifest list and sizes of all other manifests were left unchanged + assertThat(allManifests) + .filteredOn(manifest -> manifest != rewrittenManifest) + .allSatisfy( + manifest -> { + String manifestName = fileName(manifest.path()); + assertThat(manifest.length()).isEqualTo(manifestSizesBeforeRewrite.get(manifestName)); + }); + assertThat(rewrittenManifest.content()).isEqualTo(ManifestContent.DELETES); + assertThat(rewrittenManifest.length()) + .isNotEqualTo(manifestSizesBeforeRewrite.get(fileName(rewrittenManifest.path()))) + .isEqualTo(targetTable.io().newInputFile(rewrittenManifest.path()).getLength()); + } + + protected static String generateLongNestedPath(int depth) { + StringBuilder pathBuilder = new StringBuilder(); + for (int i = 1; i <= depth; i++) { + pathBuilder.append(String.format("/%03d", i)); + } + pathBuilder.append("/"); + return pathBuilder.toString(); + } + + protected void addAnyPositionDelete(Table table, String path) throws Exception { + DataFile dataFile = + StreamSupport.stream(table.snapshots().spliterator(), false) + .flatMap( + snapshot -> + StreamSupport.stream(snapshot.addedDataFiles(table.io()).spliterator(), false)) + .findAny() + .get(); + + List> deletes = Lists.newArrayList(Pair.of(dataFile.location(), 0L)); + File file = new File(path); + + DeleteFile positionDeletes = + FileHelpers.writeDeleteFile( + table, table.io().newOutputFile(file.toURI().toString()), deletes) + .first(); + + table.newRowDelta().addDeletes(positionDeletes).commit(); + } + protected void checkFileNum( int versionFileCount, int manifestListCount, From bbb5b27ec7f5a0097753174a65291b84a85f80f5 Mon Sep 17 00:00:00 2001 From: vaultah <4944562+vaultah@users.noreply.github.com> Date: Tue, 5 Aug 2025 00:43:50 +0100 Subject: [PATCH 02/18] Revert changes for 3.4 and 3.5 --- .../actions/RewriteTablePathSparkAction.java | 242 ++++++------------ .../actions/TestRewriteTablePathsAction.java | 184 +------------ .../actions/RewriteTablePathSparkAction.java | 242 ++++++------------ .../actions/TestRewriteTablePathsAction.java | 184 +------------ .../actions/RewriteTablePathSparkAction.java | 5 +- 5 files changed, 161 insertions(+), 696 deletions(-) diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteTablePathSparkAction.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteTablePathSparkAction.java index 21abf32cf0c9..fbeabfe63e0e 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteTablePathSparkAction.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteTablePathSparkAction.java @@ -22,13 +22,11 @@ import java.io.IOException; import java.io.OutputStreamWriter; import java.nio.charset.StandardCharsets; -import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Set; import java.util.UUID; import java.util.stream.Collectors; -import java.util.stream.Stream; import org.apache.iceberg.ContentFile; import org.apache.iceberg.DataFile; import org.apache.iceberg.DeleteFile; @@ -39,7 +37,6 @@ import org.apache.iceberg.RewriteTablePathUtil; import org.apache.iceberg.RewriteTablePathUtil.PositionDeleteReaderWriter; import org.apache.iceberg.RewriteTablePathUtil.RewriteResult; -import org.apache.iceberg.RewriteTablePathUtil.RewrittenFileInfo; import org.apache.iceberg.Schema; import org.apache.iceberg.Snapshot; import org.apache.iceberg.StaticTableOperations; @@ -60,7 +57,6 @@ import org.apache.iceberg.data.parquet.GenericParquetReaders; import org.apache.iceberg.data.parquet.GenericParquetWriter; import org.apache.iceberg.deletes.PositionDeleteWriter; -import org.apache.iceberg.exceptions.NotFoundException; import org.apache.iceberg.exceptions.RuntimeIOException; import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.io.DeleteSchemaUtil; @@ -72,13 +68,13 @@ 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.Lists; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.spark.JobGroupInfo; import org.apache.iceberg.spark.source.SerializableTableWithSize; import org.apache.iceberg.util.Pair; import org.apache.spark.api.java.function.ForeachFunction; import org.apache.spark.api.java.function.MapFunction; +import org.apache.spark.api.java.function.ReduceFunction; import org.apache.spark.broadcast.Broadcast; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Encoder; @@ -88,7 +84,6 @@ import org.apache.spark.sql.functions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import scala.Tuple3; public class RewriteTablePathSparkAction extends BaseSparkAction implements RewriteTablePath { @@ -287,25 +282,23 @@ private String rebuildMetadata() { RewriteResult rewriteVersionResult = rewriteVersionFiles(endMetadata); Set deltaSnapshots = deltaSnapshots(startMetadata, rewriteVersionResult.toRewrite()); - // rebuild manifest files - Set manifestsToRewrite = manifestsToRewrite(deltaSnapshots, startMetadata); - ManifestsRewriteResult rewriteManifestResult = - rewriteManifests(deltaSnapshots, endMetadata, manifestsToRewrite); - - // rebuild manifest-list files + Set manifestsToRewrite = manifestsToRewrite(deltaSnapshots, startMetadata); Set validSnapshots = Sets.difference(snapshotSet(endMetadata), snapshotSet(startMetadata)); + + // rebuild manifest-list files RewriteResult rewriteManifestListResult = validSnapshots.stream() - .map( - snapshot -> - rewriteManifestList( - snapshot, endMetadata, rewriteManifestResult.getRewrittenManifests())) + .map(snapshot -> rewriteManifestList(snapshot, endMetadata, manifestsToRewrite)) .reduce(new RewriteResult<>(), RewriteResult::append); + // rebuild manifest files + RewriteContentFileResult rewriteManifestResult = + rewriteManifests(deltaSnapshots, endMetadata, rewriteManifestListResult.toRewrite()); + // rebuild position delete files Set deleteFiles = - rewriteManifestResult.getContentFileResult().toRewrite().stream() + rewriteManifestResult.toRewrite().stream() .filter(e -> e instanceof DeleteFile) .map(e -> (DeleteFile) e) .collect(Collectors.toSet()); @@ -314,7 +307,8 @@ private String rebuildMetadata() { Set> copyPlan = Sets.newHashSet(); copyPlan.addAll(rewriteVersionResult.copyPlan()); copyPlan.addAll(rewriteManifestListResult.copyPlan()); - copyPlan.addAll(rewriteManifestResult.getContentFileResult().copyPlan()); + copyPlan.addAll(rewriteManifestResult.copyPlan()); + return saveFileList(copyPlan); } @@ -421,26 +415,27 @@ private Set> statsFileCopyPlan( * @param snapshot snapshot represented by the manifest list * @param tableMetadata metadata of table * @param manifestsToRewrite filter of manifests to rewrite. - * @return a result including a copy plan for the manifest list itself + * @return a result including a copy plan for the manifests contained in the manifest list, as + * well as for the manifest list itself */ private RewriteResult rewriteManifestList( - Snapshot snapshot, - TableMetadata tableMetadata, - Map rewrittenManifests) { + Snapshot snapshot, TableMetadata tableMetadata, Set manifestsToRewrite) { RewriteResult result = new RewriteResult<>(); String path = snapshot.manifestListLocation(); String outputPath = RewriteTablePathUtil.stagingPath(path, stagingDir); - RewriteTablePathUtil.rewriteManifestList( - snapshot, - table.io(), - tableMetadata, - rewrittenManifests, - sourcePrefix, - targetPrefix, - stagingDir, - outputPath); - + RewriteResult rewriteResult = + RewriteTablePathUtil.rewriteManifestList( + snapshot, + table.io(), + tableMetadata, + manifestsToRewrite, + sourcePrefix, + targetPrefix, + stagingDir, + outputPath); + + result.append(rewriteResult); // add the manifest list copy plan itself to the result result .copyPlan() @@ -448,58 +443,26 @@ private RewriteResult rewriteManifestList( return result; } - private Set manifestsToRewrite( + private Set manifestsToRewrite( Set deltaSnapshots, TableMetadata startMetadata) { try { Table endStaticTable = newStaticTable(endVersionName, table.io()); - Dataset allManifestsDF = manifestDS(endStaticTable).select("path"); - Set expectedManifestPaths; - + Dataset lastVersionFiles = manifestDS(endStaticTable).select("path"); if (startMetadata == null) { - expectedManifestPaths = - Sets.newHashSet(allManifestsDF.distinct().as(Encoders.STRING()).collectAsList()); + return Sets.newHashSet(lastVersionFiles.distinct().as(Encoders.STRING()).collectAsList()); } else { Set deltaSnapshotIds = deltaSnapshots.stream().map(Snapshot::snapshotId).collect(Collectors.toSet()); - expectedManifestPaths = - Sets.newHashSet( - allManifestsDF - .distinct() - .filter( - functions - .column(ManifestFile.SNAPSHOT_ID.name()) - .isInCollection(deltaSnapshotIds)) - .as(Encoders.STRING()) - .collectAsList()); + return Sets.newHashSet( + lastVersionFiles + .distinct() + .filter( + functions + .column(ManifestFile.SNAPSHOT_ID.name()) + .isInCollection(deltaSnapshotIds)) + .as(Encoders.STRING()) + .collectAsList()); } - - Set foundManifests = - deltaSnapshots.stream() - .flatMap( - s -> { - try { - return s.allManifests(table.io()).stream(); - } catch (NotFoundException e) { - LOG.warn( - "Skipping snapshot {} as its manifest list is missing (likely expired).", - s.snapshotId(), - e); - return Stream.empty(); - } - }) - .collect(Collectors.toSet()); - - Set foundManifestPaths = - foundManifests.stream().map(ManifestFile::path).collect(Collectors.toSet()); - Set missingPaths = Sets.difference(expectedManifestPaths, foundManifestPaths); - Preconditions.checkState( - missingPaths.isEmpty(), - "Could not find all expected manifests. Missing files: %s", - String.join(", ", missingPaths)); - - return foundManifests.stream() - .filter(m -> expectedManifestPaths.contains(m.path())) - .collect(Collectors.toSet()); } catch (Exception e) { throw new UnsupportedOperationException( "Unable to build the manifest files dataframe. The end version in use may contain invalid snapshots. " @@ -528,94 +491,48 @@ public RewriteContentFileResult appendDeleteFile(RewriteResult r1) { } } - public static class ManifestsRewriteResult { - private final RewriteContentFileResult contentFileResult; - private final Map rewrittenManifests; - - ManifestsRewriteResult( - RewriteContentFileResult contentFileResult, - Map rewrittenManifests) { - this.contentFileResult = contentFileResult; - this.rewrittenManifests = rewrittenManifests; - } - - public RewriteContentFileResult getContentFileResult() { - return contentFileResult; - } - - public Map getRewrittenManifests() { - return rewrittenManifests; - } - } - - /** Rewrite manifest files in a distributed manner and return the resulting manifests - * and content files selected for rewriting. - */ - private ManifestsRewriteResult rewriteManifests( + /** Rewrite manifest files in a distributed manner and return rewritten data files path pairs. */ + private RewriteContentFileResult rewriteManifests( Set deltaSnapshots, TableMetadata tableMetadata, Set toRewrite) { if (toRewrite.isEmpty()) { - return new ManifestsRewriteResult(new RewriteContentFileResult(), Maps.newHashMap()); + return new RewriteContentFileResult(); } Encoder manifestFileEncoder = Encoders.javaSerialization(ManifestFile.class); - Encoder contentResultEncoder = - Encoders.javaSerialization(RewriteContentFileResult.class); - Encoder> tupleEncoder = - Encoders.tuple(Encoders.STRING(), manifestFileEncoder, contentResultEncoder); - Dataset manifestDS = spark().createDataset(Lists.newArrayList(toRewrite), manifestFileEncoder); Set deltaSnapshotIds = deltaSnapshots.stream().map(Snapshot::snapshotId).collect(Collectors.toSet()); - RewriteContentFileResult finalContentResult = new RewriteContentFileResult(); - Iterator> resultIterator = - manifestDS - .repartition(toRewrite.size()) - .map( - toManifests( - tableBroadcast(), - sparkContext().broadcast(deltaSnapshotIds), - stagingDir, - tableMetadata.formatVersion(), - sourcePrefix, - targetPrefix), - tupleEncoder) - .toLocalIterator(); - - Map rewrittenManifests = Maps.newHashMap(); - - while (resultIterator.hasNext()) { - Tuple3 resultTuple = resultIterator.next(); - String originalManifestPath = resultTuple._1(); - ManifestFile rewrittenManifest = resultTuple._2(); - RewriteContentFileResult contentFileResult = resultTuple._3(); - String targetManifestPath = - RewriteTablePathUtil.newPath(originalManifestPath, sourcePrefix, targetPrefix); - - finalContentResult.append(contentFileResult); - finalContentResult.copyPlan().add(Pair.of(rewrittenManifest.path(), targetManifestPath)); - rewrittenManifests.put( - originalManifestPath, - new RewrittenFileInfo(targetManifestPath, rewrittenManifest.length())); - } - - return new ManifestsRewriteResult(finalContentResult, rewrittenManifests); - } - - private static MapFunction> - toManifests( - Broadcast
table, - Broadcast> deltaSnapshotIds, - String stagingLocation, - int format, - String sourcePrefix, - String targetPrefix) { + return manifestDS + .repartition(toRewrite.size()) + .map( + toManifests( + tableBroadcast(), + sparkContext().broadcast(deltaSnapshotIds), + stagingDir, + tableMetadata.formatVersion(), + sourcePrefix, + targetPrefix), + Encoders.bean(RewriteContentFileResult.class)) + // duplicates are expected here as the same data file can have different statuses + // (e.g. added and deleted) + .reduce((ReduceFunction) RewriteContentFileResult::append); + } + + private static MapFunction toManifests( + Broadcast
table, + Broadcast> deltaSnapshotIds, + String stagingLocation, + int format, + String sourcePrefix, + String targetPrefix) { return manifestFile -> { + RewriteContentFileResult result = new RewriteContentFileResult(); switch (manifestFile.content()) { case DATA: - Pair> dataFileResult = + result.appendDataFile( writeDataManifest( manifestFile, table, @@ -623,13 +540,10 @@ private ManifestsRewriteResult rewriteManifests( stagingLocation, format, sourcePrefix, - targetPrefix); - return Tuple3.apply( - manifestFile.path(), - dataFileResult.first(), - new RewriteContentFileResult().appendDataFile(dataFileResult.second())); + targetPrefix)); + break; case DELETES: - Pair> deleteFileResult = + result.appendDeleteFile( writeDeleteManifest( manifestFile, table, @@ -637,19 +551,17 @@ private ManifestsRewriteResult rewriteManifests( stagingLocation, format, sourcePrefix, - targetPrefix); - return Tuple3.apply( - manifestFile.path(), - deleteFileResult.first(), - new RewriteContentFileResult().appendDeleteFile(deleteFileResult.second())); + targetPrefix)); + break; default: throw new UnsupportedOperationException( "Unsupported manifest type: " + manifestFile.content()); } + return result; }; } - private static Pair> writeDataManifest( + private static RewriteResult writeDataManifest( ManifestFile manifestFile, Broadcast
table, Broadcast> snapshotIds, @@ -663,7 +575,7 @@ private static Pair> writeDataManifest( OutputFile outputFile = io.newOutputFile(stagingPath); Map specsById = table.getValue().specs(); Set deltaSnapshotIds = snapshotIds.value(); - return RewriteTablePathUtil.rewriteDataManifestWithResult( + return RewriteTablePathUtil.rewriteDataManifest( manifestFile, deltaSnapshotIds, outputFile, @@ -677,7 +589,7 @@ private static Pair> writeDataManifest( } } - private static Pair> writeDeleteManifest( + private static RewriteResult writeDeleteManifest( ManifestFile manifestFile, Broadcast
table, Broadcast> snapshotIds, @@ -691,7 +603,7 @@ private static Pair> writeDeleteManifest OutputFile outputFile = io.newOutputFile(stagingPath); Map specsById = table.getValue().specs(); Set deltaSnapshotIds = snapshotIds.value(); - return RewriteTablePathUtil.rewriteDeleteManifestWithResult( + return RewriteTablePathUtil.rewriteDeleteManifest( manifestFile, deltaSnapshotIds, outputFile, diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteTablePathsAction.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteTablePathsAction.java index c737d551821e..705e0074c79f 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteTablePathsAction.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteTablePathsAction.java @@ -39,8 +39,6 @@ import org.apache.iceberg.DeleteFile; import org.apache.iceberg.HasTableOperations; import org.apache.iceberg.ImmutableGenericPartitionStatisticsFile; -import org.apache.iceberg.ManifestContent; -import org.apache.iceberg.ManifestFile; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; import org.apache.iceberg.Snapshot; @@ -345,7 +343,7 @@ public void testManifestRewriteAndIncrementalCopy() throws Exception { int addedManifest = Iterables.size(rewriteManifestResult.addedManifests()); // only move version v4, which is the version generated by rewrite manifest - RewriteTablePath.Result postRewriteResult = + RewriteTablePath.Result postReweiteResult = actions() .rewriteTablePath(table) .rewriteLocationPrefix(table.location(), targetTableLocation()) @@ -354,7 +352,7 @@ public void testManifestRewriteAndIncrementalCopy() throws Exception { .execute(); // no data files need to move - checkFileNum(1, 1, addedManifest, 3, postRewriteResult); + checkFileNum(1, 1, addedManifest, 3, postReweiteResult); } @Test @@ -1124,184 +1122,6 @@ public void testKryoDeserializeBroadcastValues() { assertThat(tableBroadcast.getValue().uuid()).isEqualTo(table.uuid()); } - @Test - public void testFullRewriteUpdatesAllManifestLengthsInManifestList( - @TempDir Path rootTargetLocation) throws Exception { - String location = newTableLocation(); - Table sourceTable = createTableWithSnapshots(location, 10); - - addAnyPositionDelete( - sourceTable, removePrefix(sourceTable.location() + "/data/deeply/nested/deletes.parquet")); - - Map manifestSizesBeforeRewrite = - sourceTable.currentSnapshot().allManifests(sourceTable.io()).stream() - .collect(Collectors.toMap(m -> fileName(m.path()), m -> m.length())); - - // Rewrite table metadata to a location that's much longer than the original in order - // to make manifests larger - String targetLocation = toAbsolute(rootTargetLocation) + generateLongNestedPath(25); - RewriteTablePath.Result result = - actions() - .rewriteTablePath(sourceTable) - .rewriteLocationPrefix(newTableLocation(), targetLocation) - .execute(); - - // 1 + 11 JSON metadata files, 11 snapshots, 11 manifests, 10 data files, 1 delete file - checkFileNum(12, 11, 11, 45, result); - copyTableFiles(result); - - Table targetTable = TABLES.load(targetLocation); - - // We have rewritten all manifests. Make sure all sizes were correctly updated in manifest lists - assertThat(targetTable.currentSnapshot().allManifests(targetTable.io())) - .allSatisfy( - manifest -> { - String manifestName = fileName(manifest.path()); - assertThat(manifest.length()) - .isNotEqualTo(manifestSizesBeforeRewrite.get(manifestName)); - }) - .allSatisfy( - manifest -> { - assertThat(targetTable.io().newInputFile(manifest.path()).getLength()) - .isEqualTo(manifest.length()); - }); - } - - @Test - public void testKeepsExistingDataManifestLengthInManifestList(@TempDir Path rootTargetLocation) - throws Exception { - String location = newTableLocation(); - Table sourceTable = createTableWithSnapshots(location, 10); - - Map manifestSizesBeforeRewrite = - sourceTable.currentSnapshot().allManifests(sourceTable.io()).stream() - .collect(Collectors.toMap(m -> fileName(m.path()), m -> m.length())); - - // Rewrite just the latest table version to a location that's much longer than - // the original in order to make manifests larger - String targetLocation = toAbsolute(rootTargetLocation) + generateLongNestedPath(25); - RewriteTablePath.Result result = - actions() - .rewriteTablePath(sourceTable) - .rewriteLocationPrefix(newTableLocation(), targetLocation) - .startVersion("v10.metadata.json") - .execute(); - - // 1 metadata JSON file, 1 snapshot, 1 manifest, 4 data files - checkFileNum(1, 1, 1, 4, result); - copyTableFiles(result); - - Table targetTable = TABLES.load(targetLocation); - Snapshot lastSnapshot = targetTable.currentSnapshot(); - - List allManifests = lastSnapshot.allManifests(targetTable.io()); - ManifestFile rewrittenManifest = - Iterables.getOnlyElement( - allManifests.stream() - .filter(manifest -> manifest.snapshotId() == lastSnapshot.snapshotId()) - .collect(Collectors.toList())); - - // We have rewritten exactly one (data) manifest. Check that its size was correctly - // updated in the manifest list and sizes of all other manifests were left unchanged - assertThat(allManifests) - .filteredOn(manifest -> manifest != rewrittenManifest) - .allSatisfy( - manifest -> { - String manifestName = fileName(manifest.path()); - assertThat(manifest.length()).isEqualTo(manifestSizesBeforeRewrite.get(manifestName)); - }); - assertThat(rewrittenManifest.content()).isEqualTo(ManifestContent.DATA); - assertThat(rewrittenManifest.length()) - .isNotEqualTo(manifestSizesBeforeRewrite.get(fileName(rewrittenManifest.path()))) - .isEqualTo(targetTable.io().newInputFile(rewrittenManifest.path()).getLength()); - } - - @Test - public void testKeepsExistingDeleteManifestLengthInManifestList(@TempDir Path rootTargetLocation) - throws Exception { - String location = newTableLocation(); - Table sourceTable = createTableWithSnapshots(location, 5); - - // Add two more snapshots with just position deletes - addAnyPositionDelete( - sourceTable, - removePrefix(sourceTable.location() + "/data/deeply/nested/deletes-1.parquet")); - addAnyPositionDelete( - sourceTable, - removePrefix(sourceTable.location() + "/data/deeply/nested/deletes-2.parquet")); - - Map manifestSizesBeforeRewrite = - sourceTable.currentSnapshot().allManifests(sourceTable.io()).stream() - .collect(Collectors.toMap(m -> fileName(m.path()), m -> m.length())); - - // Rewrite just the latest table version to a location that's much longer than - // the original in order to make manifests larger - String targetLocation = toAbsolute(rootTargetLocation) + generateLongNestedPath(25); - RewriteTablePath.Result result = - actions() - .rewriteTablePath(sourceTable) - .rewriteLocationPrefix(newTableLocation(), targetLocation) - .startVersion("v7.metadata.json") - .execute(); - - // 1 metadata JSON file, 1 snapshot, 1 manifest, 1 delete file - checkFileNum(1, 1, 1, 4, result); - copyTableFiles(result); - - Table targetTable = TABLES.load(targetLocation); - Snapshot lastSnapshot = targetTable.currentSnapshot(); - - List allManifests = lastSnapshot.allManifests(targetTable.io()); - ManifestFile rewrittenManifest = - Iterables.getOnlyElement( - allManifests.stream() - .filter(manifest -> manifest.snapshotId() == lastSnapshot.snapshotId()) - .collect(Collectors.toList())); - - // We have rewritten exactly one (delete) manifest. Check that its size was correctly - // updated in the manifest list and sizes of all other manifests were left unchanged - assertThat(allManifests) - .filteredOn(manifest -> manifest != rewrittenManifest) - .allSatisfy( - manifest -> { - String manifestName = fileName(manifest.path()); - assertThat(manifest.length()).isEqualTo(manifestSizesBeforeRewrite.get(manifestName)); - }); - assertThat(rewrittenManifest.content()).isEqualTo(ManifestContent.DELETES); - assertThat(rewrittenManifest.length()) - .isNotEqualTo(manifestSizesBeforeRewrite.get(fileName(rewrittenManifest.path()))) - .isEqualTo(targetTable.io().newInputFile(rewrittenManifest.path()).getLength()); - } - - protected static String generateLongNestedPath(int depth) { - StringBuilder pathBuilder = new StringBuilder(); - for (int i = 1; i <= depth; i++) { - pathBuilder.append(String.format("/%03d", i)); - } - pathBuilder.append("/"); - return pathBuilder.toString(); - } - - protected void addAnyPositionDelete(Table table, String path) throws Exception { - DataFile dataFile = - StreamSupport.stream(table.snapshots().spliterator(), false) - .flatMap( - snapshot -> - StreamSupport.stream(snapshot.addedDataFiles(table.io()).spliterator(), false)) - .findAny() - .get(); - - List> deletes = Lists.newArrayList(Pair.of(dataFile.location(), 0L)); - File file = new File(path); - - DeleteFile positionDeletes = - FileHelpers.writeDeleteFile( - table, table.io().newOutputFile(file.toURI().toString()), deletes) - .first(); - - table.newRowDelta().addDeletes(positionDeletes).commit(); - } - protected void checkFileNum( int versionFileCount, int manifestListCount, diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteTablePathSparkAction.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteTablePathSparkAction.java index 21abf32cf0c9..fbeabfe63e0e 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteTablePathSparkAction.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteTablePathSparkAction.java @@ -22,13 +22,11 @@ import java.io.IOException; import java.io.OutputStreamWriter; import java.nio.charset.StandardCharsets; -import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Set; import java.util.UUID; import java.util.stream.Collectors; -import java.util.stream.Stream; import org.apache.iceberg.ContentFile; import org.apache.iceberg.DataFile; import org.apache.iceberg.DeleteFile; @@ -39,7 +37,6 @@ import org.apache.iceberg.RewriteTablePathUtil; import org.apache.iceberg.RewriteTablePathUtil.PositionDeleteReaderWriter; import org.apache.iceberg.RewriteTablePathUtil.RewriteResult; -import org.apache.iceberg.RewriteTablePathUtil.RewrittenFileInfo; import org.apache.iceberg.Schema; import org.apache.iceberg.Snapshot; import org.apache.iceberg.StaticTableOperations; @@ -60,7 +57,6 @@ import org.apache.iceberg.data.parquet.GenericParquetReaders; import org.apache.iceberg.data.parquet.GenericParquetWriter; import org.apache.iceberg.deletes.PositionDeleteWriter; -import org.apache.iceberg.exceptions.NotFoundException; import org.apache.iceberg.exceptions.RuntimeIOException; import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.io.DeleteSchemaUtil; @@ -72,13 +68,13 @@ 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.Lists; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.spark.JobGroupInfo; import org.apache.iceberg.spark.source.SerializableTableWithSize; import org.apache.iceberg.util.Pair; import org.apache.spark.api.java.function.ForeachFunction; import org.apache.spark.api.java.function.MapFunction; +import org.apache.spark.api.java.function.ReduceFunction; import org.apache.spark.broadcast.Broadcast; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Encoder; @@ -88,7 +84,6 @@ import org.apache.spark.sql.functions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import scala.Tuple3; public class RewriteTablePathSparkAction extends BaseSparkAction implements RewriteTablePath { @@ -287,25 +282,23 @@ private String rebuildMetadata() { RewriteResult rewriteVersionResult = rewriteVersionFiles(endMetadata); Set deltaSnapshots = deltaSnapshots(startMetadata, rewriteVersionResult.toRewrite()); - // rebuild manifest files - Set manifestsToRewrite = manifestsToRewrite(deltaSnapshots, startMetadata); - ManifestsRewriteResult rewriteManifestResult = - rewriteManifests(deltaSnapshots, endMetadata, manifestsToRewrite); - - // rebuild manifest-list files + Set manifestsToRewrite = manifestsToRewrite(deltaSnapshots, startMetadata); Set validSnapshots = Sets.difference(snapshotSet(endMetadata), snapshotSet(startMetadata)); + + // rebuild manifest-list files RewriteResult rewriteManifestListResult = validSnapshots.stream() - .map( - snapshot -> - rewriteManifestList( - snapshot, endMetadata, rewriteManifestResult.getRewrittenManifests())) + .map(snapshot -> rewriteManifestList(snapshot, endMetadata, manifestsToRewrite)) .reduce(new RewriteResult<>(), RewriteResult::append); + // rebuild manifest files + RewriteContentFileResult rewriteManifestResult = + rewriteManifests(deltaSnapshots, endMetadata, rewriteManifestListResult.toRewrite()); + // rebuild position delete files Set deleteFiles = - rewriteManifestResult.getContentFileResult().toRewrite().stream() + rewriteManifestResult.toRewrite().stream() .filter(e -> e instanceof DeleteFile) .map(e -> (DeleteFile) e) .collect(Collectors.toSet()); @@ -314,7 +307,8 @@ private String rebuildMetadata() { Set> copyPlan = Sets.newHashSet(); copyPlan.addAll(rewriteVersionResult.copyPlan()); copyPlan.addAll(rewriteManifestListResult.copyPlan()); - copyPlan.addAll(rewriteManifestResult.getContentFileResult().copyPlan()); + copyPlan.addAll(rewriteManifestResult.copyPlan()); + return saveFileList(copyPlan); } @@ -421,26 +415,27 @@ private Set> statsFileCopyPlan( * @param snapshot snapshot represented by the manifest list * @param tableMetadata metadata of table * @param manifestsToRewrite filter of manifests to rewrite. - * @return a result including a copy plan for the manifest list itself + * @return a result including a copy plan for the manifests contained in the manifest list, as + * well as for the manifest list itself */ private RewriteResult rewriteManifestList( - Snapshot snapshot, - TableMetadata tableMetadata, - Map rewrittenManifests) { + Snapshot snapshot, TableMetadata tableMetadata, Set manifestsToRewrite) { RewriteResult result = new RewriteResult<>(); String path = snapshot.manifestListLocation(); String outputPath = RewriteTablePathUtil.stagingPath(path, stagingDir); - RewriteTablePathUtil.rewriteManifestList( - snapshot, - table.io(), - tableMetadata, - rewrittenManifests, - sourcePrefix, - targetPrefix, - stagingDir, - outputPath); - + RewriteResult rewriteResult = + RewriteTablePathUtil.rewriteManifestList( + snapshot, + table.io(), + tableMetadata, + manifestsToRewrite, + sourcePrefix, + targetPrefix, + stagingDir, + outputPath); + + result.append(rewriteResult); // add the manifest list copy plan itself to the result result .copyPlan() @@ -448,58 +443,26 @@ private RewriteResult rewriteManifestList( return result; } - private Set manifestsToRewrite( + private Set manifestsToRewrite( Set deltaSnapshots, TableMetadata startMetadata) { try { Table endStaticTable = newStaticTable(endVersionName, table.io()); - Dataset allManifestsDF = manifestDS(endStaticTable).select("path"); - Set expectedManifestPaths; - + Dataset lastVersionFiles = manifestDS(endStaticTable).select("path"); if (startMetadata == null) { - expectedManifestPaths = - Sets.newHashSet(allManifestsDF.distinct().as(Encoders.STRING()).collectAsList()); + return Sets.newHashSet(lastVersionFiles.distinct().as(Encoders.STRING()).collectAsList()); } else { Set deltaSnapshotIds = deltaSnapshots.stream().map(Snapshot::snapshotId).collect(Collectors.toSet()); - expectedManifestPaths = - Sets.newHashSet( - allManifestsDF - .distinct() - .filter( - functions - .column(ManifestFile.SNAPSHOT_ID.name()) - .isInCollection(deltaSnapshotIds)) - .as(Encoders.STRING()) - .collectAsList()); + return Sets.newHashSet( + lastVersionFiles + .distinct() + .filter( + functions + .column(ManifestFile.SNAPSHOT_ID.name()) + .isInCollection(deltaSnapshotIds)) + .as(Encoders.STRING()) + .collectAsList()); } - - Set foundManifests = - deltaSnapshots.stream() - .flatMap( - s -> { - try { - return s.allManifests(table.io()).stream(); - } catch (NotFoundException e) { - LOG.warn( - "Skipping snapshot {} as its manifest list is missing (likely expired).", - s.snapshotId(), - e); - return Stream.empty(); - } - }) - .collect(Collectors.toSet()); - - Set foundManifestPaths = - foundManifests.stream().map(ManifestFile::path).collect(Collectors.toSet()); - Set missingPaths = Sets.difference(expectedManifestPaths, foundManifestPaths); - Preconditions.checkState( - missingPaths.isEmpty(), - "Could not find all expected manifests. Missing files: %s", - String.join(", ", missingPaths)); - - return foundManifests.stream() - .filter(m -> expectedManifestPaths.contains(m.path())) - .collect(Collectors.toSet()); } catch (Exception e) { throw new UnsupportedOperationException( "Unable to build the manifest files dataframe. The end version in use may contain invalid snapshots. " @@ -528,94 +491,48 @@ public RewriteContentFileResult appendDeleteFile(RewriteResult r1) { } } - public static class ManifestsRewriteResult { - private final RewriteContentFileResult contentFileResult; - private final Map rewrittenManifests; - - ManifestsRewriteResult( - RewriteContentFileResult contentFileResult, - Map rewrittenManifests) { - this.contentFileResult = contentFileResult; - this.rewrittenManifests = rewrittenManifests; - } - - public RewriteContentFileResult getContentFileResult() { - return contentFileResult; - } - - public Map getRewrittenManifests() { - return rewrittenManifests; - } - } - - /** Rewrite manifest files in a distributed manner and return the resulting manifests - * and content files selected for rewriting. - */ - private ManifestsRewriteResult rewriteManifests( + /** Rewrite manifest files in a distributed manner and return rewritten data files path pairs. */ + private RewriteContentFileResult rewriteManifests( Set deltaSnapshots, TableMetadata tableMetadata, Set toRewrite) { if (toRewrite.isEmpty()) { - return new ManifestsRewriteResult(new RewriteContentFileResult(), Maps.newHashMap()); + return new RewriteContentFileResult(); } Encoder manifestFileEncoder = Encoders.javaSerialization(ManifestFile.class); - Encoder contentResultEncoder = - Encoders.javaSerialization(RewriteContentFileResult.class); - Encoder> tupleEncoder = - Encoders.tuple(Encoders.STRING(), manifestFileEncoder, contentResultEncoder); - Dataset manifestDS = spark().createDataset(Lists.newArrayList(toRewrite), manifestFileEncoder); Set deltaSnapshotIds = deltaSnapshots.stream().map(Snapshot::snapshotId).collect(Collectors.toSet()); - RewriteContentFileResult finalContentResult = new RewriteContentFileResult(); - Iterator> resultIterator = - manifestDS - .repartition(toRewrite.size()) - .map( - toManifests( - tableBroadcast(), - sparkContext().broadcast(deltaSnapshotIds), - stagingDir, - tableMetadata.formatVersion(), - sourcePrefix, - targetPrefix), - tupleEncoder) - .toLocalIterator(); - - Map rewrittenManifests = Maps.newHashMap(); - - while (resultIterator.hasNext()) { - Tuple3 resultTuple = resultIterator.next(); - String originalManifestPath = resultTuple._1(); - ManifestFile rewrittenManifest = resultTuple._2(); - RewriteContentFileResult contentFileResult = resultTuple._3(); - String targetManifestPath = - RewriteTablePathUtil.newPath(originalManifestPath, sourcePrefix, targetPrefix); - - finalContentResult.append(contentFileResult); - finalContentResult.copyPlan().add(Pair.of(rewrittenManifest.path(), targetManifestPath)); - rewrittenManifests.put( - originalManifestPath, - new RewrittenFileInfo(targetManifestPath, rewrittenManifest.length())); - } - - return new ManifestsRewriteResult(finalContentResult, rewrittenManifests); - } - - private static MapFunction> - toManifests( - Broadcast
table, - Broadcast> deltaSnapshotIds, - String stagingLocation, - int format, - String sourcePrefix, - String targetPrefix) { + return manifestDS + .repartition(toRewrite.size()) + .map( + toManifests( + tableBroadcast(), + sparkContext().broadcast(deltaSnapshotIds), + stagingDir, + tableMetadata.formatVersion(), + sourcePrefix, + targetPrefix), + Encoders.bean(RewriteContentFileResult.class)) + // duplicates are expected here as the same data file can have different statuses + // (e.g. added and deleted) + .reduce((ReduceFunction) RewriteContentFileResult::append); + } + + private static MapFunction toManifests( + Broadcast
table, + Broadcast> deltaSnapshotIds, + String stagingLocation, + int format, + String sourcePrefix, + String targetPrefix) { return manifestFile -> { + RewriteContentFileResult result = new RewriteContentFileResult(); switch (manifestFile.content()) { case DATA: - Pair> dataFileResult = + result.appendDataFile( writeDataManifest( manifestFile, table, @@ -623,13 +540,10 @@ private ManifestsRewriteResult rewriteManifests( stagingLocation, format, sourcePrefix, - targetPrefix); - return Tuple3.apply( - manifestFile.path(), - dataFileResult.first(), - new RewriteContentFileResult().appendDataFile(dataFileResult.second())); + targetPrefix)); + break; case DELETES: - Pair> deleteFileResult = + result.appendDeleteFile( writeDeleteManifest( manifestFile, table, @@ -637,19 +551,17 @@ private ManifestsRewriteResult rewriteManifests( stagingLocation, format, sourcePrefix, - targetPrefix); - return Tuple3.apply( - manifestFile.path(), - deleteFileResult.first(), - new RewriteContentFileResult().appendDeleteFile(deleteFileResult.second())); + targetPrefix)); + break; default: throw new UnsupportedOperationException( "Unsupported manifest type: " + manifestFile.content()); } + return result; }; } - private static Pair> writeDataManifest( + private static RewriteResult writeDataManifest( ManifestFile manifestFile, Broadcast
table, Broadcast> snapshotIds, @@ -663,7 +575,7 @@ private static Pair> writeDataManifest( OutputFile outputFile = io.newOutputFile(stagingPath); Map specsById = table.getValue().specs(); Set deltaSnapshotIds = snapshotIds.value(); - return RewriteTablePathUtil.rewriteDataManifestWithResult( + return RewriteTablePathUtil.rewriteDataManifest( manifestFile, deltaSnapshotIds, outputFile, @@ -677,7 +589,7 @@ private static Pair> writeDataManifest( } } - private static Pair> writeDeleteManifest( + private static RewriteResult writeDeleteManifest( ManifestFile manifestFile, Broadcast
table, Broadcast> snapshotIds, @@ -691,7 +603,7 @@ private static Pair> writeDeleteManifest OutputFile outputFile = io.newOutputFile(stagingPath); Map specsById = table.getValue().specs(); Set deltaSnapshotIds = snapshotIds.value(); - return RewriteTablePathUtil.rewriteDeleteManifestWithResult( + return RewriteTablePathUtil.rewriteDeleteManifest( manifestFile, deltaSnapshotIds, outputFile, diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteTablePathsAction.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteTablePathsAction.java index c737d551821e..705e0074c79f 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteTablePathsAction.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteTablePathsAction.java @@ -39,8 +39,6 @@ import org.apache.iceberg.DeleteFile; import org.apache.iceberg.HasTableOperations; import org.apache.iceberg.ImmutableGenericPartitionStatisticsFile; -import org.apache.iceberg.ManifestContent; -import org.apache.iceberg.ManifestFile; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; import org.apache.iceberg.Snapshot; @@ -345,7 +343,7 @@ public void testManifestRewriteAndIncrementalCopy() throws Exception { int addedManifest = Iterables.size(rewriteManifestResult.addedManifests()); // only move version v4, which is the version generated by rewrite manifest - RewriteTablePath.Result postRewriteResult = + RewriteTablePath.Result postReweiteResult = actions() .rewriteTablePath(table) .rewriteLocationPrefix(table.location(), targetTableLocation()) @@ -354,7 +352,7 @@ public void testManifestRewriteAndIncrementalCopy() throws Exception { .execute(); // no data files need to move - checkFileNum(1, 1, addedManifest, 3, postRewriteResult); + checkFileNum(1, 1, addedManifest, 3, postReweiteResult); } @Test @@ -1124,184 +1122,6 @@ public void testKryoDeserializeBroadcastValues() { assertThat(tableBroadcast.getValue().uuid()).isEqualTo(table.uuid()); } - @Test - public void testFullRewriteUpdatesAllManifestLengthsInManifestList( - @TempDir Path rootTargetLocation) throws Exception { - String location = newTableLocation(); - Table sourceTable = createTableWithSnapshots(location, 10); - - addAnyPositionDelete( - sourceTable, removePrefix(sourceTable.location() + "/data/deeply/nested/deletes.parquet")); - - Map manifestSizesBeforeRewrite = - sourceTable.currentSnapshot().allManifests(sourceTable.io()).stream() - .collect(Collectors.toMap(m -> fileName(m.path()), m -> m.length())); - - // Rewrite table metadata to a location that's much longer than the original in order - // to make manifests larger - String targetLocation = toAbsolute(rootTargetLocation) + generateLongNestedPath(25); - RewriteTablePath.Result result = - actions() - .rewriteTablePath(sourceTable) - .rewriteLocationPrefix(newTableLocation(), targetLocation) - .execute(); - - // 1 + 11 JSON metadata files, 11 snapshots, 11 manifests, 10 data files, 1 delete file - checkFileNum(12, 11, 11, 45, result); - copyTableFiles(result); - - Table targetTable = TABLES.load(targetLocation); - - // We have rewritten all manifests. Make sure all sizes were correctly updated in manifest lists - assertThat(targetTable.currentSnapshot().allManifests(targetTable.io())) - .allSatisfy( - manifest -> { - String manifestName = fileName(manifest.path()); - assertThat(manifest.length()) - .isNotEqualTo(manifestSizesBeforeRewrite.get(manifestName)); - }) - .allSatisfy( - manifest -> { - assertThat(targetTable.io().newInputFile(manifest.path()).getLength()) - .isEqualTo(manifest.length()); - }); - } - - @Test - public void testKeepsExistingDataManifestLengthInManifestList(@TempDir Path rootTargetLocation) - throws Exception { - String location = newTableLocation(); - Table sourceTable = createTableWithSnapshots(location, 10); - - Map manifestSizesBeforeRewrite = - sourceTable.currentSnapshot().allManifests(sourceTable.io()).stream() - .collect(Collectors.toMap(m -> fileName(m.path()), m -> m.length())); - - // Rewrite just the latest table version to a location that's much longer than - // the original in order to make manifests larger - String targetLocation = toAbsolute(rootTargetLocation) + generateLongNestedPath(25); - RewriteTablePath.Result result = - actions() - .rewriteTablePath(sourceTable) - .rewriteLocationPrefix(newTableLocation(), targetLocation) - .startVersion("v10.metadata.json") - .execute(); - - // 1 metadata JSON file, 1 snapshot, 1 manifest, 4 data files - checkFileNum(1, 1, 1, 4, result); - copyTableFiles(result); - - Table targetTable = TABLES.load(targetLocation); - Snapshot lastSnapshot = targetTable.currentSnapshot(); - - List allManifests = lastSnapshot.allManifests(targetTable.io()); - ManifestFile rewrittenManifest = - Iterables.getOnlyElement( - allManifests.stream() - .filter(manifest -> manifest.snapshotId() == lastSnapshot.snapshotId()) - .collect(Collectors.toList())); - - // We have rewritten exactly one (data) manifest. Check that its size was correctly - // updated in the manifest list and sizes of all other manifests were left unchanged - assertThat(allManifests) - .filteredOn(manifest -> manifest != rewrittenManifest) - .allSatisfy( - manifest -> { - String manifestName = fileName(manifest.path()); - assertThat(manifest.length()).isEqualTo(manifestSizesBeforeRewrite.get(manifestName)); - }); - assertThat(rewrittenManifest.content()).isEqualTo(ManifestContent.DATA); - assertThat(rewrittenManifest.length()) - .isNotEqualTo(manifestSizesBeforeRewrite.get(fileName(rewrittenManifest.path()))) - .isEqualTo(targetTable.io().newInputFile(rewrittenManifest.path()).getLength()); - } - - @Test - public void testKeepsExistingDeleteManifestLengthInManifestList(@TempDir Path rootTargetLocation) - throws Exception { - String location = newTableLocation(); - Table sourceTable = createTableWithSnapshots(location, 5); - - // Add two more snapshots with just position deletes - addAnyPositionDelete( - sourceTable, - removePrefix(sourceTable.location() + "/data/deeply/nested/deletes-1.parquet")); - addAnyPositionDelete( - sourceTable, - removePrefix(sourceTable.location() + "/data/deeply/nested/deletes-2.parquet")); - - Map manifestSizesBeforeRewrite = - sourceTable.currentSnapshot().allManifests(sourceTable.io()).stream() - .collect(Collectors.toMap(m -> fileName(m.path()), m -> m.length())); - - // Rewrite just the latest table version to a location that's much longer than - // the original in order to make manifests larger - String targetLocation = toAbsolute(rootTargetLocation) + generateLongNestedPath(25); - RewriteTablePath.Result result = - actions() - .rewriteTablePath(sourceTable) - .rewriteLocationPrefix(newTableLocation(), targetLocation) - .startVersion("v7.metadata.json") - .execute(); - - // 1 metadata JSON file, 1 snapshot, 1 manifest, 1 delete file - checkFileNum(1, 1, 1, 4, result); - copyTableFiles(result); - - Table targetTable = TABLES.load(targetLocation); - Snapshot lastSnapshot = targetTable.currentSnapshot(); - - List allManifests = lastSnapshot.allManifests(targetTable.io()); - ManifestFile rewrittenManifest = - Iterables.getOnlyElement( - allManifests.stream() - .filter(manifest -> manifest.snapshotId() == lastSnapshot.snapshotId()) - .collect(Collectors.toList())); - - // We have rewritten exactly one (delete) manifest. Check that its size was correctly - // updated in the manifest list and sizes of all other manifests were left unchanged - assertThat(allManifests) - .filteredOn(manifest -> manifest != rewrittenManifest) - .allSatisfy( - manifest -> { - String manifestName = fileName(manifest.path()); - assertThat(manifest.length()).isEqualTo(manifestSizesBeforeRewrite.get(manifestName)); - }); - assertThat(rewrittenManifest.content()).isEqualTo(ManifestContent.DELETES); - assertThat(rewrittenManifest.length()) - .isNotEqualTo(manifestSizesBeforeRewrite.get(fileName(rewrittenManifest.path()))) - .isEqualTo(targetTable.io().newInputFile(rewrittenManifest.path()).getLength()); - } - - protected static String generateLongNestedPath(int depth) { - StringBuilder pathBuilder = new StringBuilder(); - for (int i = 1; i <= depth; i++) { - pathBuilder.append(String.format("/%03d", i)); - } - pathBuilder.append("/"); - return pathBuilder.toString(); - } - - protected void addAnyPositionDelete(Table table, String path) throws Exception { - DataFile dataFile = - StreamSupport.stream(table.snapshots().spliterator(), false) - .flatMap( - snapshot -> - StreamSupport.stream(snapshot.addedDataFiles(table.io()).spliterator(), false)) - .findAny() - .get(); - - List> deletes = Lists.newArrayList(Pair.of(dataFile.location(), 0L)); - File file = new File(path); - - DeleteFile positionDeletes = - FileHelpers.writeDeleteFile( - table, table.io().newOutputFile(file.toURI().toString()), deletes) - .first(); - - table.newRowDelta().addDeletes(positionDeletes).commit(); - } - protected void checkFileNum( int versionFileCount, int manifestListCount, diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteTablePathSparkAction.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteTablePathSparkAction.java index 21abf32cf0c9..eedeecbf77ab 100644 --- a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteTablePathSparkAction.java +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteTablePathSparkAction.java @@ -548,8 +548,9 @@ public Map getRewrittenManifests() { } } - /** Rewrite manifest files in a distributed manner and return the resulting manifests - * and content files selected for rewriting. + /** + * Rewrite manifest files in a distributed manner and return the resulting manifests and content + * files selected for rewriting. */ private ManifestsRewriteResult rewriteManifests( Set deltaSnapshots, TableMetadata tableMetadata, Set toRewrite) { From a2ec9c41b7c5f93a8034049688f1a7bacf35fc00 Mon Sep 17 00:00:00 2001 From: vaultah <4944562+vaultah@users.noreply.github.com> Date: Tue, 5 Aug 2025 10:45:35 +0100 Subject: [PATCH 03/18] Formatting --- .../apache/iceberg/RewriteTablePathUtil.java | 58 +++++++++---------- 1 file changed, 26 insertions(+), 32 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/RewriteTablePathUtil.java b/core/src/main/java/org/apache/iceberg/RewriteTablePathUtil.java index e5876f8a18aa..292ae236df58 100644 --- a/core/src/main/java/org/apache/iceberg/RewriteTablePathUtil.java +++ b/core/src/main/java/org/apache/iceberg/RewriteTablePathUtil.java @@ -296,6 +296,7 @@ public static RewriteResult rewriteManifestList( /** * Rewrite a manifest list representing a snapshot, replacing path references. + * * @param snapshot snapshot represented by the manifest list * @param io file io * @param tableMetadata metadata of table @@ -438,7 +439,6 @@ public static RewriteResult rewriteDataManifest( } } - /** * Rewrite a data manifest, replacing path references. * @@ -464,25 +464,19 @@ public static Pair> rewriteDataManifestWit throws IOException { PartitionSpec spec = specsById.get(manifestFile.partitionSpecId()); ManifestWriter writer = - ManifestFiles.write(format, spec, outputFile, manifestFile.snapshotId()); + ManifestFiles.write(format, spec, outputFile, manifestFile.snapshotId()); RewriteResult rewriteResult = null; try (ManifestWriter dataManifestWriter = writer; - ManifestReader reader = - ManifestFiles.read(manifestFile, io, specsById) - .select(Arrays.asList("*"))) { - rewriteResult = + ManifestReader reader = + ManifestFiles.read(manifestFile, io, specsById).select(Arrays.asList("*"))) { + rewriteResult = StreamSupport.stream(reader.entries().spliterator(), false) - .map( - entry -> - writeDataFileEntry( - entry, - snapshotIds, - spec, - sourcePrefix, - targetPrefix, - writer)) - .reduce(new RewriteResult<>(), RewriteResult::append); + .map( + entry -> + writeDataFileEntry( + entry, snapshotIds, spec, sourcePrefix, targetPrefix, writer)) + .reduce(new RewriteResult<>(), RewriteResult::append); } return Pair.of(writer.toManifestFile(), rewriteResult); } @@ -541,8 +535,8 @@ public static RewriteResult rewriteDeleteManifest( * @param targetPrefix target prefix that will replace it * @param stagingLocation staging location for rewritten files (referred delete file will be * rewritten here) - * @return rewritten manifest file and a copy plan of content files - * in the manifest that was rewritten + * @return rewritten manifest file and a copy plan of content files in the manifest that was + * rewritten * @deprecated since 1.10.0, will be removed in 1.11.0 */ @Deprecated @@ -606,26 +600,26 @@ public static Pair> rewriteDeleteManifes throws IOException { PartitionSpec spec = specsById.get(manifestFile.partitionSpecId()); ManifestWriter writer = - ManifestFiles.writeDeleteManifest(format, spec, outputFile, manifestFile.snapshotId()); + ManifestFiles.writeDeleteManifest(format, spec, outputFile, manifestFile.snapshotId()); RewriteResult rewriteResult = null; try (ManifestWriter deleteManifestWriter = writer; - ManifestReader reader = + ManifestReader reader = ManifestFiles.readDeleteManifest(manifestFile, io, specsById) .select(Arrays.asList("*"))) { - rewriteResult = + rewriteResult = StreamSupport.stream(reader.entries().spliterator(), false) - .map( - entry -> - writeDeleteFileEntry( - entry, - snapshotIds, - spec, - sourcePrefix, - targetPrefix, - stagingLocation, - writer)) - .reduce(new RewriteResult<>(), RewriteResult::append); + .map( + entry -> + writeDeleteFileEntry( + entry, + snapshotIds, + spec, + sourcePrefix, + targetPrefix, + stagingLocation, + writer)) + .reduce(new RewriteResult<>(), RewriteResult::append); } return Pair.of(writer.toManifestFile(), rewriteResult); From 5cb4fc0ad8c4cd7869e85d348bad6cba0dca9218 Mon Sep 17 00:00:00 2001 From: vaultah <4944562+vaultah@users.noreply.github.com> Date: Tue, 5 Aug 2025 12:40:23 +0100 Subject: [PATCH 04/18] Improved manifest discovery --- .../apache/iceberg/RewriteTablePathUtil.java | 26 ++--- .../spark/actions/BaseSparkAction.java | 34 +++---- .../actions/RewriteTablePathSparkAction.java | 74 ++------------ .../actions/TestRewriteTablePathsAction.java | 96 ++++++++++--------- 4 files changed, 87 insertions(+), 143 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/RewriteTablePathUtil.java b/core/src/main/java/org/apache/iceberg/RewriteTablePathUtil.java index 292ae236df58..8247a154e38f 100644 --- a/core/src/main/java/org/apache/iceberg/RewriteTablePathUtil.java +++ b/core/src/main/java/org/apache/iceberg/RewriteTablePathUtil.java @@ -301,9 +301,6 @@ public static RewriteResult rewriteManifestList( * @param io file io * @param tableMetadata metadata of table * @param rewrittenManifests information about rewritten manifest files - * @param sourcePrefix source prefix that will be replaced - * @param targetPrefix target prefix that will replace it - * @param stagingDir staging directory * @param outputPath location to write the manifest list */ public static void rewriteManifestList( @@ -311,9 +308,6 @@ public static void rewriteManifestList( FileIO io, TableMetadata tableMetadata, Map rewrittenManifests, - String sourcePrefix, - String targetPrefix, - String stagingDir, String outputPath) { OutputFile outputFile = io.newOutputFile(outputPath); @@ -321,10 +315,9 @@ public static void rewriteManifestList( manifestFiles.forEach( mf -> Preconditions.checkArgument( - mf.path().startsWith(sourcePrefix), - "Encountered manifest file %s not under the source prefix %s", - mf.path(), - sourcePrefix)); + rewrittenManifests.containsKey(mf.path()), + "Encountered manifest file %s that was not rewritten", + mf.path())); try (FileAppender writer = ManifestLists.write( @@ -336,16 +329,13 @@ public static void rewriteManifestList( snapshot.firstRowId())) { for (ManifestFile file : manifestFiles) { + String rewrittenPath = rewrittenManifests.get(file.path()).getNewPath(); + long rewrittenSize = rewrittenManifests.get(file.path()).getNewSize(); + ManifestFile newFile = file.copy(); + ((StructLike) newFile).set(0, rewrittenPath); + ((StructLike) newFile).set(1, rewrittenSize); - if (rewrittenManifests.containsKey(file.path())) { - String rewrittenPath = rewrittenManifests.get(file.path()).getNewPath(); - long rewrittenSize = rewrittenManifests.get(file.path()).getNewSize(); - ((StructLike) newFile).set(0, rewrittenPath); - ((StructLike) newFile).set(1, rewrittenSize); - } else { - ((StructLike) newFile).set(0, newPath(newFile.path(), sourcePrefix, targetPrefix)); - } writer.add(newFile); } } catch (IOException e) { diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/BaseSparkAction.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/BaseSparkAction.java index a8e82d101fbf..456dcaf7d441 100644 --- a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/BaseSparkAction.java +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/BaseSparkAction.java @@ -151,22 +151,8 @@ protected Dataset contentFileDS(Table table) { protected Dataset contentFileDS(Table table, Set snapshotIds) { Table serializableTable = SerializableTableWithSize.copyOf(table); Broadcast
tableBroadcast = sparkContext.broadcast(serializableTable); - int numShufflePartitions = spark.sessionState().conf().numShufflePartitions(); - - Dataset manifestBeanDS = - manifestDF(table, snapshotIds) - .selectExpr( - "content", - "path", - "length", - "0 as sequenceNumber", - "partition_spec_id as partitionSpecId", - "added_snapshot_id as addedSnapshotId") - .dropDuplicates("path") - .repartition(numShufflePartitions) // avoid adaptive execution combining tasks - .as(ManifestFileBean.ENCODER); - - return manifestBeanDS.flatMap(new ReadManifest(tableBroadcast), FileInfo.ENCODER); + return manifestBeanDS(table, snapshotIds) + .flatMap(new ReadManifest(tableBroadcast), FileInfo.ENCODER); } protected Dataset manifestDS(Table table) { @@ -179,6 +165,22 @@ protected Dataset manifestDS(Table table, Set snapshotIds) { .as(FileInfo.ENCODER); } + protected Dataset manifestBeanDS(Table table, Set snapshotIds) { + int numShufflePartitions = spark.sessionState().conf().numShufflePartitions(); + + return manifestDF(table, snapshotIds) + .selectExpr( + "content", + "path", + "length", + "0 as sequenceNumber", + "partition_spec_id as partitionSpecId", + "added_snapshot_id as addedSnapshotId") + .dropDuplicates("path") + .repartition(numShufflePartitions) // avoid adaptive execution combining tasks + .as(ManifestFileBean.ENCODER); + } + private Dataset manifestDF(Table table, Set snapshotIds) { Dataset manifestDF = loadMetadataTable(table, ALL_MANIFESTS); if (snapshotIds != null) { diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteTablePathSparkAction.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteTablePathSparkAction.java index eedeecbf77ab..9d525efd780a 100644 --- a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteTablePathSparkAction.java +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteTablePathSparkAction.java @@ -28,7 +28,6 @@ import java.util.Set; import java.util.UUID; import java.util.stream.Collectors; -import java.util.stream.Stream; import org.apache.iceberg.ContentFile; import org.apache.iceberg.DataFile; import org.apache.iceberg.DeleteFile; @@ -60,7 +59,6 @@ import org.apache.iceberg.data.parquet.GenericParquetReaders; import org.apache.iceberg.data.parquet.GenericParquetWriter; import org.apache.iceberg.deletes.PositionDeleteWriter; -import org.apache.iceberg.exceptions.NotFoundException; import org.apache.iceberg.exceptions.RuntimeIOException; import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.io.DeleteSchemaUtil; @@ -83,9 +81,7 @@ import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Encoder; import org.apache.spark.sql.Encoders; -import org.apache.spark.sql.Row; import org.apache.spark.sql.SparkSession; -import org.apache.spark.sql.functions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import scala.Tuple3; @@ -286,15 +282,16 @@ private String rebuildMetadata() { // rebuild version files RewriteResult rewriteVersionResult = rewriteVersionFiles(endMetadata); Set deltaSnapshots = deltaSnapshots(startMetadata, rewriteVersionResult.toRewrite()); + Set validSnapshots = + Sets.difference(snapshotSet(endMetadata), snapshotSet(startMetadata)); // rebuild manifest files - Set manifestsToRewrite = manifestsToRewrite(deltaSnapshots, startMetadata); + Set manifestsToRewrite = manifestsToRewrite(validSnapshots); + ManifestsRewriteResult rewriteManifestResult = rewriteManifests(deltaSnapshots, endMetadata, manifestsToRewrite); // rebuild manifest-list files - Set validSnapshots = - Sets.difference(snapshotSet(endMetadata), snapshotSet(startMetadata)); RewriteResult rewriteManifestListResult = validSnapshots.stream() .map( @@ -432,14 +429,7 @@ private RewriteResult rewriteManifestList( String path = snapshot.manifestListLocation(); String outputPath = RewriteTablePathUtil.stagingPath(path, stagingDir); RewriteTablePathUtil.rewriteManifestList( - snapshot, - table.io(), - tableMetadata, - rewrittenManifests, - sourcePrefix, - targetPrefix, - stagingDir, - outputPath); + snapshot, table.io(), tableMetadata, rewrittenManifests, outputPath); // add the manifest list copy plan itself to the result result @@ -448,58 +438,12 @@ private RewriteResult rewriteManifestList( return result; } - private Set manifestsToRewrite( - Set deltaSnapshots, TableMetadata startMetadata) { + private Set manifestsToRewrite(Set liveSnapshots) { try { Table endStaticTable = newStaticTable(endVersionName, table.io()); - Dataset allManifestsDF = manifestDS(endStaticTable).select("path"); - Set expectedManifestPaths; - - if (startMetadata == null) { - expectedManifestPaths = - Sets.newHashSet(allManifestsDF.distinct().as(Encoders.STRING()).collectAsList()); - } else { - Set deltaSnapshotIds = - deltaSnapshots.stream().map(Snapshot::snapshotId).collect(Collectors.toSet()); - expectedManifestPaths = - Sets.newHashSet( - allManifestsDF - .distinct() - .filter( - functions - .column(ManifestFile.SNAPSHOT_ID.name()) - .isInCollection(deltaSnapshotIds)) - .as(Encoders.STRING()) - .collectAsList()); - } - - Set foundManifests = - deltaSnapshots.stream() - .flatMap( - s -> { - try { - return s.allManifests(table.io()).stream(); - } catch (NotFoundException e) { - LOG.warn( - "Skipping snapshot {} as its manifest list is missing (likely expired).", - s.snapshotId(), - e); - return Stream.empty(); - } - }) - .collect(Collectors.toSet()); - - Set foundManifestPaths = - foundManifests.stream().map(ManifestFile::path).collect(Collectors.toSet()); - Set missingPaths = Sets.difference(expectedManifestPaths, foundManifestPaths); - Preconditions.checkState( - missingPaths.isEmpty(), - "Could not find all expected manifests. Missing files: %s", - String.join(", ", missingPaths)); - - return foundManifests.stream() - .filter(m -> expectedManifestPaths.contains(m.path())) - .collect(Collectors.toSet()); + Set liveSnapshotIds = + liveSnapshots.stream().map(Snapshot::snapshotId).collect(Collectors.toSet()); + return Sets.newHashSet(manifestBeanDS(endStaticTable, liveSnapshotIds).collectAsList()); } catch (Exception e) { throw new UnsupportedOperationException( "Unable to build the manifest files dataframe. The end version in use may contain invalid snapshots. " diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteTablePathsAction.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteTablePathsAction.java index c737d551821e..d27c4404e067 100644 --- a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteTablePathsAction.java +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteTablePathsAction.java @@ -39,7 +39,6 @@ import org.apache.iceberg.DeleteFile; import org.apache.iceberg.HasTableOperations; import org.apache.iceberg.ImmutableGenericPartitionStatisticsFile; -import org.apache.iceberg.ManifestContent; import org.apache.iceberg.ManifestFile; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; @@ -232,7 +231,8 @@ public void testStartVersion() throws Exception { .startVersion("v2.metadata.json") .execute(); - checkFileNum(1, 1, 1, 4, result); + // 1 metadata JSON file, 1 snapshot, 2 manifests, 1 data file + checkFileNum(1, 1, 2, 5, result); List> paths = readPathPairList(result.fileListLocation()); @@ -306,7 +306,8 @@ public void testTableWith3Snapshots(@TempDir Path location1, @TempDir Path locat .startVersion("v2.metadata.json") .execute(); - checkFileNum(2, 2, 2, 8, result); + // 2 metadata JSON files, 2 snapshots, 3 manifests, 2 data files + checkFileNum(2, 2, 3, 9, result); // start from the first version RewriteTablePath.Result result1 = @@ -316,6 +317,7 @@ public void testTableWith3Snapshots(@TempDir Path location1, @TempDir Path locat .startVersion("v1.metadata.json") .execute(); + // 3 metadata JSON files, 3 snapshots, 3 manifests, 3 data files checkFileNum(3, 3, 3, 12, result1); } @@ -716,8 +718,8 @@ public void testStartSnapshotWithoutValidSnapshot() throws Exception { .startVersion("v2.metadata.json") .execute(); - // 2 metadata.json, 1 manifest list file, 1 manifest files - checkFileNum(2, 1, 1, 5, result); + // 2 metadata.json, 1 manifest list file, 2 manifest files + checkFileNum(2, 1, 2, 6, result); } @Test @@ -1041,7 +1043,7 @@ public void testMetadataLocationChange() throws Exception { .startVersion(fileName(metadataFilePath)) .execute(); - checkFileNum(2, 1, 1, 5, result2); + checkFileNum(2, 1, 2, 6, result2); } @Test @@ -1152,24 +1154,28 @@ public void testFullRewriteUpdatesAllManifestLengthsInManifestList( Table targetTable = TABLES.load(targetLocation); - // We have rewritten all manifests. Make sure all sizes were correctly updated in manifest lists - assertThat(targetTable.currentSnapshot().allManifests(targetTable.io())) - .allSatisfy( - manifest -> { - String manifestName = fileName(manifest.path()); - assertThat(manifest.length()) - .isNotEqualTo(manifestSizesBeforeRewrite.get(manifestName)); - }) + // We have rewritten all 11 snapshots. Make sure all sizes were correctly updated + // across all manifest lists + assertThat(targetTable.snapshots()) .allSatisfy( - manifest -> { - assertThat(targetTable.io().newInputFile(manifest.path()).getLength()) - .isEqualTo(manifest.length()); - }); + snapshot -> + assertThat(snapshot.allManifests(targetTable.io())) + .allSatisfy( + manifest -> { + String manifestName = fileName(manifest.path()); + assertThat(manifest.length()) + .isNotEqualTo(manifestSizesBeforeRewrite.get(manifestName)); + }) + .allSatisfy( + manifest -> { + assertThat(targetTable.io().newInputFile(manifest.path()).getLength()) + .isEqualTo(manifest.length()); + })); } @Test - public void testKeepsExistingDataManifestLengthInManifestList(@TempDir Path rootTargetLocation) - throws Exception { + public void testPartialRewriteUpdatesDataManifestLengthInManifestList( + @TempDir Path rootTargetLocation) throws Exception { String location = newTableLocation(); Table sourceTable = createTableWithSnapshots(location, 10); @@ -1187,8 +1193,8 @@ public void testKeepsExistingDataManifestLengthInManifestList(@TempDir Path root .startVersion("v10.metadata.json") .execute(); - // 1 metadata JSON file, 1 snapshot, 1 manifest, 4 data files - checkFileNum(1, 1, 1, 4, result); + // 1 metadata JSON file, 1 snapshot, 10 manifests, 1 data file + checkFileNum(1, 1, 10, 13, result); copyTableFiles(result); Table targetTable = TABLES.load(targetLocation); @@ -1201,24 +1207,25 @@ public void testKeepsExistingDataManifestLengthInManifestList(@TempDir Path root .filter(manifest -> manifest.snapshotId() == lastSnapshot.snapshotId()) .collect(Collectors.toList())); - // We have rewritten exactly one (data) manifest. Check that its size was correctly - // updated in the manifest list and sizes of all other manifests were left unchanged - assertThat(allManifests) - .filteredOn(manifest -> manifest != rewrittenManifest) + // We have rewritten all manifests in one snapshot. Make sure all sizes were correctly + // updated in the manifest list + assertThat(targetTable.currentSnapshot().allManifests(targetTable.io())) .allSatisfy( manifest -> { String manifestName = fileName(manifest.path()); - assertThat(manifest.length()).isEqualTo(manifestSizesBeforeRewrite.get(manifestName)); + assertThat(manifest.length()) + .isNotEqualTo(manifestSizesBeforeRewrite.get(manifestName)); + }) + .allSatisfy( + manifest -> { + assertThat(targetTable.io().newInputFile(manifest.path()).getLength()) + .isEqualTo(manifest.length()); }); - assertThat(rewrittenManifest.content()).isEqualTo(ManifestContent.DATA); - assertThat(rewrittenManifest.length()) - .isNotEqualTo(manifestSizesBeforeRewrite.get(fileName(rewrittenManifest.path()))) - .isEqualTo(targetTable.io().newInputFile(rewrittenManifest.path()).getLength()); } @Test - public void testKeepsExistingDeleteManifestLengthInManifestList(@TempDir Path rootTargetLocation) - throws Exception { + public void testPartialRewriteUpdatesDeleteManifestLengthInManifestList( + @TempDir Path rootTargetLocation) throws Exception { String location = newTableLocation(); Table sourceTable = createTableWithSnapshots(location, 5); @@ -1244,8 +1251,8 @@ public void testKeepsExistingDeleteManifestLengthInManifestList(@TempDir Path ro .startVersion("v7.metadata.json") .execute(); - // 1 metadata JSON file, 1 snapshot, 1 manifest, 1 delete file - checkFileNum(1, 1, 1, 4, result); + // 1 metadata JSON file, 1 snapshot, 5 + 2 manifests, 1 delete file + checkFileNum(1, 1, 7, 10, result); copyTableFiles(result); Table targetTable = TABLES.load(targetLocation); @@ -1258,19 +1265,20 @@ public void testKeepsExistingDeleteManifestLengthInManifestList(@TempDir Path ro .filter(manifest -> manifest.snapshotId() == lastSnapshot.snapshotId()) .collect(Collectors.toList())); - // We have rewritten exactly one (delete) manifest. Check that its size was correctly - // updated in the manifest list and sizes of all other manifests were left unchanged - assertThat(allManifests) - .filteredOn(manifest -> manifest != rewrittenManifest) + // We have rewritten all manifests in one snapshot. Make sure all sizes were correctly + // updated in the manifest list + assertThat(targetTable.currentSnapshot().allManifests(targetTable.io())) .allSatisfy( manifest -> { String manifestName = fileName(manifest.path()); - assertThat(manifest.length()).isEqualTo(manifestSizesBeforeRewrite.get(manifestName)); + assertThat(manifest.length()) + .isNotEqualTo(manifestSizesBeforeRewrite.get(manifestName)); + }) + .allSatisfy( + manifest -> { + assertThat(targetTable.io().newInputFile(manifest.path()).getLength()) + .isEqualTo(manifest.length()); }); - assertThat(rewrittenManifest.content()).isEqualTo(ManifestContent.DELETES); - assertThat(rewrittenManifest.length()) - .isNotEqualTo(manifestSizesBeforeRewrite.get(fileName(rewrittenManifest.path()))) - .isEqualTo(targetTable.io().newInputFile(rewrittenManifest.path()).getLength()); } protected static String generateLongNestedPath(int depth) { From 9146b4873a8af6fa69bbb1c39a6317c74643c718 Mon Sep 17 00:00:00 2001 From: vaultah <4944562+vaultah@users.noreply.github.com> Date: Wed, 6 Aug 2025 19:31:23 +0100 Subject: [PATCH 05/18] Only keep manifest sizes --- .../apache/iceberg/RewriteTablePathUtil.java | 51 +++++++------------ .../actions/RewriteTablePathSparkAction.java | 51 ++++++++++--------- 2 files changed, 44 insertions(+), 58 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/RewriteTablePathUtil.java b/core/src/main/java/org/apache/iceberg/RewriteTablePathUtil.java index 8247a154e38f..2ba56ef8bf47 100644 --- a/core/src/main/java/org/apache/iceberg/RewriteTablePathUtil.java +++ b/core/src/main/java/org/apache/iceberg/RewriteTablePathUtil.java @@ -87,24 +87,6 @@ public Set> copyPlan() { } } - public static class RewrittenFileInfo implements Serializable { - private final String newPath; - private final long newSize; - - public RewrittenFileInfo(String newPath, long newSize) { - this.newPath = newPath; - this.newSize = newSize; - } - - public String getNewPath() { - return newPath; - } - - public long getNewSize() { - return newSize; - } - } - /** * Create a new table metadata object, replacing path references * @@ -300,14 +282,20 @@ public static RewriteResult rewriteManifestList( * @param snapshot snapshot represented by the manifest list * @param io file io * @param tableMetadata metadata of table - * @param rewrittenManifests information about rewritten manifest files + * @param rewrittenManifestLengths rewritten manifest files and their sizes + * @param sourcePrefix source prefix that will be replaced + * @param targetPrefix target prefix that will replace it + * @param stagingDir staging directory * @param outputPath location to write the manifest list */ public static void rewriteManifestList( Snapshot snapshot, FileIO io, TableMetadata tableMetadata, - Map rewrittenManifests, + Map rewrittenManifestLengths, + String sourcePrefix, + String targetPrefix, + String stagingDir, String outputPath) { OutputFile outputFile = io.newOutputFile(outputPath); @@ -315,7 +303,7 @@ public static void rewriteManifestList( manifestFiles.forEach( mf -> Preconditions.checkArgument( - rewrittenManifests.containsKey(mf.path()), + rewrittenManifestLengths.containsKey(mf.path()), "Encountered manifest file %s that was not rewritten", mf.path())); @@ -329,13 +317,9 @@ public static void rewriteManifestList( snapshot.firstRowId())) { for (ManifestFile file : manifestFiles) { - String rewrittenPath = rewrittenManifests.get(file.path()).getNewPath(); - long rewrittenSize = rewrittenManifests.get(file.path()).getNewSize(); - ManifestFile newFile = file.copy(); - ((StructLike) newFile).set(0, rewrittenPath); - ((StructLike) newFile).set(1, rewrittenSize); - + ((StructLike) newFile).set(0, newPath(newFile.path(), sourcePrefix, targetPrefix)); + ((StructLike) newFile).set(1, rewrittenManifestLengths.get(file.path())); writer.add(newFile); } } catch (IOException e) { @@ -440,9 +424,9 @@ public static RewriteResult rewriteDataManifest( * @param specsById map of partition specs by id * @param sourcePrefix source prefix that will be replaced * @param targetPrefix target prefix that will replace it - * @return rewritten manifest file and a copy plan for the referenced content files + * @return size of the resulting manifest file and a copy plan for the referenced content files */ - public static Pair> rewriteDataManifestWithResult( + public static Pair> rewriteDataManifestWithResult( ManifestFile manifestFile, Set snapshotIds, OutputFile outputFile, @@ -468,7 +452,7 @@ public static Pair> rewriteDataManifestWit entry, snapshotIds, spec, sourcePrefix, targetPrefix, writer)) .reduce(new RewriteResult<>(), RewriteResult::append); } - return Pair.of(writer.toManifestFile(), rewriteResult); + return Pair.of(writer.length(), rewriteResult); } /** @@ -575,9 +559,9 @@ public static RewriteResult rewriteDeleteManifest( * @param targetPrefix target prefix that will replace it * @param stagingLocation staging location for rewritten files (referred delete file will be * rewritten here) - * @return rewritten manifest file and a copy plan for the referenced content files + * @return size of the resulting manifest file and a copy plan for the referenced content files */ - public static Pair> rewriteDeleteManifestWithResult( + public static Pair> rewriteDeleteManifestWithResult( ManifestFile manifestFile, Set snapshotIds, OutputFile outputFile, @@ -611,8 +595,7 @@ public static Pair> rewriteDeleteManifes writer)) .reduce(new RewriteResult<>(), RewriteResult::append); } - - return Pair.of(writer.toManifestFile(), rewriteResult); + return Pair.of(writer.length(), rewriteResult); } private static RewriteResult writeDataFileEntry( diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteTablePathSparkAction.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteTablePathSparkAction.java index 9d525efd780a..87846f72a991 100644 --- a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteTablePathSparkAction.java +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteTablePathSparkAction.java @@ -38,7 +38,6 @@ import org.apache.iceberg.RewriteTablePathUtil; import org.apache.iceberg.RewriteTablePathUtil.PositionDeleteReaderWriter; import org.apache.iceberg.RewriteTablePathUtil.RewriteResult; -import org.apache.iceberg.RewriteTablePathUtil.RewrittenFileInfo; import org.apache.iceberg.Schema; import org.apache.iceberg.Snapshot; import org.apache.iceberg.StaticTableOperations; @@ -421,15 +420,20 @@ private Set> statsFileCopyPlan( * @return a result including a copy plan for the manifest list itself */ private RewriteResult rewriteManifestList( - Snapshot snapshot, - TableMetadata tableMetadata, - Map rewrittenManifests) { + Snapshot snapshot, TableMetadata tableMetadata, Map rewrittenManifestLengths) { RewriteResult result = new RewriteResult<>(); String path = snapshot.manifestListLocation(); String outputPath = RewriteTablePathUtil.stagingPath(path, stagingDir); RewriteTablePathUtil.rewriteManifestList( - snapshot, table.io(), tableMetadata, rewrittenManifests, outputPath); + snapshot, + table.io(), + tableMetadata, + rewrittenManifestLengths, + sourcePrefix, + targetPrefix, + stagingDir, + outputPath); // add the manifest list copy plan itself to the result result @@ -474,11 +478,10 @@ public RewriteContentFileResult appendDeleteFile(RewriteResult r1) { public static class ManifestsRewriteResult { private final RewriteContentFileResult contentFileResult; - private final Map rewrittenManifests; + private final Map rewrittenManifests; ManifestsRewriteResult( - RewriteContentFileResult contentFileResult, - Map rewrittenManifests) { + RewriteContentFileResult contentFileResult, Map rewrittenManifests) { this.contentFileResult = contentFileResult; this.rewrittenManifests = rewrittenManifests; } @@ -487,7 +490,7 @@ public RewriteContentFileResult getContentFileResult() { return contentFileResult; } - public Map getRewrittenManifests() { + public Map getRewrittenManifests() { return rewrittenManifests; } } @@ -505,8 +508,8 @@ private ManifestsRewriteResult rewriteManifests( Encoder manifestFileEncoder = Encoders.javaSerialization(ManifestFile.class); Encoder contentResultEncoder = Encoders.javaSerialization(RewriteContentFileResult.class); - Encoder> tupleEncoder = - Encoders.tuple(Encoders.STRING(), manifestFileEncoder, contentResultEncoder); + Encoder> tupleEncoder = + Encoders.tuple(Encoders.STRING(), Encoders.LONG(), contentResultEncoder); Dataset manifestDS = spark().createDataset(Lists.newArrayList(toRewrite), manifestFileEncoder); @@ -514,7 +517,7 @@ private ManifestsRewriteResult rewriteManifests( deltaSnapshots.stream().map(Snapshot::snapshotId).collect(Collectors.toSet()); RewriteContentFileResult finalContentResult = new RewriteContentFileResult(); - Iterator> resultIterator = + Iterator> resultIterator = manifestDS .repartition(toRewrite.size()) .map( @@ -528,27 +531,27 @@ private ManifestsRewriteResult rewriteManifests( tupleEncoder) .toLocalIterator(); - Map rewrittenManifests = Maps.newHashMap(); + Map rewrittenManifests = Maps.newHashMap(); while (resultIterator.hasNext()) { - Tuple3 resultTuple = resultIterator.next(); + Tuple3 resultTuple = resultIterator.next(); String originalManifestPath = resultTuple._1(); - ManifestFile rewrittenManifest = resultTuple._2(); + Long rewrittenManifestLength = resultTuple._2(); RewriteContentFileResult contentFileResult = resultTuple._3(); + String stagingManifestPath = + RewriteTablePathUtil.stagingPath(originalManifestPath, stagingDir); String targetManifestPath = RewriteTablePathUtil.newPath(originalManifestPath, sourcePrefix, targetPrefix); finalContentResult.append(contentFileResult); - finalContentResult.copyPlan().add(Pair.of(rewrittenManifest.path(), targetManifestPath)); - rewrittenManifests.put( - originalManifestPath, - new RewrittenFileInfo(targetManifestPath, rewrittenManifest.length())); + finalContentResult.copyPlan().add(Pair.of(stagingManifestPath, targetManifestPath)); + rewrittenManifests.put(originalManifestPath, rewrittenManifestLength); } return new ManifestsRewriteResult(finalContentResult, rewrittenManifests); } - private static MapFunction> + private static MapFunction> toManifests( Broadcast
table, Broadcast> deltaSnapshotIds, @@ -560,7 +563,7 @@ private ManifestsRewriteResult rewriteManifests( return manifestFile -> { switch (manifestFile.content()) { case DATA: - Pair> dataFileResult = + Pair> dataFileResult = writeDataManifest( manifestFile, table, @@ -574,7 +577,7 @@ private ManifestsRewriteResult rewriteManifests( dataFileResult.first(), new RewriteContentFileResult().appendDataFile(dataFileResult.second())); case DELETES: - Pair> deleteFileResult = + Pair> deleteFileResult = writeDeleteManifest( manifestFile, table, @@ -594,7 +597,7 @@ private ManifestsRewriteResult rewriteManifests( }; } - private static Pair> writeDataManifest( + private static Pair> writeDataManifest( ManifestFile manifestFile, Broadcast
table, Broadcast> snapshotIds, @@ -622,7 +625,7 @@ private static Pair> writeDataManifest( } } - private static Pair> writeDeleteManifest( + private static Pair> writeDeleteManifest( ManifestFile manifestFile, Broadcast
table, Broadcast> snapshotIds, From a6c280dc609d443fe9925b18a0c1f95020ff8f01 Mon Sep 17 00:00:00 2001 From: vaultah <4944562+vaultah@users.noreply.github.com> Date: Wed, 6 Aug 2025 21:54:14 +0100 Subject: [PATCH 06/18] ManifestsRewriteResult visibility --- .../spark/actions/RewriteTablePathSparkAction.java | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteTablePathSparkAction.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteTablePathSparkAction.java index 981002e96a53..01369b427f5f 100644 --- a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteTablePathSparkAction.java +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteTablePathSparkAction.java @@ -479,7 +479,7 @@ public RewriteContentFileResult appendDeleteFile(RewriteResult r1) { } } - public static class ManifestsRewriteResult { + private static class ManifestsRewriteResult { private final RewriteContentFileResult contentFileResult; private final Map rewrittenManifests; @@ -566,7 +566,7 @@ private ManifestsRewriteResult rewriteManifests( return manifestFile -> { switch (manifestFile.content()) { case DATA: - Pair> dataFileResult = + Pair> dataManifestResult = writeDataManifest( manifestFile, table, @@ -577,10 +577,10 @@ private ManifestsRewriteResult rewriteManifests( targetPrefix); return Tuple3.apply( manifestFile.path(), - dataFileResult.first(), - new RewriteContentFileResult().appendDataFile(dataFileResult.second())); + dataManifestResult.first(), + new RewriteContentFileResult().appendDataFile(dataManifestResult.second())); case DELETES: - Pair> deleteFileResult = + Pair> deleteManifestResult = writeDeleteManifest( manifestFile, table, @@ -591,8 +591,8 @@ private ManifestsRewriteResult rewriteManifests( targetPrefix); return Tuple3.apply( manifestFile.path(), - deleteFileResult.first(), - new RewriteContentFileResult().appendDeleteFile(deleteFileResult.second())); + deleteManifestResult.first(), + new RewriteContentFileResult().appendDeleteFile(deleteManifestResult.second())); default: throw new UnsupportedOperationException( "Unsupported manifest type: " + manifestFile.content()); From 0d90390345e7ef27dabe8fc801a52a9a06ff676b Mon Sep 17 00:00:00 2001 From: vaultah <4944562+vaultah@users.noreply.github.com> Date: Wed, 6 Aug 2025 22:02:06 +0100 Subject: [PATCH 07/18] Remove unused argument --- core/src/main/java/org/apache/iceberg/RewriteTablePathUtil.java | 2 -- .../iceberg/spark/actions/RewriteTablePathSparkAction.java | 1 - 2 files changed, 3 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/RewriteTablePathUtil.java b/core/src/main/java/org/apache/iceberg/RewriteTablePathUtil.java index 1e3017a13b33..b4280437f121 100644 --- a/core/src/main/java/org/apache/iceberg/RewriteTablePathUtil.java +++ b/core/src/main/java/org/apache/iceberg/RewriteTablePathUtil.java @@ -287,7 +287,6 @@ public static RewriteResult rewriteManifestList( * @param rewrittenManifestLengths rewritten manifest files and their sizes * @param sourcePrefix source prefix that will be replaced * @param targetPrefix target prefix that will replace it - * @param stagingDir staging directory * @param outputPath location to write the manifest list */ public static void rewriteManifestList( @@ -297,7 +296,6 @@ public static void rewriteManifestList( Map rewrittenManifestLengths, String sourcePrefix, String targetPrefix, - String stagingDir, String outputPath) { OutputFile outputFile = io.newOutputFile(outputPath); diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteTablePathSparkAction.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteTablePathSparkAction.java index 01369b427f5f..4f70969b991a 100644 --- a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteTablePathSparkAction.java +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteTablePathSparkAction.java @@ -435,7 +435,6 @@ private RewriteResult rewriteManifestList( rewrittenManifestLengths, sourcePrefix, targetPrefix, - stagingDir, outputPath); // add the manifest list copy plan itself to the result From 23fcfd87c2937665ebd9ac7ca8e598f98b71e2ba Mon Sep 17 00:00:00 2001 From: vaultah <4944562+vaultah@users.noreply.github.com> Date: Wed, 6 Aug 2025 22:14:32 +0100 Subject: [PATCH 08/18] Checkstyle --- .../spark/actions/TestRewriteTablePathsAction.java | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteTablePathsAction.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteTablePathsAction.java index 7a9172fec985..aff44427325b 100644 --- a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteTablePathsAction.java +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteTablePathsAction.java @@ -1377,12 +1377,13 @@ protected static String generateLongNestedPath(int depth) { return pathBuilder.toString(); } - protected void addAnyPositionDelete(Table table, String path) throws Exception { + protected void addAnyPositionDelete(Table targetTable, String path) throws Exception { DataFile dataFile = - StreamSupport.stream(table.snapshots().spliterator(), false) + StreamSupport.stream(targetTable.snapshots().spliterator(), false) .flatMap( snapshot -> - StreamSupport.stream(snapshot.addedDataFiles(table.io()).spliterator(), false)) + StreamSupport.stream( + snapshot.addedDataFiles(targetTable.io()).spliterator(), false)) .findAny() .get(); @@ -1391,10 +1392,10 @@ protected void addAnyPositionDelete(Table table, String path) throws Exception { DeleteFile positionDeletes = FileHelpers.writeDeleteFile( - table, table.io().newOutputFile(file.toURI().toString()), deletes) + targetTable, targetTable.io().newOutputFile(file.toURI().toString()), deletes) .first(); - table.newRowDelta().addDeletes(positionDeletes).commit(); + targetTable.newRowDelta().addDeletes(positionDeletes).commit(); } protected void checkFileNum( From e82a4db61dfd01f94d44fae5a888c75ebd092353 Mon Sep 17 00:00:00 2001 From: vaultah <4944562+vaultah@users.noreply.github.com> Date: Sun, 10 Aug 2025 04:45:51 +0100 Subject: [PATCH 09/18] Keep the old precondition check --- .../apache/iceberg/RewriteTablePathUtil.java | 18 ++++++++++++------ 1 file changed, 12 insertions(+), 6 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/RewriteTablePathUtil.java b/core/src/main/java/org/apache/iceberg/RewriteTablePathUtil.java index b4280437f121..318c6d468663 100644 --- a/core/src/main/java/org/apache/iceberg/RewriteTablePathUtil.java +++ b/core/src/main/java/org/apache/iceberg/RewriteTablePathUtil.java @@ -301,11 +301,17 @@ public static void rewriteManifestList( List manifestFiles = manifestFilesInSnapshot(io, snapshot); manifestFiles.forEach( - mf -> - Preconditions.checkArgument( - rewrittenManifestLengths.containsKey(mf.path()), - "Encountered manifest file %s that was not rewritten", - mf.path())); + mf -> { + Preconditions.checkArgument( + mf.path().startsWith(sourcePrefix), + "Encountered manifest file %s not under the source prefix %s", + mf.path(), + sourcePrefix); + Preconditions.checkArgument( + rewrittenManifestLengths.containsKey(mf.path()), + "Encountered manifest file %s that was not rewritten", + mf.path()); + }); try (FileAppender writer = ManifestLists.write( @@ -318,7 +324,7 @@ public static void rewriteManifestList( for (ManifestFile file : manifestFiles) { ManifestFile newFile = file.copy(); - ((StructLike) newFile).set(0, newPath(newFile.path(), sourcePrefix, targetPrefix)); + ((StructLike) newFile).set(0, newPath(file.path(), sourcePrefix, targetPrefix)); ((StructLike) newFile).set(1, rewrittenManifestLengths.get(file.path())); writer.add(newFile); } From 6a8acc9478a6e040e5711bd87832e2d2b4676f3d Mon Sep 17 00:00:00 2001 From: vaultah <4944562+vaultah@users.noreply.github.com> Date: Tue, 19 Aug 2025 22:21:53 +0100 Subject: [PATCH 10/18] Cleanup --- .../apache/iceberg/RewriteTablePathUtil.java | 4 ++-- .../actions/TestRewriteTablePathsAction.java | 23 ++++--------------- 2 files changed, 6 insertions(+), 21 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/RewriteTablePathUtil.java b/core/src/main/java/org/apache/iceberg/RewriteTablePathUtil.java index 318c6d468663..73837030dbd3 100644 --- a/core/src/main/java/org/apache/iceberg/RewriteTablePathUtil.java +++ b/core/src/main/java/org/apache/iceberg/RewriteTablePathUtil.java @@ -447,7 +447,7 @@ public static Pair> rewriteDataManifestWithResult( ManifestFiles.write(format, spec, outputFile, manifestFile.snapshotId()); RewriteResult rewriteResult = null; - try (ManifestWriter dataManifestWriter = writer; + try (writer; ManifestReader reader = ManifestFiles.read(manifestFile, io, specsById).select(Arrays.asList("*"))) { rewriteResult = @@ -582,7 +582,7 @@ public static Pair> rewriteDeleteManifestWithRes ManifestFiles.writeDeleteManifest(format, spec, outputFile, manifestFile.snapshotId()); RewriteResult rewriteResult = null; - try (ManifestWriter deleteManifestWriter = writer; + try (writer; ManifestReader reader = ManifestFiles.readDeleteManifest(manifestFile, io, specsById) .select(Arrays.asList("*"))) { diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteTablePathsAction.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteTablePathsAction.java index aff44427325b..36b0fee1b6bf 100644 --- a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteTablePathsAction.java +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteTablePathsAction.java @@ -39,7 +39,6 @@ import org.apache.iceberg.DeleteFile; import org.apache.iceberg.HasTableOperations; import org.apache.iceberg.ImmutableGenericPartitionStatisticsFile; -import org.apache.iceberg.ManifestFile; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; import org.apache.iceberg.Snapshot; @@ -1219,7 +1218,7 @@ public void testFullRewriteUpdatesAllManifestLengthsInManifestList( String location = newTableLocation(); Table sourceTable = createTableWithSnapshots(location, 10); - addAnyPositionDelete( + commitNewPositionDelete( sourceTable, removePrefix(sourceTable.location() + "/data/deeply/nested/deletes.parquet")); Map manifestSizesBeforeRewrite = @@ -1287,13 +1286,6 @@ public void testPartialRewriteUpdatesDataManifestLengthInManifestList( Table targetTable = TABLES.load(targetLocation); Snapshot lastSnapshot = targetTable.currentSnapshot(); - List allManifests = lastSnapshot.allManifests(targetTable.io()); - ManifestFile rewrittenManifest = - Iterables.getOnlyElement( - allManifests.stream() - .filter(manifest -> manifest.snapshotId() == lastSnapshot.snapshotId()) - .collect(Collectors.toList())); - // We have rewritten all manifests in one snapshot. Make sure all sizes were correctly // updated in the manifest list assertThat(targetTable.currentSnapshot().allManifests(targetTable.io())) @@ -1317,10 +1309,10 @@ public void testPartialRewriteUpdatesDeleteManifestLengthInManifestList( Table sourceTable = createTableWithSnapshots(location, 5); // Add two more snapshots with just position deletes - addAnyPositionDelete( + commitNewPositionDelete( sourceTable, removePrefix(sourceTable.location() + "/data/deeply/nested/deletes-1.parquet")); - addAnyPositionDelete( + commitNewPositionDelete( sourceTable, removePrefix(sourceTable.location() + "/data/deeply/nested/deletes-2.parquet")); @@ -1345,13 +1337,6 @@ public void testPartialRewriteUpdatesDeleteManifestLengthInManifestList( Table targetTable = TABLES.load(targetLocation); Snapshot lastSnapshot = targetTable.currentSnapshot(); - List allManifests = lastSnapshot.allManifests(targetTable.io()); - ManifestFile rewrittenManifest = - Iterables.getOnlyElement( - allManifests.stream() - .filter(manifest -> manifest.snapshotId() == lastSnapshot.snapshotId()) - .collect(Collectors.toList())); - // We have rewritten all manifests in one snapshot. Make sure all sizes were correctly // updated in the manifest list assertThat(targetTable.currentSnapshot().allManifests(targetTable.io())) @@ -1377,7 +1362,7 @@ protected static String generateLongNestedPath(int depth) { return pathBuilder.toString(); } - protected void addAnyPositionDelete(Table targetTable, String path) throws Exception { + protected void commitNewPositionDelete(Table targetTable, String path) throws Exception { DataFile dataFile = StreamSupport.stream(targetTable.snapshots().spliterator(), false) .flatMap( From 150ecd54b2633554a42863ddb514dc4bb2415cfa Mon Sep 17 00:00:00 2001 From: vaultah <4944562+vaultah@users.noreply.github.com> Date: Tue, 19 Aug 2025 22:39:43 +0100 Subject: [PATCH 11/18] longTargetTableLocation --- .../actions/TestRewriteTablePathsAction.java | 31 ++++++++++--------- 1 file changed, 16 insertions(+), 15 deletions(-) diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteTablePathsAction.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteTablePathsAction.java index 36b0fee1b6bf..08d84dad8d1b 100644 --- a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteTablePathsAction.java +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteTablePathsAction.java @@ -1213,8 +1213,7 @@ public void testNestedDirectoryStructurePreservation() throws Exception { } @Test - public void testFullRewriteUpdatesAllManifestLengthsInManifestList( - @TempDir Path rootTargetLocation) throws Exception { + public void testFullRewriteUpdatesAllManifestLengthsInManifestList() throws Exception { String location = newTableLocation(); Table sourceTable = createTableWithSnapshots(location, 10); @@ -1225,13 +1224,13 @@ public void testFullRewriteUpdatesAllManifestLengthsInManifestList( sourceTable.currentSnapshot().allManifests(sourceTable.io()).stream() .collect(Collectors.toMap(m -> fileName(m.path()), m -> m.length())); - // Rewrite table metadata to a location that's much longer than the original in order + // Rewrite table metadata to a location that's longer than the original in order // to make manifests larger - String targetLocation = toAbsolute(rootTargetLocation) + generateLongNestedPath(25); + String targetLocation = longTargetTableLocation(); RewriteTablePath.Result result = actions() .rewriteTablePath(sourceTable) - .rewriteLocationPrefix(newTableLocation(), targetLocation) + .rewriteLocationPrefix(location, targetLocation) .execute(); // 1 + 11 JSON metadata files, 11 snapshots, 11 manifests, 10 data files, 1 delete file @@ -1260,8 +1259,7 @@ public void testFullRewriteUpdatesAllManifestLengthsInManifestList( } @Test - public void testPartialRewriteUpdatesDataManifestLengthInManifestList( - @TempDir Path rootTargetLocation) throws Exception { + public void testPartialRewriteUpdatesDataManifestLengthInManifestList() throws Exception { String location = newTableLocation(); Table sourceTable = createTableWithSnapshots(location, 10); @@ -1269,13 +1267,13 @@ public void testPartialRewriteUpdatesDataManifestLengthInManifestList( sourceTable.currentSnapshot().allManifests(sourceTable.io()).stream() .collect(Collectors.toMap(m -> fileName(m.path()), m -> m.length())); - // Rewrite just the latest table version to a location that's much longer than + // Rewrite just the latest table version to a location that's longer than // the original in order to make manifests larger - String targetLocation = toAbsolute(rootTargetLocation) + generateLongNestedPath(25); + String targetLocation = longTargetTableLocation(); RewriteTablePath.Result result = actions() .rewriteTablePath(sourceTable) - .rewriteLocationPrefix(newTableLocation(), targetLocation) + .rewriteLocationPrefix(location, targetLocation) .startVersion("v10.metadata.json") .execute(); @@ -1303,8 +1301,7 @@ public void testPartialRewriteUpdatesDataManifestLengthInManifestList( } @Test - public void testPartialRewriteUpdatesDeleteManifestLengthInManifestList( - @TempDir Path rootTargetLocation) throws Exception { + public void testPartialRewriteUpdatesDeleteManifestLengthInManifestList() throws Exception { String location = newTableLocation(); Table sourceTable = createTableWithSnapshots(location, 5); @@ -1320,13 +1317,13 @@ public void testPartialRewriteUpdatesDeleteManifestLengthInManifestList( sourceTable.currentSnapshot().allManifests(sourceTable.io()).stream() .collect(Collectors.toMap(m -> fileName(m.path()), m -> m.length())); - // Rewrite just the latest table version to a location that's much longer than + // Rewrite just the latest table version to a location that's longer than // the original in order to make manifests larger - String targetLocation = toAbsolute(rootTargetLocation) + generateLongNestedPath(25); + String targetLocation = longTargetTableLocation(); RewriteTablePath.Result result = actions() .rewriteTablePath(sourceTable) - .rewriteLocationPrefix(newTableLocation(), targetLocation) + .rewriteLocationPrefix(location, targetLocation) .startVersion("v7.metadata.json") .execute(); @@ -1437,6 +1434,10 @@ protected String targetTableLocation() throws IOException { return toAbsolute(targetTableDir); } + protected String longTargetTableLocation() throws IOException { + return toAbsolute(targetTableDir) + generateLongNestedPath(5); + } + protected String stagingLocation() throws IOException { return toAbsolute(staging); } From 4d8886e48b681357d6eeef5f6b06c3bd8ab5b090 Mon Sep 17 00:00:00 2001 From: vaultah <4944562+vaultah@users.noreply.github.com> Date: Thu, 21 Aug 2025 12:17:28 +0100 Subject: [PATCH 12/18] RewriteResult.size --- .../apache/iceberg/RewriteTablePathUtil.java | 112 +++--------------- .../actions/RewriteTablePathSparkAction.java | 112 ++++++++---------- 2 files changed, 67 insertions(+), 157 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/RewriteTablePathUtil.java b/core/src/main/java/org/apache/iceberg/RewriteTablePathUtil.java index 73837030dbd3..95daa9fde39d 100644 --- a/core/src/main/java/org/apache/iceberg/RewriteTablePathUtil.java +++ b/core/src/main/java/org/apache/iceberg/RewriteTablePathUtil.java @@ -64,6 +64,7 @@ private RewriteTablePathUtil() {} public static class RewriteResult implements Serializable { private final Set toRewrite = Sets.newHashSet(); private final Set> copyPlan = Sets.newHashSet(); + private Long size = null; public RewriteResult() {} @@ -85,6 +86,14 @@ public Set toRewrite() { public Set> copyPlan() { return copyPlan; } + + public Long size() { + return size; + } + + public void setSize(long newSize) { + this.size = newSize; + } } /** @@ -308,8 +317,8 @@ public static void rewriteManifestList( mf.path(), sourcePrefix); Preconditions.checkArgument( - rewrittenManifestLengths.containsKey(mf.path()), - "Encountered manifest file %s that was not rewritten", + rewrittenManifestLengths.get(mf.path()) != null, + "Encountered manifest file %s that was not rewritten or has null length", mf.path()); }); @@ -393,9 +402,7 @@ public static RewriteResult rewriteDataManifest( * @param sourcePrefix source prefix that will be replaced * @param targetPrefix target prefix that will replace it * @return a copy plan of content files in the manifest that was rewritten - * @deprecated since 1.10.0, will be removed in 1.11.0 */ - @Deprecated public static RewriteResult rewriteDataManifest( ManifestFile manifestFile, Set snapshotIds, @@ -407,45 +414,9 @@ public static RewriteResult rewriteDataManifest( String targetPrefix) throws IOException { PartitionSpec spec = specsById.get(manifestFile.partitionSpecId()); - try (ManifestWriter writer = - ManifestFiles.write(format, spec, outputFile, manifestFile.snapshotId()); - ManifestReader reader = - ManifestFiles.read(manifestFile, io, specsById).select(Arrays.asList("*"))) { - return StreamSupport.stream(reader.entries().spliterator(), false) - .map( - entry -> - writeDataFileEntry(entry, snapshotIds, spec, sourcePrefix, targetPrefix, writer)) - .reduce(new RewriteResult<>(), RewriteResult::append); - } - } - - /** - * Rewrite a data manifest, replacing path references. - * - * @param manifestFile source manifest file to rewrite - * @param snapshotIds snapshot ids for filtering returned data manifest entries - * @param outputFile output file to rewrite manifest file to - * @param io file io - * @param format format of the manifest file - * @param specsById map of partition specs by id - * @param sourcePrefix source prefix that will be replaced - * @param targetPrefix target prefix that will replace it - * @return size of the resulting manifest file and a copy plan for the referenced content files - */ - public static Pair> rewriteDataManifestWithResult( - ManifestFile manifestFile, - Set snapshotIds, - OutputFile outputFile, - FileIO io, - int format, - Map specsById, - String sourcePrefix, - String targetPrefix) - throws IOException { - PartitionSpec spec = specsById.get(manifestFile.partitionSpecId()); + RewriteResult rewriteResult; ManifestWriter writer = ManifestFiles.write(format, spec, outputFile, manifestFile.snapshotId()); - RewriteResult rewriteResult = null; try (writer; ManifestReader reader = @@ -458,7 +429,9 @@ public static Pair> rewriteDataManifestWithResult( entry, snapshotIds, spec, sourcePrefix, targetPrefix, writer)) .reduce(new RewriteResult<>(), RewriteResult::append); } - return Pair.of(writer.length(), rewriteResult); + + rewriteResult.setSize(writer.length()); + return rewriteResult; } /** @@ -516,9 +489,7 @@ public static RewriteResult rewriteDeleteManifest( * @param stagingLocation staging location for rewritten files (referred delete file will be * rewritten here) * @return a copy plan of content files in the manifest that was rewritten - * @deprecated since 1.10.0, will be removed in 1.11.0 */ - @Deprecated public static RewriteResult rewriteDeleteManifest( ManifestFile manifestFile, Set snapshotIds, @@ -531,56 +502,9 @@ public static RewriteResult rewriteDeleteManifest( String stagingLocation) throws IOException { PartitionSpec spec = specsById.get(manifestFile.partitionSpecId()); - try (ManifestWriter writer = - ManifestFiles.writeDeleteManifest(format, spec, outputFile, manifestFile.snapshotId()); - ManifestReader reader = - ManifestFiles.readDeleteManifest(manifestFile, io, specsById) - .select(Arrays.asList("*"))) { - return StreamSupport.stream(reader.entries().spliterator(), false) - .map( - entry -> - writeDeleteFileEntry( - entry, - snapshotIds, - spec, - sourcePrefix, - targetPrefix, - stagingLocation, - writer)) - .reduce(new RewriteResult<>(), RewriteResult::append); - } - } - - /** - * Rewrite a delete manifest, replacing path references. - * - * @param manifestFile source delete manifest to rewrite - * @param snapshotIds snapshot ids for filtering returned delete manifest entries - * @param outputFile output file to rewrite manifest file to - * @param io file io - * @param format format of the manifest file - * @param specsById map of partition specs by id - * @param sourcePrefix source prefix that will be replaced - * @param targetPrefix target prefix that will replace it - * @param stagingLocation staging location for rewritten files (referred delete file will be - * rewritten here) - * @return size of the resulting manifest file and a copy plan for the referenced content files - */ - public static Pair> rewriteDeleteManifestWithResult( - ManifestFile manifestFile, - Set snapshotIds, - OutputFile outputFile, - FileIO io, - int format, - Map specsById, - String sourcePrefix, - String targetPrefix, - String stagingLocation) - throws IOException { - PartitionSpec spec = specsById.get(manifestFile.partitionSpecId()); + RewriteResult rewriteResult; ManifestWriter writer = ManifestFiles.writeDeleteManifest(format, spec, outputFile, manifestFile.snapshotId()); - RewriteResult rewriteResult = null; try (writer; ManifestReader reader = @@ -600,7 +524,9 @@ public static Pair> rewriteDeleteManifestWithRes writer)) .reduce(new RewriteResult<>(), RewriteResult::append); } - return Pair.of(writer.length(), rewriteResult); + + rewriteResult.setSize(writer.length()); + return rewriteResult; } private static RewriteResult writeDataFileEntry( diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteTablePathSparkAction.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteTablePathSparkAction.java index 4f70969b991a..5ab50c5c9535 100644 --- a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteTablePathSparkAction.java +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteTablePathSparkAction.java @@ -83,7 +83,7 @@ import org.apache.spark.sql.SparkSession; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import scala.Tuple3; +import scala.Tuple2; public class RewriteTablePathSparkAction extends BaseSparkAction implements RewriteTablePath { @@ -287,21 +287,26 @@ private String rebuildMetadata() { // rebuild manifest files Set manifestsToRewrite = manifestsToRewrite(validSnapshots); - ManifestsRewriteResult rewriteManifestResult = + Map rewriteManifestResult = rewriteManifests(deltaSnapshots, endMetadata, manifestsToRewrite); + RewriteContentFileResult allManifestsResult = new RewriteContentFileResult(); + Map rewrittenManifestLengths = Maps.newHashMap(); + rewriteManifestResult.forEach( + (path, rewriteResult) -> { + rewrittenManifestLengths.put(path, rewriteResult.size()); + allManifestsResult.append(rewriteResult); + }); + // rebuild manifest-list files RewriteResult rewriteManifestListResult = validSnapshots.stream() - .map( - snapshot -> - rewriteManifestList( - snapshot, endMetadata, rewriteManifestResult.getRewrittenManifests())) + .map(snapshot -> rewriteManifestList(snapshot, endMetadata, rewrittenManifestLengths)) .reduce(new RewriteResult<>(), RewriteResult::append); // rebuild position delete files Set deleteFiles = - rewriteManifestResult.getContentFileResult().toRewrite().stream() + allManifestsResult.toRewrite().stream() .filter(e -> e instanceof DeleteFile) .map(e -> (DeleteFile) e) .collect(Collectors.toSet()); @@ -310,7 +315,7 @@ private String rebuildMetadata() { Set> copyPlan = Sets.newHashSet(); copyPlan.addAll(rewriteVersionResult.copyPlan()); copyPlan.addAll(rewriteManifestListResult.copyPlan()); - copyPlan.addAll(rewriteManifestResult.getContentFileResult().copyPlan()); + copyPlan.addAll(allManifestsResult.copyPlan()); return saveFileList(copyPlan); } @@ -478,48 +483,28 @@ public RewriteContentFileResult appendDeleteFile(RewriteResult r1) { } } - private static class ManifestsRewriteResult { - private final RewriteContentFileResult contentFileResult; - private final Map rewrittenManifests; - - ManifestsRewriteResult( - RewriteContentFileResult contentFileResult, Map rewrittenManifests) { - this.contentFileResult = contentFileResult; - this.rewrittenManifests = rewrittenManifests; - } - - public RewriteContentFileResult getContentFileResult() { - return contentFileResult; - } - - public Map getRewrittenManifests() { - return rewrittenManifests; - } - } - /** * Rewrite manifest files in a distributed manner and return the resulting manifests and content * files selected for rewriting. */ - private ManifestsRewriteResult rewriteManifests( + private Map rewriteManifests( Set deltaSnapshots, TableMetadata tableMetadata, Set toRewrite) { if (toRewrite.isEmpty()) { - return new ManifestsRewriteResult(new RewriteContentFileResult(), Maps.newHashMap()); + return Maps.newHashMap(); } Encoder manifestFileEncoder = Encoders.javaSerialization(ManifestFile.class); - Encoder contentResultEncoder = + Encoder manifestResultEncoder = Encoders.javaSerialization(RewriteContentFileResult.class); - Encoder> tupleEncoder = - Encoders.tuple(Encoders.STRING(), Encoders.LONG(), contentResultEncoder); + Encoder> tupleEncoder = + Encoders.tuple(Encoders.STRING(), manifestResultEncoder); Dataset manifestDS = spark().createDataset(Lists.newArrayList(toRewrite), manifestFileEncoder); Set deltaSnapshotIds = deltaSnapshots.stream().map(Snapshot::snapshotId).collect(Collectors.toSet()); - RewriteContentFileResult finalContentResult = new RewriteContentFileResult(); - Iterator> resultIterator = + Iterator> resultIterator = manifestDS .repartition(toRewrite.size()) .map( @@ -533,39 +518,38 @@ private ManifestsRewriteResult rewriteManifests( tupleEncoder) .toLocalIterator(); - Map rewrittenManifests = Maps.newHashMap(); + Map rewrittenManifests = Maps.newHashMap(); while (resultIterator.hasNext()) { - Tuple3 resultTuple = resultIterator.next(); + Tuple2 resultTuple = resultIterator.next(); String originalManifestPath = resultTuple._1(); - Long rewrittenManifestLength = resultTuple._2(); - RewriteContentFileResult contentFileResult = resultTuple._3(); + RewriteContentFileResult manifestRewriteResult = resultTuple._2(); String stagingManifestPath = RewriteTablePathUtil.stagingPath(originalManifestPath, sourcePrefix, stagingDir); String targetManifestPath = RewriteTablePathUtil.newPath(originalManifestPath, sourcePrefix, targetPrefix); - finalContentResult.append(contentFileResult); - finalContentResult.copyPlan().add(Pair.of(stagingManifestPath, targetManifestPath)); - rewrittenManifests.put(originalManifestPath, rewrittenManifestLength); + manifestRewriteResult.copyPlan().add(Pair.of(stagingManifestPath, targetManifestPath)); + rewrittenManifests.put(originalManifestPath, manifestRewriteResult); } - return new ManifestsRewriteResult(finalContentResult, rewrittenManifests); + return rewrittenManifests; } - private static MapFunction> - toManifests( - Broadcast
table, - Broadcast> deltaSnapshotIds, - String stagingLocation, - int format, - String sourcePrefix, - String targetPrefix) { + private static MapFunction> toManifests( + Broadcast
table, + Broadcast> deltaSnapshotIds, + String stagingLocation, + int format, + String sourcePrefix, + String targetPrefix) { return manifestFile -> { + RewriteContentFileResult result; + switch (manifestFile.content()) { case DATA: - Pair> dataManifestResult = + RewriteResult dataManifestResult = writeDataManifest( manifestFile, table, @@ -574,12 +558,12 @@ private ManifestsRewriteResult rewriteManifests( format, sourcePrefix, targetPrefix); - return Tuple3.apply( - manifestFile.path(), - dataManifestResult.first(), - new RewriteContentFileResult().appendDataFile(dataManifestResult.second())); + + result = new RewriteContentFileResult().appendDataFile(dataManifestResult); + result.setSize(dataManifestResult.size()); + return Tuple2.apply(manifestFile.path(), result); case DELETES: - Pair> deleteManifestResult = + RewriteResult deleteManifestResult = writeDeleteManifest( manifestFile, table, @@ -588,10 +572,10 @@ private ManifestsRewriteResult rewriteManifests( format, sourcePrefix, targetPrefix); - return Tuple3.apply( - manifestFile.path(), - deleteManifestResult.first(), - new RewriteContentFileResult().appendDeleteFile(deleteManifestResult.second())); + + result = new RewriteContentFileResult().appendDeleteFile(deleteManifestResult); + result.setSize(deleteManifestResult.size()); + return Tuple2.apply(manifestFile.path(), result); default: throw new UnsupportedOperationException( "Unsupported manifest type: " + manifestFile.content()); @@ -599,7 +583,7 @@ private ManifestsRewriteResult rewriteManifests( }; } - private static Pair> writeDataManifest( + private static RewriteResult writeDataManifest( ManifestFile manifestFile, Broadcast
table, Broadcast> snapshotIds, @@ -614,7 +598,7 @@ private static Pair> writeDataManifest( OutputFile outputFile = io.newOutputFile(stagingPath); Map specsById = table.getValue().specs(); Set deltaSnapshotIds = snapshotIds.value(); - return RewriteTablePathUtil.rewriteDataManifestWithResult( + return RewriteTablePathUtil.rewriteDataManifest( manifestFile, deltaSnapshotIds, outputFile, @@ -628,7 +612,7 @@ private static Pair> writeDataManifest( } } - private static Pair> writeDeleteManifest( + private static RewriteResult writeDeleteManifest( ManifestFile manifestFile, Broadcast
table, Broadcast> snapshotIds, @@ -643,7 +627,7 @@ private static Pair> writeDeleteManifest( OutputFile outputFile = io.newOutputFile(stagingPath); Map specsById = table.getValue().specs(); Set deltaSnapshotIds = snapshotIds.value(); - return RewriteTablePathUtil.rewriteDeleteManifestWithResult( + return RewriteTablePathUtil.rewriteDeleteManifest( manifestFile, deltaSnapshotIds, outputFile, From c53b2279c6c793af749892258f367ec8708d53f3 Mon Sep 17 00:00:00 2001 From: vaultah <4944562+vaultah@users.noreply.github.com> Date: Thu, 21 Aug 2025 21:52:06 +0100 Subject: [PATCH 13/18] length --- .../org/apache/iceberg/RewriteTablePathUtil.java | 14 +++++++------- .../spark/actions/RewriteTablePathSparkAction.java | 6 +++--- 2 files changed, 10 insertions(+), 10 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/RewriteTablePathUtil.java b/core/src/main/java/org/apache/iceberg/RewriteTablePathUtil.java index 95daa9fde39d..4e5ddf17c4b1 100644 --- a/core/src/main/java/org/apache/iceberg/RewriteTablePathUtil.java +++ b/core/src/main/java/org/apache/iceberg/RewriteTablePathUtil.java @@ -64,7 +64,7 @@ private RewriteTablePathUtil() {} public static class RewriteResult implements Serializable { private final Set toRewrite = Sets.newHashSet(); private final Set> copyPlan = Sets.newHashSet(); - private Long size = null; + private Long length = null; public RewriteResult() {} @@ -87,12 +87,12 @@ public Set> copyPlan() { return copyPlan; } - public Long size() { - return size; + public Long length() { + return length; } - public void setSize(long newSize) { - this.size = newSize; + public void length(long newLength) { + this.length = newLength; } } @@ -430,7 +430,7 @@ public static RewriteResult rewriteDataManifest( .reduce(new RewriteResult<>(), RewriteResult::append); } - rewriteResult.setSize(writer.length()); + rewriteResult.length(writer.length()); return rewriteResult; } @@ -525,7 +525,7 @@ public static RewriteResult rewriteDeleteManifest( .reduce(new RewriteResult<>(), RewriteResult::append); } - rewriteResult.setSize(writer.length()); + rewriteResult.length(writer.length()); return rewriteResult; } diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteTablePathSparkAction.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteTablePathSparkAction.java index 5ab50c5c9535..aad2d91bef15 100644 --- a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteTablePathSparkAction.java +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteTablePathSparkAction.java @@ -294,7 +294,7 @@ private String rebuildMetadata() { Map rewrittenManifestLengths = Maps.newHashMap(); rewriteManifestResult.forEach( (path, rewriteResult) -> { - rewrittenManifestLengths.put(path, rewriteResult.size()); + rewrittenManifestLengths.put(path, rewriteResult.length()); allManifestsResult.append(rewriteResult); }); @@ -560,7 +560,7 @@ private static MapFunction deleteManifestResult = @@ -574,7 +574,7 @@ private static MapFunction Date: Thu, 21 Aug 2025 21:58:21 +0100 Subject: [PATCH 14/18] Readability --- .../actions/RewriteTablePathSparkAction.java | 16 +++++++++------- 1 file changed, 9 insertions(+), 7 deletions(-) diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteTablePathSparkAction.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteTablePathSparkAction.java index aad2d91bef15..ef2d733719b0 100644 --- a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteTablePathSparkAction.java +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteTablePathSparkAction.java @@ -290,13 +290,10 @@ private String rebuildMetadata() { Map rewriteManifestResult = rewriteManifests(deltaSnapshots, endMetadata, manifestsToRewrite); - RewriteContentFileResult allManifestsResult = new RewriteContentFileResult(); - Map rewrittenManifestLengths = Maps.newHashMap(); - rewriteManifestResult.forEach( - (path, rewriteResult) -> { - rewrittenManifestLengths.put(path, rewriteResult.length()); - allManifestsResult.append(rewriteResult); - }); + // Extract manifest file sizes for manifest list rewriting + Map rewrittenManifestLengths = + rewriteManifestResult.entrySet().stream() + .collect(Collectors.toMap(Map.Entry::getKey, entry -> entry.getValue().length())); // rebuild manifest-list files RewriteResult rewriteManifestListResult = @@ -304,6 +301,11 @@ private String rebuildMetadata() { .map(snapshot -> rewriteManifestList(snapshot, endMetadata, rewrittenManifestLengths)) .reduce(new RewriteResult<>(), RewriteResult::append); + // Aggregate all manifest rewrite results + RewriteContentFileResult allManifestsResult = + rewriteManifestResult.values().stream() + .reduce(new RewriteContentFileResult(), RewriteContentFileResult::append); + // rebuild position delete files Set deleteFiles = allManifestsResult.toRewrite().stream() From f809cf60020ac7a7bc47db4d8640486ab0c3546a Mon Sep 17 00:00:00 2001 From: vaultah <4944562+vaultah@users.noreply.github.com> Date: Fri, 22 Aug 2025 11:12:16 +0100 Subject: [PATCH 15/18] Revert unnecessary varible name change --- .../spark/actions/RewriteTablePathSparkAction.java | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteTablePathSparkAction.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteTablePathSparkAction.java index ef2d733719b0..867445bc266b 100644 --- a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteTablePathSparkAction.java +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteTablePathSparkAction.java @@ -287,12 +287,12 @@ private String rebuildMetadata() { // rebuild manifest files Set manifestsToRewrite = manifestsToRewrite(validSnapshots); - Map rewriteManifestResult = + Map rewriteManifestResultMap = rewriteManifests(deltaSnapshots, endMetadata, manifestsToRewrite); // Extract manifest file sizes for manifest list rewriting Map rewrittenManifestLengths = - rewriteManifestResult.entrySet().stream() + rewriteManifestResultMap.entrySet().stream() .collect(Collectors.toMap(Map.Entry::getKey, entry -> entry.getValue().length())); // rebuild manifest-list files @@ -302,13 +302,13 @@ private String rebuildMetadata() { .reduce(new RewriteResult<>(), RewriteResult::append); // Aggregate all manifest rewrite results - RewriteContentFileResult allManifestsResult = - rewriteManifestResult.values().stream() + RewriteContentFileResult rewriteManifestResult = + rewriteManifestResultMap.values().stream() .reduce(new RewriteContentFileResult(), RewriteContentFileResult::append); // rebuild position delete files Set deleteFiles = - allManifestsResult.toRewrite().stream() + rewriteManifestResult.toRewrite().stream() .filter(e -> e instanceof DeleteFile) .map(e -> (DeleteFile) e) .collect(Collectors.toSet()); @@ -317,7 +317,7 @@ private String rebuildMetadata() { Set> copyPlan = Sets.newHashSet(); copyPlan.addAll(rewriteVersionResult.copyPlan()); copyPlan.addAll(rewriteManifestListResult.copyPlan()); - copyPlan.addAll(allManifestsResult.copyPlan()); + copyPlan.addAll(rewriteManifestResult.copyPlan()); return saveFileList(copyPlan); } From d69e9f39362d9260da025488719a8b92cfeab719 Mon Sep 17 00:00:00 2001 From: vaultah <4944562+vaultah@users.noreply.github.com> Date: Tue, 26 Aug 2025 19:25:18 +0100 Subject: [PATCH 16/18] Ignore revapi error --- .palantir/revapi.yml | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/.palantir/revapi.yml b/.palantir/revapi.yml index 1c871ba30c04..216ed648789c 100644 --- a/.palantir/revapi.yml +++ b/.palantir/revapi.yml @@ -1262,6 +1262,12 @@ acceptedBreaks: \ java.lang.Object[]) throws java.lang.Exception" justification: "Reduce visibilty of deprecated method" org.apache.iceberg:iceberg-core: + - code: "java.class.defaultSerializationChanged" + old: "class org.apache.iceberg.RewriteTablePathUtil.RewriteResult" + new: "class org.apache.iceberg.RewriteTablePathUtil.RewriteResult" + justification: "Adding an optional field" - code: "java.class.removed" old: "class org.apache.iceberg.MetadataUpdate.EnableRowLineage" justification: "Removing deprecations for 1.10.0" From 0e6427ebec3c681400fe1260480efff1fd7e94c2 Mon Sep 17 00:00:00 2001 From: vaultah <4944562+vaultah@users.noreply.github.com> Date: Wed, 27 Aug 2025 11:36:03 +0100 Subject: [PATCH 17/18] Change justification --- .palantir/revapi.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.palantir/revapi.yml b/.palantir/revapi.yml index 216ed648789c..5f981040ea08 100644 --- a/.palantir/revapi.yml +++ b/.palantir/revapi.yml @@ -1267,7 +1267,7 @@ acceptedBreaks: \ java.lang.Object>" new: "class org.apache.iceberg.RewriteTablePathUtil.RewriteResult" - justification: "Adding an optional field" + justification: "Serialization across versions is not supported" - code: "java.class.removed" old: "class org.apache.iceberg.MetadataUpdate.EnableRowLineage" justification: "Removing deprecations for 1.10.0" From e86e8834847719bb4d20d04b73526790139b0f57 Mon Sep 17 00:00:00 2001 From: vaultah <4944562+vaultah@users.noreply.github.com> Date: Wed, 27 Aug 2025 12:40:13 +0100 Subject: [PATCH 18/18] Add a TODO --- .../iceberg/spark/actions/RewriteTablePathSparkAction.java | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteTablePathSparkAction.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteTablePathSparkAction.java index 867445bc266b..f769d7ce75e0 100644 --- a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteTablePathSparkAction.java +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteTablePathSparkAction.java @@ -506,6 +506,10 @@ private Map rewriteManifests( Set deltaSnapshotIds = deltaSnapshots.stream().map(Snapshot::snapshotId).collect(Collectors.toSet()); + // TODO: This implementation uses toLocalIterator to collect manifest rewrite results on + // the driver, which can be a bottleneck for tables with many manifests. This should be + // refactored to use a reducible result class with a more scalable aggregation pattern. + // See issue: https://github.com/apache/iceberg/issues/13932 Iterator> resultIterator = manifestDS .repartition(toRewrite.size())