You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hudi.apache.org by GitBox <gi...@apache.org> on 2022/04/22 09:41:18 UTC

[GitHub] [hudi] todd5167 opened a new issue, #5395: [SUPPORT] Failed to archive commits, thow error 'Directory is not empty'

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

   hudi version:  0.10.0-patch 
   fs: aws s3
   log:
   ```
   2022-04-22 17:31:51
   org.apache.flink.util.FlinkException: Global failure triggered by OperatorCoordinator for 'hoodie_stream_write' (operator ef1a0eded9d886851d0ae5a7b94ad01d).
   	at org.apache.flink.runtime.operators.coordination.OperatorCoordinatorHolder$LazyInitializedCoordinatorContext.failJob(OperatorCoordinatorHolder.java:553)
   	at org.apache.hudi.sink.StreamWriteOperatorCoordinator.lambda$start$0(StreamWriteOperatorCoordinator.java:170)
   	at org.apache.hudi.sink.utils.NonThrownExecutor.lambda$execute$0(NonThrownExecutor.java:103)
   	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
   	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
   	at java.lang.Thread.run(Thread.java:748)
   Caused by: org.apache.hudi.exception.HoodieException: Executor executes action [commits the instant 20220422093115515] error
   	... 5 more
   Caused by: org.apache.hudi.exception.HoodieCommitException: Failed to archive commits
   	at org.apache.hudi.table.HoodieTimelineArchiveLog.archive(HoodieTimelineArchiveLog.java:334)
   	at org.apache.hudi.table.HoodieTimelineArchiveLog.archiveIfRequired(HoodieTimelineArchiveLog.java:130)
   	at org.apache.hudi.client.AbstractHoodieWriteClient.archive(AbstractHoodieWriteClient.java:765)
   	at org.apache.hudi.client.HoodieFlinkWriteClient.postCommit(HoodieFlinkWriteClient.java:339)
   	at org.apache.hudi.client.AbstractHoodieWriteClient.commitStats(AbstractHoodieWriteClient.java:198)
   	at org.apache.hudi.client.HoodieFlinkWriteClient.commit(HoodieFlinkWriteClient.java:112)
   	at org.apache.hudi.sink.StreamWriteOperatorCoordinator.doCommit(StreamWriteOperatorCoordinator.java:496)
   	at org.apache.hudi.sink.StreamWriteOperatorCoordinator.commitInstant(StreamWriteOperatorCoordinator.java:472)
   	at org.apache.hudi.sink.StreamWriteOperatorCoordinator.lambda$notifyCheckpointComplete$2(StreamWriteOperatorCoordinator.java:225)
   	at org.apache.hudi.sink.utils.NonThrownExecutor.lambda$execute$0(NonThrownExecutor.java:93)
   	... 3 more
   Caused by: org.apache.hudi.exception.HoodieIOException: `s3a://sprs-qz-dw-prod-ue1/qz_ods/ac_account_transfer_record/.hoodie/.temp/20220421063410412': Directory is not empty
   	at org.apache.hudi.common.fs.FSUtils.deleteDir(FSUtils.java:655)
   	at org.apache.hudi.table.marker.DirectWriteMarkers.deleteMarkerDir(DirectWriteMarkers.java:75)
   	at org.apache.hudi.table.HoodieTimelineArchiveLog.deleteAnyLeftOverMarkers(HoodieTimelineArchiveLog.java:340)
   	at org.apache.hudi.table.HoodieTimelineArchiveLog.archive(HoodieTimelineArchiveLog.java:320)
   	... 12 more
   Caused by: org.apache.hadoop.fs.PathIsNotEmptyDirectoryException: `s3a://sprs-qz-dw-prod-ue1/qz_ods/ac_account_transfer_record/.hoodie/.temp/20220421063410412': Directory is not empty
   	at org.apache.hadoop.fs.s3a.S3AFileSystem.innerDelete(S3AFileSystem.java:1752)
   	at org.apache.hadoop.fs.s3a.S3AFileSystem.delete(S3AFileSystem.java:1697)
   	at org.apache.hudi.common.fs.HoodieWrapperFileSystem.lambda$delete$14(HoodieWrapperFileSystem.java:340)
   	at org.apache.hudi.common.fs.HoodieWrapperFileSystem.executeFuncWithTimeMetrics(HoodieWrapperFileSystem.java:100)
   	at org.apache.hudi.common.fs.HoodieWrapperFileSystem.delete(HoodieWrapperFileSystem.java:339)
   	at org.apache.hudi.common.fs.FSUtils.deleteDir(FSUtils.java:650)
   	... 15 more
   
   ```


-- 
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: commits-unsubscribe@hudi.apache.org.apache.org

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


[GitHub] [hudi] yihua commented on issue #5395: [SUPPORT] Failed to archive commits, thow error 'Directory is not empty'

