[HUDI-2633] Make precombine field optional for flink (#3874)
This commit is contained in:
@@ -394,6 +394,29 @@ public class HoodieDataSourceITCase extends AbstractTestBase {
|
||||
+ "+I[id8, Han, 56, 1970-01-01T00:00:08, par4]]");
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@MethodSource("tableTypeAndPartitioningParams")
|
||||
void testWriteAndReadWithProctimeSequence(HoodieTableType tableType, boolean hiveStylePartitioning) {
|
||||
TableEnvironment tableEnv = batchTableEnv;
|
||||
String hoodieTableDDL = sql("t1")
|
||||
.field("uuid varchar(20)")
|
||||
.field("name varchar(10)")
|
||||
.field("age int")
|
||||
.field("tss timestamp(3)") // use a different field with default precombine field 'ts'
|
||||
.field("`partition` varchar(10)")
|
||||
.option(FlinkOptions.PATH, tempFile.getAbsolutePath())
|
||||
.option(FlinkOptions.TABLE_TYPE, tableType)
|
||||
.option(FlinkOptions.HIVE_STYLE_PARTITIONING, hiveStylePartitioning)
|
||||
.end();
|
||||
tableEnv.executeSql(hoodieTableDDL);
|
||||
|
||||
execInsertSql(tableEnv, TestSQL.INSERT_SAME_KEY_T1);
|
||||
|
||||
List<Row> result1 = CollectionUtil.iterableToList(
|
||||
() -> tableEnv.sqlQuery("select * from t1").execute().collect());
|
||||
assertRowsEquals(result1, "[+I[id1, Danny, 23, 1970-01-01T00:00:01, par1]]");
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@EnumSource(value = HoodieTableType.class)
|
||||
void testBatchModeUpsertWithoutPartition(HoodieTableType tableType) {
|
||||
|
||||
@@ -18,7 +18,9 @@
|
||||
|
||||
package org.apache.hudi.table;
|
||||
|
||||
import org.apache.hudi.common.model.DefaultHoodieRecordPayload;
|
||||
import org.apache.hudi.configuration.FlinkOptions;
|
||||
import org.apache.hudi.exception.HoodieValidationException;
|
||||
import org.apache.hudi.hive.MultiPartKeysValueExtractor;
|
||||
import org.apache.hudi.hive.SlashEncodedDayPartitionValueExtractor;
|
||||
import org.apache.hudi.keygen.ComplexAvroKeyGenerator;
|
||||
@@ -32,7 +34,6 @@ import org.apache.flink.configuration.Configuration;
|
||||
import org.apache.flink.configuration.ReadableConfig;
|
||||
import org.apache.flink.table.api.DataTypes;
|
||||
import org.apache.flink.table.api.Schema;
|
||||
import org.apache.flink.table.api.ValidationException;
|
||||
import org.apache.flink.table.catalog.CatalogTable;
|
||||
import org.apache.flink.table.catalog.ObjectIdentifier;
|
||||
import org.apache.flink.table.catalog.ResolvedCatalogTable;
|
||||
@@ -84,39 +85,62 @@ public class TestHoodieTableFactory {
|
||||
|
||||
@Test
|
||||
void testRequiredOptionsForSource() {
|
||||
// miss pk and pre combine key will throw exception
|
||||
// miss pk and precombine key will throw exception
|
||||
ResolvedSchema schema1 = SchemaBuilder.instance()
|
||||
.field("f0", DataTypes.INT().notNull())
|
||||
.field("f1", DataTypes.VARCHAR(20))
|
||||
.field("f2", DataTypes.TIMESTAMP(3))
|
||||
.build();
|
||||
final MockContext sourceContext1 = MockContext.getInstance(this.conf, schema1, "f2");
|
||||
assertThrows(ValidationException.class, () -> new HoodieTableFactory().createDynamicTableSource(sourceContext1));
|
||||
assertThrows(ValidationException.class, () -> new HoodieTableFactory().createDynamicTableSink(sourceContext1));
|
||||
assertThrows(HoodieValidationException.class, () -> new HoodieTableFactory().createDynamicTableSource(sourceContext1));
|
||||
assertThrows(HoodieValidationException.class, () -> new HoodieTableFactory().createDynamicTableSink(sourceContext1));
|
||||
|
||||
// given the pk and miss the pre combine key will throw exception
|
||||
// a non-exists precombine key will throw exception
|
||||
ResolvedSchema schema2 = SchemaBuilder.instance()
|
||||
.field("f0", DataTypes.INT().notNull())
|
||||
.field("f1", DataTypes.VARCHAR(20))
|
||||
.field("f2", DataTypes.TIMESTAMP(3))
|
||||
.build();
|
||||
this.conf.setString(FlinkOptions.PRECOMBINE_FIELD, "non_exist_field");
|
||||
final MockContext sourceContext2 = MockContext.getInstance(this.conf, schema2, "f2");
|
||||
assertThrows(HoodieValidationException.class, () -> new HoodieTableFactory().createDynamicTableSource(sourceContext2));
|
||||
assertThrows(HoodieValidationException.class, () -> new HoodieTableFactory().createDynamicTableSink(sourceContext2));
|
||||
this.conf.setString(FlinkOptions.PRECOMBINE_FIELD, FlinkOptions.PRECOMBINE_FIELD.defaultValue());
|
||||
|
||||
// given the pk but miss the pre combine key will be ok
|
||||
ResolvedSchema schema3 = SchemaBuilder.instance()
|
||||
.field("f0", DataTypes.INT().notNull())
|
||||
.field("f1", DataTypes.VARCHAR(20))
|
||||
.field("f2", DataTypes.TIMESTAMP(3))
|
||||
.primaryKey("f0")
|
||||
.build();
|
||||
final MockContext sourceContext2 = MockContext.getInstance(this.conf, schema2, "f2");
|
||||
assertThrows(ValidationException.class, () -> new HoodieTableFactory().createDynamicTableSource(sourceContext2));
|
||||
assertThrows(ValidationException.class, () -> new HoodieTableFactory().createDynamicTableSink(sourceContext2));
|
||||
final MockContext sourceContext3 = MockContext.getInstance(this.conf, schema3, "f2");
|
||||
HoodieTableSource tableSource = (HoodieTableSource) new HoodieTableFactory().createDynamicTableSource(sourceContext3);
|
||||
HoodieTableSink tableSink = (HoodieTableSink) new HoodieTableFactory().createDynamicTableSink(sourceContext3);
|
||||
// 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));
|
||||
|
||||
// given pk but miss the pre combine key with DefaultHoodieRecordPayload should throw
|
||||
this.conf.setString(FlinkOptions.PAYLOAD_CLASS_NAME, DefaultHoodieRecordPayload.class.getName());
|
||||
final MockContext sourceContext4 = MockContext.getInstance(this.conf, schema3, "f2");
|
||||
|
||||
assertThrows(HoodieValidationException.class, () -> new HoodieTableFactory().createDynamicTableSource(sourceContext4));
|
||||
assertThrows(HoodieValidationException.class, () -> new HoodieTableFactory().createDynamicTableSink(sourceContext4));
|
||||
this.conf.setString(FlinkOptions.PAYLOAD_CLASS_NAME, FlinkOptions.PAYLOAD_CLASS_NAME.defaultValue());
|
||||
|
||||
// given pk and pre combine key will be ok
|
||||
ResolvedSchema schema3 = SchemaBuilder.instance()
|
||||
ResolvedSchema schema4 = SchemaBuilder.instance()
|
||||
.field("f0", DataTypes.INT().notNull())
|
||||
.field("f1", DataTypes.VARCHAR(20))
|
||||
.field("f2", DataTypes.TIMESTAMP(3))
|
||||
.field("ts", DataTypes.TIMESTAMP(3))
|
||||
.primaryKey("f0")
|
||||
.build();
|
||||
final MockContext sourceContext3 = MockContext.getInstance(this.conf, schema3, "f2");
|
||||
final MockContext sourceContext5 = MockContext.getInstance(this.conf, schema4, "f2");
|
||||
|
||||
assertDoesNotThrow(() -> new HoodieTableFactory().createDynamicTableSource(sourceContext3));
|
||||
assertDoesNotThrow(() -> new HoodieTableFactory().createDynamicTableSink(sourceContext3));
|
||||
assertDoesNotThrow(() -> new HoodieTableFactory().createDynamicTableSource(sourceContext5));
|
||||
assertDoesNotThrow(() -> new HoodieTableFactory().createDynamicTableSink(sourceContext5));
|
||||
}
|
||||
|
||||
@Test
|
||||
|
||||
@@ -35,6 +35,13 @@ public class TestSQL {
|
||||
+ "('id7','Bob',44,TIMESTAMP '1970-01-01 00:00:07','par4'),\n"
|
||||
+ "('id8','Han',56,TIMESTAMP '1970-01-01 00:00:08','par4')";
|
||||
|
||||
public static final String INSERT_SAME_KEY_T1 = "insert into t1 values\n"
|
||||
+ "('id1','Danny',23,TIMESTAMP '1970-01-01 00:00:05','par1'),\n"
|
||||
+ "('id1','Danny',23,TIMESTAMP '1970-01-01 00:00:04','par1'),\n"
|
||||
+ "('id1','Danny',23,TIMESTAMP '1970-01-01 00:00:03','par1'),\n"
|
||||
+ "('id1','Danny',23,TIMESTAMP '1970-01-01 00:00:02','par1'),\n"
|
||||
+ "('id1','Danny',23,TIMESTAMP '1970-01-01 00:00:01','par1')";
|
||||
|
||||
public static final String UPDATE_INSERT_T1 = "insert into t1 values\n"
|
||||
+ "('id1','Danny',24,TIMESTAMP '1970-01-01 00:00:01','par1'),\n"
|
||||
+ "('id2','Stephen',34,TIMESTAMP '1970-01-01 00:00:02','par1'),\n"
|
||||
|
||||
Reference in New Issue
Block a user