-
Notifications
You must be signed in to change notification settings - Fork 2.5k
[HUDI-5058]Fix flink catalog read spark table error : primary key col can not be nullable #7009
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 2 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,7 @@ | |
|
|
||
| package org.apache.hudi.util; | ||
|
|
||
| import org.apache.flink.table.api.DataTypes; | ||
| import org.apache.flink.table.types.DataType; | ||
| import org.apache.flink.table.types.logical.LocalZonedTimestampType; | ||
| import org.apache.flink.table.types.logical.LogicalType; | ||
|
|
@@ -26,10 +27,14 @@ | |
| import org.apache.flink.table.types.logical.RowType; | ||
| import org.apache.flink.table.types.logical.TimestampType; | ||
|
|
||
| import javax.annotation.Nullable; | ||
|
|
||
| import java.math.BigDecimal; | ||
| import java.time.LocalDate; | ||
| import java.time.LocalDateTime; | ||
| import java.util.ArrayList; | ||
| import java.util.Arrays; | ||
| import java.util.List; | ||
|
|
||
| /** | ||
| * Utilities for {@link org.apache.flink.table.types.DataType}. | ||
|
|
@@ -123,4 +128,39 @@ public static Object resolvePartition(String partition, DataType type) { | |
| "Can not convert %s to type %s for partition value", partition, type)); | ||
| } | ||
| } | ||
|
|
||
| /** | ||
| * Ensures the give columns of the row data type are not nullable(for example, the primary keys). | ||
| * | ||
| * @param dataType The row data type | ||
| * @param pkColumns The primary keys | ||
| * @return a new row data type if any column nullability is tweaked or the original data type | ||
| */ | ||
| public static DataType ensureColumnsAsNonNullable(DataType dataType, @Nullable List<String> pkColumns) { | ||
|
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. nit: this method should be generic no reason to couple it to primary-keys (we can call it for non-PK use-cases as well)
Contributor
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. Hello, thank you for your reviews. This method is used to convert the primary key field into not null type.
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. @waywtdcc i understand that. What i'm saying though is that there's no reason for us to couple it to primary keys (it should be generic to be able to handle any columns) |
||
| if (pkColumns == null || pkColumns.isEmpty()) { | ||
| return dataType; | ||
| } | ||
| RowType rowType = (RowType) dataType.getLogicalType(); | ||
| List<DataType> oriFieldTypes = dataType.getChildren(); | ||
|
||
| List<String> fieldNames = rowType.getFieldNames(); | ||
| List<DataType> fieldTypes = new ArrayList<>(); | ||
| boolean tweaked = false; | ||
| for (int i = 0; i < fieldNames.size(); i++) { | ||
| if (pkColumns.contains(fieldNames.get(i)) && rowType.getTypeAt(i).isNullable()) { | ||
| fieldTypes.add(oriFieldTypes.get(i).notNull()); | ||
| tweaked = true; | ||
| } else { | ||
| fieldTypes.add(oriFieldTypes.get(i)); | ||
| } | ||
| } | ||
| if (!tweaked) { | ||
| return dataType; | ||
| } | ||
| List<DataTypes.Field> fields = new ArrayList<>(); | ||
| for (int i = 0; i < fieldNames.size(); i++) { | ||
| fields.add(DataTypes.FIELD(fieldNames.get(i), fieldTypes.get(i))); | ||
| } | ||
| return DataTypes.ROW(fields.toArray(new DataTypes.Field[fields.size()])).notNull(); | ||
|
||
| } | ||
|
|
||
| } | ||
Uh oh!
There was an error while loading. Please reload this page.