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


##########
flink-dstl/flink-dstl-dfs/src/main/java/org/apache/flink/changelog/fs/ChangelogStreamHandleReaderWithCache.java:
##########
@@ -0,0 +1,203 @@
+/*
+ * 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.RefCountedFile;
+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.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.FileOutputStream;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.Random;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static org.apache.flink.changelog.fs.ChangelogStreamWrapper.wrap;
+import static org.apache.flink.changelog.fs.ChangelogStreamWrapper.wrapAndSeek;
+import static 
org.apache.flink.changelog.fs.FsStateChangelogOptions.CACHE_IDLE_TIMEOUT;
+
+/** StateChangeIterator with local cache. */
+class ChangelogStreamHandleReaderWithCache implements 
ChangelogStreamHandleReader {
+    private static final Logger LOG =
+            
LoggerFactory.getLogger(ChangelogStreamHandleReaderWithCache.class);
+
+    private static final String CACHE_FILE_SUB_DIR = "dstl-cache-file";
+    private static final String CACHE_FILE_PREFIX = "dstl";
+
+    // reference count == 1 means only cache component reference the cache file
+    private static final int NO_USING_REF_COUNT = 1;
+
+    private final File[] cacheDirectories;
+    private final AtomicInteger next;
+
+    private final ConcurrentHashMap<Path, RefCountedFile> cache = new 
ConcurrentHashMap<>();
+    private final ScheduledExecutorService cacheCleanScheduler;
+    private final long cacheIdleMillis;
+
+    ChangelogStreamHandleReaderWithCache(Configuration config) {
+        this.cacheDirectories =
+                Arrays.stream(ConfigurationUtils.parseTempDirectories(config))
+                        .map(path -> new File(path, CACHE_FILE_SUB_DIR))
+                        .toArray(File[]::new);
+        this.next = new AtomicInteger(new 
Random().nextInt(this.cacheDirectories.length));
+
+        this.cacheCleanScheduler =
+                SchedulerFactory.create(1, "ChangelogCacheFileCleanScheduler", 
LOG);
+        this.cacheIdleMillis = config.get(CACHE_IDLE_TIMEOUT).toMillis();
+    }
+
+    @Override
+    public DataInputStream openAndSeek(StreamStateHandle handle, Long offset) 
throws IOException {
+        if (!canBeCached(handle)) {
+            return wrapAndSeek(handle.openInputStream(), offset);
+        }
+
+        final FileStateHandle fileHandle = (FileStateHandle) handle;
+        final RefCountedFile refCountedFile = getRefCountedFile(fileHandle);
+
+        FileInputStream fin = openAndSeek(refCountedFile, offset);
+
+        return wrapStream(fileHandle.getFilePath(), fin);
+    }
+
+    private boolean canBeCached(StreamStateHandle handle) throws IOException {
+        if (handle instanceof FileStateHandle) {
+            FileStateHandle fileHandle = (FileStateHandle) handle;
+            return fileHandle.getFilePath().getFileSystem().isDistributedFS();
+        } else {
+            return false;
+        }
+    }
+
+    private RefCountedFile getRefCountedFile(FileStateHandle fileHandle) {
+        return cache.compute(
+                fileHandle.getFilePath(),
+                (key, oldValue) -> {
+                    if (oldValue == null) {
+                        oldValue = downloadToCacheFile(fileHandle);
+                    }
+                    oldValue.retain();
+                    return oldValue;
+                });
+    }
+
+    private RefCountedFile downloadToCacheFile(FileStateHandle fileHandle) {
+        RefCountedFile refCountedFile = null;
+
+        try (FSDataInputStream inputStream = fileHandle.openInputStream()) {
+            File directory = cacheDirectories[next.getAndIncrement() % 
cacheDirectories.length];
+            File file = File.createTempFile(CACHE_FILE_PREFIX, null, 
directory);
+
+            IOUtils.copyBytes(wrap(inputStream), new FileOutputStream(file));

Review Comment:
   1. No need to `wrap` here?
   2. The input stream is closed twice; I think we should use an overloaded 
`copyBytes` with `close=false`



##########
flink-dstl/flink-dstl-dfs/src/main/java/org/apache/flink/changelog/fs/ChangelogStreamHandleReaderWithCache.java:
##########
@@ -0,0 +1,203 @@
+/*
+ * 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.RefCountedFile;
+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.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.FileOutputStream;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.Random;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static org.apache.flink.changelog.fs.ChangelogStreamWrapper.wrap;
+import static org.apache.flink.changelog.fs.ChangelogStreamWrapper.wrapAndSeek;
+import static 
org.apache.flink.changelog.fs.FsStateChangelogOptions.CACHE_IDLE_TIMEOUT;
+
+/** StateChangeIterator with local cache. */
+class ChangelogStreamHandleReaderWithCache implements 
ChangelogStreamHandleReader {
+    private static final Logger LOG =
+            
LoggerFactory.getLogger(ChangelogStreamHandleReaderWithCache.class);
+
+    private static final String CACHE_FILE_SUB_DIR = "dstl-cache-file";
+    private static final String CACHE_FILE_PREFIX = "dstl";
+
+    // reference count == 1 means only cache component reference the cache file
+    private static final int NO_USING_REF_COUNT = 1;
+
+    private final File[] cacheDirectories;
+    private final AtomicInteger next;
+
+    private final ConcurrentHashMap<Path, RefCountedFile> cache = new 
ConcurrentHashMap<>();
+    private final ScheduledExecutorService cacheCleanScheduler;
+    private final long cacheIdleMillis;
+
+    ChangelogStreamHandleReaderWithCache(Configuration config) {
+        this.cacheDirectories =
+                Arrays.stream(ConfigurationUtils.parseTempDirectories(config))
+                        .map(path -> new File(path, CACHE_FILE_SUB_DIR))
+                        .toArray(File[]::new);
+        this.next = new AtomicInteger(new 
Random().nextInt(this.cacheDirectories.length));
+
+        this.cacheCleanScheduler =
+                SchedulerFactory.create(1, "ChangelogCacheFileCleanScheduler", 
LOG);
+        this.cacheIdleMillis = config.get(CACHE_IDLE_TIMEOUT).toMillis();
+    }
+
+    @Override
+    public DataInputStream openAndSeek(StreamStateHandle handle, Long offset) 
throws IOException {
+        if (!canBeCached(handle)) {
+            return wrapAndSeek(handle.openInputStream(), offset);
+        }
+
+        final FileStateHandle fileHandle = (FileStateHandle) handle;
+        final RefCountedFile refCountedFile = getRefCountedFile(fileHandle);
+
+        FileInputStream fin = openAndSeek(refCountedFile, offset);
+
+        return wrapStream(fileHandle.getFilePath(), fin);
+    }
+
+    private boolean canBeCached(StreamStateHandle handle) throws IOException {
+        if (handle instanceof FileStateHandle) {
+            FileStateHandle fileHandle = (FileStateHandle) handle;
+            return fileHandle.getFilePath().getFileSystem().isDistributedFS();
+        } else {
+            return false;
+        }
+    }
+
+    private RefCountedFile getRefCountedFile(FileStateHandle fileHandle) {
+        return cache.compute(
+                fileHandle.getFilePath(),
+                (key, oldValue) -> {
+                    if (oldValue == null) {
+                        oldValue = downloadToCacheFile(fileHandle);
+                    }
+                    oldValue.retain();
+                    return oldValue;
+                });
+    }
+
+    private RefCountedFile downloadToCacheFile(FileStateHandle fileHandle) {
+        RefCountedFile refCountedFile = null;
+
+        try (FSDataInputStream inputStream = fileHandle.openInputStream()) {
+            File directory = cacheDirectories[next.getAndIncrement() % 
cacheDirectories.length];
+            File file = File.createTempFile(CACHE_FILE_PREFIX, null, 
directory);
+
+            IOUtils.copyBytes(wrap(inputStream), new FileOutputStream(file));
+            LOG.debug(
+                    "download and decompress dstl file : {} to cache file : 
{}",
+                    fileHandle.getFilePath(),
+                    refCountedFile.getFile().getPath());

Review Comment:
   `refCountedFile` is only initialized on the next line.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/state/TaskExecutorStateChangelogStoragesManager.java:
##########
@@ -135,28 +148,109 @@ public void releaseStateChangelogStorageForJob(@Nonnull 
JobID jobId) {
         }
     }
 
+    @Nullable
+    StateChangelogStorageView<?> stateChangelogStorageViewForJob(
+            @Nonnull JobID jobID,
+            Configuration configuration,
+            ChangelogStateHandle changelogStateHandle)
+            throws IOException {
+        if (closed) {
+            throw new IllegalStateException(
+                    "TaskExecutorStateChangelogStoragesManager is already 
closed and cannot "
+                            + "register a new StateChangelogStorageView.");
+        }
+
+        if (!(changelogStateHandle instanceof ChangelogStateHandleStreamImpl)) 
{
+            return StateChangelogStorageLoader.loadFromStateHandle(
+                    configuration, changelogStateHandle);
+        }
+
+        synchronized (lock) {
+            
Optional<StateChangelogStorageView<ChangelogStateHandleStreamImpl>> storageView 
=
+                    changelogStorageViewsByJobId.get(jobID);
+
+            if (storageView == null) {
+                StateChangelogStorageView<?> loaded =
+                        StateChangelogStorageLoader.loadFromStateHandle(
+                                configuration, changelogStateHandle);
+                changelogStorageViewsByJobId.put(
+                        jobID,
+                        Optional.of(
+                                
(StateChangelogStorageView<ChangelogStateHandleStreamImpl>)
+                                        loaded));

Review Comment:
   Could you explain why is it `Optional`?



##########
flink-runtime/src/main/java/org/apache/flink/runtime/state/TaskExecutorStateChangelogStoragesManager.java:
##########
@@ -135,28 +148,109 @@ public void releaseStateChangelogStorageForJob(@Nonnull 
JobID jobId) {
         }
     }
 
+    @Nullable
+    StateChangelogStorageView<?> stateChangelogStorageViewForJob(
+            @Nonnull JobID jobID,
+            Configuration configuration,
+            ChangelogStateHandle changelogStateHandle)
+            throws IOException {
+        if (closed) {
+            throw new IllegalStateException(
+                    "TaskExecutorStateChangelogStoragesManager is already 
closed and cannot "
+                            + "register a new StateChangelogStorageView.");
+        }
+
+        if (!(changelogStateHandle instanceof ChangelogStateHandleStreamImpl)) 
{
+            return StateChangelogStorageLoader.loadFromStateHandle(
+                    configuration, changelogStateHandle);
+        }

Review Comment:
   This duplicates the logic of 
`ChangelogStreamHandleReaderWithCache.canBeCached()`.
   I think it's better to have it only there



##########
flink-dstl/flink-dstl-dfs/src/main/java/org/apache/flink/changelog/fs/StateChangeIteratorImpl.java:
##########
@@ -0,0 +1,51 @@
+/*
+ * 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.runtime.state.StreamStateHandle;
+import org.apache.flink.runtime.state.changelog.StateChange;
+import 
org.apache.flink.runtime.state.changelog.StateChangelogHandleStreamHandleReader;
+import org.apache.flink.util.CloseableIterator;
+
+import java.io.IOException;
+
+import static org.apache.flink.changelog.fs.ChangelogStreamWrapper.wrapAndSeek;
+
+/** StateChangeIterator default implementation. */
+class StateChangeIteratorImpl
+        implements StateChangelogHandleStreamHandleReader.StateChangeIterator {
+
+    private final ChangelogStreamHandleReader changelogStreamHandleReader;
+
+    public StateChangeIteratorImpl() {
+        this.changelogStreamHandleReader =
+                (handle, offset) -> wrapAndSeek(handle.openInputStream(), 
offset);

Review Comment:
   I'd extract this lambda into a constant in 
`ChangelogStreamHandleReader.DIRECT_READER` or `NON_CACHING_READER`.



##########
flink-dstl/flink-dstl-dfs/src/main/java/org/apache/flink/changelog/fs/ChangelogStreamHandleReaderWithCache.java:
##########
@@ -0,0 +1,203 @@
+/*
+ * 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.RefCountedFile;
+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.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.FileOutputStream;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.Random;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static org.apache.flink.changelog.fs.ChangelogStreamWrapper.wrap;
+import static org.apache.flink.changelog.fs.ChangelogStreamWrapper.wrapAndSeek;
+import static 
org.apache.flink.changelog.fs.FsStateChangelogOptions.CACHE_IDLE_TIMEOUT;
+
+/** StateChangeIterator with local cache. */
+class ChangelogStreamHandleReaderWithCache implements 
ChangelogStreamHandleReader {
+    private static final Logger LOG =
+            
LoggerFactory.getLogger(ChangelogStreamHandleReaderWithCache.class);
+
+    private static final String CACHE_FILE_SUB_DIR = "dstl-cache-file";
+    private static final String CACHE_FILE_PREFIX = "dstl";
+
+    // reference count == 1 means only cache component reference the cache file
+    private static final int NO_USING_REF_COUNT = 1;
+
+    private final File[] cacheDirectories;
+    private final AtomicInteger next;
+
+    private final ConcurrentHashMap<Path, RefCountedFile> cache = new 
ConcurrentHashMap<>();
+    private final ScheduledExecutorService cacheCleanScheduler;
+    private final long cacheIdleMillis;
+
+    ChangelogStreamHandleReaderWithCache(Configuration config) {
+        this.cacheDirectories =
+                Arrays.stream(ConfigurationUtils.parseTempDirectories(config))
+                        .map(path -> new File(path, CACHE_FILE_SUB_DIR))
+                        .toArray(File[]::new);
+        this.next = new AtomicInteger(new 
Random().nextInt(this.cacheDirectories.length));
+
+        this.cacheCleanScheduler =
+                SchedulerFactory.create(1, "ChangelogCacheFileCleanScheduler", 
LOG);
+        this.cacheIdleMillis = config.get(CACHE_IDLE_TIMEOUT).toMillis();
+    }
+
+    @Override
+    public DataInputStream openAndSeek(StreamStateHandle handle, Long offset) 
throws IOException {
+        if (!canBeCached(handle)) {
+            return wrapAndSeek(handle.openInputStream(), offset);
+        }
+
+        final FileStateHandle fileHandle = (FileStateHandle) handle;
+        final RefCountedFile refCountedFile = getRefCountedFile(fileHandle);
+
+        FileInputStream fin = openAndSeek(refCountedFile, offset);
+
+        return wrapStream(fileHandle.getFilePath(), fin);
+    }
+
+    private boolean canBeCached(StreamStateHandle handle) throws IOException {
+        if (handle instanceof FileStateHandle) {
+            FileStateHandle fileHandle = (FileStateHandle) handle;
+            return fileHandle.getFilePath().getFileSystem().isDistributedFS();
+        } else {
+            return false;
+        }
+    }
+
+    private RefCountedFile getRefCountedFile(FileStateHandle fileHandle) {
+        return cache.compute(
+                fileHandle.getFilePath(),
+                (key, oldValue) -> {
+                    if (oldValue == null) {
+                        oldValue = downloadToCacheFile(fileHandle);
+                    }
+                    oldValue.retain();
+                    return oldValue;
+                });
+    }
+
+    private RefCountedFile downloadToCacheFile(FileStateHandle fileHandle) {
+        RefCountedFile refCountedFile = null;
+
+        try (FSDataInputStream inputStream = fileHandle.openInputStream()) {
+            File directory = cacheDirectories[next.getAndIncrement() % 
cacheDirectories.length];
+            File file = File.createTempFile(CACHE_FILE_PREFIX, null, 
directory);
+
+            IOUtils.copyBytes(wrap(inputStream), new FileOutputStream(file));
+            LOG.debug(
+                    "download and decompress dstl file : {} to cache file : 
{}",
+                    fileHandle.getFilePath(),
+                    refCountedFile.getFile().getPath());
+
+            refCountedFile = new RefCountedFile(file);
+        } catch (IOException e) {
+            ExceptionUtils.rethrow(e);
+        }
+
+        return refCountedFile;
+    }
+
+    private FileInputStream openAndSeek(RefCountedFile refCountedFile, long 
offset)
+            throws IOException {
+        FileInputStream fin = new FileInputStream(refCountedFile.getFile());
+        if (offset != 0) {
+            LOG.debug("seek to {}", offset);
+            fin.getChannel().position(offset);
+        }
+        return fin;
+    }
+
+    private DataInputStream wrapStream(Path dfsPath, FileInputStream fin) {
+        return new DataInputStream(new BufferedInputStream(fin)) {
+            @Override
+            public void close() throws IOException {
+                try {
+                    super.close();
+                } finally {
+                    cache.computeIfPresent(
+                            dfsPath,
+                            (key, value) -> {
+                                value.release();
+                                if (value.getReferenceCounter() == 
NO_USING_REF_COUNT) {
+                                    cacheCleanScheduler.schedule(
+                                            () -> cleanCacheFile(dfsPath),
+                                            cacheIdleMillis,
+                                            TimeUnit.MILLISECONDS);
+                                }
+                                return value;
+                            });
+                }
+            }
+        };
+    }
+
+    private void cleanCacheFile(Path dfsPath) {
+        cache.computeIfPresent(
+                dfsPath,
+                (key, value) -> {
+                    if (value.getReferenceCounter() == NO_USING_REF_COUNT) {
+                        LOG.debug("clean cached file : {}", 
value.getFile().getPath());
+                        value.release();
+                        return null;

Review Comment:
   This `null` means the mapping will be removed, right? 



##########
flink-dstl/flink-dstl-dfs/src/main/java/org/apache/flink/changelog/fs/ChangelogStreamHandleReaderWithCache.java:
##########
@@ -0,0 +1,203 @@
+/*
+ * 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.RefCountedFile;
+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.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.FileOutputStream;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.Random;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static org.apache.flink.changelog.fs.ChangelogStreamWrapper.wrap;
+import static org.apache.flink.changelog.fs.ChangelogStreamWrapper.wrapAndSeek;
+import static 
org.apache.flink.changelog.fs.FsStateChangelogOptions.CACHE_IDLE_TIMEOUT;
+
+/** StateChangeIterator with local cache. */
+class ChangelogStreamHandleReaderWithCache implements 
ChangelogStreamHandleReader {
+    private static final Logger LOG =
+            
LoggerFactory.getLogger(ChangelogStreamHandleReaderWithCache.class);
+
+    private static final String CACHE_FILE_SUB_DIR = "dstl-cache-file";
+    private static final String CACHE_FILE_PREFIX = "dstl";
+
+    // reference count == 1 means only cache component reference the cache file
+    private static final int NO_USING_REF_COUNT = 1;
+
+    private final File[] cacheDirectories;
+    private final AtomicInteger next;
+
+    private final ConcurrentHashMap<Path, RefCountedFile> cache = new 
ConcurrentHashMap<>();
+    private final ScheduledExecutorService cacheCleanScheduler;
+    private final long cacheIdleMillis;
+
+    ChangelogStreamHandleReaderWithCache(Configuration config) {
+        this.cacheDirectories =
+                Arrays.stream(ConfigurationUtils.parseTempDirectories(config))
+                        .map(path -> new File(path, CACHE_FILE_SUB_DIR))
+                        .toArray(File[]::new);
+        this.next = new AtomicInteger(new 
Random().nextInt(this.cacheDirectories.length));
+
+        this.cacheCleanScheduler =
+                SchedulerFactory.create(1, "ChangelogCacheFileCleanScheduler", 
LOG);
+        this.cacheIdleMillis = config.get(CACHE_IDLE_TIMEOUT).toMillis();
+    }
+
+    @Override
+    public DataInputStream openAndSeek(StreamStateHandle handle, Long offset) 
throws IOException {
+        if (!canBeCached(handle)) {
+            return wrapAndSeek(handle.openInputStream(), offset);
+        }
+
+        final FileStateHandle fileHandle = (FileStateHandle) handle;
+        final RefCountedFile refCountedFile = getRefCountedFile(fileHandle);
+
+        FileInputStream fin = openAndSeek(refCountedFile, offset);
+
+        return wrapStream(fileHandle.getFilePath(), fin);
+    }
+
+    private boolean canBeCached(StreamStateHandle handle) throws IOException {
+        if (handle instanceof FileStateHandle) {
+            FileStateHandle fileHandle = (FileStateHandle) handle;
+            return fileHandle.getFilePath().getFileSystem().isDistributedFS();
+        } else {
+            return false;
+        }
+    }
+
+    private RefCountedFile getRefCountedFile(FileStateHandle fileHandle) {
+        return cache.compute(
+                fileHandle.getFilePath(),
+                (key, oldValue) -> {
+                    if (oldValue == null) {
+                        oldValue = downloadToCacheFile(fileHandle);
+                    }
+                    oldValue.retain();
+                    return oldValue;
+                });
+    }
+
+    private RefCountedFile downloadToCacheFile(FileStateHandle fileHandle) {
+        RefCountedFile refCountedFile = null;

Review Comment:
   Can't this variable can be inlined?



##########
flink-dstl/flink-dstl-dfs/src/main/java/org/apache/flink/changelog/fs/FsStateChangelogStorageForRecovery.java:
##########
@@ -33,8 +34,29 @@
 public class FsStateChangelogStorageForRecovery
         implements StateChangelogStorageView<ChangelogStateHandleStreamImpl> {
 
+    private final ChangelogStreamHandleReaderWithCache 
changelogHandleReaderWithCache;
+
+    public FsStateChangelogStorageForRecovery() {
+        this.changelogHandleReaderWithCache = null;
+    }
+
+    public FsStateChangelogStorageForRecovery(Configuration configuration) {
+        this.changelogHandleReaderWithCache =
+                new ChangelogStreamHandleReaderWithCache(configuration);
+    }
+
     @Override
     public StateChangelogHandleReader<ChangelogStateHandleStreamImpl> 
createReader() {
-        return new StateChangelogHandleStreamHandleReader(new 
StateChangeFormat());
+        return new StateChangelogHandleStreamHandleReader(
+                changelogHandleReaderWithCache != null
+                        ? new 
StateChangeIteratorImpl(changelogHandleReaderWithCache)
+                        : new StateChangeIteratorImpl());

Review Comment:
   Can't the logic of choosing `ChangelogStreamHandleReader` be moved to the 
creator of this object?
   
   So that:
   - `FsStateChangelogStorage` passes no-caching version to 
`FsStateChangelogStorageForRecovery` constructor
   - `FsStateChangelogStorageFactory` passes configuration, from which a 
caching version is built
   
   (so there is no `null` and no `if` here)



##########
flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java:
##########
@@ -1807,6 +1808,7 @@ private void releaseJobResources(JobID jobId, Exception 
cause) {
                         });
         taskManagerMetricGroup.removeJobMetricsGroup(jobId);
         changelogStoragesManager.releaseStateChangelogStorageForJob(jobId);
+        changelogStoragesManager.releaseStateChangelogStorageViewForJob(jobId);

Review Comment:
   Should these two methods be combined into a single one, e.g. 
`releaseResourcesForJob` to abstract from the details?



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