sumeetgajjar opened a new pull request, #4825:
URL: https://github.com/apache/iceberg/pull/4825

   We at Cloudera run Iceberg tests on Apache/master branch on a daily basis. 
   Ever since #4711 was merged 
`org.apache.iceberg.spark.actions.TestRemoveOrphanFilesAction3.testOlderThanTimestamp`
 is failing in our internal builds with the following assertion error:
   ```bash
   java.lang.AssertionError: Should delete only 2 files expected:<2> but was:<3>
        at org.junit.Assert.fail(Assert.java:89)
        at org.junit.Assert.failNotEquals(Assert.java:835)
        at org.junit.Assert.assertEquals(Assert.java:647)
        at 
org.apache.iceberg.spark.actions.TestRemoveOrphanFilesAction.testOlderThanTimestamp(TestRemoveOrphanFilesAction.java:433)
   ```
   
   We identified the RC as the following:
   The code execution on our VM is so fast that the millisecond precision used 
for selecting the candidate as an orphan file is simply not enough here.
   
   The test code is as follows:
   
https://github.com/apache/iceberg/blob/71282b8ca7d0c703e4fd4ad460821eaec52124ce/spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java#L416-L431
   
   The predicate used to test if a given file should be considered an orphan or 
not is as follows:
   
https://github.com/apache/iceberg/blob/71282b8ca7d0c703e4fd4ad460821eaec52124ce/spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/actions/BaseDeleteOrphanFilesSparkAction.java#L239
   
   ### Our Hypothesis
   The code execution for creating files using the three append statements in 
the above test code is so fast that all three files have the same milliseconds 
in their modificationTime. 
   Thus when the DeleteOrphanFiles action is executed it classifies all three 
files as orphans thereby failing the test.
   
   Running the test locally or in GitHub actions does not show any failure. The 
local machine runs a lot of other processes apart from the test code thus the 
execution is relatively slow giving enough time for the Predicate to work 
correctly. 
   
   ### Hypothesis test
   We used the following code to validate our hypothesis and the result is 
clearly visible in the Stdout of the test
   ```bash
   diff --git 
a/spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java
 
b/spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java
   index bcabe38e7..3e3b6b9a1 100644
   --- 
a/spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java
   +++ 
b/spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java
   @@ -424,6 +424,20 @@ public abstract class TestRemoveOrphanFilesAction 
extends SparkTestBase {
    
        df.write().mode("append").parquet(tableLocation + 
"/data/c2_trunc=AA/c3=AAAA");
    
   +    try {
   +      System.out.println("OlderThanTimeStamp -> " + timestamp);
   +      Path dfDir = new Path(tableLocation + "/data/c2_trunc=AA/c3=AAAA");
   +      FileSystem fs = 
dfDir.getFileSystem(spark.sparkContext().hadoopConfiguration());
   +      String[] inputFiles = spark.read().parquet(tableLocation + 
"/data/c2_trunc=AA/c3=AAAA").inputFiles();
   +      for (FileStatus fileStatus : fs.listStatus(dfDir)) {
   +        if (Arrays.stream(inputFiles).anyMatch(inputFileName -> 
inputFileName.contains(fileStatus.getPath().getName()))) {
   +          System.out.println("SparkInputFileMTime -> " + 
fileStatus.getPath() + " -> " + fileStatus.getModificationTime());
   +        }
   +      }
   +    } catch (IOException e) {
   +      e.printStackTrace();
   +    }
   +
        SparkActions actions = SparkActions.get();
    
        DeleteOrphanFiles.Result result = actions.deleteOrphanFiles(table) 
   ```
   
   ### Stdout of test with the above addition
   ```bash
   OlderThanTimeStamp -> 1653037291034
   SparkInputFileMTime -> 
file:/tmp/junit4128601999154387775/junit5914981474932610748/data/c2_trunc=AA/c3=AAAA/part-00000-6f36c2a7-a937-4e60-bbf5-062316b365b5-c000.snappy.parquet
 -> 1653037290000
   SparkInputFileMTime -> 
file:/tmp/junit4128601999154387775/junit5914981474932610748/data/c2_trunc=AA/c3=AAAA/part-00000-b0e3b4eb-c5cf-47df-9979-b9ec5ac6c326-c000.snappy.parquet
 -> 1653037291000
   SparkInputFileMTime -> 
file:/tmp/junit4128601999154387775/junit5914981474932610748/data/c2_trunc=AA/c3=AAAA/part-00000-13ff79b6-1625-4280-86d5-ee70a341bb3d-c000.snappy.parquet
 -> 1653037291000
   SparkInputFileMTime -> 
file:/tmp/junit4128601999154387775/junit5914981474932610748/data/c2_trunc=AA/c3=AAAA/00000-0-5e8a0bd0-2b14-4416-8f33-77bc081aa064-00001.parquet
 -> 1653037290000 
   ```


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to