[HUDI-2562] Embedded timeline server on JobManager (#3812)
This commit is contained in:
@@ -196,7 +196,7 @@ public class HoodieAppendHandle<T extends HoodieRecordPayload, I, K, O> extends
|
||||
Option<Map<String, String>> recordMetadata = hoodieRecord.getData().getMetadata();
|
||||
try {
|
||||
// Pass the isUpdateRecord to the props for HoodieRecordPayload to judge
|
||||
// Whether it is a update or insert record.
|
||||
// Whether it is an update or insert record.
|
||||
boolean isUpdateRecord = isUpdateRecord(hoodieRecord);
|
||||
// If the format can not record the operation field, nullify the DELETE payload manually.
|
||||
boolean nullifyPayload = HoodieOperation.isDelete(hoodieRecord.getOperation()) && !config.allowOperationMetadataField();
|
||||
@@ -219,7 +219,7 @@ public class HoodieAppendHandle<T extends HoodieRecordPayload, I, K, O> extends
|
||||
if (config.allowOperationMetadataField()) {
|
||||
HoodieAvroUtils.addOperationToRecord(rewriteRecord, hoodieRecord.getOperation());
|
||||
}
|
||||
if (isUpdateRecord(hoodieRecord)) {
|
||||
if (isUpdateRecord) {
|
||||
updatedRecordsWritten++;
|
||||
} else {
|
||||
insertRecordsWritten++;
|
||||
|
||||
@@ -35,7 +35,6 @@ import org.apache.hudi.common.table.HoodieTableVersion;
|
||||
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
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;
|
||||
@@ -70,7 +69,6 @@ import org.slf4j.LoggerFactory;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.text.ParseException;
|
||||
import java.util.Comparator;
|
||||
import java.util.HashMap;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
@@ -506,45 +504,9 @@ public class HoodieFlinkWriteClient<T extends HoodieRecordPayload> extends
|
||||
} else {
|
||||
writeTimer = metrics.getDeltaCommitCtx();
|
||||
}
|
||||
table.getHoodieView().sync();
|
||||
return table;
|
||||
}
|
||||
|
||||
public String getLastPendingInstant(HoodieTableType tableType) {
|
||||
final String actionType = CommitUtils.getCommitActionType(tableType);
|
||||
return getLastPendingInstant(actionType);
|
||||
}
|
||||
|
||||
public String getLastPendingInstant(String actionType) {
|
||||
HoodieTimeline unCompletedTimeline = FlinkClientUtil.createMetaClient(basePath)
|
||||
.getCommitsTimeline().filterInflightsAndRequested();
|
||||
return unCompletedTimeline.getInstants()
|
||||
.filter(x -> x.getAction().equals(actionType) && x.isInflight())
|
||||
.map(HoodieInstant::getTimestamp)
|
||||
.collect(Collectors.toList()).stream()
|
||||
.max(Comparator.naturalOrder())
|
||||
.orElse(null);
|
||||
}
|
||||
|
||||
public String getLastCompletedInstant(HoodieTableType tableType) {
|
||||
final String commitType = CommitUtils.getCommitActionType(tableType);
|
||||
HoodieTimeline completedTimeline = FlinkClientUtil.createMetaClient(basePath)
|
||||
.getCommitsTimeline().filterCompletedInstants();
|
||||
return completedTimeline.getInstants()
|
||||
.filter(x -> x.getAction().equals(commitType))
|
||||
.map(HoodieInstant::getTimestamp)
|
||||
.collect(Collectors.toList()).stream()
|
||||
.max(Comparator.naturalOrder())
|
||||
.orElse(null);
|
||||
}
|
||||
|
||||
public void transitionRequestedToInflight(String commitType, String inFlightInstant) {
|
||||
HoodieActiveTimeline activeTimeline = FlinkClientUtil.createMetaClient(basePath).getActiveTimeline();
|
||||
HoodieInstant requested = new HoodieInstant(HoodieInstant.State.REQUESTED, commitType, inFlightInstant);
|
||||
activeTimeline.transitionRequestedToInflight(requested, Option.empty(),
|
||||
config.shouldAllowMultiWriteOnSameInstant());
|
||||
}
|
||||
|
||||
public HoodieFlinkTable<T> getHoodieTable() {
|
||||
return HoodieFlinkTable.create(config, (HoodieFlinkEngineContext) context);
|
||||
}
|
||||
|
||||
@@ -85,7 +85,7 @@ public class FlinkHoodieBackedTableMetadataWriter extends HoodieBackedTableMetad
|
||||
if (!metadataMetaClient.getActiveTimeline().filterCompletedInstants().containsInstant(instantTime)) {
|
||||
// if this is a new commit being applied to metadata for the first time
|
||||
writeClient.startCommitWithTime(instantTime);
|
||||
writeClient.transitionRequestedToInflight(HoodieActiveTimeline.DELTA_COMMIT_ACTION, instantTime);
|
||||
metadataMetaClient.getActiveTimeline().transitionRequestedToInflight(HoodieActiveTimeline.DELTA_COMMIT_ACTION, 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.
|
||||
|
||||
@@ -63,14 +63,28 @@ public abstract class HoodieFlinkTable<T extends HoodieRecordPayload>
|
||||
public static <T extends HoodieRecordPayload> HoodieFlinkTable<T> create(HoodieWriteConfig config,
|
||||
HoodieFlinkEngineContext context,
|
||||
HoodieTableMetaClient metaClient) {
|
||||
return HoodieFlinkTable.create(config, context, metaClient, config.isMetadataTableEnabled());
|
||||
}
|
||||
|
||||
public static <T extends HoodieRecordPayload> HoodieFlinkTable<T> create(HoodieWriteConfig config,
|
||||
HoodieFlinkEngineContext context,
|
||||
HoodieTableMetaClient metaClient,
|
||||
boolean refreshTimeline) {
|
||||
final HoodieFlinkTable<T> hoodieFlinkTable;
|
||||
switch (metaClient.getTableType()) {
|
||||
case COPY_ON_WRITE:
|
||||
return new HoodieFlinkCopyOnWriteTable<>(config, context, metaClient);
|
||||
hoodieFlinkTable = new HoodieFlinkCopyOnWriteTable<>(config, context, metaClient);
|
||||
break;
|
||||
case MERGE_ON_READ:
|
||||
return new HoodieFlinkMergeOnReadTable<>(config, context, metaClient);
|
||||
hoodieFlinkTable = new HoodieFlinkMergeOnReadTable<>(config, context, metaClient);
|
||||
break;
|
||||
default:
|
||||
throw new HoodieException("Unsupported table type :" + metaClient.getTableType());
|
||||
}
|
||||
if (refreshTimeline) {
|
||||
hoodieFlinkTable.getHoodieView().sync();
|
||||
}
|
||||
return hoodieFlinkTable;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
||||
@@ -101,7 +101,7 @@ public class TestFlinkHoodieBloomIndex extends HoodieFlinkClientTestHarness {
|
||||
public void testLoadInvolvedFiles(boolean rangePruning, boolean treeFiltering, boolean bucketizedChecking) throws Exception {
|
||||
HoodieWriteConfig config = makeConfig(rangePruning, treeFiltering, bucketizedChecking);
|
||||
FlinkHoodieBloomIndex index = new FlinkHoodieBloomIndex(config);
|
||||
HoodieTable hoodieTable = HoodieFlinkTable.create(config, context, metaClient);
|
||||
HoodieTable hoodieTable = HoodieFlinkTable.create(config, context, metaClient, false);
|
||||
HoodieFlinkWriteableTestTable testTable = HoodieFlinkWriteableTestTable.of(hoodieTable, SCHEMA);
|
||||
|
||||
// Create some partitions, and put some files
|
||||
|
||||
Reference in New Issue
Block a user