You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by GitBox <gi...@apache.org> on 2022/05/20 09:50:53 UTC

[GitHub] [iceberg] sumeetgajjar opened a new pull request, #4825: Test: Fix flaky testOlderThanTimestamp in TestRemoveOrphanFilesAction3

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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] kbendick commented on a diff in pull request #4825: Test: Fix flaky testOlderThanTimestamp in TestRemoveOrphanFilesAction3

Posted by GitBox <gi...@apache.org>.
kbendick commented on code in PR #4825:
URL: https://github.com/apache/iceberg/pull/4825#discussion_r878591978


##########
spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java:
##########
@@ -420,7 +420,7 @@ public void testOlderThanTimestamp() throws InterruptedException {
 

Review Comment:
   I can't comment on it, but do any of the other `waitUntilAfter` calls need this change? There's one right above this comment.
   
   I would think the one below would be sufficient, as it's two in a row (essentially this is sort of like thread.sleep(1000)).



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] sumeetgajjar commented on a diff in pull request #4825: Test: Fix flaky testOlderThanTimestamp in TestRemoveOrphanFilesAction3

Posted by GitBox <gi...@apache.org>.
sumeetgajjar commented on code in PR #4825:
URL: https://github.com/apache/iceberg/pull/4825#discussion_r878847100


##########
spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java:
##########
@@ -420,7 +420,7 @@ public void testOlderThanTimestamp() throws InterruptedException {
 
     long timestamp = System.currentTimeMillis();
 
-    waitUntilAfter(System.currentTimeMillis());
+    waitUntilAfter(System.currentTimeMillis() + 1000L);

Review Comment:
   ~~Ok, let me try that.~~



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on pull request #4825: Test: Fix flaky testOlderThanTimestamp in TestRemoveOrphanFilesAction3

Posted by GitBox <gi...@apache.org>.
rdblue commented on PR #4825:
URL: https://github.com/apache/iceberg/pull/4825#issuecomment-1153276096

   Thanks, @sumeetgajjar!


-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] singhpk234 commented on a diff in pull request #4825: Test: Fix flaky testOlderThanTimestamp in TestRemoveOrphanFilesAction3

Posted by GitBox <gi...@apache.org>.
singhpk234 commented on code in PR #4825:
URL: https://github.com/apache/iceberg/pull/4825#discussion_r880867926


