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


##########
flink-filesystems/flink-hadoop-fs/src/main/java/org/apache/flink/runtime/fs/hdfs/HadoopDataInputStream.java:
##########
@@ -140,4 +144,43 @@ public void skipFully(long bytes) throws IOException {
             bytes -= fsDataInputStream.skip(bytes);
         }
     }
+
+    @Override
+    public int read(ByteBuffer byteBuffer) throws IOException {
+        // Not all internal stream supports ByteBufferReadable
+        if 
(fsDataInputStream.hasCapability(StreamCapabilities.READBYTEBUFFER)) {
+            return fsDataInputStream.read(byteBuffer);
+        } else {
+            // Fallback to read byte then put
+            int c = read();
+            if (c == -1) {
+                return -1;
+            }
+            byteBuffer.put((byte) c);
+
+            int n = 1, len = byteBuffer.remaining() + 1;
+            for (; n < len; n++) {
+                c = read();
+                if (c == -1) {
+                    break;
+                }
+                byteBuffer.put((byte) c);
+            }
+            return n;
+        }
+    }
+
+    @Override
+    public int read(long position, ByteBuffer byteBuffer) throws IOException {
+        // Not all internal stream supports ByteBufferPositionedReadable
+        if 
(fsDataInputStream.hasCapability(StreamCapabilities.PREADBYTEBUFFER)) {
+            return fsDataInputStream.read(position, byteBuffer);
+        } else {
+            // Fallback to positionable read bytes then put
+            byte[] tmp = new byte[byteBuffer.remaining()];

Review Comment:
   Even for this fallback code path, there still be a possible way to optimize 
a little bit. e.g.:
   ```
   if (byteBuffer.hasArray()) {
       fsDataInputStream.readFully(position, byteBuffer.array(), 
byteBuffer.arrayOffset(), byteBuffer.remaining());
   }
   
   ```



##########
flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/fs/ByteBufferReadableFSDataInputStream.java:
##########
@@ -0,0 +1,195 @@
+/*
+ * 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.FSDataInputStream;
+import org.apache.flink.core.fs.PositionedReadable;
+
+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(
+            FSDataInputStream originalInputStream,
+            Callable<FSDataInputStream> inputStreamBuilder,
+            int inputStreamCapacity) {
+        this.originalInputStream = originalInputStream;
+        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 readFullyFromFSDataInputStream(originalInputStream, bb);
+    }
+
+    /**
+     * Reads up to <code>ByteBuffer#remaining</code> bytes of data from the 
specific position of the
+     * input stream into a ByteBuffer. Tread-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 {
+        if (bb == null) {
+            throw new NullPointerException();
+        } else if (bb.remaining() == 0) {
+            return 0;
+        }
+
+        FSDataInputStream fsDataInputStream = readInputStreamPool.poll();
+        if (fsDataInputStream == null) {
+            fsDataInputStream = inputStreamBuilder.call();

Review Comment:
   I'd be fine with no hard limit control. It concurrent read a common case? 
Otherwise I'm thinking is it better to make `DEFAULT_INPUT_STREAM_CAPACITY` as 
`1`?



##########
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:
   I'd suggest naming this `read`. `readFully` means you must read bytes that 
can fully fill the buffer otherwise you should throw an exception, and no 
return value. While `read` has no that constraint and could return a read count.



##########
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:
   Same applied to other related methods.



##########
flink-filesystems/flink-hadoop-fs/src/main/java/org/apache/flink/runtime/fs/hdfs/HadoopDataInputStream.java:
##########
@@ -140,4 +144,43 @@ public void skipFully(long bytes) throws IOException {
             bytes -= fsDataInputStream.skip(bytes);
         }
     }
+
+    @Override
+    public int read(ByteBuffer byteBuffer) throws IOException {
+        // Not all internal stream supports ByteBufferReadable
+        if 
(fsDataInputStream.hasCapability(StreamCapabilities.READBYTEBUFFER)) {
+            return fsDataInputStream.read(byteBuffer);
+        } else {
+            // Fallback to read byte then put

Review Comment:
   Why is this fallback part different from the one in `public int read(long 
position, ByteBuffer byteBuffer)` 



##########
flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/fs/ByteBufferWritableFSDataOutputStream.java:
##########
@@ -0,0 +1,105 @@
+/*
+ * 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.FSDataOutputStream;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+/**
+ * A {@link FSDataOutputStream} delegates requests to other one and supports 
writing 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 ByteBufferWritableFSDataOutputStream extends FSDataOutputStream {
+
+    private final FSDataOutputStream originalOutputStream;
+
+    public ByteBufferWritableFSDataOutputStream(FSDataOutputStream 
originalOutputStream) {
+        this.originalOutputStream = originalOutputStream;
+    }
+
+    /**
+     * Writes <code>ByteBuffer#remaining</code> bytes from the ByteBuffer to 
this output stream. Not
+     * Thread-safe yet since the interface of write of ForSt only be accessed 
by one thread at a
+     * time.
+     *
+     * <p>If <code>bb</code> is <code>null</code>, a 
<code>NullPointerException</code> is thrown.
+     *
+     * @exception IOException if an I/O error occurs. In particular, an 
<code>IOException</code> is

Review Comment:
   ```suggestion
        * @throws IOException if an I/O error occurs. In particular, an 
<code>IOException</code> is
   ```



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