From 548298eed439b30ebd26033360af2d5f9e969c13 Mon Sep 17 00:00:00 2001 From: Russell Spitzer Date: Mon, 16 Mar 2026 16:32:36 -0500 Subject: [PATCH 1/6] Migrate callers off deprecated Snapshot file-access methods to SnapshotChanges Replace direct calls to deprecated Snapshot.addedDataFiles(io), removedDataFiles(io), addedDeleteFiles(io), and removedDeleteFiles(io) with the SnapshotChanges builder API across production and test code. Generalize SnapshotChanges to support multi-snapshot aggregation via snapshots(List) on the builder, replacing the need for SnapshotUtil.newFilesBetween in CherryPickOperation. Deprecate the remaining newFilesBetween overloads in SnapshotUtil. Update CatalogUtil, ReachableFileCleanup, and PartitionStatsHandler to pass specsById when opening manifests. --- .../java/org/apache/iceberg/CatalogUtil.java | 8 ++- .../apache/iceberg/CherryPickOperation.java | 56 +++++++++++---- .../apache/iceberg/PartitionStatsHandler.java | 3 +- .../apache/iceberg/ReachableFileCleanup.java | 14 ++-- .../org/apache/iceberg/util/SnapshotUtil.java | 34 +++++++-- .../java/org/apache/iceberg/TestRowDelta.java | 23 ++++-- .../org/apache/iceberg/io/TestDVWriters.java | 21 +++--- .../delta/TestSnapshotDeltaLakeTable.java | 8 ++- .../maintenance/operator/RewriteUtil.java | 3 +- .../TestDataFileRewriteCommitter.java | 7 +- .../operator/TestMonitorSource.java | 46 ++++++++---- .../flink/sink/TestFlinkIcebergSinkV2.java | 6 +- .../iceberg/flink/sink/TestIcebergSinkV2.java | 6 +- .../TestMetadataTableReadableMetrics.java | 4 +- .../maintenance/operator/RewriteUtil.java | 3 +- .../TestDataFileRewriteCommitter.java | 7 +- .../operator/TestMonitorSource.java | 22 ++++-- .../flink/sink/TestFlinkIcebergSinkV2.java | 6 +- .../iceberg/flink/sink/TestIcebergSinkV2.java | 6 +- .../TestMetadataTableReadableMetrics.java | 4 +- .../maintenance/operator/RewriteUtil.java | 3 +- .../TestDataFileRewriteCommitter.java | 7 +- .../operator/TestMonitorSource.java | 22 ++++-- .../flink/sink/TestFlinkIcebergSinkV2.java | 6 +- .../iceberg/flink/sink/TestIcebergSinkV2.java | 6 +- .../TestMetadataTableReadableMetrics.java | 4 +- .../iceberg/connect/IntegrationTestBase.java | 13 +++- .../connect/channel/TestCoordinator.java | 11 +-- .../iceberg/spark/PlanningBenchmark.java | 5 +- .../spark/TaskGroupPlanningBenchmark.java | 6 +- .../spark/extensions/TestBranchDDL.java | 14 ++-- .../extensions/TestMergeOnReadDelete.java | 18 +++-- .../extensions/TestMergeOnReadUpdate.java | 72 ++++++++++--------- .../TestRewriteTablePathProcedure.java | 14 +++- ...SourceParquetMultiDeleteFileBenchmark.java | 7 +- ...cebergSourceParquetPosDeleteBenchmark.java | 7 +- ...ceParquetWithUnrelatedDeleteBenchmark.java | 7 +- .../actions/TestExpireSnapshotsAction.java | 25 +++++-- .../actions/TestRewriteDataFilesAction.java | 57 ++++++++++----- .../actions/TestRewriteManifestsAction.java | 9 ++- .../actions/TestRewriteTablePathsAction.java | 43 ++++++----- .../spark/source/DataFrameWriteTestBase.java | 46 +++++++----- .../spark/source/TestDataSourceOptions.java | 28 +++++--- .../source/TestIcebergSourceTablesBase.java | 60 +++++++++++----- .../TestMetadataTableReadableMetrics.java | 4 +- .../iceberg/spark/sql/TestRefreshTable.java | 13 +++- .../spark/extensions/TestBranchDDL.java | 8 +-- .../extensions/TestMergeOnReadDelete.java | 14 +++- .../extensions/TestMergeOnReadUpdate.java | 15 +++- .../TestRewriteTablePathProcedure.java | 14 +++- ...SourceParquetMultiDeleteFileBenchmark.java | 7 +- ...cebergSourceParquetPosDeleteBenchmark.java | 7 +- ...ceParquetWithUnrelatedDeleteBenchmark.java | 7 +- .../actions/TestExpireSnapshotsAction.java | 22 ++++-- .../actions/TestRewriteDataFilesAction.java | 57 ++++++++++----- .../actions/TestRewriteManifestsAction.java | 9 ++- .../actions/TestRewriteTablePathsAction.java | 43 ++++++----- .../spark/source/DataFrameWriteTestBase.java | 46 +++++++----- .../spark/source/TestDataSourceOptions.java | 24 ++++--- .../source/TestIcebergSourceTablesBase.java | 57 ++++++++++----- .../TestMetadataTableReadableMetrics.java | 4 +- .../iceberg/spark/sql/TestRefreshTable.java | 11 ++- .../spark/extensions/TestBranchDDL.java | 8 +-- .../extensions/TestMergeOnReadDelete.java | 14 +++- .../extensions/TestMergeOnReadUpdate.java | 15 +++- .../TestRewriteTablePathProcedure.java | 8 ++- ...SourceParquetMultiDeleteFileBenchmark.java | 7 +- ...cebergSourceParquetPosDeleteBenchmark.java | 7 +- ...ceParquetWithUnrelatedDeleteBenchmark.java | 7 +- .../actions/TestExpireSnapshotsAction.java | 22 ++++-- .../actions/TestRewriteDataFilesAction.java | 8 +-- .../actions/TestRewriteManifestsAction.java | 9 ++- .../actions/TestRewriteTablePathsAction.java | 43 ++++++----- .../spark/source/DataFrameWriteTestBase.java | 7 +- .../spark/source/TestDataSourceOptions.java | 24 ++++--- .../source/TestIcebergSourceTablesBase.java | 57 ++++++++++----- .../TestMetadataTableReadableMetrics.java | 4 +- .../iceberg/spark/sql/TestRefreshTable.java | 3 +- .../spark/extensions/TestBranchDDL.java | 8 +-- .../extensions/TestMergeOnReadDelete.java | 7 +- .../extensions/TestMergeOnReadUpdate.java | 15 +++- .../TestRewriteTablePathProcedure.java | 8 ++- ...SourceParquetMultiDeleteFileBenchmark.java | 3 +- ...cebergSourceParquetPosDeleteBenchmark.java | 3 +- ...ceParquetWithUnrelatedDeleteBenchmark.java | 3 +- .../iceberg/spark/source/MicroBatchUtils.java | 4 +- .../actions/TestExpireSnapshotsAction.java | 22 ++++-- .../actions/TestRewriteDataFilesAction.java | 6 +- .../actions/TestRewriteManifestsAction.java | 9 ++- .../actions/TestRewriteTablePathsAction.java | 43 ++++++----- .../spark/source/DataFrameWriteTestBase.java | 7 +- .../spark/source/TestDataSourceOptions.java | 3 +- .../source/TestIcebergSourceTablesBase.java | 37 +++++++--- .../TestMetadataTableReadableMetrics.java | 4 +- .../iceberg/spark/sql/TestRefreshTable.java | 3 +- 95 files changed, 1048 insertions(+), 519 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/CatalogUtil.java b/core/src/main/java/org/apache/iceberg/CatalogUtil.java index 7147f98e6237..07ca55d522b3 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); } 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, TableMetadata metadata) { // 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,8 @@ 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, metadata.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..35bf9795f3da 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,17 +222,43 @@ 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 -> { + Iterable ownedManifests = + Iterables.filter( + snap.dataManifests(io), + m -> Objects.equals(m.snapshotId(), snap.snapshotId())); + return Iterables.transform( + ownedManifests, + manifest -> { + CloseableIterable> entries = + ManifestFiles.read(manifest, io, meta.specsById()).entries(); + CloseableIterable> added = + CloseableIterable.filter( + entries, e -> e.status() == ManifestEntry.Status.ADDED); + return CloseableIterable.transform(added, e -> e.file().copy()); + }); + })); + + 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); } } } 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..1c645a03ebed 100644 --- a/core/src/main/java/org/apache/iceberg/ReachableFileCleanup.java +++ b/core/src/main/java/org/apache/iceberg/ReachableFileCleanup.java @@ -81,7 +81,8 @@ public void cleanFiles( if (!manifestsToDelete.isEmpty()) { if (ExpireSnapshots.CleanupLevel.ALL == cleanupLevel) { - Set dataFilesToDelete = findFilesToDelete(manifestsToDelete, currentManifests); + Set dataFilesToDelete = + findFilesToDelete(manifestsToDelete, currentManifests, afterExpiration); LOG.debug("Deleting {} data files", dataFilesToDelete.size()); deleteFiles(dataFilesToDelete, "data"); } @@ -165,9 +166,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, + TableMetadata metadata) { Set filesToDelete = ConcurrentHashMap.newKeySet(); Tasks.foreach(manifestFilesToDelete) @@ -180,7 +182,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, metadata.specsById())) { paths.forEach(filesToDelete::add); } catch (IOException e) { throw new RuntimeIOException(e, "Failed to read manifest file: %s", manifest); @@ -208,7 +211,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, metadata.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..d3d3c84ee3f8 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,9 +311,34 @@ 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) { + List snapshots = snapshotsBetween(startSnapshotId, endSnapshotId, lookup); + return new ParallelIterable<>( + Iterables.transform(snapshots, snapshot -> snapshot.addedDataFiles(io)), + ThreadPools.getWorkerPool()); + } + /** + * @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, TableMetadata metadata, FileIO io) { + List snapshots = snapshotsBetween(startSnapshotId, endSnapshotId, metadata::snapshot); + return new ParallelIterable<>( + Iterables.transform(snapshots, snapshot -> snapshot.addedDataFiles(io)), + ThreadPools.getWorkerPool()); + } + + private static List snapshotsBetween( + Long startSnapshotId, long endSnapshotId, Function lookup) { List snapshots = Lists.newArrayList(); Snapshot lastSnapshot = null; for (Snapshot currentSnapshot : ancestorsOf(endSnapshotId, lookup)) { @@ -333,9 +359,7 @@ public static CloseableIterable newFilesBetween( lastSnapshot.snapshotId()); } - return new ParallelIterable<>( - Iterables.transform(snapshots, snapshot -> snapshot.addedDataFiles(io)), - ThreadPools.getWorkerPool()); + return snapshots; } /** 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..84090e9e6b7b 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 @@ -25,21 +25,28 @@ import java.io.IOException; import java.time.Duration; import java.util.List; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.atomic.AtomicReference; +import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.eventtime.WatermarkStrategy; import org.apache.flink.api.connector.source.util.ratelimit.RateLimiterStrategy; +import org.apache.flink.client.program.ClusterClient; import org.apache.flink.configuration.CheckpointingOptions; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.execution.JobClient; import org.apache.flink.runtime.client.JobExecutionException; +import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.graph.StreamGraph; +import org.apache.flink.test.junit5.InjectClusterClient; import org.apache.hadoop.fs.Path; import org.apache.iceberg.ContentFile; import org.apache.iceberg.DataFile; 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; @@ -157,7 +164,9 @@ void testSource() throws Exception { /** Check that the {@link MonitorSource} operator state is restored correctly. */ @Test - void testStateRestore(@TempDir File savepointDir) throws Exception { + void testStateRestore( + @TempDir File savepointDir, @InjectClusterClient ClusterClient clusterClient) + throws Exception { Table table = createTable(); insert(table, 1, "a"); TableLoader tableLoader = tableLoader(); @@ -180,7 +189,7 @@ void testStateRestore(@TempDir File savepointDir) throws Exception { events.sinkTo(result); // Start the job - Configuration conf; + String savepointPath; JobClient jobClient = null; AtomicReference firstNonEmptyEvent = new AtomicReference<>(); try { @@ -201,11 +210,11 @@ void testStateRestore(@TempDir File savepointDir) throws Exception { }); } finally { // Stop with savepoint - conf = closeJobClient(jobClient, savepointDir); + savepointPath = closeJobClient(jobClient, savepointDir); } // Restore from savepoint, create the same topology with a different env - env = StreamExecutionEnvironment.getExecutionEnvironment(conf); + env = StreamExecutionEnvironment.getExecutionEnvironment(); events = env.fromSource( new MonitorSource(tableLoader, LOW_RATE, Long.MAX_VALUE), @@ -216,13 +225,13 @@ void testStateRestore(@TempDir File savepointDir) throws Exception { events.sinkTo(resultWithSavepoint); // Make sure that the job with restored source does not read new records from the table - JobClient clientWithSavepoint = null; + StreamGraph streamGraph = env.getStreamGraph(); + streamGraph.setSavepointRestoreSettings(SavepointRestoreSettings.forPath(savepointPath)); + CompletableFuture jobIDCompletableFuture = clusterClient.submitJob(streamGraph); try { - clientWithSavepoint = env.executeAsync("Table Change Source test with savepoint"); - assertThat(resultWithSavepoint.poll(Duration.ofSeconds(5L))).isEqualTo(EMPTY_EVENT); } finally { - closeJobClient(clientWithSavepoint, null); + clusterClient.cancel(jobIDCompletableFuture.get()); } // Restore without savepoint @@ -308,7 +317,12 @@ void testSkipReplace() throws IOException { // Create a DataOperations.REPLACE snapshot DataFile dataFile = - table.snapshots().iterator().next().addedDataFiles(table.io()).iterator().next(); + SnapshotChanges.builderFor(table) + .snapshot(table.snapshots().iterator().next()) + .build() + .addedDataFiles() + .iterator() + .next(); RewriteFiles rewrite = tableLoader.loadTable().newRewrite(); // Replace the file with itself for testing purposes rewrite.deleteFile(dataFile); @@ -320,14 +334,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..84090e9e6b7b 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,12 @@ void testSkipReplace() throws IOException { // Create a DataOperations.REPLACE snapshot DataFile dataFile = - table.snapshots().iterator().next().addedDataFiles(table.io()).iterator().next(); + SnapshotChanges.builderFor(table) + .snapshot(table.snapshots().iterator().next()) + .build() + .addedDataFiles() + .iterator() + .next(); RewriteFiles rewrite = tableLoader.loadTable().newRewrite(); // Replace the file with itself for testing purposes rewrite.deleteFile(dataFile); @@ -328,14 +334,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..84090e9e6b7b 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,12 @@ void testSkipReplace() throws IOException { // Create a DataOperations.REPLACE snapshot DataFile dataFile = - table.snapshots().iterator().next().addedDataFiles(table.io()).iterator().next(); + SnapshotChanges.builderFor(table) + .snapshot(table.snapshots().iterator().next()) + .build() + .addedDataFiles() + .iterator() + .next(); RewriteFiles rewrite = tableLoader.loadTable().newRewrite(); // Replace the file with itself for testing purposes rewrite.deleteFile(dataFile); @@ -328,14 +334,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..26dde2059966 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 @@ -58,9 +58,9 @@ public void removeTable() { public static Object[][] parameters() { return new Object[][] { { - SparkCatalogConfig.SPARK.catalogName(), - SparkCatalogConfig.SPARK.implementation(), - SparkCatalogConfig.SPARK.properties() + SparkCatalogConfig.SPARK_SESSION.catalogName(), + SparkCatalogConfig.SPARK_SESSION.implementation(), + SparkCatalogConfig.SPARK_SESSION.properties() } }; } @@ -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..fa2012d7dce2 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")), @@ -241,10 +244,6 @@ private void checkDeleteFileGranularity(DeleteGranularity deleteGranularity) createTableWithDeleteGranularity( "id INT, dep STRING", "PARTITIONED BY (dep)", deleteGranularity); - sql( - "ALTER TABLE %s SET TBLPROPERTIES ('%s' '%s')", - tableName, TableProperties.DELETE_GRANULARITY, deleteGranularity); - append(tableName, new Employee(1, "hr"), new Employee(2, "hr")); append(tableName, new Employee(3, "hr"), new Employee(4, "hr")); append(tableName, new Employee(1, "hardware"), new Employee(2, "hardware")); @@ -298,7 +297,7 @@ public void testCommitUnknownException() { Table spyTable = spy(table); when(spyTable.newRowDelta()).thenReturn(spyNewRowDelta); SparkTable sparkTable = - branch == null ? new SparkTable(spyTable, false) : new SparkTable(spyTable, branch, false); + branch == null ? new SparkTable(spyTable) : SparkTable.create(spyTable, branch); ImmutableMap config = ImmutableMap.of( @@ -317,6 +316,11 @@ public void testCommitUnknownException() { .isInstanceOf(CommitStateUnknownException.class) .hasMessageStartingWith("Datacenter on Fire"); + // Manually refresh Spark table because it always pins snapshot + sparkTable = branch == null ? new SparkTable(spyTable) : SparkTable.create(spyTable, branch); + TestSparkCatalog.unsetTable(ident); + TestSparkCatalog.setTable(ident, sparkTable); + // Since write and commit succeeded, the rows should be readable assertEquals( "Should have expected rows", 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..2398bc45b19b 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; @@ -64,6 +65,7 @@ public void testUpdatePartitionGranularity() { @TestTemplate public void testPositionDeletesAreMaintainedDuringUpdate() { + assumeThat(formatVersion).isEqualTo(2); // Range distribution will produce partition scoped deletes which will not be cleaned up assumeThat(distributionMode).isNotEqualToIgnoringCase("range"); @@ -74,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( @@ -122,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( @@ -137,6 +149,32 @@ public void testUnpartitionedPositionDeletesAreMaintainedDuringUpdate() { sql("SELECT * FROM %s ORDER BY dep ASC, id ASC", selectTarget())); } + private void checkUpdateFileGranularity(DeleteGranularity deleteGranularity) { + initTable("PARTITIONED BY (dep)", deleteGranularity); + + sql("UPDATE %s SET id = id - 1 WHERE id = 1 OR id = 3", commitTarget()); + + Table table = validationCatalog.loadTable(tableIdent); + assertThat(table.snapshots()).hasSize(5); + + Snapshot currentSnapshot = SnapshotUtil.latestSnapshot(table, branch); + String expectedDeleteFilesCount = deleteGranularity == DeleteGranularity.FILE ? "4" : "2"; + validateMergeOnRead(currentSnapshot, "2", expectedDeleteFilesCount, "2"); + + assertEquals( + "Should have expected rows", + ImmutableList.of( + row(0, "hr"), + row(2, "hr"), + row(2, "hr"), + row(4, "hr"), + row(0, "it"), + row(2, "it"), + row(2, "it"), + row(4, "it")), + sql("SELECT * FROM %s ORDER BY dep ASC, id ASC", selectTarget())); + } + @TestTemplate public void testUpdateWithDVAndHistoricalPositionDeletes() { assumeThat(formatVersion).isEqualTo(2); @@ -186,39 +224,9 @@ public void testUpdateWithDVAndHistoricalPositionDeletes() { assertThat(dvs).allMatch(dv -> FileFormat.fromFileName(dv.location()) == FileFormat.PUFFIN); } - private void checkUpdateFileGranularity(DeleteGranularity deleteGranularity) { - initTable("PARTITIONED BY (dep)", deleteGranularity); - - sql("UPDATE %s SET id = id - 1 WHERE id = 1 OR id = 3", commitTarget()); - - Table table = validationCatalog.loadTable(tableIdent); - assertThat(table.snapshots()).hasSize(5); - - Snapshot currentSnapshot = SnapshotUtil.latestSnapshot(table, branch); - String expectedDeleteFilesCount = deleteGranularity == DeleteGranularity.FILE ? "4" : "2"; - validateMergeOnRead(currentSnapshot, "2", expectedDeleteFilesCount, "2"); - - assertEquals( - "Should have expected rows", - ImmutableList.of( - row(0, "hr"), - row(2, "hr"), - row(2, "hr"), - row(4, "hr"), - row(0, "it"), - row(2, "it"), - row(2, "it"), - row(4, "it")), - sql("SELECT * FROM %s ORDER BY dep ASC, id ASC", selectTarget())); - } - private void initTable(String partitionedBy, DeleteGranularity deleteGranularity) { createTableWithDeleteGranularity("id INT, dep STRING", partitionedBy, deleteGranularity); - sql( - "ALTER TABLE %s SET TBLPROPERTIES ('%s' '%s')", - tableName, TableProperties.DELETE_GRANULARITY, deleteGranularity); - append(tableName, "{ \"id\": 1, \"dep\": \"hr\" }\n" + "{ \"id\": 2, \"dep\": \"hr\" }"); append(tableName, "{ \"id\": 3, \"dep\": \"hr\" }\n" + "{ \"id\": 4, \"dep\": \"hr\" }"); append(tableName, "{ \"id\": 1, \"dep\": \"it\" }\n" + "{ \"id\": 2, \"dep\": \"it\" }"); 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..78f6b80ac948 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; @@ -126,14 +127,16 @@ public void testProcedureWithInvalidInput() { assertThatThrownBy( () -> sql("CALL %s.system.rewrite_table_path('%s')", catalogName, tableIdent)) .isInstanceOf(AnalysisException.class) - .hasMessageContaining("Missing required parameters: [source_prefix,target_prefix]"); + .hasMessageContaining( + "[REQUIRED_PARAMETER_NOT_FOUND] Cannot invoke routine `rewrite_table_path` because the parameter named `source_prefix` is required, but the routine call did not supply a value. Please update the routine call to supply an argument value (either positionally at index 0 or by name) and retry the query again. SQLSTATE: 4274K"); assertThatThrownBy( () -> sql( "CALL %s.system.rewrite_table_path('%s','%s')", catalogName, tableIdent, targetLocation)) .isInstanceOf(AnalysisException.class) - .hasMessageContaining("Missing required parameters: [target_prefix]"); + .hasMessageContaining( + "[REQUIRED_PARAMETER_NOT_FOUND] Cannot invoke routine `rewrite_table_path` because the parameter named `target_prefix` is required, but the routine call did not supply a value. Please update the routine call to supply an argument value (either positionally at index 0 or by name) and retry the query again. SQLSTATE: 4274K"); assertThatThrownBy( () -> sql( @@ -211,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/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..2ebb3ff5b4f2 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; @@ -28,8 +29,8 @@ * A benchmark that evaluates the non-vectorized read and vectorized read with pos-delete in the * Spark data source for Iceberg. * - *

This class uses a dataset with a flat schema. To run this benchmark for spark-3.4: - * ./gradlew -DsparkVersions=3.4 :iceberg-spark:iceberg-spark-3.4:jmh \ + *

This class uses a dataset with a flat schema. To run this benchmark for spark-4.1: + * ./gradlew -DsparkVersions=4.1 :iceberg-spark:iceberg-spark-4.1:jmh \ * -PjmhIncludeRegex=IcebergSourceParquetMultiDeleteFileBenchmark \ * -PjmhOutputPath=benchmark/iceberg-source-parquet-multi-delete-file-benchmark-result.txt * @@ -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..fd837e8e826e 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; @@ -28,8 +29,8 @@ * A benchmark that evaluates the non-vectorized read and vectorized read with pos-delete in the * Spark data source for Iceberg. * - *

This class uses a dataset with a flat schema. To run this benchmark for spark-3.4: - * ./gradlew -DsparkVersions=3.4 :iceberg-spark:iceberg-spark-3.4:jmh + *

This class uses a dataset with a flat schema. To run this benchmark for spark-4.1: + * ./gradlew -DsparkVersions=4.1 :iceberg-spark:iceberg-spark-4.1:jmh * -PjmhIncludeRegex=IcebergSourceParquetPosDeleteBenchmark * -PjmhOutputPath=benchmark/iceberg-source-parquet-pos-delete-benchmark-result.txt * @@ -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..273b3f1cee4d 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; @@ -28,8 +29,8 @@ * A benchmark that evaluates the non-vectorized read and vectorized read with pos-delete in the * Spark data source for Iceberg. * - *

This class uses a dataset with a flat schema. To run this benchmark for spark-3.4: - * ./gradlew -DsparkVersions=3.4 :iceberg-spark:iceberg-spark-3.4:jmh + *

This class uses a dataset with a flat schema. To run this benchmark for spark-4.1: + * ./gradlew -DsparkVersions=4.1 :iceberg-spark:iceberg-spark-4.1:jmh * -PjmhIncludeRegex=IcebergSourceParquetWithUnrelatedDeleteBenchmark * -PjmhOutputPath=benchmark/iceberg-source-parquet-with-unrelated-delete-benchmark-result.txt * @@ -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..c89e4f7852d5 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()); @@ -894,7 +906,6 @@ public void testExpireOlderThanWithDeleteInMergedManifests() { assertThat(secondSnapshot.allManifests(table.io())) .as("Should replace manifest with a rewritten manifest") .hasSize(1); - table .newFastAppend() // do not merge to keep the last snapshot's manifest valid .appendFile(FILE_C) @@ -999,7 +1010,6 @@ public void testExpireOlderThanWithRollbackAndMergedManifests() { Snapshot firstSnapshot = table.currentSnapshot(); assertThat(firstSnapshot.allManifests(table.io())).as("Should create one manifest").hasSize(1); - rightAfterSnapshot(); table.newAppend().appendFile(FILE_B).commit(); @@ -1158,6 +1168,7 @@ public void testExpireAction() { assertThat(table.snapshot(firstSnapshot.snapshotId())) .as("Should remove the oldest snapshot") .isNull(); + assertThat(pending).as("Pending deletes should contain one row").hasSize(1); assertThat(pending.get(0).getPath()) .as("Pending delete should be the expired manifest list 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..d94b83e4c973 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 @@ -22,6 +22,7 @@ import static org.apache.iceberg.data.FileHelpers.encrypt; import static org.apache.iceberg.types.Types.NestedField.optional; import static org.apache.iceberg.types.Types.NestedField.required; +import static org.apache.spark.sql.functions.col; import static org.apache.spark.sql.functions.current_date; import static org.apache.spark.sql.functions.date_add; import static org.apache.spark.sql.functions.expr; @@ -51,6 +52,7 @@ import java.util.stream.IntStream; import java.util.stream.LongStream; import java.util.stream.Stream; +import java.util.stream.StreamSupport; import org.apache.hadoop.conf.Configuration; import org.apache.iceberg.BaseTable; import org.apache.iceberg.ContentFile; @@ -127,6 +129,7 @@ import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; import org.apache.spark.sql.internal.SQLConf; +import org.apache.spark.sql.types.DataTypes; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.TestTemplate; @@ -1719,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); @@ -1798,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); @@ -1828,6 +1831,7 @@ public void testZOrderSort() { @TestTemplate public void testZOrderAllTypesSort() { + spark.conf().set("spark.sql.ansi.enabled", "false"); Table table = createTypeTestTable(); shouldHaveFiles(table, 10); @@ -1863,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); @@ -2096,6 +2100,35 @@ public void testRewriteDataFilesPreservesLineage() throws NoSuchTableException { assertEquals("Rows must match", expectedRecords, actualRecordsWithLineage); } + @TestTemplate + public void testExecutorCacheForDeleteFilesDisabled() { + Table table = createTablePartitioned(1, 1); + RewriteDataFilesSparkAction action = SparkActions.get(spark).rewriteDataFiles(table); + + // The constructor should have set the configuration to false + SparkReadConf readConf = new SparkReadConf(action.spark(), table); + assertThat(readConf.cacheDeleteFilesOnExecutors()) + .as("Executor cache for delete files should be disabled in RewriteDataFilesSparkAction") + .isFalse(); + } + + @TestTemplate + public void testZOrderUDFWithDateType() { + SparkZOrderUDF zorderUDF = new SparkZOrderUDF(1, 16, 1024); + Dataset result = + spark + .sql("SELECT DATE '2025-01-01' as test_col") + .withColumn( + "zorder_result", + zorderUDF.sortedLexicographically(col("test_col"), DataTypes.DateType)); + + assertThat(result.schema().apply("zorder_result").dataType()).isEqualTo(DataTypes.BinaryType); + List rows = result.collectAsList(); + Row row = rows.get(0); + byte[] zorderBytes = row.getAs("zorder_result"); + assertThat(zorderBytes).isNotNull().isNotEmpty(); + } + protected void shouldRewriteDataFilesWithPartitionSpec(Table table, int outputSpecId) { List rewrittenFiles = currentDataFiles(table); assertThat(rewrittenFiles).allMatch(file -> file.specId() == outputSpecId); @@ -2154,8 +2187,10 @@ protected void shouldHaveMultipleFiles(Table table) { protected void shouldHaveFiles(Table table, int numExpected) { table.refresh(); - int numFiles = Iterables.size(table.newScan().planFiles()); - assertThat(numFiles).as("Did not have the expected number of files").isEqualTo(numExpected); + List files = + StreamSupport.stream(table.newScan().planFiles().spliterator(), false) + .collect(Collectors.toList()); + assertThat(files.size()).as("Did not have the expected number of files").isEqualTo(numExpected); } protected long shouldHaveMinSequenceNumberInPartition( @@ -2597,18 +2632,6 @@ private Set cacheContents(Table table) { .build(); } - @TestTemplate - public void testExecutorCacheForDeleteFilesDisabled() { - Table table = createTablePartitioned(1, 1); - RewriteDataFilesSparkAction action = SparkActions.get(spark).rewriteDataFiles(table); - - // The constructor should have set the configuration to false - SparkReadConf readConf = new SparkReadConf(action.spark(), table, Collections.emptyMap()); - assertThat(readConf.cacheDeleteFilesOnExecutors()) - .as("Executor cache for delete files should be disabled in RewriteDataFilesSparkAction") - .isFalse(); - } - private double percentFilesRequired(Table table, String col, String value) { return percentFilesRequired(table, new String[] {col}, new String[] {value}); } 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..09dfaecdf417 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 @@ -19,7 +19,6 @@ package org.apache.iceberg.spark.source; import static org.apache.iceberg.spark.SparkSchemaUtil.convert; -import static org.apache.iceberg.spark.data.TestHelpers.assertEqualsUnsafe; import static org.apache.iceberg.types.Types.NestedField.required; import static org.assertj.core.api.Assertions.assertThat; @@ -28,19 +27,23 @@ import java.nio.file.Path; import java.util.Iterator; import java.util.List; -import org.apache.avro.generic.GenericData; 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; import org.apache.iceberg.avro.Avro; import org.apache.iceberg.avro.AvroIterable; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.data.avro.DataWriter; import org.apache.iceberg.hadoop.HadoopTables; import org.apache.iceberg.io.FileAppender; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.spark.data.RandomData; +import org.apache.iceberg.spark.data.GenericsHelpers; import org.apache.iceberg.spark.data.SparkPlannedAvroReader; import org.apache.iceberg.types.Types; import org.apache.spark.api.java.JavaRDD; @@ -61,7 +64,7 @@ protected boolean supportsDefaultValues() { } @Override - protected void writeRecords(Table table, List records) throws IOException { + protected void writeRecords(Table table, List records) throws IOException { Schema tableSchema = table.schema(); // use the table schema because ids are reassigned Dataset df = createDataset(records, tableSchema); @@ -73,18 +76,19 @@ protected void writeRecords(Table table, List records) throw table.refresh(); } - private Dataset createDataset(List records, Schema schema) - throws IOException { + private Dataset createDataset(List records, Schema schema) throws IOException { // this uses the SparkAvroReader to create a DataFrame from the list of records // it assumes that SparkAvroReader is correct File testFile = File.createTempFile("junit", null, temp.toFile()); assertThat(testFile.delete()).as("Delete should succeed").isTrue(); - try (FileAppender writer = - Avro.write(Files.localOutput(testFile)).schema(schema).named("test").build()) { - for (GenericData.Record rec : records) { - writer.add(rec); - } + try (FileAppender writer = + Avro.write(Files.localOutput(testFile)) + .schema(schema) + .createWriterFunc(DataWriter::create) + .named("test") + .build()) { + writer.addAll(records); } List rows; @@ -98,13 +102,19 @@ private Dataset createDataset(List records, Schema sche // verify that the dataframe matches assertThat(rows).hasSameSizeAs(records); - Iterator recordIter = records.iterator(); + Iterator recordIter = records.iterator(); for (InternalRow row : rows) { - assertEqualsUnsafe(schema.asStruct(), recordIter.next(), row); + GenericsHelpers.assertEqualsUnsafe(schema.asStruct(), recordIter.next(), row); } JavaRDD rdd = sc.parallelize(rows); - return spark.internalCreateDataFrame(JavaRDD.toRDD(rdd), convert(schema), false); + Preconditions.checkArgument( + spark instanceof org.apache.spark.sql.classic.SparkSession, + "Expected instance of org.apache.spark.sql.classic.SparkSession, but got: %s", + spark.getClass().getName()); + + return ((org.apache.spark.sql.classic.SparkSession) spark) + .internalCreateDataFrame(JavaRDD.toRDD(rdd), convert(schema), false); } @Test @@ -123,11 +133,11 @@ public void testAlternateLocation() throws IOException { .set(TableProperties.WRITE_DATA_LOCATION, altLocation.getAbsolutePath()) .commit(); - writeRecords(table, RandomData.generateList(table.schema(), 100, 87112L)); + 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/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..e67ec5fd62d4 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 @@ -24,6 +24,8 @@ import java.io.IOException; import java.math.RoundingMode; +import java.text.SimpleDateFormat; +import java.util.Date; import java.util.List; import java.util.Map; import org.apache.hadoop.conf.Configuration; @@ -35,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; @@ -66,6 +69,8 @@ @ExtendWith(ParameterizedTestExtension.class) public class TestDataSourceOptions extends TestBaseWithCatalog { + private static final SimpleDateFormat TIMESTAMP_FORMAT = + new SimpleDateFormat("yyyy-MM-dd HH:mm:ss.SSS"); private static final Configuration CONF = new Configuration(); private static final Schema SCHEMA = new Schema( @@ -206,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(); @@ -252,29 +257,28 @@ public void testIncrementalScanOptions() throws IOException { spark .read() .format("iceberg") - .option("snapshot-id", snapshotIds.get(3).toString()) + .option(SparkReadOptions.VERSION_AS_OF, snapshotIds.get(3).toString()) .option("start-snapshot-id", snapshotIds.get(3).toString()) .load(tableLocation) .explain()) .isInstanceOf(IllegalArgumentException.class) - .hasMessage( - "Cannot set start-snapshot-id and end-snapshot-id for incremental scans when either snapshot-id or as-of-timestamp is set"); + .hasMessage("Cannot use time travel in incremental scan"); // end-snapshot-id and as-of-timestamp are both configured. + long snapshotTimestamp = table.snapshot(snapshotIds.get(3)).timestampMillis(); + String formattedTimestamp = TIMESTAMP_FORMAT.format(new Date(snapshotTimestamp)); + assertThatThrownBy( () -> spark .read() .format("iceberg") - .option( - SparkReadOptions.AS_OF_TIMESTAMP, - Long.toString(table.snapshot(snapshotIds.get(3)).timestampMillis())) + .option(SparkReadOptions.TIMESTAMP_AS_OF, formattedTimestamp) .option("end-snapshot-id", snapshotIds.get(2).toString()) .load(tableLocation) .explain()) .isInstanceOf(IllegalArgumentException.class) - .hasMessage( - "Cannot set start-snapshot-id and end-snapshot-id for incremental scans when either snapshot-id or as-of-timestamp is set"); + .hasMessage("Cannot use time travel in incremental scan"); // only end-snapshot-id is configured. assertThatThrownBy( @@ -287,7 +291,7 @@ public void testIncrementalScanOptions() throws IOException { .explain()) .isInstanceOf(IllegalArgumentException.class) .hasMessage( - "Cannot set only end-snapshot-id for incremental scans. Please, set start-snapshot-id too."); + "Cannot set only `end-snapshot-id` for incremental scans. Please, set `start-snapshot-id` too."); // test (1st snapshot, current snapshot] incremental scan. Dataset unboundedIncrementalResult = @@ -371,7 +375,7 @@ public void testMetadataSplitSizeOptionOverrideTableProperties() throws IOExcept } @TestTemplate - public void testDefaultMetadataSplitSize() throws IOException { + public void testDefaultMetadataSplitSize() { String tableLocation = temp.resolve("iceberg-table").toFile().toString(); HadoopTables tables = new HadoopTables(CONF); @@ -509,6 +513,8 @@ public void testExtraSnapshotMetadataWithDelete() Table table = validationCatalog.loadTable(tableIdent); List snapshots = Lists.newArrayList(table.snapshots()); + + assertThat(snapshots).hasSize(2); assertThat(snapshots.get(0).summary()).doesNotContainKey("writer-thread"); assertThat(snapshots.get(1).summary()) .containsEntry("writer-thread", "test-extra-commit-message-delete-thread") 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..22f2de120a9e 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 @@ -30,11 +30,13 @@ import java.io.IOException; import java.io.UncheckedIOException; import java.nio.file.Path; +import java.text.SimpleDateFormat; import java.time.LocalDateTime; import java.time.ZoneOffset; import java.util.Arrays; import java.util.Collections; import java.util.Comparator; +import java.util.Date; import java.util.List; import java.util.Map; import java.util.StringJoiner; @@ -51,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; @@ -103,6 +106,9 @@ public abstract class TestIcebergSourceTablesBase extends TestBase { + private static final SimpleDateFormat TIMESTAMP_FORMAT = + new SimpleDateFormat("yyyy-MM-dd HH:mm:ss.SSS"); + private static final Schema SCHEMA = new Schema( optional(1, "id", Types.IntegerType.get()), optional(2, "data", Types.StringType.get())); @@ -257,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( @@ -290,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( @@ -328,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( @@ -411,6 +417,7 @@ public void testAllEntriesTable() throws Exception { assertThat(expected).as("Entries table should have 3 rows").hasSize(3); assertThat(actual).as("Actual results should have 3 rows").hasSize(3); + for (int i = 0; i < expected.size(); i += 1) { TestHelpers.assertEqualsSafe( TestHelpers.nonDerivedSchema(entriesTableDs), expected.get(i), actual.get(i)); @@ -634,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") @@ -1322,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) @@ -1391,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) @@ -1407,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) @@ -1429,7 +1444,7 @@ public void testPartitionsTable() { spark .read() .format("iceberg") - .option(SparkReadOptions.SNAPSHOT_ID, String.valueOf(firstCommitId)) + .option(SparkReadOptions.VERSION_AS_OF, String.valueOf(firstCommitId)) .load(loadLocation(tableIdentifier, "partitions")) .orderBy("partition.id") .collectAsList(); @@ -1446,6 +1461,7 @@ public void testPartitionsTable() { .load(loadLocation(tableIdentifier, "partitions")) .filter("partition.id < 2") .collectAsList(); + assertThat(filtered).as("Actual results should have one row").hasSize(1); TestHelpers.assertEqualsSafe( partitionsTable.schema().asStruct(), expected.get(0), filtered.get(0)); @@ -1457,6 +1473,7 @@ public void testPartitionsTable() { .load(loadLocation(tableIdentifier, "partitions")) .filter("partition.id < 2 or record_count=1") .collectAsList(); + assertThat(nonFiltered).as("Actual results should have two rows").hasSize(2); for (int i = 0; i < 2; i += 1) { TestHelpers.assertEqualsSafe( @@ -1671,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) @@ -1687,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) @@ -1796,7 +1821,7 @@ public synchronized void testSnapshotReadAfterAddColumn() { spark .read() .format("iceberg") - .option(SparkReadOptions.SNAPSHOT_ID, snapshotBeforeAddColumn.snapshotId()) + .option(SparkReadOptions.VERSION_AS_OF, snapshotBeforeAddColumn.snapshotId()) .load(loadLocation(tableIdentifier)); assertThat(resultDf3.orderBy("id").collectAsList()) .as("Records should match") @@ -1835,6 +1860,9 @@ public synchronized void testSnapshotReadAfterDropColumn() { table.updateSchema().deleteColumn("data").commit(); long tsAfterDropColumn = waitUntilAfter(System.currentTimeMillis()); + String formattedTsBeforeDropColumn = TIMESTAMP_FORMAT.format(new Date(tsBeforeDropColumn)); + String formattedTsAfterDropColumn = TIMESTAMP_FORMAT.format(new Date(tsAfterDropColumn)); + List newRecords = Lists.newArrayList(RowFactory.create(4, "B"), RowFactory.create(5, "C")); StructType newSparkSchema = SparkSchemaUtil.convert(SCHEMA3); @@ -1865,7 +1893,7 @@ public synchronized void testSnapshotReadAfterDropColumn() { spark .read() .format("iceberg") - .option(SparkReadOptions.AS_OF_TIMESTAMP, tsBeforeDropColumn) + .option(SparkReadOptions.TIMESTAMP_AS_OF, formattedTsBeforeDropColumn) .load(loadLocation(tableIdentifier)); assertThat(resultDf3.orderBy("id").collectAsList()) .as("Records should match") @@ -1878,7 +1906,7 @@ public synchronized void testSnapshotReadAfterDropColumn() { spark .read() .format("iceberg") - .option(SparkReadOptions.AS_OF_TIMESTAMP, tsAfterDropColumn) + .option(SparkReadOptions.TIMESTAMP_AS_OF, formattedTsAfterDropColumn) .load(loadLocation(tableIdentifier)); assertThat(resultDf4.orderBy("id").collectAsList()) .as("Records should match") @@ -1961,7 +1989,7 @@ public synchronized void testSnapshotReadAfterAddAndDropColumn() { spark .read() .format("iceberg") - .option(SparkReadOptions.SNAPSHOT_ID, snapshotBeforeAddColumn.snapshotId()) + .option(SparkReadOptions.VERSION_AS_OF, snapshotBeforeAddColumn.snapshotId()) .load(loadLocation(tableIdentifier)); assertThat(resultDf4.orderBy("id").collectAsList()) .as("Records should match") @@ -2325,7 +2353,7 @@ public void testSessionConfigSupport() { withSQLConf( // set read option through session configuration - ImmutableMap.of("spark.datasource.iceberg.snapshot-id", String.valueOf(s1)), + ImmutableMap.of("spark.datasource.iceberg.versionAsOf", String.valueOf(s1)), () -> { Dataset result = spark.read().format("iceberg").load(loadLocation(tableIdentifier)); List actual = result.as(Encoders.bean(SimpleRecord.class)).collectAsList(); @@ -2415,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/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..34e9dbc87d02 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,7 +22,9 @@ 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; import org.apache.iceberg.spark.CatalogTestBase; import org.apache.iceberg.spark.SparkCatalogConfig; @@ -50,12 +52,17 @@ public void testRefreshCommand() { // We are not allowed to change the session catalog after it has been initialized, so build a // new one if (Set.of( - SparkCatalogConfig.SPARK.catalogName(), + SparkCatalogConfig.SPARK_SESSION.catalogName(), SparkCatalogConfig.HADOOP.catalogName(), SparkCatalogConfig.REST.catalogName()) .contains(catalogName)) { spark.conf().set("spark.sql.catalog." + catalogName + ".cache-enabled", true); - spark = spark.cloneSession(); + + Preconditions.checkArgument( + spark instanceof org.apache.spark.sql.classic.SparkSession, + "Expected instance of org.apache.spark.sql.classic.SparkSession, but got: %s", + spark.getClass().getName()); + spark = ((org.apache.spark.sql.classic.SparkSession) spark).cloneSession(); } List originalExpected = ImmutableList.of(row(1, 1)); @@ -65,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/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..fa2012d7dce2 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")), @@ -294,7 +297,7 @@ public void testCommitUnknownException() { Table spyTable = spy(table); when(spyTable.newRowDelta()).thenReturn(spyNewRowDelta); SparkTable sparkTable = - branch == null ? new SparkTable(spyTable, false) : new SparkTable(spyTable, branch, false); + branch == null ? new SparkTable(spyTable) : SparkTable.create(spyTable, branch); ImmutableMap config = ImmutableMap.of( @@ -313,6 +316,11 @@ public void testCommitUnknownException() { .isInstanceOf(CommitStateUnknownException.class) .hasMessageStartingWith("Datacenter on Fire"); + // Manually refresh Spark table because it always pins snapshot + sparkTable = branch == null ? new SparkTable(spyTable) : SparkTable.create(spyTable, branch); + TestSparkCatalog.unsetTable(ident); + TestSparkCatalog.setTable(ident, sparkTable); + // Since write and commit succeeded, the rows should be readable assertEquals( "Should have expected rows", 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..78f6b80ac948 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; @@ -126,14 +127,16 @@ public void testProcedureWithInvalidInput() { assertThatThrownBy( () -> sql("CALL %s.system.rewrite_table_path('%s')", catalogName, tableIdent)) .isInstanceOf(AnalysisException.class) - .hasMessageContaining("Missing required parameters: [source_prefix,target_prefix]"); + .hasMessageContaining( + "[REQUIRED_PARAMETER_NOT_FOUND] Cannot invoke routine `rewrite_table_path` because the parameter named `source_prefix` is required, but the routine call did not supply a value. Please update the routine call to supply an argument value (either positionally at index 0 or by name) and retry the query again. SQLSTATE: 4274K"); assertThatThrownBy( () -> sql( "CALL %s.system.rewrite_table_path('%s','%s')", catalogName, tableIdent, targetLocation)) .isInstanceOf(AnalysisException.class) - .hasMessageContaining("Missing required parameters: [target_prefix]"); + .hasMessageContaining( + "[REQUIRED_PARAMETER_NOT_FOUND] Cannot invoke routine `rewrite_table_path` because the parameter named `target_prefix` is required, but the routine call did not supply a value. Please update the routine call to supply an argument value (either positionally at index 0 or by name) and retry the query again. SQLSTATE: 4274K"); assertThatThrownBy( () -> sql( @@ -211,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/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..2ebb3ff5b4f2 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; @@ -28,8 +29,8 @@ * A benchmark that evaluates the non-vectorized read and vectorized read with pos-delete in the * Spark data source for Iceberg. * - *

This class uses a dataset with a flat schema. To run this benchmark for spark-3.5: - * ./gradlew -DsparkVersions=3.5 :iceberg-spark:iceberg-spark-3.5:jmh \ + *

This class uses a dataset with a flat schema. To run this benchmark for spark-4.1: + * ./gradlew -DsparkVersions=4.1 :iceberg-spark:iceberg-spark-4.1:jmh \ * -PjmhIncludeRegex=IcebergSourceParquetMultiDeleteFileBenchmark \ * -PjmhOutputPath=benchmark/iceberg-source-parquet-multi-delete-file-benchmark-result.txt * @@ -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..fd837e8e826e 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; @@ -28,8 +29,8 @@ * A benchmark that evaluates the non-vectorized read and vectorized read with pos-delete in the * Spark data source for Iceberg. * - *

This class uses a dataset with a flat schema. To run this benchmark for spark-3.5: - * ./gradlew -DsparkVersions=3.5 :iceberg-spark:iceberg-spark-3.5:jmh + *

This class uses a dataset with a flat schema. To run this benchmark for spark-4.1: + * ./gradlew -DsparkVersions=4.1 :iceberg-spark:iceberg-spark-4.1:jmh * -PjmhIncludeRegex=IcebergSourceParquetPosDeleteBenchmark * -PjmhOutputPath=benchmark/iceberg-source-parquet-pos-delete-benchmark-result.txt * @@ -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..273b3f1cee4d 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; @@ -28,8 +29,8 @@ * A benchmark that evaluates the non-vectorized read and vectorized read with pos-delete in the * Spark data source for Iceberg. * - *

This class uses a dataset with a flat schema. To run this benchmark for spark-3.5: - * ./gradlew -DsparkVersions=3.5 :iceberg-spark:iceberg-spark-3.5:jmh + *

This class uses a dataset with a flat schema. To run this benchmark for spark-4.1: + * ./gradlew -DsparkVersions=4.1 :iceberg-spark:iceberg-spark-4.1:jmh * -PjmhIncludeRegex=IcebergSourceParquetWithUnrelatedDeleteBenchmark * -PjmhOutputPath=benchmark/iceberg-source-parquet-with-unrelated-delete-benchmark-result.txt * @@ -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..d94b83e4c973 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 @@ -22,6 +22,7 @@ import static org.apache.iceberg.data.FileHelpers.encrypt; import static org.apache.iceberg.types.Types.NestedField.optional; import static org.apache.iceberg.types.Types.NestedField.required; +import static org.apache.spark.sql.functions.col; import static org.apache.spark.sql.functions.current_date; import static org.apache.spark.sql.functions.date_add; import static org.apache.spark.sql.functions.expr; @@ -51,6 +52,7 @@ import java.util.stream.IntStream; import java.util.stream.LongStream; import java.util.stream.Stream; +import java.util.stream.StreamSupport; import org.apache.hadoop.conf.Configuration; import org.apache.iceberg.BaseTable; import org.apache.iceberg.ContentFile; @@ -127,6 +129,7 @@ import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; import org.apache.spark.sql.internal.SQLConf; +import org.apache.spark.sql.types.DataTypes; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.TestTemplate; @@ -1719,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); @@ -1798,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); @@ -1828,6 +1831,7 @@ public void testZOrderSort() { @TestTemplate public void testZOrderAllTypesSort() { + spark.conf().set("spark.sql.ansi.enabled", "false"); Table table = createTypeTestTable(); shouldHaveFiles(table, 10); @@ -1863,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); @@ -2096,6 +2100,35 @@ public void testRewriteDataFilesPreservesLineage() throws NoSuchTableException { assertEquals("Rows must match", expectedRecords, actualRecordsWithLineage); } + @TestTemplate + public void testExecutorCacheForDeleteFilesDisabled() { + Table table = createTablePartitioned(1, 1); + RewriteDataFilesSparkAction action = SparkActions.get(spark).rewriteDataFiles(table); + + // The constructor should have set the configuration to false + SparkReadConf readConf = new SparkReadConf(action.spark(), table); + assertThat(readConf.cacheDeleteFilesOnExecutors()) + .as("Executor cache for delete files should be disabled in RewriteDataFilesSparkAction") + .isFalse(); + } + + @TestTemplate + public void testZOrderUDFWithDateType() { + SparkZOrderUDF zorderUDF = new SparkZOrderUDF(1, 16, 1024); + Dataset result = + spark + .sql("SELECT DATE '2025-01-01' as test_col") + .withColumn( + "zorder_result", + zorderUDF.sortedLexicographically(col("test_col"), DataTypes.DateType)); + + assertThat(result.schema().apply("zorder_result").dataType()).isEqualTo(DataTypes.BinaryType); + List rows = result.collectAsList(); + Row row = rows.get(0); + byte[] zorderBytes = row.getAs("zorder_result"); + assertThat(zorderBytes).isNotNull().isNotEmpty(); + } + protected void shouldRewriteDataFilesWithPartitionSpec(Table table, int outputSpecId) { List rewrittenFiles = currentDataFiles(table); assertThat(rewrittenFiles).allMatch(file -> file.specId() == outputSpecId); @@ -2154,8 +2187,10 @@ protected void shouldHaveMultipleFiles(Table table) { protected void shouldHaveFiles(Table table, int numExpected) { table.refresh(); - int numFiles = Iterables.size(table.newScan().planFiles()); - assertThat(numFiles).as("Did not have the expected number of files").isEqualTo(numExpected); + List files = + StreamSupport.stream(table.newScan().planFiles().spliterator(), false) + .collect(Collectors.toList()); + assertThat(files.size()).as("Did not have the expected number of files").isEqualTo(numExpected); } protected long shouldHaveMinSequenceNumberInPartition( @@ -2597,18 +2632,6 @@ private Set cacheContents(Table table) { .build(); } - @TestTemplate - public void testExecutorCacheForDeleteFilesDisabled() { - Table table = createTablePartitioned(1, 1); - RewriteDataFilesSparkAction action = SparkActions.get(spark).rewriteDataFiles(table); - - // The constructor should have set the configuration to false - SparkReadConf readConf = new SparkReadConf(action.spark(), table, Collections.emptyMap()); - assertThat(readConf.cacheDeleteFilesOnExecutors()) - .as("Executor cache for delete files should be disabled in RewriteDataFilesSparkAction") - .isFalse(); - } - private double percentFilesRequired(Table table, String col, String value) { return percentFilesRequired(table, new String[] {col}, new String[] {value}); } 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..09dfaecdf417 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 @@ -19,7 +19,6 @@ package org.apache.iceberg.spark.source; import static org.apache.iceberg.spark.SparkSchemaUtil.convert; -import static org.apache.iceberg.spark.data.TestHelpers.assertEqualsUnsafe; import static org.apache.iceberg.types.Types.NestedField.required; import static org.assertj.core.api.Assertions.assertThat; @@ -28,19 +27,23 @@ import java.nio.file.Path; import java.util.Iterator; import java.util.List; -import org.apache.avro.generic.GenericData; 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; import org.apache.iceberg.avro.Avro; import org.apache.iceberg.avro.AvroIterable; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.data.avro.DataWriter; import org.apache.iceberg.hadoop.HadoopTables; import org.apache.iceberg.io.FileAppender; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.spark.data.RandomData; +import org.apache.iceberg.spark.data.GenericsHelpers; import org.apache.iceberg.spark.data.SparkPlannedAvroReader; import org.apache.iceberg.types.Types; import org.apache.spark.api.java.JavaRDD; @@ -61,7 +64,7 @@ protected boolean supportsDefaultValues() { } @Override - protected void writeRecords(Table table, List records) throws IOException { + protected void writeRecords(Table table, List records) throws IOException { Schema tableSchema = table.schema(); // use the table schema because ids are reassigned Dataset df = createDataset(records, tableSchema); @@ -73,18 +76,19 @@ protected void writeRecords(Table table, List records) throw table.refresh(); } - private Dataset createDataset(List records, Schema schema) - throws IOException { + private Dataset createDataset(List records, Schema schema) throws IOException { // this uses the SparkAvroReader to create a DataFrame from the list of records // it assumes that SparkAvroReader is correct File testFile = File.createTempFile("junit", null, temp.toFile()); assertThat(testFile.delete()).as("Delete should succeed").isTrue(); - try (FileAppender writer = - Avro.write(Files.localOutput(testFile)).schema(schema).named("test").build()) { - for (GenericData.Record rec : records) { - writer.add(rec); - } + try (FileAppender writer = + Avro.write(Files.localOutput(testFile)) + .schema(schema) + .createWriterFunc(DataWriter::create) + .named("test") + .build()) { + writer.addAll(records); } List rows; @@ -98,13 +102,19 @@ private Dataset createDataset(List records, Schema sche // verify that the dataframe matches assertThat(rows).hasSameSizeAs(records); - Iterator recordIter = records.iterator(); + Iterator recordIter = records.iterator(); for (InternalRow row : rows) { - assertEqualsUnsafe(schema.asStruct(), recordIter.next(), row); + GenericsHelpers.assertEqualsUnsafe(schema.asStruct(), recordIter.next(), row); } JavaRDD rdd = sc.parallelize(rows); - return spark.internalCreateDataFrame(JavaRDD.toRDD(rdd), convert(schema), false); + Preconditions.checkArgument( + spark instanceof org.apache.spark.sql.classic.SparkSession, + "Expected instance of org.apache.spark.sql.classic.SparkSession, but got: %s", + spark.getClass().getName()); + + return ((org.apache.spark.sql.classic.SparkSession) spark) + .internalCreateDataFrame(JavaRDD.toRDD(rdd), convert(schema), false); } @Test @@ -123,11 +133,11 @@ public void testAlternateLocation() throws IOException { .set(TableProperties.WRITE_DATA_LOCATION, altLocation.getAbsolutePath()) .commit(); - writeRecords(table, RandomData.generateList(table.schema(), 100, 87112L)); + 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/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..e67ec5fd62d4 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 @@ -24,6 +24,8 @@ import java.io.IOException; import java.math.RoundingMode; +import java.text.SimpleDateFormat; +import java.util.Date; import java.util.List; import java.util.Map; import org.apache.hadoop.conf.Configuration; @@ -35,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; @@ -66,6 +69,8 @@ @ExtendWith(ParameterizedTestExtension.class) public class TestDataSourceOptions extends TestBaseWithCatalog { + private static final SimpleDateFormat TIMESTAMP_FORMAT = + new SimpleDateFormat("yyyy-MM-dd HH:mm:ss.SSS"); private static final Configuration CONF = new Configuration(); private static final Schema SCHEMA = new Schema( @@ -206,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(); @@ -252,29 +257,28 @@ public void testIncrementalScanOptions() throws IOException { spark .read() .format("iceberg") - .option("snapshot-id", snapshotIds.get(3).toString()) + .option(SparkReadOptions.VERSION_AS_OF, snapshotIds.get(3).toString()) .option("start-snapshot-id", snapshotIds.get(3).toString()) .load(tableLocation) .explain()) .isInstanceOf(IllegalArgumentException.class) - .hasMessage( - "Cannot set start-snapshot-id and end-snapshot-id for incremental scans when either snapshot-id or as-of-timestamp is set"); + .hasMessage("Cannot use time travel in incremental scan"); // end-snapshot-id and as-of-timestamp are both configured. + long snapshotTimestamp = table.snapshot(snapshotIds.get(3)).timestampMillis(); + String formattedTimestamp = TIMESTAMP_FORMAT.format(new Date(snapshotTimestamp)); + assertThatThrownBy( () -> spark .read() .format("iceberg") - .option( - SparkReadOptions.AS_OF_TIMESTAMP, - Long.toString(table.snapshot(snapshotIds.get(3)).timestampMillis())) + .option(SparkReadOptions.TIMESTAMP_AS_OF, formattedTimestamp) .option("end-snapshot-id", snapshotIds.get(2).toString()) .load(tableLocation) .explain()) .isInstanceOf(IllegalArgumentException.class) - .hasMessage( - "Cannot set start-snapshot-id and end-snapshot-id for incremental scans when either snapshot-id or as-of-timestamp is set"); + .hasMessage("Cannot use time travel in incremental scan"); // only end-snapshot-id is configured. assertThatThrownBy( @@ -287,7 +291,7 @@ public void testIncrementalScanOptions() throws IOException { .explain()) .isInstanceOf(IllegalArgumentException.class) .hasMessage( - "Cannot set only end-snapshot-id for incremental scans. Please, set start-snapshot-id too."); + "Cannot set only `end-snapshot-id` for incremental scans. Please, set `start-snapshot-id` too."); // test (1st snapshot, current snapshot] incremental scan. Dataset unboundedIncrementalResult = 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..22f2de120a9e 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 @@ -30,11 +30,13 @@ import java.io.IOException; import java.io.UncheckedIOException; import java.nio.file.Path; +import java.text.SimpleDateFormat; import java.time.LocalDateTime; import java.time.ZoneOffset; import java.util.Arrays; import java.util.Collections; import java.util.Comparator; +import java.util.Date; import java.util.List; import java.util.Map; import java.util.StringJoiner; @@ -51,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; @@ -103,6 +106,9 @@ public abstract class TestIcebergSourceTablesBase extends TestBase { + private static final SimpleDateFormat TIMESTAMP_FORMAT = + new SimpleDateFormat("yyyy-MM-dd HH:mm:ss.SSS"); + private static final Schema SCHEMA = new Schema( optional(1, "id", Types.IntegerType.get()), optional(2, "data", Types.StringType.get())); @@ -257,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( @@ -290,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( @@ -328,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( @@ -635,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") @@ -1323,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) @@ -1392,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) @@ -1408,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) @@ -1430,7 +1444,7 @@ public void testPartitionsTable() { spark .read() .format("iceberg") - .option(SparkReadOptions.SNAPSHOT_ID, String.valueOf(firstCommitId)) + .option(SparkReadOptions.VERSION_AS_OF, String.valueOf(firstCommitId)) .load(loadLocation(tableIdentifier, "partitions")) .orderBy("partition.id") .collectAsList(); @@ -1674,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) @@ -1690,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) @@ -1799,7 +1821,7 @@ public synchronized void testSnapshotReadAfterAddColumn() { spark .read() .format("iceberg") - .option(SparkReadOptions.SNAPSHOT_ID, snapshotBeforeAddColumn.snapshotId()) + .option(SparkReadOptions.VERSION_AS_OF, snapshotBeforeAddColumn.snapshotId()) .load(loadLocation(tableIdentifier)); assertThat(resultDf3.orderBy("id").collectAsList()) .as("Records should match") @@ -1838,6 +1860,9 @@ public synchronized void testSnapshotReadAfterDropColumn() { table.updateSchema().deleteColumn("data").commit(); long tsAfterDropColumn = waitUntilAfter(System.currentTimeMillis()); + String formattedTsBeforeDropColumn = TIMESTAMP_FORMAT.format(new Date(tsBeforeDropColumn)); + String formattedTsAfterDropColumn = TIMESTAMP_FORMAT.format(new Date(tsAfterDropColumn)); + List newRecords = Lists.newArrayList(RowFactory.create(4, "B"), RowFactory.create(5, "C")); StructType newSparkSchema = SparkSchemaUtil.convert(SCHEMA3); @@ -1868,7 +1893,7 @@ public synchronized void testSnapshotReadAfterDropColumn() { spark .read() .format("iceberg") - .option(SparkReadOptions.AS_OF_TIMESTAMP, tsBeforeDropColumn) + .option(SparkReadOptions.TIMESTAMP_AS_OF, formattedTsBeforeDropColumn) .load(loadLocation(tableIdentifier)); assertThat(resultDf3.orderBy("id").collectAsList()) .as("Records should match") @@ -1881,7 +1906,7 @@ public synchronized void testSnapshotReadAfterDropColumn() { spark .read() .format("iceberg") - .option(SparkReadOptions.AS_OF_TIMESTAMP, tsAfterDropColumn) + .option(SparkReadOptions.TIMESTAMP_AS_OF, formattedTsAfterDropColumn) .load(loadLocation(tableIdentifier)); assertThat(resultDf4.orderBy("id").collectAsList()) .as("Records should match") @@ -1964,7 +1989,7 @@ public synchronized void testSnapshotReadAfterAddAndDropColumn() { spark .read() .format("iceberg") - .option(SparkReadOptions.SNAPSHOT_ID, snapshotBeforeAddColumn.snapshotId()) + .option(SparkReadOptions.VERSION_AS_OF, snapshotBeforeAddColumn.snapshotId()) .load(loadLocation(tableIdentifier)); assertThat(resultDf4.orderBy("id").collectAsList()) .as("Records should match") @@ -2328,7 +2353,7 @@ public void testSessionConfigSupport() { withSQLConf( // set read option through session configuration - ImmutableMap.of("spark.datasource.iceberg.snapshot-id", String.valueOf(s1)), + ImmutableMap.of("spark.datasource.iceberg.versionAsOf", String.valueOf(s1)), () -> { Dataset result = spark.read().format("iceberg").load(loadLocation(tableIdentifier)); List actual = result.as(Encoders.bean(SimpleRecord.class)).collectAsList(); @@ -2418,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/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..34e9dbc87d02 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,7 +22,9 @@ 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; import org.apache.iceberg.spark.CatalogTestBase; import org.apache.iceberg.spark.SparkCatalogConfig; @@ -55,7 +57,12 @@ public void testRefreshCommand() { SparkCatalogConfig.REST.catalogName()) .contains(catalogName)) { spark.conf().set("spark.sql.catalog." + catalogName + ".cache-enabled", true); - spark = spark.cloneSession(); + + Preconditions.checkArgument( + spark instanceof org.apache.spark.sql.classic.SparkSession, + "Expected instance of org.apache.spark.sql.classic.SparkSession, but got: %s", + spark.getClass().getName()); + spark = ((org.apache.spark.sql.classic.SparkSession) spark).cloneSession(); } List originalExpected = ImmutableList.of(row(1, 1)); @@ -65,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.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..fa2012d7dce2 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")), @@ -294,7 +297,7 @@ public void testCommitUnknownException() { Table spyTable = spy(table); when(spyTable.newRowDelta()).thenReturn(spyNewRowDelta); SparkTable sparkTable = - branch == null ? new SparkTable(spyTable, false) : new SparkTable(spyTable, branch, false); + branch == null ? new SparkTable(spyTable) : SparkTable.create(spyTable, branch); ImmutableMap config = ImmutableMap.of( @@ -313,6 +316,11 @@ public void testCommitUnknownException() { .isInstanceOf(CommitStateUnknownException.class) .hasMessageStartingWith("Datacenter on Fire"); + // Manually refresh Spark table because it always pins snapshot + sparkTable = branch == null ? new SparkTable(spyTable) : SparkTable.create(spyTable, branch); + TestSparkCatalog.unsetTable(ident); + TestSparkCatalog.setTable(ident, sparkTable); + // Since write and commit succeeded, the rows should be readable assertEquals( "Should have expected rows", 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..2ebb3ff5b4f2 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; @@ -28,8 +29,8 @@ * A benchmark that evaluates the non-vectorized read and vectorized read with pos-delete in the * Spark data source for Iceberg. * - *

This class uses a dataset with a flat schema. To run this benchmark for spark-4.0: - * ./gradlew -DsparkVersions=4.0 :iceberg-spark:iceberg-spark-4.0:jmh \ + *

This class uses a dataset with a flat schema. To run this benchmark for spark-4.1: + * ./gradlew -DsparkVersions=4.1 :iceberg-spark:iceberg-spark-4.1:jmh \ * -PjmhIncludeRegex=IcebergSourceParquetMultiDeleteFileBenchmark \ * -PjmhOutputPath=benchmark/iceberg-source-parquet-multi-delete-file-benchmark-result.txt * @@ -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..fd837e8e826e 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; @@ -28,8 +29,8 @@ * A benchmark that evaluates the non-vectorized read and vectorized read with pos-delete in the * Spark data source for Iceberg. * - *

This class uses a dataset with a flat schema. To run this benchmark for spark-4.0: - * ./gradlew -DsparkVersions=4.0 :iceberg-spark:iceberg-spark-4.0:jmh + *

This class uses a dataset with a flat schema. To run this benchmark for spark-4.1: + * ./gradlew -DsparkVersions=4.1 :iceberg-spark:iceberg-spark-4.1:jmh * -PjmhIncludeRegex=IcebergSourceParquetPosDeleteBenchmark * -PjmhOutputPath=benchmark/iceberg-source-parquet-pos-delete-benchmark-result.txt * @@ -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..273b3f1cee4d 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; @@ -28,8 +29,8 @@ * A benchmark that evaluates the non-vectorized read and vectorized read with pos-delete in the * Spark data source for Iceberg. * - *

This class uses a dataset with a flat schema. To run this benchmark for spark-4.0: - * ./gradlew -DsparkVersions=4.0 :iceberg-spark:iceberg-spark-4.0:jmh + *

This class uses a dataset with a flat schema. To run this benchmark for spark-4.1: + * ./gradlew -DsparkVersions=4.1 :iceberg-spark:iceberg-spark-4.1:jmh * -PjmhIncludeRegex=IcebergSourceParquetWithUnrelatedDeleteBenchmark * -PjmhOutputPath=benchmark/iceberg-source-parquet-with-unrelated-delete-benchmark-result.txt * @@ -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..d94b83e4c973 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); @@ -2106,7 +2106,7 @@ public void testExecutorCacheForDeleteFilesDisabled() { RewriteDataFilesSparkAction action = SparkActions.get(spark).rewriteDataFiles(table); // The constructor should have set the configuration to false - SparkReadConf readConf = new SparkReadConf(action.spark(), table, Collections.emptyMap()); + SparkReadConf readConf = new SparkReadConf(action.spark(), table); assertThat(readConf.cacheDeleteFilesOnExecutors()) .as("Executor cache for delete files should be disabled in RewriteDataFilesSparkAction") .isFalse(); 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..e67ec5fd62d4 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 @@ -24,6 +24,8 @@ import java.io.IOException; import java.math.RoundingMode; +import java.text.SimpleDateFormat; +import java.util.Date; import java.util.List; import java.util.Map; import org.apache.hadoop.conf.Configuration; @@ -35,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; @@ -66,6 +69,8 @@ @ExtendWith(ParameterizedTestExtension.class) public class TestDataSourceOptions extends TestBaseWithCatalog { + private static final SimpleDateFormat TIMESTAMP_FORMAT = + new SimpleDateFormat("yyyy-MM-dd HH:mm:ss.SSS"); private static final Configuration CONF = new Configuration(); private static final Schema SCHEMA = new Schema( @@ -206,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(); @@ -252,29 +257,28 @@ public void testIncrementalScanOptions() throws IOException { spark .read() .format("iceberg") - .option("snapshot-id", snapshotIds.get(3).toString()) + .option(SparkReadOptions.VERSION_AS_OF, snapshotIds.get(3).toString()) .option("start-snapshot-id", snapshotIds.get(3).toString()) .load(tableLocation) .explain()) .isInstanceOf(IllegalArgumentException.class) - .hasMessage( - "Cannot set start-snapshot-id and end-snapshot-id for incremental scans when either snapshot-id or as-of-timestamp is set"); + .hasMessage("Cannot use time travel in incremental scan"); // end-snapshot-id and as-of-timestamp are both configured. + long snapshotTimestamp = table.snapshot(snapshotIds.get(3)).timestampMillis(); + String formattedTimestamp = TIMESTAMP_FORMAT.format(new Date(snapshotTimestamp)); + assertThatThrownBy( () -> spark .read() .format("iceberg") - .option( - SparkReadOptions.AS_OF_TIMESTAMP, - Long.toString(table.snapshot(snapshotIds.get(3)).timestampMillis())) + .option(SparkReadOptions.TIMESTAMP_AS_OF, formattedTimestamp) .option("end-snapshot-id", snapshotIds.get(2).toString()) .load(tableLocation) .explain()) .isInstanceOf(IllegalArgumentException.class) - .hasMessage( - "Cannot set start-snapshot-id and end-snapshot-id for incremental scans when either snapshot-id or as-of-timestamp is set"); + .hasMessage("Cannot use time travel in incremental scan"); // only end-snapshot-id is configured. assertThatThrownBy( @@ -287,7 +291,7 @@ public void testIncrementalScanOptions() throws IOException { .explain()) .isInstanceOf(IllegalArgumentException.class) .hasMessage( - "Cannot set only end-snapshot-id for incremental scans. Please, set start-snapshot-id too."); + "Cannot set only `end-snapshot-id` for incremental scans. Please, set `start-snapshot-id` too."); // test (1st snapshot, current snapshot] incremental scan. Dataset unboundedIncrementalResult = 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..22f2de120a9e 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 @@ -30,11 +30,13 @@ import java.io.IOException; import java.io.UncheckedIOException; import java.nio.file.Path; +import java.text.SimpleDateFormat; import java.time.LocalDateTime; import java.time.ZoneOffset; import java.util.Arrays; import java.util.Collections; import java.util.Comparator; +import java.util.Date; import java.util.List; import java.util.Map; import java.util.StringJoiner; @@ -51,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; @@ -103,6 +106,9 @@ public abstract class TestIcebergSourceTablesBase extends TestBase { + private static final SimpleDateFormat TIMESTAMP_FORMAT = + new SimpleDateFormat("yyyy-MM-dd HH:mm:ss.SSS"); + private static final Schema SCHEMA = new Schema( optional(1, "id", Types.IntegerType.get()), optional(2, "data", Types.StringType.get())); @@ -257,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( @@ -290,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( @@ -328,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( @@ -635,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") @@ -1323,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) @@ -1392,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) @@ -1408,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) @@ -1430,7 +1444,7 @@ public void testPartitionsTable() { spark .read() .format("iceberg") - .option(SparkReadOptions.SNAPSHOT_ID, String.valueOf(firstCommitId)) + .option(SparkReadOptions.VERSION_AS_OF, String.valueOf(firstCommitId)) .load(loadLocation(tableIdentifier, "partitions")) .orderBy("partition.id") .collectAsList(); @@ -1674,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) @@ -1690,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) @@ -1799,7 +1821,7 @@ public synchronized void testSnapshotReadAfterAddColumn() { spark .read() .format("iceberg") - .option(SparkReadOptions.SNAPSHOT_ID, snapshotBeforeAddColumn.snapshotId()) + .option(SparkReadOptions.VERSION_AS_OF, snapshotBeforeAddColumn.snapshotId()) .load(loadLocation(tableIdentifier)); assertThat(resultDf3.orderBy("id").collectAsList()) .as("Records should match") @@ -1838,6 +1860,9 @@ public synchronized void testSnapshotReadAfterDropColumn() { table.updateSchema().deleteColumn("data").commit(); long tsAfterDropColumn = waitUntilAfter(System.currentTimeMillis()); + String formattedTsBeforeDropColumn = TIMESTAMP_FORMAT.format(new Date(tsBeforeDropColumn)); + String formattedTsAfterDropColumn = TIMESTAMP_FORMAT.format(new Date(tsAfterDropColumn)); + List newRecords = Lists.newArrayList(RowFactory.create(4, "B"), RowFactory.create(5, "C")); StructType newSparkSchema = SparkSchemaUtil.convert(SCHEMA3); @@ -1868,7 +1893,7 @@ public synchronized void testSnapshotReadAfterDropColumn() { spark .read() .format("iceberg") - .option(SparkReadOptions.AS_OF_TIMESTAMP, tsBeforeDropColumn) + .option(SparkReadOptions.TIMESTAMP_AS_OF, formattedTsBeforeDropColumn) .load(loadLocation(tableIdentifier)); assertThat(resultDf3.orderBy("id").collectAsList()) .as("Records should match") @@ -1881,7 +1906,7 @@ public synchronized void testSnapshotReadAfterDropColumn() { spark .read() .format("iceberg") - .option(SparkReadOptions.AS_OF_TIMESTAMP, tsAfterDropColumn) + .option(SparkReadOptions.TIMESTAMP_AS_OF, formattedTsAfterDropColumn) .load(loadLocation(tableIdentifier)); assertThat(resultDf4.orderBy("id").collectAsList()) .as("Records should match") @@ -1964,7 +1989,7 @@ public synchronized void testSnapshotReadAfterAddAndDropColumn() { spark .read() .format("iceberg") - .option(SparkReadOptions.SNAPSHOT_ID, snapshotBeforeAddColumn.snapshotId()) + .option(SparkReadOptions.VERSION_AS_OF, snapshotBeforeAddColumn.snapshotId()) .load(loadLocation(tableIdentifier)); assertThat(resultDf4.orderBy("id").collectAsList()) .as("Records should match") @@ -2328,7 +2353,7 @@ public void testSessionConfigSupport() { withSQLConf( // set read option through session configuration - ImmutableMap.of("spark.datasource.iceberg.snapshot-id", String.valueOf(s1)), + ImmutableMap.of("spark.datasource.iceberg.versionAsOf", String.valueOf(s1)), () -> { Dataset result = spark.read().format("iceberg").load(loadLocation(tableIdentifier)); List actual = result.as(Encoders.bean(SimpleRecord.class)).collectAsList(); @@ -2418,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.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); From b94fa93c2258d645e17e4eda49e9302cbe0fc4fc Mon Sep 17 00:00:00 2001 From: Russell Spitzer Date: Mon, 16 Mar 2026 19:56:12 -0500 Subject: [PATCH 2/6] Remove dead newFilesBetween(TableMetadata) overload and inline snapshotsBetween helper --- .../org/apache/iceberg/util/SnapshotUtil.java | 24 +++---------------- 1 file changed, 3 insertions(+), 21 deletions(-) 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 d3d3c84ee3f8..370bbfed336e 100644 --- a/core/src/main/java/org/apache/iceberg/util/SnapshotUtil.java +++ b/core/src/main/java/org/apache/iceberg/util/SnapshotUtil.java @@ -318,27 +318,7 @@ public static List newFiles( @Deprecated public static CloseableIterable newFilesBetween( Long startSnapshotId, long endSnapshotId, Function lookup, FileIO io) { - List snapshots = snapshotsBetween(startSnapshotId, endSnapshotId, lookup); - return new ParallelIterable<>( - Iterables.transform(snapshots, snapshot -> snapshot.addedDataFiles(io)), - ThreadPools.getWorkerPool()); - } - - /** - * @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, TableMetadata metadata, FileIO io) { - List snapshots = snapshotsBetween(startSnapshotId, endSnapshotId, metadata::snapshot); - return new ParallelIterable<>( - Iterables.transform(snapshots, snapshot -> snapshot.addedDataFiles(io)), - ThreadPools.getWorkerPool()); - } - private static List snapshotsBetween( - Long startSnapshotId, long endSnapshotId, Function lookup) { List snapshots = Lists.newArrayList(); Snapshot lastSnapshot = null; for (Snapshot currentSnapshot : ancestorsOf(endSnapshotId, lookup)) { @@ -359,7 +339,9 @@ private static List snapshotsBetween( lastSnapshot.snapshotId()); } - return snapshots; + return new ParallelIterable<>( + Iterables.transform(snapshots, snapshot -> snapshot.addedDataFiles(io)), + ThreadPools.getWorkerPool()); } /** From f0820e85c01b139bcf441e09554624e081e5ae22 Mon Sep 17 00:00:00 2001 From: Russell Spitzer Date: Mon, 16 Mar 2026 20:12:41 -0500 Subject: [PATCH 3/6] Fix 31 test files that were overwritten with latest version content instead of targeted migration Older Spark/Flink version test files (v3.4, v3.5, v4.0, flink v1.20) were accidentally replaced with the content of the latest version (v4.1, v2.1) instead of having only the SnapshotChanges migration applied in place. Restored each file from main and re-applied only the migration changes. --- .../operator/TestMonitorSource.java | 24 +++----- .../spark/extensions/TestBranchDDL.java | 6 +- .../extensions/TestMergeOnReadDelete.java | 11 ++-- .../extensions/TestMergeOnReadUpdate.java | 57 ++++++++++--------- .../TestRewriteTablePathProcedure.java | 6 +- ...SourceParquetMultiDeleteFileBenchmark.java | 4 +- ...cebergSourceParquetPosDeleteBenchmark.java | 4 +- ...ceParquetWithUnrelatedDeleteBenchmark.java | 4 +- .../actions/TestExpireSnapshotsAction.java | 3 +- .../actions/TestRewriteDataFilesAction.java | 51 +++++------------ .../spark/source/DataFrameWriteTestBase.java | 39 +++++-------- .../spark/source/TestDataSourceOptions.java | 25 ++++---- .../source/TestIcebergSourceTablesBase.java | 23 ++------ .../iceberg/spark/sql/TestRefreshTable.java | 10 +--- .../extensions/TestMergeOnReadDelete.java | 7 +-- .../TestRewriteTablePathProcedure.java | 6 +- ...SourceParquetMultiDeleteFileBenchmark.java | 4 +- ...cebergSourceParquetPosDeleteBenchmark.java | 4 +- ...ceParquetWithUnrelatedDeleteBenchmark.java | 4 +- .../actions/TestRewriteDataFilesAction.java | 51 +++++------------ .../spark/source/DataFrameWriteTestBase.java | 39 +++++-------- .../spark/source/TestDataSourceOptions.java | 21 +++---- .../source/TestIcebergSourceTablesBase.java | 20 ++----- .../iceberg/spark/sql/TestRefreshTable.java | 8 +-- .../extensions/TestMergeOnReadDelete.java | 7 +-- ...SourceParquetMultiDeleteFileBenchmark.java | 4 +- ...cebergSourceParquetPosDeleteBenchmark.java | 4 +- ...ceParquetWithUnrelatedDeleteBenchmark.java | 4 +- .../actions/TestRewriteDataFilesAction.java | 2 +- .../spark/source/TestDataSourceOptions.java | 21 +++---- .../source/TestIcebergSourceTablesBase.java | 20 ++----- 31 files changed, 180 insertions(+), 313 deletions(-) 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 84090e9e6b7b..f89c92cbf22e 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 @@ -25,21 +25,15 @@ import java.io.IOException; import java.time.Duration; import java.util.List; -import java.util.concurrent.CompletableFuture; import java.util.concurrent.atomic.AtomicReference; -import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.eventtime.WatermarkStrategy; import org.apache.flink.api.connector.source.util.ratelimit.RateLimiterStrategy; -import org.apache.flink.client.program.ClusterClient; import org.apache.flink.configuration.CheckpointingOptions; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.execution.JobClient; import org.apache.flink.runtime.client.JobExecutionException; -import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.api.graph.StreamGraph; -import org.apache.flink.test.junit5.InjectClusterClient; import org.apache.hadoop.fs.Path; import org.apache.iceberg.ContentFile; import org.apache.iceberg.DataFile; @@ -164,9 +158,7 @@ void testSource() throws Exception { /** Check that the {@link MonitorSource} operator state is restored correctly. */ @Test - void testStateRestore( - @TempDir File savepointDir, @InjectClusterClient ClusterClient clusterClient) - throws Exception { + void testStateRestore(@TempDir File savepointDir) throws Exception { Table table = createTable(); insert(table, 1, "a"); TableLoader tableLoader = tableLoader(); @@ -189,7 +181,7 @@ void testStateRestore( events.sinkTo(result); // Start the job - String savepointPath; + Configuration conf; JobClient jobClient = null; AtomicReference firstNonEmptyEvent = new AtomicReference<>(); try { @@ -210,11 +202,11 @@ void testStateRestore( }); } finally { // Stop with savepoint - savepointPath = closeJobClient(jobClient, savepointDir); + conf = closeJobClient(jobClient, savepointDir); } // Restore from savepoint, create the same topology with a different env - env = StreamExecutionEnvironment.getExecutionEnvironment(); + env = StreamExecutionEnvironment.getExecutionEnvironment(conf); events = env.fromSource( new MonitorSource(tableLoader, LOW_RATE, Long.MAX_VALUE), @@ -225,13 +217,13 @@ void testStateRestore( events.sinkTo(resultWithSavepoint); // Make sure that the job with restored source does not read new records from the table - StreamGraph streamGraph = env.getStreamGraph(); - streamGraph.setSavepointRestoreSettings(SavepointRestoreSettings.forPath(savepointPath)); - CompletableFuture jobIDCompletableFuture = clusterClient.submitJob(streamGraph); + JobClient clientWithSavepoint = null; try { + clientWithSavepoint = env.executeAsync("Table Change Source test with savepoint"); + assertThat(resultWithSavepoint.poll(Duration.ofSeconds(5L))).isEqualTo(EMPTY_EVENT); } finally { - clusterClient.cancel(jobIDCompletableFuture.get()); + closeJobClient(clientWithSavepoint, null); } // Restore without savepoint 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 26dde2059966..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 @@ -58,9 +58,9 @@ public void removeTable() { public static Object[][] parameters() { return new Object[][] { { - SparkCatalogConfig.SPARK_SESSION.catalogName(), - SparkCatalogConfig.SPARK_SESSION.implementation(), - SparkCatalogConfig.SPARK_SESSION.properties() + SparkCatalogConfig.SPARK.catalogName(), + SparkCatalogConfig.SPARK.implementation(), + SparkCatalogConfig.SPARK.properties() } }; } 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 fa2012d7dce2..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 @@ -244,6 +244,10 @@ private void checkDeleteFileGranularity(DeleteGranularity deleteGranularity) createTableWithDeleteGranularity( "id INT, dep STRING", "PARTITIONED BY (dep)", deleteGranularity); + sql( + "ALTER TABLE %s SET TBLPROPERTIES ('%s' '%s')", + tableName, TableProperties.DELETE_GRANULARITY, deleteGranularity); + append(tableName, new Employee(1, "hr"), new Employee(2, "hr")); append(tableName, new Employee(3, "hr"), new Employee(4, "hr")); append(tableName, new Employee(1, "hardware"), new Employee(2, "hardware")); @@ -297,7 +301,7 @@ public void testCommitUnknownException() { Table spyTable = spy(table); when(spyTable.newRowDelta()).thenReturn(spyNewRowDelta); SparkTable sparkTable = - branch == null ? new SparkTable(spyTable) : SparkTable.create(spyTable, branch); + branch == null ? new SparkTable(spyTable, false) : new SparkTable(spyTable, branch, false); ImmutableMap config = ImmutableMap.of( @@ -316,11 +320,6 @@ public void testCommitUnknownException() { .isInstanceOf(CommitStateUnknownException.class) .hasMessageStartingWith("Datacenter on Fire"); - // Manually refresh Spark table because it always pins snapshot - sparkTable = branch == null ? new SparkTable(spyTable) : SparkTable.create(spyTable, branch); - TestSparkCatalog.unsetTable(ident); - TestSparkCatalog.setTable(ident, sparkTable); - // Since write and commit succeeded, the rows should be readable assertEquals( "Should have expected rows", 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 2398bc45b19b..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 @@ -65,7 +65,6 @@ public void testUpdatePartitionGranularity() { @TestTemplate public void testPositionDeletesAreMaintainedDuringUpdate() { - assumeThat(formatVersion).isEqualTo(2); // Range distribution will produce partition scoped deletes which will not be cleaned up assumeThat(distributionMode).isNotEqualToIgnoringCase("range"); @@ -149,32 +148,6 @@ public void testUnpartitionedPositionDeletesAreMaintainedDuringUpdate() { sql("SELECT * FROM %s ORDER BY dep ASC, id ASC", selectTarget())); } - private void checkUpdateFileGranularity(DeleteGranularity deleteGranularity) { - initTable("PARTITIONED BY (dep)", deleteGranularity); - - sql("UPDATE %s SET id = id - 1 WHERE id = 1 OR id = 3", commitTarget()); - - Table table = validationCatalog.loadTable(tableIdent); - assertThat(table.snapshots()).hasSize(5); - - Snapshot currentSnapshot = SnapshotUtil.latestSnapshot(table, branch); - String expectedDeleteFilesCount = deleteGranularity == DeleteGranularity.FILE ? "4" : "2"; - validateMergeOnRead(currentSnapshot, "2", expectedDeleteFilesCount, "2"); - - assertEquals( - "Should have expected rows", - ImmutableList.of( - row(0, "hr"), - row(2, "hr"), - row(2, "hr"), - row(4, "hr"), - row(0, "it"), - row(2, "it"), - row(2, "it"), - row(4, "it")), - sql("SELECT * FROM %s ORDER BY dep ASC, id ASC", selectTarget())); - } - @TestTemplate public void testUpdateWithDVAndHistoricalPositionDeletes() { assumeThat(formatVersion).isEqualTo(2); @@ -224,9 +197,39 @@ public void testUpdateWithDVAndHistoricalPositionDeletes() { assertThat(dvs).allMatch(dv -> FileFormat.fromFileName(dv.location()) == FileFormat.PUFFIN); } + private void checkUpdateFileGranularity(DeleteGranularity deleteGranularity) { + initTable("PARTITIONED BY (dep)", deleteGranularity); + + sql("UPDATE %s SET id = id - 1 WHERE id = 1 OR id = 3", commitTarget()); + + Table table = validationCatalog.loadTable(tableIdent); + assertThat(table.snapshots()).hasSize(5); + + Snapshot currentSnapshot = SnapshotUtil.latestSnapshot(table, branch); + String expectedDeleteFilesCount = deleteGranularity == DeleteGranularity.FILE ? "4" : "2"; + validateMergeOnRead(currentSnapshot, "2", expectedDeleteFilesCount, "2"); + + assertEquals( + "Should have expected rows", + ImmutableList.of( + row(0, "hr"), + row(2, "hr"), + row(2, "hr"), + row(4, "hr"), + row(0, "it"), + row(2, "it"), + row(2, "it"), + row(4, "it")), + sql("SELECT * FROM %s ORDER BY dep ASC, id ASC", selectTarget())); + } + private void initTable(String partitionedBy, DeleteGranularity deleteGranularity) { createTableWithDeleteGranularity("id INT, dep STRING", partitionedBy, deleteGranularity); + sql( + "ALTER TABLE %s SET TBLPROPERTIES ('%s' '%s')", + tableName, TableProperties.DELETE_GRANULARITY, deleteGranularity); + append(tableName, "{ \"id\": 1, \"dep\": \"hr\" }\n" + "{ \"id\": 2, \"dep\": \"hr\" }"); append(tableName, "{ \"id\": 3, \"dep\": \"hr\" }\n" + "{ \"id\": 4, \"dep\": \"hr\" }"); append(tableName, "{ \"id\": 1, \"dep\": \"it\" }\n" + "{ \"id\": 2, \"dep\": \"it\" }"); 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 78f6b80ac948..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 @@ -127,16 +127,14 @@ public void testProcedureWithInvalidInput() { assertThatThrownBy( () -> sql("CALL %s.system.rewrite_table_path('%s')", catalogName, tableIdent)) .isInstanceOf(AnalysisException.class) - .hasMessageContaining( - "[REQUIRED_PARAMETER_NOT_FOUND] Cannot invoke routine `rewrite_table_path` because the parameter named `source_prefix` is required, but the routine call did not supply a value. Please update the routine call to supply an argument value (either positionally at index 0 or by name) and retry the query again. SQLSTATE: 4274K"); + .hasMessageContaining("Missing required parameters: [source_prefix,target_prefix]"); assertThatThrownBy( () -> sql( "CALL %s.system.rewrite_table_path('%s','%s')", catalogName, tableIdent, targetLocation)) .isInstanceOf(AnalysisException.class) - .hasMessageContaining( - "[REQUIRED_PARAMETER_NOT_FOUND] Cannot invoke routine `rewrite_table_path` because the parameter named `target_prefix` is required, but the routine call did not supply a value. Please update the routine call to supply an argument value (either positionally at index 0 or by name) and retry the query again. SQLSTATE: 4274K"); + .hasMessageContaining("Missing required parameters: [target_prefix]"); assertThatThrownBy( () -> sql( 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 2ebb3ff5b4f2..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 @@ -29,8 +29,8 @@ * A benchmark that evaluates the non-vectorized read and vectorized read with pos-delete in the * Spark data source for Iceberg. * - *

This class uses a dataset with a flat schema. To run this benchmark for spark-4.1: - * ./gradlew -DsparkVersions=4.1 :iceberg-spark:iceberg-spark-4.1:jmh \ + *

This class uses a dataset with a flat schema. To run this benchmark for spark-3.4: + * ./gradlew -DsparkVersions=3.4 :iceberg-spark:iceberg-spark-3.4:jmh \ * -PjmhIncludeRegex=IcebergSourceParquetMultiDeleteFileBenchmark \ * -PjmhOutputPath=benchmark/iceberg-source-parquet-multi-delete-file-benchmark-result.txt * 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 fd837e8e826e..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 @@ -29,8 +29,8 @@ * A benchmark that evaluates the non-vectorized read and vectorized read with pos-delete in the * Spark data source for Iceberg. * - *

This class uses a dataset with a flat schema. To run this benchmark for spark-4.1: - * ./gradlew -DsparkVersions=4.1 :iceberg-spark:iceberg-spark-4.1:jmh + *

This class uses a dataset with a flat schema. To run this benchmark for spark-3.4: + * ./gradlew -DsparkVersions=3.4 :iceberg-spark:iceberg-spark-3.4:jmh * -PjmhIncludeRegex=IcebergSourceParquetPosDeleteBenchmark * -PjmhOutputPath=benchmark/iceberg-source-parquet-pos-delete-benchmark-result.txt * 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 273b3f1cee4d..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 @@ -29,8 +29,8 @@ * A benchmark that evaluates the non-vectorized read and vectorized read with pos-delete in the * Spark data source for Iceberg. * - *

This class uses a dataset with a flat schema. To run this benchmark for spark-4.1: - * ./gradlew -DsparkVersions=4.1 :iceberg-spark:iceberg-spark-4.1:jmh + *

This class uses a dataset with a flat schema. To run this benchmark for spark-3.4: + * ./gradlew -DsparkVersions=3.4 :iceberg-spark:iceberg-spark-3.4:jmh * -PjmhIncludeRegex=IcebergSourceParquetWithUnrelatedDeleteBenchmark * -PjmhOutputPath=benchmark/iceberg-source-parquet-with-unrelated-delete-benchmark-result.txt * 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 c89e4f7852d5..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 @@ -906,6 +906,7 @@ public void testExpireOlderThanWithDeleteInMergedManifests() { assertThat(secondSnapshot.allManifests(table.io())) .as("Should replace manifest with a rewritten manifest") .hasSize(1); + table .newFastAppend() // do not merge to keep the last snapshot's manifest valid .appendFile(FILE_C) @@ -1010,6 +1011,7 @@ public void testExpireOlderThanWithRollbackAndMergedManifests() { Snapshot firstSnapshot = table.currentSnapshot(); assertThat(firstSnapshot.allManifests(table.io())).as("Should create one manifest").hasSize(1); + rightAfterSnapshot(); table.newAppend().appendFile(FILE_B).commit(); @@ -1168,7 +1170,6 @@ public void testExpireAction() { assertThat(table.snapshot(firstSnapshot.snapshotId())) .as("Should remove the oldest snapshot") .isNull(); - assertThat(pending).as("Pending deletes should contain one row").hasSize(1); assertThat(pending.get(0).getPath()) .as("Pending delete should be the expired manifest list 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 d94b83e4c973..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 @@ -22,7 +22,6 @@ import static org.apache.iceberg.data.FileHelpers.encrypt; import static org.apache.iceberg.types.Types.NestedField.optional; import static org.apache.iceberg.types.Types.NestedField.required; -import static org.apache.spark.sql.functions.col; import static org.apache.spark.sql.functions.current_date; import static org.apache.spark.sql.functions.date_add; import static org.apache.spark.sql.functions.expr; @@ -52,7 +51,6 @@ import java.util.stream.IntStream; import java.util.stream.LongStream; import java.util.stream.Stream; -import java.util.stream.StreamSupport; import org.apache.hadoop.conf.Configuration; import org.apache.iceberg.BaseTable; import org.apache.iceberg.ContentFile; @@ -129,7 +127,6 @@ import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; import org.apache.spark.sql.internal.SQLConf; -import org.apache.spark.sql.types.DataTypes; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.TestTemplate; @@ -1831,7 +1828,6 @@ public void testZOrderSort() { @TestTemplate public void testZOrderAllTypesSort() { - spark.conf().set("spark.sql.ansi.enabled", "false"); Table table = createTypeTestTable(); shouldHaveFiles(table, 10); @@ -2100,35 +2096,6 @@ public void testRewriteDataFilesPreservesLineage() throws NoSuchTableException { assertEquals("Rows must match", expectedRecords, actualRecordsWithLineage); } - @TestTemplate - public void testExecutorCacheForDeleteFilesDisabled() { - Table table = createTablePartitioned(1, 1); - RewriteDataFilesSparkAction action = SparkActions.get(spark).rewriteDataFiles(table); - - // The constructor should have set the configuration to false - SparkReadConf readConf = new SparkReadConf(action.spark(), table); - assertThat(readConf.cacheDeleteFilesOnExecutors()) - .as("Executor cache for delete files should be disabled in RewriteDataFilesSparkAction") - .isFalse(); - } - - @TestTemplate - public void testZOrderUDFWithDateType() { - SparkZOrderUDF zorderUDF = new SparkZOrderUDF(1, 16, 1024); - Dataset result = - spark - .sql("SELECT DATE '2025-01-01' as test_col") - .withColumn( - "zorder_result", - zorderUDF.sortedLexicographically(col("test_col"), DataTypes.DateType)); - - assertThat(result.schema().apply("zorder_result").dataType()).isEqualTo(DataTypes.BinaryType); - List rows = result.collectAsList(); - Row row = rows.get(0); - byte[] zorderBytes = row.getAs("zorder_result"); - assertThat(zorderBytes).isNotNull().isNotEmpty(); - } - protected void shouldRewriteDataFilesWithPartitionSpec(Table table, int outputSpecId) { List rewrittenFiles = currentDataFiles(table); assertThat(rewrittenFiles).allMatch(file -> file.specId() == outputSpecId); @@ -2187,10 +2154,8 @@ protected void shouldHaveMultipleFiles(Table table) { protected void shouldHaveFiles(Table table, int numExpected) { table.refresh(); - List files = - StreamSupport.stream(table.newScan().planFiles().spliterator(), false) - .collect(Collectors.toList()); - assertThat(files.size()).as("Did not have the expected number of files").isEqualTo(numExpected); + int numFiles = Iterables.size(table.newScan().planFiles()); + assertThat(numFiles).as("Did not have the expected number of files").isEqualTo(numExpected); } protected long shouldHaveMinSequenceNumberInPartition( @@ -2632,6 +2597,18 @@ private Set cacheContents(Table table) { .build(); } + @TestTemplate + public void testExecutorCacheForDeleteFilesDisabled() { + Table table = createTablePartitioned(1, 1); + RewriteDataFilesSparkAction action = SparkActions.get(spark).rewriteDataFiles(table); + + // The constructor should have set the configuration to false + SparkReadConf readConf = new SparkReadConf(action.spark(), table, Collections.emptyMap()); + assertThat(readConf.cacheDeleteFilesOnExecutors()) + .as("Executor cache for delete files should be disabled in RewriteDataFilesSparkAction") + .isFalse(); + } + private double percentFilesRequired(Table table, String col, String value) { return percentFilesRequired(table, new String[] {col}, new String[] {value}); } 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 09dfaecdf417..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 @@ -19,6 +19,7 @@ package org.apache.iceberg.spark.source; import static org.apache.iceberg.spark.SparkSchemaUtil.convert; +import static org.apache.iceberg.spark.data.TestHelpers.assertEqualsUnsafe; import static org.apache.iceberg.types.Types.NestedField.required; import static org.assertj.core.api.Assertions.assertThat; @@ -27,6 +28,7 @@ import java.nio.file.Path; import java.util.Iterator; import java.util.List; +import org.apache.avro.generic.GenericData; import org.apache.iceberg.Files; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; @@ -36,14 +38,10 @@ import org.apache.iceberg.Tables; import org.apache.iceberg.avro.Avro; import org.apache.iceberg.avro.AvroIterable; -import org.apache.iceberg.data.RandomGenericData; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.data.avro.DataWriter; import org.apache.iceberg.hadoop.HadoopTables; import org.apache.iceberg.io.FileAppender; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.spark.data.GenericsHelpers; +import org.apache.iceberg.spark.data.RandomData; import org.apache.iceberg.spark.data.SparkPlannedAvroReader; import org.apache.iceberg.types.Types; import org.apache.spark.api.java.JavaRDD; @@ -64,7 +62,7 @@ protected boolean supportsDefaultValues() { } @Override - protected void writeRecords(Table table, List records) throws IOException { + protected void writeRecords(Table table, List records) throws IOException { Schema tableSchema = table.schema(); // use the table schema because ids are reassigned Dataset df = createDataset(records, tableSchema); @@ -76,19 +74,18 @@ protected void writeRecords(Table table, List records) throws IOExceptio table.refresh(); } - private Dataset createDataset(List records, Schema schema) throws IOException { + private Dataset createDataset(List records, Schema schema) + throws IOException { // this uses the SparkAvroReader to create a DataFrame from the list of records // it assumes that SparkAvroReader is correct File testFile = File.createTempFile("junit", null, temp.toFile()); assertThat(testFile.delete()).as("Delete should succeed").isTrue(); - try (FileAppender writer = - Avro.write(Files.localOutput(testFile)) - .schema(schema) - .createWriterFunc(DataWriter::create) - .named("test") - .build()) { - writer.addAll(records); + try (FileAppender writer = + Avro.write(Files.localOutput(testFile)).schema(schema).named("test").build()) { + for (GenericData.Record rec : records) { + writer.add(rec); + } } List rows; @@ -102,19 +99,13 @@ private Dataset createDataset(List records, Schema schema) throws I // verify that the dataframe matches assertThat(rows).hasSameSizeAs(records); - Iterator recordIter = records.iterator(); + Iterator recordIter = records.iterator(); for (InternalRow row : rows) { - GenericsHelpers.assertEqualsUnsafe(schema.asStruct(), recordIter.next(), row); + assertEqualsUnsafe(schema.asStruct(), recordIter.next(), row); } JavaRDD rdd = sc.parallelize(rows); - Preconditions.checkArgument( - spark instanceof org.apache.spark.sql.classic.SparkSession, - "Expected instance of org.apache.spark.sql.classic.SparkSession, but got: %s", - spark.getClass().getName()); - - return ((org.apache.spark.sql.classic.SparkSession) spark) - .internalCreateDataFrame(JavaRDD.toRDD(rdd), convert(schema), false); + return spark.internalCreateDataFrame(JavaRDD.toRDD(rdd), convert(schema), false); } @Test @@ -133,7 +124,7 @@ public void testAlternateLocation() throws IOException { .set(TableProperties.WRITE_DATA_LOCATION, altLocation.getAbsolutePath()) .commit(); - writeRecords(table, RandomGenericData.generate(table.schema(), 100, 87112L)); + writeRecords(table, RandomData.generateList(table.schema(), 100, 87112L)); SnapshotChanges.builderFor(table) .build() 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 e67ec5fd62d4..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 @@ -24,8 +24,6 @@ import java.io.IOException; import java.math.RoundingMode; -import java.text.SimpleDateFormat; -import java.util.Date; import java.util.List; import java.util.Map; import org.apache.hadoop.conf.Configuration; @@ -69,8 +67,6 @@ @ExtendWith(ParameterizedTestExtension.class) public class TestDataSourceOptions extends TestBaseWithCatalog { - private static final SimpleDateFormat TIMESTAMP_FORMAT = - new SimpleDateFormat("yyyy-MM-dd HH:mm:ss.SSS"); private static final Configuration CONF = new Configuration(); private static final Schema SCHEMA = new Schema( @@ -257,28 +253,29 @@ public void testIncrementalScanOptions() throws IOException { spark .read() .format("iceberg") - .option(SparkReadOptions.VERSION_AS_OF, snapshotIds.get(3).toString()) + .option("snapshot-id", snapshotIds.get(3).toString()) .option("start-snapshot-id", snapshotIds.get(3).toString()) .load(tableLocation) .explain()) .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Cannot use time travel in incremental scan"); + .hasMessage( + "Cannot set start-snapshot-id and end-snapshot-id for incremental scans when either snapshot-id or as-of-timestamp is set"); // end-snapshot-id and as-of-timestamp are both configured. - long snapshotTimestamp = table.snapshot(snapshotIds.get(3)).timestampMillis(); - String formattedTimestamp = TIMESTAMP_FORMAT.format(new Date(snapshotTimestamp)); - assertThatThrownBy( () -> spark .read() .format("iceberg") - .option(SparkReadOptions.TIMESTAMP_AS_OF, formattedTimestamp) + .option( + SparkReadOptions.AS_OF_TIMESTAMP, + Long.toString(table.snapshot(snapshotIds.get(3)).timestampMillis())) .option("end-snapshot-id", snapshotIds.get(2).toString()) .load(tableLocation) .explain()) .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Cannot use time travel in incremental scan"); + .hasMessage( + "Cannot set start-snapshot-id and end-snapshot-id for incremental scans when either snapshot-id or as-of-timestamp is set"); // only end-snapshot-id is configured. assertThatThrownBy( @@ -291,7 +288,7 @@ public void testIncrementalScanOptions() throws IOException { .explain()) .isInstanceOf(IllegalArgumentException.class) .hasMessage( - "Cannot set only `end-snapshot-id` for incremental scans. Please, set `start-snapshot-id` too."); + "Cannot set only end-snapshot-id for incremental scans. Please, set start-snapshot-id too."); // test (1st snapshot, current snapshot] incremental scan. Dataset unboundedIncrementalResult = @@ -375,7 +372,7 @@ public void testMetadataSplitSizeOptionOverrideTableProperties() throws IOExcept } @TestTemplate - public void testDefaultMetadataSplitSize() { + public void testDefaultMetadataSplitSize() throws IOException { String tableLocation = temp.resolve("iceberg-table").toFile().toString(); HadoopTables tables = new HadoopTables(CONF); @@ -513,8 +510,6 @@ public void testExtraSnapshotMetadataWithDelete() Table table = validationCatalog.loadTable(tableIdent); List snapshots = Lists.newArrayList(table.snapshots()); - - assertThat(snapshots).hasSize(2); assertThat(snapshots.get(0).summary()).doesNotContainKey("writer-thread"); assertThat(snapshots.get(1).summary()) .containsEntry("writer-thread", "test-extra-commit-message-delete-thread") 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 22f2de120a9e..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 @@ -30,13 +30,11 @@ import java.io.IOException; import java.io.UncheckedIOException; import java.nio.file.Path; -import java.text.SimpleDateFormat; import java.time.LocalDateTime; import java.time.ZoneOffset; import java.util.Arrays; import java.util.Collections; import java.util.Comparator; -import java.util.Date; import java.util.List; import java.util.Map; import java.util.StringJoiner; @@ -106,9 +104,6 @@ public abstract class TestIcebergSourceTablesBase extends TestBase { - private static final SimpleDateFormat TIMESTAMP_FORMAT = - new SimpleDateFormat("yyyy-MM-dd HH:mm:ss.SSS"); - private static final Schema SCHEMA = new Schema( optional(1, "id", Types.IntegerType.get()), optional(2, "data", Types.StringType.get())); @@ -417,7 +412,6 @@ public void testAllEntriesTable() throws Exception { assertThat(expected).as("Entries table should have 3 rows").hasSize(3); assertThat(actual).as("Actual results should have 3 rows").hasSize(3); - for (int i = 0; i < expected.size(); i += 1) { TestHelpers.assertEqualsSafe( TestHelpers.nonDerivedSchema(entriesTableDs), expected.get(i), actual.get(i)); @@ -1444,7 +1438,7 @@ public void testPartitionsTable() { spark .read() .format("iceberg") - .option(SparkReadOptions.VERSION_AS_OF, String.valueOf(firstCommitId)) + .option(SparkReadOptions.SNAPSHOT_ID, String.valueOf(firstCommitId)) .load(loadLocation(tableIdentifier, "partitions")) .orderBy("partition.id") .collectAsList(); @@ -1461,7 +1455,6 @@ public void testPartitionsTable() { .load(loadLocation(tableIdentifier, "partitions")) .filter("partition.id < 2") .collectAsList(); - assertThat(filtered).as("Actual results should have one row").hasSize(1); TestHelpers.assertEqualsSafe( partitionsTable.schema().asStruct(), expected.get(0), filtered.get(0)); @@ -1473,7 +1466,6 @@ public void testPartitionsTable() { .load(loadLocation(tableIdentifier, "partitions")) .filter("partition.id < 2 or record_count=1") .collectAsList(); - assertThat(nonFiltered).as("Actual results should have two rows").hasSize(2); for (int i = 0; i < 2; i += 1) { TestHelpers.assertEqualsSafe( @@ -1821,7 +1813,7 @@ public synchronized void testSnapshotReadAfterAddColumn() { spark .read() .format("iceberg") - .option(SparkReadOptions.VERSION_AS_OF, snapshotBeforeAddColumn.snapshotId()) + .option(SparkReadOptions.SNAPSHOT_ID, snapshotBeforeAddColumn.snapshotId()) .load(loadLocation(tableIdentifier)); assertThat(resultDf3.orderBy("id").collectAsList()) .as("Records should match") @@ -1860,9 +1852,6 @@ public synchronized void testSnapshotReadAfterDropColumn() { table.updateSchema().deleteColumn("data").commit(); long tsAfterDropColumn = waitUntilAfter(System.currentTimeMillis()); - String formattedTsBeforeDropColumn = TIMESTAMP_FORMAT.format(new Date(tsBeforeDropColumn)); - String formattedTsAfterDropColumn = TIMESTAMP_FORMAT.format(new Date(tsAfterDropColumn)); - List newRecords = Lists.newArrayList(RowFactory.create(4, "B"), RowFactory.create(5, "C")); StructType newSparkSchema = SparkSchemaUtil.convert(SCHEMA3); @@ -1893,7 +1882,7 @@ public synchronized void testSnapshotReadAfterDropColumn() { spark .read() .format("iceberg") - .option(SparkReadOptions.TIMESTAMP_AS_OF, formattedTsBeforeDropColumn) + .option(SparkReadOptions.AS_OF_TIMESTAMP, tsBeforeDropColumn) .load(loadLocation(tableIdentifier)); assertThat(resultDf3.orderBy("id").collectAsList()) .as("Records should match") @@ -1906,7 +1895,7 @@ public synchronized void testSnapshotReadAfterDropColumn() { spark .read() .format("iceberg") - .option(SparkReadOptions.TIMESTAMP_AS_OF, formattedTsAfterDropColumn) + .option(SparkReadOptions.AS_OF_TIMESTAMP, tsAfterDropColumn) .load(loadLocation(tableIdentifier)); assertThat(resultDf4.orderBy("id").collectAsList()) .as("Records should match") @@ -1989,7 +1978,7 @@ public synchronized void testSnapshotReadAfterAddAndDropColumn() { spark .read() .format("iceberg") - .option(SparkReadOptions.VERSION_AS_OF, snapshotBeforeAddColumn.snapshotId()) + .option(SparkReadOptions.SNAPSHOT_ID, snapshotBeforeAddColumn.snapshotId()) .load(loadLocation(tableIdentifier)); assertThat(resultDf4.orderBy("id").collectAsList()) .as("Records should match") @@ -2353,7 +2342,7 @@ public void testSessionConfigSupport() { withSQLConf( // set read option through session configuration - ImmutableMap.of("spark.datasource.iceberg.versionAsOf", String.valueOf(s1)), + ImmutableMap.of("spark.datasource.iceberg.snapshot-id", String.valueOf(s1)), () -> { Dataset result = spark.read().format("iceberg").load(loadLocation(tableIdentifier)); List actual = result.as(Encoders.bean(SimpleRecord.class)).collectAsList(); 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 34e9dbc87d02..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 @@ -24,7 +24,6 @@ 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; import org.apache.iceberg.spark.CatalogTestBase; import org.apache.iceberg.spark.SparkCatalogConfig; @@ -52,17 +51,12 @@ public void testRefreshCommand() { // We are not allowed to change the session catalog after it has been initialized, so build a // new one if (Set.of( - SparkCatalogConfig.SPARK_SESSION.catalogName(), + SparkCatalogConfig.SPARK.catalogName(), SparkCatalogConfig.HADOOP.catalogName(), SparkCatalogConfig.REST.catalogName()) .contains(catalogName)) { spark.conf().set("spark.sql.catalog." + catalogName + ".cache-enabled", true); - - Preconditions.checkArgument( - spark instanceof org.apache.spark.sql.classic.SparkSession, - "Expected instance of org.apache.spark.sql.classic.SparkSession, but got: %s", - spark.getClass().getName()); - spark = ((org.apache.spark.sql.classic.SparkSession) spark).cloneSession(); + spark = spark.cloneSession(); } List originalExpected = ImmutableList.of(row(1, 1)); 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 fa2012d7dce2..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 @@ -297,7 +297,7 @@ public void testCommitUnknownException() { Table spyTable = spy(table); when(spyTable.newRowDelta()).thenReturn(spyNewRowDelta); SparkTable sparkTable = - branch == null ? new SparkTable(spyTable) : SparkTable.create(spyTable, branch); + branch == null ? new SparkTable(spyTable, false) : new SparkTable(spyTable, branch, false); ImmutableMap config = ImmutableMap.of( @@ -316,11 +316,6 @@ public void testCommitUnknownException() { .isInstanceOf(CommitStateUnknownException.class) .hasMessageStartingWith("Datacenter on Fire"); - // Manually refresh Spark table because it always pins snapshot - sparkTable = branch == null ? new SparkTable(spyTable) : SparkTable.create(spyTable, branch); - TestSparkCatalog.unsetTable(ident); - TestSparkCatalog.setTable(ident, sparkTable); - // Since write and commit succeeded, the rows should be readable assertEquals( "Should have expected rows", 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 78f6b80ac948..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 @@ -127,16 +127,14 @@ public void testProcedureWithInvalidInput() { assertThatThrownBy( () -> sql("CALL %s.system.rewrite_table_path('%s')", catalogName, tableIdent)) .isInstanceOf(AnalysisException.class) - .hasMessageContaining( - "[REQUIRED_PARAMETER_NOT_FOUND] Cannot invoke routine `rewrite_table_path` because the parameter named `source_prefix` is required, but the routine call did not supply a value. Please update the routine call to supply an argument value (either positionally at index 0 or by name) and retry the query again. SQLSTATE: 4274K"); + .hasMessageContaining("Missing required parameters: [source_prefix,target_prefix]"); assertThatThrownBy( () -> sql( "CALL %s.system.rewrite_table_path('%s','%s')", catalogName, tableIdent, targetLocation)) .isInstanceOf(AnalysisException.class) - .hasMessageContaining( - "[REQUIRED_PARAMETER_NOT_FOUND] Cannot invoke routine `rewrite_table_path` because the parameter named `target_prefix` is required, but the routine call did not supply a value. Please update the routine call to supply an argument value (either positionally at index 0 or by name) and retry the query again. SQLSTATE: 4274K"); + .hasMessageContaining("Missing required parameters: [target_prefix]"); assertThatThrownBy( () -> sql( 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 2ebb3ff5b4f2..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 @@ -29,8 +29,8 @@ * A benchmark that evaluates the non-vectorized read and vectorized read with pos-delete in the * Spark data source for Iceberg. * - *

This class uses a dataset with a flat schema. To run this benchmark for spark-4.1: - * ./gradlew -DsparkVersions=4.1 :iceberg-spark:iceberg-spark-4.1:jmh \ + *

This class uses a dataset with a flat schema. To run this benchmark for spark-3.5: + * ./gradlew -DsparkVersions=3.5 :iceberg-spark:iceberg-spark-3.5:jmh \ * -PjmhIncludeRegex=IcebergSourceParquetMultiDeleteFileBenchmark \ * -PjmhOutputPath=benchmark/iceberg-source-parquet-multi-delete-file-benchmark-result.txt * 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 fd837e8e826e..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 @@ -29,8 +29,8 @@ * A benchmark that evaluates the non-vectorized read and vectorized read with pos-delete in the * Spark data source for Iceberg. * - *

This class uses a dataset with a flat schema. To run this benchmark for spark-4.1: - * ./gradlew -DsparkVersions=4.1 :iceberg-spark:iceberg-spark-4.1:jmh + *

This class uses a dataset with a flat schema. To run this benchmark for spark-3.5: + * ./gradlew -DsparkVersions=3.5 :iceberg-spark:iceberg-spark-3.5:jmh * -PjmhIncludeRegex=IcebergSourceParquetPosDeleteBenchmark * -PjmhOutputPath=benchmark/iceberg-source-parquet-pos-delete-benchmark-result.txt * 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 273b3f1cee4d..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 @@ -29,8 +29,8 @@ * A benchmark that evaluates the non-vectorized read and vectorized read with pos-delete in the * Spark data source for Iceberg. * - *

This class uses a dataset with a flat schema. To run this benchmark for spark-4.1: - * ./gradlew -DsparkVersions=4.1 :iceberg-spark:iceberg-spark-4.1:jmh + *

This class uses a dataset with a flat schema. To run this benchmark for spark-3.5: + * ./gradlew -DsparkVersions=3.5 :iceberg-spark:iceberg-spark-3.5:jmh * -PjmhIncludeRegex=IcebergSourceParquetWithUnrelatedDeleteBenchmark * -PjmhOutputPath=benchmark/iceberg-source-parquet-with-unrelated-delete-benchmark-result.txt * 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 d94b83e4c973..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 @@ -22,7 +22,6 @@ import static org.apache.iceberg.data.FileHelpers.encrypt; import static org.apache.iceberg.types.Types.NestedField.optional; import static org.apache.iceberg.types.Types.NestedField.required; -import static org.apache.spark.sql.functions.col; import static org.apache.spark.sql.functions.current_date; import static org.apache.spark.sql.functions.date_add; import static org.apache.spark.sql.functions.expr; @@ -52,7 +51,6 @@ import java.util.stream.IntStream; import java.util.stream.LongStream; import java.util.stream.Stream; -import java.util.stream.StreamSupport; import org.apache.hadoop.conf.Configuration; import org.apache.iceberg.BaseTable; import org.apache.iceberg.ContentFile; @@ -129,7 +127,6 @@ import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; import org.apache.spark.sql.internal.SQLConf; -import org.apache.spark.sql.types.DataTypes; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.TestTemplate; @@ -1831,7 +1828,6 @@ public void testZOrderSort() { @TestTemplate public void testZOrderAllTypesSort() { - spark.conf().set("spark.sql.ansi.enabled", "false"); Table table = createTypeTestTable(); shouldHaveFiles(table, 10); @@ -2100,35 +2096,6 @@ public void testRewriteDataFilesPreservesLineage() throws NoSuchTableException { assertEquals("Rows must match", expectedRecords, actualRecordsWithLineage); } - @TestTemplate - public void testExecutorCacheForDeleteFilesDisabled() { - Table table = createTablePartitioned(1, 1); - RewriteDataFilesSparkAction action = SparkActions.get(spark).rewriteDataFiles(table); - - // The constructor should have set the configuration to false - SparkReadConf readConf = new SparkReadConf(action.spark(), table); - assertThat(readConf.cacheDeleteFilesOnExecutors()) - .as("Executor cache for delete files should be disabled in RewriteDataFilesSparkAction") - .isFalse(); - } - - @TestTemplate - public void testZOrderUDFWithDateType() { - SparkZOrderUDF zorderUDF = new SparkZOrderUDF(1, 16, 1024); - Dataset result = - spark - .sql("SELECT DATE '2025-01-01' as test_col") - .withColumn( - "zorder_result", - zorderUDF.sortedLexicographically(col("test_col"), DataTypes.DateType)); - - assertThat(result.schema().apply("zorder_result").dataType()).isEqualTo(DataTypes.BinaryType); - List rows = result.collectAsList(); - Row row = rows.get(0); - byte[] zorderBytes = row.getAs("zorder_result"); - assertThat(zorderBytes).isNotNull().isNotEmpty(); - } - protected void shouldRewriteDataFilesWithPartitionSpec(Table table, int outputSpecId) { List rewrittenFiles = currentDataFiles(table); assertThat(rewrittenFiles).allMatch(file -> file.specId() == outputSpecId); @@ -2187,10 +2154,8 @@ protected void shouldHaveMultipleFiles(Table table) { protected void shouldHaveFiles(Table table, int numExpected) { table.refresh(); - List files = - StreamSupport.stream(table.newScan().planFiles().spliterator(), false) - .collect(Collectors.toList()); - assertThat(files.size()).as("Did not have the expected number of files").isEqualTo(numExpected); + int numFiles = Iterables.size(table.newScan().planFiles()); + assertThat(numFiles).as("Did not have the expected number of files").isEqualTo(numExpected); } protected long shouldHaveMinSequenceNumberInPartition( @@ -2632,6 +2597,18 @@ private Set cacheContents(Table table) { .build(); } + @TestTemplate + public void testExecutorCacheForDeleteFilesDisabled() { + Table table = createTablePartitioned(1, 1); + RewriteDataFilesSparkAction action = SparkActions.get(spark).rewriteDataFiles(table); + + // The constructor should have set the configuration to false + SparkReadConf readConf = new SparkReadConf(action.spark(), table, Collections.emptyMap()); + assertThat(readConf.cacheDeleteFilesOnExecutors()) + .as("Executor cache for delete files should be disabled in RewriteDataFilesSparkAction") + .isFalse(); + } + private double percentFilesRequired(Table table, String col, String value) { return percentFilesRequired(table, new String[] {col}, new String[] {value}); } 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 09dfaecdf417..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 @@ -19,6 +19,7 @@ package org.apache.iceberg.spark.source; import static org.apache.iceberg.spark.SparkSchemaUtil.convert; +import static org.apache.iceberg.spark.data.TestHelpers.assertEqualsUnsafe; import static org.apache.iceberg.types.Types.NestedField.required; import static org.assertj.core.api.Assertions.assertThat; @@ -27,6 +28,7 @@ import java.nio.file.Path; import java.util.Iterator; import java.util.List; +import org.apache.avro.generic.GenericData; import org.apache.iceberg.Files; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; @@ -36,14 +38,10 @@ import org.apache.iceberg.Tables; import org.apache.iceberg.avro.Avro; import org.apache.iceberg.avro.AvroIterable; -import org.apache.iceberg.data.RandomGenericData; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.data.avro.DataWriter; import org.apache.iceberg.hadoop.HadoopTables; import org.apache.iceberg.io.FileAppender; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.spark.data.GenericsHelpers; +import org.apache.iceberg.spark.data.RandomData; import org.apache.iceberg.spark.data.SparkPlannedAvroReader; import org.apache.iceberg.types.Types; import org.apache.spark.api.java.JavaRDD; @@ -64,7 +62,7 @@ protected boolean supportsDefaultValues() { } @Override - protected void writeRecords(Table table, List records) throws IOException { + protected void writeRecords(Table table, List records) throws IOException { Schema tableSchema = table.schema(); // use the table schema because ids are reassigned Dataset df = createDataset(records, tableSchema); @@ -76,19 +74,18 @@ protected void writeRecords(Table table, List records) throws IOExceptio table.refresh(); } - private Dataset createDataset(List records, Schema schema) throws IOException { + private Dataset createDataset(List records, Schema schema) + throws IOException { // this uses the SparkAvroReader to create a DataFrame from the list of records // it assumes that SparkAvroReader is correct File testFile = File.createTempFile("junit", null, temp.toFile()); assertThat(testFile.delete()).as("Delete should succeed").isTrue(); - try (FileAppender writer = - Avro.write(Files.localOutput(testFile)) - .schema(schema) - .createWriterFunc(DataWriter::create) - .named("test") - .build()) { - writer.addAll(records); + try (FileAppender writer = + Avro.write(Files.localOutput(testFile)).schema(schema).named("test").build()) { + for (GenericData.Record rec : records) { + writer.add(rec); + } } List rows; @@ -102,19 +99,13 @@ private Dataset createDataset(List records, Schema schema) throws I // verify that the dataframe matches assertThat(rows).hasSameSizeAs(records); - Iterator recordIter = records.iterator(); + Iterator recordIter = records.iterator(); for (InternalRow row : rows) { - GenericsHelpers.assertEqualsUnsafe(schema.asStruct(), recordIter.next(), row); + assertEqualsUnsafe(schema.asStruct(), recordIter.next(), row); } JavaRDD rdd = sc.parallelize(rows); - Preconditions.checkArgument( - spark instanceof org.apache.spark.sql.classic.SparkSession, - "Expected instance of org.apache.spark.sql.classic.SparkSession, but got: %s", - spark.getClass().getName()); - - return ((org.apache.spark.sql.classic.SparkSession) spark) - .internalCreateDataFrame(JavaRDD.toRDD(rdd), convert(schema), false); + return spark.internalCreateDataFrame(JavaRDD.toRDD(rdd), convert(schema), false); } @Test @@ -133,7 +124,7 @@ public void testAlternateLocation() throws IOException { .set(TableProperties.WRITE_DATA_LOCATION, altLocation.getAbsolutePath()) .commit(); - writeRecords(table, RandomGenericData.generate(table.schema(), 100, 87112L)); + writeRecords(table, RandomData.generateList(table.schema(), 100, 87112L)); SnapshotChanges.builderFor(table) .build() 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 e67ec5fd62d4..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 @@ -24,8 +24,6 @@ import java.io.IOException; import java.math.RoundingMode; -import java.text.SimpleDateFormat; -import java.util.Date; import java.util.List; import java.util.Map; import org.apache.hadoop.conf.Configuration; @@ -69,8 +67,6 @@ @ExtendWith(ParameterizedTestExtension.class) public class TestDataSourceOptions extends TestBaseWithCatalog { - private static final SimpleDateFormat TIMESTAMP_FORMAT = - new SimpleDateFormat("yyyy-MM-dd HH:mm:ss.SSS"); private static final Configuration CONF = new Configuration(); private static final Schema SCHEMA = new Schema( @@ -257,28 +253,29 @@ public void testIncrementalScanOptions() throws IOException { spark .read() .format("iceberg") - .option(SparkReadOptions.VERSION_AS_OF, snapshotIds.get(3).toString()) + .option("snapshot-id", snapshotIds.get(3).toString()) .option("start-snapshot-id", snapshotIds.get(3).toString()) .load(tableLocation) .explain()) .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Cannot use time travel in incremental scan"); + .hasMessage( + "Cannot set start-snapshot-id and end-snapshot-id for incremental scans when either snapshot-id or as-of-timestamp is set"); // end-snapshot-id and as-of-timestamp are both configured. - long snapshotTimestamp = table.snapshot(snapshotIds.get(3)).timestampMillis(); - String formattedTimestamp = TIMESTAMP_FORMAT.format(new Date(snapshotTimestamp)); - assertThatThrownBy( () -> spark .read() .format("iceberg") - .option(SparkReadOptions.TIMESTAMP_AS_OF, formattedTimestamp) + .option( + SparkReadOptions.AS_OF_TIMESTAMP, + Long.toString(table.snapshot(snapshotIds.get(3)).timestampMillis())) .option("end-snapshot-id", snapshotIds.get(2).toString()) .load(tableLocation) .explain()) .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Cannot use time travel in incremental scan"); + .hasMessage( + "Cannot set start-snapshot-id and end-snapshot-id for incremental scans when either snapshot-id or as-of-timestamp is set"); // only end-snapshot-id is configured. assertThatThrownBy( @@ -291,7 +288,7 @@ public void testIncrementalScanOptions() throws IOException { .explain()) .isInstanceOf(IllegalArgumentException.class) .hasMessage( - "Cannot set only `end-snapshot-id` for incremental scans. Please, set `start-snapshot-id` too."); + "Cannot set only end-snapshot-id for incremental scans. Please, set start-snapshot-id too."); // test (1st snapshot, current snapshot] incremental scan. Dataset unboundedIncrementalResult = 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 22f2de120a9e..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 @@ -30,13 +30,11 @@ import java.io.IOException; import java.io.UncheckedIOException; import java.nio.file.Path; -import java.text.SimpleDateFormat; import java.time.LocalDateTime; import java.time.ZoneOffset; import java.util.Arrays; import java.util.Collections; import java.util.Comparator; -import java.util.Date; import java.util.List; import java.util.Map; import java.util.StringJoiner; @@ -106,9 +104,6 @@ public abstract class TestIcebergSourceTablesBase extends TestBase { - private static final SimpleDateFormat TIMESTAMP_FORMAT = - new SimpleDateFormat("yyyy-MM-dd HH:mm:ss.SSS"); - private static final Schema SCHEMA = new Schema( optional(1, "id", Types.IntegerType.get()), optional(2, "data", Types.StringType.get())); @@ -1444,7 +1439,7 @@ public void testPartitionsTable() { spark .read() .format("iceberg") - .option(SparkReadOptions.VERSION_AS_OF, String.valueOf(firstCommitId)) + .option(SparkReadOptions.SNAPSHOT_ID, String.valueOf(firstCommitId)) .load(loadLocation(tableIdentifier, "partitions")) .orderBy("partition.id") .collectAsList(); @@ -1821,7 +1816,7 @@ public synchronized void testSnapshotReadAfterAddColumn() { spark .read() .format("iceberg") - .option(SparkReadOptions.VERSION_AS_OF, snapshotBeforeAddColumn.snapshotId()) + .option(SparkReadOptions.SNAPSHOT_ID, snapshotBeforeAddColumn.snapshotId()) .load(loadLocation(tableIdentifier)); assertThat(resultDf3.orderBy("id").collectAsList()) .as("Records should match") @@ -1860,9 +1855,6 @@ public synchronized void testSnapshotReadAfterDropColumn() { table.updateSchema().deleteColumn("data").commit(); long tsAfterDropColumn = waitUntilAfter(System.currentTimeMillis()); - String formattedTsBeforeDropColumn = TIMESTAMP_FORMAT.format(new Date(tsBeforeDropColumn)); - String formattedTsAfterDropColumn = TIMESTAMP_FORMAT.format(new Date(tsAfterDropColumn)); - List newRecords = Lists.newArrayList(RowFactory.create(4, "B"), RowFactory.create(5, "C")); StructType newSparkSchema = SparkSchemaUtil.convert(SCHEMA3); @@ -1893,7 +1885,7 @@ public synchronized void testSnapshotReadAfterDropColumn() { spark .read() .format("iceberg") - .option(SparkReadOptions.TIMESTAMP_AS_OF, formattedTsBeforeDropColumn) + .option(SparkReadOptions.AS_OF_TIMESTAMP, tsBeforeDropColumn) .load(loadLocation(tableIdentifier)); assertThat(resultDf3.orderBy("id").collectAsList()) .as("Records should match") @@ -1906,7 +1898,7 @@ public synchronized void testSnapshotReadAfterDropColumn() { spark .read() .format("iceberg") - .option(SparkReadOptions.TIMESTAMP_AS_OF, formattedTsAfterDropColumn) + .option(SparkReadOptions.AS_OF_TIMESTAMP, tsAfterDropColumn) .load(loadLocation(tableIdentifier)); assertThat(resultDf4.orderBy("id").collectAsList()) .as("Records should match") @@ -1989,7 +1981,7 @@ public synchronized void testSnapshotReadAfterAddAndDropColumn() { spark .read() .format("iceberg") - .option(SparkReadOptions.VERSION_AS_OF, snapshotBeforeAddColumn.snapshotId()) + .option(SparkReadOptions.SNAPSHOT_ID, snapshotBeforeAddColumn.snapshotId()) .load(loadLocation(tableIdentifier)); assertThat(resultDf4.orderBy("id").collectAsList()) .as("Records should match") @@ -2353,7 +2345,7 @@ public void testSessionConfigSupport() { withSQLConf( // set read option through session configuration - ImmutableMap.of("spark.datasource.iceberg.versionAsOf", String.valueOf(s1)), + ImmutableMap.of("spark.datasource.iceberg.snapshot-id", String.valueOf(s1)), () -> { Dataset result = spark.read().format("iceberg").load(loadLocation(tableIdentifier)); List actual = result.as(Encoders.bean(SimpleRecord.class)).collectAsList(); 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 34e9dbc87d02..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 @@ -24,7 +24,6 @@ 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; import org.apache.iceberg.spark.CatalogTestBase; import org.apache.iceberg.spark.SparkCatalogConfig; @@ -57,12 +56,7 @@ public void testRefreshCommand() { SparkCatalogConfig.REST.catalogName()) .contains(catalogName)) { spark.conf().set("spark.sql.catalog." + catalogName + ".cache-enabled", true); - - Preconditions.checkArgument( - spark instanceof org.apache.spark.sql.classic.SparkSession, - "Expected instance of org.apache.spark.sql.classic.SparkSession, but got: %s", - spark.getClass().getName()); - spark = ((org.apache.spark.sql.classic.SparkSession) spark).cloneSession(); + spark = spark.cloneSession(); } List originalExpected = ImmutableList.of(row(1, 1)); 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 fa2012d7dce2..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 @@ -297,7 +297,7 @@ public void testCommitUnknownException() { Table spyTable = spy(table); when(spyTable.newRowDelta()).thenReturn(spyNewRowDelta); SparkTable sparkTable = - branch == null ? new SparkTable(spyTable) : SparkTable.create(spyTable, branch); + branch == null ? new SparkTable(spyTable, false) : new SparkTable(spyTable, branch, false); ImmutableMap config = ImmutableMap.of( @@ -316,11 +316,6 @@ public void testCommitUnknownException() { .isInstanceOf(CommitStateUnknownException.class) .hasMessageStartingWith("Datacenter on Fire"); - // Manually refresh Spark table because it always pins snapshot - sparkTable = branch == null ? new SparkTable(spyTable) : SparkTable.create(spyTable, branch); - TestSparkCatalog.unsetTable(ident); - TestSparkCatalog.setTable(ident, sparkTable); - // Since write and commit succeeded, the rows should be readable assertEquals( "Should have expected rows", 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 2ebb3ff5b4f2..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 @@ -29,8 +29,8 @@ * A benchmark that evaluates the non-vectorized read and vectorized read with pos-delete in the * Spark data source for Iceberg. * - *

This class uses a dataset with a flat schema. To run this benchmark for spark-4.1: - * ./gradlew -DsparkVersions=4.1 :iceberg-spark:iceberg-spark-4.1:jmh \ + *

This class uses a dataset with a flat schema. To run this benchmark for spark-4.0: + * ./gradlew -DsparkVersions=4.0 :iceberg-spark:iceberg-spark-4.0:jmh \ * -PjmhIncludeRegex=IcebergSourceParquetMultiDeleteFileBenchmark \ * -PjmhOutputPath=benchmark/iceberg-source-parquet-multi-delete-file-benchmark-result.txt * 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 fd837e8e826e..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 @@ -29,8 +29,8 @@ * A benchmark that evaluates the non-vectorized read and vectorized read with pos-delete in the * Spark data source for Iceberg. * - *

This class uses a dataset with a flat schema. To run this benchmark for spark-4.1: - * ./gradlew -DsparkVersions=4.1 :iceberg-spark:iceberg-spark-4.1:jmh + *

This class uses a dataset with a flat schema. To run this benchmark for spark-4.0: + * ./gradlew -DsparkVersions=4.0 :iceberg-spark:iceberg-spark-4.0:jmh * -PjmhIncludeRegex=IcebergSourceParquetPosDeleteBenchmark * -PjmhOutputPath=benchmark/iceberg-source-parquet-pos-delete-benchmark-result.txt * 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 273b3f1cee4d..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 @@ -29,8 +29,8 @@ * A benchmark that evaluates the non-vectorized read and vectorized read with pos-delete in the * Spark data source for Iceberg. * - *

This class uses a dataset with a flat schema. To run this benchmark for spark-4.1: - * ./gradlew -DsparkVersions=4.1 :iceberg-spark:iceberg-spark-4.1:jmh + *

This class uses a dataset with a flat schema. To run this benchmark for spark-4.0: + * ./gradlew -DsparkVersions=4.0 :iceberg-spark:iceberg-spark-4.0:jmh * -PjmhIncludeRegex=IcebergSourceParquetWithUnrelatedDeleteBenchmark * -PjmhOutputPath=benchmark/iceberg-source-parquet-with-unrelated-delete-benchmark-result.txt * 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 d94b83e4c973..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 @@ -2106,7 +2106,7 @@ public void testExecutorCacheForDeleteFilesDisabled() { RewriteDataFilesSparkAction action = SparkActions.get(spark).rewriteDataFiles(table); // The constructor should have set the configuration to false - SparkReadConf readConf = new SparkReadConf(action.spark(), table); + SparkReadConf readConf = new SparkReadConf(action.spark(), table, Collections.emptyMap()); assertThat(readConf.cacheDeleteFilesOnExecutors()) .as("Executor cache for delete files should be disabled in RewriteDataFilesSparkAction") .isFalse(); 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 e67ec5fd62d4..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 @@ -24,8 +24,6 @@ import java.io.IOException; import java.math.RoundingMode; -import java.text.SimpleDateFormat; -import java.util.Date; import java.util.List; import java.util.Map; import org.apache.hadoop.conf.Configuration; @@ -69,8 +67,6 @@ @ExtendWith(ParameterizedTestExtension.class) public class TestDataSourceOptions extends TestBaseWithCatalog { - private static final SimpleDateFormat TIMESTAMP_FORMAT = - new SimpleDateFormat("yyyy-MM-dd HH:mm:ss.SSS"); private static final Configuration CONF = new Configuration(); private static final Schema SCHEMA = new Schema( @@ -257,28 +253,29 @@ public void testIncrementalScanOptions() throws IOException { spark .read() .format("iceberg") - .option(SparkReadOptions.VERSION_AS_OF, snapshotIds.get(3).toString()) + .option("snapshot-id", snapshotIds.get(3).toString()) .option("start-snapshot-id", snapshotIds.get(3).toString()) .load(tableLocation) .explain()) .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Cannot use time travel in incremental scan"); + .hasMessage( + "Cannot set start-snapshot-id and end-snapshot-id for incremental scans when either snapshot-id or as-of-timestamp is set"); // end-snapshot-id and as-of-timestamp are both configured. - long snapshotTimestamp = table.snapshot(snapshotIds.get(3)).timestampMillis(); - String formattedTimestamp = TIMESTAMP_FORMAT.format(new Date(snapshotTimestamp)); - assertThatThrownBy( () -> spark .read() .format("iceberg") - .option(SparkReadOptions.TIMESTAMP_AS_OF, formattedTimestamp) + .option( + SparkReadOptions.AS_OF_TIMESTAMP, + Long.toString(table.snapshot(snapshotIds.get(3)).timestampMillis())) .option("end-snapshot-id", snapshotIds.get(2).toString()) .load(tableLocation) .explain()) .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Cannot use time travel in incremental scan"); + .hasMessage( + "Cannot set start-snapshot-id and end-snapshot-id for incremental scans when either snapshot-id or as-of-timestamp is set"); // only end-snapshot-id is configured. assertThatThrownBy( @@ -291,7 +288,7 @@ public void testIncrementalScanOptions() throws IOException { .explain()) .isInstanceOf(IllegalArgumentException.class) .hasMessage( - "Cannot set only `end-snapshot-id` for incremental scans. Please, set `start-snapshot-id` too."); + "Cannot set only end-snapshot-id for incremental scans. Please, set start-snapshot-id too."); // test (1st snapshot, current snapshot] incremental scan. Dataset unboundedIncrementalResult = 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 22f2de120a9e..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 @@ -30,13 +30,11 @@ import java.io.IOException; import java.io.UncheckedIOException; import java.nio.file.Path; -import java.text.SimpleDateFormat; import java.time.LocalDateTime; import java.time.ZoneOffset; import java.util.Arrays; import java.util.Collections; import java.util.Comparator; -import java.util.Date; import java.util.List; import java.util.Map; import java.util.StringJoiner; @@ -106,9 +104,6 @@ public abstract class TestIcebergSourceTablesBase extends TestBase { - private static final SimpleDateFormat TIMESTAMP_FORMAT = - new SimpleDateFormat("yyyy-MM-dd HH:mm:ss.SSS"); - private static final Schema SCHEMA = new Schema( optional(1, "id", Types.IntegerType.get()), optional(2, "data", Types.StringType.get())); @@ -1444,7 +1439,7 @@ public void testPartitionsTable() { spark .read() .format("iceberg") - .option(SparkReadOptions.VERSION_AS_OF, String.valueOf(firstCommitId)) + .option(SparkReadOptions.SNAPSHOT_ID, String.valueOf(firstCommitId)) .load(loadLocation(tableIdentifier, "partitions")) .orderBy("partition.id") .collectAsList(); @@ -1821,7 +1816,7 @@ public synchronized void testSnapshotReadAfterAddColumn() { spark .read() .format("iceberg") - .option(SparkReadOptions.VERSION_AS_OF, snapshotBeforeAddColumn.snapshotId()) + .option(SparkReadOptions.SNAPSHOT_ID, snapshotBeforeAddColumn.snapshotId()) .load(loadLocation(tableIdentifier)); assertThat(resultDf3.orderBy("id").collectAsList()) .as("Records should match") @@ -1860,9 +1855,6 @@ public synchronized void testSnapshotReadAfterDropColumn() { table.updateSchema().deleteColumn("data").commit(); long tsAfterDropColumn = waitUntilAfter(System.currentTimeMillis()); - String formattedTsBeforeDropColumn = TIMESTAMP_FORMAT.format(new Date(tsBeforeDropColumn)); - String formattedTsAfterDropColumn = TIMESTAMP_FORMAT.format(new Date(tsAfterDropColumn)); - List newRecords = Lists.newArrayList(RowFactory.create(4, "B"), RowFactory.create(5, "C")); StructType newSparkSchema = SparkSchemaUtil.convert(SCHEMA3); @@ -1893,7 +1885,7 @@ public synchronized void testSnapshotReadAfterDropColumn() { spark .read() .format("iceberg") - .option(SparkReadOptions.TIMESTAMP_AS_OF, formattedTsBeforeDropColumn) + .option(SparkReadOptions.AS_OF_TIMESTAMP, tsBeforeDropColumn) .load(loadLocation(tableIdentifier)); assertThat(resultDf3.orderBy("id").collectAsList()) .as("Records should match") @@ -1906,7 +1898,7 @@ public synchronized void testSnapshotReadAfterDropColumn() { spark .read() .format("iceberg") - .option(SparkReadOptions.TIMESTAMP_AS_OF, formattedTsAfterDropColumn) + .option(SparkReadOptions.AS_OF_TIMESTAMP, tsAfterDropColumn) .load(loadLocation(tableIdentifier)); assertThat(resultDf4.orderBy("id").collectAsList()) .as("Records should match") @@ -1989,7 +1981,7 @@ public synchronized void testSnapshotReadAfterAddAndDropColumn() { spark .read() .format("iceberg") - .option(SparkReadOptions.VERSION_AS_OF, snapshotBeforeAddColumn.snapshotId()) + .option(SparkReadOptions.SNAPSHOT_ID, snapshotBeforeAddColumn.snapshotId()) .load(loadLocation(tableIdentifier)); assertThat(resultDf4.orderBy("id").collectAsList()) .as("Records should match") @@ -2353,7 +2345,7 @@ public void testSessionConfigSupport() { withSQLConf( // set read option through session configuration - ImmutableMap.of("spark.datasource.iceberg.versionAsOf", String.valueOf(s1)), + ImmutableMap.of("spark.datasource.iceberg.snapshot-id", String.valueOf(s1)), () -> { Dataset result = spark.read().format("iceberg").load(loadLocation(tableIdentifier)); List actual = result.as(Encoders.bean(SimpleRecord.class)).collectAsList(); From edd27cdd243f913e4afb20c5225d5370abb6f064 Mon Sep 17 00:00:00 2001 From: Russell Spitzer Date: Tue, 17 Mar 2026 10:51:50 -0500 Subject: [PATCH 4/6] Address review feedback: extract helpers, pass specsById, simplify test snapshot usage --- .../apache/iceberg/CherryPickOperation.java | 34 ++++++++++--------- .../apache/iceberg/ReachableFileCleanup.java | 9 ++--- .../operator/TestMonitorSource.java | 7 +--- .../operator/TestMonitorSource.java | 7 +--- .../operator/TestMonitorSource.java | 7 +--- 5 files changed, 26 insertions(+), 38 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/CherryPickOperation.java b/core/src/main/java/org/apache/iceberg/CherryPickOperation.java index 35bf9795f3da..8c8026de5a08 100644 --- a/core/src/main/java/org/apache/iceberg/CherryPickOperation.java +++ b/core/src/main/java/org/apache/iceberg/CherryPickOperation.java @@ -231,22 +231,10 @@ private static void validateReplacedPartitions( Iterables.concat( Iterables.transform( snapshots, - snap -> { - Iterable ownedManifests = - Iterables.filter( - snap.dataManifests(io), - m -> Objects.equals(m.snapshotId(), snap.snapshotId())); - return Iterables.transform( - ownedManifests, - manifest -> { - CloseableIterable> entries = - ManifestFiles.read(manifest, io, meta.specsById()).entries(); - CloseableIterable> added = - CloseableIterable.filter( - entries, e -> e.status() == ManifestEntry.Status.ADDED); - return CloseableIterable.transform(added, e -> e.file().copy()); - }); - })); + snap -> + Iterables.transform( + manifestsCreatedBy(snap, io), + manifest -> addedDataFiles(manifest, io, meta.specsById())))); try (CloseableIterable newFiles = new ParallelIterable<>(addedFileTasks, ThreadPools.getWorkerPool())) { @@ -263,6 +251,20 @@ private static void validateReplacedPartitions( } } + 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/ReachableFileCleanup.java b/core/src/main/java/org/apache/iceberg/ReachableFileCleanup.java index 1c645a03ebed..8100ce191dfc 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; @@ -82,7 +83,7 @@ public void cleanFiles( if (!manifestsToDelete.isEmpty()) { if (ExpireSnapshots.CleanupLevel.ALL == cleanupLevel) { Set dataFilesToDelete = - findFilesToDelete(manifestsToDelete, currentManifests, afterExpiration); + findFilesToDelete(manifestsToDelete, currentManifests, afterExpiration.specsById()); LOG.debug("Deleting {} data files", dataFilesToDelete.size()); deleteFiles(dataFilesToDelete, "data"); } @@ -169,7 +170,7 @@ private Set readManifests(Set snapshots) { private Set findFilesToDelete( Set manifestFilesToDelete, Set currentManifestFiles, - TableMetadata metadata) { + Map specsById) { Set filesToDelete = ConcurrentHashMap.newKeySet(); Tasks.foreach(manifestFilesToDelete) @@ -183,7 +184,7 @@ private Set findFilesToDelete( .run( manifest -> { try (CloseableIterable paths = - ManifestFiles.readPaths(manifest, fileIO, metadata.specsById())) { + ManifestFiles.readPaths(manifest, fileIO, specsById)) { paths.forEach(filesToDelete::add); } catch (IOException e) { throw new RuntimeIOException(e, "Failed to read manifest file: %s", manifest); @@ -212,7 +213,7 @@ private Set findFilesToDelete( // Remove all the live files from the candidate deletion set try (CloseableIterable paths = - ManifestFiles.readPaths(manifest, fileIO, metadata.specsById())) { + 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/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 f89c92cbf22e..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 @@ -309,12 +309,7 @@ void testSkipReplace() throws IOException { // Create a DataOperations.REPLACE snapshot DataFile dataFile = - SnapshotChanges.builderFor(table) - .snapshot(table.snapshots().iterator().next()) - .build() - .addedDataFiles() - .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); 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 84090e9e6b7b..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 @@ -317,12 +317,7 @@ void testSkipReplace() throws IOException { // Create a DataOperations.REPLACE snapshot DataFile dataFile = - SnapshotChanges.builderFor(table) - .snapshot(table.snapshots().iterator().next()) - .build() - .addedDataFiles() - .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); 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 84090e9e6b7b..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 @@ -317,12 +317,7 @@ void testSkipReplace() throws IOException { // Create a DataOperations.REPLACE snapshot DataFile dataFile = - SnapshotChanges.builderFor(table) - .snapshot(table.snapshots().iterator().next()) - .build() - .addedDataFiles() - .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); From 5bdfd81460a83ee4741c016a7846831e286eb680 Mon Sep 17 00:00:00 2001 From: Russell Spitzer Date: Tue, 17 Mar 2026 13:00:41 -0500 Subject: [PATCH 5/6] Fix ReachableFileCleanup using afterExpiration specs to read expired manifests Expired manifests may reference partition specs that were removed from afterExpiration metadata by cleanExpiredMetadata. Use beforeExpiration specs instead, which still contain all specs including those referenced by the expired manifests. --- core/src/main/java/org/apache/iceberg/ReachableFileCleanup.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/java/org/apache/iceberg/ReachableFileCleanup.java b/core/src/main/java/org/apache/iceberg/ReachableFileCleanup.java index 8100ce191dfc..b5a565b81e7e 100644 --- a/core/src/main/java/org/apache/iceberg/ReachableFileCleanup.java +++ b/core/src/main/java/org/apache/iceberg/ReachableFileCleanup.java @@ -83,7 +83,7 @@ public void cleanFiles( if (!manifestsToDelete.isEmpty()) { if (ExpireSnapshots.CleanupLevel.ALL == cleanupLevel) { Set dataFilesToDelete = - findFilesToDelete(manifestsToDelete, currentManifests, afterExpiration.specsById()); + findFilesToDelete(manifestsToDelete, currentManifests, beforeExpiration.specsById()); LOG.debug("Deleting {} data files", dataFilesToDelete.size()); deleteFiles(dataFilesToDelete, "data"); } From 5a2308005585ab6bb97e3f58e09b90d4ea21a574 Mon Sep 17 00:00:00 2001 From: Russell Spitzer Date: Wed, 18 Mar 2026 19:27:07 -0500 Subject: [PATCH 6/6] Pass specsById directly instead of full TableMetadata in CatalogUtil.deleteFiles --- core/src/main/java/org/apache/iceberg/CatalogUtil.java | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/CatalogUtil.java b/core/src/main/java/org/apache/iceberg/CatalogUtil.java index 07ca55d522b3..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, metadata); + deleteFiles(io, manifestsToDelete, metadata.specsById()); } deleteFiles(io, Iterables.transform(manifestsToDelete, ManifestFile::path), "manifest"); @@ -164,7 +164,7 @@ public static void dropViewMetadata(FileIO io, ViewMetadata metadata) { @SuppressWarnings("DangerousStringInternUsage") private static void deleteFiles( - FileIO io, Set allManifests, TableMetadata metadata) { + 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(); @@ -178,8 +178,7 @@ private static void deleteFiles( LOG.warn("Failed to get deleted files: this may cause orphaned data files", exc)) .run( manifest -> { - try (ManifestReader reader = - ManifestFiles.open(manifest, io, metadata.specsById())) { + 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