You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2021/10/27 10:18:46 UTC

[GitHub] [flink] echauchot commented on a change in pull request #17501: [Draft][FLINK-21406][RecordFormat] build AvroParquetRecordFormat for the new FileSource

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



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

Review comment:
       I guess it is only because this PR is a draft, but of course splitting is mandatory in production. If you need some pointers on how to implement split with Parquet, you can take a look at `ParquetColumnarRowSplitReader`.

##########
File path: flink-formats/flink-parquet/pom.xml
##########
@@ -26,7 +26,7 @@ under the License.
 		<groupId>org.apache.flink</groupId>
 		<artifactId>flink-formats</artifactId>
 		<version>1.15-SNAPSHOT</version>
-		<relativePath>..</relativePath>
+		<relativePath>../pom.xml</relativePath>

Review comment:
       ditto

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

Review comment:
       I would avoid creating this abstraction unless it is needed. This class is just a passthrough wrapper to `ParquetReader`. I think it is better to call `ParquetReader` directly 

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

Review comment:
       missing the end of the comment ?

##########
File path: flink-formats/flink-avro/pom.xml
##########
@@ -26,7 +26,7 @@ under the License.
 		<groupId>org.apache.flink</groupId>
 		<artifactId>flink-formats</artifactId>
 		<version>1.15-SNAPSHOT</version>
-		<relativePath>..</relativePath>
+		<relativePath>../pom.xml</relativePath>

Review comment:
       > The batch and stream we are talking here refer to reading data not the execution. I think it is an interesting discussion whether BulkFormat alone is a good fit to run streaming execution with batch read, the latency could be an issue.
   > 
   > For the `RecordFormat`, it is considered from the architecture perspective, because the `FileRecordFormat` and `StreamFormat` are very similar. And currently, more features have been built for StreamFormat than FileRecordFormat, Compression is only used as (wrong) example which is not used for parquet, there could be more others. We should keep an eye on DRY.
   
   I was about to tell that there was indeed a confusion in the discussion between a java stream and streaming mode execution of the pipeline.

##########
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:
       +1 to supporting other types as `GenericRecord` holds avro schema which can be costly in memory use and serialisation/deserialisation.

##########
File path: flink-formats/flink-parquet/pom.xml
##########
@@ -45,6 +45,14 @@ under the License.
 			<scope>provided</scope>
 		</dependency>
 
+		<!-- Flink-avro -->
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-avro</artifactId>
+			<version>${project.version}</version>

