This is an automated email from the ASF dual-hosted git repository.

zakelly pushed a commit to branch release-2.0
in repository https://gitbox.apache.org/repos/asf/flink.git


The following commit(s) were added to refs/heads/release-2.0 by this push:
     new 263cf4e910b [FLINK-38567][state/forst] Adapt ForStFileSystem to dummy 
mkdir() implementation (#27223)
263cf4e910b is described below

commit 263cf4e910ba6c958dfd32d4bcf5330178d67c55
Author: AlexYinHan <[email protected]>
AuthorDate: Tue Nov 11 19:01:11 2025 +0800

    [FLINK-38567][state/forst] Adapt ForStFileSystem to dummy mkdir() 
implementation (#27223)
    
    (cherry picked from commit 498954941b2fc1502fad2c85423b39d0e36649f6)
---
 ...ForStFileSystemTrackingCreatedDirDecorator.java |  59 ++++++++++++
 .../flink/state/forst/fs/ForStFileSystemUtils.java |  84 +++++++++++++++++
 .../flink/state/forst/fs/ForStFlinkFileSystem.java |   8 ++
 .../state/forst/fs/StringifiedForStFileSystem.java |   2 +-
 .../forst/fs/filemapping/FileMappingManager.java   |  20 +---
 ...tFileSystemTrackingCreatedDirDecoratorTest.java | 103 +++++++++++++++++++++
 6 files changed, 257 insertions(+), 19 deletions(-)

diff --git 
a/flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/fs/ForStFileSystemTrackingCreatedDirDecorator.java
 
b/flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/fs/ForStFileSystemTrackingCreatedDirDecorator.java
new file mode 100644
index 00000000000..7d7ba568b2d
--- /dev/null
+++ 
b/flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/fs/ForStFileSystemTrackingCreatedDirDecorator.java
@@ -0,0 +1,59 @@
+/*
+ * 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.Path;
+
+import java.io.IOException;
+import java.util.HashSet;
+import java.util.Set;
+
+/**
+ * A decorator of {@link ForStFlinkFileSystem} to adapt ForSt to the 
underlying FileSystems which
+ * are implemented with incomplete mkdir(), i.e., such FileSystem 
implementation does not actually
+ * create the directory when mkdir() completes. This can lead to unexpected 
behavior when ForSt
+ * tries to assert the existence of directories by calling exists(). 
Therefore, we track the paths
+ * of the should-be-created directories and subsequently return true for 
existence checks.
+ */
+public class ForStFileSystemTrackingCreatedDirDecorator extends 
ForStFlinkFileSystem {
+    private final Set<Path> createdDirPaths = new HashSet<>();
+
+    ForStFileSystemTrackingCreatedDirDecorator(ForStFlinkFileSystem 
fileSystem) {
+        super(fileSystem);
+    }
+
+    @Override
+    public synchronized boolean mkdirs(Path path) throws IOException {
+        boolean mkdirSucceed = super.mkdirs(path);
+        if (!mkdirSucceed) {
+            return false;
+        }
+
+        createdDirPaths.add(path);
+        return true;
+    }
+
+    @Override
+    public synchronized boolean exists(final Path f) throws IOException {
+        if (createdDirPaths.contains(f)) {
+            return true;
+        }
+        return super.exists(f);
+    }
+}
diff --git 
a/flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/fs/ForStFileSystemUtils.java
 
b/flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/fs/ForStFileSystemUtils.java
new file mode 100644
index 00000000000..0674dd56777
--- /dev/null
+++ 
b/flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/fs/ForStFileSystemUtils.java
@@ -0,0 +1,84 @@
+/*
+ * 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.Path;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+import java.util.UUID;
+
+/** Utils for ForStFileSystem. */
+public class ForStFileSystemUtils {
+    private static final Logger LOG = 
LoggerFactory.getLogger(ForStFileSystemUtils.class);
+
+    private static final String DUMMY_DIR_NAME = "_dummy_dir_";
+
+    public static boolean isParentDir(@Nullable Path path, String dir) {
+        if (path == null) {
+            return false;
+        }
+        return isParentDir(path.toString(), dir);
+    }
+
+    public static boolean isParentDir(String path, String dir) {
+        if (dir.isEmpty()) {
+            return false;
+        }
+        if (dir.charAt(dir.length() - 1) == '/') {
+            return path.startsWith(dir);
+        } else {
+            return (path.startsWith(dir + "/"));
+        }
+    }
+
+    public static ForStFlinkFileSystem tryDecorate(ForStFlinkFileSystem 
fileSystem) {
+        try {
+            return isIncompleteMkdirEnabled(fileSystem)
+                    ? new 
ForStFileSystemTrackingCreatedDirDecorator(fileSystem)
+                    : fileSystem;
+        } catch (IOException e) {
+            LOG.info("Cannot decorate ForStFlinkFileSystem", e);
+        }
+        return fileSystem;
+    }
+
+    private static boolean isIncompleteMkdirEnabled(ForStFlinkFileSystem 
fileSystem)
+            throws IOException {
+        // check if the underlying FileSystem uses an incomplete mkdir 
implementation
+        Path dummyDir = new Path(fileSystem.getRemoteBase(), DUMMY_DIR_NAME + 
UUID.randomUUID());
+        if (fileSystem.mkdirs(dummyDir)) {
+            if (!fileSystem.exists(dummyDir)) {
+                return true;
+            }
+            fileSystem.delete(new Path(DUMMY_DIR_NAME), true);
+            return false;
+        } else {
+            LOG.info(
+                    "Cannot to mkdir for "
+                            + DUMMY_DIR_NAME
+                            + ", skip decoration of ForStFlinkFileSystem");
+        }
+        return false;
+    }
+}
diff --git 
a/flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/fs/ForStFlinkFileSystem.java
 
b/flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/fs/ForStFlinkFileSystem.java
index afbc5a71505..1e662a509be 100644
--- 
a/flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/fs/ForStFlinkFileSystem.java
+++ 
b/flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/fs/ForStFlinkFileSystem.java
@@ -89,6 +89,14 @@ public class ForStFlinkFileSystem extends FileSystem 
implements Closeable {
         this.fileMappingManager = new FileMappingManager(delegateFS, 
remoteBase, localBase);
     }
 
+    protected ForStFlinkFileSystem(ForStFlinkFileSystem forStFlinkFileSystem) {
+        this.localFS = forStFlinkFileSystem.localFS;
+        this.delegateFS = forStFlinkFileSystem.delegateFS;
+        this.remoteBase = forStFlinkFileSystem.remoteBase;
+        this.fileBasedCache = forStFlinkFileSystem.fileBasedCache;
+        this.fileMappingManager = forStFlinkFileSystem.fileMappingManager;
+    }
+
     /**
      * Returns a reference to the {@link FileSystem} instance for accessing 
the file system
      * identified by the given {@link URI}.
diff --git 
a/flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/fs/StringifiedForStFileSystem.java
 
b/flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/fs/StringifiedForStFileSystem.java
index 1031a76e735..f0969117827 100644
--- 
a/flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/fs/StringifiedForStFileSystem.java
+++ 
b/flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/fs/StringifiedForStFileSystem.java
@@ -33,7 +33,7 @@ public class StringifiedForStFileSystem {
     private ForStFlinkFileSystem fileSystem;
 
     public StringifiedForStFileSystem(ForStFlinkFileSystem fileSystem) {
-        this.fileSystem = fileSystem;
+        this.fileSystem = ForStFileSystemUtils.tryDecorate(fileSystem);
     }
 
     public static StringifiedForStFileSystem get(String uri) throws 
IOException {
diff --git 
a/flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/fs/filemapping/FileMappingManager.java
 
b/flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/fs/filemapping/FileMappingManager.java
index ca2ff8dbec4..95cbdbdc1a3 100644
--- 
a/flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/fs/filemapping/FileMappingManager.java
+++ 
b/flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/fs/filemapping/FileMappingManager.java
@@ -37,6 +37,8 @@ import java.util.List;
 import java.util.Map;
 import java.util.UUID;
 
+import static org.apache.flink.state.forst.fs.ForStFileSystemUtils.isParentDir;
+
 /**
  * A manager to manage file mapping of forst file system, including misc file 
mapping (remote file
  * -> local file) and linked mapping (remote file -> remote file). Note, the 
key/value of mapping
@@ -309,24 +311,6 @@ public class FileMappingManager {
         return mappingTable.getOrDefault(path, null);
     }
 
-    private boolean isParentDir(@Nullable Path path, String dir) {
-        if (path == null) {
-            return false;
-        }
-        return isParentDir(path.toString(), dir);
-    }
-
-    private boolean isParentDir(String path, String dir) {
-        if (dir.isEmpty()) {
-            return false;
-        }
-        if (dir.charAt(dir.length() - 1) == '/') {
-            return path.startsWith(dir);
-        } else {
-            return (path.startsWith(dir + "/"));
-        }
-    }
-
     public void giveUpOwnership(Path path, StreamStateHandle stateHandle) {
         MappingEntry mappingEntry = mappingTable.getOrDefault(path.toString(), 
null);
         Preconditions.checkArgument(
diff --git 
a/flink-state-backends/flink-statebackend-forst/src/test/java/org/apache/flink/state/forst/fs/ForStFileSystemTrackingCreatedDirDecoratorTest.java
 
b/flink-state-backends/flink-statebackend-forst/src/test/java/org/apache/flink/state/forst/fs/ForStFileSystemTrackingCreatedDirDecoratorTest.java
new file mode 100644
index 00000000000..20b3bee3afe
--- /dev/null
+++ 
b/flink-state-backends/flink-statebackend-forst/src/test/java/org/apache/flink/state/forst/fs/ForStFileSystemTrackingCreatedDirDecoratorTest.java
@@ -0,0 +1,103 @@
+/*
+ * 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.local.LocalFileSystem;
+
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
+
+import java.io.IOException;
+import java.nio.file.Path;
+import java.util.UUID;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Tests for {@link ForStFileSystemTrackingCreatedDirDecorator}. */
+public class ForStFileSystemTrackingCreatedDirDecoratorTest {
+    @TempDir static Path tempDir;
+
+    private static class MockLocalFileSystem extends LocalFileSystem {
+        private final boolean dummyMkdirEnabled;
+
+        public MockLocalFileSystem(boolean dummyMkdirEnabled) {
+            super();
+            this.dummyMkdirEnabled = dummyMkdirEnabled;
+        }
+
+        @Override
+        public synchronized boolean mkdirs(org.apache.flink.core.fs.Path path) 
throws IOException {
+            if (dummyMkdirEnabled) {
+                return true;
+            } else {
+                return super.mkdirs(path);
+            }
+        }
+    }
+
+    @Test
+    public void testMkdirAndCheck() throws IOException {
+        mkdirAndCheck(false);
+    }
+
+    @Test
+    public void testDummyMkdirAndCheck() throws IOException {
+        mkdirAndCheck(true);
+    }
+
+    void mkdirAndCheck(boolean enableDummyMkdir) throws IOException {
+        org.apache.flink.core.fs.Path remotePath =
+                new org.apache.flink.core.fs.Path(tempDir.toString() + 
"/remote");
+        org.apache.flink.core.fs.Path localPath =
+                new org.apache.flink.core.fs.Path(tempDir.toString() + 
"/local");
+
+        MockLocalFileSystem mockLocalFileSystem = new 
MockLocalFileSystem(enableDummyMkdir);
+        ForStFlinkFileSystem fileSystem =
+                ForStFileSystemUtils.tryDecorate(
+                        new ForStFlinkFileSystem(
+                                mockLocalFileSystem,
+                                remotePath.toString(),
+                                localPath.toString(),
+                                null));
+        if (enableDummyMkdir) {
+            
assertThat(fileSystem).isInstanceOf(ForStFileSystemTrackingCreatedDirDecorator.class);
+        }
+
+        // create a directory
+        String dirPathStr = genRandomFilePathStr();
+        org.apache.flink.core.fs.Path testMkdirPath = new 
org.apache.flink.core.fs.Path(dirPathStr);
+        fileSystem.mkdirs(testMkdirPath);
+        
assertThat(mockLocalFileSystem.exists(testMkdirPath)).isEqualTo(!enableDummyMkdir);
+        assertThat(fileSystem.exists(testMkdirPath)).isTrue();
+
+        // create sub directories
+        for (int i = 0; i < 10; i++) {
+            String subDirName = UUID.randomUUID().toString();
+            org.apache.flink.core.fs.Path testSubMkdirPath =
+                    new org.apache.flink.core.fs.Path(dirPathStr, subDirName);
+            fileSystem.mkdirs(testSubMkdirPath);
+            
assertThat(mockLocalFileSystem.exists(testSubMkdirPath)).isEqualTo(!enableDummyMkdir);
+            assertThat(fileSystem.exists(testSubMkdirPath)).isTrue();
+        }
+    }
+
+    private String genRandomFilePathStr() {
+        return tempDir.toString() + "/" + UUID.randomUUID();
+    }
+}

Reply via email to