##########
spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java:
##########
@@ -420,7 +420,7 @@ public void testOlderThanTimestamp() throws InterruptedException {
 

Review Comment:
   +1, I also see `org.apache.iceberg.spark.actions.TestRemoveOrphanFilesAction3 > orphanedFileRemovedWithParallelTasks FAILED` has become flaky in master as well, 
   
   failed iceberg run : https://github.com/apache/iceberg/runs/6577825750?check_suite_focus=true
   
   I think we should sleep by atleast 1000 ms, to handle issue like these [StackOverflow](https://stackoverflow.com/questions/24804618/get-file-mtime-with-millisecond-resolution-from-java).



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] sumeetgajjar commented on a diff in pull request #4825: Test: Fix flaky testOlderThanTimestamp in TestRemoveOrphanFilesAction3

Posted by GitBox <gi...@apache.org>.
sumeetgajjar commented on code in PR #4825:
URL: https://github.com/apache/iceberg/pull/4825#discussion_r878845854


##########
spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java:
##########
@@ -420,7 +420,7 @@ public void testOlderThanTimestamp() throws InterruptedException {
 

Review Comment:
   > I can't comment on it, but do any of the other waitUntilAfter calls need this change?
   
   This is the only place where it is necessary. 
   I took a look at all the other tests in this Suite when I filed the PR, and I did not find any places where we would require such a change.
   
   > as it's two in a row (essentially this is sort of like thread.sleep(1000)).
   
   Yes, exactly :-)



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #4825: Test: Fix flaky testOlderThanTimestamp in TestRemoveOrphanFilesAction3

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4825:
URL: https://github.com/apache/iceberg/pull/4825#discussion_r895211136


##########
spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java:
##########
@@ -420,7 +420,7 @@ public void testOlderThanTimestamp() throws InterruptedException {
 
     long timestamp = System.currentTimeMillis();
 
-    waitUntilAfter(System.currentTimeMillis());
+    waitUntilAfter(System.currentTimeMillis() + 1000L);

Review Comment:
   @sumeetgajjar, sorry, I think this was my misunderstanding. After going through your write-up more carefully, I see that the timestamps reported by the files are in seconds. So you're right: to ensure that the timestamp is different, we need to wait a full second.



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue merged pull request #4825: Test: Fix flaky testOlderThanTimestamp in TestRemoveOrphanFilesAction3

Posted by GitBox <gi...@apache.org>.
rdblue merged PR #4825:
URL: https://github.com/apache/iceberg/pull/4825


-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #4825: Test: Fix flaky testOlderThanTimestamp in TestRemoveOrphanFilesAction3

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4825:
URL: https://github.com/apache/iceberg/pull/4825#discussion_r880991367


##########
spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java:
##########
@@ -420,7 +420,7 @@ public void testOlderThanTimestamp() throws InterruptedException {
 
     long timestamp = System.currentTimeMillis();
 
-    waitUntilAfter(System.currentTimeMillis());
+    waitUntilAfter(System.currentTimeMillis() + 1000L);

Review Comment:
   I think the point is still valid. `waitUntilAfter` should be used so that you know that a certain amount of time, in milliseconds, has elapsed. That gives you the ability to create one file, wait until the next millisecond, create another, wait, etc. to make sure they don't have the same write time. It doesn't need to be a snapshot timestamp, but you shouldn't need to wait for an entire second.



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] sumeetgajjar commented on a diff in pull request #4825: Test: Fix flaky testOlderThanTimestamp in TestRemoveOrphanFilesAction3

Posted by GitBox <gi...@apache.org>.
sumeetgajjar commented on code in PR #4825:
URL: https://github.com/apache/iceberg/pull/4825#discussion_r878846273


##########
spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java:
##########
@@ -420,7 +420,7 @@ public void testOlderThanTimestamp() throws InterruptedException {
 
     long timestamp = System.currentTimeMillis();
 
-    waitUntilAfter(System.currentTimeMillis());
+    waitUntilAfter(System.currentTimeMillis() + 1000L);

Review Comment:
   Ok, let me try that as well.



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] sumeetgajjar commented on a diff in pull request #4825: Test: Fix flaky testOlderThanTimestamp in TestRemoveOrphanFilesAction3

Posted by GitBox <gi...@apache.org>.
sumeetgajjar commented on code in PR #4825:
URL: https://github.com/apache/iceberg/pull/4825#discussion_r878845854


##########
spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java:
##########
@@ -420,7 +420,7 @@ public void testOlderThanTimestamp() throws InterruptedException {
 

Review Comment:
   > I can't comment on it, but do any of the other waitUntilAfter calls need this change?
   This is the only place where it is necessary. 
   I took a look at all the other tests in this Suite when I filed the PR, and I did not find any places where we would require such a change.
   
   > as it's two in a row (essentially this is sort of like thread.sleep(1000)).
   Yes, exactly :-)
   
   



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] kbendick commented on a diff in pull request #4825: Test: Fix flaky testOlderThanTimestamp in TestRemoveOrphanFilesAction3

Posted by GitBox <gi...@apache.org>.
kbendick commented on code in PR #4825:
URL: https://github.com/apache/iceberg/pull/4825#discussion_r881230284


##########
spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java:
##########
@@ -420,7 +420,7 @@ public void testOlderThanTimestamp() throws InterruptedException {
 
     long timestamp = System.currentTimeMillis();
 
-    waitUntilAfter(System.currentTimeMillis());
+    waitUntilAfter(System.currentTimeMillis() + 1000L);

Review Comment:
   @sumeetgajjar left a comment on the main page of this PR, but check out the solution we came up with in another PR / issue discussion: https://github.com/apache/iceberg/pull/4859
   
   We noticed a different test in this suite fail in GitHub CI as well. We just made the `olderThan` argument in the future to account for it the time precision issue without having to do excessive busy waiting.



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] sumeetgajjar commented on pull request #4825: Test: Fix flaky testOlderThanTimestamp in TestRemoveOrphanFilesAction3

Posted by GitBox <gi...@apache.org>.
sumeetgajjar commented on PR #4825:
URL: https://github.com/apache/iceberg/pull/4825#issuecomment-1132711888

   Hi @kbendick @rdblue @ulmako,
   Can you please review this PR?


-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] kbendick commented on pull request #4825: Test: Fix flaky testOlderThanTimestamp in TestRemoveOrphanFilesAction3

Posted by GitBox <gi...@apache.org>.
kbendick commented on PR #4825:
URL: https://github.com/apache/iceberg/pull/4825#issuecomment-1136740153

   I’m not sure if you have seen this PR I opened today, but I noticed one of these tests fail in CI (details and link in PR summary). I opened this PR (admittedly I forgot about this one so we can close mine) - https://github.com/apache/iceberg/pull/4859
   
   TLDR - As suggested by Russell, for the test case in my PR, we changed the “olderThan” argument to be further in the future. The benefit being no unnecessary busy waiting and all of the files are still caught. I went with 5 seconds given that there was no required busy waiting.


-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] sumeetgajjar commented on pull request #4825: Test: Fix flaky testOlderThanTimestamp in TestRemoveOrphanFilesAction3

