From a75c7dfb3e7e105deccac1863ffd267a309c99c1 Mon Sep 17 00:00:00 2001 From: Alessandro Nori Date: Thu, 18 Dec 2025 20:08:50 +0100 Subject: [PATCH 1/8] add deletedOrphanFilesCount --- .../java/org/apache/iceberg/actions/DeleteOrphanFiles.java | 4 ++++ .../org/apache/iceberg/actions/BaseDeleteOrphanFiles.java | 5 ++++- .../iceberg/spark/actions/DeleteOrphanFilesSparkAction.java | 5 ++++- .../iceberg/spark/actions/DeleteOrphanFilesSparkAction.java | 5 ++++- .../iceberg/spark/actions/DeleteOrphanFilesSparkAction.java | 5 ++++- 5 files changed, 20 insertions(+), 4 deletions(-) diff --git a/api/src/main/java/org/apache/iceberg/actions/DeleteOrphanFiles.java b/api/src/main/java/org/apache/iceberg/actions/DeleteOrphanFiles.java index 4e8f80fa833f..e59e7d3b8b63 100644 --- a/api/src/main/java/org/apache/iceberg/actions/DeleteOrphanFiles.java +++ b/api/src/main/java/org/apache/iceberg/actions/DeleteOrphanFiles.java @@ -142,6 +142,10 @@ default DeleteOrphanFiles equalAuthorities(Map newEqualAuthoriti interface Result { /** Returns locations of orphan files. */ Iterable orphanFileLocations(); + + default long deletedOrphanFilesCount() { + return 0; + } } /** diff --git a/core/src/main/java/org/apache/iceberg/actions/BaseDeleteOrphanFiles.java b/core/src/main/java/org/apache/iceberg/actions/BaseDeleteOrphanFiles.java index 182c8b191e87..54ee2d391b1a 100644 --- a/core/src/main/java/org/apache/iceberg/actions/BaseDeleteOrphanFiles.java +++ b/core/src/main/java/org/apache/iceberg/actions/BaseDeleteOrphanFiles.java @@ -29,5 +29,8 @@ interface BaseDeleteOrphanFiles extends DeleteOrphanFiles { @Value.Immutable - interface Result extends DeleteOrphanFiles.Result {} + interface Result extends DeleteOrphanFiles.Result { + @Override + long deletedOrphanFilesCount(); + } } diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/DeleteOrphanFilesSparkAction.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/DeleteOrphanFilesSparkAction.java index 78662159b0bb..031214ea8468 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/DeleteOrphanFilesSparkAction.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/DeleteOrphanFilesSparkAction.java @@ -300,7 +300,10 @@ private DeleteOrphanFiles.Result deleteFiles(Dataset orphanFileDS) { LOG.info("Deleted {} orphan files", filesCount); - return ImmutableDeleteOrphanFiles.Result.builder().orphanFileLocations(orphanFileList).build(); + return ImmutableDeleteOrphanFiles.Result.builder() + .orphanFileLocations(orphanFileList) + .deletedOrphanFilesCount(filesCount) + .build(); } private void collectPathsForOutput( diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/DeleteOrphanFilesSparkAction.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/DeleteOrphanFilesSparkAction.java index 78662159b0bb..031214ea8468 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/DeleteOrphanFilesSparkAction.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/DeleteOrphanFilesSparkAction.java @@ -300,7 +300,10 @@ private DeleteOrphanFiles.Result deleteFiles(Dataset orphanFileDS) { LOG.info("Deleted {} orphan files", filesCount); - return ImmutableDeleteOrphanFiles.Result.builder().orphanFileLocations(orphanFileList).build(); + return ImmutableDeleteOrphanFiles.Result.builder() + .orphanFileLocations(orphanFileList) + .deletedOrphanFilesCount(filesCount) + .build(); } private void collectPathsForOutput( diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/DeleteOrphanFilesSparkAction.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/DeleteOrphanFilesSparkAction.java index 78662159b0bb..031214ea8468 100644 --- a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/DeleteOrphanFilesSparkAction.java +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/DeleteOrphanFilesSparkAction.java @@ -300,7 +300,10 @@ private DeleteOrphanFiles.Result deleteFiles(Dataset orphanFileDS) { LOG.info("Deleted {} orphan files", filesCount); - return ImmutableDeleteOrphanFiles.Result.builder().orphanFileLocations(orphanFileList).build(); + return ImmutableDeleteOrphanFiles.Result.builder() + .orphanFileLocations(orphanFileList) + .deletedOrphanFilesCount(filesCount) + .build(); } private void collectPathsForOutput( From 106f552b8f571d8ae0e5376e1311c44eb8d187b4 Mon Sep 17 00:00:00 2001 From: Alessandro Nori Date: Thu, 18 Dec 2025 22:11:12 +0100 Subject: [PATCH 2/8] add unit tests --- .../iceberg/actions/DeleteOrphanFiles.java | 1 + .../actions/TestRemoveOrphanFilesAction.java | 45 +++++++++++++++++++ .../actions/TestRemoveOrphanFilesAction3.java | 11 ++++- .../actions/TestRemoveOrphanFilesAction.java | 45 +++++++++++++++++++ .../actions/TestRemoveOrphanFilesAction3.java | 5 +++ .../actions/TestRemoveOrphanFilesAction.java | 45 +++++++++++++++++++ .../actions/TestRemoveOrphanFilesAction3.java | 5 +++ 7 files changed, 155 insertions(+), 2 deletions(-) diff --git a/api/src/main/java/org/apache/iceberg/actions/DeleteOrphanFiles.java b/api/src/main/java/org/apache/iceberg/actions/DeleteOrphanFiles.java index e59e7d3b8b63..521d4d47b1f8 100644 --- a/api/src/main/java/org/apache/iceberg/actions/DeleteOrphanFiles.java +++ b/api/src/main/java/org/apache/iceberg/actions/DeleteOrphanFiles.java @@ -143,6 +143,7 @@ interface Result { /** Returns locations of orphan files. */ Iterable orphanFileLocations(); + /** Returns the number of deleted orphan files. */ default long deletedOrphanFilesCount() { return 0; } diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java index 40505b856737..dffefed678e4 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java @@ -184,6 +184,9 @@ public void testDryRun() throws IOException { assertThat(result1.orphanFileLocations()) .as("Default olderThan interval should be safe") .isEmpty(); + assertThat(result1.deletedOrphanFilesCount()) + .as("Default olderThan interval should be safe") + .isEqualTo(0L); DeleteOrphanFiles.Result result2 = actions @@ -195,6 +198,9 @@ public void testDryRun() throws IOException { assertThat(result2.orphanFileLocations()) .as("Action should find 1 file") .isEqualTo(invalidFiles); + assertThat(result2.deletedOrphanFilesCount()) + .as("Action should find 1 file") + .isEqualTo((long) invalidFiles.size()); assertThat(fs.exists(new Path(invalidFiles.get(0)))) .as("Invalid file should be present") .isTrue(); @@ -210,6 +216,9 @@ public void testDryRun() throws IOException { assertThat(result3.orphanFileLocations()) .as("Streaming dry run should find 1 file") .isEqualTo(invalidFiles); + assertThat(result3.deletedOrphanFilesCount()) + .as("Streaming dry run should find 1 file") + .isEqualTo((long) invalidFiles.size()); assertThat(fs.exists(new Path(invalidFiles.get(0)))) .as("Invalid file should be present after streaming dry run") .isTrue(); @@ -223,6 +232,9 @@ public void testDryRun() throws IOException { assertThat(result4.orphanFileLocations()) .as("Action should delete 1 file") .isEqualTo(invalidFiles); + assertThat(result4.deletedOrphanFilesCount()) + .as("Action should delete 1 file") + .isEqualTo((long) invalidFiles.size()); assertThat(fs.exists(new Path(invalidFiles.get(0)))) .as("Invalid file should not be present") .isFalse(); @@ -286,6 +298,7 @@ public void testAllValidFilesAreKept() throws IOException { .execute(); assertThat(result.orphanFileLocations()).as("Should delete 4 files").hasSize(4); + assertThat(result.deletedOrphanFilesCount()).as("Should delete 4 files").isEqualTo(4L); Path dataPath = new Path(tableLocation + "/data"); FileSystem fs = dataPath.getFileSystem(spark.sessionState().newHadoopConf()); @@ -366,6 +379,7 @@ public void orphanedFileRemovedWithParallelTasks() { .containsExactlyInAnyOrder( "remove-orphan-0", "remove-orphan-1", "remove-orphan-2", "remove-orphan-3"); assertThat(deletedFiles).hasSize(4); + assertThat(result.deletedOrphanFilesCount()).as("Should delete 4 files").isEqualTo(4L); } @TestTemplate @@ -410,6 +424,7 @@ public void testWapFilesAreKept() { actions.deleteOrphanFiles(table).olderThan(System.currentTimeMillis()).execute(); assertThat(result.orphanFileLocations()).as("Should not delete any files").isEmpty(); + assertThat(result.deletedOrphanFilesCount()).as("Should not delete any files").isEqualTo(0L); } @TestTemplate @@ -440,6 +455,7 @@ public void testMetadataFolderIsIntact() { .execute(); assertThat(result.orphanFileLocations()).as("Should delete 1 file").hasSize(1); + assertThat(result.deletedOrphanFilesCount()).as("Should delete 1 file").isEqualTo(1L); Dataset resultDF = spark.read().format("iceberg").load(tableLocation); List actualRecords = @@ -478,6 +494,7 @@ public void testOlderThanTimestamp() { .execute(); assertThat(result.orphanFileLocations()).as("Should delete only 2 files").hasSize(2); + assertThat(result.deletedOrphanFilesCount()).as("Should delete only 2 files").isEqualTo(2L); } @TestTemplate @@ -509,6 +526,7 @@ public void testRemoveUnreachableMetadataVersionFiles() { assertThat(result.orphanFileLocations()) .containsExactly(tableLocation + "metadata/v1.metadata.json"); + assertThat(result.deletedOrphanFilesCount()).as("Should delete 1 file").isEqualTo(1L); List expectedRecords = Lists.newArrayList(); expectedRecords.addAll(records); @@ -545,6 +563,7 @@ public void testManyTopLevelPartitions() { .execute(); assertThat(result.orphanFileLocations()).as("Should not delete any files").isEmpty(); + assertThat(result.deletedOrphanFilesCount()).as("Should not delete any files").isEqualTo(0L); Dataset resultDF = spark.read().format("iceberg").load(tableLocation); assertThat(resultDF.count()).as("Rows count must match").isEqualTo(records.size()); @@ -575,6 +594,7 @@ public void testManyLeafPartitions() { .execute(); assertThat(result.orphanFileLocations()).as("Should not delete any files").isEmpty(); + assertThat(result.deletedOrphanFilesCount()).as("Should not delete any files").isEqualTo(0L); Dataset resultDF = spark.read().format("iceberg").load(tableLocation); assertThat(resultDF.count()).as("Row count must match").isEqualTo(records.size()); @@ -615,6 +635,7 @@ public void testHiddenPartitionPaths() { .execute(); assertThat(result.orphanFileLocations()).as("Should delete 2 files").hasSize(2); + assertThat(result.deletedOrphanFilesCount()).as("Should delete 2 files").isEqualTo(2L); } @TestTemplate @@ -655,6 +676,7 @@ public void testHiddenPartitionPathsWithPartitionEvolution() { .execute(); assertThat(result.orphanFileLocations()).as("Should delete 2 files").hasSize(2); + assertThat(result.deletedOrphanFilesCount()).as("Should delete 2 files").isEqualTo(2L); } @TestTemplate @@ -694,6 +716,7 @@ public void testHiddenPathsStartingWithPartitionNamesAreIgnored() throws IOExcep .execute(); assertThat(result.orphanFileLocations()).as("Should delete 0 files").isEmpty(); + assertThat(result.deletedOrphanFilesCount()).as("Should delete 0 files").isEqualTo(0L); assertThat(fs.exists(pathToFileInHiddenFolder)).isTrue(); } @@ -766,6 +789,9 @@ public void testRemoveOrphanFilesWithRelativeFilePath() throws IOException { assertThat(result.orphanFileLocations()) .as("Action should find 1 file") .isEqualTo(invalidFiles); + assertThat(result.deletedOrphanFilesCount()) + .as("Action should find 1 file") + .isEqualTo((long) invalidFiles.size()); assertThat(fs.exists(new Path(invalidFiles.get(0)))) .as("Invalid file should be present") .isTrue(); @@ -803,6 +829,7 @@ public void testRemoveOrphanFilesWithHadoopCatalog() throws InterruptedException .execute(); assertThat(result.orphanFileLocations()).as("Should delete only 1 file").hasSize(1); + assertThat(result.deletedOrphanFilesCount()).as("Should delete only 1 file").isEqualTo(1L); Dataset resultDF = spark.read().format("iceberg").load(table.location()); List actualRecords = @@ -838,6 +865,7 @@ public void testHiveCatalogTable() throws IOException { assertThat(result.orphanFileLocations()) .as("trash file should be removed") .contains("file:" + location + "/data/trashfile"); + assertThat(result.deletedOrphanFilesCount()).as("trash file should be removed").isEqualTo(1L); } @TestTemplate @@ -932,6 +960,9 @@ public void testCompareToFileList() throws IOException { assertThat(result1.orphanFileLocations()) .as("Default olderThan interval should be safe") .isEmpty(); + assertThat(result1.deletedOrphanFilesCount()) + .as("Default olderThan interval should be safe") + .isEqualTo(0L); DeleteOrphanFiles.Result result2 = actions @@ -943,6 +974,9 @@ public void testCompareToFileList() throws IOException { assertThat(result2.orphanFileLocations()) .as("Action should find 1 file") .isEqualTo(invalidFilePaths); + assertThat(result2.deletedOrphanFilesCount()) + .as("Action should find 1 file") + .isEqualTo((long) invalidFilePaths.size()); assertThat(fs.exists(new Path(invalidFilePaths.get(0)))) .as("Invalid file should be present") .isTrue(); @@ -956,6 +990,9 @@ public void testCompareToFileList() throws IOException { assertThat(result3.orphanFileLocations()) .as("Action should delete 1 file") .isEqualTo(invalidFilePaths); + assertThat(result3.deletedOrphanFilesCount()) + .as("Action should delete 1 file") + .isEqualTo((long) invalidFilePaths.size()); assertThat(fs.exists(new Path(invalidFilePaths.get(0)))) .as("Invalid file should not be present") .isFalse(); @@ -985,6 +1022,7 @@ public void testCompareToFileList() throws IOException { .deleteWith(s -> {}) .execute(); assertThat(result4.orphanFileLocations()).as("Action should find nothing").isEmpty(); + assertThat(result4.deletedOrphanFilesCount()).as("Action should find nothing").isEqualTo(0L); } protected long waitUntilAfter(long timestampMillis) { @@ -1064,6 +1102,7 @@ public void testRemoveOrphanFilesWithStatisticFiles() throws Exception { .execute(); Iterable orphanFileLocations = result.orphanFileLocations(); assertThat(orphanFileLocations).hasSize(1).containsExactly(statsLocation.toURI().toString()); + assertThat(result.deletedOrphanFilesCount()).as("Should delete 1 file").isEqualTo(1L); assertThat(statsLocation).as("stats file should be deleted").doesNotExist(); } @@ -1282,6 +1321,9 @@ public void testStreamResultsDeletion() throws IOException { .as("Non-streaming dry-run should return all 10 orphan files") .hasSize(10) .containsExactlyInAnyOrderElementsOf(invalidFiles); + assertThat(nonStreamingResult.deletedOrphanFilesCount()) + .as("Non-streaming dry-run should return all 10 orphan files") + .isEqualTo((long) invalidFiles.size()); DeleteOrphanFiles.Result streamingResult = SparkActions.get() @@ -1295,6 +1337,9 @@ public void testStreamResultsDeletion() throws IOException { assertThat(streamingResult.orphanFileLocations()) .as("Streaming with sample size 5 should return only 5 orphan files") .hasSize(5); + assertThat(streamingResult.deletedOrphanFilesCount()) + .as("Deleted 10 files") + .isEqualTo((long) invalidFiles.size()); for (String invalidFile : invalidFiles) { assertThat(fs.exists(new Path(invalidFile))).as("Orphan file should be deleted").isFalse(); diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction3.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction3.java index 646e5f8e70d4..2659d519bba1 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction3.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction3.java @@ -21,6 +21,7 @@ import static org.assertj.core.api.Assertions.assertThat; import java.io.File; +import java.util.stream.StreamSupport; import org.apache.iceberg.actions.DeleteOrphanFiles; import org.apache.iceberg.spark.SparkCatalog; import org.apache.iceberg.spark.SparkSchemaUtil; @@ -59,6 +60,7 @@ public void testSparkCatalogTable() throws Exception { assertThat(results.orphanFileLocations()) .as("trash file should be removed") .contains("file:" + location + trashFile); + assertThat(results.deletedOrphanFilesCount()).as("trash file should be removed").isEqualTo(1L); } @TestTemplate @@ -88,6 +90,7 @@ public void testSparkCatalogNamedHadoopTable() throws Exception { assertThat(results.orphanFileLocations()) .as("trash file should be removed") .contains("file:" + location + trashFile); + assertThat(results.deletedOrphanFilesCount()).as("trash file should be removed").isEqualTo(1L); } @TestTemplate @@ -114,9 +117,11 @@ public void testSparkCatalogNamedHiveTable() throws Exception { .deleteOrphanFiles(table.table()) .olderThan(System.currentTimeMillis() + 1000) .execute(); - assertThat(results.orphanFileLocations()) + + assertThat(StreamSupport.stream(results.orphanFileLocations().spliterator(), false)) .as("trash file should be removed") - .contains("file:" + location + trashFile); + .anyMatch(file -> file.contains("file:" + location + trashFile)); + assertThat(results.deletedOrphanFilesCount()).as("trash file should be removed").isEqualTo(1L); } @TestTemplate @@ -149,6 +154,7 @@ public void testSparkSessionCatalogHadoopTable() throws Exception { assertThat(results.orphanFileLocations()) .as("trash file should be removed") .contains("file:" + location + trashFile); + assertThat(results.deletedOrphanFilesCount()).as("trash file should be removed").isEqualTo(1L); } @TestTemplate @@ -181,6 +187,7 @@ public void testSparkSessionCatalogHiveTable() throws Exception { assertThat(results.orphanFileLocations()) .as("trash file should be removed") .contains("file:" + location + trashFile); + assertThat(results.deletedOrphanFilesCount()).as("trash file should be removed").isEqualTo(1L); } @AfterEach diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java index 40505b856737..dffefed678e4 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java @@ -184,6 +184,9 @@ public void testDryRun() throws IOException { assertThat(result1.orphanFileLocations()) .as("Default olderThan interval should be safe") .isEmpty(); + assertThat(result1.deletedOrphanFilesCount()) + .as("Default olderThan interval should be safe") + .isEqualTo(0L); DeleteOrphanFiles.Result result2 = actions @@ -195,6 +198,9 @@ public void testDryRun() throws IOException { assertThat(result2.orphanFileLocations()) .as("Action should find 1 file") .isEqualTo(invalidFiles); + assertThat(result2.deletedOrphanFilesCount()) + .as("Action should find 1 file") + .isEqualTo((long) invalidFiles.size()); assertThat(fs.exists(new Path(invalidFiles.get(0)))) .as("Invalid file should be present") .isTrue(); @@ -210,6 +216,9 @@ public void testDryRun() throws IOException { assertThat(result3.orphanFileLocations()) .as("Streaming dry run should find 1 file") .isEqualTo(invalidFiles); + assertThat(result3.deletedOrphanFilesCount()) + .as("Streaming dry run should find 1 file") + .isEqualTo((long) invalidFiles.size()); assertThat(fs.exists(new Path(invalidFiles.get(0)))) .as("Invalid file should be present after streaming dry run") .isTrue(); @@ -223,6 +232,9 @@ public void testDryRun() throws IOException { assertThat(result4.orphanFileLocations()) .as("Action should delete 1 file") .isEqualTo(invalidFiles); + assertThat(result4.deletedOrphanFilesCount()) + .as("Action should delete 1 file") + .isEqualTo((long) invalidFiles.size()); assertThat(fs.exists(new Path(invalidFiles.get(0)))) .as("Invalid file should not be present") .isFalse(); @@ -286,6 +298,7 @@ public void testAllValidFilesAreKept() throws IOException { .execute(); assertThat(result.orphanFileLocations()).as("Should delete 4 files").hasSize(4); + assertThat(result.deletedOrphanFilesCount()).as("Should delete 4 files").isEqualTo(4L); Path dataPath = new Path(tableLocation + "/data"); FileSystem fs = dataPath.getFileSystem(spark.sessionState().newHadoopConf()); @@ -366,6 +379,7 @@ public void orphanedFileRemovedWithParallelTasks() { .containsExactlyInAnyOrder( "remove-orphan-0", "remove-orphan-1", "remove-orphan-2", "remove-orphan-3"); assertThat(deletedFiles).hasSize(4); + assertThat(result.deletedOrphanFilesCount()).as("Should delete 4 files").isEqualTo(4L); } @TestTemplate @@ -410,6 +424,7 @@ public void testWapFilesAreKept() { actions.deleteOrphanFiles(table).olderThan(System.currentTimeMillis()).execute(); assertThat(result.orphanFileLocations()).as("Should not delete any files").isEmpty(); + assertThat(result.deletedOrphanFilesCount()).as("Should not delete any files").isEqualTo(0L); } @TestTemplate @@ -440,6 +455,7 @@ public void testMetadataFolderIsIntact() { .execute(); assertThat(result.orphanFileLocations()).as("Should delete 1 file").hasSize(1); + assertThat(result.deletedOrphanFilesCount()).as("Should delete 1 file").isEqualTo(1L); Dataset resultDF = spark.read().format("iceberg").load(tableLocation); List actualRecords = @@ -478,6 +494,7 @@ public void testOlderThanTimestamp() { .execute(); assertThat(result.orphanFileLocations()).as("Should delete only 2 files").hasSize(2); + assertThat(result.deletedOrphanFilesCount()).as("Should delete only 2 files").isEqualTo(2L); } @TestTemplate @@ -509,6 +526,7 @@ public void testRemoveUnreachableMetadataVersionFiles() { assertThat(result.orphanFileLocations()) .containsExactly(tableLocation + "metadata/v1.metadata.json"); + assertThat(result.deletedOrphanFilesCount()).as("Should delete 1 file").isEqualTo(1L); List expectedRecords = Lists.newArrayList(); expectedRecords.addAll(records); @@ -545,6 +563,7 @@ public void testManyTopLevelPartitions() { .execute(); assertThat(result.orphanFileLocations()).as("Should not delete any files").isEmpty(); + assertThat(result.deletedOrphanFilesCount()).as("Should not delete any files").isEqualTo(0L); Dataset resultDF = spark.read().format("iceberg").load(tableLocation); assertThat(resultDF.count()).as("Rows count must match").isEqualTo(records.size()); @@ -575,6 +594,7 @@ public void testManyLeafPartitions() { .execute(); assertThat(result.orphanFileLocations()).as("Should not delete any files").isEmpty(); + assertThat(result.deletedOrphanFilesCount()).as("Should not delete any files").isEqualTo(0L); Dataset resultDF = spark.read().format("iceberg").load(tableLocation); assertThat(resultDF.count()).as("Row count must match").isEqualTo(records.size()); @@ -615,6 +635,7 @@ public void testHiddenPartitionPaths() { .execute(); assertThat(result.orphanFileLocations()).as("Should delete 2 files").hasSize(2); + assertThat(result.deletedOrphanFilesCount()).as("Should delete 2 files").isEqualTo(2L); } @TestTemplate @@ -655,6 +676,7 @@ public void testHiddenPartitionPathsWithPartitionEvolution() { .execute(); assertThat(result.orphanFileLocations()).as("Should delete 2 files").hasSize(2); + assertThat(result.deletedOrphanFilesCount()).as("Should delete 2 files").isEqualTo(2L); } @TestTemplate @@ -694,6 +716,7 @@ public void testHiddenPathsStartingWithPartitionNamesAreIgnored() throws IOExcep .execute(); assertThat(result.orphanFileLocations()).as("Should delete 0 files").isEmpty(); + assertThat(result.deletedOrphanFilesCount()).as("Should delete 0 files").isEqualTo(0L); assertThat(fs.exists(pathToFileInHiddenFolder)).isTrue(); } @@ -766,6 +789,9 @@ public void testRemoveOrphanFilesWithRelativeFilePath() throws IOException { assertThat(result.orphanFileLocations()) .as("Action should find 1 file") .isEqualTo(invalidFiles); + assertThat(result.deletedOrphanFilesCount()) + .as("Action should find 1 file") + .isEqualTo((long) invalidFiles.size()); assertThat(fs.exists(new Path(invalidFiles.get(0)))) .as("Invalid file should be present") .isTrue(); @@ -803,6 +829,7 @@ public void testRemoveOrphanFilesWithHadoopCatalog() throws InterruptedException .execute(); assertThat(result.orphanFileLocations()).as("Should delete only 1 file").hasSize(1); + assertThat(result.deletedOrphanFilesCount()).as("Should delete only 1 file").isEqualTo(1L); Dataset resultDF = spark.read().format("iceberg").load(table.location()); List actualRecords = @@ -838,6 +865,7 @@ public void testHiveCatalogTable() throws IOException { assertThat(result.orphanFileLocations()) .as("trash file should be removed") .contains("file:" + location + "/data/trashfile"); + assertThat(result.deletedOrphanFilesCount()).as("trash file should be removed").isEqualTo(1L); } @TestTemplate @@ -932,6 +960,9 @@ public void testCompareToFileList() throws IOException { assertThat(result1.orphanFileLocations()) .as("Default olderThan interval should be safe") .isEmpty(); + assertThat(result1.deletedOrphanFilesCount()) + .as("Default olderThan interval should be safe") + .isEqualTo(0L); DeleteOrphanFiles.Result result2 = actions @@ -943,6 +974,9 @@ public void testCompareToFileList() throws IOException { assertThat(result2.orphanFileLocations()) .as("Action should find 1 file") .isEqualTo(invalidFilePaths); + assertThat(result2.deletedOrphanFilesCount()) + .as("Action should find 1 file") + .isEqualTo((long) invalidFilePaths.size()); assertThat(fs.exists(new Path(invalidFilePaths.get(0)))) .as("Invalid file should be present") .isTrue(); @@ -956,6 +990,9 @@ public void testCompareToFileList() throws IOException { assertThat(result3.orphanFileLocations()) .as("Action should delete 1 file") .isEqualTo(invalidFilePaths); + assertThat(result3.deletedOrphanFilesCount()) + .as("Action should delete 1 file") + .isEqualTo((long) invalidFilePaths.size()); assertThat(fs.exists(new Path(invalidFilePaths.get(0)))) .as("Invalid file should not be present") .isFalse(); @@ -985,6 +1022,7 @@ public void testCompareToFileList() throws IOException { .deleteWith(s -> {}) .execute(); assertThat(result4.orphanFileLocations()).as("Action should find nothing").isEmpty(); + assertThat(result4.deletedOrphanFilesCount()).as("Action should find nothing").isEqualTo(0L); } protected long waitUntilAfter(long timestampMillis) { @@ -1064,6 +1102,7 @@ public void testRemoveOrphanFilesWithStatisticFiles() throws Exception { .execute(); Iterable orphanFileLocations = result.orphanFileLocations(); assertThat(orphanFileLocations).hasSize(1).containsExactly(statsLocation.toURI().toString()); + assertThat(result.deletedOrphanFilesCount()).as("Should delete 1 file").isEqualTo(1L); assertThat(statsLocation).as("stats file should be deleted").doesNotExist(); } @@ -1282,6 +1321,9 @@ public void testStreamResultsDeletion() throws IOException { .as("Non-streaming dry-run should return all 10 orphan files") .hasSize(10) .containsExactlyInAnyOrderElementsOf(invalidFiles); + assertThat(nonStreamingResult.deletedOrphanFilesCount()) + .as("Non-streaming dry-run should return all 10 orphan files") + .isEqualTo((long) invalidFiles.size()); DeleteOrphanFiles.Result streamingResult = SparkActions.get() @@ -1295,6 +1337,9 @@ public void testStreamResultsDeletion() throws IOException { assertThat(streamingResult.orphanFileLocations()) .as("Streaming with sample size 5 should return only 5 orphan files") .hasSize(5); + assertThat(streamingResult.deletedOrphanFilesCount()) + .as("Deleted 10 files") + .isEqualTo((long) invalidFiles.size()); for (String invalidFile : invalidFiles) { assertThat(fs.exists(new Path(invalidFile))).as("Orphan file should be deleted").isFalse(); diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction3.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction3.java index 5f98287951f1..2659d519bba1 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction3.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction3.java @@ -60,6 +60,7 @@ public void testSparkCatalogTable() throws Exception { assertThat(results.orphanFileLocations()) .as("trash file should be removed") .contains("file:" + location + trashFile); + assertThat(results.deletedOrphanFilesCount()).as("trash file should be removed").isEqualTo(1L); } @TestTemplate @@ -89,6 +90,7 @@ public void testSparkCatalogNamedHadoopTable() throws Exception { assertThat(results.orphanFileLocations()) .as("trash file should be removed") .contains("file:" + location + trashFile); + assertThat(results.deletedOrphanFilesCount()).as("trash file should be removed").isEqualTo(1L); } @TestTemplate @@ -119,6 +121,7 @@ public void testSparkCatalogNamedHiveTable() throws Exception { assertThat(StreamSupport.stream(results.orphanFileLocations().spliterator(), false)) .as("trash file should be removed") .anyMatch(file -> file.contains("file:" + location + trashFile)); + assertThat(results.deletedOrphanFilesCount()).as("trash file should be removed").isEqualTo(1L); } @TestTemplate @@ -151,6 +154,7 @@ public void testSparkSessionCatalogHadoopTable() throws Exception { assertThat(results.orphanFileLocations()) .as("trash file should be removed") .contains("file:" + location + trashFile); + assertThat(results.deletedOrphanFilesCount()).as("trash file should be removed").isEqualTo(1L); } @TestTemplate @@ -183,6 +187,7 @@ public void testSparkSessionCatalogHiveTable() throws Exception { assertThat(results.orphanFileLocations()) .as("trash file should be removed") .contains("file:" + location + trashFile); + assertThat(results.deletedOrphanFilesCount()).as("trash file should be removed").isEqualTo(1L); } @AfterEach diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java index 40505b856737..b3d65363c4de 100644 --- a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java @@ -184,6 +184,9 @@ public void testDryRun() throws IOException { assertThat(result1.orphanFileLocations()) .as("Default olderThan interval should be safe") .isEmpty(); + assertThat(result1.deletedOrphanFilesCount()) + .as("Default olderThan interval should be safe") + .isEqualTo(0L); DeleteOrphanFiles.Result result2 = actions @@ -195,6 +198,9 @@ public void testDryRun() throws IOException { assertThat(result2.orphanFileLocations()) .as("Action should find 1 file") .isEqualTo(invalidFiles); + assertThat(result2.deletedOrphanFilesCount()) + .as("Action should find 1 file") + .isEqualTo(invalidFiles.size()); assertThat(fs.exists(new Path(invalidFiles.get(0)))) .as("Invalid file should be present") .isTrue(); @@ -210,6 +216,9 @@ public void testDryRun() throws IOException { assertThat(result3.orphanFileLocations()) .as("Streaming dry run should find 1 file") .isEqualTo(invalidFiles); + assertThat(result3.deletedOrphanFilesCount()) + .as("Streaming dry run should find 1 file") + .isEqualTo(invalidFiles.size()); assertThat(fs.exists(new Path(invalidFiles.get(0)))) .as("Invalid file should be present after streaming dry run") .isTrue(); @@ -223,6 +232,9 @@ public void testDryRun() throws IOException { assertThat(result4.orphanFileLocations()) .as("Action should delete 1 file") .isEqualTo(invalidFiles); + assertThat(result4.deletedOrphanFilesCount()) + .as("Action should delete 1 file") + .isEqualTo(invalidFiles.size()); assertThat(fs.exists(new Path(invalidFiles.get(0)))) .as("Invalid file should not be present") .isFalse(); @@ -286,6 +298,7 @@ public void testAllValidFilesAreKept() throws IOException { .execute(); assertThat(result.orphanFileLocations()).as("Should delete 4 files").hasSize(4); + assertThat(result.deletedOrphanFilesCount()).as("Should delete 4 files").isEqualTo(4L); Path dataPath = new Path(tableLocation + "/data"); FileSystem fs = dataPath.getFileSystem(spark.sessionState().newHadoopConf()); @@ -366,6 +379,7 @@ public void orphanedFileRemovedWithParallelTasks() { .containsExactlyInAnyOrder( "remove-orphan-0", "remove-orphan-1", "remove-orphan-2", "remove-orphan-3"); assertThat(deletedFiles).hasSize(4); + assertThat(result.deletedOrphanFilesCount()).as("Should delete 4 files").isEqualTo(4L); } @TestTemplate @@ -410,6 +424,7 @@ public void testWapFilesAreKept() { actions.deleteOrphanFiles(table).olderThan(System.currentTimeMillis()).execute(); assertThat(result.orphanFileLocations()).as("Should not delete any files").isEmpty(); + assertThat(result.deletedOrphanFilesCount()).as("Should not delete any files").isEqualTo(0L); } @TestTemplate @@ -440,6 +455,7 @@ public void testMetadataFolderIsIntact() { .execute(); assertThat(result.orphanFileLocations()).as("Should delete 1 file").hasSize(1); + assertThat(result.deletedOrphanFilesCount()).as("Should delete 1 file").isEqualTo(1L); Dataset resultDF = spark.read().format("iceberg").load(tableLocation); List actualRecords = @@ -478,6 +494,7 @@ public void testOlderThanTimestamp() { .execute(); assertThat(result.orphanFileLocations()).as("Should delete only 2 files").hasSize(2); + assertThat(result.deletedOrphanFilesCount()).as("Should delete only 2 files").isEqualTo(2L); } @TestTemplate @@ -509,6 +526,7 @@ public void testRemoveUnreachableMetadataVersionFiles() { assertThat(result.orphanFileLocations()) .containsExactly(tableLocation + "metadata/v1.metadata.json"); + assertThat(result.deletedOrphanFilesCount()).as("Should delete 1 file").isEqualTo(1L); List expectedRecords = Lists.newArrayList(); expectedRecords.addAll(records); @@ -545,6 +563,7 @@ public void testManyTopLevelPartitions() { .execute(); assertThat(result.orphanFileLocations()).as("Should not delete any files").isEmpty(); + assertThat(result.deletedOrphanFilesCount()).as("Should not delete any files").isEqualTo(0L); Dataset resultDF = spark.read().format("iceberg").load(tableLocation); assertThat(resultDF.count()).as("Rows count must match").isEqualTo(records.size()); @@ -575,6 +594,7 @@ public void testManyLeafPartitions() { .execute(); assertThat(result.orphanFileLocations()).as("Should not delete any files").isEmpty(); + assertThat(result.deletedOrphanFilesCount()).as("Should not delete any files").isEqualTo(0L); Dataset resultDF = spark.read().format("iceberg").load(tableLocation); assertThat(resultDF.count()).as("Row count must match").isEqualTo(records.size()); @@ -615,6 +635,7 @@ public void testHiddenPartitionPaths() { .execute(); assertThat(result.orphanFileLocations()).as("Should delete 2 files").hasSize(2); + assertThat(result.deletedOrphanFilesCount()).as("Should delete 2 files").isEqualTo(2L); } @TestTemplate @@ -655,6 +676,7 @@ public void testHiddenPartitionPathsWithPartitionEvolution() { .execute(); assertThat(result.orphanFileLocations()).as("Should delete 2 files").hasSize(2); + assertThat(result.deletedOrphanFilesCount()).as("Should delete 2 files").isEqualTo(2L); } @TestTemplate @@ -694,6 +716,7 @@ public void testHiddenPathsStartingWithPartitionNamesAreIgnored() throws IOExcep .execute(); assertThat(result.orphanFileLocations()).as("Should delete 0 files").isEmpty(); + assertThat(result.deletedOrphanFilesCount()).as("Should delete 0 files").isEqualTo(0L); assertThat(fs.exists(pathToFileInHiddenFolder)).isTrue(); } @@ -766,6 +789,9 @@ public void testRemoveOrphanFilesWithRelativeFilePath() throws IOException { assertThat(result.orphanFileLocations()) .as("Action should find 1 file") .isEqualTo(invalidFiles); + assertThat(result.deletedOrphanFilesCount()) + .as("Action should find 1 file") + .isEqualTo(invalidFiles.size()); assertThat(fs.exists(new Path(invalidFiles.get(0)))) .as("Invalid file should be present") .isTrue(); @@ -803,6 +829,7 @@ public void testRemoveOrphanFilesWithHadoopCatalog() throws InterruptedException .execute(); assertThat(result.orphanFileLocations()).as("Should delete only 1 file").hasSize(1); + assertThat(result.deletedOrphanFilesCount()).as("Should delete only 1 file").isEqualTo(1L); Dataset resultDF = spark.read().format("iceberg").load(table.location()); List actualRecords = @@ -838,6 +865,7 @@ public void testHiveCatalogTable() throws IOException { assertThat(result.orphanFileLocations()) .as("trash file should be removed") .contains("file:" + location + "/data/trashfile"); + assertThat(result.deletedOrphanFilesCount()).as("trash file should be removed").isEqualTo(1L); } @TestTemplate @@ -932,6 +960,9 @@ public void testCompareToFileList() throws IOException { assertThat(result1.orphanFileLocations()) .as("Default olderThan interval should be safe") .isEmpty(); + assertThat(result1.deletedOrphanFilesCount()) + .as("Default olderThan interval should be safe") + .isEqualTo(0L); DeleteOrphanFiles.Result result2 = actions @@ -943,6 +974,9 @@ public void testCompareToFileList() throws IOException { assertThat(result2.orphanFileLocations()) .as("Action should find 1 file") .isEqualTo(invalidFilePaths); + assertThat(result2.deletedOrphanFilesCount()) + .as("Action should find 1 file") + .isEqualTo(invalidFilePaths.size()); assertThat(fs.exists(new Path(invalidFilePaths.get(0)))) .as("Invalid file should be present") .isTrue(); @@ -956,6 +990,9 @@ public void testCompareToFileList() throws IOException { assertThat(result3.orphanFileLocations()) .as("Action should delete 1 file") .isEqualTo(invalidFilePaths); + assertThat(result3.deletedOrphanFilesCount()) + .as("Action should delete 1 file") + .isEqualTo(invalidFilePaths.size()); assertThat(fs.exists(new Path(invalidFilePaths.get(0)))) .as("Invalid file should not be present") .isFalse(); @@ -985,6 +1022,7 @@ public void testCompareToFileList() throws IOException { .deleteWith(s -> {}) .execute(); assertThat(result4.orphanFileLocations()).as("Action should find nothing").isEmpty(); + assertThat(result4.deletedOrphanFilesCount()).as("Action should find nothing").isEqualTo(0L); } protected long waitUntilAfter(long timestampMillis) { @@ -1064,6 +1102,7 @@ public void testRemoveOrphanFilesWithStatisticFiles() throws Exception { .execute(); Iterable orphanFileLocations = result.orphanFileLocations(); assertThat(orphanFileLocations).hasSize(1).containsExactly(statsLocation.toURI().toString()); + assertThat(result.deletedOrphanFilesCount()).as("Should delete 1 file").isEqualTo(1L); assertThat(statsLocation).as("stats file should be deleted").doesNotExist(); } @@ -1282,6 +1321,9 @@ public void testStreamResultsDeletion() throws IOException { .as("Non-streaming dry-run should return all 10 orphan files") .hasSize(10) .containsExactlyInAnyOrderElementsOf(invalidFiles); + assertThat(nonStreamingResult.deletedOrphanFilesCount()) + .as("Non-streaming dry-run should return all 10 orphan files") + .isEqualTo(invalidFiles.size()); DeleteOrphanFiles.Result streamingResult = SparkActions.get() @@ -1295,6 +1337,9 @@ public void testStreamResultsDeletion() throws IOException { assertThat(streamingResult.orphanFileLocations()) .as("Streaming with sample size 5 should return only 5 orphan files") .hasSize(5); + assertThat(streamingResult.deletedOrphanFilesCount()) + .as("Deleted 10 files") + .isEqualTo(invalidFiles.size()); for (String invalidFile : invalidFiles) { assertThat(fs.exists(new Path(invalidFile))).as("Orphan file should be deleted").isFalse(); diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction3.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction3.java index 5f98287951f1..2659d519bba1 100644 --- a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction3.java +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction3.java @@ -60,6 +60,7 @@ public void testSparkCatalogTable() throws Exception { assertThat(results.orphanFileLocations()) .as("trash file should be removed") .contains("file:" + location + trashFile); + assertThat(results.deletedOrphanFilesCount()).as("trash file should be removed").isEqualTo(1L); } @TestTemplate @@ -89,6 +90,7 @@ public void testSparkCatalogNamedHadoopTable() throws Exception { assertThat(results.orphanFileLocations()) .as("trash file should be removed") .contains("file:" + location + trashFile); + assertThat(results.deletedOrphanFilesCount()).as("trash file should be removed").isEqualTo(1L); } @TestTemplate @@ -119,6 +121,7 @@ public void testSparkCatalogNamedHiveTable() throws Exception { assertThat(StreamSupport.stream(results.orphanFileLocations().spliterator(), false)) .as("trash file should be removed") .anyMatch(file -> file.contains("file:" + location + trashFile)); + assertThat(results.deletedOrphanFilesCount()).as("trash file should be removed").isEqualTo(1L); } @TestTemplate @@ -151,6 +154,7 @@ public void testSparkSessionCatalogHadoopTable() throws Exception { assertThat(results.orphanFileLocations()) .as("trash file should be removed") .contains("file:" + location + trashFile); + assertThat(results.deletedOrphanFilesCount()).as("trash file should be removed").isEqualTo(1L); } @TestTemplate @@ -183,6 +187,7 @@ public void testSparkSessionCatalogHiveTable() throws Exception { assertThat(results.orphanFileLocations()) .as("trash file should be removed") .contains("file:" + location + trashFile); + assertThat(results.deletedOrphanFilesCount()).as("trash file should be removed").isEqualTo(1L); } @AfterEach From 05121b4ba97d9a3417425f6f4cc79c5c55396425 Mon Sep 17 00:00:00 2001 From: Alessandro Nori Date: Fri, 19 Dec 2025 08:16:10 +0100 Subject: [PATCH 3/8] rename to orphanFilesCount() --- .../iceberg/actions/DeleteOrphanFiles.java | 4 +- .../actions/BaseDeleteOrphanFiles.java | 2 +- .../actions/DeleteOrphanFilesSparkAction.java | 2 +- .../actions/TestRemoveOrphanFilesAction.java | 50 +++++++++---------- .../actions/TestRemoveOrphanFilesAction3.java | 10 ++-- .../actions/DeleteOrphanFilesSparkAction.java | 2 +- .../actions/TestRemoveOrphanFilesAction.java | 50 +++++++++---------- .../actions/TestRemoveOrphanFilesAction3.java | 10 ++-- .../actions/DeleteOrphanFilesSparkAction.java | 2 +- .../actions/TestRemoveOrphanFilesAction.java | 50 +++++++++---------- .../actions/TestRemoveOrphanFilesAction3.java | 10 ++-- 11 files changed, 96 insertions(+), 96 deletions(-) diff --git a/api/src/main/java/org/apache/iceberg/actions/DeleteOrphanFiles.java b/api/src/main/java/org/apache/iceberg/actions/DeleteOrphanFiles.java index 521d4d47b1f8..ab12a3b7c1e3 100644 --- a/api/src/main/java/org/apache/iceberg/actions/DeleteOrphanFiles.java +++ b/api/src/main/java/org/apache/iceberg/actions/DeleteOrphanFiles.java @@ -143,8 +143,8 @@ interface Result { /** Returns locations of orphan files. */ Iterable orphanFileLocations(); - /** Returns the number of deleted orphan files. */ - default long deletedOrphanFilesCount() { + /** Returns the total number of orphan files. */ + default long orphanFilesCount() { return 0; } } diff --git a/core/src/main/java/org/apache/iceberg/actions/BaseDeleteOrphanFiles.java b/core/src/main/java/org/apache/iceberg/actions/BaseDeleteOrphanFiles.java index 54ee2d391b1a..cfa7a80b8870 100644 --- a/core/src/main/java/org/apache/iceberg/actions/BaseDeleteOrphanFiles.java +++ b/core/src/main/java/org/apache/iceberg/actions/BaseDeleteOrphanFiles.java @@ -31,6 +31,6 @@ interface BaseDeleteOrphanFiles extends DeleteOrphanFiles { @Value.Immutable interface Result extends DeleteOrphanFiles.Result { @Override - long deletedOrphanFilesCount(); + long orphanFilesCount(); } } diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/DeleteOrphanFilesSparkAction.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/DeleteOrphanFilesSparkAction.java index 031214ea8468..92bfc880ad7f 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/DeleteOrphanFilesSparkAction.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/DeleteOrphanFilesSparkAction.java @@ -302,7 +302,7 @@ private DeleteOrphanFiles.Result deleteFiles(Dataset orphanFileDS) { return ImmutableDeleteOrphanFiles.Result.builder() .orphanFileLocations(orphanFileList) - .deletedOrphanFilesCount(filesCount) + .orphanFilesCount(filesCount) .build(); } diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java index dffefed678e4..64414e4513ab 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java @@ -184,7 +184,7 @@ public void testDryRun() throws IOException { assertThat(result1.orphanFileLocations()) .as("Default olderThan interval should be safe") .isEmpty(); - assertThat(result1.deletedOrphanFilesCount()) + assertThat(result1.orphanFilesCount()) .as("Default olderThan interval should be safe") .isEqualTo(0L); @@ -198,7 +198,7 @@ public void testDryRun() throws IOException { assertThat(result2.orphanFileLocations()) .as("Action should find 1 file") .isEqualTo(invalidFiles); - assertThat(result2.deletedOrphanFilesCount()) + assertThat(result2.orphanFilesCount()) .as("Action should find 1 file") .isEqualTo((long) invalidFiles.size()); assertThat(fs.exists(new Path(invalidFiles.get(0)))) @@ -216,7 +216,7 @@ public void testDryRun() throws IOException { assertThat(result3.orphanFileLocations()) .as("Streaming dry run should find 1 file") .isEqualTo(invalidFiles); - assertThat(result3.deletedOrphanFilesCount()) + assertThat(result3.orphanFilesCount()) .as("Streaming dry run should find 1 file") .isEqualTo((long) invalidFiles.size()); assertThat(fs.exists(new Path(invalidFiles.get(0)))) @@ -232,7 +232,7 @@ public void testDryRun() throws IOException { assertThat(result4.orphanFileLocations()) .as("Action should delete 1 file") .isEqualTo(invalidFiles); - assertThat(result4.deletedOrphanFilesCount()) + assertThat(result4.orphanFilesCount()) .as("Action should delete 1 file") .isEqualTo((long) invalidFiles.size()); assertThat(fs.exists(new Path(invalidFiles.get(0)))) @@ -298,7 +298,7 @@ public void testAllValidFilesAreKept() throws IOException { .execute(); assertThat(result.orphanFileLocations()).as("Should delete 4 files").hasSize(4); - assertThat(result.deletedOrphanFilesCount()).as("Should delete 4 files").isEqualTo(4L); + assertThat(result.orphanFilesCount()).as("Should delete 4 files").isEqualTo(4L); Path dataPath = new Path(tableLocation + "/data"); FileSystem fs = dataPath.getFileSystem(spark.sessionState().newHadoopConf()); @@ -379,7 +379,7 @@ public void orphanedFileRemovedWithParallelTasks() { .containsExactlyInAnyOrder( "remove-orphan-0", "remove-orphan-1", "remove-orphan-2", "remove-orphan-3"); assertThat(deletedFiles).hasSize(4); - assertThat(result.deletedOrphanFilesCount()).as("Should delete 4 files").isEqualTo(4L); + assertThat(result.orphanFilesCount()).as("Should delete 4 files").isEqualTo(4L); } @TestTemplate @@ -424,7 +424,7 @@ public void testWapFilesAreKept() { actions.deleteOrphanFiles(table).olderThan(System.currentTimeMillis()).execute(); assertThat(result.orphanFileLocations()).as("Should not delete any files").isEmpty(); - assertThat(result.deletedOrphanFilesCount()).as("Should not delete any files").isEqualTo(0L); + assertThat(result.orphanFilesCount()).as("Should not delete any files").isEqualTo(0L); } @TestTemplate @@ -455,7 +455,7 @@ public void testMetadataFolderIsIntact() { .execute(); assertThat(result.orphanFileLocations()).as("Should delete 1 file").hasSize(1); - assertThat(result.deletedOrphanFilesCount()).as("Should delete 1 file").isEqualTo(1L); + assertThat(result.orphanFilesCount()).as("Should delete 1 file").isEqualTo(1L); Dataset resultDF = spark.read().format("iceberg").load(tableLocation); List actualRecords = @@ -494,7 +494,7 @@ public void testOlderThanTimestamp() { .execute(); assertThat(result.orphanFileLocations()).as("Should delete only 2 files").hasSize(2); - assertThat(result.deletedOrphanFilesCount()).as("Should delete only 2 files").isEqualTo(2L); + assertThat(result.orphanFilesCount()).as("Should delete only 2 files").isEqualTo(2L); } @TestTemplate @@ -526,7 +526,7 @@ public void testRemoveUnreachableMetadataVersionFiles() { assertThat(result.orphanFileLocations()) .containsExactly(tableLocation + "metadata/v1.metadata.json"); - assertThat(result.deletedOrphanFilesCount()).as("Should delete 1 file").isEqualTo(1L); + assertThat(result.orphanFilesCount()).as("Should delete 1 file").isEqualTo(1L); List expectedRecords = Lists.newArrayList(); expectedRecords.addAll(records); @@ -563,7 +563,7 @@ public void testManyTopLevelPartitions() { .execute(); assertThat(result.orphanFileLocations()).as("Should not delete any files").isEmpty(); - assertThat(result.deletedOrphanFilesCount()).as("Should not delete any files").isEqualTo(0L); + assertThat(result.orphanFilesCount()).as("Should not delete any files").isEqualTo(0L); Dataset resultDF = spark.read().format("iceberg").load(tableLocation); assertThat(resultDF.count()).as("Rows count must match").isEqualTo(records.size()); @@ -594,7 +594,7 @@ public void testManyLeafPartitions() { .execute(); assertThat(result.orphanFileLocations()).as("Should not delete any files").isEmpty(); - assertThat(result.deletedOrphanFilesCount()).as("Should not delete any files").isEqualTo(0L); + assertThat(result.orphanFilesCount()).as("Should not delete any files").isEqualTo(0L); Dataset resultDF = spark.read().format("iceberg").load(tableLocation); assertThat(resultDF.count()).as("Row count must match").isEqualTo(records.size()); @@ -635,7 +635,7 @@ public void testHiddenPartitionPaths() { .execute(); assertThat(result.orphanFileLocations()).as("Should delete 2 files").hasSize(2); - assertThat(result.deletedOrphanFilesCount()).as("Should delete 2 files").isEqualTo(2L); + assertThat(result.orphanFilesCount()).as("Should delete 2 files").isEqualTo(2L); } @TestTemplate @@ -676,7 +676,7 @@ public void testHiddenPartitionPathsWithPartitionEvolution() { .execute(); assertThat(result.orphanFileLocations()).as("Should delete 2 files").hasSize(2); - assertThat(result.deletedOrphanFilesCount()).as("Should delete 2 files").isEqualTo(2L); + assertThat(result.orphanFilesCount()).as("Should delete 2 files").isEqualTo(2L); } @TestTemplate @@ -716,7 +716,7 @@ public void testHiddenPathsStartingWithPartitionNamesAreIgnored() throws IOExcep .execute(); assertThat(result.orphanFileLocations()).as("Should delete 0 files").isEmpty(); - assertThat(result.deletedOrphanFilesCount()).as("Should delete 0 files").isEqualTo(0L); + assertThat(result.orphanFilesCount()).as("Should delete 0 files").isEqualTo(0L); assertThat(fs.exists(pathToFileInHiddenFolder)).isTrue(); } @@ -789,7 +789,7 @@ public void testRemoveOrphanFilesWithRelativeFilePath() throws IOException { assertThat(result.orphanFileLocations()) .as("Action should find 1 file") .isEqualTo(invalidFiles); - assertThat(result.deletedOrphanFilesCount()) + assertThat(result.orphanFilesCount()) .as("Action should find 1 file") .isEqualTo((long) invalidFiles.size()); assertThat(fs.exists(new Path(invalidFiles.get(0)))) @@ -829,7 +829,7 @@ public void testRemoveOrphanFilesWithHadoopCatalog() throws InterruptedException .execute(); assertThat(result.orphanFileLocations()).as("Should delete only 1 file").hasSize(1); - assertThat(result.deletedOrphanFilesCount()).as("Should delete only 1 file").isEqualTo(1L); + assertThat(result.orphanFilesCount()).as("Should delete only 1 file").isEqualTo(1L); Dataset resultDF = spark.read().format("iceberg").load(table.location()); List actualRecords = @@ -865,7 +865,7 @@ public void testHiveCatalogTable() throws IOException { assertThat(result.orphanFileLocations()) .as("trash file should be removed") .contains("file:" + location + "/data/trashfile"); - assertThat(result.deletedOrphanFilesCount()).as("trash file should be removed").isEqualTo(1L); + assertThat(result.orphanFilesCount()).as("trash file should be removed").isEqualTo(1L); } @TestTemplate @@ -960,7 +960,7 @@ public void testCompareToFileList() throws IOException { assertThat(result1.orphanFileLocations()) .as("Default olderThan interval should be safe") .isEmpty(); - assertThat(result1.deletedOrphanFilesCount()) + assertThat(result1.orphanFilesCount()) .as("Default olderThan interval should be safe") .isEqualTo(0L); @@ -974,7 +974,7 @@ public void testCompareToFileList() throws IOException { assertThat(result2.orphanFileLocations()) .as("Action should find 1 file") .isEqualTo(invalidFilePaths); - assertThat(result2.deletedOrphanFilesCount()) + assertThat(result2.orphanFilesCount()) .as("Action should find 1 file") .isEqualTo((long) invalidFilePaths.size()); assertThat(fs.exists(new Path(invalidFilePaths.get(0)))) @@ -990,7 +990,7 @@ public void testCompareToFileList() throws IOException { assertThat(result3.orphanFileLocations()) .as("Action should delete 1 file") .isEqualTo(invalidFilePaths); - assertThat(result3.deletedOrphanFilesCount()) + assertThat(result3.orphanFilesCount()) .as("Action should delete 1 file") .isEqualTo((long) invalidFilePaths.size()); assertThat(fs.exists(new Path(invalidFilePaths.get(0)))) @@ -1022,7 +1022,7 @@ public void testCompareToFileList() throws IOException { .deleteWith(s -> {}) .execute(); assertThat(result4.orphanFileLocations()).as("Action should find nothing").isEmpty(); - assertThat(result4.deletedOrphanFilesCount()).as("Action should find nothing").isEqualTo(0L); + assertThat(result4.orphanFilesCount()).as("Action should find nothing").isEqualTo(0L); } protected long waitUntilAfter(long timestampMillis) { @@ -1102,7 +1102,7 @@ public void testRemoveOrphanFilesWithStatisticFiles() throws Exception { .execute(); Iterable orphanFileLocations = result.orphanFileLocations(); assertThat(orphanFileLocations).hasSize(1).containsExactly(statsLocation.toURI().toString()); - assertThat(result.deletedOrphanFilesCount()).as("Should delete 1 file").isEqualTo(1L); + assertThat(result.orphanFilesCount()).as("Should delete 1 file").isEqualTo(1L); assertThat(statsLocation).as("stats file should be deleted").doesNotExist(); } @@ -1321,7 +1321,7 @@ public void testStreamResultsDeletion() throws IOException { .as("Non-streaming dry-run should return all 10 orphan files") .hasSize(10) .containsExactlyInAnyOrderElementsOf(invalidFiles); - assertThat(nonStreamingResult.deletedOrphanFilesCount()) + assertThat(nonStreamingResult.orphanFilesCount()) .as("Non-streaming dry-run should return all 10 orphan files") .isEqualTo((long) invalidFiles.size()); @@ -1337,7 +1337,7 @@ public void testStreamResultsDeletion() throws IOException { assertThat(streamingResult.orphanFileLocations()) .as("Streaming with sample size 5 should return only 5 orphan files") .hasSize(5); - assertThat(streamingResult.deletedOrphanFilesCount()) + assertThat(streamingResult.orphanFilesCount()) .as("Deleted 10 files") .isEqualTo((long) invalidFiles.size()); diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction3.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction3.java index 2659d519bba1..88ac800b158f 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction3.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction3.java @@ -60,7 +60,7 @@ public void testSparkCatalogTable() throws Exception { assertThat(results.orphanFileLocations()) .as("trash file should be removed") .contains("file:" + location + trashFile); - assertThat(results.deletedOrphanFilesCount()).as("trash file should be removed").isEqualTo(1L); + assertThat(results.orphanFilesCount()).as("trash file should be removed").isEqualTo(1L); } @TestTemplate @@ -90,7 +90,7 @@ public void testSparkCatalogNamedHadoopTable() throws Exception { assertThat(results.orphanFileLocations()) .as("trash file should be removed") .contains("file:" + location + trashFile); - assertThat(results.deletedOrphanFilesCount()).as("trash file should be removed").isEqualTo(1L); + assertThat(results.orphanFilesCount()).as("trash file should be removed").isEqualTo(1L); } @TestTemplate @@ -121,7 +121,7 @@ public void testSparkCatalogNamedHiveTable() throws Exception { assertThat(StreamSupport.stream(results.orphanFileLocations().spliterator(), false)) .as("trash file should be removed") .anyMatch(file -> file.contains("file:" + location + trashFile)); - assertThat(results.deletedOrphanFilesCount()).as("trash file should be removed").isEqualTo(1L); + assertThat(results.orphanFilesCount()).as("trash file should be removed").isEqualTo(1L); } @TestTemplate @@ -154,7 +154,7 @@ public void testSparkSessionCatalogHadoopTable() throws Exception { assertThat(results.orphanFileLocations()) .as("trash file should be removed") .contains("file:" + location + trashFile); - assertThat(results.deletedOrphanFilesCount()).as("trash file should be removed").isEqualTo(1L); + assertThat(results.orphanFilesCount()).as("trash file should be removed").isEqualTo(1L); } @TestTemplate @@ -187,7 +187,7 @@ public void testSparkSessionCatalogHiveTable() throws Exception { assertThat(results.orphanFileLocations()) .as("trash file should be removed") .contains("file:" + location + trashFile); - assertThat(results.deletedOrphanFilesCount()).as("trash file should be removed").isEqualTo(1L); + assertThat(results.orphanFilesCount()).as("trash file should be removed").isEqualTo(1L); } @AfterEach diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/DeleteOrphanFilesSparkAction.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/DeleteOrphanFilesSparkAction.java index 031214ea8468..92bfc880ad7f 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/DeleteOrphanFilesSparkAction.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/DeleteOrphanFilesSparkAction.java @@ -302,7 +302,7 @@ private DeleteOrphanFiles.Result deleteFiles(Dataset orphanFileDS) { return ImmutableDeleteOrphanFiles.Result.builder() .orphanFileLocations(orphanFileList) - .deletedOrphanFilesCount(filesCount) + .orphanFilesCount(filesCount) .build(); } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java index dffefed678e4..64414e4513ab 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java @@ -184,7 +184,7 @@ public void testDryRun() throws IOException { assertThat(result1.orphanFileLocations()) .as("Default olderThan interval should be safe") .isEmpty(); - assertThat(result1.deletedOrphanFilesCount()) + assertThat(result1.orphanFilesCount()) .as("Default olderThan interval should be safe") .isEqualTo(0L); @@ -198,7 +198,7 @@ public void testDryRun() throws IOException { assertThat(result2.orphanFileLocations()) .as("Action should find 1 file") .isEqualTo(invalidFiles); - assertThat(result2.deletedOrphanFilesCount()) + assertThat(result2.orphanFilesCount()) .as("Action should find 1 file") .isEqualTo((long) invalidFiles.size()); assertThat(fs.exists(new Path(invalidFiles.get(0)))) @@ -216,7 +216,7 @@ public void testDryRun() throws IOException { assertThat(result3.orphanFileLocations()) .as("Streaming dry run should find 1 file") .isEqualTo(invalidFiles); - assertThat(result3.deletedOrphanFilesCount()) + assertThat(result3.orphanFilesCount()) .as("Streaming dry run should find 1 file") .isEqualTo((long) invalidFiles.size()); assertThat(fs.exists(new Path(invalidFiles.get(0)))) @@ -232,7 +232,7 @@ public void testDryRun() throws IOException { assertThat(result4.orphanFileLocations()) .as("Action should delete 1 file") .isEqualTo(invalidFiles); - assertThat(result4.deletedOrphanFilesCount()) + assertThat(result4.orphanFilesCount()) .as("Action should delete 1 file") .isEqualTo((long) invalidFiles.size()); assertThat(fs.exists(new Path(invalidFiles.get(0)))) @@ -298,7 +298,7 @@ public void testAllValidFilesAreKept() throws IOException { .execute(); assertThat(result.orphanFileLocations()).as("Should delete 4 files").hasSize(4); - assertThat(result.deletedOrphanFilesCount()).as("Should delete 4 files").isEqualTo(4L); + assertThat(result.orphanFilesCount()).as("Should delete 4 files").isEqualTo(4L); Path dataPath = new Path(tableLocation + "/data"); FileSystem fs = dataPath.getFileSystem(spark.sessionState().newHadoopConf()); @@ -379,7 +379,7 @@ public void orphanedFileRemovedWithParallelTasks() { .containsExactlyInAnyOrder( "remove-orphan-0", "remove-orphan-1", "remove-orphan-2", "remove-orphan-3"); assertThat(deletedFiles).hasSize(4); - assertThat(result.deletedOrphanFilesCount()).as("Should delete 4 files").isEqualTo(4L); + assertThat(result.orphanFilesCount()).as("Should delete 4 files").isEqualTo(4L); } @TestTemplate @@ -424,7 +424,7 @@ public void testWapFilesAreKept() { actions.deleteOrphanFiles(table).olderThan(System.currentTimeMillis()).execute(); assertThat(result.orphanFileLocations()).as("Should not delete any files").isEmpty(); - assertThat(result.deletedOrphanFilesCount()).as("Should not delete any files").isEqualTo(0L); + assertThat(result.orphanFilesCount()).as("Should not delete any files").isEqualTo(0L); } @TestTemplate @@ -455,7 +455,7 @@ public void testMetadataFolderIsIntact() { .execute(); assertThat(result.orphanFileLocations()).as("Should delete 1 file").hasSize(1); - assertThat(result.deletedOrphanFilesCount()).as("Should delete 1 file").isEqualTo(1L); + assertThat(result.orphanFilesCount()).as("Should delete 1 file").isEqualTo(1L); Dataset resultDF = spark.read().format("iceberg").load(tableLocation); List actualRecords = @@ -494,7 +494,7 @@ public void testOlderThanTimestamp() { .execute(); assertThat(result.orphanFileLocations()).as("Should delete only 2 files").hasSize(2); - assertThat(result.deletedOrphanFilesCount()).as("Should delete only 2 files").isEqualTo(2L); + assertThat(result.orphanFilesCount()).as("Should delete only 2 files").isEqualTo(2L); } @TestTemplate @@ -526,7 +526,7 @@ public void testRemoveUnreachableMetadataVersionFiles() { assertThat(result.orphanFileLocations()) .containsExactly(tableLocation + "metadata/v1.metadata.json"); - assertThat(result.deletedOrphanFilesCount()).as("Should delete 1 file").isEqualTo(1L); + assertThat(result.orphanFilesCount()).as("Should delete 1 file").isEqualTo(1L); List expectedRecords = Lists.newArrayList(); expectedRecords.addAll(records); @@ -563,7 +563,7 @@ public void testManyTopLevelPartitions() { .execute(); assertThat(result.orphanFileLocations()).as("Should not delete any files").isEmpty(); - assertThat(result.deletedOrphanFilesCount()).as("Should not delete any files").isEqualTo(0L); + assertThat(result.orphanFilesCount()).as("Should not delete any files").isEqualTo(0L); Dataset resultDF = spark.read().format("iceberg").load(tableLocation); assertThat(resultDF.count()).as("Rows count must match").isEqualTo(records.size()); @@ -594,7 +594,7 @@ public void testManyLeafPartitions() { .execute(); assertThat(result.orphanFileLocations()).as("Should not delete any files").isEmpty(); - assertThat(result.deletedOrphanFilesCount()).as("Should not delete any files").isEqualTo(0L); + assertThat(result.orphanFilesCount()).as("Should not delete any files").isEqualTo(0L); Dataset resultDF = spark.read().format("iceberg").load(tableLocation); assertThat(resultDF.count()).as("Row count must match").isEqualTo(records.size()); @@ -635,7 +635,7 @@ public void testHiddenPartitionPaths() { .execute(); assertThat(result.orphanFileLocations()).as("Should delete 2 files").hasSize(2); - assertThat(result.deletedOrphanFilesCount()).as("Should delete 2 files").isEqualTo(2L); + assertThat(result.orphanFilesCount()).as("Should delete 2 files").isEqualTo(2L); } @TestTemplate @@ -676,7 +676,7 @@ public void testHiddenPartitionPathsWithPartitionEvolution() { .execute(); assertThat(result.orphanFileLocations()).as("Should delete 2 files").hasSize(2); - assertThat(result.deletedOrphanFilesCount()).as("Should delete 2 files").isEqualTo(2L); + assertThat(result.orphanFilesCount()).as("Should delete 2 files").isEqualTo(2L); } @TestTemplate @@ -716,7 +716,7 @@ public void testHiddenPathsStartingWithPartitionNamesAreIgnored() throws IOExcep .execute(); assertThat(result.orphanFileLocations()).as("Should delete 0 files").isEmpty(); - assertThat(result.deletedOrphanFilesCount()).as("Should delete 0 files").isEqualTo(0L); + assertThat(result.orphanFilesCount()).as("Should delete 0 files").isEqualTo(0L); assertThat(fs.exists(pathToFileInHiddenFolder)).isTrue(); } @@ -789,7 +789,7 @@ public void testRemoveOrphanFilesWithRelativeFilePath() throws IOException { assertThat(result.orphanFileLocations()) .as("Action should find 1 file") .isEqualTo(invalidFiles); - assertThat(result.deletedOrphanFilesCount()) + assertThat(result.orphanFilesCount()) .as("Action should find 1 file") .isEqualTo((long) invalidFiles.size()); assertThat(fs.exists(new Path(invalidFiles.get(0)))) @@ -829,7 +829,7 @@ public void testRemoveOrphanFilesWithHadoopCatalog() throws InterruptedException .execute(); assertThat(result.orphanFileLocations()).as("Should delete only 1 file").hasSize(1); - assertThat(result.deletedOrphanFilesCount()).as("Should delete only 1 file").isEqualTo(1L); + assertThat(result.orphanFilesCount()).as("Should delete only 1 file").isEqualTo(1L); Dataset resultDF = spark.read().format("iceberg").load(table.location()); List actualRecords = @@ -865,7 +865,7 @@ public void testHiveCatalogTable() throws IOException { assertThat(result.orphanFileLocations()) .as("trash file should be removed") .contains("file:" + location + "/data/trashfile"); - assertThat(result.deletedOrphanFilesCount()).as("trash file should be removed").isEqualTo(1L); + assertThat(result.orphanFilesCount()).as("trash file should be removed").isEqualTo(1L); } @TestTemplate @@ -960,7 +960,7 @@ public void testCompareToFileList() throws IOException { assertThat(result1.orphanFileLocations()) .as("Default olderThan interval should be safe") .isEmpty(); - assertThat(result1.deletedOrphanFilesCount()) + assertThat(result1.orphanFilesCount()) .as("Default olderThan interval should be safe") .isEqualTo(0L); @@ -974,7 +974,7 @@ public void testCompareToFileList() throws IOException { assertThat(result2.orphanFileLocations()) .as("Action should find 1 file") .isEqualTo(invalidFilePaths); - assertThat(result2.deletedOrphanFilesCount()) + assertThat(result2.orphanFilesCount()) .as("Action should find 1 file") .isEqualTo((long) invalidFilePaths.size()); assertThat(fs.exists(new Path(invalidFilePaths.get(0)))) @@ -990,7 +990,7 @@ public void testCompareToFileList() throws IOException { assertThat(result3.orphanFileLocations()) .as("Action should delete 1 file") .isEqualTo(invalidFilePaths); - assertThat(result3.deletedOrphanFilesCount()) + assertThat(result3.orphanFilesCount()) .as("Action should delete 1 file") .isEqualTo((long) invalidFilePaths.size()); assertThat(fs.exists(new Path(invalidFilePaths.get(0)))) @@ -1022,7 +1022,7 @@ public void testCompareToFileList() throws IOException { .deleteWith(s -> {}) .execute(); assertThat(result4.orphanFileLocations()).as("Action should find nothing").isEmpty(); - assertThat(result4.deletedOrphanFilesCount()).as("Action should find nothing").isEqualTo(0L); + assertThat(result4.orphanFilesCount()).as("Action should find nothing").isEqualTo(0L); } protected long waitUntilAfter(long timestampMillis) { @@ -1102,7 +1102,7 @@ public void testRemoveOrphanFilesWithStatisticFiles() throws Exception { .execute(); Iterable orphanFileLocations = result.orphanFileLocations(); assertThat(orphanFileLocations).hasSize(1).containsExactly(statsLocation.toURI().toString()); - assertThat(result.deletedOrphanFilesCount()).as("Should delete 1 file").isEqualTo(1L); + assertThat(result.orphanFilesCount()).as("Should delete 1 file").isEqualTo(1L); assertThat(statsLocation).as("stats file should be deleted").doesNotExist(); } @@ -1321,7 +1321,7 @@ public void testStreamResultsDeletion() throws IOException { .as("Non-streaming dry-run should return all 10 orphan files") .hasSize(10) .containsExactlyInAnyOrderElementsOf(invalidFiles); - assertThat(nonStreamingResult.deletedOrphanFilesCount()) + assertThat(nonStreamingResult.orphanFilesCount()) .as("Non-streaming dry-run should return all 10 orphan files") .isEqualTo((long) invalidFiles.size()); @@ -1337,7 +1337,7 @@ public void testStreamResultsDeletion() throws IOException { assertThat(streamingResult.orphanFileLocations()) .as("Streaming with sample size 5 should return only 5 orphan files") .hasSize(5); - assertThat(streamingResult.deletedOrphanFilesCount()) + assertThat(streamingResult.orphanFilesCount()) .as("Deleted 10 files") .isEqualTo((long) invalidFiles.size()); diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction3.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction3.java index 2659d519bba1..88ac800b158f 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction3.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction3.java @@ -60,7 +60,7 @@ public void testSparkCatalogTable() throws Exception { assertThat(results.orphanFileLocations()) .as("trash file should be removed") .contains("file:" + location + trashFile); - assertThat(results.deletedOrphanFilesCount()).as("trash file should be removed").isEqualTo(1L); + assertThat(results.orphanFilesCount()).as("trash file should be removed").isEqualTo(1L); } @TestTemplate @@ -90,7 +90,7 @@ public void testSparkCatalogNamedHadoopTable() throws Exception { assertThat(results.orphanFileLocations()) .as("trash file should be removed") .contains("file:" + location + trashFile); - assertThat(results.deletedOrphanFilesCount()).as("trash file should be removed").isEqualTo(1L); + assertThat(results.orphanFilesCount()).as("trash file should be removed").isEqualTo(1L); } @TestTemplate @@ -121,7 +121,7 @@ public void testSparkCatalogNamedHiveTable() throws Exception { assertThat(StreamSupport.stream(results.orphanFileLocations().spliterator(), false)) .as("trash file should be removed") .anyMatch(file -> file.contains("file:" + location + trashFile)); - assertThat(results.deletedOrphanFilesCount()).as("trash file should be removed").isEqualTo(1L); + assertThat(results.orphanFilesCount()).as("trash file should be removed").isEqualTo(1L); } @TestTemplate @@ -154,7 +154,7 @@ public void testSparkSessionCatalogHadoopTable() throws Exception { assertThat(results.orphanFileLocations()) .as("trash file should be removed") .contains("file:" + location + trashFile); - assertThat(results.deletedOrphanFilesCount()).as("trash file should be removed").isEqualTo(1L); + assertThat(results.orphanFilesCount()).as("trash file should be removed").isEqualTo(1L); } @TestTemplate @@ -187,7 +187,7 @@ public void testSparkSessionCatalogHiveTable() throws Exception { assertThat(results.orphanFileLocations()) .as("trash file should be removed") .contains("file:" + location + trashFile); - assertThat(results.deletedOrphanFilesCount()).as("trash file should be removed").isEqualTo(1L); + assertThat(results.orphanFilesCount()).as("trash file should be removed").isEqualTo(1L); } @AfterEach diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/DeleteOrphanFilesSparkAction.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/DeleteOrphanFilesSparkAction.java index 031214ea8468..92bfc880ad7f 100644 --- a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/DeleteOrphanFilesSparkAction.java +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/DeleteOrphanFilesSparkAction.java @@ -302,7 +302,7 @@ private DeleteOrphanFiles.Result deleteFiles(Dataset orphanFileDS) { return ImmutableDeleteOrphanFiles.Result.builder() .orphanFileLocations(orphanFileList) - .deletedOrphanFilesCount(filesCount) + .orphanFilesCount(filesCount) .build(); } diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java index b3d65363c4de..7429b4c8aad4 100644 --- a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java @@ -184,7 +184,7 @@ public void testDryRun() throws IOException { assertThat(result1.orphanFileLocations()) .as("Default olderThan interval should be safe") .isEmpty(); - assertThat(result1.deletedOrphanFilesCount()) + assertThat(result1.orphanFilesCount()) .as("Default olderThan interval should be safe") .isEqualTo(0L); @@ -198,7 +198,7 @@ public void testDryRun() throws IOException { assertThat(result2.orphanFileLocations()) .as("Action should find 1 file") .isEqualTo(invalidFiles); - assertThat(result2.deletedOrphanFilesCount()) + assertThat(result2.orphanFilesCount()) .as("Action should find 1 file") .isEqualTo(invalidFiles.size()); assertThat(fs.exists(new Path(invalidFiles.get(0)))) @@ -216,7 +216,7 @@ public void testDryRun() throws IOException { assertThat(result3.orphanFileLocations()) .as("Streaming dry run should find 1 file") .isEqualTo(invalidFiles); - assertThat(result3.deletedOrphanFilesCount()) + assertThat(result3.orphanFilesCount()) .as("Streaming dry run should find 1 file") .isEqualTo(invalidFiles.size()); assertThat(fs.exists(new Path(invalidFiles.get(0)))) @@ -232,7 +232,7 @@ public void testDryRun() throws IOException { assertThat(result4.orphanFileLocations()) .as("Action should delete 1 file") .isEqualTo(invalidFiles); - assertThat(result4.deletedOrphanFilesCount()) + assertThat(result4.orphanFilesCount()) .as("Action should delete 1 file") .isEqualTo(invalidFiles.size()); assertThat(fs.exists(new Path(invalidFiles.get(0)))) @@ -298,7 +298,7 @@ public void testAllValidFilesAreKept() throws IOException { .execute(); assertThat(result.orphanFileLocations()).as("Should delete 4 files").hasSize(4); - assertThat(result.deletedOrphanFilesCount()).as("Should delete 4 files").isEqualTo(4L); + assertThat(result.orphanFilesCount()).as("Should delete 4 files").isEqualTo(4L); Path dataPath = new Path(tableLocation + "/data"); FileSystem fs = dataPath.getFileSystem(spark.sessionState().newHadoopConf()); @@ -379,7 +379,7 @@ public void orphanedFileRemovedWithParallelTasks() { .containsExactlyInAnyOrder( "remove-orphan-0", "remove-orphan-1", "remove-orphan-2", "remove-orphan-3"); assertThat(deletedFiles).hasSize(4); - assertThat(result.deletedOrphanFilesCount()).as("Should delete 4 files").isEqualTo(4L); + assertThat(result.orphanFilesCount()).as("Should delete 4 files").isEqualTo(4L); } @TestTemplate @@ -424,7 +424,7 @@ public void testWapFilesAreKept() { actions.deleteOrphanFiles(table).olderThan(System.currentTimeMillis()).execute(); assertThat(result.orphanFileLocations()).as("Should not delete any files").isEmpty(); - assertThat(result.deletedOrphanFilesCount()).as("Should not delete any files").isEqualTo(0L); + assertThat(result.orphanFilesCount()).as("Should not delete any files").isEqualTo(0L); } @TestTemplate @@ -455,7 +455,7 @@ public void testMetadataFolderIsIntact() { .execute(); assertThat(result.orphanFileLocations()).as("Should delete 1 file").hasSize(1); - assertThat(result.deletedOrphanFilesCount()).as("Should delete 1 file").isEqualTo(1L); + assertThat(result.orphanFilesCount()).as("Should delete 1 file").isEqualTo(1L); Dataset resultDF = spark.read().format("iceberg").load(tableLocation); List actualRecords = @@ -494,7 +494,7 @@ public void testOlderThanTimestamp() { .execute(); assertThat(result.orphanFileLocations()).as("Should delete only 2 files").hasSize(2); - assertThat(result.deletedOrphanFilesCount()).as("Should delete only 2 files").isEqualTo(2L); + assertThat(result.orphanFilesCount()).as("Should delete only 2 files").isEqualTo(2L); } @TestTemplate @@ -526,7 +526,7 @@ public void testRemoveUnreachableMetadataVersionFiles() { assertThat(result.orphanFileLocations()) .containsExactly(tableLocation + "metadata/v1.metadata.json"); - assertThat(result.deletedOrphanFilesCount()).as("Should delete 1 file").isEqualTo(1L); + assertThat(result.orphanFilesCount()).as("Should delete 1 file").isEqualTo(1L); List expectedRecords = Lists.newArrayList(); expectedRecords.addAll(records); @@ -563,7 +563,7 @@ public void testManyTopLevelPartitions() { .execute(); assertThat(result.orphanFileLocations()).as("Should not delete any files").isEmpty(); - assertThat(result.deletedOrphanFilesCount()).as("Should not delete any files").isEqualTo(0L); + assertThat(result.orphanFilesCount()).as("Should not delete any files").isEqualTo(0L); Dataset resultDF = spark.read().format("iceberg").load(tableLocation); assertThat(resultDF.count()).as("Rows count must match").isEqualTo(records.size()); @@ -594,7 +594,7 @@ public void testManyLeafPartitions() { .execute(); assertThat(result.orphanFileLocations()).as("Should not delete any files").isEmpty(); - assertThat(result.deletedOrphanFilesCount()).as("Should not delete any files").isEqualTo(0L); + assertThat(result.orphanFilesCount()).as("Should not delete any files").isEqualTo(0L); Dataset resultDF = spark.read().format("iceberg").load(tableLocation); assertThat(resultDF.count()).as("Row count must match").isEqualTo(records.size()); @@ -635,7 +635,7 @@ public void testHiddenPartitionPaths() { .execute(); assertThat(result.orphanFileLocations()).as("Should delete 2 files").hasSize(2); - assertThat(result.deletedOrphanFilesCount()).as("Should delete 2 files").isEqualTo(2L); + assertThat(result.orphanFilesCount()).as("Should delete 2 files").isEqualTo(2L); } @TestTemplate @@ -676,7 +676,7 @@ public void testHiddenPartitionPathsWithPartitionEvolution() { .execute(); assertThat(result.orphanFileLocations()).as("Should delete 2 files").hasSize(2); - assertThat(result.deletedOrphanFilesCount()).as("Should delete 2 files").isEqualTo(2L); + assertThat(result.orphanFilesCount()).as("Should delete 2 files").isEqualTo(2L); } @TestTemplate @@ -716,7 +716,7 @@ public void testHiddenPathsStartingWithPartitionNamesAreIgnored() throws IOExcep .execute(); assertThat(result.orphanFileLocations()).as("Should delete 0 files").isEmpty(); - assertThat(result.deletedOrphanFilesCount()).as("Should delete 0 files").isEqualTo(0L); + assertThat(result.orphanFilesCount()).as("Should delete 0 files").isEqualTo(0L); assertThat(fs.exists(pathToFileInHiddenFolder)).isTrue(); } @@ -789,7 +789,7 @@ public void testRemoveOrphanFilesWithRelativeFilePath() throws IOException { assertThat(result.orphanFileLocations()) .as("Action should find 1 file") .isEqualTo(invalidFiles); - assertThat(result.deletedOrphanFilesCount()) + assertThat(result.orphanFilesCount()) .as("Action should find 1 file") .isEqualTo(invalidFiles.size()); assertThat(fs.exists(new Path(invalidFiles.get(0)))) @@ -829,7 +829,7 @@ public void testRemoveOrphanFilesWithHadoopCatalog() throws InterruptedException .execute(); assertThat(result.orphanFileLocations()).as("Should delete only 1 file").hasSize(1); - assertThat(result.deletedOrphanFilesCount()).as("Should delete only 1 file").isEqualTo(1L); + assertThat(result.orphanFilesCount()).as("Should delete only 1 file").isEqualTo(1L); Dataset resultDF = spark.read().format("iceberg").load(table.location()); List actualRecords = @@ -865,7 +865,7 @@ public void testHiveCatalogTable() throws IOException { assertThat(result.orphanFileLocations()) .as("trash file should be removed") .contains("file:" + location + "/data/trashfile"); - assertThat(result.deletedOrphanFilesCount()).as("trash file should be removed").isEqualTo(1L); + assertThat(result.orphanFilesCount()).as("trash file should be removed").isEqualTo(1L); } @TestTemplate @@ -960,7 +960,7 @@ public void testCompareToFileList() throws IOException { assertThat(result1.orphanFileLocations()) .as("Default olderThan interval should be safe") .isEmpty(); - assertThat(result1.deletedOrphanFilesCount()) + assertThat(result1.orphanFilesCount()) .as("Default olderThan interval should be safe") .isEqualTo(0L); @@ -974,7 +974,7 @@ public void testCompareToFileList() throws IOException { assertThat(result2.orphanFileLocations()) .as("Action should find 1 file") .isEqualTo(invalidFilePaths); - assertThat(result2.deletedOrphanFilesCount()) + assertThat(result2.orphanFilesCount()) .as("Action should find 1 file") .isEqualTo(invalidFilePaths.size()); assertThat(fs.exists(new Path(invalidFilePaths.get(0)))) @@ -990,7 +990,7 @@ public void testCompareToFileList() throws IOException { assertThat(result3.orphanFileLocations()) .as("Action should delete 1 file") .isEqualTo(invalidFilePaths); - assertThat(result3.deletedOrphanFilesCount()) + assertThat(result3.orphanFilesCount()) .as("Action should delete 1 file") .isEqualTo(invalidFilePaths.size()); assertThat(fs.exists(new Path(invalidFilePaths.get(0)))) @@ -1022,7 +1022,7 @@ public void testCompareToFileList() throws IOException { .deleteWith(s -> {}) .execute(); assertThat(result4.orphanFileLocations()).as("Action should find nothing").isEmpty(); - assertThat(result4.deletedOrphanFilesCount()).as("Action should find nothing").isEqualTo(0L); + assertThat(result4.orphanFilesCount()).as("Action should find nothing").isEqualTo(0L); } protected long waitUntilAfter(long timestampMillis) { @@ -1102,7 +1102,7 @@ public void testRemoveOrphanFilesWithStatisticFiles() throws Exception { .execute(); Iterable orphanFileLocations = result.orphanFileLocations(); assertThat(orphanFileLocations).hasSize(1).containsExactly(statsLocation.toURI().toString()); - assertThat(result.deletedOrphanFilesCount()).as("Should delete 1 file").isEqualTo(1L); + assertThat(result.orphanFilesCount()).as("Should delete 1 file").isEqualTo(1L); assertThat(statsLocation).as("stats file should be deleted").doesNotExist(); } @@ -1321,7 +1321,7 @@ public void testStreamResultsDeletion() throws IOException { .as("Non-streaming dry-run should return all 10 orphan files") .hasSize(10) .containsExactlyInAnyOrderElementsOf(invalidFiles); - assertThat(nonStreamingResult.deletedOrphanFilesCount()) + assertThat(nonStreamingResult.orphanFilesCount()) .as("Non-streaming dry-run should return all 10 orphan files") .isEqualTo(invalidFiles.size()); @@ -1337,7 +1337,7 @@ public void testStreamResultsDeletion() throws IOException { assertThat(streamingResult.orphanFileLocations()) .as("Streaming with sample size 5 should return only 5 orphan files") .hasSize(5); - assertThat(streamingResult.deletedOrphanFilesCount()) + assertThat(streamingResult.orphanFilesCount()) .as("Deleted 10 files") .isEqualTo(invalidFiles.size()); diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction3.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction3.java index 2659d519bba1..88ac800b158f 100644 --- a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction3.java +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction3.java @@ -60,7 +60,7 @@ public void testSparkCatalogTable() throws Exception { assertThat(results.orphanFileLocations()) .as("trash file should be removed") .contains("file:" + location + trashFile); - assertThat(results.deletedOrphanFilesCount()).as("trash file should be removed").isEqualTo(1L); + assertThat(results.orphanFilesCount()).as("trash file should be removed").isEqualTo(1L); } @TestTemplate @@ -90,7 +90,7 @@ public void testSparkCatalogNamedHadoopTable() throws Exception { assertThat(results.orphanFileLocations()) .as("trash file should be removed") .contains("file:" + location + trashFile); - assertThat(results.deletedOrphanFilesCount()).as("trash file should be removed").isEqualTo(1L); + assertThat(results.orphanFilesCount()).as("trash file should be removed").isEqualTo(1L); } @TestTemplate @@ -121,7 +121,7 @@ public void testSparkCatalogNamedHiveTable() throws Exception { assertThat(StreamSupport.stream(results.orphanFileLocations().spliterator(), false)) .as("trash file should be removed") .anyMatch(file -> file.contains("file:" + location + trashFile)); - assertThat(results.deletedOrphanFilesCount()).as("trash file should be removed").isEqualTo(1L); + assertThat(results.orphanFilesCount()).as("trash file should be removed").isEqualTo(1L); } @TestTemplate @@ -154,7 +154,7 @@ public void testSparkSessionCatalogHadoopTable() throws Exception { assertThat(results.orphanFileLocations()) .as("trash file should be removed") .contains("file:" + location + trashFile); - assertThat(results.deletedOrphanFilesCount()).as("trash file should be removed").isEqualTo(1L); + assertThat(results.orphanFilesCount()).as("trash file should be removed").isEqualTo(1L); } @TestTemplate @@ -187,7 +187,7 @@ public void testSparkSessionCatalogHiveTable() throws Exception { assertThat(results.orphanFileLocations()) .as("trash file should be removed") .contains("file:" + location + trashFile); - assertThat(results.deletedOrphanFilesCount()).as("trash file should be removed").isEqualTo(1L); + assertThat(results.orphanFilesCount()).as("trash file should be removed").isEqualTo(1L); } @AfterEach From 09c99386eb7507ff7a93c7ded29347de64bc57ac Mon Sep 17 00:00:00 2001 From: Alessandro Nori Date: Tue, 6 Jan 2026 18:08:28 +0100 Subject: [PATCH 4/8] update default orphanFilesCount to -1 --- .../main/java/org/apache/iceberg/actions/DeleteOrphanFiles.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/api/src/main/java/org/apache/iceberg/actions/DeleteOrphanFiles.java b/api/src/main/java/org/apache/iceberg/actions/DeleteOrphanFiles.java index ab12a3b7c1e3..a14fe0b41a6f 100644 --- a/api/src/main/java/org/apache/iceberg/actions/DeleteOrphanFiles.java +++ b/api/src/main/java/org/apache/iceberg/actions/DeleteOrphanFiles.java @@ -145,7 +145,7 @@ interface Result { /** Returns the total number of orphan files. */ default long orphanFilesCount() { - return 0; + return -1; } } From 4fc403c37bd62697941020c547b15a71be56b2d5 Mon Sep 17 00:00:00 2001 From: Alessandro Nori Date: Tue, 6 Jan 2026 19:51:07 +0100 Subject: [PATCH 5/8] port to spark 4.1 --- .../actions/DeleteOrphanFilesSparkAction.java | 5 ++- .../actions/TestRemoveOrphanFilesAction.java | 45 +++++++++++++++++++ .../actions/TestRemoveOrphanFilesAction3.java | 5 +++ 3 files changed, 54 insertions(+), 1 deletion(-) diff --git a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/actions/DeleteOrphanFilesSparkAction.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/actions/DeleteOrphanFilesSparkAction.java index 78662159b0bb..92bfc880ad7f 100644 --- a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/actions/DeleteOrphanFilesSparkAction.java +++ b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/actions/DeleteOrphanFilesSparkAction.java @@ -300,7 +300,10 @@ private DeleteOrphanFiles.Result deleteFiles(Dataset orphanFileDS) { LOG.info("Deleted {} orphan files", filesCount); - return ImmutableDeleteOrphanFiles.Result.builder().orphanFileLocations(orphanFileList).build(); + return ImmutableDeleteOrphanFiles.Result.builder() + .orphanFileLocations(orphanFileList) + .orphanFilesCount(filesCount) + .build(); } private void collectPathsForOutput( diff --git a/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java index 40505b856737..64414e4513ab 100644 --- a/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java +++ b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java @@ -184,6 +184,9 @@ public void testDryRun() throws IOException { assertThat(result1.orphanFileLocations()) .as("Default olderThan interval should be safe") .isEmpty(); + assertThat(result1.orphanFilesCount()) + .as("Default olderThan interval should be safe") + .isEqualTo(0L); DeleteOrphanFiles.Result result2 = actions @@ -195,6 +198,9 @@ public void testDryRun() throws IOException { assertThat(result2.orphanFileLocations()) .as("Action should find 1 file") .isEqualTo(invalidFiles); + assertThat(result2.orphanFilesCount()) + .as("Action should find 1 file") + .isEqualTo((long) invalidFiles.size()); assertThat(fs.exists(new Path(invalidFiles.get(0)))) .as("Invalid file should be present") .isTrue(); @@ -210,6 +216,9 @@ public void testDryRun() throws IOException { assertThat(result3.orphanFileLocations()) .as("Streaming dry run should find 1 file") .isEqualTo(invalidFiles); + assertThat(result3.orphanFilesCount()) + .as("Streaming dry run should find 1 file") + .isEqualTo((long) invalidFiles.size()); assertThat(fs.exists(new Path(invalidFiles.get(0)))) .as("Invalid file should be present after streaming dry run") .isTrue(); @@ -223,6 +232,9 @@ public void testDryRun() throws IOException { assertThat(result4.orphanFileLocations()) .as("Action should delete 1 file") .isEqualTo(invalidFiles); + assertThat(result4.orphanFilesCount()) + .as("Action should delete 1 file") + .isEqualTo((long) invalidFiles.size()); assertThat(fs.exists(new Path(invalidFiles.get(0)))) .as("Invalid file should not be present") .isFalse(); @@ -286,6 +298,7 @@ public void testAllValidFilesAreKept() throws IOException { .execute(); assertThat(result.orphanFileLocations()).as("Should delete 4 files").hasSize(4); + assertThat(result.orphanFilesCount()).as("Should delete 4 files").isEqualTo(4L); Path dataPath = new Path(tableLocation + "/data"); FileSystem fs = dataPath.getFileSystem(spark.sessionState().newHadoopConf()); @@ -366,6 +379,7 @@ public void orphanedFileRemovedWithParallelTasks() { .containsExactlyInAnyOrder( "remove-orphan-0", "remove-orphan-1", "remove-orphan-2", "remove-orphan-3"); assertThat(deletedFiles).hasSize(4); + assertThat(result.orphanFilesCount()).as("Should delete 4 files").isEqualTo(4L); } @TestTemplate @@ -410,6 +424,7 @@ public void testWapFilesAreKept() { actions.deleteOrphanFiles(table).olderThan(System.currentTimeMillis()).execute(); assertThat(result.orphanFileLocations()).as("Should not delete any files").isEmpty(); + assertThat(result.orphanFilesCount()).as("Should not delete any files").isEqualTo(0L); } @TestTemplate @@ -440,6 +455,7 @@ public void testMetadataFolderIsIntact() { .execute(); assertThat(result.orphanFileLocations()).as("Should delete 1 file").hasSize(1); + assertThat(result.orphanFilesCount()).as("Should delete 1 file").isEqualTo(1L); Dataset resultDF = spark.read().format("iceberg").load(tableLocation); List actualRecords = @@ -478,6 +494,7 @@ public void testOlderThanTimestamp() { .execute(); assertThat(result.orphanFileLocations()).as("Should delete only 2 files").hasSize(2); + assertThat(result.orphanFilesCount()).as("Should delete only 2 files").isEqualTo(2L); } @TestTemplate @@ -509,6 +526,7 @@ public void testRemoveUnreachableMetadataVersionFiles() { assertThat(result.orphanFileLocations()) .containsExactly(tableLocation + "metadata/v1.metadata.json"); + assertThat(result.orphanFilesCount()).as("Should delete 1 file").isEqualTo(1L); List expectedRecords = Lists.newArrayList(); expectedRecords.addAll(records); @@ -545,6 +563,7 @@ public void testManyTopLevelPartitions() { .execute(); assertThat(result.orphanFileLocations()).as("Should not delete any files").isEmpty(); + assertThat(result.orphanFilesCount()).as("Should not delete any files").isEqualTo(0L); Dataset resultDF = spark.read().format("iceberg").load(tableLocation); assertThat(resultDF.count()).as("Rows count must match").isEqualTo(records.size()); @@ -575,6 +594,7 @@ public void testManyLeafPartitions() { .execute(); assertThat(result.orphanFileLocations()).as("Should not delete any files").isEmpty(); + assertThat(result.orphanFilesCount()).as("Should not delete any files").isEqualTo(0L); Dataset resultDF = spark.read().format("iceberg").load(tableLocation); assertThat(resultDF.count()).as("Row count must match").isEqualTo(records.size()); @@ -615,6 +635,7 @@ public void testHiddenPartitionPaths() { .execute(); assertThat(result.orphanFileLocations()).as("Should delete 2 files").hasSize(2); + assertThat(result.orphanFilesCount()).as("Should delete 2 files").isEqualTo(2L); } @TestTemplate @@ -655,6 +676,7 @@ public void testHiddenPartitionPathsWithPartitionEvolution() { .execute(); assertThat(result.orphanFileLocations()).as("Should delete 2 files").hasSize(2); + assertThat(result.orphanFilesCount()).as("Should delete 2 files").isEqualTo(2L); } @TestTemplate @@ -694,6 +716,7 @@ public void testHiddenPathsStartingWithPartitionNamesAreIgnored() throws IOExcep .execute(); assertThat(result.orphanFileLocations()).as("Should delete 0 files").isEmpty(); + assertThat(result.orphanFilesCount()).as("Should delete 0 files").isEqualTo(0L); assertThat(fs.exists(pathToFileInHiddenFolder)).isTrue(); } @@ -766,6 +789,9 @@ public void testRemoveOrphanFilesWithRelativeFilePath() throws IOException { assertThat(result.orphanFileLocations()) .as("Action should find 1 file") .isEqualTo(invalidFiles); + assertThat(result.orphanFilesCount()) + .as("Action should find 1 file") + .isEqualTo((long) invalidFiles.size()); assertThat(fs.exists(new Path(invalidFiles.get(0)))) .as("Invalid file should be present") .isTrue(); @@ -803,6 +829,7 @@ public void testRemoveOrphanFilesWithHadoopCatalog() throws InterruptedException .execute(); assertThat(result.orphanFileLocations()).as("Should delete only 1 file").hasSize(1); + assertThat(result.orphanFilesCount()).as("Should delete only 1 file").isEqualTo(1L); Dataset resultDF = spark.read().format("iceberg").load(table.location()); List actualRecords = @@ -838,6 +865,7 @@ public void testHiveCatalogTable() throws IOException { assertThat(result.orphanFileLocations()) .as("trash file should be removed") .contains("file:" + location + "/data/trashfile"); + assertThat(result.orphanFilesCount()).as("trash file should be removed").isEqualTo(1L); } @TestTemplate @@ -932,6 +960,9 @@ public void testCompareToFileList() throws IOException { assertThat(result1.orphanFileLocations()) .as("Default olderThan interval should be safe") .isEmpty(); + assertThat(result1.orphanFilesCount()) + .as("Default olderThan interval should be safe") + .isEqualTo(0L); DeleteOrphanFiles.Result result2 = actions @@ -943,6 +974,9 @@ public void testCompareToFileList() throws IOException { assertThat(result2.orphanFileLocations()) .as("Action should find 1 file") .isEqualTo(invalidFilePaths); + assertThat(result2.orphanFilesCount()) + .as("Action should find 1 file") + .isEqualTo((long) invalidFilePaths.size()); assertThat(fs.exists(new Path(invalidFilePaths.get(0)))) .as("Invalid file should be present") .isTrue(); @@ -956,6 +990,9 @@ public void testCompareToFileList() throws IOException { assertThat(result3.orphanFileLocations()) .as("Action should delete 1 file") .isEqualTo(invalidFilePaths); + assertThat(result3.orphanFilesCount()) + .as("Action should delete 1 file") + .isEqualTo((long) invalidFilePaths.size()); assertThat(fs.exists(new Path(invalidFilePaths.get(0)))) .as("Invalid file should not be present") .isFalse(); @@ -985,6 +1022,7 @@ public void testCompareToFileList() throws IOException { .deleteWith(s -> {}) .execute(); assertThat(result4.orphanFileLocations()).as("Action should find nothing").isEmpty(); + assertThat(result4.orphanFilesCount()).as("Action should find nothing").isEqualTo(0L); } protected long waitUntilAfter(long timestampMillis) { @@ -1064,6 +1102,7 @@ public void testRemoveOrphanFilesWithStatisticFiles() throws Exception { .execute(); Iterable orphanFileLocations = result.orphanFileLocations(); assertThat(orphanFileLocations).hasSize(1).containsExactly(statsLocation.toURI().toString()); + assertThat(result.orphanFilesCount()).as("Should delete 1 file").isEqualTo(1L); assertThat(statsLocation).as("stats file should be deleted").doesNotExist(); } @@ -1282,6 +1321,9 @@ public void testStreamResultsDeletion() throws IOException { .as("Non-streaming dry-run should return all 10 orphan files") .hasSize(10) .containsExactlyInAnyOrderElementsOf(invalidFiles); + assertThat(nonStreamingResult.orphanFilesCount()) + .as("Non-streaming dry-run should return all 10 orphan files") + .isEqualTo((long) invalidFiles.size()); DeleteOrphanFiles.Result streamingResult = SparkActions.get() @@ -1295,6 +1337,9 @@ public void testStreamResultsDeletion() throws IOException { assertThat(streamingResult.orphanFileLocations()) .as("Streaming with sample size 5 should return only 5 orphan files") .hasSize(5); + assertThat(streamingResult.orphanFilesCount()) + .as("Deleted 10 files") + .isEqualTo((long) invalidFiles.size()); for (String invalidFile : invalidFiles) { assertThat(fs.exists(new Path(invalidFile))).as("Orphan file should be deleted").isFalse(); diff --git a/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction3.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction3.java index 5f98287951f1..88ac800b158f 100644 --- a/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction3.java +++ b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction3.java @@ -60,6 +60,7 @@ public void testSparkCatalogTable() throws Exception { assertThat(results.orphanFileLocations()) .as("trash file should be removed") .contains("file:" + location + trashFile); + assertThat(results.orphanFilesCount()).as("trash file should be removed").isEqualTo(1L); } @TestTemplate @@ -89,6 +90,7 @@ public void testSparkCatalogNamedHadoopTable() throws Exception { assertThat(results.orphanFileLocations()) .as("trash file should be removed") .contains("file:" + location + trashFile); + assertThat(results.orphanFilesCount()).as("trash file should be removed").isEqualTo(1L); } @TestTemplate @@ -119,6 +121,7 @@ public void testSparkCatalogNamedHiveTable() throws Exception { assertThat(StreamSupport.stream(results.orphanFileLocations().spliterator(), false)) .as("trash file should be removed") .anyMatch(file -> file.contains("file:" + location + trashFile)); + assertThat(results.orphanFilesCount()).as("trash file should be removed").isEqualTo(1L); } @TestTemplate @@ -151,6 +154,7 @@ public void testSparkSessionCatalogHadoopTable() throws Exception { assertThat(results.orphanFileLocations()) .as("trash file should be removed") .contains("file:" + location + trashFile); + assertThat(results.orphanFilesCount()).as("trash file should be removed").isEqualTo(1L); } @TestTemplate @@ -183,6 +187,7 @@ public void testSparkSessionCatalogHiveTable() throws Exception { assertThat(results.orphanFileLocations()) .as("trash file should be removed") .contains("file:" + location + trashFile); + assertThat(results.orphanFilesCount()).as("trash file should be removed").isEqualTo(1L); } @AfterEach From ee6bf77ca55d8c0b489a423756f96338d42a6b7e Mon Sep 17 00:00:00 2001 From: Alessandro Nori Date: Thu, 8 Jan 2026 11:42:56 +0100 Subject: [PATCH 6/8] restore default to 0 --- .../main/java/org/apache/iceberg/actions/DeleteOrphanFiles.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/api/src/main/java/org/apache/iceberg/actions/DeleteOrphanFiles.java b/api/src/main/java/org/apache/iceberg/actions/DeleteOrphanFiles.java index a14fe0b41a6f..ab12a3b7c1e3 100644 --- a/api/src/main/java/org/apache/iceberg/actions/DeleteOrphanFiles.java +++ b/api/src/main/java/org/apache/iceberg/actions/DeleteOrphanFiles.java @@ -145,7 +145,7 @@ interface Result { /** Returns the total number of orphan files. */ default long orphanFilesCount() { - return -1; + return 0; } } From 378de6fd1909468f6cbd94e4be2817d5720e845b Mon Sep 17 00:00:00 2001 From: Alessandro Nori Date: Thu, 22 Jan 2026 21:45:40 +0100 Subject: [PATCH 7/8] keep changes only for latest Spark version --- .../actions/BaseDeleteOrphanFiles.java | 5 ++- .../actions/DeleteOrphanFilesSparkAction.java | 5 +-- .../actions/TestRemoveOrphanFilesAction.java | 45 ------------------- .../actions/TestRemoveOrphanFilesAction3.java | 11 +---- .../actions/DeleteOrphanFilesSparkAction.java | 5 +-- .../actions/TestRemoveOrphanFilesAction.java | 45 ------------------- .../actions/TestRemoveOrphanFilesAction3.java | 5 --- .../actions/DeleteOrphanFilesSparkAction.java | 5 +-- .../actions/TestRemoveOrphanFilesAction.java | 45 ------------------- .../actions/TestRemoveOrphanFilesAction3.java | 5 --- 10 files changed, 9 insertions(+), 167 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/actions/BaseDeleteOrphanFiles.java b/core/src/main/java/org/apache/iceberg/actions/BaseDeleteOrphanFiles.java index cfa7a80b8870..fc87dad49de0 100644 --- a/core/src/main/java/org/apache/iceberg/actions/BaseDeleteOrphanFiles.java +++ b/core/src/main/java/org/apache/iceberg/actions/BaseDeleteOrphanFiles.java @@ -31,6 +31,9 @@ interface BaseDeleteOrphanFiles extends DeleteOrphanFiles { @Value.Immutable interface Result extends DeleteOrphanFiles.Result { @Override - long orphanFilesCount(); + @Value.Default + default long orphanFilesCount() { + return 0; + } } } diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/DeleteOrphanFilesSparkAction.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/DeleteOrphanFilesSparkAction.java index 92bfc880ad7f..78662159b0bb 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/DeleteOrphanFilesSparkAction.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/DeleteOrphanFilesSparkAction.java @@ -300,10 +300,7 @@ private DeleteOrphanFiles.Result deleteFiles(Dataset orphanFileDS) { LOG.info("Deleted {} orphan files", filesCount); - return ImmutableDeleteOrphanFiles.Result.builder() - .orphanFileLocations(orphanFileList) - .orphanFilesCount(filesCount) - .build(); + return ImmutableDeleteOrphanFiles.Result.builder().orphanFileLocations(orphanFileList).build(); } private void collectPathsForOutput( diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java index 64414e4513ab..40505b856737 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java @@ -184,9 +184,6 @@ public void testDryRun() throws IOException { assertThat(result1.orphanFileLocations()) .as("Default olderThan interval should be safe") .isEmpty(); - assertThat(result1.orphanFilesCount()) - .as("Default olderThan interval should be safe") - .isEqualTo(0L); DeleteOrphanFiles.Result result2 = actions @@ -198,9 +195,6 @@ public void testDryRun() throws IOException { assertThat(result2.orphanFileLocations()) .as("Action should find 1 file") .isEqualTo(invalidFiles); - assertThat(result2.orphanFilesCount()) - .as("Action should find 1 file") - .isEqualTo((long) invalidFiles.size()); assertThat(fs.exists(new Path(invalidFiles.get(0)))) .as("Invalid file should be present") .isTrue(); @@ -216,9 +210,6 @@ public void testDryRun() throws IOException { assertThat(result3.orphanFileLocations()) .as("Streaming dry run should find 1 file") .isEqualTo(invalidFiles); - assertThat(result3.orphanFilesCount()) - .as("Streaming dry run should find 1 file") - .isEqualTo((long) invalidFiles.size()); assertThat(fs.exists(new Path(invalidFiles.get(0)))) .as("Invalid file should be present after streaming dry run") .isTrue(); @@ -232,9 +223,6 @@ public void testDryRun() throws IOException { assertThat(result4.orphanFileLocations()) .as("Action should delete 1 file") .isEqualTo(invalidFiles); - assertThat(result4.orphanFilesCount()) - .as("Action should delete 1 file") - .isEqualTo((long) invalidFiles.size()); assertThat(fs.exists(new Path(invalidFiles.get(0)))) .as("Invalid file should not be present") .isFalse(); @@ -298,7 +286,6 @@ public void testAllValidFilesAreKept() throws IOException { .execute(); assertThat(result.orphanFileLocations()).as("Should delete 4 files").hasSize(4); - assertThat(result.orphanFilesCount()).as("Should delete 4 files").isEqualTo(4L); Path dataPath = new Path(tableLocation + "/data"); FileSystem fs = dataPath.getFileSystem(spark.sessionState().newHadoopConf()); @@ -379,7 +366,6 @@ public void orphanedFileRemovedWithParallelTasks() { .containsExactlyInAnyOrder( "remove-orphan-0", "remove-orphan-1", "remove-orphan-2", "remove-orphan-3"); assertThat(deletedFiles).hasSize(4); - assertThat(result.orphanFilesCount()).as("Should delete 4 files").isEqualTo(4L); } @TestTemplate @@ -424,7 +410,6 @@ public void testWapFilesAreKept() { actions.deleteOrphanFiles(table).olderThan(System.currentTimeMillis()).execute(); assertThat(result.orphanFileLocations()).as("Should not delete any files").isEmpty(); - assertThat(result.orphanFilesCount()).as("Should not delete any files").isEqualTo(0L); } @TestTemplate @@ -455,7 +440,6 @@ public void testMetadataFolderIsIntact() { .execute(); assertThat(result.orphanFileLocations()).as("Should delete 1 file").hasSize(1); - assertThat(result.orphanFilesCount()).as("Should delete 1 file").isEqualTo(1L); Dataset resultDF = spark.read().format("iceberg").load(tableLocation); List actualRecords = @@ -494,7 +478,6 @@ public void testOlderThanTimestamp() { .execute(); assertThat(result.orphanFileLocations()).as("Should delete only 2 files").hasSize(2); - assertThat(result.orphanFilesCount()).as("Should delete only 2 files").isEqualTo(2L); } @TestTemplate @@ -526,7 +509,6 @@ public void testRemoveUnreachableMetadataVersionFiles() { assertThat(result.orphanFileLocations()) .containsExactly(tableLocation + "metadata/v1.metadata.json"); - assertThat(result.orphanFilesCount()).as("Should delete 1 file").isEqualTo(1L); List expectedRecords = Lists.newArrayList(); expectedRecords.addAll(records); @@ -563,7 +545,6 @@ public void testManyTopLevelPartitions() { .execute(); assertThat(result.orphanFileLocations()).as("Should not delete any files").isEmpty(); - assertThat(result.orphanFilesCount()).as("Should not delete any files").isEqualTo(0L); Dataset resultDF = spark.read().format("iceberg").load(tableLocation); assertThat(resultDF.count()).as("Rows count must match").isEqualTo(records.size()); @@ -594,7 +575,6 @@ public void testManyLeafPartitions() { .execute(); assertThat(result.orphanFileLocations()).as("Should not delete any files").isEmpty(); - assertThat(result.orphanFilesCount()).as("Should not delete any files").isEqualTo(0L); Dataset resultDF = spark.read().format("iceberg").load(tableLocation); assertThat(resultDF.count()).as("Row count must match").isEqualTo(records.size()); @@ -635,7 +615,6 @@ public void testHiddenPartitionPaths() { .execute(); assertThat(result.orphanFileLocations()).as("Should delete 2 files").hasSize(2); - assertThat(result.orphanFilesCount()).as("Should delete 2 files").isEqualTo(2L); } @TestTemplate @@ -676,7 +655,6 @@ public void testHiddenPartitionPathsWithPartitionEvolution() { .execute(); assertThat(result.orphanFileLocations()).as("Should delete 2 files").hasSize(2); - assertThat(result.orphanFilesCount()).as("Should delete 2 files").isEqualTo(2L); } @TestTemplate @@ -716,7 +694,6 @@ public void testHiddenPathsStartingWithPartitionNamesAreIgnored() throws IOExcep .execute(); assertThat(result.orphanFileLocations()).as("Should delete 0 files").isEmpty(); - assertThat(result.orphanFilesCount()).as("Should delete 0 files").isEqualTo(0L); assertThat(fs.exists(pathToFileInHiddenFolder)).isTrue(); } @@ -789,9 +766,6 @@ public void testRemoveOrphanFilesWithRelativeFilePath() throws IOException { assertThat(result.orphanFileLocations()) .as("Action should find 1 file") .isEqualTo(invalidFiles); - assertThat(result.orphanFilesCount()) - .as("Action should find 1 file") - .isEqualTo((long) invalidFiles.size()); assertThat(fs.exists(new Path(invalidFiles.get(0)))) .as("Invalid file should be present") .isTrue(); @@ -829,7 +803,6 @@ public void testRemoveOrphanFilesWithHadoopCatalog() throws InterruptedException .execute(); assertThat(result.orphanFileLocations()).as("Should delete only 1 file").hasSize(1); - assertThat(result.orphanFilesCount()).as("Should delete only 1 file").isEqualTo(1L); Dataset resultDF = spark.read().format("iceberg").load(table.location()); List actualRecords = @@ -865,7 +838,6 @@ public void testHiveCatalogTable() throws IOException { assertThat(result.orphanFileLocations()) .as("trash file should be removed") .contains("file:" + location + "/data/trashfile"); - assertThat(result.orphanFilesCount()).as("trash file should be removed").isEqualTo(1L); } @TestTemplate @@ -960,9 +932,6 @@ public void testCompareToFileList() throws IOException { assertThat(result1.orphanFileLocations()) .as("Default olderThan interval should be safe") .isEmpty(); - assertThat(result1.orphanFilesCount()) - .as("Default olderThan interval should be safe") - .isEqualTo(0L); DeleteOrphanFiles.Result result2 = actions @@ -974,9 +943,6 @@ public void testCompareToFileList() throws IOException { assertThat(result2.orphanFileLocations()) .as("Action should find 1 file") .isEqualTo(invalidFilePaths); - assertThat(result2.orphanFilesCount()) - .as("Action should find 1 file") - .isEqualTo((long) invalidFilePaths.size()); assertThat(fs.exists(new Path(invalidFilePaths.get(0)))) .as("Invalid file should be present") .isTrue(); @@ -990,9 +956,6 @@ public void testCompareToFileList() throws IOException { assertThat(result3.orphanFileLocations()) .as("Action should delete 1 file") .isEqualTo(invalidFilePaths); - assertThat(result3.orphanFilesCount()) - .as("Action should delete 1 file") - .isEqualTo((long) invalidFilePaths.size()); assertThat(fs.exists(new Path(invalidFilePaths.get(0)))) .as("Invalid file should not be present") .isFalse(); @@ -1022,7 +985,6 @@ public void testCompareToFileList() throws IOException { .deleteWith(s -> {}) .execute(); assertThat(result4.orphanFileLocations()).as("Action should find nothing").isEmpty(); - assertThat(result4.orphanFilesCount()).as("Action should find nothing").isEqualTo(0L); } protected long waitUntilAfter(long timestampMillis) { @@ -1102,7 +1064,6 @@ public void testRemoveOrphanFilesWithStatisticFiles() throws Exception { .execute(); Iterable orphanFileLocations = result.orphanFileLocations(); assertThat(orphanFileLocations).hasSize(1).containsExactly(statsLocation.toURI().toString()); - assertThat(result.orphanFilesCount()).as("Should delete 1 file").isEqualTo(1L); assertThat(statsLocation).as("stats file should be deleted").doesNotExist(); } @@ -1321,9 +1282,6 @@ public void testStreamResultsDeletion() throws IOException { .as("Non-streaming dry-run should return all 10 orphan files") .hasSize(10) .containsExactlyInAnyOrderElementsOf(invalidFiles); - assertThat(nonStreamingResult.orphanFilesCount()) - .as("Non-streaming dry-run should return all 10 orphan files") - .isEqualTo((long) invalidFiles.size()); DeleteOrphanFiles.Result streamingResult = SparkActions.get() @@ -1337,9 +1295,6 @@ public void testStreamResultsDeletion() throws IOException { assertThat(streamingResult.orphanFileLocations()) .as("Streaming with sample size 5 should return only 5 orphan files") .hasSize(5); - assertThat(streamingResult.orphanFilesCount()) - .as("Deleted 10 files") - .isEqualTo((long) invalidFiles.size()); for (String invalidFile : invalidFiles) { assertThat(fs.exists(new Path(invalidFile))).as("Orphan file should be deleted").isFalse(); diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction3.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction3.java index 88ac800b158f..646e5f8e70d4 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction3.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction3.java @@ -21,7 +21,6 @@ import static org.assertj.core.api.Assertions.assertThat; import java.io.File; -import java.util.stream.StreamSupport; import org.apache.iceberg.actions.DeleteOrphanFiles; import org.apache.iceberg.spark.SparkCatalog; import org.apache.iceberg.spark.SparkSchemaUtil; @@ -60,7 +59,6 @@ public void testSparkCatalogTable() throws Exception { assertThat(results.orphanFileLocations()) .as("trash file should be removed") .contains("file:" + location + trashFile); - assertThat(results.orphanFilesCount()).as("trash file should be removed").isEqualTo(1L); } @TestTemplate @@ -90,7 +88,6 @@ public void testSparkCatalogNamedHadoopTable() throws Exception { assertThat(results.orphanFileLocations()) .as("trash file should be removed") .contains("file:" + location + trashFile); - assertThat(results.orphanFilesCount()).as("trash file should be removed").isEqualTo(1L); } @TestTemplate @@ -117,11 +114,9 @@ public void testSparkCatalogNamedHiveTable() throws Exception { .deleteOrphanFiles(table.table()) .olderThan(System.currentTimeMillis() + 1000) .execute(); - - assertThat(StreamSupport.stream(results.orphanFileLocations().spliterator(), false)) + assertThat(results.orphanFileLocations()) .as("trash file should be removed") - .anyMatch(file -> file.contains("file:" + location + trashFile)); - assertThat(results.orphanFilesCount()).as("trash file should be removed").isEqualTo(1L); + .contains("file:" + location + trashFile); } @TestTemplate @@ -154,7 +149,6 @@ public void testSparkSessionCatalogHadoopTable() throws Exception { assertThat(results.orphanFileLocations()) .as("trash file should be removed") .contains("file:" + location + trashFile); - assertThat(results.orphanFilesCount()).as("trash file should be removed").isEqualTo(1L); } @TestTemplate @@ -187,7 +181,6 @@ public void testSparkSessionCatalogHiveTable() throws Exception { assertThat(results.orphanFileLocations()) .as("trash file should be removed") .contains("file:" + location + trashFile); - assertThat(results.orphanFilesCount()).as("trash file should be removed").isEqualTo(1L); } @AfterEach diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/DeleteOrphanFilesSparkAction.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/DeleteOrphanFilesSparkAction.java index 92bfc880ad7f..78662159b0bb 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/DeleteOrphanFilesSparkAction.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/DeleteOrphanFilesSparkAction.java @@ -300,10 +300,7 @@ private DeleteOrphanFiles.Result deleteFiles(Dataset orphanFileDS) { LOG.info("Deleted {} orphan files", filesCount); - return ImmutableDeleteOrphanFiles.Result.builder() - .orphanFileLocations(orphanFileList) - .orphanFilesCount(filesCount) - .build(); + return ImmutableDeleteOrphanFiles.Result.builder().orphanFileLocations(orphanFileList).build(); } private void collectPathsForOutput( diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java index 64414e4513ab..40505b856737 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java @@ -184,9 +184,6 @@ public void testDryRun() throws IOException { assertThat(result1.orphanFileLocations()) .as("Default olderThan interval should be safe") .isEmpty(); - assertThat(result1.orphanFilesCount()) - .as("Default olderThan interval should be safe") - .isEqualTo(0L); DeleteOrphanFiles.Result result2 = actions @@ -198,9 +195,6 @@ public void testDryRun() throws IOException { assertThat(result2.orphanFileLocations()) .as("Action should find 1 file") .isEqualTo(invalidFiles); - assertThat(result2.orphanFilesCount()) - .as("Action should find 1 file") - .isEqualTo((long) invalidFiles.size()); assertThat(fs.exists(new Path(invalidFiles.get(0)))) .as("Invalid file should be present") .isTrue(); @@ -216,9 +210,6 @@ public void testDryRun() throws IOException { assertThat(result3.orphanFileLocations()) .as("Streaming dry run should find 1 file") .isEqualTo(invalidFiles); - assertThat(result3.orphanFilesCount()) - .as("Streaming dry run should find 1 file") - .isEqualTo((long) invalidFiles.size()); assertThat(fs.exists(new Path(invalidFiles.get(0)))) .as("Invalid file should be present after streaming dry run") .isTrue(); @@ -232,9 +223,6 @@ public void testDryRun() throws IOException { assertThat(result4.orphanFileLocations()) .as("Action should delete 1 file") .isEqualTo(invalidFiles); - assertThat(result4.orphanFilesCount()) - .as("Action should delete 1 file") - .isEqualTo((long) invalidFiles.size()); assertThat(fs.exists(new Path(invalidFiles.get(0)))) .as("Invalid file should not be present") .isFalse(); @@ -298,7 +286,6 @@ public void testAllValidFilesAreKept() throws IOException { .execute(); assertThat(result.orphanFileLocations()).as("Should delete 4 files").hasSize(4); - assertThat(result.orphanFilesCount()).as("Should delete 4 files").isEqualTo(4L); Path dataPath = new Path(tableLocation + "/data"); FileSystem fs = dataPath.getFileSystem(spark.sessionState().newHadoopConf()); @@ -379,7 +366,6 @@ public void orphanedFileRemovedWithParallelTasks() { .containsExactlyInAnyOrder( "remove-orphan-0", "remove-orphan-1", "remove-orphan-2", "remove-orphan-3"); assertThat(deletedFiles).hasSize(4); - assertThat(result.orphanFilesCount()).as("Should delete 4 files").isEqualTo(4L); } @TestTemplate @@ -424,7 +410,6 @@ public void testWapFilesAreKept() { actions.deleteOrphanFiles(table).olderThan(System.currentTimeMillis()).execute(); assertThat(result.orphanFileLocations()).as("Should not delete any files").isEmpty(); - assertThat(result.orphanFilesCount()).as("Should not delete any files").isEqualTo(0L); } @TestTemplate @@ -455,7 +440,6 @@ public void testMetadataFolderIsIntact() { .execute(); assertThat(result.orphanFileLocations()).as("Should delete 1 file").hasSize(1); - assertThat(result.orphanFilesCount()).as("Should delete 1 file").isEqualTo(1L); Dataset resultDF = spark.read().format("iceberg").load(tableLocation); List actualRecords = @@ -494,7 +478,6 @@ public void testOlderThanTimestamp() { .execute(); assertThat(result.orphanFileLocations()).as("Should delete only 2 files").hasSize(2); - assertThat(result.orphanFilesCount()).as("Should delete only 2 files").isEqualTo(2L); } @TestTemplate @@ -526,7 +509,6 @@ public void testRemoveUnreachableMetadataVersionFiles() { assertThat(result.orphanFileLocations()) .containsExactly(tableLocation + "metadata/v1.metadata.json"); - assertThat(result.orphanFilesCount()).as("Should delete 1 file").isEqualTo(1L); List expectedRecords = Lists.newArrayList(); expectedRecords.addAll(records); @@ -563,7 +545,6 @@ public void testManyTopLevelPartitions() { .execute(); assertThat(result.orphanFileLocations()).as("Should not delete any files").isEmpty(); - assertThat(result.orphanFilesCount()).as("Should not delete any files").isEqualTo(0L); Dataset resultDF = spark.read().format("iceberg").load(tableLocation); assertThat(resultDF.count()).as("Rows count must match").isEqualTo(records.size()); @@ -594,7 +575,6 @@ public void testManyLeafPartitions() { .execute(); assertThat(result.orphanFileLocations()).as("Should not delete any files").isEmpty(); - assertThat(result.orphanFilesCount()).as("Should not delete any files").isEqualTo(0L); Dataset resultDF = spark.read().format("iceberg").load(tableLocation); assertThat(resultDF.count()).as("Row count must match").isEqualTo(records.size()); @@ -635,7 +615,6 @@ public void testHiddenPartitionPaths() { .execute(); assertThat(result.orphanFileLocations()).as("Should delete 2 files").hasSize(2); - assertThat(result.orphanFilesCount()).as("Should delete 2 files").isEqualTo(2L); } @TestTemplate @@ -676,7 +655,6 @@ public void testHiddenPartitionPathsWithPartitionEvolution() { .execute(); assertThat(result.orphanFileLocations()).as("Should delete 2 files").hasSize(2); - assertThat(result.orphanFilesCount()).as("Should delete 2 files").isEqualTo(2L); } @TestTemplate @@ -716,7 +694,6 @@ public void testHiddenPathsStartingWithPartitionNamesAreIgnored() throws IOExcep .execute(); assertThat(result.orphanFileLocations()).as("Should delete 0 files").isEmpty(); - assertThat(result.orphanFilesCount()).as("Should delete 0 files").isEqualTo(0L); assertThat(fs.exists(pathToFileInHiddenFolder)).isTrue(); } @@ -789,9 +766,6 @@ public void testRemoveOrphanFilesWithRelativeFilePath() throws IOException { assertThat(result.orphanFileLocations()) .as("Action should find 1 file") .isEqualTo(invalidFiles); - assertThat(result.orphanFilesCount()) - .as("Action should find 1 file") - .isEqualTo((long) invalidFiles.size()); assertThat(fs.exists(new Path(invalidFiles.get(0)))) .as("Invalid file should be present") .isTrue(); @@ -829,7 +803,6 @@ public void testRemoveOrphanFilesWithHadoopCatalog() throws InterruptedException .execute(); assertThat(result.orphanFileLocations()).as("Should delete only 1 file").hasSize(1); - assertThat(result.orphanFilesCount()).as("Should delete only 1 file").isEqualTo(1L); Dataset resultDF = spark.read().format("iceberg").load(table.location()); List actualRecords = @@ -865,7 +838,6 @@ public void testHiveCatalogTable() throws IOException { assertThat(result.orphanFileLocations()) .as("trash file should be removed") .contains("file:" + location + "/data/trashfile"); - assertThat(result.orphanFilesCount()).as("trash file should be removed").isEqualTo(1L); } @TestTemplate @@ -960,9 +932,6 @@ public void testCompareToFileList() throws IOException { assertThat(result1.orphanFileLocations()) .as("Default olderThan interval should be safe") .isEmpty(); - assertThat(result1.orphanFilesCount()) - .as("Default olderThan interval should be safe") - .isEqualTo(0L); DeleteOrphanFiles.Result result2 = actions @@ -974,9 +943,6 @@ public void testCompareToFileList() throws IOException { assertThat(result2.orphanFileLocations()) .as("Action should find 1 file") .isEqualTo(invalidFilePaths); - assertThat(result2.orphanFilesCount()) - .as("Action should find 1 file") - .isEqualTo((long) invalidFilePaths.size()); assertThat(fs.exists(new Path(invalidFilePaths.get(0)))) .as("Invalid file should be present") .isTrue(); @@ -990,9 +956,6 @@ public void testCompareToFileList() throws IOException { assertThat(result3.orphanFileLocations()) .as("Action should delete 1 file") .isEqualTo(invalidFilePaths); - assertThat(result3.orphanFilesCount()) - .as("Action should delete 1 file") - .isEqualTo((long) invalidFilePaths.size()); assertThat(fs.exists(new Path(invalidFilePaths.get(0)))) .as("Invalid file should not be present") .isFalse(); @@ -1022,7 +985,6 @@ public void testCompareToFileList() throws IOException { .deleteWith(s -> {}) .execute(); assertThat(result4.orphanFileLocations()).as("Action should find nothing").isEmpty(); - assertThat(result4.orphanFilesCount()).as("Action should find nothing").isEqualTo(0L); } protected long waitUntilAfter(long timestampMillis) { @@ -1102,7 +1064,6 @@ public void testRemoveOrphanFilesWithStatisticFiles() throws Exception { .execute(); Iterable orphanFileLocations = result.orphanFileLocations(); assertThat(orphanFileLocations).hasSize(1).containsExactly(statsLocation.toURI().toString()); - assertThat(result.orphanFilesCount()).as("Should delete 1 file").isEqualTo(1L); assertThat(statsLocation).as("stats file should be deleted").doesNotExist(); } @@ -1321,9 +1282,6 @@ public void testStreamResultsDeletion() throws IOException { .as("Non-streaming dry-run should return all 10 orphan files") .hasSize(10) .containsExactlyInAnyOrderElementsOf(invalidFiles); - assertThat(nonStreamingResult.orphanFilesCount()) - .as("Non-streaming dry-run should return all 10 orphan files") - .isEqualTo((long) invalidFiles.size()); DeleteOrphanFiles.Result streamingResult = SparkActions.get() @@ -1337,9 +1295,6 @@ public void testStreamResultsDeletion() throws IOException { assertThat(streamingResult.orphanFileLocations()) .as("Streaming with sample size 5 should return only 5 orphan files") .hasSize(5); - assertThat(streamingResult.orphanFilesCount()) - .as("Deleted 10 files") - .isEqualTo((long) invalidFiles.size()); for (String invalidFile : invalidFiles) { assertThat(fs.exists(new Path(invalidFile))).as("Orphan file should be deleted").isFalse(); diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction3.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction3.java index 88ac800b158f..5f98287951f1 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction3.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction3.java @@ -60,7 +60,6 @@ public void testSparkCatalogTable() throws Exception { assertThat(results.orphanFileLocations()) .as("trash file should be removed") .contains("file:" + location + trashFile); - assertThat(results.orphanFilesCount()).as("trash file should be removed").isEqualTo(1L); } @TestTemplate @@ -90,7 +89,6 @@ public void testSparkCatalogNamedHadoopTable() throws Exception { assertThat(results.orphanFileLocations()) .as("trash file should be removed") .contains("file:" + location + trashFile); - assertThat(results.orphanFilesCount()).as("trash file should be removed").isEqualTo(1L); } @TestTemplate @@ -121,7 +119,6 @@ public void testSparkCatalogNamedHiveTable() throws Exception { assertThat(StreamSupport.stream(results.orphanFileLocations().spliterator(), false)) .as("trash file should be removed") .anyMatch(file -> file.contains("file:" + location + trashFile)); - assertThat(results.orphanFilesCount()).as("trash file should be removed").isEqualTo(1L); } @TestTemplate @@ -154,7 +151,6 @@ public void testSparkSessionCatalogHadoopTable() throws Exception { assertThat(results.orphanFileLocations()) .as("trash file should be removed") .contains("file:" + location + trashFile); - assertThat(results.orphanFilesCount()).as("trash file should be removed").isEqualTo(1L); } @TestTemplate @@ -187,7 +183,6 @@ public void testSparkSessionCatalogHiveTable() throws Exception { assertThat(results.orphanFileLocations()) .as("trash file should be removed") .contains("file:" + location + trashFile); - assertThat(results.orphanFilesCount()).as("trash file should be removed").isEqualTo(1L); } @AfterEach diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/DeleteOrphanFilesSparkAction.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/DeleteOrphanFilesSparkAction.java index 92bfc880ad7f..78662159b0bb 100644 --- a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/DeleteOrphanFilesSparkAction.java +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/DeleteOrphanFilesSparkAction.java @@ -300,10 +300,7 @@ private DeleteOrphanFiles.Result deleteFiles(Dataset orphanFileDS) { LOG.info("Deleted {} orphan files", filesCount); - return ImmutableDeleteOrphanFiles.Result.builder() - .orphanFileLocations(orphanFileList) - .orphanFilesCount(filesCount) - .build(); + return ImmutableDeleteOrphanFiles.Result.builder().orphanFileLocations(orphanFileList).build(); } private void collectPathsForOutput( diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java index 7429b4c8aad4..40505b856737 100644 --- a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java @@ -184,9 +184,6 @@ public void testDryRun() throws IOException { assertThat(result1.orphanFileLocations()) .as("Default olderThan interval should be safe") .isEmpty(); - assertThat(result1.orphanFilesCount()) - .as("Default olderThan interval should be safe") - .isEqualTo(0L); DeleteOrphanFiles.Result result2 = actions @@ -198,9 +195,6 @@ public void testDryRun() throws IOException { assertThat(result2.orphanFileLocations()) .as("Action should find 1 file") .isEqualTo(invalidFiles); - assertThat(result2.orphanFilesCount()) - .as("Action should find 1 file") - .isEqualTo(invalidFiles.size()); assertThat(fs.exists(new Path(invalidFiles.get(0)))) .as("Invalid file should be present") .isTrue(); @@ -216,9 +210,6 @@ public void testDryRun() throws IOException { assertThat(result3.orphanFileLocations()) .as("Streaming dry run should find 1 file") .isEqualTo(invalidFiles); - assertThat(result3.orphanFilesCount()) - .as("Streaming dry run should find 1 file") - .isEqualTo(invalidFiles.size()); assertThat(fs.exists(new Path(invalidFiles.get(0)))) .as("Invalid file should be present after streaming dry run") .isTrue(); @@ -232,9 +223,6 @@ public void testDryRun() throws IOException { assertThat(result4.orphanFileLocations()) .as("Action should delete 1 file") .isEqualTo(invalidFiles); - assertThat(result4.orphanFilesCount()) - .as("Action should delete 1 file") - .isEqualTo(invalidFiles.size()); assertThat(fs.exists(new Path(invalidFiles.get(0)))) .as("Invalid file should not be present") .isFalse(); @@ -298,7 +286,6 @@ public void testAllValidFilesAreKept() throws IOException { .execute(); assertThat(result.orphanFileLocations()).as("Should delete 4 files").hasSize(4); - assertThat(result.orphanFilesCount()).as("Should delete 4 files").isEqualTo(4L); Path dataPath = new Path(tableLocation + "/data"); FileSystem fs = dataPath.getFileSystem(spark.sessionState().newHadoopConf()); @@ -379,7 +366,6 @@ public void orphanedFileRemovedWithParallelTasks() { .containsExactlyInAnyOrder( "remove-orphan-0", "remove-orphan-1", "remove-orphan-2", "remove-orphan-3"); assertThat(deletedFiles).hasSize(4); - assertThat(result.orphanFilesCount()).as("Should delete 4 files").isEqualTo(4L); } @TestTemplate @@ -424,7 +410,6 @@ public void testWapFilesAreKept() { actions.deleteOrphanFiles(table).olderThan(System.currentTimeMillis()).execute(); assertThat(result.orphanFileLocations()).as("Should not delete any files").isEmpty(); - assertThat(result.orphanFilesCount()).as("Should not delete any files").isEqualTo(0L); } @TestTemplate @@ -455,7 +440,6 @@ public void testMetadataFolderIsIntact() { .execute(); assertThat(result.orphanFileLocations()).as("Should delete 1 file").hasSize(1); - assertThat(result.orphanFilesCount()).as("Should delete 1 file").isEqualTo(1L); Dataset resultDF = spark.read().format("iceberg").load(tableLocation); List actualRecords = @@ -494,7 +478,6 @@ public void testOlderThanTimestamp() { .execute(); assertThat(result.orphanFileLocations()).as("Should delete only 2 files").hasSize(2); - assertThat(result.orphanFilesCount()).as("Should delete only 2 files").isEqualTo(2L); } @TestTemplate @@ -526,7 +509,6 @@ public void testRemoveUnreachableMetadataVersionFiles() { assertThat(result.orphanFileLocations()) .containsExactly(tableLocation + "metadata/v1.metadata.json"); - assertThat(result.orphanFilesCount()).as("Should delete 1 file").isEqualTo(1L); List expectedRecords = Lists.newArrayList(); expectedRecords.addAll(records); @@ -563,7 +545,6 @@ public void testManyTopLevelPartitions() { .execute(); assertThat(result.orphanFileLocations()).as("Should not delete any files").isEmpty(); - assertThat(result.orphanFilesCount()).as("Should not delete any files").isEqualTo(0L); Dataset resultDF = spark.read().format("iceberg").load(tableLocation); assertThat(resultDF.count()).as("Rows count must match").isEqualTo(records.size()); @@ -594,7 +575,6 @@ public void testManyLeafPartitions() { .execute(); assertThat(result.orphanFileLocations()).as("Should not delete any files").isEmpty(); - assertThat(result.orphanFilesCount()).as("Should not delete any files").isEqualTo(0L); Dataset resultDF = spark.read().format("iceberg").load(tableLocation); assertThat(resultDF.count()).as("Row count must match").isEqualTo(records.size()); @@ -635,7 +615,6 @@ public void testHiddenPartitionPaths() { .execute(); assertThat(result.orphanFileLocations()).as("Should delete 2 files").hasSize(2); - assertThat(result.orphanFilesCount()).as("Should delete 2 files").isEqualTo(2L); } @TestTemplate @@ -676,7 +655,6 @@ public void testHiddenPartitionPathsWithPartitionEvolution() { .execute(); assertThat(result.orphanFileLocations()).as("Should delete 2 files").hasSize(2); - assertThat(result.orphanFilesCount()).as("Should delete 2 files").isEqualTo(2L); } @TestTemplate @@ -716,7 +694,6 @@ public void testHiddenPathsStartingWithPartitionNamesAreIgnored() throws IOExcep .execute(); assertThat(result.orphanFileLocations()).as("Should delete 0 files").isEmpty(); - assertThat(result.orphanFilesCount()).as("Should delete 0 files").isEqualTo(0L); assertThat(fs.exists(pathToFileInHiddenFolder)).isTrue(); } @@ -789,9 +766,6 @@ public void testRemoveOrphanFilesWithRelativeFilePath() throws IOException { assertThat(result.orphanFileLocations()) .as("Action should find 1 file") .isEqualTo(invalidFiles); - assertThat(result.orphanFilesCount()) - .as("Action should find 1 file") - .isEqualTo(invalidFiles.size()); assertThat(fs.exists(new Path(invalidFiles.get(0)))) .as("Invalid file should be present") .isTrue(); @@ -829,7 +803,6 @@ public void testRemoveOrphanFilesWithHadoopCatalog() throws InterruptedException .execute(); assertThat(result.orphanFileLocations()).as("Should delete only 1 file").hasSize(1); - assertThat(result.orphanFilesCount()).as("Should delete only 1 file").isEqualTo(1L); Dataset resultDF = spark.read().format("iceberg").load(table.location()); List actualRecords = @@ -865,7 +838,6 @@ public void testHiveCatalogTable() throws IOException { assertThat(result.orphanFileLocations()) .as("trash file should be removed") .contains("file:" + location + "/data/trashfile"); - assertThat(result.orphanFilesCount()).as("trash file should be removed").isEqualTo(1L); } @TestTemplate @@ -960,9 +932,6 @@ public void testCompareToFileList() throws IOException { assertThat(result1.orphanFileLocations()) .as("Default olderThan interval should be safe") .isEmpty(); - assertThat(result1.orphanFilesCount()) - .as("Default olderThan interval should be safe") - .isEqualTo(0L); DeleteOrphanFiles.Result result2 = actions @@ -974,9 +943,6 @@ public void testCompareToFileList() throws IOException { assertThat(result2.orphanFileLocations()) .as("Action should find 1 file") .isEqualTo(invalidFilePaths); - assertThat(result2.orphanFilesCount()) - .as("Action should find 1 file") - .isEqualTo(invalidFilePaths.size()); assertThat(fs.exists(new Path(invalidFilePaths.get(0)))) .as("Invalid file should be present") .isTrue(); @@ -990,9 +956,6 @@ public void testCompareToFileList() throws IOException { assertThat(result3.orphanFileLocations()) .as("Action should delete 1 file") .isEqualTo(invalidFilePaths); - assertThat(result3.orphanFilesCount()) - .as("Action should delete 1 file") - .isEqualTo(invalidFilePaths.size()); assertThat(fs.exists(new Path(invalidFilePaths.get(0)))) .as("Invalid file should not be present") .isFalse(); @@ -1022,7 +985,6 @@ public void testCompareToFileList() throws IOException { .deleteWith(s -> {}) .execute(); assertThat(result4.orphanFileLocations()).as("Action should find nothing").isEmpty(); - assertThat(result4.orphanFilesCount()).as("Action should find nothing").isEqualTo(0L); } protected long waitUntilAfter(long timestampMillis) { @@ -1102,7 +1064,6 @@ public void testRemoveOrphanFilesWithStatisticFiles() throws Exception { .execute(); Iterable orphanFileLocations = result.orphanFileLocations(); assertThat(orphanFileLocations).hasSize(1).containsExactly(statsLocation.toURI().toString()); - assertThat(result.orphanFilesCount()).as("Should delete 1 file").isEqualTo(1L); assertThat(statsLocation).as("stats file should be deleted").doesNotExist(); } @@ -1321,9 +1282,6 @@ public void testStreamResultsDeletion() throws IOException { .as("Non-streaming dry-run should return all 10 orphan files") .hasSize(10) .containsExactlyInAnyOrderElementsOf(invalidFiles); - assertThat(nonStreamingResult.orphanFilesCount()) - .as("Non-streaming dry-run should return all 10 orphan files") - .isEqualTo(invalidFiles.size()); DeleteOrphanFiles.Result streamingResult = SparkActions.get() @@ -1337,9 +1295,6 @@ public void testStreamResultsDeletion() throws IOException { assertThat(streamingResult.orphanFileLocations()) .as("Streaming with sample size 5 should return only 5 orphan files") .hasSize(5); - assertThat(streamingResult.orphanFilesCount()) - .as("Deleted 10 files") - .isEqualTo(invalidFiles.size()); for (String invalidFile : invalidFiles) { assertThat(fs.exists(new Path(invalidFile))).as("Orphan file should be deleted").isFalse(); diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction3.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction3.java index 88ac800b158f..5f98287951f1 100644 --- a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction3.java +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction3.java @@ -60,7 +60,6 @@ public void testSparkCatalogTable() throws Exception { assertThat(results.orphanFileLocations()) .as("trash file should be removed") .contains("file:" + location + trashFile); - assertThat(results.orphanFilesCount()).as("trash file should be removed").isEqualTo(1L); } @TestTemplate @@ -90,7 +89,6 @@ public void testSparkCatalogNamedHadoopTable() throws Exception { assertThat(results.orphanFileLocations()) .as("trash file should be removed") .contains("file:" + location + trashFile); - assertThat(results.orphanFilesCount()).as("trash file should be removed").isEqualTo(1L); } @TestTemplate @@ -121,7 +119,6 @@ public void testSparkCatalogNamedHiveTable() throws Exception { assertThat(StreamSupport.stream(results.orphanFileLocations().spliterator(), false)) .as("trash file should be removed") .anyMatch(file -> file.contains("file:" + location + trashFile)); - assertThat(results.orphanFilesCount()).as("trash file should be removed").isEqualTo(1L); } @TestTemplate @@ -154,7 +151,6 @@ public void testSparkSessionCatalogHadoopTable() throws Exception { assertThat(results.orphanFileLocations()) .as("trash file should be removed") .contains("file:" + location + trashFile); - assertThat(results.orphanFilesCount()).as("trash file should be removed").isEqualTo(1L); } @TestTemplate @@ -187,7 +183,6 @@ public void testSparkSessionCatalogHiveTable() throws Exception { assertThat(results.orphanFileLocations()) .as("trash file should be removed") .contains("file:" + location + trashFile); - assertThat(results.orphanFilesCount()).as("trash file should be removed").isEqualTo(1L); } @AfterEach From cc281f8691e08c9fb7985dcd32e45dc5ae922509 Mon Sep 17 00:00:00 2001 From: Alessandro Nori Date: Mon, 26 Jan 2026 10:54:59 +0100 Subject: [PATCH 8/8] update assertion description --- .../iceberg/spark/actions/TestRemoveOrphanFilesAction.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java index 64414e4513ab..0d2a5c0a4daf 100644 --- a/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java +++ b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java @@ -185,7 +185,7 @@ public void testDryRun() throws IOException { .as("Default olderThan interval should be safe") .isEmpty(); assertThat(result1.orphanFilesCount()) - .as("Default olderThan interval should be safe") + .as("Should not find any orphan file using default olderThan interval") .isEqualTo(0L); DeleteOrphanFiles.Result result2 = @@ -961,7 +961,7 @@ public void testCompareToFileList() throws IOException { .as("Default olderThan interval should be safe") .isEmpty(); assertThat(result1.orphanFilesCount()) - .as("Default olderThan interval should be safe") + .as("Should not find any orphan file using default olderThan interval") .isEqualTo(0L); DeleteOrphanFiles.Result result2 =