zoltar9264 commented on code in PR #20152:
URL: https://github.com/apache/flink/pull/20152#discussion_r932168266


##########
flink-dstl/flink-dstl-dfs/src/main/java/org/apache/flink/changelog/fs/ChangelogHandleReaderWithCache.java:
##########
@@ -0,0 +1,184 @@
+/*
+ * 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.changelog.fs;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.ConfigurationUtils;
+import org.apache.flink.core.fs.FSDataInputStream;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.core.fs.RefCountedBufferingFileStream;
+import org.apache.flink.core.fs.RefCountedFileWithStream;
+import org.apache.flink.core.fs.RefCountedTmpFileCreator;
+import org.apache.flink.runtime.state.StreamStateHandle;
+import org.apache.flink.runtime.state.filesystem.FileStateHandle;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.IOUtils;
+import org.apache.flink.util.function.BiFunctionWithException;
+import org.apache.flink.util.function.FunctionWithException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedInputStream;
+import java.io.DataInputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.flink.changelog.fs.ChangelogStreamWrapper.wrap;
+import static org.apache.flink.changelog.fs.ChangelogStreamWrapper.wrapAndSeek;
+import static 
org.apache.flink.configuration.StateChangelogOptions.PERIODIC_MATERIALIZATION_INTERVAL;
+
+/** StateChangeIterator with local cache. */
+class ChangelogHandleReaderWithCache
+        implements BiFunctionWithException<StreamStateHandle, Long, 
DataInputStream, IOException>,
+                AutoCloseable {
+    private static final Logger LOG = 
LoggerFactory.getLogger(ChangelogHandleReaderWithCache.class);
+
+    private static final String CACHE_FILE_SUB_DIR = "dstl-cache-file";
+
+    private final FunctionWithException<File, RefCountedFileWithStream, 
IOException>
+            cacheFileCreator;
+    private final ConcurrentMap<Path, RefCountedBufferingFileStream> cache =
+            new ConcurrentHashMap<>();
+    private final ScheduledExecutorService cacheCleanScheduler;
+    private final long cacheIdleMillis;
+
+    ChangelogHandleReaderWithCache(Configuration config) {
+        File[] tempFiles =
+                Arrays.stream(ConfigurationUtils.parseTempDirectories(config))
+                        .map(path -> new File(path, CACHE_FILE_SUB_DIR))
+                        .toArray(File[]::new);
+
+        this.cacheCleanScheduler =
+                SchedulerFactory.create(1, "ChangelogCacheFileCleanScheduler", 
LOG);
+        // TODO: 2022/5/31 consider adding a new options for cache idle
+        this.cacheIdleMillis = 
config.get(PERIODIC_MATERIALIZATION_INTERVAL).toMillis();
+        this.cacheFileCreator = 
RefCountedTmpFileCreator.inDirectories(tempFiles);
+    }
+
+    @Override
+    public DataInputStream apply(StreamStateHandle handle, Long offset) throws 
IOException {
+        if (!(handle instanceof FileStateHandle)) {
+            return wrapAndSeek(handle.openInputStream(), offset);
+        }
+
+        FileStateHandle fileHandle = (FileStateHandle) handle;
+        DataInputStream input;
+
+        if (fileHandle.getFilePath().getFileSystem().isDistributedFS()) {
+
+            Path dfsPath = fileHandle.getFilePath();
+
+            final RefCountedBufferingFileStream refCountedFileStream =
+                    cache.computeIfAbsent(
+                            dfsPath,
+                            key -> {
+                                RefCountedBufferingFileStream fileStream = 
null;
+                                FSDataInputStream handleInputStream = null;
+
+                                try {
+                                    fileStream =
+                                            
RefCountedBufferingFileStream.openNew(cacheFileCreator);
+                                    handleInputStream = 
handle.openInputStream();
+                                    IOUtils.copyBytes(wrap(handleInputStream), 
fileStream);
+                                    LOG.debug(
+                                            "download and decompress dstl file 
: {} to cached file : {}",
+                                            fileHandle.getFilePath(),
+                                            
fileStream.getInputFile().getPath());
+                                } catch (IOException e) {
+                                    ExceptionUtils.rethrow(e);
+                                } finally {
+                                    if (handleInputStream != null) {
+                                        
IOUtils.closeQuietly(handleInputStream);
+                                    }
+                                }
+
+                                return fileStream;
+                            });
+
+            FileInputStream fin = null;
+            synchronized (cache) {
+                if (refCountedFileStream.getReferenceCounter() >= 1) {
+                    fin = new 
FileInputStream(refCountedFileStream.getInputFile());
+                    if (offset != 0) {
+                        LOG.debug("seek to {}", offset);
+                        fin.getChannel().position(offset);
+                    }
+                    refCountedFileStream.retain();
+                } else {
+                    // cache file already be deleted, try again
+                    return apply(handle, offset);
+                }
+            }
+
+            input =
+                    new DataInputStream(new BufferedInputStream(fin)) {
+                        @Override
+                        public void close() throws IOException {
+                            super.close();
+                            refCountedFileStream.release();
+                            if (refCountedFileStream.getReferenceCounter() == 
1) {
+                                cacheCleanScheduler.schedule(
+                                        () -> cleanCacheFile(dfsPath, 
refCountedFileStream),
+                                        cacheIdleMillis,
+                                        TimeUnit.MILLISECONDS);
+                            }
+                        }
+                    };
+        } else {
+            input = wrapAndSeek(handle.openInputStream(), offset);
+        }
+
+        return input;
+    }
+
+    private void cleanCacheFile(Path dfsPath, RefCountedBufferingFileStream 
refCountedFileStream) {
+        synchronized (cache) {
+            if (refCountedFileStream.getReferenceCounter() == 1) {
+                LOG.debug("clean cached file : {}", 
refCountedFileStream.getInputFile().getPath());
+                cache.remove(dfsPath);
+                refCountedFileStream.release();
+            }
+        }
+    }
+
+    @Override
+    public void close() throws Exception {
+        cacheCleanScheduler.shutdownNow();
+        if (!cacheCleanScheduler.awaitTermination(5, TimeUnit.SECONDS)) {
+            LOG.warn(
+                    "Unable to cleanly shutdown cache clean scheduler of "
+                            + "ChangelogHandleReaderWithCache in 5s");
+        }
+
+        Iterator<RefCountedBufferingFileStream> iterator = 
cache.values().iterator();
+        while (iterator.hasNext()) {
+            RefCountedBufferingFileStream cacheFile = iterator.next();
+            iterator.remove();
+            while (!cacheFile.release()) {}

Review Comment:
   Hi @rkhachatryan , here is to delete the cache file immediately. Generally, 
the reference count of all cached files should be 1 when close 
`ChangelogHandleReaderWithCache`, so there will be only a few loops.



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