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

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_r234579748
 
 

 ##########
 File path: 
flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/ParquetInputFormat.java
 ##########
 @@ -0,0 +1,283 @@
+/*
+ * 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;
+
+import org.apache.flink.api.common.io.CheckpointableInputFormat;
+import org.apache.flink.api.common.io.FileInputFormat;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.typeutils.RowTypeInfo;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.core.fs.FileInputSplit;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.formats.parquet.utils.ParquetRecordReader;
+import org.apache.flink.formats.parquet.utils.ParquetSchemaConverter;
+import org.apache.flink.formats.parquet.utils.RowReadSupport;
+import org.apache.flink.metrics.Counter;
+import org.apache.flink.types.Row;
+import org.apache.flink.util.Preconditions;
+
+import org.apache.parquet.ParquetReadOptions;
+import org.apache.parquet.filter2.compat.FilterCompat;
+import org.apache.parquet.hadoop.ParquetFileReader;
+import org.apache.parquet.hadoop.util.HadoopInputFile;
+import org.apache.parquet.io.InputFile;
+import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.Type;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * The base InputFormat class to read from Parquet files.
+ * For specific return types the {@link #convert(Row)} method need to be 
implemented.
+ *
+ * <P>Using {@link ParquetRecordReader} to read files instead of {@link 
org.apache.flink.core.fs.FSDataInputStream},
+ * we override {@link #open(FileInputSplit)} and {@link #close()} to change 
the behaviors.
+ *
+ * @param <E> The type of record to read.
+ */
+public abstract class ParquetInputFormat<E>
+       extends FileInputFormat<E>
+       implements CheckpointableInputFormat<FileInputSplit, Tuple2<Long, 
Long>> {
+
+       private static final long serialVersionUID = 1L;
+
+       private static final Logger LOG = 
LoggerFactory.getLogger(ParquetInputFormat.class);
+
+       /**
+        * The flag to specify whether to skip file splits with wrong schema.
+        */
+       private boolean skipWrongSchemaFileSplit = false;
+
+       /**
+        * The flag to specify whether to skip corrupted record.
+        */
+       private boolean skipCorruptedRecord = false;
+
+       private transient Counter recordConsumed;
+
+       private transient MessageType expectedFileSchema;
+
+       private TypeInformation[] fieldTypes;
+
+       private String[] fieldNames;
+
+       private boolean skipThisSplit = false;
+
+       private transient ParquetRecordReader<Row> parquetRecordReader;
+
+
+       /**
+        * Read parquet files with given parquet file schema.
+        *
+        * @param path The path of the file to read.
+        * @param messageType schema of parquet file
+        */
+
+       protected ParquetInputFormat(Path path, MessageType messageType) {
+               super(path);
+               this.expectedFileSchema = checkNotNull(messageType, 
"messageType");
+               RowTypeInfo rowTypeInfo = (RowTypeInfo) 
ParquetSchemaConverter.fromParquetType(expectedFileSchema);
+               this.fieldTypes = rowTypeInfo.getFieldTypes();
+               this.fieldNames = rowTypeInfo.getFieldNames();
+               // read whole parquet file as one file split
+               this.unsplittable = true;
+       }
+
+       @Override
+       public void configure(Configuration parameters) {
+               super.configure(parameters);
+
+               if (!this.skipWrongSchemaFileSplit) {
+                       this.skipWrongSchemaFileSplit = 
parameters.getBoolean(SKIP_WRONG_SCHEMA_SPLITS, false);
+               }
+
+               if (this.skipCorruptedRecord) {
+                       this.skipCorruptedRecord = 
parameters.getBoolean(SKIP_CORRUPTED_RECORD, false);
+               }
+
+               
parquetRecordReader.setSkipCorruptedRecord(this.skipCorruptedRecord);
+       }
+
+       public void selectFields(String[] fieldNames) {
 
 Review comment:
   Add JavaDocs, e.g., "Configures the fields to be read and returned by the 
`ParquetInputFormat`. Selected fields must be present in the configured schema."

----------------------------------------------------------------
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