Posted by GitBox <gi...@apache.org>.
yihua commented on issue #5395:
URL: https://github.com/apache/hudi/issues/5395#issuecomment-1110326566

   All the subdirs are deleted in parallel before executing L686 above.  It is likely that there are concurrent operations on the same instant, e.g., one trying to delete the `.temp/<instant_time>` while the other adding new markers to `.temp/<instant_time>`.  @todd5167 Once you provide more information on the timeline as @codope suggested, we can dig deeper into the root cause.


-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] todd5167 commented on issue #5395: [SUPPORT] Failed to archive commits, thow error 'Directory is not empty'

Posted by GitBox <gi...@apache.org>.
todd5167 commented on issue #5395:
URL: https://github.com/apache/hudi/issues/5395#issuecomment-1112815298

   > So you do not write into same table from two separate Flink jobs.
   
   yes


-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] danny0405 commented on issue #5395: [SUPPORT] Failed to archive commits, thow error 'Directory is not empty'

Posted by GitBox <gi...@apache.org>.
danny0405 commented on issue #5395:
URL: https://github.com/apache/hudi/issues/5395#issuecomment-1111652198

   > deleteAnyLeftOverMarkers
   
   I'm not quite sure, we actually clean the MARKER folder each time we commit an instant, and here we do it again when the instant was archived? (curious about the background here ?), and like what @codope saied, the call in [#L682-L684](https://github.com/apache/hudi/blob/762623a15cfeba6f3fe936c238d660685ae62b50/hudi-common/src/main/java/org/apache/hudi/common/fs/FSUtils.java#L682-L684) already delete recursively first.
   
   The instant commit and archiving both happens in JobManager in single thread, so there should not be parallelism problem.
   
   @todd5167 Did you write to the same table from two separate Flink jobs ?


-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] nsivabalan commented on issue #5395: [SUPPORT] Failed to archive commits, thow error 'Directory is not empty'

Posted by GitBox <gi...@apache.org>.
nsivabalan commented on issue #5395:
URL: https://github.com/apache/hudi/issues/5395#issuecomment-1126596850

   @todd5167 : can you respond to Danny's question above. and if you got it working, can you let us know what was the root cause and how did you get around it.


-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] todd5167 commented on issue #5395: [SUPPORT] Failed to archive commits, thow error 'Directory is not empty'

Posted by GitBox <gi...@apache.org>.
todd5167 commented on issue #5395:
URL: https://github.com/apache/hudi/issues/5395#issuecomment-1110621179

   @yihua   Because this problem causes the flink job to restart frequently, I use fs.delete(dirPath, true) to delete the folder.After I changed the code again, the ./hoodie folder was cleaned up by me. I re-consume historical data and incremental data again.


-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] codope commented on issue #5395: [SUPPORT] Failed to archive commits, thow error 'Directory is not empty'

Posted by GitBox <gi...@apache.org>.
codope commented on issue #5395:
URL: https://github.com/apache/hudi/issues/5395#issuecomment-1109807235

   @todd5167 Did you retry? From stacktrace it looks like recursive delete failed at some point. Can you please share hudi write configs and data timeline (in `.hoodie` directory)? Also, what was the state of `.temp/20220421063410412` directory at the time? If it's still intact can you please send the `ls -lrt .temp/20220421063410412` ?


-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] codope commented on issue #5395: [SUPPORT] Failed to archive commits, thow error 'Directory is not empty'

Posted by GitBox <gi...@apache.org>.
codope commented on issue #5395:
URL: https://github.com/apache/hudi/issues/5395#issuecomment-1109809687

   Looks like we don't allow recursive delete in some cases https://github.com/apache/hudi/blob/762623a15cfeba6f3fe936c238d660685ae62b50/hudi-common/src/main/java/org/apache/hudi/common/fs/FSUtils.java#L686
   
   @yihua do you know why that's the case?


-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] danny0405 commented on issue #5395: [SUPPORT] Failed to archive commits, thow error 'Directory is not empty'

Posted by GitBox <gi...@apache.org>.
danny0405 commented on issue #5395:
URL: https://github.com/apache/hudi/issues/5395#issuecomment-1111847010

   So you do not write into same table from two separate Flink jobs.


-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] yihua commented on issue #5395: [SUPPORT] Failed to archive commits, thow error 'Directory is not empty'

Posted by GitBox <gi...@apache.org>.
yihua commented on issue #5395:
URL: https://github.com/apache/hudi/issues/5395#issuecomment-1111356586

   @todd5167 Got it.  If you can also provide the steps in your setup to reproduce the problem, we can try to reproduce the problem on our end as well to identify the root cause.  @danny0405 are you aware of a similar issue of deleting marker directory in Flink?


-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] todd5167 commented on issue #5395: [SUPPORT] Failed to archive commits, thow error 'Directory is not empty'

Posted by GitBox <gi...@apache.org>.
todd5167 commented on issue #5395:
URL: https://github.com/apache/hudi/issues/5395#issuecomment-1127500228

   @nsivabalan  @danny0405    I deleted the ./hoodie folder and re-executed it because I needed to restore the task.Therefore, I can't see the Action information.


-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] yihua commented on issue #5395: [SUPPORT] Failed to archive commits, thow error 'Directory is not empty'