Posted by GitBox <gi...@apache.org>.
sumeetgajjar commented on PR #4825:
URL: https://github.com/apache/iceberg/pull/4825#issuecomment-1139330018

   > I’m not sure if you have seen this PR I opened today, but I noticed one of these tests fail in CI (details and link in PR summary). I opened the following PR for that test case (admittedly I forgot about this one so we can close mine if you’d like) - #4859
   > 
   > TLDR - As suggested by Russell, for the test case in my PR, we changed the “olderThan” argument to be further in the future. The benefit being no unnecessary busy waiting and all of the files are still caught. I went with 5 seconds given that the solution in the other PR adds no busy waiting and that specific test removes _every_ orphan file so the olderThan argument doesn’t need to be very precise… just far enough “in the future” relative to the timestamp of the files that Spark writes to grab them all.
   
   Hi @kbendick - thanks for the suggestion, however, the test that you are fixing in #4859 (TestRemoveOrphanFilesAction#orphanedFileRemovedWithParallelTasks) requires all the files to be removed. Thus providing a future time would ensure that Predicate selects all the files as candidates for removal.
   
   
   However, for this PR (TestRemoveOrphanFilesAction#testOlderThanTimestamp), we intentionally want to wait until a second has passed to avoid scenarios like [mtime from java is truncated to seconds](https://stackoverflow.com/questions/24804618/get-file-mtime-with-millisecond-resolution-from-java) where we loose tracking millisecond precision while getting lastModification time.
   
   
   The flow for this test is as follows:
   - spark writes non-table files to tableLocation
   - spark writes non-table files to tableLocation
   - spark writes non-table files to tableLocation
   - delete orphan files such that only files created from the first two writes are removed and the file created from the third write is preserved
   
   And because of this caveat of preserving the files from the third, we'll have to busy wait for a second.
   
   


-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] sumeetgajjar commented on a diff in pull request #4825: Test: Fix flaky testOlderThanTimestamp in TestRemoveOrphanFilesAction3

Posted by GitBox <gi...@apache.org>.
sumeetgajjar commented on code in PR #4825:
URL: https://github.com/apache/iceberg/pull/4825#discussion_r878848478


##########
spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java:
##########
@@ -420,7 +420,7 @@ public void testOlderThanTimestamp() throws InterruptedException {
 
     long timestamp = System.currentTimeMillis();
 
-    waitUntilAfter(System.currentTimeMillis());
+    waitUntilAfter(System.currentTimeMillis() + 1000L);

Review Comment:
   So I believe in this explicit case, using the timestamp from the snapshot summary won't be useful. The primary reason being we are trying to delete parquet files at the "data/c2_trunc=AA/c3=AAAA" location which are not managed by Iceberg.
   https://github.com/apache/iceberg/blob/566b2fe3f31f4a53f5a26aca1c4239085de4a994/spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java#L416-L417
   
   Had it been the case if the supposed to be orphan files were created using iceberg we could have leveraged the timestamp from the snapshot summary.



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] kbendick commented on a diff in pull request #4825: Test: Fix flaky testOlderThanTimestamp in TestRemoveOrphanFilesAction3

Posted by GitBox <gi...@apache.org>.
kbendick commented on code in PR #4825:
URL: https://github.com/apache/iceberg/pull/4825#discussion_r878591654


##########
spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java:
##########
@@ -420,7 +420,7 @@ public void testOlderThanTimestamp() throws InterruptedException {
 
     long timestamp = System.currentTimeMillis();
 
-    waitUntilAfter(System.currentTimeMillis());
+    waitUntilAfter(System.currentTimeMillis() + 1000L);

Review Comment:
   What would be better, if possible, would be to access the timestamp from the snapshot summary and then wait until after that (which is what we do in many other tests).
   
   However, given that this is a Spark test, the time it would take to access the summary from the commit means that it would likely take longer to do that than any of the `waitUntilAfter` (i.e. that time would likely have passed by at least a few hundred milliseconds on any machine).



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] sumeetgajjar commented on a diff in pull request #4825: Test: Fix flaky testOlderThanTimestamp in TestRemoveOrphanFilesAction3

Posted by GitBox <gi...@apache.org>.
sumeetgajjar commented on code in PR #4825:
URL: https://github.com/apache/iceberg/pull/4825#discussion_r881207933


##########
spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java:
##########
@@ -420,7 +420,7 @@ public void testOlderThanTimestamp() throws InterruptedException {
 
     long timestamp = System.currentTimeMillis();
 
-    waitUntilAfter(System.currentTimeMillis());
+    waitUntilAfter(System.currentTimeMillis() + 1000L);

Review Comment:
   > That gives you the ability to create one file, wait until the next millisecond, create another, wait, etc. to make sure they don't have the same write time.
   
   Hi @rdblue - I agree with you. 
   Writing the file at the very next millisecond should do the trick here but I believe that is what the original code is doing and still, we are seeing the failure. 
   
   We experimented with various values less than 1000L milliseconds but none of them got us a 100% success rate for the test. 1000L was the lowest value that gave a 100% success rate.
   
   I can investigate more if required or use the 1000L value. Using a value of 1000L would make this test equivalent to what we had before #4711 
   
   Please let me know your thoughts.



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org