Re: [PR] [FLINK-35045][state] Introduce ForStFlinkFileSystem to support reading and writing with ByteBuffer [flink]

2024-04-19 Thread via GitHub


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


##
flink-filesystems/flink-hadoop-fs/src/main/java/org/apache/flink/runtime/fs/hdfs/HadoopDataInputStream.java:
##
@@ -140,4 +144,56 @@ 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 {
+if (byteBuffer.hasArray()) {
+int len = byteBuffer.remaining();
+fsDataInputStream.readFully(byteBuffer.array(), 
byteBuffer.arrayOffset(), len);
+return len;
+} 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)) {

Review Comment:
   Hello!  Unfortunately this causes a compile error when using Hadoop 3.2.3 
(found in the nightly builds).  I created  FLINK-35175 to address this.



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



Re: [PR] [FLINK-35045][state] Introduce ForStFlinkFileSystem to support reading and writing with ByteBuffer [flink]

2024-04-18 Thread via GitHub


masteryhx closed pull request #24632: [FLINK-35045][state] Introduce 
ForStFlinkFileSystem to support reading and writing with ByteBuffer
URL: https://github.com/apache/flink/pull/24632


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



Re: [PR] [FLINK-35045][state] Introduce ForStFlinkFileSystem to support reading and writing with ByteBuffer [flink]

2024-04-18 Thread via GitHub


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


##
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}.
+ *
+ * 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 readInputStreamPool;
+
+private final Callable inputStreamBuilder;
+
+public ByteBufferReadableFSDataInputStream(
+FSDataInputStream originalInputStream,
+Callable inputStreamBuilder,
+int inputStreamCapacity) {
+this.originalInputStream = originalInputStream;
+this.inputStreamBuilder = inputStreamBuilder;
+this.readInputStreamPool = new 
LinkedBlockingQueue<>(inputStreamCapacity);
+}
+
+/**
+ * Reads up to ByteBuffer#remaining 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 bb is null.
+ */
+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 ByteBuffer#remaining 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 bb is null.
+ */
+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 think concurrent read may be a common case which is controlled in ForSt 
random read.
   I'd prefer to remain this currently, maybe we could test it later and give a 
better default value for it, WDYT?



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

Re: [PR] [FLINK-35045][state] Introduce ForStFlinkFileSystem to support reading and writing with ByteBuffer [flink]

2024-04-18 Thread via GitHub


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}.
+ *
+ * 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 readInputStreamPool;
+
+private final Callable inputStreamBuilder;
+
+public ByteBufferReadableFSDataInputStream(
+Callable 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 ByteBuffer#remaining 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 bb is null.
+ */
+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 ByteBuffer#remaining 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 bb is null.
+ */
+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 

Re: [PR] [FLINK-35045][state] Introduce ForStFlinkFileSystem to support reading and writing with ByteBuffer [flink]

2024-04-18 Thread via GitHub


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


##
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:
   Since position read may fallback to just use the method of position read 
with extra byte array in hadoop, I'd prefer to implement this logic without 
extra array overhead.



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



Re: [PR] [FLINK-35045][state] Introduce ForStFlinkFileSystem to support reading and writing with ByteBuffer [flink]

2024-04-17 Thread via GitHub


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}.
+ *
+ * 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 readInputStreamPool;
+
+private final Callable inputStreamBuilder;
+
+public ByteBufferReadableFSDataInputStream(
+FSDataInputStream originalInputStream,
+Callable inputStreamBuilder,
+int inputStreamCapacity) {
+this.originalInputStream = originalInputStream;
+this.inputStreamBuilder = inputStreamBuilder;
+this.readInputStreamPool = new 
LinkedBlockingQueue<>(inputStreamCapacity);
+}
+
+/**
+ * Reads up to ByteBuffer#remaining 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 bb is null.
+ */
+public int readFully(ByteBuffer bb) throws 

Re: [PR] [FLINK-35045][state] Introduce ForStFlinkFileSystem to support reading and writing with ByteBuffer [flink]

2024-04-17 Thread via GitHub


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


