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.
+ *
+ * <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:
   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

Reply via email to