1
0

[HUDI-2925] Fix duplicate cleaning of same files when unfinished clean operations are present using a config. (#4212)

Co-authored-by: sivabalan <n.siva.b@gmail.com>
This commit is contained in:
Prashant Wason
2022-02-21 18:53:03 -08:00
committed by GitHub
parent 0c950181aa
commit 0dee8edc97
7 changed files with 118 additions and 15 deletions

View File

@@ -767,21 +767,28 @@ public abstract class BaseHoodieWriteClient<T extends HoodieRecordPayload, I, K,
if (!tableServicesEnabled(config)) {
return null;
}
if (scheduleInline) {
scheduleTableServiceInternal(cleanInstantTime, Option.empty(), TableServiceType.CLEAN);
}
LOG.info("Cleaner started");
final Timer.Context timerContext = metrics.getCleanCtx();
LOG.info("Cleaned failed attempts if any");
CleanerUtils.rollbackFailedWrites(config.getFailedWritesCleanPolicy(),
HoodieTimeline.CLEAN_ACTION, () -> rollbackFailedWrites(skipLocking));
HoodieCleanMetadata metadata = createTable(config, hadoopConf).clean(context, cleanInstantTime, skipLocking);
if (timerContext != null && metadata != null) {
long durationMs = metrics.getDurationInMs(timerContext.stop());
metrics.updateCleanMetrics(durationMs, metadata.getTotalFilesDeleted());
LOG.info("Cleaned " + metadata.getTotalFilesDeleted() + " files"
+ " Earliest Retained Instant :" + metadata.getEarliestCommitToRetain()
+ " cleanerElapsedMs" + durationMs);
HoodieCleanMetadata metadata = null;
HoodieTable table = createTable(config, hadoopConf);
if (config.allowMultipleCleans() || !table.getActiveTimeline().getCleanerTimeline().filterInflightsAndRequested().firstInstant().isPresent()) {
LOG.info("Cleaner started");
// proceed only if multiple clean schedules are enabled or if there are no pending cleans.
if (scheduleInline) {
scheduleTableServiceInternal(cleanInstantTime, Option.empty(), TableServiceType.CLEAN);
table.getMetaClient().reloadActiveTimeline();
}
metadata = table.clean(context, cleanInstantTime, skipLocking);
if (timerContext != null && metadata != null) {
long durationMs = metrics.getDurationInMs(timerContext.stop());
metrics.updateCleanMetrics(durationMs, metadata.getTotalFilesDeleted());
LOG.info("Cleaned " + metadata.getTotalFilesDeleted() + " files"
+ " Earliest Retained Instant :" + metadata.getEarliestCommitToRetain()
+ " cleanerElapsedMs" + durationMs);
}
}
return metadata;
}

View File

@@ -278,6 +278,13 @@ public class HoodieCompactionConfig extends HoodieConfig {
.withDocumentation("The average record size. If not explicitly specified, hudi will compute the "
+ "record size estimate compute dynamically based on commit metadata. "
+ " This is critical in computing the insert parallelism and bin-packing inserts into small files.");
public static final ConfigProperty<Boolean> ALLOW_MULTIPLE_CLEANS = ConfigProperty
.key("hoodie.clean.allow.multiple")
.defaultValue(true)
.sinceVersion("0.11.0")
.withDocumentation("Allows scheduling/executing multiple cleans by enabling this config. If users prefer to strictly ensure clean requests should be mutually exclusive, "
+ ".i.e. a 2nd clean will not be scheduled if another clean is not yet completed to avoid repeat cleaning of same files, they might want to disable this config.");
public static final ConfigProperty<Integer> ARCHIVE_MERGE_FILES_BATCH_SIZE = ConfigProperty
.key("hoodie.archive.merge.files.batch.size")
@@ -642,6 +649,11 @@ public class HoodieCompactionConfig extends HoodieConfig {
return this;
}
public Builder allowMultipleCleans(boolean allowMultipleCleanSchedules) {
compactionConfig.setValue(ALLOW_MULTIPLE_CLEANS, String.valueOf(allowMultipleCleanSchedules));
return this;
}
public Builder withCleanerParallelism(int cleanerParallelism) {
compactionConfig.setValue(CLEANER_PARALLELISM_VALUE, String.valueOf(cleanerParallelism));
return this;

View File

@@ -1117,6 +1117,10 @@ public class HoodieWriteConfig extends HoodieConfig {
return getInt(HoodieCompactionConfig.COPY_ON_WRITE_RECORD_SIZE_ESTIMATE);
}
public boolean allowMultipleCleans() {
return getBoolean(HoodieCompactionConfig.ALLOW_MULTIPLE_CLEANS);
}
public boolean shouldAutoTuneInsertSplits() {
return getBoolean(HoodieCompactionConfig.COPY_ON_WRITE_AUTO_SPLIT_INSERTS);
}

View File

@@ -65,7 +65,7 @@ public abstract class BaseActionExecutor<T extends HoodieRecordPayload, I, K, O,
* Writes clean metadata to table metadata.
* @param metadata clean metadata of interest.
*/
protected final void writeTableMetadata(HoodieCleanMetadata metadata) {
protected final void writeTableMetadata(HoodieCleanMetadata metadata, String instantTime) {
table.getMetadataWriter(instantTime).ifPresent(w -> w.update(metadata, instantTime));
}

View File

@@ -208,7 +208,7 @@ public class CleanActionExecutor<T extends HoodieRecordPayload, I, K, O> extends
if (!skipLocking) {
this.txnManager.beginTransaction(Option.empty(), Option.empty());
}
writeTableMetadata(metadata);
writeTableMetadata(metadata, inflightInstant.getTimestamp());
table.getActiveTimeline().transitionCleanInflightToComplete(inflightInstant,
TimelineMetadataUtils.serializeCleanMetadata(metadata));
LOG.info("Marked clean started on " + inflightInstant.getTimestamp() + " as complete");
@@ -240,9 +240,13 @@ public class CleanActionExecutor<T extends HoodieRecordPayload, I, K, O> extends
LOG.warn("Failed to perform previous clean operation, instant: " + hoodieInstant, e);
}
}
table.getMetaClient().reloadActiveTimeline();
if (config.isMetadataTableEnabled()) {
table.getHoodieView().sync();
}
});
table.getMetaClient().reloadActiveTimeline();
}
// return the last clean metadata for now
// TODO (NA) : Clean only the earliest pending clean just like how we do for other table services
// This requires the CleanActionExecutor to be refactored as BaseCommitActionExecutor