##
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}.
+ *
+ * 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 readInputStreamPool;
+
+private final Callable inputStreamBuilder;
+
+public ByteBufferReadableFSDataInputStream(
+FSDataInputStream originalInputStream,
+Callable inputStreamBuilder,
+int inputStreamCapacity) {
+this.originalInputStream = originalInputStream;
+this.inputStreamBuilder = inputStreamBuilder;
+this.readInputStreamPool = new 
LinkedBlockingQueue<>(inputStreamCapacity);
+}
+
+/**
+ * Reads up to ByteBuffer#remaining 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 bb is null.
+ */
+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 ByteBuffer#remaining 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 bb is null.
+ */
+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();
+}
+
+int result;
+if (fsDataInputStream instanceof PositionedReadable) {
+byte[] tmp = new byte[bb.remaining()];
+((PositionedReadable) fsDataInputStream).readFully(position, tmp, 
0, tmp.length);
+bb.put(tmp);
+result = tmp.length;
+} else {
+fsDataInputStream.seek(position);
+result = 

Re: [PR] [FLINK-35045][state] Introduce ForStFlinkFileSystem to support reading and writing with ByteBuffer [flink]

2024-04-17 Thread via GitHub


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


##
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}.
+ *
+ * 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 readInputStreamPool;
+
+private final Callable inputStreamBuilder;
+
+public ByteBufferReadableFSDataInputStream(
+FSDataInputStream originalInputStream,
+Callable inputStreamBuilder,
+int inputStreamCapacity) {
+this.originalInputStream = originalInputStream;
+this.inputStreamBuilder = inputStreamBuilder;
+this.readInputStreamPool = new 
LinkedBlockingQueue<>(inputStreamCapacity);
+}
+
+/**
+ * Reads up to ByteBuffer#remaining 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 bb is null.
+ */
+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 ByteBuffer#remaining 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 bb is null.
+ */
+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:
   `take()` should be called if we want to control it strictly.
   I am thinking whether we should block or just create a new one to use 
temporarily if inputStreamCapacity is not enough, WDYT ?



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

Re: [PR] [FLINK-35045][state] Introduce ForStFlinkFileSystem to support reading and writing with ByteBuffer [flink]

2024-04-17 Thread via GitHub


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


##
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}.
+ *
+ * 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 readInputStreamPool;
+
+private final Callable inputStreamBuilder;
+
+public ByteBufferReadableFSDataInputStream(
+FSDataInputStream originalInputStream,
+Callable inputStreamBuilder,
+int inputStreamCapacity) {
+this.originalInputStream = originalInputStream;
+this.inputStreamBuilder = inputStreamBuilder;
+this.readInputStreamPool = new 
LinkedBlockingQueue<>(inputStreamCapacity);
+}
+
+/**
+ * Reads up to ByteBuffer#remaining 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 bb is null.
+ */
+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 ByteBuffer#remaining 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 bb is null.
+ */
+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:
   Right. It's controlled by inputStreamCapacity.



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



Re: [PR] [FLINK-35045][state] Introduce ForStFlinkFileSystem to support reading and writing with ByteBuffer [flink]

2024-04-17 Thread via GitHub


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


