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



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

Review comment:
       About the pool size. `ParquetVectorizedInputFormat` is an abstract class 
whose only subclass is `ParquetColumnarRowInputFormat`. In 
`ParquetColumnarRowInputFormat` the pool size is constantly set to 1.
   
   The comments in `ParquetColumnarRowInputFormat#numBatchesToCirculate` state 
that
   
   > In a VectorizedColumnBatch, the dictionary will be lazied deserialized. If 
there are multiple batches at the same time, there may be thread safety 
problems, because the deserialization of the dictionary depends on some 
internal structures. We need set numBatchesToCirculate to 1.
   
   This PR is facing the same problem with `ParquetColumnarRowInputFormat`, as 
deserializing avro records also need some internal avro structures.
   
   ---
   
   About the `StreamFormat` draft by Stephan a year ago, I don't know about 
that. I've shown that draft to @JingsongLi and he told me that he and Stephan 
has discussed this draft before and agree that we should not implement avro 
FLIP-27 interfaces based on `StreamForamat`. So it remains as a draft and isn't 
committed.




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