kasured opened a new issue, #5843: URL: https://github.com/apache/hudi/issues/5843
**Describe the problem you faced** Under certain circumstances Hoodies can create a commit on a timeline far in the future (days and years). **To Reproduce** At the moment there is no clear way to reproduce the issue. It was discovered with the same symptoms as described in https://github.com/apache/hudi/issues/2533 when Hoodies denied to rollback the commit because there is another commit that it has found in the future. **Expected behavior** Hoodie does not create successful commits far in the future **Environment Description** * EMR version: 6.5.0 * Hudi version : 0.9.0-amzn-1 * Spark version : 3.1.2 * Hadoop version : 3.2.1 * Storage (HDFS/S3/GCS..) : S3 * Running on Docker? (yes/no) : no **Additional context** * We have a single Spark Application with multiple Structured Streaming queries running at the same time as described in https://github.com/apache/hudi/issues/5298 * We are using COW with a single writer model and we have `hoodie.cleaner.policy.failed.writes=true` by default * It is expected that Spark Driver can be restarted by YARN so if there is an inflight commit that has not been completed it will be rolled-back. In method `BaseRollbackActionExecutor#validateRollbackCommitSequence` there is a logic that prevents the roll back because the future commit has been found on the timeline. Which is also understandable ``` private void validateRollbackCommitSequence() { // Continue to provide the same behavior if policy is EAGER (similar to pendingRollback logic). This is required // since with LAZY rollback we support parallel writing which can allow a new inflight while rollback is ongoing // Remove this once we support LAZY rollback of failed writes by default as parallel writing becomes the default // writer mode. if (config.getFailedWritesCleanPolicy().isEager()) { ``` * After looking into the s3 location we were able to see the following ``` 2022-06-01 08:43:16 12147 20220601084314.clean.inflight 2022-06-01 08:43:16 12147 20220601084314.clean.requested 2022-06-01 08:43:17 10731 20220601084314.clean 2022-06-01 09:00:54 0 20220629090052.commit.requested 2022-06-01 09:05:50 54609 20220629090052.inflight 2022-06-01 09:08:26 68699 20220629090052.commit 2022-06-01 09:55:09 0 20220601095507.commit.requested 2022-06-01 10:04:21 56777 20220601095507.inflight 2022-06-01 10:13:10 68725 20220601095507.commit 2022-06-01 12:00:46 0 20220601120045.commit.requested ``` * As can be seen above Hoodie were expectedly trying to roll back the non-completed commit 20220601120045 but it found the completed commit 20220629090052 which is more that 28 days in the future as per Hoodie instant time * We have checked another locations and it looks like there are even commits had been created and completed in 2023 ``` 2022-06-11 16:05:28 2337 20220611160525.clean.requested 2022-06-11 16:33:15 8789 20220611163013.commit 2022-06-11 16:30:14 0 20220611163013.commit.requested 2022-06-11 16:30:22 3928 20220611163013.inflight 2022-05-26 08:30:53 1746 20231226083050.clean 2022-05-26 08:30:52 1752 20231226083050.clean.inflight 2022-05-26 08:30:52 1752 20231226083050.clean.requested ``` * The logic of creating the instant time is in the `HoodieActiveTimeline#createNewInstantTime` ``` private static AtomicReference<String> lastInstantTime = new AtomicReference<>(String.valueOf(Integer.MIN_VALUE)); /** * Returns next instant time that adds N milliseconds in the {@link #COMMIT_FORMATTER} format. * Ensures each instant time is atleast 1 second apart since we create instant times at second granularity */ public static String createNewInstantTime(long milliseconds) { return lastInstantTime.updateAndGet((oldVal) -> { String newCommitTime; do { newCommitTime = HoodieActiveTimeline.COMMIT_FORMATTER.format(new Date(System.currentTimeMillis() + milliseconds)); } while (HoodieTimeline.compareTimestamps(newCommitTime, LESSER_THAN_OR_EQUALS, oldVal)); return newCommitTime; }); } ``` * I am planning to reproduce this issue but it looks like Hoodies might get in the state when optimistic locking with CAS approach causes multiple threads to update the value until that condition disappears and the commit can be created and more importantly flushed * This might cause unexpected querying results when doing incremental querying. **Stacktrace** ```Caused by: org.apache.hudi.exception.HoodieRollbackException: Found commits after time :20220601120045, please rollback greater commits first``` -- 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]
