masteryhx commented on code in PR #24632:
URL: https://github.com/apache/flink/pull/24632#discussion_r1570106555


##########
flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/fs/ByteBufferReadableFSDataInputStream.java:
##########
@@ -0,0 +1,210 @@
+/*
+ * 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.state.forst.fs;
+
+import org.apache.flink.core.fs.ByteBufferReadable;
+import org.apache.flink.core.fs.FSDataInputStream;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Queue;
+import java.util.concurrent.Callable;
+import java.util.concurrent.LinkedBlockingQueue;
+
+/**
+ * A {@link FSDataInputStream} delegates requests to other one and supports 
reading data with {@link
+ * ByteBuffer}.
+ *
+ * <p>All methods in this class maybe used by ForSt, please start a discussion 
firstly if it has to
+ * be modified.
+ */
+public class ByteBufferReadableFSDataInputStream extends FSDataInputStream {
+
+    private final FSDataInputStream originalInputStream;
+
+    /**
+     * InputStream Pool which provides multiple input streams to random read 
concurrently. An input
+     * stream should only be used by a thread at a point in time.
+     */
+    private final Queue<FSDataInputStream> readInputStreamPool;
+
+    private final Callable<FSDataInputStream> inputStreamBuilder;
+
+    public ByteBufferReadableFSDataInputStream(
+            Callable<FSDataInputStream> inputStreamBuilder, int 
inputStreamCapacity)
+            throws IOException {
+        try {
+            this.originalInputStream = inputStreamBuilder.call();
+        } catch (Exception e) {
+            throw new IOException("Exception when build original input 
stream", e);
+        }
+        this.inputStreamBuilder = inputStreamBuilder;
+        this.readInputStreamPool = new 
LinkedBlockingQueue<>(inputStreamCapacity);
+    }
+
+    /**
+     * Reads up to <code>ByteBuffer#remaining</code> bytes of data from the 
input stream into a
+     * ByteBuffer. Not Thread-safe yet since the interface of sequential read 
of ForSt only be
+     * accessed by one thread at a time.
+     *
+     * @param bb the buffer into which the data is read.
+     * @return the total number of bytes read into the buffer.
+     * @exception IOException If the first byte cannot be read for any reason 
other than end of
+     *     file, or if the input stream has been closed, or if some other I/O 
error occurs.
+     * @exception NullPointerException If <code>bb</code> is <code>null</code>.
+     */
+    public int readFully(ByteBuffer bb) throws IOException {
+        if (bb == null) {
+            throw new NullPointerException();
+        } else if (bb.remaining() == 0) {
+            return 0;
+        }
+        return originalInputStream instanceof ByteBufferReadable
+                ? ((ByteBufferReadable) originalInputStream).read(bb)
+                : readFullyFromFSDataInputStream(originalInputStream, bb);
+    }
+
+    /**
+     * Reads up to <code>ByteBuffer#remaining</code> bytes of data from the 
specific position of the
+     * input stream into a ByteBuffer. Thread-safe since the interface of 
random read of ForSt may
+     * be concurrently accessed by multiple threads. TODO: Support to split 
this method to other
+     * class.
+     *
+     * @param position the start offset in input stream at which the data is 
read.
+     * @param bb the buffer into which the data is read.
+     * @return the total number of bytes read into the buffer.
+     * @exception IOException If the first byte cannot be read for any reason 
other than end of
+     *     file, or if the input stream has been closed, or if some other I/O 
error occurs.
+     * @exception NullPointerException If <code>bb</code> is <code>null</code>.
+     */
+    public int readFully(long position, ByteBuffer bb) throws Exception {

Review Comment:
   Thanks for the suggestion.
   Considering current version of ForSt relys on this method, I'd prefer to 
modify this when next version of ForSt is ready. WDYT ?
   I just added TODO on the above method.



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