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

@@ -104,4 +104,8 @@ public enum WriteOperationType {
public static boolean isChangingRecords(WriteOperationType operationType) {
return operationType == UPSERT || operationType == UPSERT_PREPPED || operationType == DELETE;
}
public static boolean isOverwrite(WriteOperationType operationType) {
return operationType == INSERT_OVERWRITE || operationType == INSERT_OVERWRITE_TABLE;
}
}

View File

@@ -40,6 +40,17 @@ public class CommitUtils {
private static final Logger LOG = LogManager.getLogger(CommitUtils.class);
/**
* Gets the commit action type for given write operation and table type.
*/
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 getCommitActionType(tableType);
}
}
/**
* Gets the commit action type for given table type.
*/