JingGe commented on a change in pull request #17501:
URL: https://github.com/apache/flink/pull/17501#discussion_r765238437



##########
File path: 
flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/avro/AvroParquetRecordFormat.java
##########
@@ -0,0 +1,236 @@
+/*
+ * 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.avro;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connector.file.src.reader.StreamFormat;
+import org.apache.flink.connector.file.src.util.CheckpointedPosition;
+import org.apache.flink.core.fs.FSDataInputStream;
+
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.avro.reflect.ReflectData;
+import org.apache.avro.specific.SpecificData;
+import org.apache.parquet.avro.AvroParquetReader;
+import org.apache.parquet.hadoop.ParquetReader;
+import org.apache.parquet.io.DelegatingSeekableInputStream;
+import org.apache.parquet.io.InputFile;
+import org.apache.parquet.io.SeekableInputStream;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+
+/** */
+public class AvroParquetRecordFormat<E> implements StreamFormat<E> {
+
+    private static final long serialVersionUID = 1L;
+
+    static final Logger LOG = 
LoggerFactory.getLogger(AvroParquetRecordFormat.class);
+
+    private final TypeInformation<E> type;
+
+    AvroParquetRecordFormat(TypeInformation<E> type) {
+        this.type = type;
+    }
+
+    /**
+     * Creates a new reader to read avro {@link GenericRecord} from Parquet 
input stream.
+     *
+     * <p>Several wrapper classes haven be created to Flink abstraction become 
compatible with the
+     * parquet abstraction. Please refer to the inner classes {@link 
AvroParquetRecordReader},
+     * {@link ParquetInputFile}, {@link FSDataInputStreamAdapter} for details.
+     */
+    @Override
+    public Reader<E> createReader(
+            Configuration config, FSDataInputStream stream, long fileLen, long 
splitEnd)
+            throws IOException {
+
+        // current version does not support splitting.
+        checkNotSplit(fileLen, splitEnd);
+
+        return new AvroParquetRecordReader<E>(
+                AvroParquetReader.<E>builder(new ParquetInputFile(stream, 
fileLen))
+                        .withDataModel(getDataModel())
+                        .build());
+    }
+
+    /**
+     * Restores the reader from a checkpointed position. It is in fact 
identical since only {@link
+     * CheckpointedPosition#NO_OFFSET} as the {@code restoredOffset} is 
support.
+     */
+    @Override
+    public Reader<E> restoreReader(
+            Configuration config,
+            FSDataInputStream stream,
+            long restoredOffset,
+            long fileLen,
+            long splitEnd)
+            throws IOException {
+
+        // current version does not support splitting.
+        checkNotSplit(fileLen, splitEnd);
+
+        checkArgument(
+                restoredOffset == CheckpointedPosition.NO_OFFSET,
+                "The restoredOffset should always be NO_OFFSET");
+
+        return createReader(config, stream, fileLen, splitEnd);
+    }
+
+    @VisibleForTesting
+    GenericData getDataModel() {
+        Class<E> typeClass = getProducedType().getTypeClass();
+        if 
(org.apache.avro.specific.SpecificRecordBase.class.isAssignableFrom(typeClass)) 
{
+            return SpecificData.get();
+        } else if 
(org.apache.avro.generic.GenericRecord.class.isAssignableFrom(typeClass)) {
+            return GenericData.get();
+        } else {
+            return ReflectData.get();
+        }
+    }
+
+    /** Current version does not support splitting. */
+    @Override
+    public boolean isSplittable() {
+        return false;
+    }
+
+    /**
+     * Gets the type produced by this format. This type will be the type 
produced by the file source
+     * as a whole.
+     */
+    @Override
+    public TypeInformation<E> getProducedType() {
+        return type;
+    }
+
+    private static void checkNotSplit(long fileLen, long splitEnd) {
+        if (splitEnd != fileLen) {
+            throw new IllegalArgumentException(
+                    String.format(
+                            "Current version of AvroParquetRecordFormat is not 
splittable, "
+                                    + "but found split end (%d) different from 
file length (%d)",
+                            splitEnd, fileLen));
+        }
+    }
+
+    /**
+     * {@link StreamFormat.Reader} implementation. Using {@link ParquetReader} 
internally to read
+     * avro {@link GenericRecord} from parquet {@link InputFile}.
+     */
+    private static class AvroParquetRecordReader<E> implements 
StreamFormat.Reader<E> {
+
+        private final ParquetReader<E> parquetReader;
+
+        private final long offset;
+        private long skipCount;
+        private final boolean checkpointed;
+
+        private AvroParquetRecordReader(ParquetReader<E> parquetReader) {
+            this(parquetReader, CheckpointedPosition.NO_OFFSET, 0, false);
+        }
+
+        private AvroParquetRecordReader(
+                ParquetReader<E> parquetReader, long offset, long skipCount, 
boolean checkpointed) {
+            this.parquetReader = parquetReader;
+            this.offset = offset;
+            this.skipCount = skipCount;
+            this.checkpointed = checkpointed;
+        }
+
+        @Nullable
+        @Override
+        public E read() throws IOException {
+            E record = parquetReader.read();
+            incrementPosition();
+            return record;
+        }
+
+        @Override
+        public void close() throws IOException {
+            parquetReader.close();
+        }
+
+        @Nullable
+        @Override
+        public CheckpointedPosition getCheckpointedPosition() {
+            return checkpointed ? new CheckpointedPosition(offset, skipCount) 
: null;

Review comment:
       Thanks for your effort for researching and providing the details code 
logics. Yes, using the low level API ParquetFileReader is another option I've 
considered and finally got the feeling that it'd be better to use with 
`BulkFormat` directly like 'ParquetVectorizedInputFormat' did instead of with 
`StreamFormat` for the following reasons:
   
   -  the read logic is built in the internal low level class 
`InternalParquetRecordReader` with package private visibility in parquet-hadoop 
lib which uses another low level class `ParquetFileReader` internally. This 
makes the implementation of StreamFormat very complicated. I think the design 
idea of StreamFormat is to simplify the implementation. They do not seem to 
work together.
   
   -  `ParquetFileReader`reads data in batch mode, i.e. `PageReadStore pages = 
reader.readNextFilteredRowGroup();`. If we build these logic into 
StreamFormat(`AvroParquetRecordFormat` in this case), `AvroParquetRecordFormat` 
has to take over the role `InternalParquetRecordReader` does, including but not 
limited to 
         1. read `PageReadStore` in batch mode. 
         2. manage `PageReadStore`, i.e. read next page when all records in the 
current page have been consumed and cache it. 
         3. manage the read index within the current `PageReadStore` because 
StreamFormat has its own setting for read size, etc. All of these make 
`AvroParquetRecordFormat` become the `BulkFormat` instead of `StreamFormat`  
   
   - `StreamFormat` can only be used via `StreamFormatAdapter`, which means 
everything we will do with the low level APIs for parquet-hadoop lib should 
have not conflict with the built-in logic provided by `StreamFormatAdapter`.
   
   Now we could see if we build these logics into a `StreamFormat` 
implementation, i.e. `AvroParquetRecordFormat`, all convenient built-in logic 
provided by the `StreamFormatAdapter` turns into obstacles. There is also a 
violation of single responsibility principle, i.e. 
`AvroParquetRecordFormat`will take some responsibility of `BulkFormat`. I guess 
this were the reasons why 'ParquetVectorizedInputFormat' implemented 
`BulkFormat` instead of `StreamFormat`.
   
   In order to build a unified parquet implementation for both Table API and 
DataStream API, it makes more sense to consider building these code into a 
`BulkFormat` implementation class. Speaking of "solve both things at once", 
since the output data types are different, `RowData` vs. `Avro`, extra 
converter logic should be introduced into the architecture design. This is 
beyond the scope of this PR. I would suggest to open another ticket to focus on 
it. Depending on how complicated the issue will be and how big the impact it 
will have on the current code base, a new FLIP might be required.
   
   Current implementation follows the design idea of `StreamFormat` and keep 
everything on the high level and simple. It is therefore easy to implement and 
easy for user to understand. It is a good fit for simple use cases. It has no 
conflict to the other solution based on the code mentioned above teams up with 
`BulkFormat`. 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


Reply via email to