-
Notifications
You must be signed in to change notification settings - Fork 2.5k
[HUDI-5004] Allow nested field as primary key and preCombineField in flink sql #6915
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -18,6 +18,8 @@ | |
|
|
||
| package org.apache.hudi.table; | ||
|
|
||
| import org.apache.avro.Schema; | ||
| import org.apache.hudi.avro.HoodieAvroUtils; | ||
| import org.apache.hudi.common.model.DefaultHoodieRecordPayload; | ||
| import org.apache.hudi.common.util.StringUtils; | ||
| import org.apache.hudi.configuration.FlinkOptions; | ||
|
|
@@ -120,6 +122,7 @@ public Set<ConfigOption<?>> optionalOptions() { | |
| */ | ||
| private void sanityCheck(Configuration conf, ResolvedSchema schema) { | ||
| List<String> fields = schema.getColumnNames(); | ||
| Schema inferredSchema = AvroSchemaConverter.convertToSchema(schema.toPhysicalRowDataType().notNull().getLogicalType()); | ||
|
|
||
| // validate record key in pk absence. | ||
| if (!schema.getPrimaryKey().isPresent()) { | ||
|
|
@@ -132,7 +135,7 @@ private void sanityCheck(Configuration conf, ResolvedSchema schema) { | |
| } | ||
|
|
||
| Arrays.stream(recordKeys) | ||
| .filter(field -> !fields.contains(field)) | ||
| .filter(field -> !HoodieAvroUtils.getField(inferredSchema, field).isPresent()) | ||
| .findAny() | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Do we support nested primary key also in this patch ?
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Uhm, from what i understand, Spark-SQL supports nested primaryKey and precombineField. The changes I made here is to standardize the validations in Spark and Flink. |
||
| .ifPresent(f -> { | ||
| throw new HoodieValidationException("Field '" + f + "' specified in option " | ||
|
|
@@ -142,7 +145,7 @@ private void sanityCheck(Configuration conf, ResolvedSchema schema) { | |
|
|
||
| // validate pre_combine key | ||
| String preCombineField = conf.get(FlinkOptions.PRECOMBINE_FIELD); | ||
| if (!fields.contains(preCombineField)) { | ||
| if (!HoodieAvroUtils.getField(inferredSchema, preCombineField).isPresent()) { | ||
| if (OptionsResolver.isDefaultHoodieRecordPayloadClazz(conf)) { | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. We may also need to validate the nested field names.
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I agree. This change is to standardize the validations between Flink and Spark. As such, no checks on the nested field names were made.
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. @danny0405 I have added the feature to validate nested fields for Flink. Can you please help to review this PR again? Thank you. |
||
| throw new HoodieValidationException("Option '" + FlinkOptions.PRECOMBINE_FIELD.key() | ||
| + "' is required for payload class: " + DefaultHoodieRecordPayload.class.getName()); | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -165,6 +165,35 @@ void testRequiredOptionsForSource() { | |
|
|
||
| assertDoesNotThrow(() -> new HoodieTableFactory().createDynamicTableSource(sourceContext6)); | ||
| assertDoesNotThrow(() -> new HoodieTableFactory().createDynamicTableSink(sourceContext6)); | ||
|
|
||
| // nested pk field is allowed | ||
| ResolvedSchema schema6 = SchemaBuilder.instance() | ||
| .field("f0", | ||
| DataTypes.ROW(DataTypes.FIELD("id", DataTypes.INT()), DataTypes.FIELD("date", DataTypes.VARCHAR(20)))) | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Can we write a IT test in
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Hmmm, do you mean add IT tests? Or remove the UT i included here and rewrite them as IT tests? |
||
| .field("f1", DataTypes.VARCHAR(20)) | ||
| .field("f2", DataTypes.TIMESTAMP(3)) | ||
| .field("ts", DataTypes.TIMESTAMP(3)) | ||
| .build(); | ||
| this.conf.setString(FlinkOptions.RECORD_KEY_FIELD, "f0.id"); | ||
| this.conf.setString(FlinkOptions.PRECOMBINE_FIELD, "f2"); | ||
| final MockContext sourceContext7 = MockContext.getInstance(this.conf, schema6, "f2"); | ||
|
|
||
| assertDoesNotThrow(() -> new HoodieTableFactory().createDynamicTableSource(sourceContext7)); | ||
| assertDoesNotThrow(() -> new HoodieTableFactory().createDynamicTableSink(sourceContext7)); | ||
|
|
||
| // nested precombine field is allowed | ||
| ResolvedSchema schema7 = SchemaBuilder.instance() | ||
| .field("f0", DataTypes.INT().notNull()) | ||
| .field("f1", DataTypes.VARCHAR(20)) | ||
| .field("f2", DataTypes.TIMESTAMP(3)) | ||
| .field("ts", DataTypes.ROW(DataTypes.FIELD("year", DataTypes.INT()), DataTypes.FIELD("MONTH", DataTypes.INT()))) | ||
| .build(); | ||
| this.conf.setString(FlinkOptions.RECORD_KEY_FIELD, "f0"); | ||
| this.conf.setString(FlinkOptions.PRECOMBINE_FIELD, "f2.year"); | ||
| final MockContext sourceContext8 = MockContext.getInstance(this.conf, schema7, "f2"); | ||
|
|
||
| assertDoesNotThrow(() -> new HoodieTableFactory().createDynamicTableSource(sourceContext8)); | ||
| assertDoesNotThrow(() -> new HoodieTableFactory().createDynamicTableSink(sourceContext8)); | ||
| } | ||
|
|
||
| @Test | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
There is no need to convert the
ResolvedSchemaas an avro schema for validation, theResolvedSchema#getColumnDataTypescan fetch the data type of each field.Also we need to fix the
RowDataKeyGen#getRecordKeyfor nested primary keys.Uh oh!
There was an error while loading. Please reload this page.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Sure, I will take a look.
The main reasons for doing this is:
AvroSchemaConverter.convertToSchemawas already imported and used somewhere else in the code so, just reuseLet me try to see if we can use the
ResolvedSchemainstead, will get back to you.Got it!