1
0

[HUDI-2684] Use DefaultHoodieRecordPayload when precombine field is specified specifically (#3922)

This commit is contained in:
Danny Chan
2021-11-04 16:23:36 +08:00
committed by GitHub
parent 8932c4f7b5
commit 689020f303
4 changed files with 100 additions and 0 deletions

View File

@@ -19,6 +19,7 @@
package org.apache.hudi.table;
import org.apache.hudi.common.model.DefaultHoodieRecordPayload;
import org.apache.hudi.common.model.EventTimeAvroPayload;
import org.apache.hudi.configuration.FlinkOptions;
import org.apache.hudi.configuration.OptionsResolver;
import org.apache.hudi.exception.HoodieValidationException;
@@ -148,6 +149,11 @@ public class HoodieTableFactory implements DynamicTableSourceFactory, DynamicTab
throw new HoodieValidationException("Field " + preCombineField + " does not exist in the table schema."
+ "Please check '" + FlinkOptions.PRECOMBINE_FIELD.key() + "' option.");
}
} else if (FlinkOptions.isDefaultValueDefined(conf, FlinkOptions.PAYLOAD_CLASS_NAME)) {
// if precombine field is specified but payload clazz is default,
// use DefaultHoodieRecordPayload to make sure the precombine field is always taken for
// comparing.
conf.setString(FlinkOptions.PAYLOAD_CLASS_NAME, EventTimeAvroPayload.class.getName());
}
}

View File

@@ -19,6 +19,7 @@
package org.apache.hudi.table;
import org.apache.hudi.common.model.DefaultHoodieRecordPayload;
import org.apache.hudi.common.model.EventTimeAvroPayload;
import org.apache.hudi.configuration.FlinkOptions;
import org.apache.hudi.exception.HoodieValidationException;
import org.apache.hudi.hive.MultiPartKeysValueExtractor;
@@ -120,6 +121,9 @@ public class TestHoodieTableFactory {
// the precombine field is overwritten
assertThat(tableSource.getConf().getString(FlinkOptions.PRECOMBINE_FIELD), is(FlinkOptions.NO_PRE_COMBINE));
assertThat(tableSink.getConf().getString(FlinkOptions.PRECOMBINE_FIELD), is(FlinkOptions.NO_PRE_COMBINE));
// precombine field not specified, use the default payload clazz
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()));
// given pk but miss the pre combine key with DefaultHoodieRecordPayload should throw
this.conf.setString(FlinkOptions.PAYLOAD_CLASS_NAME, DefaultHoodieRecordPayload.class.getName());
@@ -141,6 +145,11 @@ public class TestHoodieTableFactory {
assertDoesNotThrow(() -> new HoodieTableFactory().createDynamicTableSource(sourceContext5));
assertDoesNotThrow(() -> new HoodieTableFactory().createDynamicTableSink(sourceContext5));
// precombine field specified(default ts), use DefaultHoodieRecordPayload as payload clazz
HoodieTableSource tableSource5 = (HoodieTableSource) new HoodieTableFactory().createDynamicTableSource(sourceContext5);
HoodieTableSink tableSink5 = (HoodieTableSink) new HoodieTableFactory().createDynamicTableSink(sourceContext5);
assertThat(tableSource5.getConf().getString(FlinkOptions.PAYLOAD_CLASS_NAME), is(EventTimeAvroPayload.class.getName()));
assertThat(tableSink5.getConf().getString(FlinkOptions.PAYLOAD_CLASS_NAME), is(EventTimeAvroPayload.class.getName()));
}
@Test