[
https://issues.apache.org/jira/browse/HUDI-5408?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
sivabalan narayanan updated HUDI-5408:
--------------------------------------
Description:
when compaction failed after completing in MDT but before completing in DT. and
later when we re-attempt to apply the same compaction instant to MDT, we might
miss to rollback any partially failed commit in MDT.
Code of interest in SparkHoodieBackedTableMetadataWriter:
{code:java}
if (!metadataMetaClient.getActiveTimeline().containsInstant(instantTime)) {
// if this is a new commit being applied to metadata for the first time
writeClient.startCommitWithTime(instantTime);
} else {
Option<HoodieInstant> alreadyCompletedInstant =
metadataMetaClient.getActiveTimeline().filterCompletedInstants().filter(entry
-> entry.getTimestamp().equals(instantTime)).lastInstant();
if (alreadyCompletedInstant.isPresent()) {
// 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.
HoodieActiveTimeline.deleteInstantFile(metadataMetaClient.getFs(),
metadataMetaClient.getMetaPath(), alreadyCompletedInstant.get());
metadataMetaClient.reloadActiveTimeline();
}
// If the alreadyCompletedInstant is empty, that means there is a requested
or inflight
// instant with the same instant time. This happens for data table clean
action which
// reuses the same instant time without rollback first. It is a no-op here
as the
// clean plan is the same, so we don't need to delete the requested and
inflight instant
// files in the active timeline.
} {code}
incase of else block, if there happen to be a partially failed commit in MDT,
we may miss to roll it back.
we might need to fix the flow.
Imp to consider:
even before attempting compaction, we should ensure there are no partially
failed commits in MDT. If not, we need to ensure we consider list of valid
instants while executing the compaction.
was:
when compaction failed after completing in MDT but before completing in DT. and
later when we re-attempt to apply the same compaction instant to MDT, we might
miss to rollback any partially failed commit in MDT.
Code of interest in SparkHoodieBackedTableMetadataWriter:
{code:java}
if (!metadataMetaClient.getActiveTimeline().containsInstant(instantTime)) {
// if this is a new commit being applied to metadata for the first time
writeClient.startCommitWithTime(instantTime);
} else {
Option<HoodieInstant> alreadyCompletedInstant =
metadataMetaClient.getActiveTimeline().filterCompletedInstants().filter(entry
-> entry.getTimestamp().equals(instantTime)).lastInstant();
if (alreadyCompletedInstant.isPresent()) {
// 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.
HoodieActiveTimeline.deleteInstantFile(metadataMetaClient.getFs(),
metadataMetaClient.getMetaPath(), alreadyCompletedInstant.get());
metadataMetaClient.reloadActiveTimeline();
}
// If the alreadyCompletedInstant is empty, that means there is a requested
or inflight
// instant with the same instant time. This happens for data table clean
action which
// reuses the same instant time without rollback first. It is a no-op here
as the
// clean plan is the same, so we don't need to delete the requested and
inflight instant
// files in the active timeline.
} {code}
incase of else block, if there happen to be a partially failed commit in MDT,
we may miss to roll it back.
we might need to fix the flow.
> Partially failed commits in MDT is not rolledback in all cases
> --------------------------------------------------------------
>
> Key: HUDI-5408
> URL: https://issues.apache.org/jira/browse/HUDI-5408
> Project: Apache Hudi
> Issue Type: Bug
> Components: metadata
> Reporter: sivabalan narayanan
> Assignee: sivabalan narayanan
> Priority: Critical
> Labels: pull-request-available
> Fix For: 0.13.0
>
>
> when compaction failed after completing in MDT but before completing in DT.
> and later when we re-attempt to apply the same compaction instant to MDT, we
> might miss to rollback any partially failed commit in MDT.
> Code of interest in SparkHoodieBackedTableMetadataWriter:
> {code:java}
> if (!metadataMetaClient.getActiveTimeline().containsInstant(instantTime)) {
> // if this is a new commit being applied to metadata for the first time
> writeClient.startCommitWithTime(instantTime);
> } else {
> Option<HoodieInstant> alreadyCompletedInstant =
> metadataMetaClient.getActiveTimeline().filterCompletedInstants().filter(entry
> -> entry.getTimestamp().equals(instantTime)).lastInstant();
> if (alreadyCompletedInstant.isPresent()) {
> // 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.
> HoodieActiveTimeline.deleteInstantFile(metadataMetaClient.getFs(),
> metadataMetaClient.getMetaPath(), alreadyCompletedInstant.get());
> metadataMetaClient.reloadActiveTimeline();
> }
> // If the alreadyCompletedInstant is empty, that means there is a requested
> or inflight
> // instant with the same instant time. This happens for data table clean
> action which
> // reuses the same instant time without rollback first. It is a no-op here
> as the
> // clean plan is the same, so we don't need to delete the requested and
> inflight instant
> // files in the active timeline.
> } {code}
> incase of else block, if there happen to be a partially failed commit in MDT,
> we may miss to roll it back.
> we might need to fix the flow.
>
> Imp to consider:
> even before attempting compaction, we should ensure there are no partially
> failed commits in MDT. If not, we need to ensure we consider list of valid
> instants while executing the compaction.
>
>
--
This message was sent by Atlassian Jira
(v8.20.10#820010)