Posted by GitBox <gi...@apache.org>.
yihua commented on issue #5395:
URL: https://github.com/apache/hudi/issues/5395#issuecomment-1169704897

   @todd5167 I'm going to close this ticket as you have got past it.  Feel free to reopen the issue if you see it again.


-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] yihua commented on issue #5395: [SUPPORT] Failed to archive commits, thow error 'Directory is not empty'

Posted by GitBox <gi...@apache.org>.
yihua commented on issue #5395:
URL: https://github.com/apache/hudi/issues/5395#issuecomment-1110600966

   @todd5167 the point is that even before `fs.delete(dirPath, false)` is called, the subfolders should have been deleted already.  So I'm wondering why the subfolders are not deleted in the first place.  If you're not comfortable sharing the whole `.hoodie` folder, could you simply list the `.hoodie` folder with `aws s3 ls <table_path>/.hoodie`?  Or you can take a screenshot of the files in `.hoodie` folder in S3 Web UI.  I'm interested in the timeline of the table, and the type of instance `20220421063410412`, whether it is deltacommit, compaction, etc.
   
   > I have restored from flink savepoint multiple times and keep getting this error. 
   
   I guess you mean you tried to restart the flink job but kept hitting the same exception.  I assume you're not talking about Hudi restore or savepoint?


-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] todd5167 commented on issue #5395: [SUPPORT] Failed to archive commits, thow error 'Directory is not empty'

Posted by GitBox <gi...@apache.org>.
todd5167 commented on issue #5395:
URL: https://github.com/apache/hudi/issues/5395#issuecomment-1111689002

   > > deleteAnyLeftOverMarkers
   > 
   > I'm not quite sure, we actually clean the MARKER folder each time we commit an instant, and here we do it again when the instant was archived? (curious about the background here ?), and like what @codope saied, the call in [#L682-L684](https://github.com/apache/hudi/blob/762623a15cfeba6f3fe936c238d660685ae62b50/hudi-common/src/main/java/org/apache/hudi/common/fs/FSUtils.java#L682-L684) already delete recursively first.
   > 
   > The instant commit and archiving both happens in JobManager in single thread, so there should not be parallelism problem.
   > 
   > @todd5167 Did you write to the same table from two separate Flink jobs ?
   
   @danny0405  A flink job corresponds to a hudi table.   checkpoint config:
   ```
   Interval	10m 0s
   Timeout	 3m 0s
   Minimum Pause Between Checkpoints	0ms
   Maximum Concurrent Checkpoints	2
   ```
   
   
   


-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] yihua closed issue #5395: [SUPPORT] Failed to archive commits, thow error 'Directory is not empty'

Posted by GitBox <gi...@apache.org>.
yihua closed issue #5395: [SUPPORT]  Failed to archive commits,  thow  error 'Directory is not empty'
URL: https://github.com/apache/hudi/issues/5395


-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] todd5167 commented on issue #5395: [SUPPORT] Failed to archive commits, thow error 'Directory is not empty'

Posted by GitBox <gi...@apache.org>.
todd5167 commented on issue #5395:
URL: https://github.com/apache/hudi/issues/5395#issuecomment-1110469761

   @yihua @codope  Flink writes hudi configuration information as follows. I have restored from flink savepoint multiple times and keep getting this error.  When the exception occurs .temp/20220421063410412 has subfolders.    My temporary solution is, boolean result = fs.delete(dirPath, **true**); not sure what's wrong with this.
   ```
     'connector' = 'hudi',
     'path' = 's3a://xxxxxxx/hudi/ac_withdraw_record',
     'table.type' = 'MERGE_ON_READ',
     'write.bucket_assign.tasks' = '2',
     'write.tasks' = '2',
     'changelog.enabled' = 'false',
     'hoodie.cleaner.policy' = 'KEEP_LATEST_FILE_VERSIONS' ,
     'hoodie.cleaner.fileversions.retained' = '2' ,
     'write.task.max.size' = '2048',
     'write.index_bootstrap.tasks' = '4',
     'index.bootstrap.enabled' = 'true',
     'index.global.enabled' = 'false',
     'compaction.tasks' = '4',
     'compaction.max_memory' = '1024',
     'hoodie.compact.inline.trigger.strategy' = 'NUM_COMMITS',
     'hive_sync.enable' = 'true',
     'hive_sync.mode' = 'glue',
     'hive_sync.table' = 'ac_withdraw_record',
     'hive_sync.skip_ro_suffix' = 'true',
     'hive_sync.db' = 'hudi_meta', 
     'hoodie.compact.inline.max.delta.commits' = '6' 
   ```


-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] danny0405 commented on issue #5395: [SUPPORT] Failed to archive commits, thow error 'Directory is not empty'

Posted by GitBox <gi...@apache.org>.
danny0405 commented on issue #5395:
URL: https://github.com/apache/hudi/issues/5395#issuecomment-1112906982

   @todd5167 What kind of instant of time `20220421063410412`, is it a delta commit or commit or cleaning ?


-- 
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: commits-unsubscribe@hudi.apache.org

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