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

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_r224230977
 
 

 ##########
 File path: 
flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/ParquetInputFormat.java
 ##########
 @@ -0,0 +1,191 @@
+/*
+ * 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.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;
+
+/**
+ * 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);
+
+       private transient Counter recordConsumed;
+
+       protected RowTypeInfo readType;
+
+       protected boolean isStandard;
+
+       protected final TypeInformation[] fieldTypes;
+
+       protected final String[] fieldNames;
+
+       protected transient ParquetRecordReader<Row> parquetRecordReader;
+
+       protected transient long recordsReadSinceLastSync;
+
+       protected long lastSyncedBlock = -1L;
+
+       protected ParquetInputFormat(Path path, TypeInformation[] fieldTypes, 
String[] fieldNames, boolean isStandard) {
+               super(path);
+               this.readType = new RowTypeInfo(fieldTypes, fieldNames);
+               this.fieldTypes = readType.getFieldTypes();
+               this.fieldNames = readType.getFieldNames();
+               this.unsplittable = true;
+               this.isStandard = isStandard;
+       }
+
+       @Override
+       public Tuple2<Long, Long> getCurrentState() {
+               return new Tuple2<>(this.lastSyncedBlock, 
this.recordsReadSinceLastSync);
+       }
+
+       @Override
+       public void open(FileInputSplit split) throws IOException {
 
 Review comment:
   Yes, I saw that. I'm just worried what happens if we open a new split and a 
checkpoint is triggered before the first row is read. The chance is small (and 
might not even exist at all if split opening and reading of the first record 
are done in the same synchronized block), but it is also a very simple change 
to guarantee correctness here.

----------------------------------------------------------------
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:
[email protected]


> 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