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



##########
File path: 
flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/AbstractAvroBulkFormat.java
##########
@@ -0,0 +1,207 @@
+/*
+ * 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.avro;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connector.file.src.FileSourceSplit;
+import org.apache.flink.connector.file.src.reader.BulkFormat;
+import org.apache.flink.connector.file.src.util.CheckpointedPosition;
+import org.apache.flink.connector.file.src.util.IteratorResultIterator;
+import org.apache.flink.connector.file.src.util.Pool;
+import org.apache.flink.core.fs.FileSystem;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.formats.avro.utils.FSDataInputStreamWrapper;
+
+import org.apache.avro.Schema;
+import org.apache.avro.file.DataFileReader;
+import org.apache.avro.file.SeekableInput;
+import org.apache.avro.generic.GenericDatumReader;
+import org.apache.avro.io.DatumReader;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+import java.util.Iterator;
+
+/** Provides a {@link BulkFormat} for Avro records. */
+@Internal
+public abstract class AbstractAvroBulkFormat<A, T, SplitT extends 
FileSourceSplit>
+        implements BulkFormat<T, SplitT> {
+
+    private static final long serialVersionUID = 1L;
+
+    protected final Schema readerSchema;
+
+    protected AbstractAvroBulkFormat(Schema readerSchema) {
+        this.readerSchema = readerSchema;
+    }
+
+    @Override
+    public AvroReader createReader(Configuration config, SplitT split) throws 
IOException {
+        open(split);
+        return createReader(split);
+    }
+
+    @Override
+    public AvroReader restoreReader(Configuration config, SplitT split) throws 
IOException {
+        open(split);
+        return createReader(split);
+    }
+
+    @Override
+    public boolean isSplittable() {
+        return true;
+    }
+
+    private AvroReader createReader(SplitT split) throws IOException {
+        long end = split.offset() + split.length();
+        if (split.getReaderPosition().isPresent()) {
+            CheckpointedPosition position = split.getReaderPosition().get();
+            return new AvroReader(
+                    split.path(),
+                    split.offset(),
+                    end,
+                    position.getOffset(),
+                    position.getRecordsAfterOffset());
+        } else {
+            return new AvroReader(split.path(), split.offset(), end, -1, 0);
+        }
+    }
+
+    protected void open(SplitT split) {}
+
+    protected abstract T convert(A record);
+
+    protected abstract A createReusedAvroRecord();
+
+    private class AvroReader implements BulkFormat.Reader<T> {
+
+        private final DataFileReader<A> reader;
+
+        private final long end;
+        private final Pool<A> pool;
+
+        private long currentBlockStart;
+        private long currentRecordsToSkip;
+
+        private AvroReader(Path path, long offset, long end, long blockStart, 
long recordsToSkip)
+                throws IOException {
+            A reuse = createReusedAvroRecord();
+
+            this.reader = createReaderFromPath(path);
+            if (blockStart >= 0) {
+                reader.seek(blockStart);
+            } else {
+                reader.sync(offset);
+            }
+            for (int i = 0; i < recordsToSkip; i++) {
+                reader.next(reuse);
+            }
+
+            this.end = end;
+            this.pool = new Pool<>(1);
+            this.pool.add(reuse);
+
+            this.currentBlockStart = reader.previousSync();
+            this.currentRecordsToSkip = recordsToSkip;
+        }
+
+        private DataFileReader<A> createReaderFromPath(Path path) throws 
IOException {
+            FileSystem fileSystem = path.getFileSystem();
+            DatumReader<A> datumReader = new GenericDatumReader<>(null, 
readerSchema);
+            SeekableInput in =
+                    new FSDataInputStreamWrapper(
+                            fileSystem.open(path), 
fileSystem.getFileStatus(path).getLen());
+            return (DataFileReader<A>) DataFileReader.openReader(in, 
datumReader);
+        }
+
+        @Nullable
+        @Override
+        public RecordIterator<T> readBatch() throws IOException {
+            A reuse;
+            try {
+                reuse = pool.pollEntry();
+            } catch (InterruptedException e) {
+                Thread.currentThread().interrupt();
+                throw new RuntimeException(
+                        "Interrupted while waiting for the previous batch to 
be consumed", e);
+            }
+
+            if (!readNextBlock()) {
+                pool.recycler().recycle(reuse);
+                return null;
+            }
+
+            currentBlockStart = reader.previousSync();
+            Iterator<T> iterator =
+                    new AvroBlockIterator(
+                            reader.getBlockCount() - currentRecordsToSkip, 
reader, reuse);
+            long recordsToSkip = currentRecordsToSkip;
+            currentRecordsToSkip = 0;
+            return new IteratorResultIterator<>(
+                    iterator,
+                    currentBlockStart,
+                    recordsToSkip,
+                    () -> pool.recycler().recycle(reuse));

Review comment:
       The design of the new FileSource is to separate the fetch thread and the 
task thread. Let all blocked I/O reading be done in the fetch thread, so that 
the task threads are blocking-free while reading the data. It is recommended to 
make batch objects ready in the fetch thread. Afaik, first of all, this 
implementation builds a iterator which will lazily read data and run I/O 
blocked reading within the task thread. This is against the design of the new 
FileSource. Second, because the pool size is set to 1 and recycle is called 
when `releaseBatch()` is called, which means an iterator with only one element 
is built for each `readBatch()` call. All Flink built-in logic to optimise the 
reading has been bypassed. It might be good to reconsider it. Benchmark should 
be one option.




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