[ 
https://issues.apache.org/jira/browse/HUDI-3624?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17507236#comment-17507236
 ] 

Ethan Guo commented on HUDI-3624:
---------------------------------

Another instance of this failure with the stacktrace:
{code:java}
org.apache.hudi.exception.HoodieIOException: Failed to create file 
file:/Users/ethan/Work/scripts/mt_rollout_testing/deploy_c_multi_writer/c3_cow_010mt_011mt/test_table/.hoodie/metadata/.hoodie/20220314105017883.deltacommit.requested
    at 
org.apache.hudi.common.table.timeline.HoodieActiveTimeline.createImmutableFileInPath(HoodieActiveTimeline.java:673)
    at 
org.apache.hudi.common.table.timeline.HoodieActiveTimeline.createFileInMetaPath(HoodieActiveTimeline.java:655)
    at 
org.apache.hudi.common.table.timeline.HoodieActiveTimeline.createNewInstant(HoodieActiveTimeline.java:163)
    at 
org.apache.hudi.client.BaseHoodieWriteClient.startCommit(BaseHoodieWriteClient.java:894)
    at 
org.apache.hudi.client.BaseHoodieWriteClient.startCommitWithTime(BaseHoodieWriteClient.java:876)
    at 
org.apache.hudi.client.BaseHoodieWriteClient.startCommitWithTime(BaseHoodieWriteClient.java:859)
    at 
org.apache.hudi.metadata.SparkHoodieBackedTableMetadataWriter.commit(SparkHoodieBackedTableMetadataWriter.java:141)
    at 
org.apache.hudi.metadata.HoodieBackedTableMetadataWriter.processAndCommit(HoodieBackedTableMetadataWriter.java:670)
    at 
org.apache.hudi.metadata.HoodieBackedTableMetadataWriter.update(HoodieBackedTableMetadataWriter.java:694)
    at 
org.apache.hudi.table.action.BaseActionExecutor.lambda$writeTableMetadata$1(BaseActionExecutor.java:69)
    at org.apache.hudi.common.util.Option.ifPresent(Option.java:96)
    at 
org.apache.hudi.table.action.BaseActionExecutor.writeTableMetadata(BaseActionExecutor.java:69)
    at 
org.apache.hudi.table.action.clean.CleanActionExecutor.runClean(CleanActionExecutor.java:211)
    at 
org.apache.hudi.table.action.clean.CleanActionExecutor.runPendingClean(CleanActionExecutor.java:176)
    at 
org.apache.hudi.table.action.clean.CleanActionExecutor.lambda$execute$6(CleanActionExecutor.java:238)
    at java.util.ArrayList.forEach(ArrayList.java:1259)
    at 
org.apache.hudi.table.action.clean.CleanActionExecutor.execute(CleanActionExecutor.java:232)
    at 
org.apache.hudi.table.HoodieSparkCopyOnWriteTable.clean(HoodieSparkCopyOnWriteTable.java:339)
    at 
org.apache.hudi.client.BaseHoodieWriteClient.clean(BaseHoodieWriteClient.java:780)
    at 
org.apache.hudi.client.BaseHoodieWriteClient.clean(BaseHoodieWriteClient.java:737)
    at 
org.apache.hudi.async.AsyncCleanerService.lambda$startService$0(AsyncCleanerService.java:55)
    at 
java.util.concurrent.CompletableFuture$AsyncSupply.run(CompletableFuture.java:1604)
    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.hadoop.fs.FileAlreadyExistsException: File already 
exists: 
file:/Users/ethan/Work/scripts/mt_rollout_testing/deploy_c_multi_writer/c3_cow_010mt_011mt/test_table/.hoodie/metadata/.hoodie/20220314105017883.deltacommit.requested
    at 
org.apache.hadoop.fs.RawLocalFileSystem.create(RawLocalFileSystem.java:421)
    at 
org.apache.hadoop.fs.RawLocalFileSystem.create(RawLocalFileSystem.java:459)
    at 
