[
https://issues.apache.org/jira/browse/HUDI-3624?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
ASF GitHub Bot updated HUDI-3624:
---------------------------------
Labels: pull-request-available (was: )
> 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
> Labels: pull-request-available
> 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)