[
https://issues.apache.org/jira/browse/HIVE-26395?focusedWorklogId=791783&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-791783
]
ASF GitHub Bot logged work on HIVE-26395:
-----------------------------------------
Author: ASF GitHub Bot
Created on: 17/Jul/22 18:55
Start Date: 17/Jul/22 18:55
Worklog Time Spent: 10m
Work Description: amansinha100 commented on code in PR #3443:
URL: https://github.com/apache/hive/pull/3443#discussion_r922878384
##########
ql/src/java/org/apache/hadoop/hive/ql/ddl/table/create/CreateTableOperation.java:
##########
@@ -53,8 +59,33 @@ public CreateTableOperation(DDLOperationContext context,
CreateTableDesc desc) {
super(context, desc);
}
+ private void readSchemaFromFile() throws HiveException {
Review Comment:
nit: Add a brief comment since this is the main method for schema inference.
##########
ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java:
##########
@@ -13477,6 +13480,23 @@ private boolean hasConstraints(final List<FieldSchema>
partCols, final List<SQLD
}
return false;
}
+
+ boolean doesSupportSchemaInference(String fileFormat) throws
SemanticException {
Review Comment:
Pls add a comment. Also, this could be a public static utility method that
takes fileFormat and the conf parameters.
##########
hcatalog/core/src/main/java/org/apache/hive/hcatalog/cli/SemanticAnalysis/CreateTableHook.java:
##########
@@ -88,10 +85,6 @@ public ASTNode preAnalyze(HiveSemanticAnalyzerHookContext
context,
case HiveParser.TOK_ALTERTABLE_BUCKETS:
break;
- case HiveParser.TOK_LIKETABLE:
Review Comment:
Since this patch removes this LIKE TABLE support from the hcatalog
(presumably because it is not used), it would be good to mention this
explicitly in the commit message.
##########
serde/src/java/org/apache/hadoop/hive/serde2/SchemaInference.java:
##########
@@ -0,0 +1,34 @@
+/*
+ * 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.hadoop.hive.serde2;
+
+import org.apache.hadoop.hive.metastore.api.FieldSchema;
+import java.util.List;
+import org.apache.hadoop.conf.Configuration;
+
+public interface SchemaInference {
+ /**
+ * Infer Hive compatible schema from provided file. The purpose of this
method is to optionally
+ * allow SerDes to implement schema inference for CREATE TABLE LIKE FILE
support.
+ *
+ * @param conf Hadoop Configuration
+ * @param file Fully qualified path to file to infer schema from (hadoop
compatible URI + filename)
+ * @return List of FieldSchema that was derived from the provided file
Review Comment:
nit: Add a @throw for the SerDeException
##########
ql/src/java/org/apache/hadoop/hive/ql/ddl/table/create/CreateTableOperation.java:
##########
@@ -53,8 +59,33 @@ public CreateTableOperation(DDLOperationContext context,
CreateTableDesc desc) {
super(context, desc);
}
+ private void readSchemaFromFile() throws HiveException {
+ String fileFormat = desc.getLikeFileFormat();
+ StorageFormatFactory storageFormatFactory = new StorageFormatFactory();
+ StorageFormatDescriptor descriptor = storageFormatFactory.get(fileFormat);
+ if (descriptor == null) {
+ // normal operation should never hit this since analysis has already
verified this exists
Review Comment:
The comment here seems to indicate this should be an assert rather than an
exception.
##########
ql/src/java/org/apache/hadoop/hive/ql/io/parquet/serde/ParquetHiveSerDe.java:
##########
@@ -234,4 +251,161 @@ StructTypeInfo prune() {
return (StructTypeInfo) TypeInfoFactory.getStructTypeInfo(newNames,
newTypes);
}
}
+
+ // ReadSchema interface implementation
+ private String convertGroupType(GroupType group) throws SerDeException {
+ boolean first = true;
+ StringBuilder sb = new StringBuilder(serdeConstants.STRUCT_TYPE_NAME +
"<");
+ for (Type field: group.getFields()) {
+ if (first) {
+ first = false;
+ } else {
+ sb.append(",");
+ }
+ // fieldName:typeName
+
sb.append(field.getName()).append(":").append(convertParquetTypeToFieldType(field));
+ }
+ sb.append(">");
+ // struct<fieldName1:int, fieldName2:map<string : int>, etc
+ return sb.toString();
+ }
+
+ private String convertPrimitiveType(PrimitiveType primitive) throws
SerDeException {
+ switch (primitive.getPrimitiveTypeName()) {
+ case INT96:
+ return serdeConstants.TIMESTAMP_TYPE_NAME;
+ case INT32:
+ return serdeConstants.INT_TYPE_NAME;
+ case INT64:
+ return serdeConstants.BIGINT_TYPE_NAME;
+ case BOOLEAN:
+ return serdeConstants.BOOLEAN_TYPE_NAME;
+ case FLOAT:
+ return serdeConstants.FLOAT_TYPE_NAME;
+ case DOUBLE:
+ return serdeConstants.DOUBLE_TYPE_NAME;
+ case BINARY:
+ return serdeConstants.BINARY_TYPE_NAME;
+ default:
+ throw new SerDeException("Unhandled parquet primitive type " +
primitive.getPrimitiveTypeName());
+ }
+ }
+
+ private String convertParquetIntLogicalType(Type parquetType) throws
SerDeException {
+ IntLogicalTypeAnnotation intLogicalType = (IntLogicalTypeAnnotation)
parquetType.getLogicalTypeAnnotation();
+ PrimitiveType primitiveType = parquetType.asPrimitiveType();
+ // check to see if primitive type handling is implemented
+ switch (primitiveType.getPrimitiveTypeName()) {
+ case INT32:
+ case INT64:
+ break;
+ default:
+ throw new SerDeException("Unhandled parquet int logical type " +
intLogicalType);
+ }
+
+ if (!intLogicalType.isSigned()) {
+ throw new SerDeException("Unhandled parquet int logical type (unsigned
types are not supported) " + intLogicalType);
+ }
+
+ switch (intLogicalType.getBitWidth()) {
+ case 8: return serdeConstants.TINYINT_TYPE_NAME;
+ case 16: return serdeConstants.SMALLINT_TYPE_NAME;
+ case 32: return serdeConstants.INT_TYPE_NAME;
+ case 64: return serdeConstants.BIGINT_TYPE_NAME;
+ }
+
+ throw new SerDeException("Unhandled parquet int logical type " +
intLogicalType);
+ }
+
+ private String createMapType(String keyType, String valueType) {
+ // examples: map<string, int>, map<string : struct<i : int>>
+ return serdeConstants.MAP_TYPE_NAME + "<" + keyType + "," + valueType +
">";
+ }
+
+ private String convertParquetMapLogicalTypeAnnotation(Type parquetType)
throws SerDeException {
+ MapLogicalTypeAnnotation mType = (MapLogicalTypeAnnotation)
parquetType.getLogicalTypeAnnotation();
+ GroupType gType = parquetType.asGroupType();
+ Type innerField = gType.getType(0);
+ GroupType innerGroup = innerField.asGroupType();
+ Type key = innerGroup.getType(0);
+ Type value = innerGroup.getType(1);
+ return createMapType(convertParquetTypeToFieldType(key),
convertParquetTypeToFieldType(value));
+ }
+
+ private String createArrayType(String fieldType) {
+ // examples: array<int>, array<struct<i:int>>, array<map<string : int>>
+ return serdeConstants.LIST_TYPE_NAME + "<" + fieldType + ">";
+ }
+
+ private String convertParquetListLogicalTypeAnnotation(Type parquetType)
throws SerDeException {
+ ListLogicalTypeAnnotation mType = (ListLogicalTypeAnnotation)
parquetType.getLogicalTypeAnnotation();
+ GroupType gType = parquetType.asGroupType();
+ Type innerField = gType.getType(0);
+ if (innerField.isPrimitive() || innerField.getOriginalType() != null) {
+ return createArrayType(convertParquetTypeToFieldType(innerField));
+ }
+
+ GroupType innerGroup = innerField.asGroupType();
+ if (innerGroup.getFieldCount() != 1) {
+ return createArrayType(convertGroupType(innerGroup));
+ }
+
+ return
createArrayType(convertParquetTypeToFieldType(innerGroup.getType(0)));
+ }
+
+ private String createDecimalType(int precision, int scale) {
+ // example: decimal(10, 4)
+ return serdeConstants.DECIMAL_TYPE_NAME + "(" + precision + "," + scale +
")";
+ }
+
+ private String convertLogicalType(Type type) throws SerDeException {
+ LogicalTypeAnnotation lType = type.getLogicalTypeAnnotation();
+ if (lType instanceof IntLogicalTypeAnnotation) {
+ return convertParquetIntLogicalType(type);
+ } else if (lType instanceof StringLogicalTypeAnnotation) {
+ return serdeConstants.STRING_TYPE_NAME;
+ } else if (lType instanceof DecimalLogicalTypeAnnotation) {
+ DecimalLogicalTypeAnnotation dType = (DecimalLogicalTypeAnnotation)
lType;
+ return createDecimalType(dType.getPrecision(), dType.getScale());
+ } else if (lType instanceof MapLogicalTypeAnnotation) {
+ return convertParquetMapLogicalTypeAnnotation(type);
+ } else if (lType instanceof ListLogicalTypeAnnotation) {
+ return convertParquetListLogicalTypeAnnotation(type);
+ } else if (lType instanceof DateLogicalTypeAnnotation) {
+ // assuming 32 bit int
+ return serdeConstants.DATE_TYPE_NAME;
+ }
+ throw new SerDeException("Unhandled logical type " + lType);
+ }
+
+ private String convertParquetTypeToFieldType(Type type) throws
SerDeException {
+ if (type.getLogicalTypeAnnotation() != null) {
+ return convertLogicalType(type);
+ } else if (type.isPrimitive()) {
+ return convertPrimitiveType(type.asPrimitiveType());
+ }
+ return convertGroupType(type.asGroupType());
+ }
+
+ private FieldSchema convertParquetTypeToFieldSchema(Type type) throws
SerDeException {
+ String columnName = type.getName();
+ String typeName = convertParquetTypeToFieldType(type);
+ return new FieldSchema(columnName, typeName, "Inferred from Parquet
file.");
+ }
+
+ public List<FieldSchema> readSchema(Configuration conf, String file) throws
SerDeException {
+ ParquetMetadata footer;
+ try {
+ footer = ParquetFileReader.readFooter(conf, new Path(file),
ParquetMetadataConverter.NO_FILTER);
+ } catch (Exception e) {
+ throw new SerDeException("Failed to read parquet footer:", e);
+ }
+
+ MessageType msg = footer.getFileMetaData().getSchema();
+ List<FieldSchema> schema = new ArrayList<>();
+ for (Type field: msg.getFields()) {
+ schema.add(convertParquetTypeToFieldSchema(field));
+ }
+ return schema;
Review Comment:
For debuggability, you may want to add a DEBUG level log message for the
inferred schema.
Issue Time Tracking
-------------------
Worklog Id: (was: 791783)
Time Spent: 1h 50m (was: 1h 40m)
> Support CREATE TABLE LIKE FILE for PARQUET
> ------------------------------------------
>
> Key: HIVE-26395
> URL: https://issues.apache.org/jira/browse/HIVE-26395
> Project: Hive
> Issue Type: New Feature
> Components: HiveServer2
> Reporter: John Sherman
> Assignee: John Sherman
> Priority: Major
> Labels: pull-request-available
> Time Spent: 1h 50m
> Remaining Estimate: 0h
>
> The intent is to allow a user to create a table and derive the schema from a
> user provided parquet file. A secondary goal is to generalize this support so
> other SerDes/formats could implement the feature easily.
> The proposed syntax is:
> CREATE TABLE <tableName> LIKE FILE <fileFormat> 'path to file';
> Example being:
> {code:java}
> CREATE TABLE like_test_all_types LIKE FILE PARQUET
> '${system:test.tmp.dir}/test_all_types/000000_0';{code}
> with partitioning
> {code}
> CREATE TABLE like_test_partitioning LIKE FILE PARQUET
> '${system:test.tmp.dir}/test_all_types/000000_0' PARTITIONED BY (year STRING,
> month STRING);
> {code}
--
This message was sent by Atlassian Jira
(v8.20.10#820010)