org.apache.hadoop.fs.ChecksumFileSystem$ChecksumFSOutputSummer.<init>(ChecksumFileSystem.java:433)
    at 
org.apache.hadoop.fs.ChecksumFileSystem.create(ChecksumFileSystem.java:521)
    at 
org.apache.hadoop.fs.ChecksumFileSystem.create(ChecksumFileSystem.java:500)
    at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:1195)
    at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:1175)
    at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:1064)
    at 
org.apache.hudi.common.fs.HoodieWrapperFileSystem.lambda$create$2(HoodieWrapperFileSystem.java:222)
    at 
org.apache.hudi.common.fs.HoodieWrapperFileSystem.executeFuncWithTimeMetrics(HoodieWrapperFileSystem.java:101)
    at 
org.apache.hudi.common.fs.HoodieWrapperFileSystem.create(HoodieWrapperFileSystem.java:221)
    at 
org.apache.hudi.common.table.timeline.HoodieActiveTimeline.createImmutableFileInPath(HoodieActiveTimeline.java:668)
    ... 24 more {code}

> Re-attempt of failed cleaning from DT to MDT fails 
> ---------------------------------------------------
>
>                 Key: HUDI-3624
>                 URL: https://issues.apache.org/jira/browse/HUDI-3624
>             Project: Apache Hudi
>          Issue Type: Bug
>          Components: metadata
>            Reporter: sivabalan narayanan
>            Assignee: Ethan Guo
>            Priority: Blocker
>             Fix For: 0.11.0
>
>
> C5.clean.requested in DT 
> C5.clean.inflight in DT
> and then we try to apply this to MDT. 
> C5.deltacommit.requested and C5.deltacommit.inflight
> and crashed. 
>  
> If pipeline is restarted, cleaner will just go ahead retry the pending clean. 
> It will not trigger rollback like compaction or clustering. So, This fails in 
> MDT, since we only check for completed operation and avoid calling 
> writeClient.startCommit(newInstantTime). 
>  
> SparkHoodieMetadataTableWriter 
> {code:java}
> if 
> (!metadataMetaClient.getActiveTimeline().filterCompletedInstants().containsInstant(instantTime))
>  {
>   // if this is a new commit being applied to metadata for the first time
>   writeClient.startCommitWithTime(instantTime);
> } else {
>   // this code path refers to a re-attempted commit that got committed to 
> metadata table, but failed in datatable.
>   // for eg, lets say compaction c1 on 1st attempt succeeded in metadata 
> table and failed before committing to datatable.
>   // when retried again, data table will first rollback pending compaction. 
> these will be applied to metadata table, but all changes
>   // are upserts to metadata table and so only a new delta commit will be 
> created.
>   // once rollback is complete, compaction will be retried again, which will 
> eventually hit this code block where the respective commit is
>   // already part of completed commit. So, we have to manually remove the 
> completed instant and proceed.
>   // and it is for the same reason we enabled 
> withAllowMultiWriteOnSameInstant for metadata table.
>   HoodieInstant alreadyCompletedInstant = 
> metadataMetaClient.getActiveTimeline().filterCompletedInstants().filter(entry 
> -> entry.getTimestamp().equals(instantTime)).lastInstant().get();
>   HoodieActiveTimeline.deleteInstantFile(metadataMetaClient.getFs(), 
> metadataMetaClient.getMetaPath(), alreadyCompletedInstant);
>   metadataMetaClient.reloadActiveTimeline();
> }
> List<WriteStatus> statuses = 
> writeClient.upsertPreppedRecords(preppedRecordRDD, instantTime).collect();
>  {code}
> In above code snippet, the if condition should be fixed to check for any 
> instant in timeline and not just completed. 
> And within else block, if there is a completed instant, we should delete. if 
> not, its a no-op. 
>  
>  
>  



--
This message was sent by Atlassian Jira
(v8.20.1#820001)

Reply via email to