|
| 1 | +/* |
| 2 | + * Licensed to the Apache Software Foundation (ASF) under one |
| 3 | + * or more contributor license agreements. See the NOTICE file |
| 4 | + * distributed with this work for additional information |
| 5 | + * regarding copyright ownership. The ASF licenses this file |
| 6 | + * to you under the Apache License, Version 2.0 (the |
| 7 | + * "License"); you may not use this file except in compliance |
| 8 | + * with the License. You may obtain a copy of the License at |
| 9 | + * |
| 10 | + * http://www.apache.org/licenses/LICENSE-2.0 |
| 11 | + * |
| 12 | + * Unless required by applicable law or agreed to in writing, software |
| 13 | + * distributed under the License is distributed on an "AS IS" BASIS, |
| 14 | + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
| 15 | + * See the License for the specific language governing permissions and |
| 16 | + * limitations under the License. |
| 17 | + */ |
| 18 | + |
| 19 | +package org.apache.hudi.table.catalog; |
| 20 | + |
| 21 | +import org.apache.hudi.common.util.StringUtils; |
| 22 | + |
| 23 | +import org.apache.flink.table.api.DataTypes; |
| 24 | +import org.apache.flink.table.api.TableSchema; |
| 25 | +import org.apache.flink.table.types.DataType; |
| 26 | +import org.apache.flink.table.types.logical.LogicalType; |
| 27 | +import org.apache.hadoop.hive.metastore.api.FieldSchema; |
| 28 | +import org.apache.hadoop.hive.metastore.api.Table; |
| 29 | +import org.apache.hadoop.hive.serde2.typeinfo.CharTypeInfo; |
| 30 | +import org.apache.hadoop.hive.serde2.typeinfo.DecimalTypeInfo; |
| 31 | +import org.apache.hadoop.hive.serde2.typeinfo.ListTypeInfo; |
| 32 | +import org.apache.hadoop.hive.serde2.typeinfo.MapTypeInfo; |
| 33 | +import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo; |
| 34 | +import org.apache.hadoop.hive.serde2.typeinfo.StructTypeInfo; |
| 35 | +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo; |
| 36 | +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils; |
| 37 | +import org.apache.hadoop.hive.serde2.typeinfo.VarcharTypeInfo; |
| 38 | + |
| 39 | +import java.util.ArrayList; |
| 40 | +import java.util.Collections; |
| 41 | +import java.util.List; |
| 42 | + |
| 43 | +import static org.apache.flink.util.Preconditions.checkNotNull; |
| 44 | + |
| 45 | +/** |
| 46 | + * hive table options. |
| 47 | + */ |
| 48 | +public class HiveTableOptions { |
| 49 | + /** Get field names from field schemas. */ |
| 50 | + public static List<String> getFieldNames(List<FieldSchema> fieldSchemas) { |
| 51 | + List<String> names = new ArrayList<>(fieldSchemas.size()); |
| 52 | + for (FieldSchema fs : fieldSchemas) { |
| 53 | + names.add(fs.getName()); |
| 54 | + } |
| 55 | + return names; |
| 56 | + } |
| 57 | + |
| 58 | + public static org.apache.flink.table.api.Schema convertTableSchema(Table hiveTable) { |
| 59 | + List<FieldSchema> allCols = new ArrayList<>(hiveTable.getSd().getCols()); |
| 60 | + allCols.addAll(hiveTable.getPartitionKeys()); |
| 61 | + |
| 62 | + String pkConstraintName = hiveTable.getParameters().get(TableOptionProperties.PK_CONSTRAINT_NAME); |
| 63 | + List<String> primaryColNames = StringUtils.isNullOrEmpty(pkConstraintName) |
| 64 | + ? Collections.EMPTY_LIST |
| 65 | + : StringUtils.split(hiveTable.getParameters().get(TableOptionProperties.PK_COLUMNS),","); |
| 66 | + |
| 67 | + String[] colNames = new String[allCols.size()]; |
| 68 | + DataType[] colTypes = new DataType[allCols.size()]; |
| 69 | + |
| 70 | + for (int i = 0; i < allCols.size(); i++) { |
| 71 | + FieldSchema fs = allCols.get(i); |
| 72 | + |
| 73 | + colNames[i] = fs.getName(); |
| 74 | + colTypes[i] = |
| 75 | + toFlinkType(TypeInfoUtils.getTypeInfoFromTypeString(fs.getType())); |
| 76 | + if (primaryColNames.contains(colNames[i])) { |
| 77 | + colTypes[i] = colTypes[i].notNull(); |
| 78 | + } |
| 79 | + } |
| 80 | + |
| 81 | + org.apache.flink.table.api.Schema.Builder builder = org.apache.flink.table.api.Schema.newBuilder().fromFields(colNames, colTypes); |
| 82 | + if (!StringUtils.isNullOrEmpty(pkConstraintName)) { |
| 83 | + builder.primaryKeyNamed(pkConstraintName, primaryColNames); |
| 84 | + } |
| 85 | + |
| 86 | + return builder.build(); |
| 87 | + } |
| 88 | + |
| 89 | + /** |
| 90 | + * Convert Hive data type to a Flink data type. |
| 91 | + * |
| 92 | + * @param hiveType a Hive data type |
| 93 | + * @return the corresponding Flink data type |
| 94 | + */ |
| 95 | + public static DataType toFlinkType(TypeInfo hiveType) { |
| 96 | + checkNotNull(hiveType, "hiveType cannot be null"); |
| 97 | + |
| 98 | + switch (hiveType.getCategory()) { |
| 99 | + case PRIMITIVE: |
| 100 | + return toFlinkPrimitiveType((PrimitiveTypeInfo) hiveType); |
| 101 | + case LIST: |
| 102 | + ListTypeInfo listTypeInfo = (ListTypeInfo) hiveType; |
| 103 | + return DataTypes.ARRAY(toFlinkType(listTypeInfo.getListElementTypeInfo())); |
| 104 | + case MAP: |
| 105 | + MapTypeInfo mapTypeInfo = (MapTypeInfo) hiveType; |
| 106 | + return DataTypes.MAP( |
| 107 | + toFlinkType(mapTypeInfo.getMapKeyTypeInfo()), |
| 108 | + toFlinkType(mapTypeInfo.getMapValueTypeInfo())); |
| 109 | + case STRUCT: |
| 110 | + StructTypeInfo structTypeInfo = (StructTypeInfo) hiveType; |
| 111 | + |
| 112 | + List<String> names = structTypeInfo.getAllStructFieldNames(); |
| 113 | + List<TypeInfo> typeInfos = structTypeInfo.getAllStructFieldTypeInfos(); |
| 114 | + |
| 115 | + DataTypes.Field[] fields = new DataTypes.Field[names.size()]; |
| 116 | + |
| 117 | + for (int i = 0; i < fields.length; i++) { |
| 118 | + fields[i] = DataTypes.FIELD(names.get(i), toFlinkType(typeInfos.get(i))); |
| 119 | + } |
| 120 | + |
| 121 | + return DataTypes.ROW(fields); |
| 122 | + default: |
| 123 | + throw new UnsupportedOperationException( |
| 124 | + String.format("Flink doesn't support Hive data type %s yet.", hiveType)); |
| 125 | + } |
| 126 | + } |
| 127 | + |
| 128 | + private static DataType toFlinkPrimitiveType(PrimitiveTypeInfo hiveType) { |
| 129 | + checkNotNull(hiveType, "hiveType cannot be null"); |
| 130 | + |
| 131 | + switch (hiveType.getPrimitiveCategory()) { |
| 132 | + case CHAR: |
| 133 | + return DataTypes.CHAR(((CharTypeInfo) hiveType).getLength()); |
| 134 | + case VARCHAR: |
| 135 | + return DataTypes.VARCHAR(((VarcharTypeInfo) hiveType).getLength()); |
| 136 | + case STRING: |
| 137 | + return DataTypes.STRING(); |
| 138 | + case BOOLEAN: |
| 139 | + return DataTypes.BOOLEAN(); |
| 140 | + case BYTE: |
| 141 | + return DataTypes.TINYINT(); |
| 142 | + case SHORT: |
| 143 | + return DataTypes.SMALLINT(); |
| 144 | + case INT: |
| 145 | + return DataTypes.INT(); |
| 146 | + case LONG: |
| 147 | + return DataTypes.BIGINT(); |
| 148 | + case FLOAT: |
| 149 | + return DataTypes.FLOAT(); |
| 150 | + case DOUBLE: |
| 151 | + return DataTypes.DOUBLE(); |
| 152 | + case DATE: |
| 153 | + return DataTypes.DATE(); |
| 154 | + case TIMESTAMP: |
| 155 | + return DataTypes.TIMESTAMP(9); |
| 156 | + case BINARY: |
| 157 | + return DataTypes.BYTES(); |
| 158 | + case DECIMAL: |
| 159 | + DecimalTypeInfo decimalTypeInfo = (DecimalTypeInfo) hiveType; |
| 160 | + return DataTypes.DECIMAL( |
| 161 | + decimalTypeInfo.getPrecision(), decimalTypeInfo.getScale()); |
| 162 | + default: |
| 163 | + throw new UnsupportedOperationException( |
| 164 | + String.format( |
| 165 | + "Flink doesn't support Hive primitive type %s yet", hiveType)); |
| 166 | + } |
| 167 | + } |
| 168 | + |
| 169 | + /** Create Hive columns from Flink TableSchema. */ |
| 170 | + public static List<FieldSchema> createHiveColumns(TableSchema schema) { |
| 171 | + String[] fieldNames = schema.getFieldNames(); |
| 172 | + DataType[] fieldTypes = schema.getFieldDataTypes(); |
| 173 | + |
| 174 | + List<FieldSchema> columns = new ArrayList<>(fieldNames.length); |
| 175 | + |
| 176 | + for (int i = 0; i < fieldNames.length; i++) { |
| 177 | + columns.add( |
| 178 | + new FieldSchema( |
| 179 | + fieldNames[i], |
| 180 | + toHiveTypeInfo(fieldTypes[i], true).getTypeName(), |
| 181 | + null)); |
| 182 | + } |
| 183 | + |
| 184 | + return columns; |
| 185 | + } |
| 186 | + |
| 187 | + /** |
| 188 | + * Convert Flink DataType to Hive TypeInfo. For types with a precision parameter, e.g. |
| 189 | + * timestamp, the supported precisions in Hive and Flink can be different. Therefore the |
| 190 | + * conversion will fail for those types if the precision is not supported by Hive and |
| 191 | + * checkPrecision is true. |
| 192 | + * |
| 193 | + * @param dataType a Flink DataType |
| 194 | + * @param checkPrecision whether to fail the conversion if the precision of the DataType is not |
| 195 | + * supported by Hive |
| 196 | + * @return the corresponding Hive data type |
| 197 | + */ |
| 198 | + public static TypeInfo toHiveTypeInfo(DataType dataType, boolean checkPrecision) { |
| 199 | + checkNotNull(dataType, "type cannot be null"); |
| 200 | + LogicalType logicalType = dataType.getLogicalType(); |
| 201 | + return logicalType.accept(new TypeInfoLogicalTypeVisitor(dataType, checkPrecision)); |
| 202 | + } |
| 203 | +} |
0 commit comments