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



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

Review comment:
       When is offset != 0?

##########
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();
+        }
+    }

Review comment:
       We should inject the model with the factory methods. There we already 
have the knowledge where it's a specific, generic, or reflective type and we 
don't need to infer that here.

##########
File path: 
flink-formats/flink-parquet/src/test/java/org/apache/flink/formats/parquet/avro/Datum.java
##########
@@ -0,0 +1,55 @@
+/*
+ * 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 java.io.Serializable;
+
+/** Test datum. */
+public class Datum implements Serializable {

Review comment:
       Does this even need to be serializable?

##########
File path: 
flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/avro/AvroParquetReaders.java
##########
@@ -0,0 +1,94 @@
+/*
+ * 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.java.typeutils.TypeExtractor;
+import org.apache.flink.formats.avro.typeutils.AvroTypeInfo;
+import org.apache.flink.formats.avro.typeutils.GenericRecordAvroTypeInfo;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.avro.specific.SpecificRecordBase;
+
+/**
+ * Convenience builder to create {@link AvroParquetRecordFormat} instances for 
the different Avro
+ * types.
+ */
+public class AvroParquetReaders {
+
+    /**
+     * Creates a new {@link AvroParquetRecordFormat} that reads the parquet 
file into Avro {@link
+     * org.apache.avro.specific.SpecificRecord SpecificRecords}.
+     *
+     * <p>To read into Avro {@link GenericRecord GenericRecords}, use the 
{@link
+     * #forGenericRecord(Schema)} method.
+     *
+     * @see #forGenericRecord(Schema)
+     */
+    public static <T extends SpecificRecordBase> AvroParquetRecordFormat<T> 
forSpecificRecord(
+            final Class<T> typeClass) {
+        return new AvroParquetRecordFormat<>(new AvroTypeInfo<>(typeClass));
+    }
+
+    /**
+     * Creates a new {@link AvroParquetRecordFormat} that reads the parquet 
file into Avro records
+     * via reflection.
+     *
+     * <p>To read into Avro {@link GenericRecord GenericRecords}, use the 
{@link
+     * #forGenericRecord(Schema)} method.
+     *
+     * <p>To read into Avro {@link org.apache.avro.specific.SpecificRecord 
SpecificRecords}, use the
+     * {@link #forSpecificRecord(Class)} method.
+     *
+     * @see #forGenericRecord(Schema)
+     * @see #forSpecificRecord(Class)
+     */
+    public static <T> AvroParquetRecordFormat<T> forReflectRecord(final 
Class<T> typeClass) {
+        if (SpecificRecordBase.class.isAssignableFrom(typeClass)) {
+            throw new IllegalArgumentException(
+                    "Please use AvroParquetReaders.forSpecificRecord(Class<T>) 
for SpecificRecord.");

Review comment:
       I wonder if we can do it directly and just log a warning. It's not like 
`GenericRecord`, where we simply lack information.

##########
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:
       I'm not fully satisfied with our ability to recover here. So I looked 
into what the AvroParquetReader is doing internally:
   
   ```
               // Injected
               GenericData model = GenericData.get();
               org.apache.hadoop.conf.Configuration conf = new 
org.apache.hadoop.conf.Configuration();
   
               // Low level reader - fetch metadata
               ParquetFileReader reader = null;
               MessageType fileSchema = reader.getFileMetaData().getSchema();
               Map<String, String> metaData = 
reader.getFileMetaData().getKeyValueMetaData();
   
               // init Avro specific things
               AvroReadSupport<T> readSupport = new AvroReadSupport<>(model);
               ReadSupport.ReadContext readContext =
                       readSupport.init(
                               new InitContext(
                                     conf,
                                       metaData.entrySet().stream()
                                               .collect(Collectors.toMap(e -> 
e.getKey(), e -> Collections.singleton(e.getValue()))),
                                       fileSchema));
               RecordMaterializer<T> recordMaterializer = 
readSupport.prepareForRead(conf, metaData, fileSchema, readContext);
               MessageType requestedSchema = readContext.getRequestedSchema();
   
               // prepare record reader
               ColumnIOFactory columnIOFactory = new 
ColumnIOFactory(reader.getFileMetaData().getCreatedBy());
               MessageColumnIO columnIO = 
columnIOFactory.getColumnIO(requestedSchema, fileSchema, true);
   
               // for recovery
               while (...) {
                 reader.skipNextRowGroup();
               }
   
               // for reading
               PageReadStore pages;
               for (int block = 0; (pages = reader.readNextRowGroup()) != null; 
block++) {
                   RecordReader<T> recordReader = 
columnIO.getRecordReader(pages, recordMaterializer);
                   for (int i = 0; i < pages.getRowCount(); i++) {
                       T record = recordReader.read();
                       emit record;
                   }
               }
   ```
   
   Here we can easily track the block. Even better most of that snippet is 
already implemented in `ParquetVectorizedInputFormat`, so we may be able to 
solve both things at once. WDYT?

##########
File path: 
flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/src/reader/StreamFormat.java
##########
@@ -157,6 +166,88 @@
             long splitEnd)
             throws IOException;
 
+    /**
+     * Creates a new reader to read in this format. This method is called when 
a fresh reader is
+     * created for a split that was assigned from the enumerator. This method 
may also be called on
+     * recovery from a checkpoint, if the reader never stored an offset in the 
checkpoint (see
+     * {@link #restoreReader(Configuration, Path, long, long, long)} for 
details.
+     *
+     * <p>Provide the default implementation, subclasses are therefore not 
forced to implement it.
+     * Compare to the {@link #createReader(Configuration, FSDataInputStream, 
long, long)}, This
+     * method put the focus on the {@link Path}. The default implementation 
adapts information given
+     * by method arguments to {@link FSDataInputStream} and calls {@link
+     * #createReader(Configuration, FSDataInputStream, long, long)}.
+     *
+     * <p>If the format is {@link #isSplittable() splittable}, then the {@code 
inputStream} is
+     * positioned to the beginning of the file split, otherwise it will be at 
position zero.
+     */
+    default StreamFormat.Reader<T> createReader(
+            Configuration config, Path filePath, long splitOffset, long 
splitLength)
+            throws IOException {

Review comment:
       Who is invoking the new methods? Is it the framework or the user?

##########
File path: 
flink-formats/flink-parquet/src/test/java/org/apache/flink/formats/parquet/avro/AvroParquetRecordFormatTest.java
##########
@@ -0,0 +1,272 @@
+/*
+ * 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.serialization.BulkWriter;
+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.FileSystem;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.formats.parquet.ParquetWriterFactory;
+import org.apache.flink.formats.parquet.generated.Address;
+
+import org.apache.avro.Schema;
+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.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Objects;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+
+/**
+ * Unit test for {@link AvroParquetRecordFormat} and {@link
+ * org.apache.flink.connector.file.src.reader.StreamFormat}.
+ */
+class AvroParquetRecordFormatTest {

Review comment:
       Good test coverage. Should the ITCase also have 1 test for specific and 
reflective?




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