1
0

[HUDI-1729] Asynchronous Hive sync and commits cleaning for Flink writer (#2732)

This commit is contained in:
Danny Chan
2021-03-29 10:47:29 +08:00
committed by GitHub
parent ecbd389a3f
commit d415d45416
23 changed files with 704 additions and 41 deletions

View File

@@ -20,6 +20,7 @@ package org.apache.hudi.client;
import org.apache.hudi.client.common.HoodieFlinkEngineContext;
import org.apache.hudi.client.embedded.EmbeddedTimelineService;
import org.apache.hudi.client.utils.TransactionUtils;
import org.apache.hudi.common.engine.HoodieEngineContext;
import org.apache.hudi.common.model.HoodieCommitMetadata;
import org.apache.hudi.common.model.HoodieKey;
@@ -38,6 +39,7 @@ import org.apache.hudi.common.util.CommitUtils;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieCommitException;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.hudi.exception.HoodieNotSupportedException;
import org.apache.hudi.index.FlinkHoodieIndex;
import org.apache.hudi.index.HoodieIndex;
@@ -49,6 +51,8 @@ import org.apache.hudi.io.MiniBatchHandle;
import org.apache.hudi.table.BulkInsertPartitioner;
import org.apache.hudi.table.HoodieFlinkTable;
import org.apache.hudi.table.HoodieTable;
import org.apache.hudi.table.HoodieTimelineArchiveLog;
import org.apache.hudi.table.MarkerFiles;
import org.apache.hudi.table.action.HoodieWriteMetadata;
import org.apache.hudi.table.action.compact.FlinkCompactHelpers;
import org.apache.hudi.table.upgrade.FlinkUpgradeDowngrade;
@@ -117,7 +121,7 @@ public class HoodieFlinkWriteClient<T extends HoodieRecordPayload> extends
@Override
public List<HoodieRecord<T>> filterExists(List<HoodieRecord<T>> hoodieRecords) {
// Create a Hoodie table which encapsulated the commits and files visible
HoodieFlinkTable<T> table = HoodieFlinkTable.create(config, (HoodieFlinkEngineContext) context);
HoodieFlinkTable<T> table = getHoodieTable();
Timer.Context indexTimer = metrics.getIndexCtx();
List<HoodieRecord<T>> recordsWithLocation = getIndex().tagLocation(hoodieRecords, context, table);
metrics.updateIndexMetrics(LOOKUP_STR, metrics.getDurationInMs(indexTimer == null ? 0L : indexTimer.stop()));
@@ -198,6 +202,47 @@ public class HoodieFlinkWriteClient<T extends HoodieRecordPayload> extends
return postWrite(result, instantTime, table);
}
@Override
protected void preWrite(String instantTime, WriteOperationType writeOperationType, HoodieTableMetaClient metaClient) {
setOperationType(writeOperationType);
this.lastCompletedTxnAndMetadata = TransactionUtils.getLastCompletedTxnInstantAndMetadata(metaClient);
this.txnManager.beginTransaction(Option.of(new HoodieInstant(HoodieInstant.State.INFLIGHT, metaClient.getCommitActionType(), instantTime)), lastCompletedTxnAndMetadata
.isPresent()
? Option.of(lastCompletedTxnAndMetadata.get().getLeft()) : Option.empty());
try {
syncTableMetadata();
} finally {
this.txnManager.endTransaction();
}
// remove the async cleaning
}
/**
* Starts async cleaning service for finished commits.
*
* <p>The Flink write client is designed to write data set as buckets
* but cleaning action should trigger after all the write actions within a
* checkpoint finish.
*/
public void startAsyncCleaning() {
this.asyncCleanerService = AsyncCleanerService.startAsyncCleaningIfEnabled(this);
}
/**
* Blocks and wait for the async cleaning service to finish.
*
* <p>The Flink write client is designed to write data set as buckets
* but cleaning action should trigger after all the write actions within a
* checkpoint finish.
*/
public void waitForCleaningFinish() {
if (this.asyncCleanerService != null) {
LOG.info("Cleaner has been spawned already. Waiting for it to finish");
AsyncCleanerService.waitForCompletion(asyncCleanerService);
LOG.info("Cleaner has finished");
}
}
@Override
protected List<WriteStatus> postWrite(HoodieWriteMetadata<List<WriteStatus>> result,
String instantTime,
@@ -208,12 +253,37 @@ public class HoodieFlinkWriteClient<T extends HoodieRecordPayload> extends
return result.getWriteStatuses();
}
/**
* Post commit is rewrite to be invoked after a successful commit.
*
* <p>The Flink write client is designed to write data set as buckets
* but cleaning action should trigger after all the write actions within a
* checkpoint finish.
*
* @param instantTime The latest successful commit time
*/
public void postCommit(String instantTime) {
try {
HoodieTable<?, ?, ?, ?> table = createTable(config, hadoopConf);
// Delete the marker directory for the instant.
new MarkerFiles(createTable(config, hadoopConf), instantTime)
.quietDeleteMarkerDir(context, config.getMarkersDeleteParallelism());
// We cannot have unbounded commit files. Archive commits if we have to archive
HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(config, table);
archiveLog.archiveIfRequired(context);
} catch (IOException ioe) {
throw new HoodieIOException(ioe.getMessage(), ioe);
} finally {
this.heartbeatClient.stop(instantTime);
}
}
@Override
public void commitCompaction(
String compactionInstantTime,
List<WriteStatus> writeStatuses,
Option<Map<String, String>> extraMetadata) throws IOException {
HoodieFlinkTable<T> table = HoodieFlinkTable.create(config, (HoodieFlinkEngineContext) context);
HoodieFlinkTable<T> table = getHoodieTable();
HoodieCommitMetadata metadata = FlinkCompactHelpers.newInstance().createCompactionMetadata(
table, compactionInstantTime, writeStatuses, config.getSchema());
extraMetadata.ifPresent(m -> m.forEach(metadata::addMetadata));
@@ -318,7 +388,7 @@ public class HoodieFlinkWriteClient<T extends HoodieRecordPayload> extends
setWriteSchemaForDeletes(metaClient);
}
// Create a Hoodie table which encapsulated the commits and files visible
HoodieFlinkTable<T> table = HoodieFlinkTable.create(config, (HoodieFlinkEngineContext) context, metaClient);
HoodieFlinkTable<T> table = getHoodieTable();
if (table.getMetaClient().getCommitActionType().equals(HoodieTimeline.COMMIT_ACTION)) {
writeTimer = metrics.getCommitCtx();
} else {
@@ -328,16 +398,14 @@ public class HoodieFlinkWriteClient<T extends HoodieRecordPayload> extends
}
public List<String> getInflightsAndRequestedInstants(String commitType) {
HoodieFlinkTable<T> table = HoodieFlinkTable.create(config, (HoodieFlinkEngineContext) context);
HoodieTimeline unCompletedTimeline = table.getMetaClient().getCommitsTimeline().filterInflightsAndRequested();
HoodieTimeline unCompletedTimeline = getHoodieTable().getMetaClient().getCommitsTimeline().filterInflightsAndRequested();
return unCompletedTimeline.getInstants().filter(x -> x.getAction().equals(commitType)).map(HoodieInstant::getTimestamp)
.collect(Collectors.toList());
}
public String getInflightAndRequestedInstant(String tableType) {
final String commitType = CommitUtils.getCommitActionType(HoodieTableType.valueOf(tableType));
HoodieFlinkTable<T> table = HoodieFlinkTable.create(config, (HoodieFlinkEngineContext) context);
HoodieTimeline unCompletedTimeline = table.getMetaClient().getCommitsTimeline().filterInflightsAndRequested();
HoodieTimeline unCompletedTimeline = getHoodieTable().getMetaClient().getCommitsTimeline().filterInflightsAndRequested();
return unCompletedTimeline.getInstants()
.filter(x -> x.getAction().equals(commitType))
.map(HoodieInstant::getTimestamp)
@@ -348,8 +416,7 @@ public class HoodieFlinkWriteClient<T extends HoodieRecordPayload> extends
public String getLastCompletedInstant(String tableType) {
final String commitType = CommitUtils.getCommitActionType(HoodieTableType.valueOf(tableType));
HoodieFlinkTable<T> table = HoodieFlinkTable.create(config, (HoodieFlinkEngineContext) context);
HoodieTimeline completedTimeline = table.getMetaClient().getCommitsTimeline().filterCompletedInstants();
HoodieTimeline completedTimeline = getHoodieTable().getMetaClient().getCommitsTimeline().filterCompletedInstants();
return completedTimeline.getInstants()
.filter(x -> x.getAction().equals(commitType))
.map(HoodieInstant::getTimestamp)
@@ -359,7 +426,7 @@ public class HoodieFlinkWriteClient<T extends HoodieRecordPayload> extends
}
public void deletePendingInstant(String tableType, String instant) {
HoodieFlinkTable<T> table = HoodieFlinkTable.create(config, (HoodieFlinkEngineContext) context);
HoodieFlinkTable<T> table = getHoodieTable();
String commitType = CommitUtils.getCommitActionType(HoodieTableType.valueOf(tableType));
HoodieActiveTimeline activeTimeline = table.getMetaClient().getActiveTimeline();
activeTimeline.deletePending(HoodieInstant.State.INFLIGHT, commitType, instant);
@@ -367,7 +434,7 @@ public class HoodieFlinkWriteClient<T extends HoodieRecordPayload> extends
}
public void transitionRequestedToInflight(String tableType, String inFlightInstant) {
HoodieFlinkTable<T> table = HoodieFlinkTable.create(config, (HoodieFlinkEngineContext) context);
HoodieFlinkTable<T> table = getHoodieTable();
HoodieActiveTimeline activeTimeline = table.getActiveTimeline();
String commitType = CommitUtils.getCommitActionType(HoodieTableType.valueOf(tableType));
HoodieInstant requested = new HoodieInstant(HoodieInstant.State.REQUESTED, commitType, inFlightInstant);
@@ -376,7 +443,7 @@ public class HoodieFlinkWriteClient<T extends HoodieRecordPayload> extends
}
public void rollbackInflightCompaction(HoodieInstant inflightInstant) {
HoodieFlinkTable<T> table = HoodieFlinkTable.create(config, (HoodieFlinkEngineContext) context);
HoodieFlinkTable<T> table = getHoodieTable();
HoodieTimeline pendingCompactionTimeline = table.getActiveTimeline().filterPendingCompactionTimeline();
if (pendingCompactionTimeline.containsInstant(inflightInstant)) {
rollbackInflightCompaction(inflightInstant, table);