1
0

[HUDI-1522] Add a new pipeline for Flink writer (#2430)

* [HUDI-1522] Add a new pipeline for Flink writer
This commit is contained in:
Danny Chan
2021-01-28 08:53:13 +08:00
committed by GitHub
parent 7b2e658ac0
commit bc0325f6ea
40 changed files with 3613 additions and 302 deletions

View File

@@ -25,12 +25,14 @@ import org.apache.hudi.common.model.HoodieCommitMetadata;
import org.apache.hudi.common.model.HoodieKey;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieRecordPayload;
import org.apache.hudi.common.model.HoodieTableType;
import org.apache.hudi.common.model.HoodieWriteStat;
import org.apache.hudi.common.model.WriteOperationType;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.HoodieTableVersion;
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.HoodieNotSupportedException;
@@ -46,6 +48,7 @@ import com.codahale.metrics.Timer;
import org.apache.hadoop.conf.Configuration;
import java.io.IOException;
import java.util.Comparator;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
@@ -218,4 +221,35 @@ public class HoodieFlinkWriteClient<T extends HoodieRecordPayload> extends
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();
return unCompletedTimeline.getInstants()
.filter(x -> x.getAction().equals(commitType))
.map(HoodieInstant::getTimestamp)
.collect(Collectors.toList()).stream()
.max(Comparator.naturalOrder())
.orElse(null);
}
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();
return completedTimeline.getInstants()
.filter(x -> x.getAction().equals(commitType))
.map(HoodieInstant::getTimestamp)
.collect(Collectors.toList()).stream()
.max(Comparator.naturalOrder())
.orElse(null);
}
public void deletePendingInstant(String tableType, String instant) {
HoodieFlinkTable<T> table = HoodieFlinkTable.create(config, (HoodieFlinkEngineContext) context);
String commitType = CommitUtils.getCommitActionType(HoodieTableType.valueOf(tableType));
table.getMetaClient().getActiveTimeline()
.deletePending(new HoodieInstant(HoodieInstant.State.REQUESTED, commitType, instant));
}
}

View File

@@ -54,9 +54,7 @@ public class FlinkInMemoryStateIndex<T extends HoodieRecordPayload> extends Flin
if (context.getRuntimeContext() != null) {
MapStateDescriptor<HoodieKey, HoodieRecordLocation> indexStateDesc =
new MapStateDescriptor<>("indexState", TypeInformation.of(HoodieKey.class), TypeInformation.of(HoodieRecordLocation.class));
if (context.getRuntimeContext() != null) {
mapState = context.getRuntimeContext().getMapState(indexStateDesc);
}
mapState = context.getRuntimeContext().getMapState(indexStateDesc);
}
}