Zakelly commented on code in PR #25818: URL: https://github.com/apache/flink/pull/25818#discussion_r1895650983
########## flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/fs/ForStFlinkFileSystem.java: ########## @@ -340,8 +373,9 @@ public boolean delete(Path path, boolean recursive) throws IOException { if (localPathTuple.f0) { success = localFS.delete(localPathTuple.f1, recursive); // delete from local } - success |= delegateFS.delete(path, recursive); // and delete from remote + success |= fileMappingManager.deleteFile(path, recursive); Review Comment: I'm wondering if the `create` won't affect the fileMapping, then how `fileMappingManager.deleteFile` delete the created but not mapped files? ########## flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/fs/filemapping/FileMappingManager.java: ########## @@ -0,0 +1,148 @@ +/* + * 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.filemapping; + +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.core.fs.FileStatus; +import org.apache.flink.core.fs.FileSystem; +import org.apache.flink.core.fs.Path; +import org.apache.flink.util.Preconditions; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; + +/** + * A manager to manage file mapping of forst file system. Only interact with + * copy()/link()/exist()/delete()/list(), create() won't use file mapping. + */ +public class FileMappingManager { + + private static final Logger LOG = LoggerFactory.getLogger(FileMappingManager.class); + + private FileSystem fileSystem; + + private HashMap<String, MappingEntry> mappingTable; + + public FileMappingManager(FileSystem fileSystem) { + this.fileSystem = fileSystem; + this.mappingTable = new HashMap<>(); + } + + /** Called by link/copy. */ + public int put(String src, String dst) { + if (src == dst) { // self link is not supported + return -1; + } + MappingEntry sourceEntry = mappingTable.get(src); + if (sourceEntry != null) { + sourceEntry.retain(); + mappingTable.putIfAbsent(dst, sourceEntry); + } else { + sourceEntry = new MappingEntry(fileSystem, src); + mappingTable.put(src, sourceEntry); + mappingTable.put(dst, sourceEntry); + } + LOG.trace("put: {} -> {}", src, dst); + return 0; + } + + public String originalPath(String fileName) { + MappingEntry entry = mappingTable.getOrDefault(fileName, null); + if (entry != null) { + return entry.sourcePath; + } + return fileName; + } + + public List<String> listByPrefix(String path) { + List<String> linkedPaths = new ArrayList<>(); + for (String key : mappingTable.keySet()) { + if (key.startsWith(path)) { + linkedPaths.add(key); + } + } + return linkedPaths; + } + + public void renameFile(String src, String dst) { + List<String> toRename = new ArrayList<>(); + for (String key : mappingTable.keySet()) { + if (key.startsWith(src)) { + toRename.add(key); + } + MappingEntry sourceEntry = mappingTable.get(key); + if (sourceEntry.sourcePath.startsWith(src)) { + sourceEntry.sourcePath = sourceEntry.sourcePath.replace(src, dst); + } + } + if (toRename.size() > 0) { + for (String key : toRename) { + MappingEntry sourceEntry = mappingTable.remove(key); + String renamedDst = key.replace(src, dst); + LOG.trace("rename: {} -> {}", key, renamedDst); + mappingTable.put(renamedDst, sourceEntry); + } + } + } + + public boolean deleteFile(Path file, boolean recursive) throws IOException { Review Comment: But still you use lots of `file.toString` here ########## flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/fs/filemapping/FileMappingManager.java: ########## @@ -0,0 +1,160 @@ +/* + * 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.filemapping; + +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.core.fs.FileStatus; +import org.apache.flink.core.fs.FileSystem; +import org.apache.flink.core.fs.Path; +import org.apache.flink.util.Preconditions; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; + +/** + * A manager to manage file mapping of forst file system. Only interact with + * copy()/link()/exist()/delete()/list(), create() won't use file mapping. + */ +public class FileMappingManager { + + private static final Logger LOG = LoggerFactory.getLogger(FileMappingManager.class); + + private FileSystem fileSystem; + + private HashMap<String, MappingEntry> mappingTable; + + public FileMappingManager(FileSystem fileSystem) { + this.fileSystem = fileSystem; + this.mappingTable = new HashMap<>(); + } + + /** Called by link/copy. */ + public int put(String src, String dst) { + if (src == dst) { // self link is not supported + return -1; + } + MappingEntry sourceEntry = mappingTable.get(src); + if (sourceEntry != null) { + sourceEntry.retain(); + mappingTable.putIfAbsent(dst, sourceEntry); + } else { + sourceEntry = new MappingEntry(fileSystem, src); + mappingTable.put(src, sourceEntry); + mappingTable.put(dst, sourceEntry); + } + LOG.trace("put: {} -> {}", src, dst); + return 0; + } + + public String originalPath(String fileName) { + MappingEntry entry = mappingTable.getOrDefault(fileName, null); + if (entry != null) { + return entry.sourcePath; + } + return fileName; + } + + public List<String> listByPrefix(String path) { + List<String> linkedPaths = new ArrayList<>(); + for (String key : mappingTable.keySet()) { + if (key.startsWith(path)) { + linkedPaths.add(key); + } + } + return linkedPaths; + } + + public void renameFile(String src, String dst) { + List<String> toRename = new ArrayList<>(); + for (String key : mappingTable.keySet()) { + if (key.equals(src) || isParentDir(key, src)) { + toRename.add(key); + } + MappingEntry sourceEntry = mappingTable.get(key); + if (sourceEntry.sourcePath.equals(src) || isParentDir(sourceEntry.sourcePath, src)) { + sourceEntry.sourcePath = sourceEntry.sourcePath.replace(src, dst); + } + } + if (toRename.size() > 0) { + for (String key : toRename) { + MappingEntry sourceEntry = mappingTable.remove(key); + String renamedDst = key.replace(src, dst); + LOG.trace("rename: {} -> {}", key, renamedDst); + mappingTable.put(renamedDst, sourceEntry); + } + } + } + + public boolean deleteFile(Path file, boolean recursive) throws IOException { + MappingEntry entry = mappingTable.getOrDefault(file.toString(), null); + LOG.trace("delete: {}, source:{}", file, entry == null ? "" : entry.sourcePath); + if (entry != null) { + entry.release(); + mappingTable.remove(file.toString()); + return true; + } + boolean matchedDir = + mappingTable.keySet().stream().anyMatch(key -> key.startsWith(file.toString())); Review Comment: Should be `isParentDir` here? -- 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