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



##########
File path: 
flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/AbstractAvroBulkFormat.java
##########
@@ -0,0 +1,190 @@
+/*
+ * 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.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.core.fs.FileSystem;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.formats.avro.utils.FSDataInputStreamWrapper;
+
+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;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+
+/** Provides a {@link BulkFormat} for Avro records. */
+public abstract class AbstractAvroBulkFormat<A, T, SplitT extends 
FileSourceSplit>
+        implements BulkFormat<T, SplitT> {
+
+    private static final long serialVersionUID = 1L;
+
+    @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) {}
+
+    abstract T convert(A record);
+
+    abstract A getReusedAvroObject();
+
+    private class AvroReader implements BulkFormat.Reader<T> {
+
+        private final DataFileReader<A> reader;
+        private final A reuse;
+
+        private final long end;
+        private final BlockingQueue<Boolean> blockingQueue;
+
+        private long currentBlockStart;
+        private long currentRecordsToSkip;
+
+        private AvroReader(Path path, long offset, long end, long blockStart, 
long recordsToSkip)
+                throws IOException {
+            this.reader = createReaderFromPath(path);
+            this.reuse = getReusedAvroObject();
+            if (blockStart >= 0) {
+                reader.seek(blockStart);
+            } else {
+                reader.sync(offset);
+            }
+            for (int i = 0; i < recordsToSkip; i++) {
+                reader.next(reuse);
+            }
+
+            this.end = end;
+            this.blockingQueue = new LinkedBlockingQueue<>(1);
+
+            this.currentBlockStart = reader.previousSync();
+            this.currentRecordsToSkip = recordsToSkip;
+        }
+
+        private DataFileReader<A> createReaderFromPath(Path path) throws 
IOException {
+            FileSystem fileSystem = path.getFileSystem();
+            DatumReader<A> datumReader = new GenericDatumReader<>();
+            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 {
+            if (reachEnd()) {
+                return null;
+            }
+
+            try {
+                blockingQueue.put(true);

Review comment:
       `SingleThreadMultiplexSourceReaderBase` indeed reads splits with only 
one thread. But `BulkFormat`s are used in fetching threads, not in reading 
threads. The big picture of a FLIP-27 source is that splits fetching and the 
actual readings are done in separate threads. They form a producer-consumer 
module and communicate with a `FutureCompletingBlockingQueue`. See [this 
code](https://github.com/apache/flink/blob/713f0661d319c920d3cc2ce98b0d791e81673454/flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/source/reader/SingleThreadMultiplexSourceReaderBase.java#L76).
   
   Note that in this PR, the results of an avro block is lazily produced. So 
when a block is enqueued, the contents of that block hasn't been read. In [this 
code](https://github.com/apache/flink/blob/99c2a415e9eeefafacf70762b6f54070f7911ceb/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/src/impl/FileSourceSplitReader.java#L64)
 you can see that the fetch task repeatedly adds blocks to the queue until the 
avro reader states that there is no more blocks. At this time the fetch task 
will close the avro reader without consuming the records. Thus when the file 
connector reader comes and tries to read from avro blocks, an exception stating 
that the avro reader has been closed will be thrown.




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