Zakelly commented on code in PR #25561: URL: https://github.com/apache/flink/pull/25561#discussion_r1812408205
########## flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/fs/ByteBufferReadableFSDataInputStream.java: ########## @@ -44,9 +44,9 @@ public class ByteBufferReadableFSDataInputStream extends FSDataInputStream { * 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 Queue<FSDataInputStream> readInputStreamPool; // 可能是CachedFSIS Review Comment: nit. ########## flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/fs/ForStFlinkFileSystem.java: ########## @@ -46,21 +54,44 @@ public class ForStFlinkFileSystem extends FileSystem { // TODO: make it configurable private static final int DEFAULT_INPUT_STREAM_CAPACITY = 32; + private static final long SST_FILE_SIZE = 1024 * 1024 * 64; + private static final Map<String, String> remoteLocalMapping = new ConcurrentHashMap<>(); private static final Function<String, Boolean> miscFileFilter = s -> !s.endsWith(".sst"); + private static Path cacheBase; + private static long cacheCapacity = Long.MAX_VALUE; private final FileSystem localFS; private final FileSystem delegateFS; private final String remoteBase; private final Function<String, Boolean> localFileFilter; private final String localBase; + @Nullable private final FileBasedCache fileBasedCache; - public ForStFlinkFileSystem(FileSystem delegateFS, String remoteBase, String localBase) { + public ForStFlinkFileSystem( + FileSystem delegateFS, + String remoteBase, + String localBase, + @Nullable FileBasedCache fileBasedCache) { this.localFS = FileSystem.getLocalFileSystem(); this.delegateFS = delegateFS; this.localFileFilter = miscFileFilter; this.remoteBase = remoteBase; this.localBase = localBase; + this.fileBasedCache = fileBasedCache; + } + + /** + * Configure cache for ForStFlinkFileSystem. + * + * @param path the cache base path. + * @param cacheCap the cache capacity. + */ + public static void configureCache(Path path, long cacheCap) { Review Comment: I'd suggest we change the jni part about initialization, creating `ForStFlinkFileSystem` instance in java side. ########## flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/fs/cache/FileCacheEntry.java: ########## @@ -0,0 +1,129 @@ +/* + * 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.cache; + +import org.apache.flink.core.fs.FSDataInputStream; +import org.apache.flink.core.fs.FSDataOutputStream; +import org.apache.flink.core.fs.FileSystem; +import org.apache.flink.core.fs.Path; +import org.apache.flink.runtime.asyncprocessing.ReferenceCounted; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; + +import java.io.IOException; + +/** + * A file cache entry that encapsulates file and the size of the file, and provides methods to read + * or write file. Not thread safe. + */ +public class FileCacheEntry extends ReferenceCounted { Review Comment: I'd suggest decouple read and write functionalities. Cache is only for read here. ########## flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/fs/cache/FileCacheEntry.java: ########## @@ -0,0 +1,129 @@ +/* + * 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.cache; + +import org.apache.flink.core.fs.FSDataInputStream; +import org.apache.flink.core.fs.FSDataOutputStream; +import org.apache.flink.core.fs.FileSystem; +import org.apache.flink.core.fs.Path; +import org.apache.flink.runtime.asyncprocessing.ReferenceCounted; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; + +import java.io.IOException; + +/** + * A file cache entry that encapsulates file and the size of the file, and provides methods to read + * or write file. Not thread safe. + */ +public class FileCacheEntry extends ReferenceCounted { + private static final Logger LOG = LoggerFactory.getLogger(FileCacheEntry.class); + + /** The reference of file cache. */ + final FileBasedCache fileBasedCache; + + /** The file system of cache. */ + final FileSystem cacheFs; + + /** The original path of file. */ + final Path originalPath; + + /** The path in cache. */ + final Path cachePath; + + /** The size of file. */ + long entrySize; + + volatile FSDataInputStream fsDataInputStream; + + volatile FSDataOutputStream fsDataOutputStream; + + /** Whether the entry is writing. */ + volatile boolean writing; + + volatile boolean closed; + + FileCacheEntry(FileBasedCache fileBasedCache, Path originalPath, Path cachePath) { + super(1); + this.fileBasedCache = fileBasedCache; + this.cacheFs = fileBasedCache.cacheFs; + this.originalPath = originalPath; + this.cachePath = cachePath; + this.entrySize = 0; + this.writing = true; + this.closed = false; + } + + FSDataInputStream open4Read() throws IOException { + if (!closed && !writing && tryRetain() > 0) { + if (fsDataInputStream == null) { + fsDataInputStream = cacheFs.open(cachePath); Review Comment: is `fsDataInputStream` thread safe? ########## flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/fs/cache/ResourceChecker.java: ########## @@ -0,0 +1,66 @@ +/* + * 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.cache; + +/** Space checker. */ +public interface ResourceChecker { Review Comment: How about `CacheLimit` as class name? The word `resource` is too generic. ########## flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/fs/cache/ResourceChecker.java: ########## @@ -0,0 +1,66 @@ +/* + * 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.cache; + +/** Space checker. */ +public interface ResourceChecker { + /** Type of file space checker. */ + enum Type { + // default production space checker of + OS, + FIXED_CAPACITY, + } + + /** + * Whether the resource usage is exceeded the upperbound. + * + * @param toAddSize + * @return true if the resource usage is overflow, false otherwise. + */ + boolean isOverflow(long toAddSize); + + /** + * Get the current free resource. + * + * @return the current free resource. + */ + long getCurrentFreeResource(); + + /** + * Get the current usage space. + * + * @return the current usage space. + */ + long getUsageResource(); + + /** + * Acquire resource. + * + * @param toAddSize + * @return true if the space is enough, false otherwise. + */ + boolean acquireResource(long toAddSize); + + /** + * Release resource. + * + * @param toReleaseSize + */ + void releaseResource(long toReleaseSize); Review Comment: Remove the word `resource`? ########## flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/fs/cache/FileCacheEntry.java: ########## @@ -0,0 +1,129 @@ +/* + * 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.cache; + +import org.apache.flink.core.fs.FSDataInputStream; +import org.apache.flink.core.fs.FSDataOutputStream; +import org.apache.flink.core.fs.FileSystem; +import org.apache.flink.core.fs.Path; +import org.apache.flink.runtime.asyncprocessing.ReferenceCounted; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; + +import java.io.IOException; + +/** + * A file cache entry that encapsulates file and the size of the file, and provides methods to read + * or write file. Not thread safe. + */ +public class FileCacheEntry extends ReferenceCounted { + private static final Logger LOG = LoggerFactory.getLogger(FileCacheEntry.class); + + /** The reference of file cache. */ + final FileBasedCache fileBasedCache; + + /** The file system of cache. */ + final FileSystem cacheFs; + + /** The original path of file. */ + final Path originalPath; + + /** The path in cache. */ + final Path cachePath; + + /** The size of file. */ + long entrySize; + + volatile FSDataInputStream fsDataInputStream; + + volatile FSDataOutputStream fsDataOutputStream; + + /** Whether the entry is writing. */ + volatile boolean writing; + + volatile boolean closed; + + FileCacheEntry(FileBasedCache fileBasedCache, Path originalPath, Path cachePath) { + super(1); + this.fileBasedCache = fileBasedCache; + this.cacheFs = fileBasedCache.cacheFs; + this.originalPath = originalPath; + this.cachePath = cachePath; + this.entrySize = 0; + this.writing = true; + this.closed = false; + } + + FSDataInputStream open4Read() throws IOException { Review Comment: Use `For` instead of `4`? ########## flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/ForStOptions.java: ########## @@ -58,6 +58,22 @@ public class ForStOptions { "The remote directory where ForSt puts its SST files, fallback to %s if not configured.", LOCAL_DIRECTORIES.key())); + public static final ConfigOption<String> CACHE_DIRECTORY = + ConfigOptions.key("state.backend.forst.cache-dir") + .stringType() + .noDefaultValue() + .withDescription( + String.format( + "The directory where ForSt caches its SST files, fallback to %s/cache if not configured.", + LOCAL_DIRECTORIES.key())); + + public static final ConfigOption<Long> CACHE_CAPACITY = + ConfigOptions.key("state.backend.forst.cache-capacity") + .longType() + .defaultValue(-1L) + .withDescription( + "The capacity capacity of cache, a negative value means no cache will be used. When this value is greater than the actual available space, the actual available space will be used as the upper limit."); + Review Comment: How about 2 options: 1. state.backend.forst.cache.size-based-limit 2. state.backend.forst.cache.disk-remaining-space-based-limit ########## flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/fs/cache/ResourceChecker.java: ########## @@ -0,0 +1,66 @@ +/* + * 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.cache; + +/** Space checker. */ +public interface ResourceChecker { + /** Type of file space checker. */ + enum Type { + // default production space checker of + OS, + FIXED_CAPACITY, + } + + /** + * Whether the resource usage is exceeded the upperbound. + * + * @param toAddSize + * @return true if the resource usage is overflow, false otherwise. + */ + boolean isOverflow(long toAddSize); + + /** + * Get the current free resource. + * + * @return the current free resource. + */ + long getCurrentFreeResource(); + + /** + * Get the current usage space. + * + * @return the current usage space. + */ + long getUsageResource(); Review Comment: Seems useless. ########## flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/fs/cache/ResourceChecker.java: ########## @@ -0,0 +1,66 @@ +/* + * 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.cache; + +/** Space checker. */ +public interface ResourceChecker { + /** Type of file space checker. */ + enum Type { + // default production space checker of + OS, + FIXED_CAPACITY, + } + + /** + * Whether the resource usage is exceeded the upperbound. + * + * @param toAddSize + * @return true if the resource usage is overflow, false otherwise. + */ + boolean isOverflow(long toAddSize); + + /** + * Get the current free resource. + * + * @return the current free resource. + */ + long getCurrentFreeResource(); + + /** + * Get the current usage space. + * + * @return the current usage space. + */ + long getUsageResource(); Review Comment: And another interface may be useful: `boolean canAddNewEntry(long toAdd);` -- 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