diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/HoodieTableFactory.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/HoodieTableFactory.java index 83816044acb18..bbab47236a89c 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/HoodieTableFactory.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/HoodieTableFactory.java @@ -171,8 +171,8 @@ private void sanityCheck(Configuration conf, ResolvedSchema schema) { if (!OptionsResolver.isAppendMode(conf)) { checkRecordKey(conf, schema); - checkPreCombineKey(conf, schema); } + checkPreCombineKey(conf, schema); } /** diff --git a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/TestHoodieTableFactory.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/TestHoodieTableFactory.java index c6522cf32d136..d3a48ae63b7ad 100644 --- a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/TestHoodieTableFactory.java +++ b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/TestHoodieTableFactory.java @@ -109,6 +109,9 @@ void testRequiredOptions() { final MockContext sourceContext11 = MockContext.getInstance(this.conf, schema1, "f2"); assertDoesNotThrow(() -> new HoodieTableFactory().createDynamicTableSource(sourceContext11)); assertDoesNotThrow(() -> new HoodieTableFactory().createDynamicTableSink(sourceContext11)); + //miss the pre combine key will be ok + HoodieTableSink tableSink11 = (HoodieTableSink) new HoodieTableFactory().createDynamicTableSink(sourceContext11); + assertThat(tableSink11.getConf().getString(FlinkOptions.PRECOMBINE_FIELD), is(FlinkOptions.NO_PRE_COMBINE)); this.conf.set(FlinkOptions.OPERATION, FlinkOptions.OPERATION.defaultValue()); // a non-exists precombine key will throw exception @@ -140,6 +143,12 @@ void testRequiredOptions() { assertThat(tableSource.getConf().getString(FlinkOptions.PAYLOAD_CLASS_NAME), is(FlinkOptions.PAYLOAD_CLASS_NAME.defaultValue())); assertThat(tableSink.getConf().getString(FlinkOptions.PAYLOAD_CLASS_NAME), is(FlinkOptions.PAYLOAD_CLASS_NAME.defaultValue())); + // append mode given the pk but miss the pre combine key will be ok + this.conf.set(FlinkOptions.OPERATION, "insert"); + HoodieTableSink tableSink3 = (HoodieTableSink) new HoodieTableFactory().createDynamicTableSink(sourceContext3); + assertThat(tableSink3.getConf().getString(FlinkOptions.PRECOMBINE_FIELD), is(FlinkOptions.NO_PRE_COMBINE)); + this.conf.set(FlinkOptions.OPERATION, FlinkOptions.OPERATION.defaultValue()); + this.conf.setString(FlinkOptions.PAYLOAD_CLASS_NAME, DefaultHoodieRecordPayload.class.getName()); final MockContext sourceContext4 = MockContext.getInstance(this.conf, schema3, "f2");