Review comment:
       maybe put this dep `<optional>true</optional>` as it was done for the older flink-parquet component (see https://github.com/apache/flink/pull/15156#pullrequestreview-672808431)

##########
File path: flink-formats/flink-avro/pom.xml
##########
@@ -26,7 +26,7 @@ under the License.
 		<groupId>org.apache.flink</groupId>
 		<artifactId>flink-formats</artifactId>
 		<version>1.15-SNAPSHOT</version>
-		<relativePath>..</relativePath>
+		<relativePath>../pom.xml</relativePath>

Review comment:
       
   > In general, all formats should support batch and streaming execution. As an example that `BulkFormat`s are also applicable to streaming executions you can take a look at this docstring [1]. The docstring mentions checkpoints and how the last offset/position is tracked. Checkpointing is not supported in batch execution. The difference between `BulkFormat` and `FileRecordFormat` is how the underlying reader interacts with the filesystem. `BulkFormats` usually always read batches of data i.e. parquet reader always reads blocks/rowgroups as on the other hand `FileRecordFormat` usually reads the file line by line.
   > 
   > After looking through the `AvroParquetReader` I think your assumption is right. We cannot implement a bulk format here because the reader does not expose any information about the underlying block/rowgroup structure.
   
   +1 on what Fabian said. IMHO, I think this avro/parquet format should implement `BulkFormat` directly cf [this discussion](https://lists.apache.org/thread.html/re3a5724ba3e68d63cd2b83d9d14c41cdcb7547e7c46c6c5e5b7aeb73%40%3Cdev.flink.apache.org%3E) we had with Jingsong. Regarding block/rowgroup support with BulkFormat: could it be done by implementing `BulkFormat.Reader#readBatch `?

##########
File path: flink-formats/flink-avro/pom.xml
##########
@@ -26,7 +26,7 @@ under the License.
 		<groupId>org.apache.flink</groupId>
 		<artifactId>flink-formats</artifactId>
 		<version>1.15-SNAPSHOT</version>
-		<relativePath>..</relativePath>
+		<relativePath>../pom.xml</relativePath>

Review comment:
       > Why was this change needed?
   
   as @fapaul I don't think changing the relativePath is needed but maybe I missed something.
   

##########
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:
       nit: please rename to `ParquetAvroRecordFormat` for concistency with existing (Flink 1.13) `ParquetAvroInputFormat`

##########
File path: flink-formats/flink-parquet/src/test/java/org/apache/flink/formats/parquet/avro/AvroParquetRecordFormatTest.java
##########
@@ -0,0 +1,211 @@
+/*
+ * 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.avro.Schema;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.generic.GenericRecord;
+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.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.RecordFormat}.
+ */
+class AvroParquetRecordFormatTest {
+
+    private static final String USER_PARQUET_FILE = "user.parquet";
+
+    private static Path path;
+    private static Schema schema;
+    private static List<GenericRecord> records = new ArrayList<>(3);
+
+    @TempDir static java.nio.file.Path temporaryFolder;
+
+    /**
+     * Create a parquet file in the {@code TEMPORARY_FOLDER} directory.
+     *
+     * @throws IOException if new file can not be created.
+     */
+    @BeforeAll
+    static void setup() throws IOException {
+        schema =
+                new Schema.Parser()
+                        .parse(
+                                "{\"type\": \"record\", "
+                                        + "\"name\": \"User\", "
+                                        + "\"fields\": [\n"
+                                        + "        {\"name\": \"name\", \"type\": \"string\" },\n"
+                                        + "        {\"name\": \"favoriteNumber\",  \"type\": [\"int\", \"null\"] },\n"
+                                        + "        {\"name\": \"favoriteColor\", \"type\": [\"string\", \"null\"] }\n"
+                                        + "    ]\n"
+                                        + "    }");
+
+        records.add(createUser("Peter", 1, "red"));
+        records.add(createUser("Tom", 2, "yellow"));
+        records.add(createUser("Jack", 3, "green"));
+
+        path = new Path(temporaryFolder.resolve(USER_PARQUET_FILE).toUri());
+
+        ParquetWriterFactory<GenericRecord> writerFactory =
+                ParquetAvroWriters.forGenericRecord(schema);
+        BulkWriter<GenericRecord> writer =
+                writerFactory.create(
+                        path.getFileSystem().create(path, FileSystem.WriteMode.OVERWRITE));
+
+        for (GenericRecord record : records) {
+            writer.addElement(record);
+        }
+
+        writer.flush();
+        writer.finish();
+    }
+
+    @Test
+    void testCreateReader() throws IOException {
+        StreamFormat.Reader<GenericRecord> reader =
+                new AvroParquetRecordFormat(schema)
+                        .createReader(
+                                new Configuration(),
+                                path,
+                                0,
+                                path.getFileSystem().getFileStatus(path).getLen());
+        for (GenericRecord record : records) {
+            assertUserEquals(Objects.requireNonNull(reader.read()), record);
+        }
+    }
+
+    /** Expect exception since splitting is not supported now. */
+    @Test
+    void testCreateReaderWithSplitting() {
+        assertThrows(
+                IllegalArgumentException.class,
+                () ->
+                        new AvroParquetRecordFormat(schema)
+                                .createReader(new Configuration(), path, 5, 5));
+    }
+
+    @Test
+    void testCreateReaderWithNullPath() {
+        assertThrows(
+                NullPointerException.class,
+                () ->
+                        new AvroParquetRecordFormat(schema)
+                                .createReader(new Configuration(), (Path) null, 0, 0));
+    }
+
+    @Test
+    void testRestoreReaderWithNoOffset() {
+        assertThrows(
+                IllegalArgumentException.class,
+                () ->
+                        new AvroParquetRecordFormat(schema)
+                                .restoreReader(
+                                        new Configuration(),
+                                        path,
+                                        CheckpointedPosition.NO_OFFSET,
+                                        0,
+                                        path.getFileSystem().getFileStatus(path).getLen()));
+    }
+
+    @Test
+    void testRestoreReader() throws IOException {

Review comment:
       nit: please rename to `testReadWithRestoredReader`




-- 
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-unsubscribe@flink.apache.org

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