##
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}.
+ *
+ * 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 readInputStreamPool;
+
+private final Callable inputStreamBuilder;
+
+public ByteBufferReadableFSDataInputStream(
+FSDataInputStream originalInputStream,
+Callable inputStreamBuilder,
+int inputStreamCapacity) {
+this.originalInputStream = originalInputStream;
+this.inputStreamBuilder = inputStreamBuilder;
+this.readInputStreamPool = new 
LinkedBlockingQueue<>(inputStreamCapacity);
+}
+
+/**
+ * Reads up to ByteBuffer#remaining 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 bb is null.
+ */
+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 ByteBuffer#remaining 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 bb is null.
+ */
+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();
+}
+
+int result;
+if (fsDataInputStream instanceof PositionedReadable) {
+byte[] tmp = new byte[bb.remaining()];

Review Comment:
   I reconstructed it as below:
   1. reconstruct `PositionedReadable` to introduce `ByteBufferReadable` which 
supports reading and positionable read with ByteBuffer.
   2. For some filesystems doesn't support, it will rollback to 

Re: [PR] [FLINK-35045][state] Introduce ForStFlinkFileSystem to support reading and writing with ByteBuffer [flink]

2024-04-16 Thread via GitHub


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


##
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}.
+ *
+ * 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 readInputStreamPool;
+
+private final Callable inputStreamBuilder;
+
+public ByteBufferReadableFSDataInputStream(
+FSDataInputStream originalInputStream,
+Callable inputStreamBuilder,
+int inputStreamCapacity) {
+this.originalInputStream = originalInputStream;
+this.inputStreamBuilder = inputStreamBuilder;
+this.readInputStreamPool = new 
LinkedBlockingQueue<>(inputStreamCapacity);
+}
+
+/**
+ * Reads up to ByteBuffer#remaining 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 bb is null.
+ */
+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 ByteBuffer#remaining 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 bb is null.
+ */
+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();
+}
+
+int result;
+if (fsDataInputStream instanceof PositionedReadable) {
+byte[] tmp = new byte[bb.remaining()];

Review Comment:
   Can this be optimized? Without introducing another temporary heap array?



##
flink-core/src/main/java/org/apache/flink/core/fs/PositionedReadable.java:
##
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software 

Re: [PR] [FLINK-35045][state] Introduce ForStFlinkFileSystem to support reading and writing with ByteBuffer [flink]

2024-04-16 Thread via GitHub


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


##
flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/fs/ByteBufferWritableFSDataOutputStream.java:
##
@@ -0,0 +1,102 @@
+/*
+ * 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}.
+ *
+ * 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 ByteBuffer#remaining bytes from the ByteBuffer to 
this output stream. Not
+ * Tread-safe yet since the interface of write of ForSt only be accessed 
by one thread at a

Review Comment:
   typo: Thread-safe?



##
flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/fs/ForStFlinkFileSystem.java:
##
@@ -0,0 +1,153 @@
+/*
+ * 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.BlockLocation;
+import org.apache.flink.core.fs.FileStatus;
+import org.apache.flink.core.fs.FileSystem;
+import org.apache.flink.core.fs.FileSystemKind;
+import org.apache.flink.core.fs.Path;
+
+import java.io.IOException;
+import java.net.URI;
+
+/**
+ * A {@link FileSystem} delegates some requests to file system loaded by Flink 
FileSystem mechanism.
+ *
+ * All methods in this class maybe used by ForSt, please start a discussion 
firstly if it has to
+ * be modified.
+ */
+public class ForStFlinkFileSystem extends FileSystem {
+
+private final FileSystem delegateFS;
+
+public ForStFlinkFileSystem(FileSystem delegateFS) {
+this.delegateFS = delegateFS;
+}
+
+/**
+ * Returns a reference to the {@link FileSystem} instance for accessing 
the file system
+ * identified by the given {@link URI}.
+ *
+ * @param uri the {@link URI} identifying the file system.
+ * @return a reference to the {@link FileSystem} instance for accessing 
the file system
+ * identified by the given {@link URI}.
+ * @throws IOException thrown if a reference to the file system instance 
could not be obtained.
+ */
+public static FileSystem get(URI uri) throws IOException {
+return new ForStFlinkFileSystem(FileSystem.get(uri));
+}
+
+/**
+ * Create ByteBufferWritableFSDataOutputStream from specific path which 
supports to write data
+ * to ByteBuffer with {@link 
org.apache.flink.core.fs.FileSystem.WriteMode#OVERWRITE} mode.
+ *
+ * @param path The file path to write to.
+ * @return The stream to the new file at the target path.
+ * @throws IOException Thrown, if the stream could not be opened because 
of an I/O, or because a
+ * file already exists at that path and the write mode indicates to 
not overwrite the file.
+ */
+public 

Re: [PR] [FLINK-35045][state] Introduce ForStFlinkFileSystem to support reading and writing with ByteBuffer [flink]

2024-04-15 Thread via GitHub


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


##
flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/fs/ForStFlinkFileSystem.java:
##
@@ -0,0 +1,153 @@
+/*
+ * 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.BlockLocation;
+import org.apache.flink.core.fs.FileStatus;
+import org.apache.flink.core.fs.FileSystem;
+import org.apache.flink.core.fs.FileSystemKind;
+import org.apache.flink.core.fs.Path;
+
+import java.io.IOException;
+import java.net.URI;
+
+/**
+ * A {@link FileSystem} delegates some requests to file system loaded by Flink 
FileSystem mechanism.
+ *
+ * All methods in this class maybe used by ForSt, please start a discussion 
firstly if it has to
+ * be modified.
+ */
+public class ForStFlinkFileSystem extends FileSystem {
+
+private final FileSystem delegateFS;
+
+public ForStFlinkFileSystem(FileSystem delegateFS) {
+this.delegateFS = delegateFS;
+}
+
+/**
+ * Returns a reference to the {@link FileSystem} instance for accessing 
the file system
+ * identified by the given {@link URI}.
+ *
+ * @param uri the {@link URI} identifying the file system.
+ * @return a reference to the {@link FileSystem} instance for accessing 
the file system
+ * identified by the given {@link URI}.
+ * @throws IOException thrown if a reference to the file system instance 
could not be obtained.
+ */
+public static FileSystem get(URI uri) throws IOException {
+return new ForStFlinkFileSystem(FileSystem.get(uri));
+}
+
+/**
+ * Create ByteBufferWritableFSDataOutputStream from specific path which 
supports to write data
+ * to ByteBuffer with {@link 
org.apache.flink.core.fs.FileSystem.WriteMode#OVERWRITE} mode.
+ *
+ * @param path The file path to write to.
+ * @return The stream to the new file at the target path.
+ * @throws IOException Thrown, if the stream could not be opened because 
of an I/O, or because a
+ * file already exists at that path and the write mode indicates to 
not overwrite the file.
+ */
+public ByteBufferWritableFSDataOutputStream create(Path path) throws 
IOException {
+return create(path, WriteMode.OVERWRITE);
+}
+
+@Override
+public ByteBufferWritableFSDataOutputStream create(Path path, WriteMode 
overwriteMode)
+throws IOException {
+return new 
ByteBufferWritableFSDataOutputStream(delegateFS.create(path, overwriteMode));
+}
+
+@Override
+public ByteBufferReadableFSDataInputStream open(Path path, int bufferSize) 
throws IOException {
+return new ByteBufferReadableFSDataInputStream(
+delegateFS.open(path, bufferSize), () -> delegateFS.open(path, 
bufferSize), 32);
+}
+
+@Override
+public ByteBufferReadableFSDataInputStream open(Path path) throws 
IOException {
+// TODO: make inputStreamCapacity configurable
+return new ByteBufferReadableFSDataInputStream(
+delegateFS.open(path), () -> delegateFS.open(path), 32);
+}
+
+@Override
+public boolean rename(Path src, Path dst) throws IOException {
+// The rename is not atomic for ForSt. Some FileSystems e.g. HDFS, OSS 
does not allow a
+// renaming if the target already exists. So, we delete the target 
before attempting the
+// rename.
+if (delegateFS.exists(dst)) {
+boolean deleted = delegateFS.delete(dst, false);

Review Comment:
   If I remember correctly, it is added because the usage of ForSt/RocksDB's 
`RenameFile` may overwrite the target file in some scenarios. e.g. when 
renaming `CURRENT` file.
   Some FileSystems e.g. PosixFileSystem also just overwrite the target file 
when renaming.



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


Re: [PR] [FLINK-35045][state] Introduce ForStFlinkFileSystem to support reading and writing with ByteBuffer [flink]

2024-04-15 Thread via GitHub


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


##
flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/fs/ByteBufferReadableFSDataInputStream.java:
##
@@ -0,0 +1,191 @@
+/*
+ * 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}.
+ *
+ * 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;
+
+private final Queue readInputStreamPool;
+
+private final Callable inputStreamBuilder;
+
+public ByteBufferReadableFSDataInputStream(
+FSDataInputStream originalInputStream,
+Callable inputStreamBuilder,
+int inputStreamCapacity) {
+this.originalInputStream = originalInputStream;
+this.inputStreamBuilder = inputStreamBuilder;
+this.readInputStreamPool = new 
LinkedBlockingQueue<>(inputStreamCapacity);
+}
+
+/**
+ * Reads up to ByteBuffer#remaining bytes of data from the 
input stream into a
+ * ByteBuffer. Not Tread-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 bb is null.
+ */
+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 ByteBuffer#remaining 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

Review Comment:
   typo: Tread-safe -> Thread-safe



##
flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/fs/ForStFlinkFileSystem.java:
##
@@ -0,0 +1,153 @@
+/*
+ * 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.BlockLocation;
+import org.apache.flink.core.fs.FileStatus;
+import org.apache.flink.core.fs.FileSystem;
+import org.apache.flink.core.fs.FileSystemKind;
+import org.apache.flink.core.fs.Path;
+
+import java.io.IOException;
+import java.net.URI;
+
+/**
+ * A {@link FileSystem} delegates some requests to file system loaded by Flink 
FileSystem mechanism.
+ *
+ * All methods in this class maybe used by ForSt, please start a discussion 
firstly if it has 

Re: [PR] [FLINK-35045][state] Introduce ForStFlinkFileSystem to support reading and writing with ByteBuffer [flink]

2024-04-10 Thread via GitHub


masteryhx commented on PR #24632:
URL: https://github.com/apache/flink/pull/24632#issuecomment-2047200220

   @flinkbot run azure


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



Re: [PR] [FLINK-35045][state] Introduce ForStFlinkFileSystem to support reading and writing with ByteBuffer [flink]

2024-04-08 Thread via GitHub


flinkbot commented on PR #24632:
URL: https://github.com/apache/flink/pull/24632#issuecomment-2042228701

   
   ## CI report:
   
   * aadf8efad983aa85d8d484211c6d9eb90dee3d84 UNKNOWN
   
   
   Bot commands
 The @flinkbot bot supports the following commands:
   
- `@flinkbot run azure` re-run the last Azure build
   


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