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



##########
File path: 
flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/avro/AvroParquetRecordFormat.java
##########
@@ -0,0 +1,189 @@
+/*
+ * 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.api.common.typeinfo.TypeInformation;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connector.file.src.reader.RecordFormat;
+import org.apache.flink.core.fs.FSDataInputStream;
+import org.apache.flink.formats.avro.typeutils.GenericRecordAvroTypeInfo;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.generic.GenericRecord;
+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 javax.annotation.Nullable;
+
+import java.io.IOException;
+
+/** */
+public class AvroParquetRecordFormat implements RecordFormat<GenericRecord> {

Review comment:
       Can we also support custom types or only `GenericRecord`?

##########
File path: 
flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/avro/AvroParquetRecordFormat.java
##########
@@ -0,0 +1,189 @@
+/*
+ * 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.api.common.typeinfo.TypeInformation;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connector.file.src.reader.RecordFormat;
+import org.apache.flink.core.fs.FSDataInputStream;
+import org.apache.flink.formats.avro.typeutils.GenericRecordAvroTypeInfo;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.generic.GenericRecord;
+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 javax.annotation.Nullable;
+
+import java.io.IOException;
+
+/** */
+public class AvroParquetRecordFormat implements RecordFormat<GenericRecord> {
+
+    private final transient Schema schema;
+
+    public AvroParquetRecordFormat(Schema schema) {
+        this.schema = schema;
+    }
+
+    /**
+     * 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 
GenericRecordReader}, {@link
+     * ParquetInputFile}, {@link FSDataInputStreamAdapter} for details.
+     */
+    @Override
+    public Reader<GenericRecord> createReader(
+            Configuration config, FSDataInputStream stream, long fileLen, long 
splitEnd)
+            throws IOException {
+
+        // current version does not support splitting.
+        checkNotSplit(fileLen, splitEnd);
+
+        return new GenericRecordReader(
+                AvroParquetReader.<GenericRecord>builder(new 
ParquetInputFile(stream, fileLen))
+                        .withDataModel(GenericData.get())
+                        .build());
+    }
+
+    /**
+     * Restores the reader from a checkpointed position. Since current version 
does not support
+     * splitting,
+     */
+    @Override
+    public Reader<GenericRecord> restoreReader(
+            Configuration config,
+            FSDataInputStream stream,
+            long restoredOffset,
+            long fileLen,
+            long splitEnd)
+            throws IOException {
+
+        // current version does not support splitting.
+        checkNotSplit(fileLen, splitEnd);
+
+        // current version just ignore the splitOffset and use restoredOffset
+        stream.seek(restoredOffset);
+
+        return createReader(config, stream, fileLen, splitEnd);
+    }
+
+    /** Current version does not support splitting. */
+    @Override
+    public boolean isSplittable() {
+        return false;

Review comment:
       Do you plan to support the splits later? I saw that you can pass a file 
range. 
   
   ```java
                   AvroParquetReader.<GenericRecord>builder(new 
ParquetInputFile(stream, fileLen))
                           .withFileRange()
   ```
   
   




-- 
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: issues-unsubscr...@flink.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Reply via email to