abhisheksahani91 opened a new issue, #11745:
URL: https://github.com/apache/hudi/issues/11745

   **_Tips before filing an issue_**
   
   - Have you gone through our [FAQs](https://hudi.apache.org/learn/faq/)?
   
   - Join the mailing list to engage in conversations and get faster support at 
[email protected].
   
   - If you have triaged this as a bug, then file an 
[issue](https://issues.apache.org/jira/projects/HUDI/issues) directly.
   
   **Describe the problem you faced**
   
   We want to disable the Async Compaction and allow only scheduling the 
compaction in the delta streamer process. 
   For Compaction, we have a standalone process utilizing HoodieCompactor of 
hudi-utilities  
   
   what we are observing in logs is that the AsyncCompaction service is still 
running along with the deltastreamer/ingestion process  and it is competing 
with the standalone compaction process for the timeline compaction instant that 
has already gone to the compaction completed stage  
   
   
   **To Reproduce**
   
   Steps to reproduce the behavior:
   
   1. in deltastreamer add the following config to disable compaction and allow 
scheduling compaction instant
       a. hoodie.compact.inline=false
       b. hoodie.compact.schedule.inline=true
   2. use HoodieCompactor util for offline compaction 
   
   
   **Expected behavior**
   deltastreamer/ingestion process should only schedule the compaction 
   A clear and concise description of what you expected to happen.
   
   **Environment Description**
   
   * Hudi version : 0.12.1
   
   * Spark version : 3.3
   
   * Hive version :
   
   * Hadoop version :
   
   * Storage (HDFS/S3/GCS..) : s3
   
   * Running on Docker? (yes/no) : no
   
   
   **Additional context**
   
   Add any other context about the problem here.
   
   We are observing that Async Compaction is still running along with the 
DeltaStreamer and it is failing as the Offline Compaction process already 
completes the compaction
   
   **Stacktrace**
   
   ```Add the stack trace of the error.```
   LOGS that shows AsyncCompaction is still running and it failed as compaction 
instant is already completed by the offline compactor util.
   
   24/08/08 16:42:46 ERROR AsyncCompactService: Compactor executor failed
   org.apache.hudi.exception.HoodieIOException: Failed to create 
files3://g24x7-eds-datalake/hudi_data_lake_prod10/prod_eds_datalake_ups_latest_v3/.hoodie/20240808160446342.commit
        at 
org.apache.hudi.common.fs.HoodieWrapperFileSystem.createImmutableFileInPath(HoodieWrapperFileSystem.java:1035)
 ~[hudi-aws-bundle-0.12.1.jar:0.12.1]
        at 
org.apache.hudi.common.table.timeline.HoodieActiveTimeline.transitionState(HoodieActiveTimeline.java:582)
 ~[hudi-aws-bundle-0.12.1.jar:0.12.1]
        at 
org.apache.hudi.common.table.timeline.HoodieActiveTimeline.transitionState(HoodieActiveTimeline.java:558)
 ~[hudi-aws-bundle-0.12.1.jar:0.12.1]
        at 
org.apache.hudi.common.table.timeline.HoodieActiveTimeline.transitionCompactionInflightToComplete(HoodieActiveTimeline.java:435)
 ~[hudi-aws-bundle-0.12.1.jar:0.12.1]
        at 
org.apache.hudi.table.action.compact.CompactHelpers.completeInflightCompaction(CompactHelpers.java:78)
 ~[hudi-spark3-bundle_2.12-0.12.1.jar:0.12.1]
        at 
org.apache.hudi.client.SparkRDDWriteClient.completeCompaction(SparkRDDWriteClient.java:316)
 ~[hudi-spark3-bundle_2.12-0.12.1.jar:0.12.1]
        at 
org.apache.hudi.client.SparkRDDWriteClient.commitCompaction(SparkRDDWriteClient.java:300)
 ~[hudi-spark3-bundle_2.12-0.12.1.jar:0.12.1]
        at 
org.apache.hudi.client.HoodieSparkCompactor.compact(HoodieSparkCompactor.java:62)
 ~[hudi-spark3-bundle_2.12-0.12.1.jar:0.12.1]
        at 
org.apache.hudi.async.AsyncCompactService.lambda$null$0(AsyncCompactService.java:84)
 ~[hudi-spark3-bundle_2.12-0.12.1.jar:0.12.1]
        at 
java.util.concurrent.CompletableFuture$AsyncSupply.run(CompletableFuture.java:1604)
 ~[?:1.8.0_412]
        at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) 
~[?:1.8.0_412]
        at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) 
~[?:1.8.0_412]
        at java.lang.Thread.run(Thread.java:750) ~[?:1.8.0_412]
   **Caused by: org.apache.hadoop.fs.FileAlreadyExistsException: File already 
