Skip to content

Commit

Permalink
[HUDI-5058] Fix flink catalog read spark table error : primary key co…
Browse files Browse the repository at this point in the history
…l can not be nullable (#7009)
  • Loading branch information
waywtdcc authored Oct 26, 2022
1 parent fb5fab5 commit 20e8ec7
Show file tree
Hide file tree
Showing 2 changed files with 55 additions and 4 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -34,6 +34,7 @@
import org.apache.hudi.sync.common.util.ConfigUtils;
import org.apache.hudi.table.format.FilePathUtils;
import org.apache.hudi.util.AvroSchemaConverter;
import org.apache.hudi.util.DataTypeUtils;
import org.apache.hudi.util.StreamerUtil;

import org.apache.avro.Schema;
Expand Down Expand Up @@ -70,6 +71,7 @@
import org.apache.flink.table.catalog.stats.CatalogColumnStatistics;
import org.apache.flink.table.catalog.stats.CatalogTableStatistics;
import org.apache.flink.table.expressions.Expression;
import org.apache.flink.table.types.DataType;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hive.conf.HiveConf;
import org.apache.hadoop.hive.metastore.IMetaStoreClient;
Expand Down Expand Up @@ -402,17 +404,22 @@ public CatalogBaseTable getTable(ObjectPath tablePath) throws TableNotExistExcep
String path = hiveTable.getSd().getLocation();
Map<String, String> parameters = hiveTable.getParameters();
Schema latestTableSchema = StreamerUtil.getLatestTableSchema(path, hiveConf);
String pkColumnsStr = parameters.get(FlinkOptions.RECORD_KEY_FIELD.key());
List<String> pkColumns = StringUtils.isNullOrEmpty(pkColumnsStr)
? null : StringUtils.split(pkColumnsStr, ",");
org.apache.flink.table.api.Schema schema;
if (latestTableSchema != null) {
// if the table is initialized from spark, the write schema is nullable for pk columns.
DataType tableDataType = DataTypeUtils.ensureColumnsAsNonNullable(
AvroSchemaConverter.convertToDataType(latestTableSchema), pkColumns);
org.apache.flink.table.api.Schema.Builder builder = org.apache.flink.table.api.Schema.newBuilder()
.fromRowDataType(AvroSchemaConverter.convertToDataType(latestTableSchema));
.fromRowDataType(tableDataType);
String pkConstraintName = parameters.get(PK_CONSTRAINT_NAME);
String pkColumns = parameters.get(FlinkOptions.RECORD_KEY_FIELD.key());
if (!StringUtils.isNullOrEmpty(pkConstraintName)) {
// pkColumns expect not to be null
builder.primaryKeyNamed(pkConstraintName, StringUtils.split(pkColumns, ","));
builder.primaryKeyNamed(pkConstraintName, pkColumns);
} else if (pkColumns != null) {
builder.primaryKey(StringUtils.split(pkColumns, ","));
builder.primaryKey(pkColumns);
}
schema = builder.build();
} else {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand All @@ -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}.
Expand Down Expand Up @@ -123,4 +128,43 @@ 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, datatype logicaltype must be rowtype
* @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) {
if (pkColumns == null || pkColumns.isEmpty()) {
return dataType;
}
LogicalType dataTypeLogicalType = dataType.getLogicalType();
if (!(dataTypeLogicalType instanceof RowType)) {
throw new RuntimeException("The datatype to be converted must be row type, but this type is :" + dataTypeLogicalType.getClass());
}
RowType rowType = (RowType) dataTypeLogicalType;
List<DataType> originalFieldTypes = 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(originalFieldTypes.get(i).notNull());
tweaked = true;
} else {
fieldTypes.add(originalFieldTypes.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.stream().toArray(DataTypes.Field[]::new)).notNull();
}

}

0 comments on commit 20e8ec7

Please sign in to comment.