diff --git a/core/src/main/java/org/apache/iceberg/CatalogUtil.java b/core/src/main/java/org/apache/iceberg/CatalogUtil.java index 7147f98e6237..2b400ccebc8b 100644 --- a/core/src/main/java/org/apache/iceberg/CatalogUtil.java +++ b/core/src/main/java/org/apache/iceberg/CatalogUtil.java @@ -127,7 +127,7 @@ public static void dropTableData(FileIO io, TableMetadata metadata) { if (gcEnabled) { // delete data files only if we are sure this won't corrupt other tables - deleteFiles(io, manifestsToDelete); + deleteFiles(io, manifestsToDelete, metadata.specsById()); } deleteFiles(io, Iterables.transform(manifestsToDelete, ManifestFile::path), "manifest"); @@ -163,7 +163,8 @@ public static void dropViewMetadata(FileIO io, ViewMetadata metadata) { } @SuppressWarnings("DangerousStringInternUsage") - private static void deleteFiles(FileIO io, Set allManifests) { + private static void deleteFiles( + FileIO io, Set allManifests, Map specsById) { // keep track of deleted files in a map that can be cleaned up when memory runs low Map deletedFiles = new MapMaker().concurrencyLevel(ThreadPools.WORKER_THREAD_POOL_SIZE).weakKeys().makeMap(); @@ -177,7 +178,7 @@ private static void deleteFiles(FileIO io, Set allManifests) { LOG.warn("Failed to get deleted files: this may cause orphaned data files", exc)) .run( manifest -> { - try (ManifestReader reader = ManifestFiles.open(manifest, io)) { + try (ManifestReader reader = ManifestFiles.open(manifest, io, specsById)) { List pathsToDelete = Lists.newArrayList(); for (ManifestEntry entry : reader.entries()) { // intern the file path because the weak key map uses identity (==) instead of diff --git a/core/src/main/java/org/apache/iceberg/CherryPickOperation.java b/core/src/main/java/org/apache/iceberg/CherryPickOperation.java index ee1435b5f187..8c8026de5a08 100644 --- a/core/src/main/java/org/apache/iceberg/CherryPickOperation.java +++ b/core/src/main/java/org/apache/iceberg/CherryPickOperation.java @@ -22,14 +22,19 @@ import java.io.UncheckedIOException; import java.util.List; import java.util.Map; +import java.util.Objects; import org.apache.iceberg.exceptions.CherrypickAncestorCommitException; import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.io.FileIO; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Iterables; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.util.ParallelIterable; import org.apache.iceberg.util.PartitionSet; import org.apache.iceberg.util.PropertyUtil; import org.apache.iceberg.util.SnapshotUtil; +import org.apache.iceberg.util.ThreadPools; import org.apache.iceberg.util.WapUtil; /** @@ -40,14 +45,12 @@ */ class CherryPickOperation extends MergingSnapshotProducer { - private final FileIO io; private Snapshot cherrypickSnapshot = null; private boolean requireFastForward = false; private PartitionSet replacedPartitions = null; CherryPickOperation(String tableName, TableOperations ops) { super(tableName, ops); - this.io = ops.io(); } @Override @@ -161,7 +164,8 @@ protected void validate(TableMetadata base, Snapshot snapshot) { // case if (!isFastForward(base)) { validateNonAncestor(base, cherrypickSnapshot.snapshotId()); - validateReplacedPartitions(base, cherrypickSnapshot.parentId(), replacedPartitions, io); + validateReplacedPartitions( + base, cherrypickSnapshot.parentId(), replacedPartitions, ops().io()); WapUtil.validateWapPublish(base, cherrypickSnapshot.snapshotId()); } } @@ -218,21 +222,49 @@ private static void validateReplacedPartitions( parentId == null || isCurrentAncestor(meta, parentId), "Cannot cherry-pick overwrite, based on non-ancestor of the current state: %s", parentId); - try (CloseableIterable newFiles = - SnapshotUtil.newFilesBetween( - parentId, meta.currentSnapshot().snapshotId(), meta::snapshot, io)) { - for (DataFile newFile : newFiles) { - ValidationException.check( - !replacedPartitions.contains(newFile.specId(), newFile.partition()), - "Cannot cherry-pick replace partitions with changed partition: %s", - newFile.partition()); + List snapshots = + Lists.newArrayList( + SnapshotUtil.ancestorsBetween( + meta.currentSnapshot().snapshotId(), parentId, meta::snapshot)); + if (!snapshots.isEmpty()) { + Iterable> addedFileTasks = + Iterables.concat( + Iterables.transform( + snapshots, + snap -> + Iterables.transform( + manifestsCreatedBy(snap, io), + manifest -> addedDataFiles(manifest, io, meta.specsById())))); + + try (CloseableIterable newFiles = + new ParallelIterable<>(addedFileTasks, ThreadPools.getWorkerPool())) { + for (DataFile newFile : newFiles) { + ValidationException.check( + !replacedPartitions.contains(newFile.specId(), newFile.partition()), + "Cannot cherry-pick replace partitions with changed partition: %s", + newFile.partition()); + } + } catch (IOException e) { + throw new UncheckedIOException("Failed to validate replaced partitions", e); } - } catch (IOException ioe) { - throw new UncheckedIOException("Failed to validate replaced partitions", ioe); } } } + private static Iterable manifestsCreatedBy(Snapshot snapshot, FileIO io) { + return Iterables.filter( + snapshot.dataManifests(io), m -> Objects.equals(m.snapshotId(), snapshot.snapshotId())); + } + + private static CloseableIterable addedDataFiles( + ManifestFile manifest, FileIO io, Map specsById) { + CloseableIterable> entries = + ManifestFiles.read(manifest, io, specsById).entries(); + CloseableIterable> added = + CloseableIterable.filter(entries, e -> e.status() == ManifestEntry.Status.ADDED); + return CloseableIterable.transform(added, e -> e.file().copy()); + } + private static Long lookupAncestorBySourceSnapshot(TableMetadata meta, long snapshotId) { String snapshotIdStr = String.valueOf(snapshotId); for (long ancestorId : currentAncestors(meta)) { diff --git a/core/src/main/java/org/apache/iceberg/PartitionStatsHandler.java b/core/src/main/java/org/apache/iceberg/PartitionStatsHandler.java index 89aabfcc6af2..9420095f94a3 100644 --- a/core/src/main/java/org/apache/iceberg/PartitionStatsHandler.java +++ b/core/src/main/java/org/apache/iceberg/PartitionStatsHandler.java @@ -499,7 +499,8 @@ private static PartitionMap computeStats( private static PartitionMap collectStatsForManifest( Table table, ManifestFile manifest, StructType partitionType, boolean incremental) { List projection = BaseScan.scanColumns(manifest.content()); - try (ManifestReader reader = ManifestFiles.open(manifest, table.io()).select(projection)) { + try (ManifestReader reader = + ManifestFiles.open(manifest, table.io(), table.specs()).select(projection)) { PartitionMap statsMap = PartitionMap.create(table.specs()); int specId = manifest.partitionSpecId(); PartitionSpec spec = table.specs().get(specId); diff --git a/core/src/main/java/org/apache/iceberg/ReachableFileCleanup.java b/core/src/main/java/org/apache/iceberg/ReachableFileCleanup.java index e860c896a477..b5a565b81e7e 100644 --- a/core/src/main/java/org/apache/iceberg/ReachableFileCleanup.java +++ b/core/src/main/java/org/apache/iceberg/ReachableFileCleanup.java @@ -19,6 +19,7 @@ package org.apache.iceberg; import java.io.IOException; +import java.util.Map; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ExecutorService; @@ -81,7 +82,8 @@ public void cleanFiles( if (!manifestsToDelete.isEmpty()) { if (ExpireSnapshots.CleanupLevel.ALL == cleanupLevel) { - Set dataFilesToDelete = findFilesToDelete(manifestsToDelete, currentManifests); + Set dataFilesToDelete = + findFilesToDelete(manifestsToDelete, currentManifests, beforeExpiration.specsById()); LOG.debug("Deleting {} data files", dataFilesToDelete.size()); deleteFiles(dataFilesToDelete, "data"); } @@ -165,9 +167,10 @@ private Set readManifests(Set snapshots) { return manifestFiles; } - // Helper to determine data files to delete private Set findFilesToDelete( - Set manifestFilesToDelete, Set currentManifestFiles) { + Set manifestFilesToDelete, + Set currentManifestFiles, + Map specsById) { Set filesToDelete = ConcurrentHashMap.newKeySet(); Tasks.foreach(manifestFilesToDelete) @@ -180,7 +183,8 @@ private Set findFilesToDelete( "Failed to determine live files in manifest {}. Retrying", item.path(), exc)) .run( manifest -> { - try (CloseableIterable paths = ManifestFiles.readPaths(manifest, fileIO)) { + try (CloseableIterable paths = + ManifestFiles.readPaths(manifest, fileIO, specsById)) { paths.forEach(filesToDelete::add); } catch (IOException e) { throw new RuntimeIOException(e, "Failed to read manifest file: %s", manifest); @@ -208,7 +212,8 @@ private Set findFilesToDelete( } // Remove all the live files from the candidate deletion set - try (CloseableIterable paths = ManifestFiles.readPaths(manifest, fileIO)) { + try (CloseableIterable paths = + ManifestFiles.readPaths(manifest, fileIO, specsById)) { paths.forEach(filesToDelete::remove); } catch (IOException e) { throw new RuntimeIOException(e, "Failed to read manifest file: %s", manifest); diff --git a/core/src/main/java/org/apache/iceberg/util/SnapshotUtil.java b/core/src/main/java/org/apache/iceberg/util/SnapshotUtil.java index dccc23c2e21f..370bbfed336e 100644 --- a/core/src/main/java/org/apache/iceberg/util/SnapshotUtil.java +++ b/core/src/main/java/org/apache/iceberg/util/SnapshotUtil.java @@ -28,6 +28,7 @@ import org.apache.iceberg.HistoryEntry; import org.apache.iceberg.Schema; import org.apache.iceberg.Snapshot; +import org.apache.iceberg.SnapshotChanges; import org.apache.iceberg.SnapshotRef; import org.apache.iceberg.Table; import org.apache.iceberg.TableMetadata; @@ -284,8 +285,8 @@ private static Iterable toIds(Iterable snapshots) { } /** - * @deprecated will be removed in 1.12.0, use {@link #newFilesBetween(Long, long, Function, - * FileIO)} instead. + * @deprecated will be removed in 1.12.0, use {@link SnapshotChanges} with {@link + * #ancestorsBetween(long, Long, Function)} instead. */ @Deprecated public static List newFiles( @@ -310,6 +311,11 @@ public static List newFiles( return newFiles; } + /** + * @deprecated will be removed in 1.12.0, use {@link SnapshotChanges} with {@link + * #ancestorsBetween(long, Long, Function)} instead. + */ + @Deprecated public static CloseableIterable newFilesBetween( Long startSnapshotId, long endSnapshotId, Function lookup, FileIO io) { diff --git a/core/src/test/java/org/apache/iceberg/TestRowDelta.java b/core/src/test/java/org/apache/iceberg/TestRowDelta.java index 749dbdc6fe53..b8a2a462b158 100644 --- a/core/src/test/java/org/apache/iceberg/TestRowDelta.java +++ b/core/src/test/java/org/apache/iceberg/TestRowDelta.java @@ -1946,7 +1946,10 @@ public void testDuplicateDVsAreMerged() throws IOException { commit(table, rowDelta1, branch); Iterable addedDeleteFiles = - latestSnapshot(table, branch).addedDeleteFiles(table.io()); + SnapshotChanges.builderFor(table) + .snapshot(latestSnapshot(table, branch)) + .build() + .addedDeleteFiles(); assertThat(Iterables.size(addedDeleteFiles)).isEqualTo(1); DeleteFile mergedDV = Iterables.getOnlyElement(addedDeleteFiles); @@ -2004,7 +2007,8 @@ public void testDuplicateDVsMergedMultipleSpecs() throws IOException { Snapshot snapshot = latestSnapshot(table, branch); // Expect 3 merged DVs, one per data file - Iterable addedDeleteFiles = snapshot.addedDeleteFiles(table.io()); + Iterable addedDeleteFiles = + SnapshotChanges.builderFor(table).snapshot(snapshot).build().addedDeleteFiles(); List mergedDVs = Lists.newArrayList(addedDeleteFiles); assertThat(mergedDVs).hasSize(3); // Should be a Puffin produced per merged DV spec @@ -2067,7 +2071,10 @@ public void testDuplicateDVsAreMergedForMultipleReferenceFiles() throws IOExcept // Expect two merged DVs, one per data file Iterable addedDeleteFiles = - latestSnapshot(table, branch).addedDeleteFiles(table.io()); + SnapshotChanges.builderFor(table) + .snapshot(latestSnapshot(table, branch)) + .build() + .addedDeleteFiles(); List mergedDVs = Lists.newArrayList(addedDeleteFiles); assertThat(mergedDVs).hasSize(2); @@ -2119,7 +2126,10 @@ public void testDuplicateDVsAndValidDV() throws IOException { // Expect two DVs: one merged for dataFile1 and deleteFile2 Iterable addedDeleteFiles = - latestSnapshot(table, branch).addedDeleteFiles(table.io()); + SnapshotChanges.builderFor(table) + .snapshot(latestSnapshot(table, branch)) + .build() + .addedDeleteFiles(); List committedDVs = Lists.newArrayList(addedDeleteFiles); assertThat(committedDVs).hasSize(2); @@ -2167,7 +2177,10 @@ public void testDuplicateDVsAreMergedAndEqDelete() throws IOException { commit(table, rowDelta, branch); Iterable addedDeleteFiles = - latestSnapshot(table, branch).addedDeleteFiles(table.io()); + SnapshotChanges.builderFor(table) + .snapshot(latestSnapshot(table, branch)) + .build() + .addedDeleteFiles(); List committedDeletes = Lists.newArrayList(addedDeleteFiles); // 1 DV + 1 equality delete diff --git a/data/src/test/java/org/apache/iceberg/io/TestDVWriters.java b/data/src/test/java/org/apache/iceberg/io/TestDVWriters.java index 70330901f4bb..97957660907f 100644 --- a/data/src/test/java/org/apache/iceberg/io/TestDVWriters.java +++ b/data/src/test/java/org/apache/iceberg/io/TestDVWriters.java @@ -35,6 +35,7 @@ import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.RowDelta; import org.apache.iceberg.Snapshot; +import org.apache.iceberg.SnapshotChanges; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; import org.apache.iceberg.TestHelpers; @@ -150,8 +151,9 @@ public void testRewriteDVs() throws IOException { // commit the first DV commit(result1); - assertThat(table.currentSnapshot().addedDeleteFiles(table.io())).hasSize(1); - assertThat(table.currentSnapshot().removedDeleteFiles(table.io())).isEmpty(); + SnapshotChanges changes1 = SnapshotChanges.builderFor(table).build(); + assertThat(changes1.addedDeleteFiles()).hasSize(1); + assertThat(changes1.removedDeleteFiles()).isEmpty(); // verify correctness after committing the first DV assertRows(ImmutableList.of(toRow(1, "aaa"), toRow(3, "aaa"))); @@ -174,8 +176,9 @@ public void testRewriteDVs() throws IOException { // replace DVs commit(result2); - assertThat(table.currentSnapshot().addedDeleteFiles(table.io())).hasSize(1); - assertThat(table.currentSnapshot().removedDeleteFiles(table.io())).hasSize(1); + SnapshotChanges changes2 = SnapshotChanges.builderFor(table).build(); + assertThat(changes2.addedDeleteFiles()).hasSize(1); + assertThat(changes2.removedDeleteFiles()).hasSize(1); // verify correctness after replacing DVs assertRows(ImmutableList.of(toRow(1, "aaa"))); @@ -220,8 +223,9 @@ public void testRewriteFileScopedPositionDeletes() throws IOException { // replace the position delete file with the DV commit(result); - assertThat(table.currentSnapshot().addedDeleteFiles(table.io())).hasSize(1); - assertThat(table.currentSnapshot().removedDeleteFiles(table.io())).hasSize(1); + SnapshotChanges changes = SnapshotChanges.builderFor(table).build(); + assertThat(changes.addedDeleteFiles()).hasSize(1); + assertThat(changes.removedDeleteFiles()).hasSize(1); // verify correctness assertRows(ImmutableList.of(toRow(3, "aaa"))); @@ -299,8 +303,9 @@ public void testApplyPartitionScopedPositionDeletes() throws IOException { // commit the DV, ensuring the position delete file remains commit(result); - assertThat(table.currentSnapshot().addedDeleteFiles(table.io())).hasSize(1); - assertThat(table.currentSnapshot().removedDeleteFiles(table.io())).isEmpty(); + SnapshotChanges changes = SnapshotChanges.builderFor(table).build(); + assertThat(changes.addedDeleteFiles()).hasSize(1); + assertThat(changes.removedDeleteFiles()).isEmpty(); // verify correctness with DVs and position delete files assertRows(ImmutableList.of(toRow(3, "aaa"), toRow(6, "aaa"))); diff --git a/delta-lake/src/integration/java/org/apache/iceberg/delta/TestSnapshotDeltaLakeTable.java b/delta-lake/src/integration/java/org/apache/iceberg/delta/TestSnapshotDeltaLakeTable.java index b2dae682d296..c71d435f3228 100644 --- a/delta-lake/src/integration/java/org/apache/iceberg/delta/TestSnapshotDeltaLakeTable.java +++ b/delta-lake/src/integration/java/org/apache/iceberg/delta/TestSnapshotDeltaLakeTable.java @@ -47,6 +47,7 @@ import org.apache.commons.codec.DecoderException; import org.apache.commons.codec.net.URLCodec; import org.apache.iceberg.Snapshot; +import org.apache.iceberg.SnapshotChanges; import org.apache.iceberg.SnapshotRef; import org.apache.iceberg.Table; import org.apache.iceberg.catalog.TableIdentifier; @@ -459,9 +460,10 @@ private void checkDataFilePathsIntegrity( deltaLog.update().getAllFiles().stream() .map(f -> getFullFilePath(f.getPath(), deltaLog.getPath().toString())) .collect(Collectors.toList()); - icebergTable - .currentSnapshot() - .addedDataFiles(icebergTable.io()) + SnapshotChanges.builderFor(icebergTable) + .snapshot(icebergTable.currentSnapshot()) + .build() + .addedDataFiles() .forEach( dataFile -> { assertThat(URI.create(dataFile.location()).isAbsolute()).isTrue(); diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/RewriteUtil.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/RewriteUtil.java index 9b19a50b092d..8a8a2fa194d4 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/RewriteUtil.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/RewriteUtil.java @@ -27,6 +27,7 @@ import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; import org.apache.flink.streaming.util.ProcessFunctionTestHarnesses; import org.apache.iceberg.DataFile; +import org.apache.iceberg.SnapshotChanges; import org.apache.iceberg.SnapshotRef; import org.apache.iceberg.Table; import org.apache.iceberg.expressions.Expressions; @@ -88,6 +89,6 @@ static List executeRewrite( static Set newDataFiles(Table table) { table.refresh(); - return Sets.newHashSet(table.currentSnapshot().addedDataFiles(table.io())); + return Sets.newHashSet(SnapshotChanges.builderFor(table).build().addedDataFiles()); } } diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestDataFileRewriteCommitter.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestDataFileRewriteCommitter.java index cdd40eb2a2bf..ded958538ff0 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestDataFileRewriteCommitter.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestDataFileRewriteCommitter.java @@ -31,6 +31,7 @@ import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; import org.apache.iceberg.ContentFile; import org.apache.iceberg.DataFile; +import org.apache.iceberg.SnapshotChanges; import org.apache.iceberg.SnapshotRef; import org.apache.iceberg.Table; import org.apache.iceberg.flink.maintenance.api.Trigger; @@ -262,9 +263,9 @@ private static void assertDataFiles( assertThat(table.currentSnapshot().summary().get(TOTAL_DATA_FILES)) .isEqualTo(String.valueOf(expectedCurrent)); - Set actualAdded = Sets.newHashSet(table.currentSnapshot().addedDataFiles(table.io())); - Set actualRemoved = - Sets.newHashSet(table.currentSnapshot().removedDataFiles(table.io())); + SnapshotChanges changes = SnapshotChanges.builderFor(table).build(); + Set actualAdded = Sets.newHashSet(changes.addedDataFiles()); + Set actualRemoved = Sets.newHashSet(changes.removedDataFiles()); assertThat(actualAdded.stream().map(DataFile::location).collect(Collectors.toSet())) .isEqualTo(expectedAdded.stream().map(DataFile::location).collect(Collectors.toSet())); assertThat(actualRemoved.stream().map(DataFile::location).collect(Collectors.toSet())) diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestMonitorSource.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestMonitorSource.java index c561c7054eae..34bb14336511 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestMonitorSource.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestMonitorSource.java @@ -40,6 +40,7 @@ import org.apache.iceberg.DeleteFile; import org.apache.iceberg.FileFormat; import org.apache.iceberg.RewriteFiles; +import org.apache.iceberg.SnapshotChanges; import org.apache.iceberg.Table; import org.apache.iceberg.data.GenericAppenderHelper; import org.apache.iceberg.data.RandomGenericData; @@ -308,7 +309,7 @@ void testSkipReplace() throws IOException { // Create a DataOperations.REPLACE snapshot DataFile dataFile = - table.snapshots().iterator().next().addedDataFiles(table.io()).iterator().next(); + SnapshotChanges.builderFor(table).build().addedDataFiles().iterator().next(); RewriteFiles rewrite = tableLoader.loadTable().newRewrite(); // Replace the file with itself for testing purposes rewrite.deleteFile(dataFile); @@ -320,14 +321,18 @@ void testSkipReplace() throws IOException { } private static long firstFileLength(Table table) { - return table.currentSnapshot().addedDataFiles(table.io()).iterator().next().fileSizeInBytes(); + return SnapshotChanges.builderFor(table) + .build() + .addedDataFiles() + .iterator() + .next() + .fileSizeInBytes(); } private static TableChange tableChangeWithLastSnapshot(Table table, TableChange previous) { - List dataFiles = - Lists.newArrayList(table.currentSnapshot().addedDataFiles(table.io()).iterator()); - List deleteFiles = - Lists.newArrayList(table.currentSnapshot().addedDeleteFiles(table.io()).iterator()); + SnapshotChanges changes = SnapshotChanges.builderFor(table).build(); + List dataFiles = Lists.newArrayList(changes.addedDataFiles().iterator()); + List deleteFiles = Lists.newArrayList(changes.addedDeleteFiles().iterator()); long dataSize = dataFiles.stream().mapToLong(ContentFile::fileSizeInBytes).sum(); long deleteRecordCount = deleteFiles.stream().mapToLong(DeleteFile::recordCount).sum(); diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java index 9ef7c707821e..fac1b8226d3c 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java @@ -35,6 +35,7 @@ import org.apache.iceberg.MetadataColumns; import org.apache.iceberg.ParameterizedTestExtension; import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.SnapshotChanges; import org.apache.iceberg.SnapshotRef; import org.apache.iceberg.TableProperties; import org.apache.iceberg.data.Record; @@ -244,11 +245,12 @@ public void testDeleteStats() throws Exception { expectedRecords, SnapshotRef.MAIN_BRANCH); - DeleteFile deleteFile = table.currentSnapshot().addedDeleteFiles(table.io()).iterator().next(); + SnapshotChanges changes = SnapshotChanges.builderFor(table).build(); + DeleteFile deleteFile = changes.addedDeleteFiles().iterator().next(); String fromStat = new String( deleteFile.lowerBounds().get(MetadataColumns.DELETE_FILE_PATH.fieldId()).array()); - DataFile dataFile = table.currentSnapshot().addedDataFiles(table.io()).iterator().next(); + DataFile dataFile = changes.addedDataFiles().iterator().next(); assumeThat(fromStat).isEqualTo(dataFile.location()); } } diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkV2.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkV2.java index 590b3f6c984c..71cec082a09e 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkV2.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkV2.java @@ -36,6 +36,7 @@ import org.apache.iceberg.ParameterizedTestExtension; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Snapshot; +import org.apache.iceberg.SnapshotChanges; import org.apache.iceberg.SnapshotRef; import org.apache.iceberg.TableProperties; import org.apache.iceberg.data.Record; @@ -227,11 +228,12 @@ public void testDeleteStats() throws Exception { expectedRecords, SnapshotRef.MAIN_BRANCH); - DeleteFile deleteFile = table.currentSnapshot().addedDeleteFiles(table.io()).iterator().next(); + SnapshotChanges changes = SnapshotChanges.builderFor(table).build(); + DeleteFile deleteFile = changes.addedDeleteFiles().iterator().next(); String fromStat = new String( deleteFile.lowerBounds().get(MetadataColumns.DELETE_FILE_PATH.fieldId()).array()); - DataFile dataFile = table.currentSnapshot().addedDataFiles(table.io()).iterator().next(); + DataFile dataFile = changes.addedDataFiles().iterator().next(); assumeThat(fromStat).isEqualTo(dataFile.location()); } diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestMetadataTableReadableMetrics.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestMetadataTableReadableMetrics.java index 488969bab045..af3a79ff67b8 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestMetadataTableReadableMetrics.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestMetadataTableReadableMetrics.java @@ -38,6 +38,7 @@ import org.apache.iceberg.Parameters; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; +import org.apache.iceberg.SnapshotChanges; import org.apache.iceberg.Table; import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.catalog.TableIdentifier; @@ -222,7 +223,8 @@ public void testPrimitiveColumns() throws Exception { // size of the column to increase. For example, with Parquet 1.14.x the // uncompressed size has been added to allow for better allocation of memory upfront. // Therefore, we look the sizes up, rather than hardcoding them - DataFile dataFile = table.currentSnapshot().addedDataFiles(table.io()).iterator().next(); + DataFile dataFile = + SnapshotChanges.builderFor(table).build().addedDataFiles().iterator().next(); Map columnSizeStats = dataFile.columnSizes(); Row binaryCol = diff --git a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/RewriteUtil.java b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/RewriteUtil.java index 9b19a50b092d..8a8a2fa194d4 100644 --- a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/RewriteUtil.java +++ b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/RewriteUtil.java @@ -27,6 +27,7 @@ import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; import org.apache.flink.streaming.util.ProcessFunctionTestHarnesses; import org.apache.iceberg.DataFile; +import org.apache.iceberg.SnapshotChanges; import org.apache.iceberg.SnapshotRef; import org.apache.iceberg.Table; import org.apache.iceberg.expressions.Expressions; @@ -88,6 +89,6 @@ static List executeRewrite( static Set newDataFiles(Table table) { table.refresh(); - return Sets.newHashSet(table.currentSnapshot().addedDataFiles(table.io())); + return Sets.newHashSet(SnapshotChanges.builderFor(table).build().addedDataFiles()); } } diff --git a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestDataFileRewriteCommitter.java b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestDataFileRewriteCommitter.java index cdd40eb2a2bf..ded958538ff0 100644 --- a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestDataFileRewriteCommitter.java +++ b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestDataFileRewriteCommitter.java @@ -31,6 +31,7 @@ import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; import org.apache.iceberg.ContentFile; import org.apache.iceberg.DataFile; +import org.apache.iceberg.SnapshotChanges; import org.apache.iceberg.SnapshotRef; import org.apache.iceberg.Table; import org.apache.iceberg.flink.maintenance.api.Trigger; @@ -262,9 +263,9 @@ private static void assertDataFiles( assertThat(table.currentSnapshot().summary().get(TOTAL_DATA_FILES)) .isEqualTo(String.valueOf(expectedCurrent)); - Set actualAdded = Sets.newHashSet(table.currentSnapshot().addedDataFiles(table.io())); - Set actualRemoved = - Sets.newHashSet(table.currentSnapshot().removedDataFiles(table.io())); + SnapshotChanges changes = SnapshotChanges.builderFor(table).build(); + Set actualAdded = Sets.newHashSet(changes.addedDataFiles()); + Set actualRemoved = Sets.newHashSet(changes.removedDataFiles()); assertThat(actualAdded.stream().map(DataFile::location).collect(Collectors.toSet())) .isEqualTo(expectedAdded.stream().map(DataFile::location).collect(Collectors.toSet())); assertThat(actualRemoved.stream().map(DataFile::location).collect(Collectors.toSet())) diff --git a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestMonitorSource.java b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestMonitorSource.java index 9c1ea2f2295a..3dca6c421c76 100644 --- a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestMonitorSource.java +++ b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestMonitorSource.java @@ -46,6 +46,7 @@ import org.apache.iceberg.DeleteFile; import org.apache.iceberg.FileFormat; import org.apache.iceberg.RewriteFiles; +import org.apache.iceberg.SnapshotChanges; import org.apache.iceberg.Table; import org.apache.iceberg.data.GenericAppenderHelper; import org.apache.iceberg.data.RandomGenericData; @@ -316,7 +317,7 @@ void testSkipReplace() throws IOException { // Create a DataOperations.REPLACE snapshot DataFile dataFile = - table.snapshots().iterator().next().addedDataFiles(table.io()).iterator().next(); + SnapshotChanges.builderFor(table).build().addedDataFiles().iterator().next(); RewriteFiles rewrite = tableLoader.loadTable().newRewrite(); // Replace the file with itself for testing purposes rewrite.deleteFile(dataFile); @@ -328,14 +329,18 @@ void testSkipReplace() throws IOException { } private static long firstFileLength(Table table) { - return table.currentSnapshot().addedDataFiles(table.io()).iterator().next().fileSizeInBytes(); + return SnapshotChanges.builderFor(table) + .build() + .addedDataFiles() + .iterator() + .next() + .fileSizeInBytes(); } private static TableChange tableChangeWithLastSnapshot(Table table, TableChange previous) { - List dataFiles = - Lists.newArrayList(table.currentSnapshot().addedDataFiles(table.io()).iterator()); - List deleteFiles = - Lists.newArrayList(table.currentSnapshot().addedDeleteFiles(table.io()).iterator()); + SnapshotChanges changes = SnapshotChanges.builderFor(table).build(); + List dataFiles = Lists.newArrayList(changes.addedDataFiles().iterator()); + List deleteFiles = Lists.newArrayList(changes.addedDeleteFiles().iterator()); long dataSize = dataFiles.stream().mapToLong(ContentFile::fileSizeInBytes).sum(); long deleteRecordCount = deleteFiles.stream().mapToLong(DeleteFile::recordCount).sum(); diff --git a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java index 9ef7c707821e..fac1b8226d3c 100644 --- a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java +++ b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java @@ -35,6 +35,7 @@ import org.apache.iceberg.MetadataColumns; import org.apache.iceberg.ParameterizedTestExtension; import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.SnapshotChanges; import org.apache.iceberg.SnapshotRef; import org.apache.iceberg.TableProperties; import org.apache.iceberg.data.Record; @@ -244,11 +245,12 @@ public void testDeleteStats() throws Exception { expectedRecords, SnapshotRef.MAIN_BRANCH); - DeleteFile deleteFile = table.currentSnapshot().addedDeleteFiles(table.io()).iterator().next(); + SnapshotChanges changes = SnapshotChanges.builderFor(table).build(); + DeleteFile deleteFile = changes.addedDeleteFiles().iterator().next(); String fromStat = new String( deleteFile.lowerBounds().get(MetadataColumns.DELETE_FILE_PATH.fieldId()).array()); - DataFile dataFile = table.currentSnapshot().addedDataFiles(table.io()).iterator().next(); + DataFile dataFile = changes.addedDataFiles().iterator().next(); assumeThat(fromStat).isEqualTo(dataFile.location()); } } diff --git a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkV2.java b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkV2.java index 590b3f6c984c..71cec082a09e 100644 --- a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkV2.java +++ b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkV2.java @@ -36,6 +36,7 @@ import org.apache.iceberg.ParameterizedTestExtension; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Snapshot; +import org.apache.iceberg.SnapshotChanges; import org.apache.iceberg.SnapshotRef; import org.apache.iceberg.TableProperties; import org.apache.iceberg.data.Record; @@ -227,11 +228,12 @@ public void testDeleteStats() throws Exception { expectedRecords, SnapshotRef.MAIN_BRANCH); - DeleteFile deleteFile = table.currentSnapshot().addedDeleteFiles(table.io()).iterator().next(); + SnapshotChanges changes = SnapshotChanges.builderFor(table).build(); + DeleteFile deleteFile = changes.addedDeleteFiles().iterator().next(); String fromStat = new String( deleteFile.lowerBounds().get(MetadataColumns.DELETE_FILE_PATH.fieldId()).array()); - DataFile dataFile = table.currentSnapshot().addedDataFiles(table.io()).iterator().next(); + DataFile dataFile = changes.addedDataFiles().iterator().next(); assumeThat(fromStat).isEqualTo(dataFile.location()); } diff --git a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/source/TestMetadataTableReadableMetrics.java b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/source/TestMetadataTableReadableMetrics.java index 488969bab045..af3a79ff67b8 100644 --- a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/source/TestMetadataTableReadableMetrics.java +++ b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/source/TestMetadataTableReadableMetrics.java @@ -38,6 +38,7 @@ import org.apache.iceberg.Parameters; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; +import org.apache.iceberg.SnapshotChanges; import org.apache.iceberg.Table; import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.catalog.TableIdentifier; @@ -222,7 +223,8 @@ public void testPrimitiveColumns() throws Exception { // size of the column to increase. For example, with Parquet 1.14.x the // uncompressed size has been added to allow for better allocation of memory upfront. // Therefore, we look the sizes up, rather than hardcoding them - DataFile dataFile = table.currentSnapshot().addedDataFiles(table.io()).iterator().next(); + DataFile dataFile = + SnapshotChanges.builderFor(table).build().addedDataFiles().iterator().next(); Map columnSizeStats = dataFile.columnSizes(); Row binaryCol = diff --git a/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/RewriteUtil.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/RewriteUtil.java index 9b19a50b092d..8a8a2fa194d4 100644 --- a/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/RewriteUtil.java +++ b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/RewriteUtil.java @@ -27,6 +27,7 @@ import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; import org.apache.flink.streaming.util.ProcessFunctionTestHarnesses; import org.apache.iceberg.DataFile; +import org.apache.iceberg.SnapshotChanges; import org.apache.iceberg.SnapshotRef; import org.apache.iceberg.Table; import org.apache.iceberg.expressions.Expressions; @@ -88,6 +89,6 @@ static List executeRewrite( static Set newDataFiles(Table table) { table.refresh(); - return Sets.newHashSet(table.currentSnapshot().addedDataFiles(table.io())); + return Sets.newHashSet(SnapshotChanges.builderFor(table).build().addedDataFiles()); } } diff --git a/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestDataFileRewriteCommitter.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestDataFileRewriteCommitter.java index cdd40eb2a2bf..ded958538ff0 100644 --- a/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestDataFileRewriteCommitter.java +++ b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestDataFileRewriteCommitter.java @@ -31,6 +31,7 @@ import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; import org.apache.iceberg.ContentFile; import org.apache.iceberg.DataFile; +import org.apache.iceberg.SnapshotChanges; import org.apache.iceberg.SnapshotRef; import org.apache.iceberg.Table; import org.apache.iceberg.flink.maintenance.api.Trigger; @@ -262,9 +263,9 @@ private static void assertDataFiles( assertThat(table.currentSnapshot().summary().get(TOTAL_DATA_FILES)) .isEqualTo(String.valueOf(expectedCurrent)); - Set actualAdded = Sets.newHashSet(table.currentSnapshot().addedDataFiles(table.io())); - Set actualRemoved = - Sets.newHashSet(table.currentSnapshot().removedDataFiles(table.io())); + SnapshotChanges changes = SnapshotChanges.builderFor(table).build(); + Set actualAdded = Sets.newHashSet(changes.addedDataFiles()); + Set actualRemoved = Sets.newHashSet(changes.removedDataFiles()); assertThat(actualAdded.stream().map(DataFile::location).collect(Collectors.toSet())) .isEqualTo(expectedAdded.stream().map(DataFile::location).collect(Collectors.toSet())); assertThat(actualRemoved.stream().map(DataFile::location).collect(Collectors.toSet())) diff --git a/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestMonitorSource.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestMonitorSource.java index 9c1ea2f2295a..3dca6c421c76 100644 --- a/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestMonitorSource.java +++ b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestMonitorSource.java @@ -46,6 +46,7 @@ import org.apache.iceberg.DeleteFile; import org.apache.iceberg.FileFormat; import org.apache.iceberg.RewriteFiles; +import org.apache.iceberg.SnapshotChanges; import org.apache.iceberg.Table; import org.apache.iceberg.data.GenericAppenderHelper; import org.apache.iceberg.data.RandomGenericData; @@ -316,7 +317,7 @@ void testSkipReplace() throws IOException { // Create a DataOperations.REPLACE snapshot DataFile dataFile = - table.snapshots().iterator().next().addedDataFiles(table.io()).iterator().next(); + SnapshotChanges.builderFor(table).build().addedDataFiles().iterator().next(); RewriteFiles rewrite = tableLoader.loadTable().newRewrite(); // Replace the file with itself for testing purposes rewrite.deleteFile(dataFile); @@ -328,14 +329,18 @@ void testSkipReplace() throws IOException { } private static long firstFileLength(Table table) { - return table.currentSnapshot().addedDataFiles(table.io()).iterator().next().fileSizeInBytes(); + return SnapshotChanges.builderFor(table) + .build() + .addedDataFiles() + .iterator() + .next() + .fileSizeInBytes(); } private static TableChange tableChangeWithLastSnapshot(Table table, TableChange previous) { - List dataFiles = - Lists.newArrayList(table.currentSnapshot().addedDataFiles(table.io()).iterator()); - List deleteFiles = - Lists.newArrayList(table.currentSnapshot().addedDeleteFiles(table.io()).iterator()); + SnapshotChanges changes = SnapshotChanges.builderFor(table).build(); + List dataFiles = Lists.newArrayList(changes.addedDataFiles().iterator()); + List deleteFiles = Lists.newArrayList(changes.addedDeleteFiles().iterator()); long dataSize = dataFiles.stream().mapToLong(ContentFile::fileSizeInBytes).sum(); long deleteRecordCount = deleteFiles.stream().mapToLong(DeleteFile::recordCount).sum(); diff --git a/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java index 9ef7c707821e..fac1b8226d3c 100644 --- a/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java +++ b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java @@ -35,6 +35,7 @@ import org.apache.iceberg.MetadataColumns; import org.apache.iceberg.ParameterizedTestExtension; import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.SnapshotChanges; import org.apache.iceberg.SnapshotRef; import org.apache.iceberg.TableProperties; import org.apache.iceberg.data.Record; @@ -244,11 +245,12 @@ public void testDeleteStats() throws Exception { expectedRecords, SnapshotRef.MAIN_BRANCH); - DeleteFile deleteFile = table.currentSnapshot().addedDeleteFiles(table.io()).iterator().next(); + SnapshotChanges changes = SnapshotChanges.builderFor(table).build(); + DeleteFile deleteFile = changes.addedDeleteFiles().iterator().next(); String fromStat = new String( deleteFile.lowerBounds().get(MetadataColumns.DELETE_FILE_PATH.fieldId()).array()); - DataFile dataFile = table.currentSnapshot().addedDataFiles(table.io()).iterator().next(); + DataFile dataFile = changes.addedDataFiles().iterator().next(); assumeThat(fromStat).isEqualTo(dataFile.location()); } } diff --git a/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkV2.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkV2.java index 590b3f6c984c..71cec082a09e 100644 --- a/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkV2.java +++ b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkV2.java @@ -36,6 +36,7 @@ import org.apache.iceberg.ParameterizedTestExtension; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Snapshot; +import org.apache.iceberg.SnapshotChanges; import org.apache.iceberg.SnapshotRef; import org.apache.iceberg.TableProperties; import org.apache.iceberg.data.Record; @@ -227,11 +228,12 @@ public void testDeleteStats() throws Exception { expectedRecords, SnapshotRef.MAIN_BRANCH); - DeleteFile deleteFile = table.currentSnapshot().addedDeleteFiles(table.io()).iterator().next(); + SnapshotChanges changes = SnapshotChanges.builderFor(table).build(); + DeleteFile deleteFile = changes.addedDeleteFiles().iterator().next(); String fromStat = new String( deleteFile.lowerBounds().get(MetadataColumns.DELETE_FILE_PATH.fieldId()).array()); - DataFile dataFile = table.currentSnapshot().addedDataFiles(table.io()).iterator().next(); + DataFile dataFile = changes.addedDataFiles().iterator().next(); assumeThat(fromStat).isEqualTo(dataFile.location()); } diff --git a/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/TestMetadataTableReadableMetrics.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/TestMetadataTableReadableMetrics.java index 488969bab045..af3a79ff67b8 100644 --- a/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/TestMetadataTableReadableMetrics.java +++ b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/TestMetadataTableReadableMetrics.java @@ -38,6 +38,7 @@ import org.apache.iceberg.Parameters; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; +import org.apache.iceberg.SnapshotChanges; import org.apache.iceberg.Table; import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.catalog.TableIdentifier; @@ -222,7 +223,8 @@ public void testPrimitiveColumns() throws Exception { // size of the column to increase. For example, with Parquet 1.14.x the // uncompressed size has been added to allow for better allocation of memory upfront. // Therefore, we look the sizes up, rather than hardcoding them - DataFile dataFile = table.currentSnapshot().addedDataFiles(table.io()).iterator().next(); + DataFile dataFile = + SnapshotChanges.builderFor(table).build().addedDataFiles().iterator().next(); Map columnSizeStats = dataFile.columnSizes(); Row binaryCol = diff --git a/kafka-connect/kafka-connect-runtime/src/integration/java/org/apache/iceberg/connect/IntegrationTestBase.java b/kafka-connect/kafka-connect-runtime/src/integration/java/org/apache/iceberg/connect/IntegrationTestBase.java index 8b5b1ddea34b..8c011ef537df 100644 --- a/kafka-connect/kafka-connect-runtime/src/integration/java/org/apache/iceberg/connect/IntegrationTestBase.java +++ b/kafka-connect/kafka-connect-runtime/src/integration/java/org/apache/iceberg/connect/IntegrationTestBase.java @@ -31,6 +31,7 @@ import org.apache.iceberg.DataFile; import org.apache.iceberg.DeleteFile; import org.apache.iceberg.Snapshot; +import org.apache.iceberg.SnapshotChanges; import org.apache.iceberg.Table; import org.apache.iceberg.catalog.Catalog; import org.apache.iceberg.catalog.Namespace; @@ -135,12 +136,20 @@ public boolean matches(String str) { protected List dataFiles(TableIdentifier tableIdentifier, String branch) { Table table = catalog.loadTable(tableIdentifier); - return Lists.newArrayList(latestSnapshot(table, branch).addedDataFiles(table.io())); + return Lists.newArrayList( + SnapshotChanges.builderFor(table) + .snapshot(latestSnapshot(table, branch)) + .build() + .addedDataFiles()); } protected List deleteFiles(TableIdentifier tableIdentifier, String branch) { Table table = catalog.loadTable(tableIdentifier); - return Lists.newArrayList(latestSnapshot(table, branch).addedDeleteFiles(table.io())); + return Lists.newArrayList( + SnapshotChanges.builderFor(table) + .snapshot(latestSnapshot(table, branch)) + .build() + .addedDeleteFiles()); } private Snapshot latestSnapshot(Table table, String branch) { diff --git a/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/channel/TestCoordinator.java b/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/channel/TestCoordinator.java index 05526eab5c68..ed370fcdad35 100644 --- a/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/channel/TestCoordinator.java +++ b/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/channel/TestCoordinator.java @@ -32,6 +32,7 @@ import org.apache.iceberg.FileFormat; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Snapshot; +import org.apache.iceberg.SnapshotChanges; import org.apache.iceberg.Table; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.connect.events.AvroUtil; @@ -72,8 +73,9 @@ public void testCommitAppend() { Snapshot snapshot = snapshots.get(0); assertThat(snapshot.operation()).isEqualTo(DataOperations.APPEND); - assertThat(snapshot.addedDataFiles(table.io())).hasSize(1); - assertThat(snapshot.addedDeleteFiles(table.io())).isEmpty(); + SnapshotChanges changes = SnapshotChanges.builderFor(table).snapshot(snapshot).build(); + assertThat(changes.addedDataFiles()).hasSize(1); + assertThat(changes.addedDeleteFiles()).isEmpty(); assertThat(snapshot.summary()) .containsEntry(COMMIT_ID_SNAPSHOT_PROP, commitId.toString()) @@ -99,8 +101,9 @@ public void testCommitDelta() { Snapshot snapshot = snapshots.get(0); assertThat(snapshot.operation()).isEqualTo(DataOperations.OVERWRITE); - assertThat(snapshot.addedDataFiles(table.io())).hasSize(1); - assertThat(snapshot.addedDeleteFiles(table.io())).hasSize(1); + SnapshotChanges changes = SnapshotChanges.builderFor(table).snapshot(snapshot).build(); + assertThat(changes.addedDataFiles()).hasSize(1); + assertThat(changes.addedDeleteFiles()).hasSize(1); assertThat(snapshot.summary()) .containsEntry(COMMIT_ID_SNAPSHOT_PROP, commitId.toString()) diff --git a/spark/v3.4/spark-extensions/src/jmh/java/org/apache/iceberg/spark/PlanningBenchmark.java b/spark/v3.4/spark-extensions/src/jmh/java/org/apache/iceberg/spark/PlanningBenchmark.java index 2b189c242867..05aa9602a323 100644 --- a/spark/v3.4/spark-extensions/src/jmh/java/org/apache/iceberg/spark/PlanningBenchmark.java +++ b/spark/v3.4/spark-extensions/src/jmh/java/org/apache/iceberg/spark/PlanningBenchmark.java @@ -39,6 +39,7 @@ import org.apache.iceberg.RowLevelOperationMode; import org.apache.iceberg.ScanTask; import org.apache.iceberg.Schema; +import org.apache.iceberg.SnapshotChanges; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; import org.apache.iceberg.expressions.Expression; @@ -222,14 +223,14 @@ private void dropTable() { private DataFile loadAddedDataFile() { table.refresh(); - Iterable dataFiles = table.currentSnapshot().addedDataFiles(table.io()); + Iterable dataFiles = SnapshotChanges.builderFor(table).build().addedDataFiles(); return Iterables.getOnlyElement(dataFiles); } private DeleteFile loadAddedDeleteFile() { table.refresh(); - Iterable deleteFiles = table.currentSnapshot().addedDeleteFiles(table.io()); + Iterable deleteFiles = SnapshotChanges.builderFor(table).build().addedDeleteFiles(); return Iterables.getOnlyElement(deleteFiles); } diff --git a/spark/v3.4/spark-extensions/src/jmh/java/org/apache/iceberg/spark/TaskGroupPlanningBenchmark.java b/spark/v3.4/spark-extensions/src/jmh/java/org/apache/iceberg/spark/TaskGroupPlanningBenchmark.java index 42b367af521d..2b85a8e385ec 100644 --- a/spark/v3.4/spark-extensions/src/jmh/java/org/apache/iceberg/spark/TaskGroupPlanningBenchmark.java +++ b/spark/v3.4/spark-extensions/src/jmh/java/org/apache/iceberg/spark/TaskGroupPlanningBenchmark.java @@ -39,6 +39,7 @@ import org.apache.iceberg.ScanTask; import org.apache.iceberg.ScanTaskGroup; import org.apache.iceberg.Schema; +import org.apache.iceberg.SnapshotChanges; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; import org.apache.iceberg.io.CloseableIterable; @@ -161,14 +162,15 @@ private void loadFileTasks() { private DataFile loadAddedDataFile() { table.refresh(); - Iterable addedDataFiles = table.currentSnapshot().addedDataFiles(table.io()); + Iterable addedDataFiles = SnapshotChanges.builderFor(table).build().addedDataFiles(); return Iterables.getOnlyElement(addedDataFiles); } private DeleteFile loadAddedDeleteFile() { table.refresh(); - Iterable addedDeleteFiles = table.currentSnapshot().addedDeleteFiles(table.io()); + Iterable addedDeleteFiles = + SnapshotChanges.builderFor(table).build().addedDeleteFiles(); return Iterables.getOnlyElement(addedDeleteFiles); } diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestBranchDDL.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestBranchDDL.java index 95a90dd42e9a..54b3c4b11b98 100644 --- a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestBranchDDL.java +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestBranchDDL.java @@ -114,8 +114,8 @@ public void testCreateBranchOnEmptyTable() { SnapshotChanges.builderFor(table).snapshot(snapshot).build(); assertThat(changes.addedDataFiles()).isEmpty(); assertThat(changes.removedDataFiles()).isEmpty(); - assertThat(snapshot.addedDeleteFiles(table.io())).isEmpty(); - assertThat(snapshot.removedDeleteFiles(table.io())).isEmpty(); + assertThat(changes.addedDeleteFiles()).isEmpty(); + assertThat(changes.removedDeleteFiles()).isEmpty(); }); } @@ -377,8 +377,8 @@ public void testCreateOrReplaceBranchOnEmptyTable() { SnapshotChanges.builderFor(table).snapshot(snapshot).build(); assertThat(changes.addedDataFiles()).isEmpty(); assertThat(changes.removedDataFiles()).isEmpty(); - assertThat(snapshot.addedDeleteFiles(table.io())).isEmpty(); - assertThat(snapshot.removedDeleteFiles(table.io())).isEmpty(); + assertThat(changes.addedDeleteFiles()).isEmpty(); + assertThat(changes.removedDeleteFiles()).isEmpty(); }); } diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeOnReadDelete.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeOnReadDelete.java index 0d9be093c96c..c78bb14fc72b 100644 --- a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeOnReadDelete.java +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeOnReadDelete.java @@ -36,6 +36,7 @@ import org.apache.iceberg.RowDelta; import org.apache.iceberg.RowLevelOperationMode; import org.apache.iceberg.Snapshot; +import org.apache.iceberg.SnapshotChanges; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; import org.apache.iceberg.deletes.DeleteGranularity; @@ -140,7 +141,8 @@ public void testPositionDeletesAreMaintainedDuringDelete() throws NoSuchTableExc Table table = validationCatalog.loadTable(tableIdent); Snapshot latest = SnapshotUtil.latestSnapshot(table, branch); - assertThat(latest.removedDeleteFiles(table.io())).hasSize(1); + assertThat(SnapshotChanges.builderFor(table).snapshot(latest).build().removedDeleteFiles()) + .hasSize(1); assertEquals( "Should have expected rows", ImmutableList.of(row(1, "b"), row(2, "e")), @@ -181,7 +183,8 @@ public void testUnpartitionedPositionDeletesAreMaintainedDuringDelete() Table table = validationCatalog.loadTable(tableIdent); Snapshot latest = SnapshotUtil.latestSnapshot(table, branch); - assertThat(latest.removedDeleteFiles(table.io())).hasSize(1); + assertThat(SnapshotChanges.builderFor(table).snapshot(latest).build().removedDeleteFiles()) + .hasSize(1); assertEquals( "Should have expected rows", ImmutableList.of(row(1, "b"), row(2, "e")), diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeOnReadUpdate.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeOnReadUpdate.java index 7fa9ffcd5f89..a876e6d66b93 100644 --- a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeOnReadUpdate.java +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeOnReadUpdate.java @@ -30,6 +30,7 @@ import org.apache.iceberg.ParameterizedTestExtension; import org.apache.iceberg.RowLevelOperationMode; import org.apache.iceberg.Snapshot; +import org.apache.iceberg.SnapshotChanges; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; import org.apache.iceberg.deletes.DeleteGranularity; @@ -74,7 +75,12 @@ public void testPositionDeletesAreMaintainedDuringUpdate() { String expectedDeleteFilesCount = "2"; validateMergeOnRead(currentSnapshot, "2", expectedDeleteFilesCount, "2"); - assertThat(currentSnapshot.removedDeleteFiles(table.io())).hasSize(2); + assertThat( + SnapshotChanges.builderFor(table) + .snapshot(currentSnapshot) + .build() + .removedDeleteFiles()) + .hasSize(2); assertEquals( "Should have expected rows", ImmutableList.of( @@ -122,7 +128,12 @@ public void testUnpartitionedPositionDeletesAreMaintainedDuringUpdate() { expectedDeleteFilesCount = "2"; validateMergeOnRead(currentSnapshot, "1", expectedDeleteFilesCount, "1"); - assertThat(currentSnapshot.removedDeleteFiles(table.io())).hasSize(2); + assertThat( + SnapshotChanges.builderFor(table) + .snapshot(currentSnapshot) + .build() + .removedDeleteFiles()) + .hasSize(2); assertEquals( "Should have expected rows", ImmutableList.of( diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteTablePathProcedure.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteTablePathProcedure.java index d1db48b0bf9e..ceb3077c5670 100644 --- a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteTablePathProcedure.java +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteTablePathProcedure.java @@ -30,6 +30,7 @@ import org.apache.iceberg.HasTableOperations; import org.apache.iceberg.ParameterizedTestExtension; import org.apache.iceberg.RewriteTablePathUtil; +import org.apache.iceberg.SnapshotChanges; import org.apache.iceberg.Table; import org.apache.iceberg.TableUtil; import org.apache.iceberg.data.FileHelpers; @@ -211,7 +212,12 @@ public void testRewriteTablePathWithManifestAndDeleteCounts() throws IOException List> rowsToDelete = Lists.newArrayList( Pair.of( - table.currentSnapshot().addedDataFiles(table.io()).iterator().next().location(), + SnapshotChanges.builderFor(table) + .build() + .addedDataFiles() + .iterator() + .next() + .location(), 0L)); File file = new File(removePrefix(table.location()) + "/data/deletes.parquet"); diff --git a/spark/v3.4/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetMultiDeleteFileBenchmark.java b/spark/v3.4/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetMultiDeleteFileBenchmark.java index 1afdae1bb8e8..b61636d9b93e 100644 --- a/spark/v3.4/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetMultiDeleteFileBenchmark.java +++ b/spark/v3.4/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetMultiDeleteFileBenchmark.java @@ -21,6 +21,7 @@ import java.io.IOException; import org.apache.iceberg.DataFile; import org.apache.iceberg.FileFormat; +import org.apache.iceberg.SnapshotChanges; import org.apache.iceberg.spark.source.IcebergSourceDeleteBenchmark; import org.openjdk.jmh.annotations.Param; @@ -44,7 +45,7 @@ protected void appendData() throws IOException { writeData(fileNum); table().refresh(); - for (DataFile file : table().currentSnapshot().addedDataFiles(table().io())) { + for (DataFile file : SnapshotChanges.builderFor(table()).build().addedDataFiles()) { writePosDeletes(file.location(), NUM_ROWS, 0.25, numDeleteFile); } } diff --git a/spark/v3.4/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetPosDeleteBenchmark.java b/spark/v3.4/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetPosDeleteBenchmark.java index e948fc4b7b8f..82b1c850b832 100644 --- a/spark/v3.4/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetPosDeleteBenchmark.java +++ b/spark/v3.4/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetPosDeleteBenchmark.java @@ -21,6 +21,7 @@ import java.io.IOException; import org.apache.iceberg.DataFile; import org.apache.iceberg.FileFormat; +import org.apache.iceberg.SnapshotChanges; import org.apache.iceberg.spark.source.IcebergSourceDeleteBenchmark; import org.openjdk.jmh.annotations.Param; @@ -46,7 +47,7 @@ protected void appendData() throws IOException { if (percentDeleteRow > 0) { // add pos-deletes table().refresh(); - for (DataFile file : table().currentSnapshot().addedDataFiles(table().io())) { + for (DataFile file : SnapshotChanges.builderFor(table()).build().addedDataFiles()) { writePosDeletes(file.location(), NUM_ROWS, percentDeleteRow); } } diff --git a/spark/v3.4/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetWithUnrelatedDeleteBenchmark.java b/spark/v3.4/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetWithUnrelatedDeleteBenchmark.java index f545759b8f40..8d45e9ecaf56 100644 --- a/spark/v3.4/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetWithUnrelatedDeleteBenchmark.java +++ b/spark/v3.4/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetWithUnrelatedDeleteBenchmark.java @@ -21,6 +21,7 @@ import java.io.IOException; import org.apache.iceberg.DataFile; import org.apache.iceberg.FileFormat; +import org.apache.iceberg.SnapshotChanges; import org.apache.iceberg.spark.source.IcebergSourceDeleteBenchmark; import org.openjdk.jmh.annotations.Param; @@ -46,7 +47,7 @@ protected void appendData() throws IOException { writeData(fileNum); table().refresh(); - for (DataFile file : table().currentSnapshot().addedDataFiles(table().io())) { + for (DataFile file : SnapshotChanges.builderFor(table()).build().addedDataFiles()) { writePosDeletesWithNoise( file.location(), NUM_ROWS, diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestExpireSnapshotsAction.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestExpireSnapshotsAction.java index afbe747093ea..d6416e1dd262 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestExpireSnapshotsAction.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestExpireSnapshotsAction.java @@ -47,6 +47,7 @@ import org.apache.iceberg.ReachableFileUtil; import org.apache.iceberg.Schema; import org.apache.iceberg.Snapshot; +import org.apache.iceberg.SnapshotChanges; import org.apache.iceberg.Table; import org.apache.iceberg.TableMetadata; import org.apache.iceberg.TableProperties; @@ -628,8 +629,10 @@ public void testWithExpiringDanglingStageCommit() { expectedDeletes.add(snapshotA.manifestListLocation()); // Files should be deleted of dangling staged snapshot - snapshotB - .addedDataFiles(table.io()) + SnapshotChanges.builderFor(table) + .snapshot(snapshotB) + .build() + .addedDataFiles() .forEach( i -> { expectedDeletes.add(i.location()); @@ -699,7 +702,10 @@ public void testWithCherryPickTableSnapshot() { Lists.newArrayList(snapshotB, snapshotC, snapshotD) .forEach( i -> { - i.addedDataFiles(table.io()) + SnapshotChanges.builderFor(table) + .snapshot(i) + .build() + .addedDataFiles() .forEach( item -> { assertThat(deletedFiles).doesNotContain(item.location()); @@ -748,7 +754,10 @@ public void testWithExpiringStagedThenCherrypick() { Lists.newArrayList(snapshotB) .forEach( i -> { - i.addedDataFiles(table.io()) + SnapshotChanges.builderFor(table) + .snapshot(i) + .build() + .addedDataFiles() .forEach( item -> { assertThat(deletedFiles).doesNotContain(item.location()); @@ -768,7 +777,10 @@ public void testWithExpiringStagedThenCherrypick() { Lists.newArrayList(snapshotB, snapshotD) .forEach( i -> { - i.addedDataFiles(table.io()) + SnapshotChanges.builderFor(table) + .snapshot(i) + .build() + .addedDataFiles() .forEach( item -> { assertThat(deletedFiles).doesNotContain(item.location()); diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteDataFilesAction.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteDataFilesAction.java index da07ae242827..8239d62950b6 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteDataFilesAction.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteDataFilesAction.java @@ -1719,7 +1719,7 @@ public void testAutoSortShuffleOutput() { assertThat(result.rewriteResults()).as("Should have 1 fileGroups").hasSize(1); assertThat(result.rewrittenBytesCount()).isEqualTo(dataSizeBefore); assertThat(result.rewriteResults()).as("Should have 1 fileGroups").hasSize(1); - assertThat(table.currentSnapshot().addedDataFiles(table.io())) + assertThat(SnapshotChanges.builderFor(table).build().addedDataFiles()) .as("Should have written 40+ files") .hasSizeGreaterThanOrEqualTo(40); @@ -1798,7 +1798,7 @@ public void testZOrderSort() { assertThat(result.rewriteResults()).as("Should have 1 fileGroups").hasSize(1); assertThat(result.rewrittenBytesCount()).isEqualTo(dataSizeBefore); - assertThat(table.currentSnapshot().addedDataFiles(table.io())) + assertThat(SnapshotChanges.builderFor(table).build().addedDataFiles()) .as("Should have written 40+ files") .hasSizeGreaterThanOrEqualTo(40); @@ -1863,7 +1863,7 @@ public void testZOrderAllTypesSort() { assertThat(result.rewriteResults()).as("Should have 1 fileGroups").hasSize(1); assertThat(result.rewrittenBytesCount()).isEqualTo(dataSizeBefore); - assertThat(table.currentSnapshot().addedDataFiles(table.io())) + assertThat(SnapshotChanges.builderFor(table).build().addedDataFiles()) .as("Should have written 1 file") .hasSize(1); diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteManifestsAction.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteManifestsAction.java index 4f95416878e2..ccef1916dae7 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteManifestsAction.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteManifestsAction.java @@ -58,6 +58,7 @@ import org.apache.iceberg.RowDelta; import org.apache.iceberg.Schema; import org.apache.iceberg.Snapshot; +import org.apache.iceberg.SnapshotChanges; import org.apache.iceberg.StructLike; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; @@ -728,7 +729,9 @@ public void testRewriteSmallManifestsNonPartitionedV2Table() { table.refresh(); Snapshot snapshot1 = table.currentSnapshot(); - DataFile file1 = Iterables.getOnlyElement(snapshot1.addedDataFiles(table.io())); + DataFile file1 = + Iterables.getOnlyElement( + SnapshotChanges.builderFor(table).snapshot(snapshot1).build().addedDataFiles()); List records2 = Lists.newArrayList(new ThreeColumnRecord(2, "CCCC", "CCCC")); writeRecords(records2); @@ -736,7 +739,9 @@ public void testRewriteSmallManifestsNonPartitionedV2Table() { table.refresh(); Snapshot snapshot2 = table.currentSnapshot(); - DataFile file2 = Iterables.getOnlyElement(snapshot2.addedDataFiles(table.io())); + DataFile file2 = + Iterables.getOnlyElement( + SnapshotChanges.builderFor(table).snapshot(snapshot2).build().addedDataFiles()); List manifests = table.currentSnapshot().allManifests(table.io()); assertThat(manifests).as("Should have 2 manifests before rewrite").hasSize(2); 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 6044368a46c3..dae721b1d73d 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 @@ -48,6 +48,7 @@ import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; import org.apache.iceberg.Snapshot; +import org.apache.iceberg.SnapshotChanges; import org.apache.iceberg.StaticTableOperations; import org.apache.iceberg.Table; import org.apache.iceberg.TableMetadata; @@ -435,9 +436,9 @@ private void runPositionDeletesTest(String fileFormat) throws Exception { List> deletes = Lists.newArrayList( Pair.of( - tableWithPosDeletes - .currentSnapshot() - .addedDataFiles(tableWithPosDeletes.io()) + SnapshotChanges.builderFor(tableWithPosDeletes) + .build() + .addedDataFiles() .iterator() .next() .location(), @@ -482,7 +483,7 @@ private void runPositionDeletesTest(String fileFormat) throws Exception { @TestTemplate public void testPositionDeleteWithRow() throws Exception { String dataFileLocation = - table.currentSnapshot().addedDataFiles(table.io()).iterator().next().location(); + SnapshotChanges.builderFor(table).build().addedDataFiles().iterator().next().location(); List> deletes = Lists.newArrayList(); OutputFile deleteFile = table @@ -532,7 +533,15 @@ public void testPositionDeletesAcrossFiles() throws Exception { .isEqualTo(2); Stream allFiles = StreamSupport.stream(table.snapshots().spliterator(), false) - .flatMap(s -> StreamSupport.stream(s.addedDataFiles(table.io()).spliterator(), false)); + .flatMap( + s -> + StreamSupport.stream( + SnapshotChanges.builderFor(table) + .snapshot(s) + .build() + .addedDataFiles() + .spliterator(), + false)); List> deletes = allFiles.map(f -> Pair.of((CharSequence) f.location(), 0L)).collect(Collectors.toList()); @@ -779,7 +788,10 @@ public void testRewritePathWithNonLiveEntry() throws Exception { Snapshot oldest = SnapshotUtil.oldestAncestor(tableWith3Snaps); String oldestDataFilePath = Iterables.getOnlyElement( - tableWith3Snaps.snapshot(oldest.snapshotId()).addedDataFiles(tableWith3Snaps.io())) + SnapshotChanges.builderFor(tableWith3Snaps) + .snapshot(tableWith3Snaps.snapshot(oldest.snapshotId())) + .build() + .addedDataFiles()) .location(); String deletedDataFilePathInTargetLocation = String.format("%sdata/%s", targetTableLocation(), fileName(oldestDataFilePath)); @@ -1303,27 +1315,14 @@ public void testNestedDirectoryStructurePreservation() throws Exception { // Create position delete files with same names in different nested directories // This simulates the scenario tested in // TestRewriteTablePathUtil.testStagingPathPreservesDirectoryStructure + SnapshotChanges sourceChanges = SnapshotChanges.builderFor(sourceTable).build(); List> deletes1 = Lists.newArrayList( - Pair.of( - sourceTable - .currentSnapshot() - .addedDataFiles(sourceTable.io()) - .iterator() - .next() - .location(), - 0L)); + Pair.of(sourceChanges.addedDataFiles().iterator().next().location(), 0L)); List> deletes2 = Lists.newArrayList( - Pair.of( - sourceTable - .currentSnapshot() - .addedDataFiles(sourceTable.io()) - .iterator() - .next() - .location(), - 0L)); + Pair.of(sourceChanges.addedDataFiles().iterator().next().location(), 0L)); // Create delete files with same name in different nested paths (hash1/ and hash2/) File file1 = diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/DataFrameWriteTestBase.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/DataFrameWriteTestBase.java index 05dec42a031a..80cc3556b3c4 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/DataFrameWriteTestBase.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/DataFrameWriteTestBase.java @@ -32,6 +32,7 @@ import org.apache.iceberg.Files; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; +import org.apache.iceberg.SnapshotChanges; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; import org.apache.iceberg.Tables; @@ -125,9 +126,9 @@ public void testAlternateLocation() throws IOException { writeRecords(table, RandomData.generateList(table.schema(), 100, 87112L)); - table - .currentSnapshot() - .addedDataFiles(table.io()) + SnapshotChanges.builderFor(table) + .build() + .addedDataFiles() .forEach( dataFile -> assertThat(dataFile.location()) diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestDataSourceOptions.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestDataSourceOptions.java index 99d3f38ee7eb..a7e8d76969b6 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestDataSourceOptions.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestDataSourceOptions.java @@ -35,6 +35,7 @@ import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; import org.apache.iceberg.Snapshot; +import org.apache.iceberg.SnapshotChanges; import org.apache.iceberg.SnapshotSummary; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; @@ -206,7 +207,7 @@ public void testSplitOptionsOverridesTableProperties() throws IOException { .save(tableLocation); List files = - Lists.newArrayList(icebergTable.currentSnapshot().addedDataFiles(icebergTable.io())); + Lists.newArrayList(SnapshotChanges.builderFor(icebergTable).build().addedDataFiles()); assertThat(files).as("Should have written 1 file").hasSize(1); long fileSize = files.get(0).fileSizeInBytes(); diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java index 53c9ac6b2257..cc78b89cba7e 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java @@ -51,6 +51,7 @@ import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; import org.apache.iceberg.Snapshot; +import org.apache.iceberg.SnapshotChanges; import org.apache.iceberg.StructLike; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; @@ -257,7 +258,7 @@ public void testEntriesTableDataFilePrune() { .save(loadLocation(tableIdentifier)); table.refresh(); - DataFile file = table.currentSnapshot().addedDataFiles(table.io()).iterator().next(); + DataFile file = SnapshotChanges.builderFor(table).build().addedDataFiles().iterator().next(); List singleActual = rowsToJava( @@ -290,7 +291,7 @@ public void testEntriesTableDataFilePruneMulti() { .save(loadLocation(tableIdentifier)); table.refresh(); - DataFile file = table.currentSnapshot().addedDataFiles(table.io()).iterator().next(); + DataFile file = SnapshotChanges.builderFor(table).build().addedDataFiles().iterator().next(); List multiActual = rowsToJava( @@ -328,7 +329,7 @@ public void testFilesSelectMap() { .save(loadLocation(tableIdentifier)); table.refresh(); - DataFile file = table.currentSnapshot().addedDataFiles(table.io()).iterator().next(); + DataFile file = SnapshotChanges.builderFor(table).build().addedDataFiles().iterator().next(); List multiActual = rowsToJava( @@ -634,7 +635,7 @@ public void testFilesUnpartitionedTable() throws Exception { table.refresh(); DataFile toDelete = - Iterables.getOnlyElement(table.currentSnapshot().addedDataFiles(table.io())); + Iterables.getOnlyElement(SnapshotChanges.builderFor(table).build().addedDataFiles()); // add a second file df2.select("id", "data") @@ -1322,7 +1323,7 @@ public void testUnpartitionedPartitionsTable() { .set("file_count", 1) .set( "total_data_file_size_in_bytes", - totalSizeInBytes(table.currentSnapshot().addedDataFiles(table.io()))) + totalSizeInBytes(SnapshotChanges.builderFor(table).build().addedDataFiles())) .set("position_delete_record_count", 0L) .set("position_delete_file_count", 0) .set("equality_delete_record_count", 0L) @@ -1391,7 +1392,11 @@ public void testPartitionsTable() { .set("file_count", 1) .set( "total_data_file_size_in_bytes", - totalSizeInBytes(table.snapshot(firstCommitId).addedDataFiles(table.io()))) + totalSizeInBytes( + SnapshotChanges.builderFor(table) + .snapshot(table.snapshot(firstCommitId)) + .build() + .addedDataFiles())) .set("position_delete_record_count", 0L) .set("position_delete_file_count", 0) .set("equality_delete_record_count", 0L) @@ -1407,7 +1412,11 @@ public void testPartitionsTable() { .set("file_count", 1) .set( "total_data_file_size_in_bytes", - totalSizeInBytes(table.snapshot(secondCommitId).addedDataFiles(table.io()))) + totalSizeInBytes( + SnapshotChanges.builderFor(table) + .snapshot(table.snapshot(secondCommitId)) + .build() + .addedDataFiles())) .set("position_delete_record_count", 0L) .set("position_delete_file_count", 0) .set("equality_delete_record_count", 0L) @@ -1671,7 +1680,11 @@ public void testPartitionsTableDeleteStats() { .set("file_count", 1) .set( "total_data_file_size_in_bytes", - totalSizeInBytes(table.snapshot(firstCommitId).addedDataFiles(table.io()))) + totalSizeInBytes( + SnapshotChanges.builderFor(table) + .snapshot(table.snapshot(firstCommitId)) + .build() + .addedDataFiles())) .set("position_delete_record_count", 0L) .set("position_delete_file_count", 0) .set("equality_delete_record_count", 0L) @@ -1687,7 +1700,11 @@ public void testPartitionsTableDeleteStats() { .set("file_count", 1) .set( "total_data_file_size_in_bytes", - totalSizeInBytes(table.snapshot(firstCommitId).addedDataFiles(table.io()))) + totalSizeInBytes( + SnapshotChanges.builderFor(table) + .snapshot(table.snapshot(firstCommitId)) + .build() + .addedDataFiles())) .set("position_delete_record_count", 2L) // should be incremented now .set("position_delete_file_count", 2) // should be incremented now .set("equality_delete_record_count", 0L) @@ -2415,7 +2432,7 @@ private DeleteFile writePosDeleteFile(Table table) { private DeleteFile writePosDeleteFile(Table table, long pos) { DataFile dataFile = - Iterables.getFirst(table.currentSnapshot().addedDataFiles(table.io()), null); + Iterables.getFirst(SnapshotChanges.builderFor(table).build().addedDataFiles(), null); PartitionSpec dataFileSpec = table.specs().get(dataFile.specId()); StructLike dataFilePartition = dataFile.partition(); diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestMetadataTableReadableMetrics.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestMetadataTableReadableMetrics.java index c21ccd0100db..f74446b2f416 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestMetadataTableReadableMetrics.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestMetadataTableReadableMetrics.java @@ -33,6 +33,7 @@ import org.apache.iceberg.Parameters; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; +import org.apache.iceberg.SnapshotChanges; import org.apache.iceberg.Table; import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.catalog.TableIdentifier; @@ -200,7 +201,8 @@ private GenericRecord createNestedRecord(Long longCol, Double doubleCol) { @TestTemplate public void testPrimitiveColumns() throws Exception { Table table = createPrimitiveTable(); - DataFile dataFile = table.currentSnapshot().addedDataFiles(table.io()).iterator().next(); + DataFile dataFile = + SnapshotChanges.builderFor(table).build().addedDataFiles().iterator().next(); Map columnSizeStats = dataFile.columnSizes(); Object[] binaryCol = diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestRefreshTable.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestRefreshTable.java index 426c68447724..8b1edf9c7075 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestRefreshTable.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestRefreshTable.java @@ -22,6 +22,7 @@ import java.util.Set; import org.apache.iceberg.DataFile; import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.SnapshotChanges; import org.apache.iceberg.Table; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.spark.CatalogTestBase; @@ -65,7 +66,7 @@ public void testRefreshCommand() { // Modify table outside of spark, it should be cached so Spark should see the same value after // mutation Table table = validationCatalog.loadTable(tableIdent); - DataFile file = table.currentSnapshot().addedDataFiles(table.io()).iterator().next(); + DataFile file = SnapshotChanges.builderFor(table).build().addedDataFiles().iterator().next(); table.newDelete().deleteFile(file).commit(); List cachedActual = sql("SELECT * FROM %s", tableName); diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestBranchDDL.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestBranchDDL.java index 75a8cf9203f1..26dde2059966 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestBranchDDL.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestBranchDDL.java @@ -114,8 +114,8 @@ public void testCreateBranchOnEmptyTable() { SnapshotChanges.builderFor(table).snapshot(snapshot).build(); assertThat(changes.addedDataFiles()).isEmpty(); assertThat(changes.removedDataFiles()).isEmpty(); - assertThat(snapshot.addedDeleteFiles(table.io())).isEmpty(); - assertThat(snapshot.removedDeleteFiles(table.io())).isEmpty(); + assertThat(changes.addedDeleteFiles()).isEmpty(); + assertThat(changes.removedDeleteFiles()).isEmpty(); }); } @@ -377,8 +377,8 @@ public void testCreateOrReplaceBranchOnEmptyTable() { SnapshotChanges.builderFor(table).snapshot(snapshot).build(); assertThat(changes.addedDataFiles()).isEmpty(); assertThat(changes.removedDataFiles()).isEmpty(); - assertThat(snapshot.addedDeleteFiles(table.io())).isEmpty(); - assertThat(snapshot.removedDeleteFiles(table.io())).isEmpty(); + assertThat(changes.addedDeleteFiles()).isEmpty(); + assertThat(changes.removedDeleteFiles()).isEmpty(); }); } diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeOnReadDelete.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeOnReadDelete.java index bf9c53f82dbd..11643c495400 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeOnReadDelete.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeOnReadDelete.java @@ -36,6 +36,7 @@ import org.apache.iceberg.RowDelta; import org.apache.iceberg.RowLevelOperationMode; import org.apache.iceberg.Snapshot; +import org.apache.iceberg.SnapshotChanges; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; import org.apache.iceberg.deletes.DeleteGranularity; @@ -140,7 +141,8 @@ public void testPositionDeletesAreMaintainedDuringDelete() throws NoSuchTableExc Table table = validationCatalog.loadTable(tableIdent); Snapshot latest = SnapshotUtil.latestSnapshot(table, branch); - assertThat(latest.removedDeleteFiles(table.io())).hasSize(1); + assertThat(SnapshotChanges.builderFor(table).snapshot(latest).build().removedDeleteFiles()) + .hasSize(1); assertEquals( "Should have expected rows", ImmutableList.of(row(1, "b"), row(2, "e")), @@ -181,7 +183,8 @@ public void testUnpartitionedPositionDeletesAreMaintainedDuringDelete() Table table = validationCatalog.loadTable(tableIdent); Snapshot latest = SnapshotUtil.latestSnapshot(table, branch); - assertThat(latest.removedDeleteFiles(table.io())).hasSize(1); + assertThat(SnapshotChanges.builderFor(table).snapshot(latest).build().removedDeleteFiles()) + .hasSize(1); assertEquals( "Should have expected rows", ImmutableList.of(row(1, "b"), row(2, "e")), diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeOnReadUpdate.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeOnReadUpdate.java index 1bec21b9b68d..2398bc45b19b 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeOnReadUpdate.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeOnReadUpdate.java @@ -30,6 +30,7 @@ import org.apache.iceberg.ParameterizedTestExtension; import org.apache.iceberg.RowLevelOperationMode; import org.apache.iceberg.Snapshot; +import org.apache.iceberg.SnapshotChanges; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; import org.apache.iceberg.deletes.DeleteGranularity; @@ -75,7 +76,12 @@ public void testPositionDeletesAreMaintainedDuringUpdate() { String expectedDeleteFilesCount = "2"; validateMergeOnRead(currentSnapshot, "2", expectedDeleteFilesCount, "2"); - assertThat(currentSnapshot.removedDeleteFiles(table.io())).hasSize(2); + assertThat( + SnapshotChanges.builderFor(table) + .snapshot(currentSnapshot) + .build() + .removedDeleteFiles()) + .hasSize(2); assertEquals( "Should have expected rows", ImmutableList.of( @@ -123,7 +129,12 @@ public void testUnpartitionedPositionDeletesAreMaintainedDuringUpdate() { expectedDeleteFilesCount = "2"; validateMergeOnRead(currentSnapshot, "1", expectedDeleteFilesCount, "1"); - assertThat(currentSnapshot.removedDeleteFiles(table.io())).hasSize(2); + assertThat( + SnapshotChanges.builderFor(table) + .snapshot(currentSnapshot) + .build() + .removedDeleteFiles()) + .hasSize(2); assertEquals( "Should have expected rows", ImmutableList.of( diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteTablePathProcedure.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteTablePathProcedure.java index d1db48b0bf9e..ceb3077c5670 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteTablePathProcedure.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteTablePathProcedure.java @@ -30,6 +30,7 @@ import org.apache.iceberg.HasTableOperations; import org.apache.iceberg.ParameterizedTestExtension; import org.apache.iceberg.RewriteTablePathUtil; +import org.apache.iceberg.SnapshotChanges; import org.apache.iceberg.Table; import org.apache.iceberg.TableUtil; import org.apache.iceberg.data.FileHelpers; @@ -211,7 +212,12 @@ public void testRewriteTablePathWithManifestAndDeleteCounts() throws IOException List> rowsToDelete = Lists.newArrayList( Pair.of( - table.currentSnapshot().addedDataFiles(table.io()).iterator().next().location(), + SnapshotChanges.builderFor(table) + .build() + .addedDataFiles() + .iterator() + .next() + .location(), 0L)); File file = new File(removePrefix(table.location()) + "/data/deletes.parquet"); diff --git a/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetMultiDeleteFileBenchmark.java b/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetMultiDeleteFileBenchmark.java index f410fcc905fe..4427d58b6477 100644 --- a/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetMultiDeleteFileBenchmark.java +++ b/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetMultiDeleteFileBenchmark.java @@ -21,6 +21,7 @@ import java.io.IOException; import org.apache.iceberg.DataFile; import org.apache.iceberg.FileFormat; +import org.apache.iceberg.SnapshotChanges; import org.apache.iceberg.spark.source.IcebergSourceDeleteBenchmark; import org.openjdk.jmh.annotations.Param; @@ -44,7 +45,7 @@ protected void appendData() throws IOException { writeData(fileNum); table().refresh(); - for (DataFile file : table().currentSnapshot().addedDataFiles(table().io())) { + for (DataFile file : SnapshotChanges.builderFor(table()).build().addedDataFiles()) { writePosDeletes(file.location(), NUM_ROWS, 0.25, numDeleteFile); } } diff --git a/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetPosDeleteBenchmark.java b/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetPosDeleteBenchmark.java index b145a8178898..03e2ebc06309 100644 --- a/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetPosDeleteBenchmark.java +++ b/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetPosDeleteBenchmark.java @@ -21,6 +21,7 @@ import java.io.IOException; import org.apache.iceberg.DataFile; import org.apache.iceberg.FileFormat; +import org.apache.iceberg.SnapshotChanges; import org.apache.iceberg.spark.source.IcebergSourceDeleteBenchmark; import org.openjdk.jmh.annotations.Param; @@ -46,7 +47,7 @@ protected void appendData() throws IOException { if (percentDeleteRow > 0) { // add pos-deletes table().refresh(); - for (DataFile file : table().currentSnapshot().addedDataFiles(table().io())) { + for (DataFile file : SnapshotChanges.builderFor(table()).build().addedDataFiles()) { writePosDeletes(file.location(), NUM_ROWS, percentDeleteRow); } } diff --git a/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetWithUnrelatedDeleteBenchmark.java b/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetWithUnrelatedDeleteBenchmark.java index b0439fe07ee8..dd35dde1fd65 100644 --- a/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetWithUnrelatedDeleteBenchmark.java +++ b/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetWithUnrelatedDeleteBenchmark.java @@ -21,6 +21,7 @@ import java.io.IOException; import org.apache.iceberg.DataFile; import org.apache.iceberg.FileFormat; +import org.apache.iceberg.SnapshotChanges; import org.apache.iceberg.spark.source.IcebergSourceDeleteBenchmark; import org.openjdk.jmh.annotations.Param; @@ -46,7 +47,7 @@ protected void appendData() throws IOException { writeData(fileNum); table().refresh(); - for (DataFile file : table().currentSnapshot().addedDataFiles(table().io())) { + for (DataFile file : SnapshotChanges.builderFor(table()).build().addedDataFiles()) { writePosDeletesWithNoise( file.location(), NUM_ROWS, diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestExpireSnapshotsAction.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestExpireSnapshotsAction.java index 7e07c66e0650..c89e4f7852d5 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestExpireSnapshotsAction.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestExpireSnapshotsAction.java @@ -47,6 +47,7 @@ import org.apache.iceberg.ReachableFileUtil; import org.apache.iceberg.Schema; import org.apache.iceberg.Snapshot; +import org.apache.iceberg.SnapshotChanges; import org.apache.iceberg.Table; import org.apache.iceberg.TableMetadata; import org.apache.iceberg.TableProperties; @@ -628,8 +629,10 @@ public void testWithExpiringDanglingStageCommit() { expectedDeletes.add(snapshotA.manifestListLocation()); // Files should be deleted of dangling staged snapshot - snapshotB - .addedDataFiles(table.io()) + SnapshotChanges.builderFor(table) + .snapshot(snapshotB) + .build() + .addedDataFiles() .forEach( i -> { expectedDeletes.add(i.location()); @@ -699,7 +702,10 @@ public void testWithCherryPickTableSnapshot() { Lists.newArrayList(snapshotB, snapshotC, snapshotD) .forEach( i -> { - i.addedDataFiles(table.io()) + SnapshotChanges.builderFor(table) + .snapshot(i) + .build() + .addedDataFiles() .forEach( item -> { assertThat(deletedFiles).doesNotContain(item.location()); @@ -748,7 +754,10 @@ public void testWithExpiringStagedThenCherrypick() { Lists.newArrayList(snapshotB) .forEach( i -> { - i.addedDataFiles(table.io()) + SnapshotChanges.builderFor(table) + .snapshot(i) + .build() + .addedDataFiles() .forEach( item -> { assertThat(deletedFiles).doesNotContain(item.location()); @@ -768,7 +777,10 @@ public void testWithExpiringStagedThenCherrypick() { Lists.newArrayList(snapshotB, snapshotD) .forEach( i -> { - i.addedDataFiles(table.io()) + SnapshotChanges.builderFor(table) + .snapshot(i) + .build() + .addedDataFiles() .forEach( item -> { assertThat(deletedFiles).doesNotContain(item.location()); diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteDataFilesAction.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteDataFilesAction.java index da07ae242827..8239d62950b6 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteDataFilesAction.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteDataFilesAction.java @@ -1719,7 +1719,7 @@ public void testAutoSortShuffleOutput() { assertThat(result.rewriteResults()).as("Should have 1 fileGroups").hasSize(1); assertThat(result.rewrittenBytesCount()).isEqualTo(dataSizeBefore); assertThat(result.rewriteResults()).as("Should have 1 fileGroups").hasSize(1); - assertThat(table.currentSnapshot().addedDataFiles(table.io())) + assertThat(SnapshotChanges.builderFor(table).build().addedDataFiles()) .as("Should have written 40+ files") .hasSizeGreaterThanOrEqualTo(40); @@ -1798,7 +1798,7 @@ public void testZOrderSort() { assertThat(result.rewriteResults()).as("Should have 1 fileGroups").hasSize(1); assertThat(result.rewrittenBytesCount()).isEqualTo(dataSizeBefore); - assertThat(table.currentSnapshot().addedDataFiles(table.io())) + assertThat(SnapshotChanges.builderFor(table).build().addedDataFiles()) .as("Should have written 40+ files") .hasSizeGreaterThanOrEqualTo(40); @@ -1863,7 +1863,7 @@ public void testZOrderAllTypesSort() { assertThat(result.rewriteResults()).as("Should have 1 fileGroups").hasSize(1); assertThat(result.rewrittenBytesCount()).isEqualTo(dataSizeBefore); - assertThat(table.currentSnapshot().addedDataFiles(table.io())) + assertThat(SnapshotChanges.builderFor(table).build().addedDataFiles()) .as("Should have written 1 file") .hasSize(1); diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteManifestsAction.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteManifestsAction.java index 4f95416878e2..ccef1916dae7 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteManifestsAction.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteManifestsAction.java @@ -58,6 +58,7 @@ import org.apache.iceberg.RowDelta; import org.apache.iceberg.Schema; import org.apache.iceberg.Snapshot; +import org.apache.iceberg.SnapshotChanges; import org.apache.iceberg.StructLike; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; @@ -728,7 +729,9 @@ public void testRewriteSmallManifestsNonPartitionedV2Table() { table.refresh(); Snapshot snapshot1 = table.currentSnapshot(); - DataFile file1 = Iterables.getOnlyElement(snapshot1.addedDataFiles(table.io())); + DataFile file1 = + Iterables.getOnlyElement( + SnapshotChanges.builderFor(table).snapshot(snapshot1).build().addedDataFiles()); List records2 = Lists.newArrayList(new ThreeColumnRecord(2, "CCCC", "CCCC")); writeRecords(records2); @@ -736,7 +739,9 @@ public void testRewriteSmallManifestsNonPartitionedV2Table() { table.refresh(); Snapshot snapshot2 = table.currentSnapshot(); - DataFile file2 = Iterables.getOnlyElement(snapshot2.addedDataFiles(table.io())); + DataFile file2 = + Iterables.getOnlyElement( + SnapshotChanges.builderFor(table).snapshot(snapshot2).build().addedDataFiles()); List manifests = table.currentSnapshot().allManifests(table.io()); assertThat(manifests).as("Should have 2 manifests before rewrite").hasSize(2); 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 6044368a46c3..dae721b1d73d 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 @@ -48,6 +48,7 @@ import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; import org.apache.iceberg.Snapshot; +import org.apache.iceberg.SnapshotChanges; import org.apache.iceberg.StaticTableOperations; import org.apache.iceberg.Table; import org.apache.iceberg.TableMetadata; @@ -435,9 +436,9 @@ private void runPositionDeletesTest(String fileFormat) throws Exception { List> deletes = Lists.newArrayList( Pair.of( - tableWithPosDeletes - .currentSnapshot() - .addedDataFiles(tableWithPosDeletes.io()) + SnapshotChanges.builderFor(tableWithPosDeletes) + .build() + .addedDataFiles() .iterator() .next() .location(), @@ -482,7 +483,7 @@ private void runPositionDeletesTest(String fileFormat) throws Exception { @TestTemplate public void testPositionDeleteWithRow() throws Exception { String dataFileLocation = - table.currentSnapshot().addedDataFiles(table.io()).iterator().next().location(); + SnapshotChanges.builderFor(table).build().addedDataFiles().iterator().next().location(); List> deletes = Lists.newArrayList(); OutputFile deleteFile = table @@ -532,7 +533,15 @@ public void testPositionDeletesAcrossFiles() throws Exception { .isEqualTo(2); Stream allFiles = StreamSupport.stream(table.snapshots().spliterator(), false) - .flatMap(s -> StreamSupport.stream(s.addedDataFiles(table.io()).spliterator(), false)); + .flatMap( + s -> + StreamSupport.stream( + SnapshotChanges.builderFor(table) + .snapshot(s) + .build() + .addedDataFiles() + .spliterator(), + false)); List> deletes = allFiles.map(f -> Pair.of((CharSequence) f.location(), 0L)).collect(Collectors.toList()); @@ -779,7 +788,10 @@ public void testRewritePathWithNonLiveEntry() throws Exception { Snapshot oldest = SnapshotUtil.oldestAncestor(tableWith3Snaps); String oldestDataFilePath = Iterables.getOnlyElement( - tableWith3Snaps.snapshot(oldest.snapshotId()).addedDataFiles(tableWith3Snaps.io())) + SnapshotChanges.builderFor(tableWith3Snaps) + .snapshot(tableWith3Snaps.snapshot(oldest.snapshotId())) + .build() + .addedDataFiles()) .location(); String deletedDataFilePathInTargetLocation = String.format("%sdata/%s", targetTableLocation(), fileName(oldestDataFilePath)); @@ -1303,27 +1315,14 @@ public void testNestedDirectoryStructurePreservation() throws Exception { // Create position delete files with same names in different nested directories // This simulates the scenario tested in // TestRewriteTablePathUtil.testStagingPathPreservesDirectoryStructure + SnapshotChanges sourceChanges = SnapshotChanges.builderFor(sourceTable).build(); List> deletes1 = Lists.newArrayList( - Pair.of( - sourceTable - .currentSnapshot() - .addedDataFiles(sourceTable.io()) - .iterator() - .next() - .location(), - 0L)); + Pair.of(sourceChanges.addedDataFiles().iterator().next().location(), 0L)); List> deletes2 = Lists.newArrayList( - Pair.of( - sourceTable - .currentSnapshot() - .addedDataFiles(sourceTable.io()) - .iterator() - .next() - .location(), - 0L)); + Pair.of(sourceChanges.addedDataFiles().iterator().next().location(), 0L)); // Create delete files with same name in different nested paths (hash1/ and hash2/) File file1 = diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/DataFrameWriteTestBase.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/DataFrameWriteTestBase.java index 05dec42a031a..80cc3556b3c4 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/DataFrameWriteTestBase.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/DataFrameWriteTestBase.java @@ -32,6 +32,7 @@ import org.apache.iceberg.Files; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; +import org.apache.iceberg.SnapshotChanges; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; import org.apache.iceberg.Tables; @@ -125,9 +126,9 @@ public void testAlternateLocation() throws IOException { writeRecords(table, RandomData.generateList(table.schema(), 100, 87112L)); - table - .currentSnapshot() - .addedDataFiles(table.io()) + SnapshotChanges.builderFor(table) + .build() + .addedDataFiles() .forEach( dataFile -> assertThat(dataFile.location()) diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestDataSourceOptions.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestDataSourceOptions.java index a7702b169a60..9ad73baa3dc4 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestDataSourceOptions.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestDataSourceOptions.java @@ -35,6 +35,7 @@ import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; import org.apache.iceberg.Snapshot; +import org.apache.iceberg.SnapshotChanges; import org.apache.iceberg.SnapshotSummary; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; @@ -206,7 +207,7 @@ public void testSplitOptionsOverridesTableProperties() throws IOException { .save(tableLocation); List files = - Lists.newArrayList(icebergTable.currentSnapshot().addedDataFiles(icebergTable.io())); + Lists.newArrayList(SnapshotChanges.builderFor(icebergTable).build().addedDataFiles()); assertThat(files).as("Should have written 1 file").hasSize(1); long fileSize = files.get(0).fileSizeInBytes(); diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java index 67647925e59d..be00461f2e4b 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java @@ -51,6 +51,7 @@ import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; import org.apache.iceberg.Snapshot; +import org.apache.iceberg.SnapshotChanges; import org.apache.iceberg.StructLike; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; @@ -257,7 +258,7 @@ public void testEntriesTableDataFilePrune() { .save(loadLocation(tableIdentifier)); table.refresh(); - DataFile file = table.currentSnapshot().addedDataFiles(table.io()).iterator().next(); + DataFile file = SnapshotChanges.builderFor(table).build().addedDataFiles().iterator().next(); List singleActual = rowsToJava( @@ -290,7 +291,7 @@ public void testEntriesTableDataFilePruneMulti() { .save(loadLocation(tableIdentifier)); table.refresh(); - DataFile file = table.currentSnapshot().addedDataFiles(table.io()).iterator().next(); + DataFile file = SnapshotChanges.builderFor(table).build().addedDataFiles().iterator().next(); List multiActual = rowsToJava( @@ -328,7 +329,7 @@ public void testFilesSelectMap() { .save(loadLocation(tableIdentifier)); table.refresh(); - DataFile file = table.currentSnapshot().addedDataFiles(table.io()).iterator().next(); + DataFile file = SnapshotChanges.builderFor(table).build().addedDataFiles().iterator().next(); List multiActual = rowsToJava( @@ -635,7 +636,7 @@ public void testFilesUnpartitionedTable() throws Exception { table.refresh(); DataFile toDelete = - Iterables.getOnlyElement(table.currentSnapshot().addedDataFiles(table.io())); + Iterables.getOnlyElement(SnapshotChanges.builderFor(table).build().addedDataFiles()); // add a second file df2.select("id", "data") @@ -1323,7 +1324,7 @@ public void testUnpartitionedPartitionsTable() { .set("file_count", 1) .set( "total_data_file_size_in_bytes", - totalSizeInBytes(table.currentSnapshot().addedDataFiles(table.io()))) + totalSizeInBytes(SnapshotChanges.builderFor(table).build().addedDataFiles())) .set("position_delete_record_count", 0L) .set("position_delete_file_count", 0) .set("equality_delete_record_count", 0L) @@ -1392,7 +1393,11 @@ public void testPartitionsTable() { .set("file_count", 1) .set( "total_data_file_size_in_bytes", - totalSizeInBytes(table.snapshot(firstCommitId).addedDataFiles(table.io()))) + totalSizeInBytes( + SnapshotChanges.builderFor(table) + .snapshot(table.snapshot(firstCommitId)) + .build() + .addedDataFiles())) .set("position_delete_record_count", 0L) .set("position_delete_file_count", 0) .set("equality_delete_record_count", 0L) @@ -1408,7 +1413,11 @@ public void testPartitionsTable() { .set("file_count", 1) .set( "total_data_file_size_in_bytes", - totalSizeInBytes(table.snapshot(secondCommitId).addedDataFiles(table.io()))) + totalSizeInBytes( + SnapshotChanges.builderFor(table) + .snapshot(table.snapshot(secondCommitId)) + .build() + .addedDataFiles())) .set("position_delete_record_count", 0L) .set("position_delete_file_count", 0) .set("equality_delete_record_count", 0L) @@ -1674,7 +1683,11 @@ public void testPartitionsTableDeleteStats() { .set("file_count", 1) .set( "total_data_file_size_in_bytes", - totalSizeInBytes(table.snapshot(firstCommitId).addedDataFiles(table.io()))) + totalSizeInBytes( + SnapshotChanges.builderFor(table) + .snapshot(table.snapshot(firstCommitId)) + .build() + .addedDataFiles())) .set("position_delete_record_count", 0L) .set("position_delete_file_count", 0) .set("equality_delete_record_count", 0L) @@ -1690,7 +1703,11 @@ public void testPartitionsTableDeleteStats() { .set("file_count", 1) .set( "total_data_file_size_in_bytes", - totalSizeInBytes(table.snapshot(firstCommitId).addedDataFiles(table.io()))) + totalSizeInBytes( + SnapshotChanges.builderFor(table) + .snapshot(table.snapshot(firstCommitId)) + .build() + .addedDataFiles())) .set("position_delete_record_count", 2L) // should be incremented now .set("position_delete_file_count", 2) // should be incremented now .set("equality_delete_record_count", 0L) @@ -2418,7 +2435,7 @@ private DeleteFile writePosDeleteFile(Table table) { private DeleteFile writePosDeleteFile(Table table, long pos) { DataFile dataFile = - Iterables.getFirst(table.currentSnapshot().addedDataFiles(table.io()), null); + Iterables.getFirst(SnapshotChanges.builderFor(table).build().addedDataFiles(), null); PartitionSpec dataFileSpec = table.specs().get(dataFile.specId()); StructLike dataFilePartition = dataFile.partition(); diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestMetadataTableReadableMetrics.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestMetadataTableReadableMetrics.java index c21ccd0100db..f74446b2f416 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestMetadataTableReadableMetrics.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestMetadataTableReadableMetrics.java @@ -33,6 +33,7 @@ import org.apache.iceberg.Parameters; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; +import org.apache.iceberg.SnapshotChanges; import org.apache.iceberg.Table; import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.catalog.TableIdentifier; @@ -200,7 +201,8 @@ private GenericRecord createNestedRecord(Long longCol, Double doubleCol) { @TestTemplate public void testPrimitiveColumns() throws Exception { Table table = createPrimitiveTable(); - DataFile dataFile = table.currentSnapshot().addedDataFiles(table.io()).iterator().next(); + DataFile dataFile = + SnapshotChanges.builderFor(table).build().addedDataFiles().iterator().next(); Map columnSizeStats = dataFile.columnSizes(); Object[] binaryCol = diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestRefreshTable.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestRefreshTable.java index 1bfeaded61c9..b5c357e6363e 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestRefreshTable.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestRefreshTable.java @@ -22,6 +22,7 @@ import java.util.Set; import org.apache.iceberg.DataFile; import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.SnapshotChanges; import org.apache.iceberg.Table; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.spark.CatalogTestBase; @@ -65,7 +66,7 @@ public void testRefreshCommand() { // Modify table outside of spark, it should be cached so Spark should see the same value after // mutation Table table = validationCatalog.loadTable(tableIdent); - DataFile file = table.currentSnapshot().addedDataFiles(table.io()).iterator().next(); + DataFile file = SnapshotChanges.builderFor(table).build().addedDataFiles().iterator().next(); table.newDelete().deleteFile(file).commit(); List cachedActual = sql("SELECT * FROM %s", tableName); diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestBranchDDL.java b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestBranchDDL.java index 75a8cf9203f1..26dde2059966 100644 --- a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestBranchDDL.java +++ b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestBranchDDL.java @@ -114,8 +114,8 @@ public void testCreateBranchOnEmptyTable() { SnapshotChanges.builderFor(table).snapshot(snapshot).build(); assertThat(changes.addedDataFiles()).isEmpty(); assertThat(changes.removedDataFiles()).isEmpty(); - assertThat(snapshot.addedDeleteFiles(table.io())).isEmpty(); - assertThat(snapshot.removedDeleteFiles(table.io())).isEmpty(); + assertThat(changes.addedDeleteFiles()).isEmpty(); + assertThat(changes.removedDeleteFiles()).isEmpty(); }); } @@ -377,8 +377,8 @@ public void testCreateOrReplaceBranchOnEmptyTable() { SnapshotChanges.builderFor(table).snapshot(snapshot).build(); assertThat(changes.addedDataFiles()).isEmpty(); assertThat(changes.removedDataFiles()).isEmpty(); - assertThat(snapshot.addedDeleteFiles(table.io())).isEmpty(); - assertThat(snapshot.removedDeleteFiles(table.io())).isEmpty(); + assertThat(changes.addedDeleteFiles()).isEmpty(); + assertThat(changes.removedDeleteFiles()).isEmpty(); }); } diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeOnReadDelete.java b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeOnReadDelete.java index bf9c53f82dbd..11643c495400 100644 --- a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeOnReadDelete.java +++ b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeOnReadDelete.java @@ -36,6 +36,7 @@ import org.apache.iceberg.RowDelta; import org.apache.iceberg.RowLevelOperationMode; import org.apache.iceberg.Snapshot; +import org.apache.iceberg.SnapshotChanges; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; import org.apache.iceberg.deletes.DeleteGranularity; @@ -140,7 +141,8 @@ public void testPositionDeletesAreMaintainedDuringDelete() throws NoSuchTableExc Table table = validationCatalog.loadTable(tableIdent); Snapshot latest = SnapshotUtil.latestSnapshot(table, branch); - assertThat(latest.removedDeleteFiles(table.io())).hasSize(1); + assertThat(SnapshotChanges.builderFor(table).snapshot(latest).build().removedDeleteFiles()) + .hasSize(1); assertEquals( "Should have expected rows", ImmutableList.of(row(1, "b"), row(2, "e")), @@ -181,7 +183,8 @@ public void testUnpartitionedPositionDeletesAreMaintainedDuringDelete() Table table = validationCatalog.loadTable(tableIdent); Snapshot latest = SnapshotUtil.latestSnapshot(table, branch); - assertThat(latest.removedDeleteFiles(table.io())).hasSize(1); + assertThat(SnapshotChanges.builderFor(table).snapshot(latest).build().removedDeleteFiles()) + .hasSize(1); assertEquals( "Should have expected rows", ImmutableList.of(row(1, "b"), row(2, "e")), diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeOnReadUpdate.java b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeOnReadUpdate.java index 1bec21b9b68d..2398bc45b19b 100644 --- a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeOnReadUpdate.java +++ b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeOnReadUpdate.java @@ -30,6 +30,7 @@ import org.apache.iceberg.ParameterizedTestExtension; import org.apache.iceberg.RowLevelOperationMode; import org.apache.iceberg.Snapshot; +import org.apache.iceberg.SnapshotChanges; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; import org.apache.iceberg.deletes.DeleteGranularity; @@ -75,7 +76,12 @@ public void testPositionDeletesAreMaintainedDuringUpdate() { String expectedDeleteFilesCount = "2"; validateMergeOnRead(currentSnapshot, "2", expectedDeleteFilesCount, "2"); - assertThat(currentSnapshot.removedDeleteFiles(table.io())).hasSize(2); + assertThat( + SnapshotChanges.builderFor(table) + .snapshot(currentSnapshot) + .build() + .removedDeleteFiles()) + .hasSize(2); assertEquals( "Should have expected rows", ImmutableList.of( @@ -123,7 +129,12 @@ public void testUnpartitionedPositionDeletesAreMaintainedDuringUpdate() { expectedDeleteFilesCount = "2"; validateMergeOnRead(currentSnapshot, "1", expectedDeleteFilesCount, "1"); - assertThat(currentSnapshot.removedDeleteFiles(table.io())).hasSize(2); + assertThat( + SnapshotChanges.builderFor(table) + .snapshot(currentSnapshot) + .build() + .removedDeleteFiles()) + .hasSize(2); assertEquals( "Should have expected rows", ImmutableList.of( diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteTablePathProcedure.java b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteTablePathProcedure.java index 0c83f34219b1..78f6b80ac948 100644 --- a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteTablePathProcedure.java +++ b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteTablePathProcedure.java @@ -30,6 +30,7 @@ import org.apache.iceberg.HasTableOperations; import org.apache.iceberg.ParameterizedTestExtension; import org.apache.iceberg.RewriteTablePathUtil; +import org.apache.iceberg.SnapshotChanges; import org.apache.iceberg.Table; import org.apache.iceberg.TableUtil; import org.apache.iceberg.data.FileHelpers; @@ -213,7 +214,12 @@ public void testRewriteTablePathWithManifestAndDeleteCounts() throws IOException List> rowsToDelete = Lists.newArrayList( Pair.of( - table.currentSnapshot().addedDataFiles(table.io()).iterator().next().location(), + SnapshotChanges.builderFor(table) + .build() + .addedDataFiles() + .iterator() + .next() + .location(), 0L)); File file = new File(removePrefix(table.location()) + "/data/deletes.parquet"); diff --git a/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetMultiDeleteFileBenchmark.java b/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetMultiDeleteFileBenchmark.java index 7891890dff4a..bbf099fae433 100644 --- a/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetMultiDeleteFileBenchmark.java +++ b/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetMultiDeleteFileBenchmark.java @@ -21,6 +21,7 @@ import java.io.IOException; import org.apache.iceberg.DataFile; import org.apache.iceberg.FileFormat; +import org.apache.iceberg.SnapshotChanges; import org.apache.iceberg.spark.source.IcebergSourceDeleteBenchmark; import org.openjdk.jmh.annotations.Param; @@ -44,7 +45,7 @@ protected void appendData() throws IOException { writeData(fileNum); table().refresh(); - for (DataFile file : table().currentSnapshot().addedDataFiles(table().io())) { + for (DataFile file : SnapshotChanges.builderFor(table()).build().addedDataFiles()) { writePosDeletes(file.location(), NUM_ROWS, 0.25, numDeleteFile); } } diff --git a/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetPosDeleteBenchmark.java b/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetPosDeleteBenchmark.java index 3c6dfa6bd94f..f8767797cad3 100644 --- a/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetPosDeleteBenchmark.java +++ b/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetPosDeleteBenchmark.java @@ -21,6 +21,7 @@ import java.io.IOException; import org.apache.iceberg.DataFile; import org.apache.iceberg.FileFormat; +import org.apache.iceberg.SnapshotChanges; import org.apache.iceberg.spark.source.IcebergSourceDeleteBenchmark; import org.openjdk.jmh.annotations.Param; @@ -46,7 +47,7 @@ protected void appendData() throws IOException { if (percentDeleteRow > 0) { // add pos-deletes table().refresh(); - for (DataFile file : table().currentSnapshot().addedDataFiles(table().io())) { + for (DataFile file : SnapshotChanges.builderFor(table()).build().addedDataFiles()) { writePosDeletes(file.location(), NUM_ROWS, percentDeleteRow); } } diff --git a/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetWithUnrelatedDeleteBenchmark.java b/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetWithUnrelatedDeleteBenchmark.java index 01096ac79649..8337d67182ee 100644 --- a/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetWithUnrelatedDeleteBenchmark.java +++ b/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetWithUnrelatedDeleteBenchmark.java @@ -21,6 +21,7 @@ import java.io.IOException; import org.apache.iceberg.DataFile; import org.apache.iceberg.FileFormat; +import org.apache.iceberg.SnapshotChanges; import org.apache.iceberg.spark.source.IcebergSourceDeleteBenchmark; import org.openjdk.jmh.annotations.Param; @@ -46,7 +47,7 @@ protected void appendData() throws IOException { writeData(fileNum); table().refresh(); - for (DataFile file : table().currentSnapshot().addedDataFiles(table().io())) { + for (DataFile file : SnapshotChanges.builderFor(table()).build().addedDataFiles()) { writePosDeletesWithNoise( file.location(), NUM_ROWS, diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestExpireSnapshotsAction.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestExpireSnapshotsAction.java index 7e07c66e0650..c89e4f7852d5 100644 --- a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestExpireSnapshotsAction.java +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestExpireSnapshotsAction.java @@ -47,6 +47,7 @@ import org.apache.iceberg.ReachableFileUtil; import org.apache.iceberg.Schema; import org.apache.iceberg.Snapshot; +import org.apache.iceberg.SnapshotChanges; import org.apache.iceberg.Table; import org.apache.iceberg.TableMetadata; import org.apache.iceberg.TableProperties; @@ -628,8 +629,10 @@ public void testWithExpiringDanglingStageCommit() { expectedDeletes.add(snapshotA.manifestListLocation()); // Files should be deleted of dangling staged snapshot - snapshotB - .addedDataFiles(table.io()) + SnapshotChanges.builderFor(table) + .snapshot(snapshotB) + .build() + .addedDataFiles() .forEach( i -> { expectedDeletes.add(i.location()); @@ -699,7 +702,10 @@ public void testWithCherryPickTableSnapshot() { Lists.newArrayList(snapshotB, snapshotC, snapshotD) .forEach( i -> { - i.addedDataFiles(table.io()) + SnapshotChanges.builderFor(table) + .snapshot(i) + .build() + .addedDataFiles() .forEach( item -> { assertThat(deletedFiles).doesNotContain(item.location()); @@ -748,7 +754,10 @@ public void testWithExpiringStagedThenCherrypick() { Lists.newArrayList(snapshotB) .forEach( i -> { - i.addedDataFiles(table.io()) + SnapshotChanges.builderFor(table) + .snapshot(i) + .build() + .addedDataFiles() .forEach( item -> { assertThat(deletedFiles).doesNotContain(item.location()); @@ -768,7 +777,10 @@ public void testWithExpiringStagedThenCherrypick() { Lists.newArrayList(snapshotB, snapshotD) .forEach( i -> { - i.addedDataFiles(table.io()) + SnapshotChanges.builderFor(table) + .snapshot(i) + .build() + .addedDataFiles() .forEach( item -> { assertThat(deletedFiles).doesNotContain(item.location()); diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteDataFilesAction.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteDataFilesAction.java index 75fe91bc6dc5..256ab9bcd0dc 100644 --- a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteDataFilesAction.java +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteDataFilesAction.java @@ -1722,7 +1722,7 @@ public void testAutoSortShuffleOutput() { assertThat(result.rewriteResults()).as("Should have 1 fileGroups").hasSize(1); assertThat(result.rewrittenBytesCount()).isEqualTo(dataSizeBefore); assertThat(result.rewriteResults()).as("Should have 1 fileGroups").hasSize(1); - assertThat(table.currentSnapshot().addedDataFiles(table.io())) + assertThat(SnapshotChanges.builderFor(table).build().addedDataFiles()) .as("Should have written 40+ files") .hasSizeGreaterThanOrEqualTo(40); @@ -1801,7 +1801,7 @@ public void testZOrderSort() { assertThat(result.rewriteResults()).as("Should have 1 fileGroups").hasSize(1); assertThat(result.rewrittenBytesCount()).isEqualTo(dataSizeBefore); - assertThat(table.currentSnapshot().addedDataFiles(table.io())) + assertThat(SnapshotChanges.builderFor(table).build().addedDataFiles()) .as("Should have written 40+ files") .hasSizeGreaterThanOrEqualTo(40); @@ -1867,7 +1867,7 @@ public void testZOrderAllTypesSort() { assertThat(result.rewriteResults()).as("Should have 1 fileGroups").hasSize(1); assertThat(result.rewrittenBytesCount()).isEqualTo(dataSizeBefore); - assertThat(table.currentSnapshot().addedDataFiles(table.io())) + assertThat(SnapshotChanges.builderFor(table).build().addedDataFiles()) .as("Should have written 1 file") .hasSize(1); diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteManifestsAction.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteManifestsAction.java index 4f95416878e2..ccef1916dae7 100644 --- a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteManifestsAction.java +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteManifestsAction.java @@ -58,6 +58,7 @@ import org.apache.iceberg.RowDelta; import org.apache.iceberg.Schema; import org.apache.iceberg.Snapshot; +import org.apache.iceberg.SnapshotChanges; import org.apache.iceberg.StructLike; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; @@ -728,7 +729,9 @@ public void testRewriteSmallManifestsNonPartitionedV2Table() { table.refresh(); Snapshot snapshot1 = table.currentSnapshot(); - DataFile file1 = Iterables.getOnlyElement(snapshot1.addedDataFiles(table.io())); + DataFile file1 = + Iterables.getOnlyElement( + SnapshotChanges.builderFor(table).snapshot(snapshot1).build().addedDataFiles()); List records2 = Lists.newArrayList(new ThreeColumnRecord(2, "CCCC", "CCCC")); writeRecords(records2); @@ -736,7 +739,9 @@ public void testRewriteSmallManifestsNonPartitionedV2Table() { table.refresh(); Snapshot snapshot2 = table.currentSnapshot(); - DataFile file2 = Iterables.getOnlyElement(snapshot2.addedDataFiles(table.io())); + DataFile file2 = + Iterables.getOnlyElement( + SnapshotChanges.builderFor(table).snapshot(snapshot2).build().addedDataFiles()); List manifests = table.currentSnapshot().allManifests(table.io()); assertThat(manifests).as("Should have 2 manifests before rewrite").hasSize(2); 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 6044368a46c3..dae721b1d73d 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 @@ -48,6 +48,7 @@ import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; import org.apache.iceberg.Snapshot; +import org.apache.iceberg.SnapshotChanges; import org.apache.iceberg.StaticTableOperations; import org.apache.iceberg.Table; import org.apache.iceberg.TableMetadata; @@ -435,9 +436,9 @@ private void runPositionDeletesTest(String fileFormat) throws Exception { List> deletes = Lists.newArrayList( Pair.of( - tableWithPosDeletes - .currentSnapshot() - .addedDataFiles(tableWithPosDeletes.io()) + SnapshotChanges.builderFor(tableWithPosDeletes) + .build() + .addedDataFiles() .iterator() .next() .location(), @@ -482,7 +483,7 @@ private void runPositionDeletesTest(String fileFormat) throws Exception { @TestTemplate public void testPositionDeleteWithRow() throws Exception { String dataFileLocation = - table.currentSnapshot().addedDataFiles(table.io()).iterator().next().location(); + SnapshotChanges.builderFor(table).build().addedDataFiles().iterator().next().location(); List> deletes = Lists.newArrayList(); OutputFile deleteFile = table @@ -532,7 +533,15 @@ public void testPositionDeletesAcrossFiles() throws Exception { .isEqualTo(2); Stream allFiles = StreamSupport.stream(table.snapshots().spliterator(), false) - .flatMap(s -> StreamSupport.stream(s.addedDataFiles(table.io()).spliterator(), false)); + .flatMap( + s -> + StreamSupport.stream( + SnapshotChanges.builderFor(table) + .snapshot(s) + .build() + .addedDataFiles() + .spliterator(), + false)); List> deletes = allFiles.map(f -> Pair.of((CharSequence) f.location(), 0L)).collect(Collectors.toList()); @@ -779,7 +788,10 @@ public void testRewritePathWithNonLiveEntry() throws Exception { Snapshot oldest = SnapshotUtil.oldestAncestor(tableWith3Snaps); String oldestDataFilePath = Iterables.getOnlyElement( - tableWith3Snaps.snapshot(oldest.snapshotId()).addedDataFiles(tableWith3Snaps.io())) + SnapshotChanges.builderFor(tableWith3Snaps) + .snapshot(tableWith3Snaps.snapshot(oldest.snapshotId())) + .build() + .addedDataFiles()) .location(); String deletedDataFilePathInTargetLocation = String.format("%sdata/%s", targetTableLocation(), fileName(oldestDataFilePath)); @@ -1303,27 +1315,14 @@ public void testNestedDirectoryStructurePreservation() throws Exception { // Create position delete files with same names in different nested directories // This simulates the scenario tested in // TestRewriteTablePathUtil.testStagingPathPreservesDirectoryStructure + SnapshotChanges sourceChanges = SnapshotChanges.builderFor(sourceTable).build(); List> deletes1 = Lists.newArrayList( - Pair.of( - sourceTable - .currentSnapshot() - .addedDataFiles(sourceTable.io()) - .iterator() - .next() - .location(), - 0L)); + Pair.of(sourceChanges.addedDataFiles().iterator().next().location(), 0L)); List> deletes2 = Lists.newArrayList( - Pair.of( - sourceTable - .currentSnapshot() - .addedDataFiles(sourceTable.io()) - .iterator() - .next() - .location(), - 0L)); + Pair.of(sourceChanges.addedDataFiles().iterator().next().location(), 0L)); // Create delete files with same name in different nested paths (hash1/ and hash2/) File file1 = diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/DataFrameWriteTestBase.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/DataFrameWriteTestBase.java index 9b5d8f5978ba..09dfaecdf417 100644 --- a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/DataFrameWriteTestBase.java +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/DataFrameWriteTestBase.java @@ -30,6 +30,7 @@ import org.apache.iceberg.Files; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; +import org.apache.iceberg.SnapshotChanges; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; import org.apache.iceberg.Tables; @@ -134,9 +135,9 @@ public void testAlternateLocation() throws IOException { writeRecords(table, RandomGenericData.generate(table.schema(), 100, 87112L)); - table - .currentSnapshot() - .addedDataFiles(table.io()) + SnapshotChanges.builderFor(table) + .build() + .addedDataFiles() .forEach( dataFile -> assertThat(dataFile.location()) diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestDataSourceOptions.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestDataSourceOptions.java index a7702b169a60..9ad73baa3dc4 100644 --- a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestDataSourceOptions.java +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestDataSourceOptions.java @@ -35,6 +35,7 @@ import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; import org.apache.iceberg.Snapshot; +import org.apache.iceberg.SnapshotChanges; import org.apache.iceberg.SnapshotSummary; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; @@ -206,7 +207,7 @@ public void testSplitOptionsOverridesTableProperties() throws IOException { .save(tableLocation); List files = - Lists.newArrayList(icebergTable.currentSnapshot().addedDataFiles(icebergTable.io())); + Lists.newArrayList(SnapshotChanges.builderFor(icebergTable).build().addedDataFiles()); assertThat(files).as("Should have written 1 file").hasSize(1); long fileSize = files.get(0).fileSizeInBytes(); diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java index 67647925e59d..be00461f2e4b 100644 --- a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java @@ -51,6 +51,7 @@ import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; import org.apache.iceberg.Snapshot; +import org.apache.iceberg.SnapshotChanges; import org.apache.iceberg.StructLike; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; @@ -257,7 +258,7 @@ public void testEntriesTableDataFilePrune() { .save(loadLocation(tableIdentifier)); table.refresh(); - DataFile file = table.currentSnapshot().addedDataFiles(table.io()).iterator().next(); + DataFile file = SnapshotChanges.builderFor(table).build().addedDataFiles().iterator().next(); List singleActual = rowsToJava( @@ -290,7 +291,7 @@ public void testEntriesTableDataFilePruneMulti() { .save(loadLocation(tableIdentifier)); table.refresh(); - DataFile file = table.currentSnapshot().addedDataFiles(table.io()).iterator().next(); + DataFile file = SnapshotChanges.builderFor(table).build().addedDataFiles().iterator().next(); List multiActual = rowsToJava( @@ -328,7 +329,7 @@ public void testFilesSelectMap() { .save(loadLocation(tableIdentifier)); table.refresh(); - DataFile file = table.currentSnapshot().addedDataFiles(table.io()).iterator().next(); + DataFile file = SnapshotChanges.builderFor(table).build().addedDataFiles().iterator().next(); List multiActual = rowsToJava( @@ -635,7 +636,7 @@ public void testFilesUnpartitionedTable() throws Exception { table.refresh(); DataFile toDelete = - Iterables.getOnlyElement(table.currentSnapshot().addedDataFiles(table.io())); + Iterables.getOnlyElement(SnapshotChanges.builderFor(table).build().addedDataFiles()); // add a second file df2.select("id", "data") @@ -1323,7 +1324,7 @@ public void testUnpartitionedPartitionsTable() { .set("file_count", 1) .set( "total_data_file_size_in_bytes", - totalSizeInBytes(table.currentSnapshot().addedDataFiles(table.io()))) + totalSizeInBytes(SnapshotChanges.builderFor(table).build().addedDataFiles())) .set("position_delete_record_count", 0L) .set("position_delete_file_count", 0) .set("equality_delete_record_count", 0L) @@ -1392,7 +1393,11 @@ public void testPartitionsTable() { .set("file_count", 1) .set( "total_data_file_size_in_bytes", - totalSizeInBytes(table.snapshot(firstCommitId).addedDataFiles(table.io()))) + totalSizeInBytes( + SnapshotChanges.builderFor(table) + .snapshot(table.snapshot(firstCommitId)) + .build() + .addedDataFiles())) .set("position_delete_record_count", 0L) .set("position_delete_file_count", 0) .set("equality_delete_record_count", 0L) @@ -1408,7 +1413,11 @@ public void testPartitionsTable() { .set("file_count", 1) .set( "total_data_file_size_in_bytes", - totalSizeInBytes(table.snapshot(secondCommitId).addedDataFiles(table.io()))) + totalSizeInBytes( + SnapshotChanges.builderFor(table) + .snapshot(table.snapshot(secondCommitId)) + .build() + .addedDataFiles())) .set("position_delete_record_count", 0L) .set("position_delete_file_count", 0) .set("equality_delete_record_count", 0L) @@ -1674,7 +1683,11 @@ public void testPartitionsTableDeleteStats() { .set("file_count", 1) .set( "total_data_file_size_in_bytes", - totalSizeInBytes(table.snapshot(firstCommitId).addedDataFiles(table.io()))) + totalSizeInBytes( + SnapshotChanges.builderFor(table) + .snapshot(table.snapshot(firstCommitId)) + .build() + .addedDataFiles())) .set("position_delete_record_count", 0L) .set("position_delete_file_count", 0) .set("equality_delete_record_count", 0L) @@ -1690,7 +1703,11 @@ public void testPartitionsTableDeleteStats() { .set("file_count", 1) .set( "total_data_file_size_in_bytes", - totalSizeInBytes(table.snapshot(firstCommitId).addedDataFiles(table.io()))) + totalSizeInBytes( + SnapshotChanges.builderFor(table) + .snapshot(table.snapshot(firstCommitId)) + .build() + .addedDataFiles())) .set("position_delete_record_count", 2L) // should be incremented now .set("position_delete_file_count", 2) // should be incremented now .set("equality_delete_record_count", 0L) @@ -2418,7 +2435,7 @@ private DeleteFile writePosDeleteFile(Table table) { private DeleteFile writePosDeleteFile(Table table, long pos) { DataFile dataFile = - Iterables.getFirst(table.currentSnapshot().addedDataFiles(table.io()), null); + Iterables.getFirst(SnapshotChanges.builderFor(table).build().addedDataFiles(), null); PartitionSpec dataFileSpec = table.specs().get(dataFile.specId()); StructLike dataFilePartition = dataFile.partition(); diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestMetadataTableReadableMetrics.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestMetadataTableReadableMetrics.java index c21ccd0100db..f74446b2f416 100644 --- a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestMetadataTableReadableMetrics.java +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestMetadataTableReadableMetrics.java @@ -33,6 +33,7 @@ import org.apache.iceberg.Parameters; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; +import org.apache.iceberg.SnapshotChanges; import org.apache.iceberg.Table; import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.catalog.TableIdentifier; @@ -200,7 +201,8 @@ private GenericRecord createNestedRecord(Long longCol, Double doubleCol) { @TestTemplate public void testPrimitiveColumns() throws Exception { Table table = createPrimitiveTable(); - DataFile dataFile = table.currentSnapshot().addedDataFiles(table.io()).iterator().next(); + DataFile dataFile = + SnapshotChanges.builderFor(table).build().addedDataFiles().iterator().next(); Map columnSizeStats = dataFile.columnSizes(); Object[] binaryCol = diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestRefreshTable.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestRefreshTable.java index 6becc07b2db2..34e9dbc87d02 100644 --- a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestRefreshTable.java +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestRefreshTable.java @@ -22,6 +22,7 @@ import java.util.Set; import org.apache.iceberg.DataFile; import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.SnapshotChanges; import org.apache.iceberg.Table; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; @@ -71,7 +72,7 @@ public void testRefreshCommand() { // Modify table outside of spark, it should be cached so Spark should see the same value after // mutation Table table = validationCatalog.loadTable(tableIdent); - DataFile file = table.currentSnapshot().addedDataFiles(table.io()).iterator().next(); + DataFile file = SnapshotChanges.builderFor(table).build().addedDataFiles().iterator().next(); table.newDelete().deleteFile(file).commit(); List cachedActual = sql("SELECT * FROM %s", tableName); diff --git a/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestBranchDDL.java b/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestBranchDDL.java index 75a8cf9203f1..26dde2059966 100644 --- a/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestBranchDDL.java +++ b/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestBranchDDL.java @@ -114,8 +114,8 @@ public void testCreateBranchOnEmptyTable() { SnapshotChanges.builderFor(table).snapshot(snapshot).build(); assertThat(changes.addedDataFiles()).isEmpty(); assertThat(changes.removedDataFiles()).isEmpty(); - assertThat(snapshot.addedDeleteFiles(table.io())).isEmpty(); - assertThat(snapshot.removedDeleteFiles(table.io())).isEmpty(); + assertThat(changes.addedDeleteFiles()).isEmpty(); + assertThat(changes.removedDeleteFiles()).isEmpty(); }); } @@ -377,8 +377,8 @@ public void testCreateOrReplaceBranchOnEmptyTable() { SnapshotChanges.builderFor(table).snapshot(snapshot).build(); assertThat(changes.addedDataFiles()).isEmpty(); assertThat(changes.removedDataFiles()).isEmpty(); - assertThat(snapshot.addedDeleteFiles(table.io())).isEmpty(); - assertThat(snapshot.removedDeleteFiles(table.io())).isEmpty(); + assertThat(changes.addedDeleteFiles()).isEmpty(); + assertThat(changes.removedDeleteFiles()).isEmpty(); }); } diff --git a/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeOnReadDelete.java b/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeOnReadDelete.java index 892fc14dd95b..fa2012d7dce2 100644 --- a/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeOnReadDelete.java +++ b/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeOnReadDelete.java @@ -36,6 +36,7 @@ import org.apache.iceberg.RowDelta; import org.apache.iceberg.RowLevelOperationMode; import org.apache.iceberg.Snapshot; +import org.apache.iceberg.SnapshotChanges; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; import org.apache.iceberg.deletes.DeleteGranularity; @@ -140,7 +141,8 @@ public void testPositionDeletesAreMaintainedDuringDelete() throws NoSuchTableExc Table table = validationCatalog.loadTable(tableIdent); Snapshot latest = SnapshotUtil.latestSnapshot(table, branch); - assertThat(latest.removedDeleteFiles(table.io())).hasSize(1); + assertThat(SnapshotChanges.builderFor(table).snapshot(latest).build().removedDeleteFiles()) + .hasSize(1); assertEquals( "Should have expected rows", ImmutableList.of(row(1, "b"), row(2, "e")), @@ -181,7 +183,8 @@ public void testUnpartitionedPositionDeletesAreMaintainedDuringDelete() Table table = validationCatalog.loadTable(tableIdent); Snapshot latest = SnapshotUtil.latestSnapshot(table, branch); - assertThat(latest.removedDeleteFiles(table.io())).hasSize(1); + assertThat(SnapshotChanges.builderFor(table).snapshot(latest).build().removedDeleteFiles()) + .hasSize(1); assertEquals( "Should have expected rows", ImmutableList.of(row(1, "b"), row(2, "e")), diff --git a/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeOnReadUpdate.java b/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeOnReadUpdate.java index 1bec21b9b68d..2398bc45b19b 100644 --- a/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeOnReadUpdate.java +++ b/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeOnReadUpdate.java @@ -30,6 +30,7 @@ import org.apache.iceberg.ParameterizedTestExtension; import org.apache.iceberg.RowLevelOperationMode; import org.apache.iceberg.Snapshot; +import org.apache.iceberg.SnapshotChanges; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; import org.apache.iceberg.deletes.DeleteGranularity; @@ -75,7 +76,12 @@ public void testPositionDeletesAreMaintainedDuringUpdate() { String expectedDeleteFilesCount = "2"; validateMergeOnRead(currentSnapshot, "2", expectedDeleteFilesCount, "2"); - assertThat(currentSnapshot.removedDeleteFiles(table.io())).hasSize(2); + assertThat( + SnapshotChanges.builderFor(table) + .snapshot(currentSnapshot) + .build() + .removedDeleteFiles()) + .hasSize(2); assertEquals( "Should have expected rows", ImmutableList.of( @@ -123,7 +129,12 @@ public void testUnpartitionedPositionDeletesAreMaintainedDuringUpdate() { expectedDeleteFilesCount = "2"; validateMergeOnRead(currentSnapshot, "1", expectedDeleteFilesCount, "1"); - assertThat(currentSnapshot.removedDeleteFiles(table.io())).hasSize(2); + assertThat( + SnapshotChanges.builderFor(table) + .snapshot(currentSnapshot) + .build() + .removedDeleteFiles()) + .hasSize(2); assertEquals( "Should have expected rows", ImmutableList.of( diff --git a/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteTablePathProcedure.java b/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteTablePathProcedure.java index 0c83f34219b1..78f6b80ac948 100644 --- a/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteTablePathProcedure.java +++ b/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteTablePathProcedure.java @@ -30,6 +30,7 @@ import org.apache.iceberg.HasTableOperations; import org.apache.iceberg.ParameterizedTestExtension; import org.apache.iceberg.RewriteTablePathUtil; +import org.apache.iceberg.SnapshotChanges; import org.apache.iceberg.Table; import org.apache.iceberg.TableUtil; import org.apache.iceberg.data.FileHelpers; @@ -213,7 +214,12 @@ public void testRewriteTablePathWithManifestAndDeleteCounts() throws IOException List> rowsToDelete = Lists.newArrayList( Pair.of( - table.currentSnapshot().addedDataFiles(table.io()).iterator().next().location(), + SnapshotChanges.builderFor(table) + .build() + .addedDataFiles() + .iterator() + .next() + .location(), 0L)); File file = new File(removePrefix(table.location()) + "/data/deletes.parquet"); diff --git a/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetMultiDeleteFileBenchmark.java b/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetMultiDeleteFileBenchmark.java index 9110ff071fbe..2ebb3ff5b4f2 100644 --- a/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetMultiDeleteFileBenchmark.java +++ b/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetMultiDeleteFileBenchmark.java @@ -21,6 +21,7 @@ import java.io.IOException; import org.apache.iceberg.DataFile; import org.apache.iceberg.FileFormat; +import org.apache.iceberg.SnapshotChanges; import org.apache.iceberg.spark.source.IcebergSourceDeleteBenchmark; import org.openjdk.jmh.annotations.Param; @@ -44,7 +45,7 @@ protected void appendData() throws IOException { writeData(fileNum); table().refresh(); - for (DataFile file : table().currentSnapshot().addedDataFiles(table().io())) { + for (DataFile file : SnapshotChanges.builderFor(table()).build().addedDataFiles()) { writePosDeletes(file.location(), NUM_ROWS, 0.25, numDeleteFile); } } diff --git a/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetPosDeleteBenchmark.java b/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetPosDeleteBenchmark.java index daf7c295c31e..fd837e8e826e 100644 --- a/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetPosDeleteBenchmark.java +++ b/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetPosDeleteBenchmark.java @@ -21,6 +21,7 @@ import java.io.IOException; import org.apache.iceberg.DataFile; import org.apache.iceberg.FileFormat; +import org.apache.iceberg.SnapshotChanges; import org.apache.iceberg.spark.source.IcebergSourceDeleteBenchmark; import org.openjdk.jmh.annotations.Param; @@ -46,7 +47,7 @@ protected void appendData() throws IOException { if (percentDeleteRow > 0) { // add pos-deletes table().refresh(); - for (DataFile file : table().currentSnapshot().addedDataFiles(table().io())) { + for (DataFile file : SnapshotChanges.builderFor(table()).build().addedDataFiles()) { writePosDeletes(file.location(), NUM_ROWS, percentDeleteRow); } } diff --git a/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetWithUnrelatedDeleteBenchmark.java b/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetWithUnrelatedDeleteBenchmark.java index ab83b688f55c..273b3f1cee4d 100644 --- a/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetWithUnrelatedDeleteBenchmark.java +++ b/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetWithUnrelatedDeleteBenchmark.java @@ -21,6 +21,7 @@ import java.io.IOException; import org.apache.iceberg.DataFile; import org.apache.iceberg.FileFormat; +import org.apache.iceberg.SnapshotChanges; import org.apache.iceberg.spark.source.IcebergSourceDeleteBenchmark; import org.openjdk.jmh.annotations.Param; @@ -46,7 +47,7 @@ protected void appendData() throws IOException { writeData(fileNum); table().refresh(); - for (DataFile file : table().currentSnapshot().addedDataFiles(table().io())) { + for (DataFile file : SnapshotChanges.builderFor(table()).build().addedDataFiles()) { writePosDeletesWithNoise( file.location(), NUM_ROWS, diff --git a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/MicroBatchUtils.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/MicroBatchUtils.java index 360e0e8fb685..7c73e3f416e3 100644 --- a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/MicroBatchUtils.java +++ b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/MicroBatchUtils.java @@ -19,6 +19,7 @@ package org.apache.iceberg.spark.source; import org.apache.iceberg.Snapshot; +import org.apache.iceberg.SnapshotChanges; import org.apache.iceberg.SnapshotSummary; import org.apache.iceberg.Table; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; @@ -61,7 +62,8 @@ static long addedFilesCount(Table table, Snapshot snapshot) { long addedFilesCount = PropertyUtil.propertyAsLong(snapshot.summary(), SnapshotSummary.ADDED_FILES_PROP, -1); return addedFilesCount == -1 - ? Iterables.size(snapshot.addedDataFiles(table.io())) + ? Iterables.size( + SnapshotChanges.builderFor(table).snapshot(snapshot).build().addedDataFiles()) : addedFilesCount; } } diff --git a/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/actions/TestExpireSnapshotsAction.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/actions/TestExpireSnapshotsAction.java index 7e07c66e0650..c89e4f7852d5 100644 --- a/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/actions/TestExpireSnapshotsAction.java +++ b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/actions/TestExpireSnapshotsAction.java @@ -47,6 +47,7 @@ import org.apache.iceberg.ReachableFileUtil; import org.apache.iceberg.Schema; import org.apache.iceberg.Snapshot; +import org.apache.iceberg.SnapshotChanges; import org.apache.iceberg.Table; import org.apache.iceberg.TableMetadata; import org.apache.iceberg.TableProperties; @@ -628,8 +629,10 @@ public void testWithExpiringDanglingStageCommit() { expectedDeletes.add(snapshotA.manifestListLocation()); // Files should be deleted of dangling staged snapshot - snapshotB - .addedDataFiles(table.io()) + SnapshotChanges.builderFor(table) + .snapshot(snapshotB) + .build() + .addedDataFiles() .forEach( i -> { expectedDeletes.add(i.location()); @@ -699,7 +702,10 @@ public void testWithCherryPickTableSnapshot() { Lists.newArrayList(snapshotB, snapshotC, snapshotD) .forEach( i -> { - i.addedDataFiles(table.io()) + SnapshotChanges.builderFor(table) + .snapshot(i) + .build() + .addedDataFiles() .forEach( item -> { assertThat(deletedFiles).doesNotContain(item.location()); @@ -748,7 +754,10 @@ public void testWithExpiringStagedThenCherrypick() { Lists.newArrayList(snapshotB) .forEach( i -> { - i.addedDataFiles(table.io()) + SnapshotChanges.builderFor(table) + .snapshot(i) + .build() + .addedDataFiles() .forEach( item -> { assertThat(deletedFiles).doesNotContain(item.location()); @@ -768,7 +777,10 @@ public void testWithExpiringStagedThenCherrypick() { Lists.newArrayList(snapshotB, snapshotD) .forEach( i -> { - i.addedDataFiles(table.io()) + SnapshotChanges.builderFor(table) + .snapshot(i) + .build() + .addedDataFiles() .forEach( item -> { assertThat(deletedFiles).doesNotContain(item.location()); diff --git a/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteDataFilesAction.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteDataFilesAction.java index 9857fea9ad17..d94b83e4c973 100644 --- a/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteDataFilesAction.java +++ b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteDataFilesAction.java @@ -1722,7 +1722,7 @@ public void testAutoSortShuffleOutput() { assertThat(result.rewriteResults()).as("Should have 1 fileGroups").hasSize(1); assertThat(result.rewrittenBytesCount()).isEqualTo(dataSizeBefore); assertThat(result.rewriteResults()).as("Should have 1 fileGroups").hasSize(1); - assertThat(table.currentSnapshot().addedDataFiles(table.io())) + assertThat(SnapshotChanges.builderFor(table).build().addedDataFiles()) .as("Should have written 40+ files") .hasSizeGreaterThanOrEqualTo(40); @@ -1801,7 +1801,7 @@ public void testZOrderSort() { assertThat(result.rewriteResults()).as("Should have 1 fileGroups").hasSize(1); assertThat(result.rewrittenBytesCount()).isEqualTo(dataSizeBefore); - assertThat(table.currentSnapshot().addedDataFiles(table.io())) + assertThat(SnapshotChanges.builderFor(table).build().addedDataFiles()) .as("Should have written 40+ files") .hasSizeGreaterThanOrEqualTo(40); @@ -1867,7 +1867,7 @@ public void testZOrderAllTypesSort() { assertThat(result.rewriteResults()).as("Should have 1 fileGroups").hasSize(1); assertThat(result.rewrittenBytesCount()).isEqualTo(dataSizeBefore); - assertThat(table.currentSnapshot().addedDataFiles(table.io())) + assertThat(SnapshotChanges.builderFor(table).build().addedDataFiles()) .as("Should have written 1 file") .hasSize(1); diff --git a/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteManifestsAction.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteManifestsAction.java index 4f95416878e2..ccef1916dae7 100644 --- a/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteManifestsAction.java +++ b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteManifestsAction.java @@ -58,6 +58,7 @@ import org.apache.iceberg.RowDelta; import org.apache.iceberg.Schema; import org.apache.iceberg.Snapshot; +import org.apache.iceberg.SnapshotChanges; import org.apache.iceberg.StructLike; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; @@ -728,7 +729,9 @@ public void testRewriteSmallManifestsNonPartitionedV2Table() { table.refresh(); Snapshot snapshot1 = table.currentSnapshot(); - DataFile file1 = Iterables.getOnlyElement(snapshot1.addedDataFiles(table.io())); + DataFile file1 = + Iterables.getOnlyElement( + SnapshotChanges.builderFor(table).snapshot(snapshot1).build().addedDataFiles()); List records2 = Lists.newArrayList(new ThreeColumnRecord(2, "CCCC", "CCCC")); writeRecords(records2); @@ -736,7 +739,9 @@ public void testRewriteSmallManifestsNonPartitionedV2Table() { table.refresh(); Snapshot snapshot2 = table.currentSnapshot(); - DataFile file2 = Iterables.getOnlyElement(snapshot2.addedDataFiles(table.io())); + DataFile file2 = + Iterables.getOnlyElement( + SnapshotChanges.builderFor(table).snapshot(snapshot2).build().addedDataFiles()); List manifests = table.currentSnapshot().allManifests(table.io()); assertThat(manifests).as("Should have 2 manifests before rewrite").hasSize(2); diff --git a/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteTablePathsAction.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteTablePathsAction.java index 6044368a46c3..dae721b1d73d 100644 --- a/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteTablePathsAction.java +++ b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteTablePathsAction.java @@ -48,6 +48,7 @@ import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; import org.apache.iceberg.Snapshot; +import org.apache.iceberg.SnapshotChanges; import org.apache.iceberg.StaticTableOperations; import org.apache.iceberg.Table; import org.apache.iceberg.TableMetadata; @@ -435,9 +436,9 @@ private void runPositionDeletesTest(String fileFormat) throws Exception { List> deletes = Lists.newArrayList( Pair.of( - tableWithPosDeletes - .currentSnapshot() - .addedDataFiles(tableWithPosDeletes.io()) + SnapshotChanges.builderFor(tableWithPosDeletes) + .build() + .addedDataFiles() .iterator() .next() .location(), @@ -482,7 +483,7 @@ private void runPositionDeletesTest(String fileFormat) throws Exception { @TestTemplate public void testPositionDeleteWithRow() throws Exception { String dataFileLocation = - table.currentSnapshot().addedDataFiles(table.io()).iterator().next().location(); + SnapshotChanges.builderFor(table).build().addedDataFiles().iterator().next().location(); List> deletes = Lists.newArrayList(); OutputFile deleteFile = table @@ -532,7 +533,15 @@ public void testPositionDeletesAcrossFiles() throws Exception { .isEqualTo(2); Stream allFiles = StreamSupport.stream(table.snapshots().spliterator(), false) - .flatMap(s -> StreamSupport.stream(s.addedDataFiles(table.io()).spliterator(), false)); + .flatMap( + s -> + StreamSupport.stream( + SnapshotChanges.builderFor(table) + .snapshot(s) + .build() + .addedDataFiles() + .spliterator(), + false)); List> deletes = allFiles.map(f -> Pair.of((CharSequence) f.location(), 0L)).collect(Collectors.toList()); @@ -779,7 +788,10 @@ public void testRewritePathWithNonLiveEntry() throws Exception { Snapshot oldest = SnapshotUtil.oldestAncestor(tableWith3Snaps); String oldestDataFilePath = Iterables.getOnlyElement( - tableWith3Snaps.snapshot(oldest.snapshotId()).addedDataFiles(tableWith3Snaps.io())) + SnapshotChanges.builderFor(tableWith3Snaps) + .snapshot(tableWith3Snaps.snapshot(oldest.snapshotId())) + .build() + .addedDataFiles()) .location(); String deletedDataFilePathInTargetLocation = String.format("%sdata/%s", targetTableLocation(), fileName(oldestDataFilePath)); @@ -1303,27 +1315,14 @@ public void testNestedDirectoryStructurePreservation() throws Exception { // Create position delete files with same names in different nested directories // This simulates the scenario tested in // TestRewriteTablePathUtil.testStagingPathPreservesDirectoryStructure + SnapshotChanges sourceChanges = SnapshotChanges.builderFor(sourceTable).build(); List> deletes1 = Lists.newArrayList( - Pair.of( - sourceTable - .currentSnapshot() - .addedDataFiles(sourceTable.io()) - .iterator() - .next() - .location(), - 0L)); + Pair.of(sourceChanges.addedDataFiles().iterator().next().location(), 0L)); List> deletes2 = Lists.newArrayList( - Pair.of( - sourceTable - .currentSnapshot() - .addedDataFiles(sourceTable.io()) - .iterator() - .next() - .location(), - 0L)); + Pair.of(sourceChanges.addedDataFiles().iterator().next().location(), 0L)); // Create delete files with same name in different nested paths (hash1/ and hash2/) File file1 = diff --git a/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/DataFrameWriteTestBase.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/DataFrameWriteTestBase.java index 9b5d8f5978ba..09dfaecdf417 100644 --- a/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/DataFrameWriteTestBase.java +++ b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/DataFrameWriteTestBase.java @@ -30,6 +30,7 @@ import org.apache.iceberg.Files; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; +import org.apache.iceberg.SnapshotChanges; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; import org.apache.iceberg.Tables; @@ -134,9 +135,9 @@ public void testAlternateLocation() throws IOException { writeRecords(table, RandomGenericData.generate(table.schema(), 100, 87112L)); - table - .currentSnapshot() - .addedDataFiles(table.io()) + SnapshotChanges.builderFor(table) + .build() + .addedDataFiles() .forEach( dataFile -> assertThat(dataFile.location()) diff --git a/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestDataSourceOptions.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestDataSourceOptions.java index 5fee9624e340..e67ec5fd62d4 100644 --- a/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestDataSourceOptions.java +++ b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestDataSourceOptions.java @@ -37,6 +37,7 @@ import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; import org.apache.iceberg.Snapshot; +import org.apache.iceberg.SnapshotChanges; import org.apache.iceberg.SnapshotSummary; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; @@ -210,7 +211,7 @@ public void testSplitOptionsOverridesTableProperties() throws IOException { .save(tableLocation); List files = - Lists.newArrayList(icebergTable.currentSnapshot().addedDataFiles(icebergTable.io())); + Lists.newArrayList(SnapshotChanges.builderFor(icebergTable).build().addedDataFiles()); assertThat(files).as("Should have written 1 file").hasSize(1); long fileSize = files.get(0).fileSizeInBytes(); diff --git a/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java index 3e5cc26d9c59..22f2de120a9e 100644 --- a/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java +++ b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java @@ -53,6 +53,7 @@ import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; import org.apache.iceberg.Snapshot; +import org.apache.iceberg.SnapshotChanges; import org.apache.iceberg.StructLike; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; @@ -262,7 +263,7 @@ public void testEntriesTableDataFilePrune() { .save(loadLocation(tableIdentifier)); table.refresh(); - DataFile file = table.currentSnapshot().addedDataFiles(table.io()).iterator().next(); + DataFile file = SnapshotChanges.builderFor(table).build().addedDataFiles().iterator().next(); List singleActual = rowsToJava( @@ -295,7 +296,7 @@ public void testEntriesTableDataFilePruneMulti() { .save(loadLocation(tableIdentifier)); table.refresh(); - DataFile file = table.currentSnapshot().addedDataFiles(table.io()).iterator().next(); + DataFile file = SnapshotChanges.builderFor(table).build().addedDataFiles().iterator().next(); List multiActual = rowsToJava( @@ -333,7 +334,7 @@ public void testFilesSelectMap() { .save(loadLocation(tableIdentifier)); table.refresh(); - DataFile file = table.currentSnapshot().addedDataFiles(table.io()).iterator().next(); + DataFile file = SnapshotChanges.builderFor(table).build().addedDataFiles().iterator().next(); List multiActual = rowsToJava( @@ -640,7 +641,7 @@ public void testFilesUnpartitionedTable() throws Exception { table.refresh(); DataFile toDelete = - Iterables.getOnlyElement(table.currentSnapshot().addedDataFiles(table.io())); + Iterables.getOnlyElement(SnapshotChanges.builderFor(table).build().addedDataFiles()); // add a second file df2.select("id", "data") @@ -1328,7 +1329,7 @@ public void testUnpartitionedPartitionsTable() { .set("file_count", 1) .set( "total_data_file_size_in_bytes", - totalSizeInBytes(table.currentSnapshot().addedDataFiles(table.io()))) + totalSizeInBytes(SnapshotChanges.builderFor(table).build().addedDataFiles())) .set("position_delete_record_count", 0L) .set("position_delete_file_count", 0) .set("equality_delete_record_count", 0L) @@ -1397,7 +1398,11 @@ public void testPartitionsTable() { .set("file_count", 1) .set( "total_data_file_size_in_bytes", - totalSizeInBytes(table.snapshot(firstCommitId).addedDataFiles(table.io()))) + totalSizeInBytes( + SnapshotChanges.builderFor(table) + .snapshot(table.snapshot(firstCommitId)) + .build() + .addedDataFiles())) .set("position_delete_record_count", 0L) .set("position_delete_file_count", 0) .set("equality_delete_record_count", 0L) @@ -1413,7 +1418,11 @@ public void testPartitionsTable() { .set("file_count", 1) .set( "total_data_file_size_in_bytes", - totalSizeInBytes(table.snapshot(secondCommitId).addedDataFiles(table.io()))) + totalSizeInBytes( + SnapshotChanges.builderFor(table) + .snapshot(table.snapshot(secondCommitId)) + .build() + .addedDataFiles())) .set("position_delete_record_count", 0L) .set("position_delete_file_count", 0) .set("equality_delete_record_count", 0L) @@ -1679,7 +1688,11 @@ public void testPartitionsTableDeleteStats() { .set("file_count", 1) .set( "total_data_file_size_in_bytes", - totalSizeInBytes(table.snapshot(firstCommitId).addedDataFiles(table.io()))) + totalSizeInBytes( + SnapshotChanges.builderFor(table) + .snapshot(table.snapshot(firstCommitId)) + .build() + .addedDataFiles())) .set("position_delete_record_count", 0L) .set("position_delete_file_count", 0) .set("equality_delete_record_count", 0L) @@ -1695,7 +1708,11 @@ public void testPartitionsTableDeleteStats() { .set("file_count", 1) .set( "total_data_file_size_in_bytes", - totalSizeInBytes(table.snapshot(firstCommitId).addedDataFiles(table.io()))) + totalSizeInBytes( + SnapshotChanges.builderFor(table) + .snapshot(table.snapshot(firstCommitId)) + .build() + .addedDataFiles())) .set("position_delete_record_count", 2L) // should be incremented now .set("position_delete_file_count", 2) // should be incremented now .set("equality_delete_record_count", 0L) @@ -2426,7 +2443,7 @@ private DeleteFile writePosDeleteFile(Table table) { private DeleteFile writePosDeleteFile(Table table, long pos) { DataFile dataFile = - Iterables.getFirst(table.currentSnapshot().addedDataFiles(table.io()), null); + Iterables.getFirst(SnapshotChanges.builderFor(table).build().addedDataFiles(), null); PartitionSpec dataFileSpec = table.specs().get(dataFile.specId()); StructLike dataFilePartition = dataFile.partition(); diff --git a/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestMetadataTableReadableMetrics.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestMetadataTableReadableMetrics.java index c21ccd0100db..f74446b2f416 100644 --- a/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestMetadataTableReadableMetrics.java +++ b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestMetadataTableReadableMetrics.java @@ -33,6 +33,7 @@ import org.apache.iceberg.Parameters; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; +import org.apache.iceberg.SnapshotChanges; import org.apache.iceberg.Table; import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.catalog.TableIdentifier; @@ -200,7 +201,8 @@ private GenericRecord createNestedRecord(Long longCol, Double doubleCol) { @TestTemplate public void testPrimitiveColumns() throws Exception { Table table = createPrimitiveTable(); - DataFile dataFile = table.currentSnapshot().addedDataFiles(table.io()).iterator().next(); + DataFile dataFile = + SnapshotChanges.builderFor(table).build().addedDataFiles().iterator().next(); Map columnSizeStats = dataFile.columnSizes(); Object[] binaryCol = diff --git a/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/sql/TestRefreshTable.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/sql/TestRefreshTable.java index 6becc07b2db2..34e9dbc87d02 100644 --- a/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/sql/TestRefreshTable.java +++ b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/sql/TestRefreshTable.java @@ -22,6 +22,7 @@ import java.util.Set; import org.apache.iceberg.DataFile; import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.SnapshotChanges; import org.apache.iceberg.Table; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; @@ -71,7 +72,7 @@ public void testRefreshCommand() { // Modify table outside of spark, it should be cached so Spark should see the same value after // mutation Table table = validationCatalog.loadTable(tableIdent); - DataFile file = table.currentSnapshot().addedDataFiles(table.io()).iterator().next(); + DataFile file = SnapshotChanges.builderFor(table).build().addedDataFiles().iterator().next(); table.newDelete().deleteFile(file).commit(); List cachedActual = sql("SELECT * FROM %s", tableName);