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}.
+ *
+ * <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
+     * 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.
+ *
+ * <p>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);

Review Comment:
   nit: let 32 as a **static final default variable** in 
ByteBufferReadableFSDataInputStream?



##########
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();
+        }
+
+        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 = readFullyFromFSDataInputStream(fsDataInputStream, bb);
+        }
+
+        if (!readInputStreamPool.offer(fsDataInputStream)) {
+            fsDataInputStream.close();
+        }

Review Comment:
   I suggest putting the "readInputStreamPool.offer(fsDataInputStream)"  into a 
"try-finally" block, so that ensure all the inputStreams can be closed when an 
IO exception is encountered.



##########
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.
+ *
+ * <p>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:
   Thanks for your explanation, I got it.



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