-
Notifications
You must be signed in to change notification settings - Fork 3k
Hive: Using Hive schema to create tables and partition specification #1612
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
356f723
4ff16a8
a40b6c2
e3b3905
1d71fee
133f188
87202bf
0fecd94
aefbca0
3c3f1be
ae751a6
b4b985e
1a3b610
a6792e2
4956970
6f881bb
ad53a0a
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 |
|---|---|---|
| @@ -0,0 +1,118 @@ | ||
| /* | ||
| * Licensed to the Apache Software Foundation (ASF) under one | ||
| * or more contributor license agreements. See the NOTICE file | ||
| * distributed with this work for additional information | ||
| * regarding copyright ownership. The ASF licenses this file | ||
| * to you under the Apache License, Version 2.0 (the | ||
| * "License"); you may not use this file except in compliance | ||
| * with the License. You may obtain a copy of the License at | ||
| * | ||
| * http://www.apache.org/licenses/LICENSE-2.0 | ||
| * | ||
| * Unless required by applicable law or agreed to in writing, | ||
| * software distributed under the License is distributed on an | ||
| * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY | ||
| * KIND, either express or implied. See the License for the | ||
| * specific language governing permissions and limitations | ||
| * under the License. | ||
| */ | ||
|
|
||
| package org.apache.iceberg.mr.hive; | ||
|
|
||
| import java.util.ArrayList; | ||
| import java.util.List; | ||
| import org.apache.hadoop.hive.serde2.typeinfo.DecimalTypeInfo; | ||
| import org.apache.hadoop.hive.serde2.typeinfo.ListTypeInfo; | ||
| import org.apache.hadoop.hive.serde2.typeinfo.MapTypeInfo; | ||
| import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo; | ||
| import org.apache.hadoop.hive.serde2.typeinfo.StructTypeInfo; | ||
| import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo; | ||
| import org.apache.iceberg.Schema; | ||
| import org.apache.iceberg.types.Type; | ||
| import org.apache.iceberg.types.Types; | ||
|
|
||
| class HiveSchemaConverter { | ||
| private int id; | ||
|
|
||
| private HiveSchemaConverter() { | ||
| id = 0; | ||
| } | ||
|
|
||
| static Schema convert(List<String> names, List<TypeInfo> typeInfos) { | ||
| HiveSchemaConverter converter = new HiveSchemaConverter(); | ||
| return new Schema(converter.convertInternal(names, typeInfos)); | ||
| } | ||
|
|
||
| List<Types.NestedField> convertInternal(List<String> names, List<TypeInfo> typeInfos) { | ||
| List<Types.NestedField> result = new ArrayList<>(names.size()); | ||
| for (int i = 0; i < names.size(); ++i) { | ||
| result.add(Types.NestedField.optional(id++, names.get(i), convert(typeInfos.get(i)))); | ||
| } | ||
|
|
||
| return result; | ||
| } | ||
|
|
||
| Type convert(TypeInfo typeInfo) { | ||
| switch (typeInfo.getCategory()) { | ||
| case PRIMITIVE: | ||
| switch (((PrimitiveTypeInfo) typeInfo).getPrimitiveCategory()) { | ||
| case FLOAT: | ||
| return Types.FloatType.get(); | ||
| case DOUBLE: | ||
| return Types.DoubleType.get(); | ||
| case BOOLEAN: | ||
| return Types.BooleanType.get(); | ||
| case BYTE: | ||
| case SHORT: | ||
pvary marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| throw new IllegalArgumentException("Unsupported Hive type (" + | ||
| ((PrimitiveTypeInfo) typeInfo).getPrimitiveCategory() + | ||
| ") for Iceberg tables. Consider using INT/INTEGER type instead."); | ||
| case INT: | ||
| return Types.IntegerType.get(); | ||
| case LONG: | ||
| return Types.LongType.get(); | ||
| case BINARY: | ||
| return Types.BinaryType.get(); | ||
| case CHAR: | ||
| case VARCHAR: | ||
| throw new IllegalArgumentException("Unsupported Hive type (" + | ||
| ((PrimitiveTypeInfo) typeInfo).getPrimitiveCategory() + | ||
| ") for Iceberg tables. Consider using STRING type instead."); | ||
|
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. I would be fine mapping these to string.
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. Let's discuss this on he dev list where we are talking about the schema mapping. |
||
| case STRING: | ||
| return Types.StringType.get(); | ||
| case TIMESTAMP: | ||
| return Types.TimestampType.withoutZone(); | ||
| case DATE: | ||
| return Types.DateType.get(); | ||
| case DECIMAL: | ||
| DecimalTypeInfo decimalTypeInfo = (DecimalTypeInfo) typeInfo; | ||
| return Types.DecimalType.of(decimalTypeInfo.precision(), decimalTypeInfo.scale()); | ||
| case INTERVAL_YEAR_MONTH: | ||
| case INTERVAL_DAY_TIME: | ||
| default: | ||
| throw new IllegalArgumentException("Unsupported Hive type (" + | ||
| ((PrimitiveTypeInfo) typeInfo).getPrimitiveCategory() + | ||
| ") for Iceberg tables."); | ||
| } | ||
| case STRUCT: | ||
| StructTypeInfo structTypeInfo = (StructTypeInfo) typeInfo; | ||
| List<Types.NestedField> fields = | ||
| convertInternal(structTypeInfo.getAllStructFieldNames(), structTypeInfo.getAllStructFieldTypeInfos()); | ||
| return Types.StructType.of(fields); | ||
| case MAP: | ||
| MapTypeInfo mapTypeInfo = (MapTypeInfo) typeInfo; | ||
| Type keyType = convert(mapTypeInfo.getMapKeyTypeInfo()); | ||
| Type valueType = convert(mapTypeInfo.getMapValueTypeInfo()); | ||
| int keyId = id++; | ||
| int valueId = id++; | ||
| return Types.MapType.ofOptional(keyId, valueId, keyType, valueType); | ||
| case LIST: | ||
| ListTypeInfo listTypeInfo = (ListTypeInfo) typeInfo; | ||
| Type listType = convert(listTypeInfo.getListElementTypeInfo()); | ||
| return Types.ListType.ofOptional(id++, listType); | ||
| case UNION: | ||
| default: | ||
| throw new IllegalArgumentException("Unknown type " + typeInfo.getCategory()); | ||
| } | ||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,71 @@ | ||
| /* | ||
| * Licensed to the Apache Software Foundation (ASF) under one | ||
| * or more contributor license agreements. See the NOTICE file | ||
| * distributed with this work for additional information | ||
| * regarding copyright ownership. The ASF licenses this file | ||
| * to you under the Apache License, Version 2.0 (the | ||
| * "License"); you may not use this file except in compliance | ||
| * with the License. You may obtain a copy of the License at | ||
| * | ||
| * http://www.apache.org/licenses/LICENSE-2.0 | ||
| * | ||
| * Unless required by applicable law or agreed to in writing, | ||
| * software distributed under the License is distributed on an | ||
| * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY | ||
| * KIND, either express or implied. See the License for the | ||
| * specific language governing permissions and limitations | ||
| * under the License. | ||
| */ | ||
|
|
||
| package org.apache.iceberg.mr.hive; | ||
|
|
||
| import java.util.ArrayList; | ||
| import java.util.Collections; | ||
| import java.util.List; | ||
| import org.apache.hadoop.hive.metastore.api.FieldSchema; | ||
| import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo; | ||
| import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils; | ||
| import org.apache.iceberg.Schema; | ||
| import org.slf4j.Logger; | ||
| import org.slf4j.LoggerFactory; | ||
|
|
||
| public class HiveSchemaUtil { | ||
| private static final Logger LOG = LoggerFactory.getLogger(HiveSchemaUtil.class); | ||
|
|
||
| private HiveSchemaUtil() { | ||
| } | ||
|
|
||
| /** | ||
| * Converts the list of Hive FieldSchemas to an Iceberg schema. | ||
| * <p> | ||
| * The list should contain the columns and the partition columns as well. | ||
| * @param fieldSchemas The list of the columns | ||
| * @return An equivalent Iceberg Schema | ||
| */ | ||
| public static Schema schema(List<FieldSchema> fieldSchemas) { | ||
| List<String> names = new ArrayList<>(fieldSchemas.size()); | ||
| List<TypeInfo> typeInfos = new ArrayList<>(fieldSchemas.size()); | ||
|
|
||
| for (FieldSchema col : fieldSchemas) { | ||
| names.add(col.getName()); | ||
| typeInfos.add(TypeInfoUtils.getTypeInfoFromTypeString(col.getType())); | ||
| } | ||
|
|
||
| return HiveSchemaConverter.convert(names, typeInfos); | ||
| } | ||
|
|
||
| /** | ||
| * Converts the Hive properties defining the columns to an Iceberg schema. | ||
| * @param columnNames The property containing the column names | ||
| * @param columnTypes The property containing the column types | ||
| * @param columnNameDelimiter The name delimiter | ||
| * @return The Iceberg schema | ||
| */ | ||
| public static Schema schema(String columnNames, String columnTypes, String columnNameDelimiter) { | ||
| // Parse the configuration parameters | ||
| List<String> names = new ArrayList<>(); | ||
| Collections.addAll(names, columnNames.split(columnNameDelimiter)); | ||
|
|
||
| return HiveSchemaConverter.convert(names, TypeInfoUtils.getTypeInfosFromTypeString(columnTypes)); | ||
| } | ||
| } |
Uh oh!
There was an error while loading. Please reload this page.