[HUDI-2633] Make precombine field optional for flink (#3874)
This commit is contained in:
@@ -241,6 +241,7 @@ public class FlinkOptions extends HoodieConfig {
|
||||
.defaultValue("upsert")
|
||||
.withDescription("The write operation, that this write should do");
|
||||
|
||||
public static final String NO_PRE_COMBINE = "no_precombine";
|
||||
public static final ConfigOption<String> PRECOMBINE_FIELD = ConfigOptions
|
||||
.key("write.precombine.field")
|
||||
.stringType()
|
||||
|
||||
@@ -18,6 +18,7 @@
|
||||
|
||||
package org.apache.hudi.configuration;
|
||||
|
||||
import org.apache.hudi.common.model.DefaultHoodieRecordPayload;
|
||||
import org.apache.hudi.common.model.WriteOperationType;
|
||||
|
||||
import org.apache.flink.configuration.Configuration;
|
||||
@@ -67,4 +68,20 @@ public class OptionsResolver {
|
||||
.toUpperCase(Locale.ROOT)
|
||||
.equals(FlinkOptions.TABLE_TYPE_COPY_ON_WRITE);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns whether the payload clazz is {@link DefaultHoodieRecordPayload}.
|
||||
*/
|
||||
public static boolean isDefaultHoodieRecordPayloadClazz(Configuration conf) {
|
||||
return conf.getString(FlinkOptions.PAYLOAD_CLASS_NAME).contains(DefaultHoodieRecordPayload.class.getSimpleName());
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the preCombine field
|
||||
* or null if the value is set as {@link FlinkOptions#NO_PRE_COMBINE}.
|
||||
*/
|
||||
public static String getPreCombineField(Configuration conf) {
|
||||
final String preCombineField = conf.getString(FlinkOptions.PRECOMBINE_FIELD);
|
||||
return preCombineField.equals(FlinkOptions.NO_PRE_COMBINE) ? null : preCombineField;
|
||||
}
|
||||
}
|
||||
|
||||
@@ -238,8 +238,10 @@ public class StreamWriteOperatorCoordinator
|
||||
public void notifyCheckpointAborted(long checkpointId) {
|
||||
// once the checkpoint was aborted, unblock the writer tasks to
|
||||
// reuse the last instant.
|
||||
executor.execute(() -> sendCommitAckEvents(checkpointId),
|
||||
"unblock data write with aborted checkpoint %s", checkpointId);
|
||||
if (!WriteMetadataEvent.BOOTSTRAP_INSTANT.equals(this.instant)) {
|
||||
executor.execute(() -> sendCommitAckEvents(checkpointId),
|
||||
"unblock data write with aborted checkpoint %s", checkpointId);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
|
||||
@@ -26,6 +26,7 @@ import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.ReflectionUtils;
|
||||
import org.apache.hudi.common.util.ValidationUtils;
|
||||
import org.apache.hudi.configuration.FlinkOptions;
|
||||
import org.apache.hudi.configuration.OptionsResolver;
|
||||
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.flink.configuration.Configuration;
|
||||
@@ -55,13 +56,14 @@ public class PayloadCreation implements Serializable {
|
||||
}
|
||||
|
||||
public static PayloadCreation instance(Configuration conf) throws Exception {
|
||||
boolean shouldCombine = conf.getBoolean(FlinkOptions.PRE_COMBINE)
|
||||
String preCombineField = OptionsResolver.getPreCombineField(conf);
|
||||
boolean needCombine = conf.getBoolean(FlinkOptions.PRE_COMBINE)
|
||||
|| WriteOperationType.fromValue(conf.getString(FlinkOptions.OPERATION)) == WriteOperationType.UPSERT;
|
||||
String preCombineField = null;
|
||||
boolean shouldCombine = needCombine && preCombineField != null;
|
||||
|
||||
final Class<?>[] argTypes;
|
||||
final Constructor<?> constructor;
|
||||
if (shouldCombine) {
|
||||
preCombineField = conf.getString(FlinkOptions.PRECOMBINE_FIELD);
|
||||
argTypes = new Class<?>[] {GenericRecord.class, Comparable.class};
|
||||
} else {
|
||||
argTypes = new Class<?>[] {Option.class};
|
||||
|
||||
@@ -18,6 +18,7 @@
|
||||
|
||||
package org.apache.hudi.table;
|
||||
|
||||
import org.apache.hudi.common.model.DefaultHoodieRecordPayload;
|
||||
import org.apache.hudi.configuration.FlinkOptions;
|
||||
import org.apache.hudi.configuration.OptionsResolver;
|
||||
import org.apache.hudi.exception.HoodieValidationException;
|
||||
@@ -117,20 +118,36 @@ public class HoodieTableFactory implements DynamicTableSourceFactory, DynamicTab
|
||||
|
||||
// validate record key in pk absence.
|
||||
if (!schema.getPrimaryKey().isPresent()) {
|
||||
Arrays.stream(conf.get(FlinkOptions.RECORD_KEY_FIELD).split(","))
|
||||
String[] recordKeys = conf.get(FlinkOptions.RECORD_KEY_FIELD).split(",");
|
||||
if (recordKeys.length == 1
|
||||
&& FlinkOptions.RECORD_KEY_FIELD.defaultValue().equals(recordKeys[0])
|
||||
&& !fields.contains(recordKeys[0])) {
|
||||
throw new HoodieValidationException("Primary key definition is required, use either PRIMARY KEY syntax "
|
||||
+ "or option '" + FlinkOptions.RECORD_KEY_FIELD.key() + "' to specify.");
|
||||
}
|
||||
|
||||
Arrays.stream(recordKeys)
|
||||
.filter(field -> !fields.contains(field))
|
||||
.findAny()
|
||||
.ifPresent(f -> {
|
||||
throw new ValidationException("Field '" + f + "' does not exist in the table schema."
|
||||
+ "Please define primary key or modify 'hoodie.datasource.write.recordkey.field' option.");
|
||||
throw new HoodieValidationException("Field '" + f + "' specified in option "
|
||||
+ "'" + FlinkOptions.RECORD_KEY_FIELD.key() + "' does not exist in the table schema.");
|
||||
});
|
||||
}
|
||||
|
||||
// validate pre_combine key
|
||||
String preCombineField = conf.get(FlinkOptions.PRECOMBINE_FIELD);
|
||||
if (!fields.contains(preCombineField)) {
|
||||
throw new ValidationException("Field " + preCombineField + " does not exist in the table schema."
|
||||
+ "Please check 'write.precombine.field' option.");
|
||||
if (OptionsResolver.isDefaultHoodieRecordPayloadClazz(conf)) {
|
||||
throw new HoodieValidationException("Option '" + FlinkOptions.PRECOMBINE_FIELD.key()
|
||||
+ "' is required for payload class: " + DefaultHoodieRecordPayload.class.getName());
|
||||
}
|
||||
if (preCombineField.equals(FlinkOptions.PRECOMBINE_FIELD.defaultValue())) {
|
||||
conf.setString(FlinkOptions.PRECOMBINE_FIELD, FlinkOptions.NO_PRE_COMBINE);
|
||||
} else {
|
||||
throw new HoodieValidationException("Field " + preCombineField + " does not exist in the table schema."
|
||||
+ "Please check '" + FlinkOptions.PRECOMBINE_FIELD.key() + "' option.");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
Reference in New Issue
Block a user