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



##########
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:
       > why `StreamFormatAdapter` doesn't have any sync primitive in its 
`readBatch` method
   
   This depends on the implementation of each specific format.
   
   `StreamFormatAdapter` has extracted all records in a batch into memory once 
the batch is produced, so it is ok for the reader to close before the batch is 
consumed because the records are already there and there is no need for a 
reader anymore.
   
   This PR, on the other hand, does not extract all records into memory when a 
batch is produced. It only stores the iterator. To iterate through the batch 
when consuming, the reader is still needed. `ParquetColumnarRowInputFormat` 
also takes this method, see 
[here](https://github.com/apache/flink/blob/413ff6ae7e7a1bd6c5fe495a141f02d383165776/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/ParquetColumnarRowInputFormat.java#L98).
   
   > each `readBatch` reuses the same instance of `FSDataInputStream`, which is 
definitely not suitable for multithreaded usage
   
   I think the "reader" here is sort of misleading. I'll now refer to the 
fetcher in FLIP-27 as "producer", the reader (NOT avro reader) in FLIP-27 as 
"consumer". Avro reader only lives in producers and not in consumers.
   
   That `FSDataInputStream` is only used for that file split, while each file 
split is only handled by a single producer, so there is no multi-threaded 
problem here. But producers and consumers are in different threads. The same 
`FSDataInputStream` is handled by the same producer, and the results are given 
to some consumers. So it is true that reading is done sequentially for files.
   
   FLIP-27 separates producers and consumers to prevent checkpoint blockage if 
it takes too long to read from an external system.
   
   ---
   
   I think the best way to understand this problem is to remove this blocking 
queue. Add some outputs at the beginning of 
`AbstractAvroBulkFormat.AvroReader#readBatch`, 
`AbstractAvroBulkFormat.AvroReader#close` and the `hasNext` or `next` method of 
the iterator in `AbstractAvroBulkFormat.AvroReader#readBatch`. You'll see that 
avro reader is closed before all records are consumed and an 
`AsyncClosingException` will be thrown when trying to read records from the 
batch.




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