[
https://issues.apache.org/jira/browse/FLINK-7243?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16645671#comment-16645671
]
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_r224248653
##########
File path:
flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/ParquetInputFormat.java
##########
@@ -0,0 +1,241 @@
+/*
+ * 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;
+
+ private final TypeInformation[] fieldTypes;
+
+ private final String[] fieldNames;
+
+ private boolean skipThisSplit = false;
+
+ private transient ParquetRecordReader<Row> parquetRecordReader;
+
+ private transient long recordsReadSinceLastSync;
+
+ private long lastSyncedBlock = -1L;
+
+ /**
+ * Read parquet files with given result parquet schema.
+ *
+ * @param path The path of the file to read.
+ * @param messageType schema of read result
+ */
+
+ protected ParquetInputFormat(Path path, MessageType messageType) {
+ super(path);
+ RowTypeInfo readType = (RowTypeInfo)
ParquetSchemaConverter.fromParquetType(messageType);
+ this.fieldTypes = readType.getFieldTypes();
+ this.fieldNames = readType.getFieldNames();
+ // read whole parquet file as one file split
+ this.unsplittable = true;
+ }
+
+ /**
+ * Read parquet files with given result field names and types.
+ *
+ * @param path The path of the file to read.
+ * @param fieldTypes field types of read result of fields
+ * @param fieldNames field names to read, which can be subset of the
parquet schema
+ */
+ protected ParquetInputFormat(Path path, TypeInformation[] fieldTypes,
String[] fieldNames) {
+ super(path);
+ this.fieldTypes = fieldTypes;
+ this.fieldNames = fieldNames;
+ // read whole parquet file as one file split
+ this.unsplittable = true;
+ }
+
+ @Override
+ public Tuple2<Long, Long> getCurrentState() {
+ return new Tuple2<>(this.lastSyncedBlock,
this.recordsReadSinceLastSync);
+ }
+
+ @Override
+ public void open(FileInputSplit split) throws IOException {
+ org.apache.hadoop.conf.Configuration configuration = new
org.apache.hadoop.conf.Configuration();
+ InputFile inputFile =
+ HadoopInputFile.fromPath(new
org.apache.hadoop.fs.Path(split.getPath().toUri()), configuration);
+ ParquetReadOptions options =
ParquetReadOptions.builder().build();
+ ParquetFileReader fileReader = new ParquetFileReader(inputFile,
options);
+ MessageType schema = fileReader.getFileMetaData().getSchema();
+ this.skipThisSplit = false;
+ MessageType readSchema = getReadSchema(schema);
+ this.parquetRecordReader = new ParquetRecordReader<>(new
RowReadSupport(), readSchema, FilterCompat.NOOP);
Review comment:
Wouldn't it make more sense to have different `ReadSupport` implementations
for the different target types (Map, Pojo, Row), instead of always reading as
`Row` and from there converting to the other types?
----------------------------------------------------------------
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 & 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)