1
0

[HUDI-1788] Insert overwrite (table) for Flink writer (#2808)

Supports `INSERT OVERWRITE` and `INSERT OVERWRITE TABLE` for Flink
writer.
This commit is contained in:
Danny Chan
2021-04-14 10:23:37 +08:00
committed by GitHub
parent 65844a8d29
commit ab4a7b0b4a
23 changed files with 523 additions and 93 deletions

View File

@@ -29,10 +29,7 @@ import org.apache.hudi.common.config.TypedProperties;
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.WriteOperationType;
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.common.util.ReflectionUtils;
import org.apache.hudi.common.util.StringUtils;
@@ -193,14 +190,6 @@ public class DataSourceUtils {
return new SparkRDDWriteClient<>(new HoodieSparkEngineContext(jssc), createHoodieConfig(schemaStr, basePath, tblName, parameters));
}
public static String getCommitActionType(WriteOperationType operation, HoodieTableType tableType) {
if (operation == WriteOperationType.INSERT_OVERWRITE || operation == WriteOperationType.INSERT_OVERWRITE_TABLE) {
return HoodieTimeline.REPLACE_COMMIT_ACTION;
} else {
return CommitUtils.getCommitActionType(tableType);
}
}
public static HoodieWriteResult doWriteOperation(SparkRDDWriteClient client, JavaRDD<HoodieRecord> hoodieRecords,
String instantTime, WriteOperationType operation) throws HoodieException {
switch (operation) {

View File

@@ -18,7 +18,6 @@
package org.apache.hudi.internal;
import org.apache.hudi.DataSourceUtils;
import org.apache.hudi.client.SparkRDDWriteClient;
import org.apache.hudi.client.common.HoodieSparkEngineContext;
import org.apache.hudi.common.model.HoodieWriteStat;
@@ -27,6 +26,7 @@ import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.table.timeline.HoodieInstant.State;
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.HoodieException;
@@ -79,7 +79,7 @@ public class DataSourceInternalWriterHelper {
public void commit(List<HoodieWriteStat> writeStatList) {
try {
writeClient.commitStats(instantTime, writeStatList, Option.of(new HashMap<>()),
DataSourceUtils.getCommitActionType(operationType, metaClient.getTableType()));
CommitUtils.getCommitActionType(operationType, metaClient.getTableType()));
} catch (Exception ioe) {
throw new HoodieException(ioe.getMessage(), ioe);
} finally {