[ 
https://issues.apache.org/jira/browse/FLINK-7243?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16707836#comment-16707836
 ] 

ASF GitHub Bot commented on FLINK-7243:
---------------------------------------

fhueske commented on a change in pull request #6483: 
[FLINK-7243][flink-formats] Add parquet input format
URL: https://github.com/apache/flink/pull/6483#discussion_r238359839
 
 

 ##########
 File path: 
flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/utils/ParquetSchemaConverter.java
 ##########
 @@ -0,0 +1,359 @@
+/*
+ * 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.flink.formats.parquet.utils;
+
+import org.apache.flink.api.common.typeinfo.BasicArrayTypeInfo;
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.common.typeinfo.SqlTimeTypeInfo;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.typeutils.MapTypeInfo;
+import org.apache.flink.api.java.typeutils.ObjectArrayTypeInfo;
+import org.apache.flink.api.java.typeutils.RowTypeInfo;
+
+import org.apache.parquet.avro.AvroSchemaConverter;
+import org.apache.parquet.schema.GroupType;
+import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.OriginalType;
+import org.apache.parquet.schema.PrimitiveType;
+import org.apache.parquet.schema.Type;
+import org.apache.parquet.schema.Types;
+
+import java.lang.reflect.Array;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Schema converter converts Parquet schema to and from Flink internal types.
+ */
+public class ParquetSchemaConverter {
+       public static final String MAP_KEY = "key";
+       public static final String MAP_VALUE = "value";
+       public static final String LIST_ARRAY_TYPE = "array";
+       public static final String LIST_ELEMENT = "element";
+       public static final String LIST_GROUP_NAME = "list";
+       public static final String MESSAGE_ROOT = "root";
+       private static final AvroSchemaConverter SCHEMA_CONVERTER = new 
AvroSchemaConverter();
+
+       public static TypeInformation<?> fromParquetType(MessageType type) {
+               return convertFields(type.getFields());
+       }
+
+       /**
+        * Converts Flink Internal Type to Parquet schema.
+        *
+        * @param typeInformation  flink type information
+        * @param isStandard is standard LIST and MAP schema or back-compatible 
schema
+        * @return Parquet schema
+        */
+       public static MessageType toParquetType(TypeInformation<?> 
typeInformation, boolean isStandard) {
+               return (MessageType) convertField(null, typeInformation, 
Type.Repetition.OPTIONAL, isStandard);
+       }
+
+       private static TypeInformation<?> convertFields(List<Type> 
parquetFields) {
+               List<TypeInformation<?>> types = new ArrayList<>();
+               List<String> names = new ArrayList<>();
+               for (Type field : parquetFields) {
+                       TypeInformation<?> subType = convertField(field);
+                       if (subType != null) {
+                               types.add(subType);
+                               names.add(field.getName());
+                       }
+               }
+
+               return new RowTypeInfo(types.toArray(new 
TypeInformation<?>[types.size()]),
+                       names.toArray(new String[names.size()]));
+       }
+
+       private static TypeInformation<?> convertField(final Type fieldType) {
+               TypeInformation<?> typeInfo = null;
+               if (fieldType.isPrimitive()) {
+                       OriginalType originalType = fieldType.getOriginalType();
+                       PrimitiveType primitiveType = 
fieldType.asPrimitiveType();
+                       switch (primitiveType.getPrimitiveTypeName()) {
+                               case BINARY:
+                                       if (originalType != null) {
+                                               switch (originalType) {
+                                                       case DECIMAL:
+                                                               typeInfo = 
BasicTypeInfo.BIG_DEC_TYPE_INFO;
+                                                               break;
+                                                       case UTF8:
+                                                       case ENUM:
+                                                       case JSON:
+                                                       case BSON:
+                                                               typeInfo = 
BasicTypeInfo.STRING_TYPE_INFO;
+                                                               break;
+                                                       default:
+                                                               throw new 
UnsupportedOperationException("Unsupported original type : " + 
originalType.name()
+                                                                       + " for 
primitive type BINARY");
+                                               }
+                                       } else {
+                                               typeInfo = 
BasicTypeInfo.STRING_TYPE_INFO;
+                                       }
+                                       break;
+                               case BOOLEAN:
+                                       typeInfo = 
BasicTypeInfo.BOOLEAN_TYPE_INFO;
+                                       break;
+                               case INT32:
+                                       if (originalType != null) {
+                                               switch (originalType) {
+                                                       case TIME_MICROS:
+                                                       case TIME_MILLIS:
+                                                               typeInfo = 
SqlTimeTypeInfo.TIME;
+                                                               break;
+                                                       case TIMESTAMP_MICROS:
+                                                       case TIMESTAMP_MILLIS:
+                                                               typeInfo = 
SqlTimeTypeInfo.TIMESTAMP;
+                                                               break;
+                                                       case DATE:
+                                                               typeInfo = 
SqlTimeTypeInfo.DATE;
+                                                               break;
+                                                       case UINT_8:
+                                                       case UINT_16:
+                                                       case UINT_32:
+                                                       case INT_8:
 
 Review comment:
   should this be `Types.BYTE`?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> Add ParquetInputFormat
> ----------------------
>
>                 Key: FLINK-7243
>                 URL: https://issues.apache.org/jira/browse/FLINK-7243
>             Project: Flink
>          Issue Type: Sub-task
>          Components: Table API &amp; SQL
>            Reporter: godfrey he
>            Assignee: Zhenqiu Huang
>            Priority: Major
>              Labels: pull-request-available
>
> Add a {{ParquetInputFormat}} to read data from a Apache Parquet file. 



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to