exists:s3://g24x7-eds-datalake/hudi_data_lake_prod10/prod_eds_datalake_ups_latest_v3/.hoodie/20240808160446342.commit**
   
   LOGS that show Async Compaction Service is executing along with the delta 
streamer process
   24/08/08 16:04:50 INFO BaseHoodieWriteClient: Scheduling compaction at 
instant time :20240808160446342
   24/08/08 16:04:50 INFO HoodieCompactor: Compacting 
s3://g24x7-eds-datalake/hudi_data_lake_prod10/prod_eds_datalake_ups_latest_v3 
with commit 20240808160446342
   24/08/08 16:24:01 INFO AsyncCompactService: Starting Compaction for instant 
[==>20240808160446342__compaction__REQUESTED]
   24/08/08 16:24:02 INFO MergeOnReadRollbackActionExecutor: Rolling back 
instant [==>20240808160446342__compaction__INFLIGHT]
   24/08/08 16:24:02 INFO MergeOnReadRollbackActionExecutor: Unpublished 
[==>20240808160446342__compaction__INFLIGHT]
   24/08/08 16:24:02 INFO BaseRollbackActionExecutor: Rolled back inflight 
instant 20240808160446342
   24/08/08 16:24:04 WARN BaseRollbackActionExecutor: Rollback finished without 
deleting inflight instant file. 
Instant=[==>20240808160446342__compaction__INFLIGHT]
   24/08/08 16:24:05 INFO BaseRollbackActionExecutor: Rollback of Commits 
[20240808160446342] is complete
   24/08/08 16:24:05 INFO HoodieActiveTimeline: Deleting instant 
[==>20240808160446342__compaction__INFLIGHT]
   24/08/08 16:24:05 INFO HoodieActiveTimeline: Removed instant 
[==>20240808160446342__compaction__INFLIGHT]
   
   24/08/08 16:42:46 ERROR AsyncCompactService: Compactor executor failed
   org.apache.hudi.exception.HoodieIOException: Failed to create 
files3://g24x7-eds-datalake/hudi_data_lake_prod10/prod_eds_datalake_ups_latest_v3/.hoodie/20240808160446342.commit
        at 
org.apache.hudi.common.fs.HoodieWrapperFileSystem.createImmutableFileInPath(HoodieWrapperFileSystem.java:1035)
 ~[hudi-aws-bundle-0.12.1.jar:0.12.1]
        at 
org.apache.hudi.common.table.timeline.HoodieActiveTimeline.transitionState(HoodieActiveTimeline.java:582)
 ~[hudi-aws-bundle-0.12.1.jar:0.12.1]
        at 
org.apache.hudi.common.table.timeline.HoodieActiveTimeline.transitionState(HoodieActiveTimeline.java:558)
 ~[hudi-aws-bundle-0.12.1.jar:0.12.1]
        at 
org.apache.hudi.common.table.timeline.HoodieActiveTimeline.transitionCompactionInflightToComplete(HoodieActiveTimeline.java:435)
 ~[hudi-aws-bundle-0.12.1.jar:0.12.1]
        at 
org.apache.hudi.table.action.compact.CompactHelpers.completeInflightCompaction(CompactHelpers.java:78)
 ~[hudi-spark3-bundle_2.12-0.12.1.jar:0.12.1]
        at 
org.apache.hudi.client.SparkRDDWriteClient.completeCompaction(SparkRDDWriteClient.java:316)
 ~[hudi-spark3-bundle_2.12-0.12.1.jar:0.12.1]
        at 
org.apache.hudi.client.SparkRDDWriteClient.commitCompaction(SparkRDDWriteClient.java:300)
 ~[hudi-spark3-bundle_2.12-0.12.1.jar:0.12.1]
        at 
org.apache.hudi.client.HoodieSparkCompactor.compact(HoodieSparkCompactor.java:62)
 ~[hudi-spark3-bundle_2.12-0.12.1.jar:0.12.1]
        at 
org.apache.hudi.async.AsyncCompactService.lambda$null$0(AsyncCompactService.java:84)
 ~[hudi-spark3-bundle_2.12-0.12.1.jar:0.12.1]
        at 
java.util.concurrent.CompletableFuture$AsyncSupply.run(CompletableFuture.java:1604)
 ~[?:1.8.0_412]
        at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) 
~[?:1.8.0_412]
        at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) 
~[?:1.8.0_412]
        at java.lang.Thread.run(Thread.java:750) ~[?:1.8.0_412]
   Caused by: org.apache.hadoop.fs.FileAlreadyExistsException: File already 
exists:s3://g24x7-eds-datalake/hudi_data_lake_prod10/prod_eds_datalake_ups_latest_v3/.hoodie/20240808160446342.commit
   
   


-- 
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]

Reply via email to