1
0

[HUDI-1704] Use PRIMARY KEY syntax to define record keys for Flink Hudi table (#2694)

The SQL PRIMARY KEY semantics is very same with Hoodie record key, using
PRIMARY KEY is more straight-forward way instead of a table option:
hoodie.datasource.write.recordkey.field.

After this change, both PRIMARY KEY and table option can define hoodie
record key, while the PRIMARY KEY has higher priority if both are
defined.

Note: a column with PRIMARY KEY constraint is forced to be non-nullable.
This commit is contained in:
Danny Chan
2021-03-18 20:21:52 +08:00
committed by GitHub
parent 968488fa3a
commit f1e0018f12
4 changed files with 168 additions and 20 deletions

View File

@@ -19,11 +19,14 @@
package org.apache.hudi.table;
import org.apache.hudi.configuration.FlinkOptions;
import org.apache.hudi.keygen.ComplexAvroKeyGenerator;
import org.apache.hudi.util.StreamerUtil;
import org.apache.hudi.utils.TestConfigurations;
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.TableSchema;
import org.apache.flink.table.catalog.CatalogTable;
import org.apache.flink.table.catalog.CatalogTableImpl;
import org.apache.flink.table.catalog.ObjectIdentifier;
@@ -36,6 +39,7 @@ import org.junit.jupiter.api.io.TempDir;
import java.io.File;
import java.io.IOException;
import java.util.Collections;
import java.util.List;
import java.util.Objects;
import static org.hamcrest.CoreMatchers.is;
@@ -86,6 +90,38 @@ public class TestHoodieTableFactory {
assertNull(conf2.get(FlinkOptions.READ_AVRO_SCHEMA), "expect schema string as null");
}
@Test
void testSetupHoodieKeyOptionsForSource() {
this.conf.setString(FlinkOptions.RECORD_KEY_FIELD, "dummyField");
this.conf.setString(FlinkOptions.KEYGEN_CLASS, "dummyKeyGenClass");
// definition with simple primary key and partition path
TableSchema schema1 = TableSchema.builder()
.field("f0", DataTypes.INT().notNull())
.field("f1", DataTypes.VARCHAR(20))
.field("f2", DataTypes.TIMESTAMP(3))
.primaryKey("f0")
.build();
final MockSourceContext sourceContext1 = MockSourceContext.getInstance(this.conf, schema1, "f2");
final HoodieTableSource tableSource1 = (HoodieTableSource) new HoodieTableFactory().createTableSource(sourceContext1);
final Configuration conf1 = tableSource1.getConf();
assertThat(conf1.get(FlinkOptions.RECORD_KEY_FIELD), is("f0"));
assertThat(conf1.get(FlinkOptions.KEYGEN_CLASS), is("dummyKeyGenClass"));
// definition with complex primary keys and partition paths
this.conf.setString(FlinkOptions.KEYGEN_CLASS, FlinkOptions.KEYGEN_CLASS.defaultValue());
TableSchema schema2 = TableSchema.builder()
.field("f0", DataTypes.INT().notNull())
.field("f1", DataTypes.VARCHAR(20).notNull())
.field("f2", DataTypes.TIMESTAMP(3))
.primaryKey("f0", "f1")
.build();
final MockSourceContext sourceContext2 = MockSourceContext.getInstance(this.conf, schema2, "f2");
final HoodieTableSource tableSource2 = (HoodieTableSource) new HoodieTableFactory().createTableSource(sourceContext2);
final Configuration conf2 = tableSource2.getConf();
assertThat(conf2.get(FlinkOptions.RECORD_KEY_FIELD), is("f0,f1"));
assertThat(conf2.get(FlinkOptions.KEYGEN_CLASS), is(ComplexAvroKeyGenerator.class.getName()));
}
@Test
void testInferAvroSchemaForSink() {
// infer the schema if not specified
@@ -102,6 +138,38 @@ public class TestHoodieTableFactory {
assertNull(conf2.get(FlinkOptions.READ_AVRO_SCHEMA), "expect schema string as null");
}
@Test
void testSetupHoodieKeyOptionsForSink() {
this.conf.setString(FlinkOptions.RECORD_KEY_FIELD, "dummyField");
this.conf.setString(FlinkOptions.KEYGEN_CLASS, "dummyKeyGenClass");
// definition with simple primary key and partition path
TableSchema schema1 = TableSchema.builder()
.field("f0", DataTypes.INT().notNull())
.field("f1", DataTypes.VARCHAR(20))
.field("f2", DataTypes.TIMESTAMP(3))
.primaryKey("f0")
.build();
final MockSinkContext sinkContext1 = MockSinkContext.getInstance(this.conf, schema1, "f2");
final HoodieTableSink tableSink1 = (HoodieTableSink) new HoodieTableFactory().createTableSink(sinkContext1);
final Configuration conf1 = tableSink1.getConf();
assertThat(conf1.get(FlinkOptions.RECORD_KEY_FIELD), is("f0"));
assertThat(conf1.get(FlinkOptions.KEYGEN_CLASS), is("dummyKeyGenClass"));
// definition with complex primary keys and partition paths
this.conf.setString(FlinkOptions.KEYGEN_CLASS, FlinkOptions.KEYGEN_CLASS.defaultValue());
TableSchema schema2 = TableSchema.builder()
.field("f0", DataTypes.INT().notNull())
.field("f1", DataTypes.VARCHAR(20).notNull())
.field("f2", DataTypes.TIMESTAMP(3))
.primaryKey("f0", "f1")
.build();
final MockSinkContext sinkContext2 = MockSinkContext.getInstance(this.conf, schema2, "f2");
final HoodieTableSink tableSink2 = (HoodieTableSink) new HoodieTableFactory().createTableSink(sinkContext2);
final Configuration conf2 = tableSink2.getConf();
assertThat(conf2.get(FlinkOptions.RECORD_KEY_FIELD), is("f0,f1"));
assertThat(conf2.get(FlinkOptions.KEYGEN_CLASS), is(ComplexAvroKeyGenerator.class.getName()));
}
// -------------------------------------------------------------------------
// Inner Class
// -------------------------------------------------------------------------
@@ -111,13 +179,25 @@ public class TestHoodieTableFactory {
*/
private static class MockSourceContext implements TableSourceFactory.Context {
private final Configuration conf;
private final TableSchema schema;
private final List<String> partitions;
private MockSourceContext(Configuration conf) {
private MockSourceContext(Configuration conf, TableSchema schema, List<String> partitions) {
this.conf = conf;
this.schema = schema;
this.partitions = partitions;
}
static MockSourceContext getInstance(Configuration conf) {
return new MockSourceContext(conf);
return getInstance(conf, TestConfigurations.TABLE_SCHEMA, Collections.singletonList("partition"));
}
static MockSourceContext getInstance(Configuration conf, TableSchema schema, String partition) {
return getInstance(conf, schema, Collections.singletonList(partition));
}
static MockSourceContext getInstance(Configuration conf, TableSchema schema, List<String> partitions) {
return new MockSourceContext(conf, schema, partitions);
}
@Override
@@ -127,8 +207,7 @@ public class TestHoodieTableFactory {
@Override
public CatalogTable getTable() {
return new CatalogTableImpl(TestConfigurations.TABLE_SCHEMA, Collections.singletonList("partition"),
conf.toMap(), "mock source table");
return new CatalogTableImpl(schema, partitions, conf.toMap(), "mock source table");
}
@Override
@@ -142,13 +221,25 @@ public class TestHoodieTableFactory {
*/
private static class MockSinkContext implements TableSinkFactory.Context {
private final Configuration conf;
private final TableSchema schema;
private final List<String> partitions;
private MockSinkContext(Configuration conf) {
private MockSinkContext(Configuration conf, TableSchema schema, List<String> partitions) {
this.conf = conf;
this.schema = schema;
this.partitions = partitions;
}
static MockSinkContext getInstance(Configuration conf) {
return new MockSinkContext(conf);
return getInstance(conf, TestConfigurations.TABLE_SCHEMA, "partition");
}
static MockSinkContext getInstance(Configuration conf, TableSchema schema, String partition) {
return getInstance(conf, schema, Collections.singletonList(partition));
}
static MockSinkContext getInstance(Configuration conf, TableSchema schema, List<String> partitions) {
return new MockSinkContext(conf, schema, partitions);
}
@Override
@@ -158,8 +249,7 @@ public class TestHoodieTableFactory {
@Override
public CatalogTable getTable() {
return new CatalogTableImpl(TestConfigurations.TABLE_SCHEMA, Collections.singletonList("partition"),
conf.toMap(), "mock sink table");
return new CatalogTableImpl(this.schema, this.partitions, conf.toMap(), "mock sink table");
}
@Override

View File

@@ -73,7 +73,8 @@ public class TestConfigurations {
+ " name varchar(10),\n"
+ " age int,\n"
+ " ts timestamp(3),\n"
+ " `partition` varchar(20)\n"
+ " `partition` varchar(20),\n"
+ " PRIMARY KEY(uuid) NOT ENFORCED\n"
+ ")\n"
+ "PARTITIONED BY (`partition`)\n"
+ "with (\n"