-
Notifications
You must be signed in to change notification settings - Fork 37
Hive Metastore: Merge hive and avro schema if not consistent #55
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 |
|---|---|---|
| @@ -0,0 +1,272 @@ | ||
| /* | ||
| * 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.hive.legacy; | ||
|
|
||
| import java.util.ArrayList; | ||
| import java.util.Arrays; | ||
| import java.util.List; | ||
| import org.apache.avro.Schema; | ||
| import org.apache.commons.lang3.StringUtils; | ||
| import org.apache.hadoop.hive.serde2.avro.AvroSerDe; | ||
| import org.apache.hadoop.hive.serde2.avro.AvroSerdeUtils; | ||
| import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; | ||
| import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector; | ||
| 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.hadoop.hive.serde2.typeinfo.UnionTypeInfo; | ||
| import org.apache.iceberg.relocated.com.google.common.collect.Lists; | ||
| import org.codehaus.jackson.node.JsonNodeFactory; | ||
|
|
||
|
|
||
| public class HiveTypeToAvroType { | ||
| private int recordCounter; | ||
| private final String namespace; | ||
|
|
||
| private final boolean mkFieldsOptional; | ||
|
|
||
| // Additional numeric type, similar to other logical type names in AvroSerde | ||
| private static final String SHORT_TYPE_NAME = "short"; | ||
| private static final String BYTE_TYPE_NAME = "byte"; | ||
|
|
||
| public HiveTypeToAvroType(String namespace, boolean mkFieldsOptional) { | ||
|
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. Does it make sense to convert this to a utility class and move those parameters to |
||
| this.recordCounter = 0; | ||
| this.namespace = namespace; | ||
|
|
||
| this.mkFieldsOptional = mkFieldsOptional; | ||
| } | ||
|
|
||
| Schema convertFieldsTypeInfoToAvroSchema(String recordNamespace, String recordName, List<String> fieldNames, | ||
| List<TypeInfo> fieldTypeInfos) { | ||
|
Comment on lines
+59
to
+60
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 avoid using list of field names and list of field types throughout the PR? For example instead of of |
||
| final List<Schema.Field> fields = new ArrayList<>(); | ||
| for (int i = 0; i < fieldNames.size(); ++i) { | ||
| final TypeInfo fieldTypeInfo = fieldTypeInfos.get(i); | ||
| String fieldName = fieldNames.get(i); | ||
| fieldName = removePrefix(fieldName); | ||
|
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 need this? I think field names being passed here are relative, they come from
Member
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. Yeah, I think it can be removed? also since we are dealing with hive, I feel the field names won't contain 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. This should be easy to verify. |
||
|
|
||
| // If there's a structType in the schema, we will use "recordNamespace.fieldName" instead of the | ||
| // autogenerated record name. The recordNamespace is composed of its parent's field names recursively. | ||
| // This mimics the logic of spark-avro. | ||
| // We will set the recordName to be capitalized, and the recordNameSpace will be in lower case | ||
| final Schema schema = convertTypeInfoToAvroSchema(fieldTypeInfo, recordNamespace + "." + recordName.toLowerCase(), | ||
| StringUtils.capitalize(fieldName)); | ||
| final Schema.Field f = new Schema.Field(fieldName, schema, null, null); | ||
| fields.add(f); | ||
| } | ||
|
|
||
| final Schema recordSchema = Schema.createRecord(recordName, null, namespace + recordNamespace, false); | ||
| recordSchema.setFields(fields); | ||
| return recordSchema; | ||
| } | ||
|
|
||
| Schema convertTypeInfoToAvroSchema(TypeInfo typeInfo, String recordNamespace, String recordName) { | ||
| Schema schema; | ||
| ObjectInspector.Category category = typeInfo.getCategory(); | ||
|
|
||
| switch (category) { | ||
| case STRUCT: | ||
| // We don't cache the structType because otherwise it could be possible that a field | ||
| // "lastname" is of type "firstname", where firstname is a compiled class. | ||
| // This will lead to ambiguity. | ||
|
Comment on lines
+88
to
+90
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 am not sure what this comment means. Which cache are we referring to? |
||
| schema = parseSchemaFromStruct((StructTypeInfo) typeInfo, recordNamespace, recordName); | ||
|
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 rename this and other methods to something like |
||
| break; | ||
| case LIST: | ||
| schema = parseSchemaFromList((ListTypeInfo) typeInfo, recordNamespace, recordName); | ||
| break; | ||
| case MAP: | ||
| schema = parseSchemaFromMap((MapTypeInfo) typeInfo, recordNamespace, recordName); | ||
| break; | ||
| case PRIMITIVE: | ||
| schema = parseSchemaFromPrimitive((PrimitiveTypeInfo) typeInfo); | ||
| break; | ||
| case UNION: | ||
| schema = parseSchemaFromUnion((UnionTypeInfo) typeInfo, recordNamespace, recordName); | ||
| break; | ||
| default: | ||
| throw new UnsupportedOperationException("Conversion from " + category + " not supported"); | ||
|
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. is not supported |
||
| } | ||
|
|
||
| if (mkFieldsOptional) { | ||
| return wrapInNullableUnion(schema); | ||
| } | ||
| return schema; | ||
| } | ||
|
|
||
| private Schema parseSchemaFromUnion(UnionTypeInfo typeInfo, final String recordNamespace, final String recordName) { | ||
| List<TypeInfo> typeInfos = typeInfo.getAllUnionObjectTypeInfos(); | ||
|
|
||
| // A union might contain duplicate struct typeinfos because the underlying Avro union has two Record types with | ||
| // different names but the same internal structure. | ||
| // For example, in tracking.CommunicationRequestEvent.specificRequest, | ||
| // PropGenerated and PropExternalCommunication have the same structure. In case of duplicate typeinfos, we generate | ||
|
Comment on lines
+120
to
+121
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. Best not to mention actual table and field names. |
||
| // a new record type for the duplicates. | ||
| List<Schema> schemas = new ArrayList<>(); | ||
|
|
||
| for (TypeInfo ti : typeInfos) { | ||
|
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. ti --> typeInfo |
||
| Schema candidate; | ||
| if (ti instanceof StructTypeInfo) { | ||
| StructTypeInfo sti = (StructTypeInfo) ti; | ||
|
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. sti --> structTypeInfo |
||
|
|
||
| // In case we have several structType in the same level, | ||
| // we need to add numbers to the record name to distinguish them from each other. | ||
| final String newRecordName = recordName + recordCounter; | ||
| recordCounter += 1; | ||
|
|
||
| candidate = parseSchemaFromStruct(sti, recordNamespace, newRecordName); | ||
| } else { // not a struct type | ||
| candidate = convertTypeInfoToAvroSchema(ti, recordNamespace, recordName); | ||
| } | ||
|
|
||
| // Remove nullable wrapping from nested schemas before adding | ||
| schemas.add(AvroSerdeUtils.isNullableType(candidate) ? AvroSerdeUtils.getOtherTypeFromNullableType(candidate) | ||
| : candidate); | ||
| } | ||
|
|
||
| return Schema.createUnion(schemas); | ||
| } | ||
|
|
||
| // Previously, Hive use recordType[N] as the recordName for each structType, | ||
| // with the change we made in LIHADOOP-36761, the new record name will be in the form of "structNamespace.structName" | ||
| private Schema parseSchemaFromStruct(final StructTypeInfo typeInfo, final String recordNamespace, | ||
| final String recordName) { | ||
| final Schema recordSchema = convertFieldsTypeInfoToAvroSchema(recordNamespace, recordName, | ||
| typeInfo.getAllStructFieldNames(), typeInfo.getAllStructFieldTypeInfos()); | ||
|
|
||
| return recordSchema; | ||
| } | ||
|
|
||
| private Schema parseSchemaFromList(final ListTypeInfo typeInfo, final String recordNamespace, | ||
| final String recordName) { | ||
| Schema listSchema = convertTypeInfoToAvroSchema(typeInfo.getListElementTypeInfo(), recordNamespace, recordName); | ||
| return Schema.createArray(listSchema); | ||
| } | ||
|
|
||
| private Schema parseSchemaFromMap(final MapTypeInfo typeInfo, final String recordNamespace, final String recordName) { | ||
| final TypeInfo keyTypeInfo = typeInfo.getMapKeyTypeInfo(); | ||
| final PrimitiveObjectInspector.PrimitiveCategory pc = ((PrimitiveTypeInfo) keyTypeInfo).getPrimitiveCategory(); | ||
| if (pc != PrimitiveObjectInspector.PrimitiveCategory.STRING) { | ||
| throw new UnsupportedOperationException("Key of Map can only be a String"); | ||
| } | ||
|
|
||
| final TypeInfo valueTypeInfo = typeInfo.getMapValueTypeInfo(); | ||
| final Schema valueSchema = convertTypeInfoToAvroSchema(valueTypeInfo, recordNamespace, recordName); | ||
|
|
||
| return Schema.createMap(valueSchema); | ||
| } | ||
|
|
||
| private Schema parseSchemaFromPrimitive(PrimitiveTypeInfo primitiveTypeInfo) { | ||
| Schema schema; | ||
| switch (primitiveTypeInfo.getPrimitiveCategory()) { | ||
| case LONG: | ||
| schema = Schema.create(Schema.Type.LONG); | ||
| break; | ||
|
|
||
| case DATE: | ||
| schema = Schema.create(Schema.Type.INT); | ||
| schema.addProp(AvroSerDe.AVRO_PROP_LOGICAL_TYPE, AvroSerDe.DATE_TYPE_NAME); | ||
| break; | ||
|
|
||
| case TIMESTAMP: | ||
| schema = Schema.create(Schema.Type.LONG); | ||
| schema.addProp(AvroSerDe.AVRO_PROP_LOGICAL_TYPE, AvroSerDe.TIMESTAMP_TYPE_NAME); | ||
| break; | ||
|
|
||
| case BINARY: | ||
| schema = Schema.create(Schema.Type.BYTES); | ||
| break; | ||
| case BOOLEAN: | ||
| schema = Schema.create(Schema.Type.BOOLEAN); | ||
| break; | ||
|
|
||
| case DOUBLE: | ||
| schema = Schema.create(Schema.Type.DOUBLE); | ||
| break; | ||
|
|
||
| case DECIMAL: | ||
| DecimalTypeInfo dti = (DecimalTypeInfo) primitiveTypeInfo; | ||
| JsonNodeFactory factory = JsonNodeFactory.instance; | ||
| schema = Schema.create(Schema.Type.BYTES); | ||
| schema.addProp(AvroSerDe.AVRO_PROP_LOGICAL_TYPE, AvroSerDe.DECIMAL_TYPE_NAME); | ||
| schema.addProp(AvroSerDe.AVRO_PROP_PRECISION, factory.numberNode(dti.getPrecision())); | ||
| schema.addProp(AvroSerDe.AVRO_PROP_SCALE, factory.numberNode(dti.getScale())); | ||
| break; | ||
|
|
||
| case FLOAT: | ||
| schema = Schema.create(Schema.Type.FLOAT); | ||
| break; | ||
|
|
||
| case BYTE: | ||
| schema = Schema.create(Schema.Type.INT); | ||
| schema.addProp(AvroSerDe.AVRO_PROP_LOGICAL_TYPE, BYTE_TYPE_NAME); | ||
| break; | ||
|
|
||
| case SHORT: | ||
| schema = Schema.create(Schema.Type.INT); | ||
| schema.addProp(AvroSerDe.AVRO_PROP_LOGICAL_TYPE, SHORT_TYPE_NAME); | ||
| break; | ||
|
|
||
| case INT: | ||
| schema = Schema.create(Schema.Type.INT); | ||
| break; | ||
|
|
||
| case CHAR: | ||
| case STRING: | ||
| case VARCHAR: | ||
| schema = Schema.create(Schema.Type.STRING); | ||
| break; | ||
|
|
||
| case VOID: | ||
| schema = Schema.create(Schema.Type.NULL); | ||
| break; | ||
|
|
||
| default: | ||
| throw new UnsupportedOperationException(primitiveTypeInfo + " is not supported."); | ||
| } | ||
| return schema; | ||
| } | ||
|
|
||
| private static Schema wrapInNullableUnion(Schema schema) { | ||
| Schema wrappedSchema = schema; | ||
| switch (schema.getType()) { | ||
| case NULL: | ||
| break; | ||
| case UNION: | ||
| List<Schema> unionSchemas = Lists.newArrayList(Schema.create(Schema.Type.NULL)); | ||
| unionSchemas.addAll(schema.getTypes()); | ||
| wrappedSchema = Schema.createUnion(unionSchemas); | ||
| break; | ||
| default: | ||
| wrappedSchema = Schema.createUnion(Arrays.asList(Schema.create(Schema.Type.NULL), schema)); | ||
| break; | ||
| } | ||
| return wrappedSchema; | ||
| } | ||
|
|
||
| private static String removePrefix(String name) { | ||
| final int idx = name.lastIndexOf('.'); | ||
| if (idx > 0) { | ||
| return name.substring(idx + 1); | ||
| } | ||
| return name; | ||
| } | ||
| } | ||
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.
Can we add test cases?
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.
will do later, but the integration test I ran already passed all the tables.