1
0

[HUDI-2274] Allows INSERT duplicates for Flink MOR table (#3403)

This commit is contained in:
Danny Chan
2021-08-06 10:30:52 +08:00
committed by GitHub
parent 0dcd6a8fca
commit b7586a5632
9 changed files with 20 additions and 46 deletions

View File

@@ -209,11 +209,11 @@ public class FlinkOptions extends HoodieConfig {
.defaultValue(TABLE_TYPE_COPY_ON_WRITE)
.withDescription("Type of table to write. COPY_ON_WRITE (or) MERGE_ON_READ");
public static final ConfigOption<Boolean> INSERT_ALLOW_DUP = ConfigOptions
.key("write.insert.allow_dup")
public static final ConfigOption<Boolean> INSERT_DEDUP = ConfigOptions
.key("write.insert.deduplicate")
.booleanType()
.defaultValue(true)
.withDescription("Whether to allow data duplication for INSERT operation, if enabled, writes the base files directly, default true");
.withDescription("Whether to deduplicate for INSERT operation, if disabled, writes the base files directly, default true");
public static final ConfigOption<String> OPERATION = ConfigOptions
.key("write.operation")

View File

@@ -69,8 +69,8 @@ public class FlinkStreamerConfig extends Configuration {
@Parameter(names = {"--table-type"}, description = "Type of table. COPY_ON_WRITE (or) MERGE_ON_READ.", required = true)
public String tableType;
@Parameter(names = {"--insert-allow-dup"}, description = "Whether to allow data duplication for INSERT operation, if enabled, writes the base files directly.", required = true)
public Boolean insertAllowDup = true;
@Parameter(names = {"--insert-dedup"}, description = "Whether to deduplicate for INSERT operation, if disabled, writes the base files directly.", required = true)
public Boolean insertDedup = true;
@Parameter(names = {"--props"}, description = "Path to properties file on localfs or dfs, with configurations for "
+ "hoodie client, schema provider, key generator and data source. For hoodie client props, sane defaults are "
@@ -308,7 +308,7 @@ public class FlinkStreamerConfig extends Configuration {
conf.setString(FlinkOptions.TABLE_NAME, config.targetTableName);
// copy_on_write works same as COPY_ON_WRITE
conf.setString(FlinkOptions.TABLE_TYPE, config.tableType.toUpperCase());
conf.setBoolean(FlinkOptions.INSERT_ALLOW_DUP, config.insertAllowDup);
conf.setBoolean(FlinkOptions.INSERT_DEDUP, config.insertDedup);
conf.setString(FlinkOptions.OPERATION, config.operation.value());
conf.setString(FlinkOptions.PRECOMBINE_FIELD, config.sourceOrderingField);
conf.setString(FlinkOptions.PAYLOAD_CLASS, config.payloadClassName);

View File

@@ -23,6 +23,7 @@ import org.apache.hudi.hive.MultiPartKeysValueExtractor;
import org.apache.hudi.keygen.ComplexAvroKeyGenerator;
import org.apache.hudi.keygen.NonpartitionedAvroKeyGenerator;
import org.apache.hudi.util.AvroSchemaConverter;
import org.apache.hudi.util.StreamerUtil;
import org.apache.flink.configuration.ConfigOption;
import org.apache.flink.configuration.Configuration;
@@ -129,11 +130,6 @@ public class HoodieTableFactory implements DynamicTableSourceFactory, DynamicTab
throw new ValidationException("Field " + preCombineField + " does not exist in the table schema."
+ "Please check 'write.precombine.field' option.");
}
if (conf.getString(FlinkOptions.TABLE_TYPE).toUpperCase().equals(FlinkOptions.TABLE_TYPE_MERGE_ON_READ)
&& conf.getBoolean(FlinkOptions.INSERT_ALLOW_DUP)) {
throw new ValidationException("Option 'write.insert.allow_dup' is only allowed for COPY_ON_WRITE table.");
}
}
/**
@@ -214,6 +210,11 @@ public class HoodieTableFactory implements DynamicTableSourceFactory, DynamicTab
// if compaction schedule is on, tweak the target io to 500GB
conf.setLong(FlinkOptions.COMPACTION_TARGET_IO, 500 * 1024L);
}
if (StreamerUtil.allowDuplicateInserts(conf)) {
// no need for compaction if insert duplicates is allowed
conf.setBoolean(FlinkOptions.COMPACTION_ASYNC_ENABLED, false);
conf.setBoolean(FlinkOptions.COMPACTION_SCHEDULE_ENABLED, false);
}
}
/**

View File

@@ -350,6 +350,6 @@ public class StreamerUtil {
public static boolean allowDuplicateInserts(Configuration conf) {
WriteOperationType operationType = WriteOperationType.fromValue(conf.getString(FlinkOptions.OPERATION));
return operationType == WriteOperationType.INSERT && conf.getBoolean(FlinkOptions.INSERT_ALLOW_DUP);
return operationType == WriteOperationType.INSERT && !conf.getBoolean(FlinkOptions.INSERT_DEDUP);
}
}