You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2022/07/04 09:53:08 UTC

[GitHub] [flink] zoltar9264 opened a new pull request, #20152: [FLINK-27155][changelog] Reduce multiple reads to the same Changelog …

zoltar9264 opened a new pull request, #20152:
URL: https://github.com/apache/flink/pull/20152

   …file in the same taskmanager during restore
   
   ## What is the purpose of the change
   
   Reduce multiple reads to the same changelog file in the same taskmanager during restare, described in [FLINK-27155](https://issues.apache.org/jira/browse/FLINK-27155).
   
   ## Brief change log
   
     - extract file open from StateChangeFormat to StateChangeIteratorImpl and StateChangeIteratorWithCache
     - add StateChangeIteratorWithCache for cache dstl file to local
     - make FsStateChangelogStorageForRecovery use StateChangeIteratorWithCache for create StateChangelogHandleReader
   
   
   ## Does this pull request potentially affect one of the following parts:
   
     - Dependencies (does it add or upgrade a dependency): no
     - The public API, i.e., is any changed class annotated with `@Public(Evolving)`: no
     - The serializers: no
     - The runtime per-record code paths (performance sensitive): no
     - Anything that affects deployment or recovery: JobManager (and its components), Checkpointing, Kubernetes/Yarn, ZooKeeper: yes
     - The S3 file system connector: no
   
   ## Documentation
   
     - Does this pull request introduce a new feature? no


-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] rkhachatryan commented on a diff in pull request #20152: [FLINK-27155][changelog] Reduce multiple reads to the same Changelog …

Posted by GitBox <gi...@apache.org>.
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] rkhachatryan commented on a diff in pull request #20152: [FLINK-27155][changelog] Reduce multiple reads to the same Changelog …

Posted by GitBox <gi...@apache.org>.
rkhachatryan commented on code in PR #20152:
URL: https://github.com/apache/flink/pull/20152#discussion_r930871288


##########
flink-runtime/src/main/java/org/apache/flink/runtime/state/changelog/StateChangelogStorageLoader.java:
##########
@@ -51,6 +55,9 @@ public class StateChangelogStorageLoader {
     private static final HashMap<String, StateChangelogStorageFactory>
             STATE_CHANGELOG_STORAGE_FACTORIES = new HashMap<>();
 
+    private static final ConcurrentHashMap<JobID, StateChangelogStorageView<?>>
+            changelogStorageViewsByJobId = new ConcurrentHashMap<>();

Review Comment:
   This mapping is similar to `TaskExecutorStateChangelogStoragesManager.changelogStoragesByJobId` - actually it's the same cache but for writing.
   
   I think it would be more consistent to have both maps and the related code in the same class (`TaskExecutorStateChangelogStoragesManager`?).



##########
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);
+

Review Comment:
   According to the javadoc, `parseTempDirectories`
   > Extracts the task manager directories for temporary files
   
   While this component (`ChangelogHandleReaderWithCache`) is per job.
   That means that different jobs will conflict because they will use the same folder and the same file `CACHE_FILE_SUB_DIR`.
   
   Or am I missing something?



##########
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);

Review Comment:
   This means the entire invocation will happen inside `synchronized`..



##########
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) {

Review Comment:
   Could you explain why do we need `synhronized` here at all?



##########
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);

Review Comment:
   Is there any particular reason to close the stream quitely?
   If not, the code can be greatly simplified with try-with-resources and returning `null` after `rethrow`. 



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

Review Comment:
   This method seems a bit complex.
   
   I'd extract some methods from it, for example:
   1. `canBeCached` to check `instanceof` and `isDistributedFS`
   1. `openHandle` to open `fileHandle` and add something reference counted into the map
   1. `openStream` to actually open a steam from a map entry
   1. `wrapStream` into `DataInputStream` and `BufferedInputStream` and to update ref count on close 
   
   The resulting mehtod could look like this (assuming addressing comments about `syncrhonized`):
   ```
       @Override
       public DataInputStream apply(StreamStateHandle handle, Long offset) throws IOException {
           if (!canBeCached(handle)) {
               return wrapAndSeek(handle.openInputStream(), offset);
           }
   
           RefCountedBufferingFileStream refCountedFile =
                   getRefCountedBufferingFileStream((FileStateHandle) handle);
   
           if (refCountedFile.getReferenceCounter() < 1) { 
               // cache file already be deleted, try again
               return apply(handle, offset); 
           }
   
           FileInputStream fin = openStream(offset, refCountedFile);
   
           return wrapStream(
                   ((FileStateHandle) handle).getFilePath(), refCountedFile, fin);
       }
   ```
   



##########
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();

Review Comment:
   I think `retain` must be called much earlier - even before adding an entry to the map (in `computeIfAbsent`). Otherwise, the entry can be invalidated while this thread is using it.
   



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

Review Comment:
   It would be easier to follow the code having a specific interface (`apply` gives a lot of implementations and usages).



##########
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) {

Review Comment:
   The usage of `1` as unused seems counter-intuitive to me.
   But I think if `retain` is called earlier as proposed above then it should be `0`.



##########
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:
   1. Do we need to wait here at all?
   2. If yes, could it be less reource-intensive, e.g. pausing for 50ms in each iteration?



##########
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();

Review Comment:
   Should it be wrapped into `try` to guarantee decrementing the RC?



##########
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();

Review Comment:
   I think this should actually be implemented in this PR unless there is a reason not to do so.



##########
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());

Review Comment:
   Maybe I'm missing something, but `RefCountedBufferingFileStream`/`RefCountedFileWithStream` doesn't seem suitable here (and in general TBH). 
   It is adds buffered output stream capabilities over `RefCountedFile`; but in this case, there is only one writer. Therefore, the output stream from it is not used. Instead, readers call `getInputFile` every time and create and input stream from it.
   
   So how about using `RefCountedFile` here?



##########
flink-runtime/src/main/java/org/apache/flink/runtime/state/changelog/StateChangelogStorageLoader.java:
##########
@@ -51,6 +55,9 @@ public class StateChangelogStorageLoader {
     private static final HashMap<String, StateChangelogStorageFactory>
             STATE_CHANGELOG_STORAGE_FACTORIES = new HashMap<>();
 
+    private static final ConcurrentHashMap<JobID, StateChangelogStorageView<?>>
+            changelogStorageViewsByJobId = new ConcurrentHashMap<>();

Review Comment:
   Another concern is at most one type of changelog per job. 
   Let's consider the following case:
   1. Recover from changelog snapshot, implementation `x`, but use implementation `y` for newer snapshots
   2. Perform a checkpoint (`y`)
   3. Task failure (TM stays online)
   4. Recover from checkpoint (`y`)
   
   In (4), wouldn't implementation `x` be used, while it should be `y`?



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] zoltar9264 commented on pull request #20152: [FLINK-27155][changelog] Reduce multiple reads to the same Changelog …

Posted by GitBox <gi...@apache.org>.
zoltar9264 commented on PR #20152:
URL: https://github.com/apache/flink/pull/20152#issuecomment-1178465547

   Thanks a lot for your advice @rkhachatryan , I feel like I can learn a lot from it. Since there are still some personal matters this week, I will be considering your suggestions one by one next week. Thanks again !


-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] rkhachatryan commented on a diff in pull request #20152: [FLINK-27155][changelog] Reduce multiple reads to the same Changelog …

Posted by GitBox <gi...@apache.org>.
rkhachatryan commented on code in PR #20152:
URL: https://github.com/apache/flink/pull/20152#discussion_r940154643


##########
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:
   > Yes.
   > But as long as the DSTL implementation is using ChangelogStateHandleStreamImpl, then the cache can be used.
   
   `FsStateChangelogStorageFactory` isn't universal, other implementations might not use `StateChangeFormat` at all.
   
   I think it might be solved by introducing map<jobId, map<dstl-identifier, dstl>>.
   However, given that there is only one production implementation ATM, the release timeframe, and that it adds some complexity, I think it's fine to assume single implementation.
   If so, the aforementioned branch isn't necessary.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] zoltar9264 commented on a diff in pull request #20152: [FLINK-27155][changelog] Reduce multiple reads to the same Changelog …

Posted by GitBox <gi...@apache.org>.
zoltar9264 commented on code in PR #20152:
URL: https://github.com/apache/flink/pull/20152#discussion_r940050889


##########
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:
   Sorry I'm missing the specific compression flag bit in `StreamStateHanle`. If another DSTL implementation does not use the first bit as a compression flag, there will be problems.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] zoltar9264 commented on a diff in pull request #20152: [FLINK-27155][changelog] Reduce multiple reads to the same Changelog …

Posted by GitBox <gi...@apache.org>.
zoltar9264 commented on code in PR #20152:
URL: https://github.com/apache/flink/pull/20152#discussion_r928356369


##########
flink-dstl/flink-dstl-dfs/src/main/java/org/apache/flink/changelog/fs/StateChangeIteratorWithCache.java:
##########
@@ -0,0 +1,367 @@
+/*
+ * 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.Path;
+import org.apache.flink.core.memory.DataInputViewStreamWrapper;
+import org.apache.flink.runtime.state.StreamStateHandle;
+import org.apache.flink.runtime.state.changelog.StateChange;
+import org.apache.flink.runtime.state.filesystem.FileStateHandle;
+import org.apache.flink.util.CloseableIterator;
+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.io.InputStream;
+import java.io.OutputStream;
+import java.nio.file.Files;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+
+import static org.apache.flink.configuration.StateChangelogOptions.PERIODIC_MATERIALIZATION_INTERVAL;
+
+/** StateChangeIterator with local cache. */
+class StateChangeIteratorWithCache extends StateChangeIteratorImpl {

Review Comment:
   Thanks @rkhachatryan , I rename the cache component to 'ChangelogHandleReaderWithCache', and make it is only responsible for caching.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] rkhachatryan commented on a diff in pull request #20152: [FLINK-27155][changelog] Reduce multiple reads to the same Changelog …

Posted by GitBox <gi...@apache.org>.
rkhachatryan commented on code in PR #20152:
URL: https://github.com/apache/flink/pull/20152#discussion_r930493192


##########
flink-dstl/flink-dstl-dfs/src/main/java/org/apache/flink/changelog/fs/StateChangeIteratorWithCache.java:
##########
@@ -0,0 +1,367 @@
+/*
+ * 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.Path;
+import org.apache.flink.core.memory.DataInputViewStreamWrapper;
+import org.apache.flink.runtime.state.StreamStateHandle;
+import org.apache.flink.runtime.state.changelog.StateChange;
+import org.apache.flink.runtime.state.filesystem.FileStateHandle;
+import org.apache.flink.util.CloseableIterator;
+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.io.InputStream;
+import java.io.OutputStream;
+import java.nio.file.Files;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+
+import static org.apache.flink.configuration.StateChangelogOptions.PERIODIC_MATERIALIZATION_INTERVAL;
+
+/** StateChangeIterator with local cache. */
+class StateChangeIteratorWithCache extends StateChangeIteratorImpl {
+    private static final Logger LOG = LoggerFactory.getLogger(StateChangeIteratorWithCache.class);
+
+    private static final String CACHE_FILE_PREFIX = "dstl-";
+
+    private final File cacheDir;
+    private final ConcurrentMap<Path, FileCache> cache = new ConcurrentHashMap<>();
+    private final ScheduledExecutorService cacheCleanScheduler;
+    private final ExecutorService downloadExecutor;
+    private final long cacheIdleMillis;
+
+    StateChangeIteratorWithCache(ExecutorService downloadExecutor, Configuration config) {
+        // TODO: 2022/5/31 add a new options for cache idle
+        long cacheIdleMillis = config.get(PERIODIC_MATERIALIZATION_INTERVAL).toMillis();
+        File cacheDir = ConfigurationUtils.getRandomTempDirectory(config);
+
+        this.cacheCleanScheduler =
+                SchedulerFactory.create(1, "ChangelogCacheFileCleanScheduler", LOG);
+        this.downloadExecutor = downloadExecutor;
+        this.cacheIdleMillis = cacheIdleMillis;
+        this.cacheDir = cacheDir;
+    }
+
+    @Override
+    public CloseableIterator<StateChange> read(StreamStateHandle handle, long offset)
+            throws IOException {
+
+        if (!(handle instanceof FileStateHandle)) {
+            return new StateChangeFormat().read(wrapAndSeek(handle.openInputStream(), offset));
+        }
+
+        FileStateHandle fileHandle = (FileStateHandle) handle;
+        DataInputStream input;
+
+        if (fileHandle.getFilePath().getFileSystem().isDistributedFS()) {
+
+            Path dfsPath = fileHandle.getFilePath();
+            FileCache fileCache =
+                    cache.computeIfAbsent(
+                            dfsPath,
+                            key -> {
+                                FileCache fCache = new FileCache(cacheDir);
+                                downloadExecutor.execute(() -> downloadFile(fileHandle, fCache));
+                                return fCache;
+                            });
+
+            FileInputStream fin = fileCache.openAndSeek(offset);
+
+            input =
+                    new DataInputStream(new BufferedInputStream(fin)) {
+                        @Override
+                        public void close() throws IOException {
+                            super.close();
+                            if (fileCache.getRefCount() == 0) {
+                                cacheCleanScheduler.schedule(
+                                        () -> cleanFileCache(dfsPath, fileCache),
+                                        cacheIdleMillis,
+                                        TimeUnit.MILLISECONDS);
+                            }
+                        }
+                    };
+        } else {
+            input = wrapAndSeek(handle.openInputStream(), offset);
+        }
+
+        return new StateChangeFormat().read(input);
+    }
+
+    private DataInputViewStreamWrapper wrapAndSeek(InputStream stream, long offset)
+            throws IOException {
+        DataInputViewStreamWrapper wrappedStream = wrap(stream);
+        if (offset != 0) {
+            LOG.debug("seek to {}", offset);
+            wrappedStream.skipBytesToRead((int) offset);
+        }
+        return wrappedStream;
+    }
+
+    private void downloadFile(FileStateHandle handle, FileCache fileCache) {
+        try {
+            IOUtils.copyBytes(
+                    wrap(handle.openInputStream()), fileCache.getOutputStreamForSaveCacheData());
+            LOG.debug(
+                    "download and decompress dstl file : {} to local cache file : {}",
+                    handle.getFilePath(),
+                    fileCache.getFilePath());
+
+        } catch (IOException e) {
+            fileCache.setSaveCacheDataException(e);
+        }
+    }
+
+    private void cleanFileCache(Path dfsPath, FileCache fileCache) {
+        if (fileCache.getRefCount() == 0) {
+            LOG.debug("clean local cache file : {}", fileCache.getFilePath());
+            cache.remove(dfsPath);
+            fileCache.discard();
+        }
+    }
+
+    static class FileCache {
+
+        private final File cacheDir;
+        private final AtomicLong writeInBytes;
+        private final AtomicBoolean writeComplete;
+        private final AtomicInteger refCount;
+        private final CountDownLatch readLatch;
+
+        private volatile File file;
+        private volatile FileOutputStream fo;
+        private volatile Exception saveCacheDataException;
+
+        FileCache(File cacheDir) {
+            this.cacheDir = cacheDir;
+            this.writeInBytes = new AtomicLong(0);
+            this.writeComplete = new AtomicBoolean(false);
+            this.refCount = new AtomicInteger(0);
+            this.readLatch = new CountDownLatch(1);
+        }
+
+        String getFilePath() {
+            return this.file.getAbsolutePath();
+        }
+
+        OutputStream getOutputStreamForSaveCacheData() throws IOException {
+            synchronized (this) {
+                if (fo == null) {
+                    file = File.createTempFile(CACHE_FILE_PREFIX, null, cacheDir);
+                    fo = new FileOutputStream(file);
+                    readLatch.countDown();
+                } else {
+                    throw new IllegalStateException("only can get OutputStream once !");
+                }
+            }
+
+            return new OutputStream() {
+                @Override
+                public void write(int b) throws IOException {
+                    fo.write(b);
+                    writeInBytes.incrementAndGet();
+                }
+
+                @Override
+                public void write(byte[] b, int off, int len) throws IOException {
+                    fo.write(b, off, len);
+                    writeInBytes.addAndGet(len);
+                }
+
+                @Override
+                public void close() throws IOException {
+                    fo.close();
+                    writeComplete.set(true);
+                }
+            };
+        }
+
+        void setSaveCacheDataException(Exception e) {
+            this.saveCacheDataException = e;
+        }
+
+        int getRefCount() {
+            return refCount.get();
+        }
+
+        private void handoverException() throws IOException {
+            if (saveCacheDataException != null) {
+                throw new IOException(
+                        "there is a exception when save data to cache file : ",
+                        saveCacheDataException);
+            }
+        }
+
+        FileInputStream open() throws IOException {
+            return open0();
+        }
+
+        FileInputStream openAndSeek(long offset) throws IOException {
+            FileInputStream fin = open0();
+            if (offset != 0) {
+                LOG.debug("seek to {}", offset);
+                fin.getChannel().position(offset);
+            }
+            return fin;
+        }
+
+        private FileInputStream open0() throws IOException {
+            try {
+                readLatch.await();
+            } catch (InterruptedException e) {
+                Thread.currentThread().interrupt();
+            }
+
+            refCount.incrementAndGet();
+
+            return new FileInputStream(file) {

Review Comment:
   I think it makes sense @zoltar9264:
   - the new implementation doesn't seem to be affected much by access serialization
   - and there should be no performance regression after this PR



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] rkhachatryan commented on a diff in pull request #20152: [FLINK-27155][changelog] Reduce multiple reads to the same Changelog …

Posted by GitBox <gi...@apache.org>.
rkhachatryan commented on code in PR #20152:
URL: https://github.com/apache/flink/pull/20152#discussion_r935386530


##########
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();

Review Comment:
   I see `retain` is still called **after** `computeIfAbsent`. That forces to:
   - add `synchronized`
   - re-download the file if it was deleted in the meantime (
   ```
   // cache file already be deleted, try again
   return getRefCountedFile(fileHandle);
   ```
   )
   
   Are there any reasons NOT to put `retain()` inside `downloadToCacheFile` ?



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] rkhachatryan commented on a diff in pull request #20152: [FLINK-27155][changelog] Reduce multiple reads to the same Changelog …

Posted by GitBox <gi...@apache.org>.
rkhachatryan commented on code in PR #20152:
URL: https://github.com/apache/flink/pull/20152#discussion_r934989507


##########
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();

Review Comment:
   I'd keep the time unit in it, so `dstl.dfs.download.local-cache.idle-timeout-ms` would be better IMO.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] rkhachatryan commented on pull request #20152: [FLINK-27155][changelog] Reduce multiple reads to the same Changelog …

Posted by GitBox <gi...@apache.org>.
rkhachatryan commented on PR #20152:
URL: https://github.com/apache/flink/pull/20152#issuecomment-1191578017

   Hi @zoltar9264, would you like to continue this work? It would be great to have it in Flink 1.16. 
   The feature freeze for 1.16 is scheduled for Aug 9, so there's not much time left.
   If you have other items, I or my colleauges could work on it if you don't mind.


-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] zoltar9264 commented on a diff in pull request #20152: [FLINK-27155][changelog] Reduce multiple reads to the same Changelog …

Posted by GitBox <gi...@apache.org>.
zoltar9264 commented on code in PR #20152:
URL: https://github.com/apache/flink/pull/20152#discussion_r940047906


##########
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:
   Yes.
   But as long as the DSTL implementation is using `ChangelogStateHangleStreamImpl`, then the cache can be used.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] zoltar9264 commented on a diff in pull request #20152: [FLINK-27155][changelog] Reduce multiple reads to the same Changelog …

Posted by GitBox <gi...@apache.org>.
zoltar9264 commented on code in PR #20152:
URL: https://github.com/apache/flink/pull/20152#discussion_r940050889


##########
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:
   Sorry @rkhachatryan , I'm missing the specific compression flag bit in `StreamStateHandle`. If another DSTL implementation does not use the first bit as a compression flag, there will be problems. Is that possible ? 
   
   About switching DSTL implementation , do you have some suggestions ?



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] zoltar9264 commented on a diff in pull request #20152: [FLINK-27155][changelog] Reduce multiple reads to the same Changelog …

Posted by GitBox <gi...@apache.org>.
zoltar9264 commented on code in PR #20152:
URL: https://github.com/apache/flink/pull/20152#discussion_r932178917


##########
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();

Review Comment:
   Thanks @rkhachatryan , How about `dstl.dfs.download.local-cache.idle-millis` and make default value same to  `PERIODIC_MATERIALIZATION_INTERVAL` ?



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] zoltar9264 commented on pull request #20152: [FLINK-27155][changelog] Reduce multiple reads to the same Changelog …

Posted by GitBox <gi...@apache.org>.
zoltar9264 commented on PR #20152:
URL: https://github.com/apache/flink/pull/20152#issuecomment-1208470388

   Hi @rkhachatryan , I have modified `ChangelogRescalingITCase` to cover `ChangelogStreamHandleReaderWithCache`.


-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] zoltar9264 commented on pull request #20152: [FLINK-27155][changelog] Reduce multiple reads to the same Changelog …

Posted by GitBox <gi...@apache.org>.
zoltar9264 commented on PR #20152:
URL: https://github.com/apache/flink/pull/20152#issuecomment-1208878901

   Thanks @rkhachatryan !  
   I'm very grateful for your help over the past few weeks, I've learned a lot in it, and I'm sorry for the trouble I've caused you due to my inexperience.


-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] rkhachatryan commented on a diff in pull request #20152: [FLINK-27155][changelog] Reduce multiple reads to the same Changelog …

Posted by GitBox <gi...@apache.org>.
rkhachatryan commented on code in PR #20152:
URL: https://github.com/apache/flink/pull/20152#discussion_r938502206


##########
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:
   Hi @zoltar9264 , no worries, thanks for updating the PR! :)
   
   1. Makes sense
   2. Both input and output streams should be closed once eventually; I think it can be done better using try-with-resources, rather than inside `copyBytes`. I don't have a strong preference though.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] zoltar9264 commented on a diff in pull request #20152: [FLINK-27155][changelog] Reduce multiple reads to the same Changelog …

Posted by GitBox <gi...@apache.org>.
zoltar9264 commented on code in PR #20152:
URL: https://github.com/apache/flink/pull/20152#discussion_r938501754


##########
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:
   This should indeed always return a non-null object, I will amend it later.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] zoltar9264 commented on a diff in pull request #20152: [FLINK-27155][changelog] Reduce multiple reads to the same Changelog …

Posted by GitBox <gi...@apache.org>.
zoltar9264 commented on code in PR #20152:
URL: https://github.com/apache/flink/pull/20152#discussion_r940167634


##########
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:
   Ok, I will remove that branch and leave an annotation to remind this .
   
   And by the way , I'm trying to add a test as you mentioned.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] rkhachatryan commented on a diff in pull request #20152: [FLINK-27155][changelog] Reduce multiple reads to the same Changelog …

Posted by GitBox <gi...@apache.org>.
rkhachatryan commented on code in PR #20152:
URL: https://github.com/apache/flink/pull/20152#discussion_r935352864


##########
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);
+

Review Comment:
   I'm not concerned with the deletion, but rather with conflicts inside the directory in case when a TM runs the tasks of more than one job.
   
   Okay, I see `createTempFile` is used which should prevent the conflicts.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] rkhachatryan commented on a diff in pull request #20152: [FLINK-27155][changelog] Reduce multiple reads to the same Changelog …

Posted by GitBox <gi...@apache.org>.
rkhachatryan commented on code in PR #20152:
URL: https://github.com/apache/flink/pull/20152#discussion_r935409659


##########
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:
   Thanks for the explanation.
   If it's not waiting, isn't it more clear to delete the file explicitly like this
   `Files.deleteIfExists(cacheFile.getFile().toPath());`
   ?



##########
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:
   Thanks for the explanation.
   If it's not waiting, isn't it more clear to delete the file explicitly like this
   `Files.deleteIfExists(cacheFile.getFile().toPath());`
   ?



##########
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:
   Thanks for the explanation.
   If it's not waiting, isn't it more clear to delete the file explicitly like this
   `Files.deleteIfExists(cacheFile.getFile().toPath());`
   ?



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] zoltar9264 commented on pull request #20152: [FLINK-27155][changelog] Reduce multiple reads to the same Changelog …

Posted by GitBox <gi...@apache.org>.
zoltar9264 commented on PR #20152:
URL: https://github.com/apache/flink/pull/20152#issuecomment-1207796016

   @flinkbot run azure


-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] rkhachatryan merged pull request #20152: [FLINK-27155][changelog] Reduce multiple reads to the same Changelog …

Posted by GitBox <gi...@apache.org>.
rkhachatryan merged PR #20152:
URL: https://github.com/apache/flink/pull/20152


-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] rkhachatryan commented on a diff in pull request #20152: [FLINK-27155][changelog] Reduce multiple reads to the same Changelog …

Posted by GitBox <gi...@apache.org>.
rkhachatryan commented on code in PR #20152:
URL: https://github.com/apache/flink/pull/20152#discussion_r940040701


##########
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:
   I don't see how does it solve the problem of switching DSTL implementation.
   If different DSTL implementation use `ChangelogStateHandleStreamImpl`  then this branch will be skipped and a cached object will be returned, right?



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] zoltar9264 commented on a diff in pull request #20152: [FLINK-27155][changelog] Reduce multiple reads to the same Changelog …

Posted by GitBox <gi...@apache.org>.
zoltar9264 commented on code in PR #20152:
URL: https://github.com/apache/flink/pull/20152#discussion_r928359054


##########
flink-dstl/flink-dstl-dfs/src/main/java/org/apache/flink/changelog/fs/StateChangeIteratorWithCache.java:
##########
@@ -0,0 +1,367 @@
+/*
+ * 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.Path;
+import org.apache.flink.core.memory.DataInputViewStreamWrapper;
+import org.apache.flink.runtime.state.StreamStateHandle;
+import org.apache.flink.runtime.state.changelog.StateChange;
+import org.apache.flink.runtime.state.filesystem.FileStateHandle;
+import org.apache.flink.util.CloseableIterator;
+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.io.InputStream;
+import java.io.OutputStream;
+import java.nio.file.Files;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+
+import static org.apache.flink.configuration.StateChangelogOptions.PERIODIC_MATERIALIZATION_INTERVAL;
+
+/** StateChangeIterator with local cache. */
+class StateChangeIteratorWithCache extends StateChangeIteratorImpl {
+    private static final Logger LOG = LoggerFactory.getLogger(StateChangeIteratorWithCache.class);
+
+    private static final String CACHE_FILE_PREFIX = "dstl-";
+
+    private final File cacheDir;
+    private final ConcurrentMap<Path, FileCache> cache = new ConcurrentHashMap<>();
+    private final ScheduledExecutorService cacheCleanScheduler;
+    private final ExecutorService downloadExecutor;
+    private final long cacheIdleMillis;
+
+    StateChangeIteratorWithCache(ExecutorService downloadExecutor, Configuration config) {
+        // TODO: 2022/5/31 add a new options for cache idle
+        long cacheIdleMillis = config.get(PERIODIC_MATERIALIZATION_INTERVAL).toMillis();
+        File cacheDir = ConfigurationUtils.getRandomTempDirectory(config);
+
+        this.cacheCleanScheduler =
+                SchedulerFactory.create(1, "ChangelogCacheFileCleanScheduler", LOG);
+        this.downloadExecutor = downloadExecutor;
+        this.cacheIdleMillis = cacheIdleMillis;
+        this.cacheDir = cacheDir;
+    }
+
+    @Override
+    public CloseableIterator<StateChange> read(StreamStateHandle handle, long offset)
+            throws IOException {
+
+        if (!(handle instanceof FileStateHandle)) {
+            return new StateChangeFormat().read(wrapAndSeek(handle.openInputStream(), offset));
+        }
+
+        FileStateHandle fileHandle = (FileStateHandle) handle;
+        DataInputStream input;
+
+        if (fileHandle.getFilePath().getFileSystem().isDistributedFS()) {
+
+            Path dfsPath = fileHandle.getFilePath();
+            FileCache fileCache =
+                    cache.computeIfAbsent(
+                            dfsPath,
+                            key -> {
+                                FileCache fCache = new FileCache(cacheDir);
+                                downloadExecutor.execute(() -> downloadFile(fileHandle, fCache));
+                                return fCache;
+                            });
+
+            FileInputStream fin = fileCache.openAndSeek(offset);
+
+            input =
+                    new DataInputStream(new BufferedInputStream(fin)) {
+                        @Override
+                        public void close() throws IOException {
+                            super.close();
+                            if (fileCache.getRefCount() == 0) {
+                                cacheCleanScheduler.schedule(
+                                        () -> cleanFileCache(dfsPath, fileCache),
+                                        cacheIdleMillis,
+                                        TimeUnit.MILLISECONDS);
+                            }
+                        }
+                    };
+        } else {
+            input = wrapAndSeek(handle.openInputStream(), offset);
+        }
+
+        return new StateChangeFormat().read(input);
+    }
+
+    private DataInputViewStreamWrapper wrapAndSeek(InputStream stream, long offset)
+            throws IOException {
+        DataInputViewStreamWrapper wrappedStream = wrap(stream);
+        if (offset != 0) {
+            LOG.debug("seek to {}", offset);
+            wrappedStream.skipBytesToRead((int) offset);
+        }
+        return wrappedStream;
+    }
+
+    private void downloadFile(FileStateHandle handle, FileCache fileCache) {
+        try {
+            IOUtils.copyBytes(
+                    wrap(handle.openInputStream()), fileCache.getOutputStreamForSaveCacheData());
+            LOG.debug(
+                    "download and decompress dstl file : {} to local cache file : {}",
+                    handle.getFilePath(),
+                    fileCache.getFilePath());
+
+        } catch (IOException e) {
+            fileCache.setSaveCacheDataException(e);
+        }
+    }
+
+    private void cleanFileCache(Path dfsPath, FileCache fileCache) {
+        if (fileCache.getRefCount() == 0) {
+            LOG.debug("clean local cache file : {}", fileCache.getFilePath());
+            cache.remove(dfsPath);
+            fileCache.discard();
+        }
+    }
+
+    static class FileCache {
+
+        private final File cacheDir;
+        private final AtomicLong writeInBytes;
+        private final AtomicBoolean writeComplete;
+        private final AtomicInteger refCount;
+        private final CountDownLatch readLatch;
+
+        private volatile File file;
+        private volatile FileOutputStream fo;
+        private volatile Exception saveCacheDataException;
+
+        FileCache(File cacheDir) {
+            this.cacheDir = cacheDir;
+            this.writeInBytes = new AtomicLong(0);
+            this.writeComplete = new AtomicBoolean(false);
+            this.refCount = new AtomicInteger(0);
+            this.readLatch = new CountDownLatch(1);
+        }
+
+        String getFilePath() {
+            return this.file.getAbsolutePath();
+        }
+
+        OutputStream getOutputStreamForSaveCacheData() throws IOException {
+            synchronized (this) {
+                if (fo == null) {
+                    file = File.createTempFile(CACHE_FILE_PREFIX, null, cacheDir);
+                    fo = new FileOutputStream(file);
+                    readLatch.countDown();
+                } else {
+                    throw new IllegalStateException("only can get OutputStream once !");
+                }
+            }
+
+            return new OutputStream() {
+                @Override
+                public void write(int b) throws IOException {
+                    fo.write(b);
+                    writeInBytes.incrementAndGet();
+                }
+
+                @Override
+                public void write(byte[] b, int off, int len) throws IOException {
+                    fo.write(b, off, len);
+                    writeInBytes.addAndGet(len);
+                }
+
+                @Override
+                public void close() throws IOException {
+                    fo.close();
+                    writeComplete.set(true);
+                }
+            };
+        }
+
+        void setSaveCacheDataException(Exception e) {
+            this.saveCacheDataException = e;
+        }
+
+        int getRefCount() {
+            return refCount.get();
+        }
+
+        private void handoverException() throws IOException {
+            if (saveCacheDataException != null) {
+                throw new IOException(
+                        "there is a exception when save data to cache file : ",
+                        saveCacheDataException);
+            }
+        }
+
+        FileInputStream open() throws IOException {
+            return open0();
+        }
+
+        FileInputStream openAndSeek(long offset) throws IOException {
+            FileInputStream fin = open0();
+            if (offset != 0) {
+                LOG.debug("seek to {}", offset);
+                fin.getChannel().position(offset);
+            }
+            return fin;
+        }
+
+        private FileInputStream open0() throws IOException {
+            try {
+                readLatch.await();
+            } catch (InterruptedException e) {
+                Thread.currentThread().interrupt();
+            }
+
+            refCount.incrementAndGet();
+
+            return new FileInputStream(file) {

Review Comment:
   I tend to open a new ticket to discuss serialize reads, WDYT @rkhachatryan  ?



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] rkhachatryan commented on a diff in pull request #20152: [FLINK-27155][changelog] Reduce multiple reads to the same Changelog …

Posted by GitBox <gi...@apache.org>.
rkhachatryan commented on code in PR #20152:
URL: https://github.com/apache/flink/pull/20152#discussion_r935386530


##########
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();

Review Comment:
   I see `retain` is still called **after** `computeIfAbsent`. That forces to:
   - add `synchronized`
   - re-download the file if it was deleted in the meantime (
   ```
   // cache file already be deleted, try again
   return getRefCountedFile(fileHandle);
   ```
   )
   
   Are there any reasons NOT to put retain inside `downloadToCacheFile` ?



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] rkhachatryan commented on a diff in pull request #20152: [FLINK-27155][changelog] Reduce multiple reads to the same Changelog …

Posted by GitBox <gi...@apache.org>.
rkhachatryan commented on code in PR #20152:
URL: https://github.com/apache/flink/pull/20152#discussion_r940012380


##########
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:
   I meant that this branch can be removed completely without noticable performance drop, or am I missing something?



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] zoltar9264 commented on a diff in pull request #20152: [FLINK-27155][changelog] Reduce multiple reads to the same Changelog …

Posted by GitBox <gi...@apache.org>.
zoltar9264 commented on code in PR #20152:
URL: https://github.com/apache/flink/pull/20152#discussion_r932143434


##########
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);
+

Review Comment:
   Thanks @rkhachatryan , do you mean spread cache files into separate directories by job id ?  Since the cache file is deleted when the TM exits, and it does not need to be managed by the user. I'm not sure if this is necessary.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] zoltar9264 commented on a diff in pull request #20152: [FLINK-27155][changelog] Reduce multiple reads to the same Changelog …

Posted by GitBox <gi...@apache.org>.
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] zoltar9264 commented on a diff in pull request #20152: [FLINK-27155][changelog] Reduce multiple reads to the same Changelog …

Posted by GitBox <gi...@apache.org>.
zoltar9264 commented on code in PR #20152:
URL: https://github.com/apache/flink/pull/20152#discussion_r932160253


##########
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) {

Review Comment:
   Hi @rkhachatryan , cache file with be delete automatically when reference count decrease to 0, but we need a fixed delay. The implementation here is indeed a bit tricky. Do you have some suggestion ?



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] zoltar9264 commented on a diff in pull request #20152: [FLINK-27155][changelog] Reduce multiple reads to the same Changelog …

Posted by GitBox <gi...@apache.org>.
zoltar9264 commented on code in PR #20152:
URL: https://github.com/apache/flink/pull/20152#discussion_r934217853


##########
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) {

Review Comment:
   Hi @rkhachatryan , synchronized here make sure no other thread decrease the ref count and delete the cache file. 



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] rkhachatryan commented on a diff in pull request #20152: [FLINK-27155][changelog] Reduce multiple reads to the same Changelog …

Posted by GitBox <gi...@apache.org>.
rkhachatryan commented on code in PR #20152:
URL: https://github.com/apache/flink/pull/20152#discussion_r916034348


##########
flink-dstl/flink-dstl-dfs/src/main/java/org/apache/flink/changelog/fs/StateChangeIteratorWithCache.java:
##########
@@ -0,0 +1,367 @@
+/*
+ * 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.Path;
+import org.apache.flink.core.memory.DataInputViewStreamWrapper;
+import org.apache.flink.runtime.state.StreamStateHandle;
+import org.apache.flink.runtime.state.changelog.StateChange;
+import org.apache.flink.runtime.state.filesystem.FileStateHandle;
+import org.apache.flink.util.CloseableIterator;
+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.io.InputStream;
+import java.io.OutputStream;
+import java.nio.file.Files;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+
+import static org.apache.flink.configuration.StateChangelogOptions.PERIODIC_MATERIALIZATION_INTERVAL;
+
+/** StateChangeIterator with local cache. */
+class StateChangeIteratorWithCache extends StateChangeIteratorImpl {

Review Comment:
   This class currently has two responsibilities: 1. Caching; 2. Iteration
   
   This leads to code duplication and higher complexity IMO.
   
   Can it only be responsible for caching? I.e for creating `FSDataInputStream` from `StreamStateHandle`?
   Then, we'd have a trivial implementation `handle -> handle.openInputStream` and a caching one.
   And then it could be injected into the iterator in `FsStateChangelogStorageForRecovery.createReader`.
   
   WDYT?



##########
flink-dstl/flink-dstl-dfs/src/main/java/org/apache/flink/changelog/fs/StateChangeIteratorWithCache.java:
##########
@@ -0,0 +1,367 @@
+/*
+ * 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.Path;
+import org.apache.flink.core.memory.DataInputViewStreamWrapper;
+import org.apache.flink.runtime.state.StreamStateHandle;
+import org.apache.flink.runtime.state.changelog.StateChange;
+import org.apache.flink.runtime.state.filesystem.FileStateHandle;
+import org.apache.flink.util.CloseableIterator;
+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.io.InputStream;
+import java.io.OutputStream;
+import java.nio.file.Files;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+
+import static org.apache.flink.configuration.StateChangelogOptions.PERIODIC_MATERIALIZATION_INTERVAL;
+
+/** StateChangeIterator with local cache. */
+class StateChangeIteratorWithCache extends StateChangeIteratorImpl {
+    private static final Logger LOG = LoggerFactory.getLogger(StateChangeIteratorWithCache.class);
+
+    private static final String CACHE_FILE_PREFIX = "dstl-";
+
+    private final File cacheDir;
+    private final ConcurrentMap<Path, FileCache> cache = new ConcurrentHashMap<>();
+    private final ScheduledExecutorService cacheCleanScheduler;
+    private final ExecutorService downloadExecutor;
+    private final long cacheIdleMillis;
+
+    StateChangeIteratorWithCache(ExecutorService downloadExecutor, Configuration config) {
+        // TODO: 2022/5/31 add a new options for cache idle
+        long cacheIdleMillis = config.get(PERIODIC_MATERIALIZATION_INTERVAL).toMillis();
+        File cacheDir = ConfigurationUtils.getRandomTempDirectory(config);
+
+        this.cacheCleanScheduler =
+                SchedulerFactory.create(1, "ChangelogCacheFileCleanScheduler", LOG);
+        this.downloadExecutor = downloadExecutor;
+        this.cacheIdleMillis = cacheIdleMillis;
+        this.cacheDir = cacheDir;
+    }
+
+    @Override
+    public CloseableIterator<StateChange> read(StreamStateHandle handle, long offset)
+            throws IOException {
+
+        if (!(handle instanceof FileStateHandle)) {
+            return new StateChangeFormat().read(wrapAndSeek(handle.openInputStream(), offset));
+        }
+
+        FileStateHandle fileHandle = (FileStateHandle) handle;
+        DataInputStream input;
+
+        if (fileHandle.getFilePath().getFileSystem().isDistributedFS()) {
+
+            Path dfsPath = fileHandle.getFilePath();
+            FileCache fileCache =
+                    cache.computeIfAbsent(
+                            dfsPath,
+                            key -> {
+                                FileCache fCache = new FileCache(cacheDir);
+                                downloadExecutor.execute(() -> downloadFile(fileHandle, fCache));
+                                return fCache;
+                            });
+
+            FileInputStream fin = fileCache.openAndSeek(offset);
+
+            input =
+                    new DataInputStream(new BufferedInputStream(fin)) {
+                        @Override
+                        public void close() throws IOException {
+                            super.close();
+                            if (fileCache.getRefCount() == 0) {
+                                cacheCleanScheduler.schedule(
+                                        () -> cleanFileCache(dfsPath, fileCache),
+                                        cacheIdleMillis,
+                                        TimeUnit.MILLISECONDS);
+                            }
+                        }
+                    };
+        } else {
+            input = wrapAndSeek(handle.openInputStream(), offset);
+        }
+
+        return new StateChangeFormat().read(input);
+    }
+
+    private DataInputViewStreamWrapper wrapAndSeek(InputStream stream, long offset)
+            throws IOException {
+        DataInputViewStreamWrapper wrappedStream = wrap(stream);
+        if (offset != 0) {
+            LOG.debug("seek to {}", offset);
+            wrappedStream.skipBytesToRead((int) offset);
+        }
+        return wrappedStream;
+    }
+
+    private void downloadFile(FileStateHandle handle, FileCache fileCache) {
+        try {
+            IOUtils.copyBytes(
+                    wrap(handle.openInputStream()), fileCache.getOutputStreamForSaveCacheData());
+            LOG.debug(
+                    "download and decompress dstl file : {} to local cache file : {}",
+                    handle.getFilePath(),
+                    fileCache.getFilePath());
+
+        } catch (IOException e) {
+            fileCache.setSaveCacheDataException(e);
+        }
+    }
+
+    private void cleanFileCache(Path dfsPath, FileCache fileCache) {
+        if (fileCache.getRefCount() == 0) {
+            LOG.debug("clean local cache file : {}", fileCache.getFilePath());
+            cache.remove(dfsPath);
+            fileCache.discard();
+        }
+    }
+
+    static class FileCache {
+
+        private final File cacheDir;
+        private final AtomicLong writeInBytes;
+        private final AtomicBoolean writeComplete;
+        private final AtomicInteger refCount;
+        private final CountDownLatch readLatch;
+
+        private volatile File file;
+        private volatile FileOutputStream fo;
+        private volatile Exception saveCacheDataException;
+
+        FileCache(File cacheDir) {
+            this.cacheDir = cacheDir;
+            this.writeInBytes = new AtomicLong(0);
+            this.writeComplete = new AtomicBoolean(false);
+            this.refCount = new AtomicInteger(0);
+            this.readLatch = new CountDownLatch(1);
+        }
+
+        String getFilePath() {
+            return this.file.getAbsolutePath();
+        }
+
+        OutputStream getOutputStreamForSaveCacheData() throws IOException {
+            synchronized (this) {
+                if (fo == null) {
+                    file = File.createTempFile(CACHE_FILE_PREFIX, null, cacheDir);
+                    fo = new FileOutputStream(file);
+                    readLatch.countDown();
+                } else {
+                    throw new IllegalStateException("only can get OutputStream once !");
+                }
+            }
+
+            return new OutputStream() {
+                @Override
+                public void write(int b) throws IOException {
+                    fo.write(b);
+                    writeInBytes.incrementAndGet();
+                }
+
+                @Override
+                public void write(byte[] b, int off, int len) throws IOException {
+                    fo.write(b, off, len);
+                    writeInBytes.addAndGet(len);
+                }
+
+                @Override
+                public void close() throws IOException {
+                    fo.close();
+                    writeComplete.set(true);
+                }
+            };
+        }
+
+        void setSaveCacheDataException(Exception e) {
+            this.saveCacheDataException = e;
+        }
+
+        int getRefCount() {
+            return refCount.get();
+        }
+
+        private void handoverException() throws IOException {
+            if (saveCacheDataException != null) {
+                throw new IOException(
+                        "there is a exception when save data to cache file : ",
+                        saveCacheDataException);
+            }
+        }
+
+        FileInputStream open() throws IOException {
+            return open0();
+        }
+
+        FileInputStream openAndSeek(long offset) throws IOException {
+            FileInputStream fin = open0();
+            if (offset != 0) {
+                LOG.debug("seek to {}", offset);
+                fin.getChannel().position(offset);
+            }
+            return fin;
+        }
+
+        private FileInputStream open0() throws IOException {
+            try {
+                readLatch.await();
+            } catch (InterruptedException e) {
+                Thread.currentThread().interrupt();
+            }
+
+            refCount.incrementAndGet();
+
+            return new FileInputStream(file) {

Review Comment:
   This creates a new file descriptor and potentially calls `fopen` for the same file - for every offset in it.
   Can we reuse the stream, at least for the same thread?
   Or maybe it would be simpler to serialize all accesses to the same file in this PR? Seems like it will change the solution substantially, e.g. by alleviating all the concurrency in `FileCache`.



##########
flink-dstl/flink-dstl-dfs/src/main/java/org/apache/flink/changelog/fs/StateChangeIteratorWithCache.java:
##########
@@ -0,0 +1,367 @@
+/*
+ * 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.Path;
+import org.apache.flink.core.memory.DataInputViewStreamWrapper;
+import org.apache.flink.runtime.state.StreamStateHandle;
+import org.apache.flink.runtime.state.changelog.StateChange;
+import org.apache.flink.runtime.state.filesystem.FileStateHandle;
+import org.apache.flink.util.CloseableIterator;
+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.io.InputStream;
+import java.io.OutputStream;
+import java.nio.file.Files;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+
+import static org.apache.flink.configuration.StateChangelogOptions.PERIODIC_MATERIALIZATION_INTERVAL;
+
+/** StateChangeIterator with local cache. */
+class StateChangeIteratorWithCache extends StateChangeIteratorImpl {
+    private static final Logger LOG = LoggerFactory.getLogger(StateChangeIteratorWithCache.class);
+
+    private static final String CACHE_FILE_PREFIX = "dstl-";
+
+    private final File cacheDir;
+    private final ConcurrentMap<Path, FileCache> cache = new ConcurrentHashMap<>();
+    private final ScheduledExecutorService cacheCleanScheduler;
+    private final ExecutorService downloadExecutor;
+    private final long cacheIdleMillis;
+
+    StateChangeIteratorWithCache(ExecutorService downloadExecutor, Configuration config) {
+        // TODO: 2022/5/31 add a new options for cache idle
+        long cacheIdleMillis = config.get(PERIODIC_MATERIALIZATION_INTERVAL).toMillis();
+        File cacheDir = ConfigurationUtils.getRandomTempDirectory(config);
+
+        this.cacheCleanScheduler =
+                SchedulerFactory.create(1, "ChangelogCacheFileCleanScheduler", LOG);
+        this.downloadExecutor = downloadExecutor;
+        this.cacheIdleMillis = cacheIdleMillis;
+        this.cacheDir = cacheDir;
+    }
+
+    @Override
+    public CloseableIterator<StateChange> read(StreamStateHandle handle, long offset)
+            throws IOException {
+
+        if (!(handle instanceof FileStateHandle)) {
+            return new StateChangeFormat().read(wrapAndSeek(handle.openInputStream(), offset));
+        }
+
+        FileStateHandle fileHandle = (FileStateHandle) handle;
+        DataInputStream input;
+
+        if (fileHandle.getFilePath().getFileSystem().isDistributedFS()) {
+
+            Path dfsPath = fileHandle.getFilePath();
+            FileCache fileCache =
+                    cache.computeIfAbsent(
+                            dfsPath,
+                            key -> {
+                                FileCache fCache = new FileCache(cacheDir);
+                                downloadExecutor.execute(() -> downloadFile(fileHandle, fCache));
+                                return fCache;
+                            });
+
+            FileInputStream fin = fileCache.openAndSeek(offset);
+
+            input =
+                    new DataInputStream(new BufferedInputStream(fin)) {
+                        @Override
+                        public void close() throws IOException {
+                            super.close();

Review Comment:
   Wrap with `try/finally`?



##########
flink-dstl/flink-dstl-dfs/src/main/java/org/apache/flink/changelog/fs/StateChangeIteratorWithCache.java:
##########
@@ -0,0 +1,367 @@
+/*
+ * 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.Path;
+import org.apache.flink.core.memory.DataInputViewStreamWrapper;
+import org.apache.flink.runtime.state.StreamStateHandle;
+import org.apache.flink.runtime.state.changelog.StateChange;
+import org.apache.flink.runtime.state.filesystem.FileStateHandle;
+import org.apache.flink.util.CloseableIterator;
+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.io.InputStream;
+import java.io.OutputStream;
+import java.nio.file.Files;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+
+import static org.apache.flink.configuration.StateChangelogOptions.PERIODIC_MATERIALIZATION_INTERVAL;
+
+/** StateChangeIterator with local cache. */
+class StateChangeIteratorWithCache extends StateChangeIteratorImpl {
+    private static final Logger LOG = LoggerFactory.getLogger(StateChangeIteratorWithCache.class);
+
+    private static final String CACHE_FILE_PREFIX = "dstl-";
+
+    private final File cacheDir;
+    private final ConcurrentMap<Path, FileCache> cache = new ConcurrentHashMap<>();
+    private final ScheduledExecutorService cacheCleanScheduler;
+    private final ExecutorService downloadExecutor;
+    private final long cacheIdleMillis;
+
+    StateChangeIteratorWithCache(ExecutorService downloadExecutor, Configuration config) {
+        // TODO: 2022/5/31 add a new options for cache idle
+        long cacheIdleMillis = config.get(PERIODIC_MATERIALIZATION_INTERVAL).toMillis();
+        File cacheDir = ConfigurationUtils.getRandomTempDirectory(config);
+
+        this.cacheCleanScheduler =
+                SchedulerFactory.create(1, "ChangelogCacheFileCleanScheduler", LOG);
+        this.downloadExecutor = downloadExecutor;
+        this.cacheIdleMillis = cacheIdleMillis;
+        this.cacheDir = cacheDir;
+    }
+
+    @Override
+    public CloseableIterator<StateChange> read(StreamStateHandle handle, long offset)
+            throws IOException {
+
+        if (!(handle instanceof FileStateHandle)) {
+            return new StateChangeFormat().read(wrapAndSeek(handle.openInputStream(), offset));
+        }
+
+        FileStateHandle fileHandle = (FileStateHandle) handle;
+        DataInputStream input;
+
+        if (fileHandle.getFilePath().getFileSystem().isDistributedFS()) {
+
+            Path dfsPath = fileHandle.getFilePath();
+            FileCache fileCache =
+                    cache.computeIfAbsent(
+                            dfsPath,
+                            key -> {
+                                FileCache fCache = new FileCache(cacheDir);
+                                downloadExecutor.execute(() -> downloadFile(fileHandle, fCache));
+                                return fCache;
+                            });
+
+            FileInputStream fin = fileCache.openAndSeek(offset);
+
+            input =
+                    new DataInputStream(new BufferedInputStream(fin)) {
+                        @Override
+                        public void close() throws IOException {
+                            super.close();
+                            if (fileCache.getRefCount() == 0) {
+                                cacheCleanScheduler.schedule(
+                                        () -> cleanFileCache(dfsPath, fileCache),
+                                        cacheIdleMillis,
+                                        TimeUnit.MILLISECONDS);
+                            }
+                        }
+                    };
+        } else {
+            input = wrapAndSeek(handle.openInputStream(), offset);
+        }
+
+        return new StateChangeFormat().read(input);
+    }
+
+    private DataInputViewStreamWrapper wrapAndSeek(InputStream stream, long offset)
+            throws IOException {
+        DataInputViewStreamWrapper wrappedStream = wrap(stream);
+        if (offset != 0) {
+            LOG.debug("seek to {}", offset);
+            wrappedStream.skipBytesToRead((int) offset);
+        }
+        return wrappedStream;
+    }
+
+    private void downloadFile(FileStateHandle handle, FileCache fileCache) {
+        try {
+            IOUtils.copyBytes(
+                    wrap(handle.openInputStream()), fileCache.getOutputStreamForSaveCacheData());
+            LOG.debug(
+                    "download and decompress dstl file : {} to local cache file : {}",
+                    handle.getFilePath(),
+                    fileCache.getFilePath());
+
+        } catch (IOException e) {
+            fileCache.setSaveCacheDataException(e);
+        }
+    }
+
+    private void cleanFileCache(Path dfsPath, FileCache fileCache) {
+        if (fileCache.getRefCount() == 0) {
+            LOG.debug("clean local cache file : {}", fileCache.getFilePath());
+            cache.remove(dfsPath);
+            fileCache.discard();

Review Comment:
   I think there is a race condition here:
   - thread T1 checks the count and sees 0, 
   - T2 increments the count and starts reading
   - T1 removes the file from map and discards it (while it's being read)



##########
flink-dstl/flink-dstl-dfs/src/main/java/org/apache/flink/changelog/fs/StateChangeIteratorWithCache.java:
##########
@@ -0,0 +1,367 @@
+/*
+ * 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.Path;
+import org.apache.flink.core.memory.DataInputViewStreamWrapper;
+import org.apache.flink.runtime.state.StreamStateHandle;
+import org.apache.flink.runtime.state.changelog.StateChange;
+import org.apache.flink.runtime.state.filesystem.FileStateHandle;
+import org.apache.flink.util.CloseableIterator;
+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.io.InputStream;
+import java.io.OutputStream;
+import java.nio.file.Files;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+
+import static org.apache.flink.configuration.StateChangelogOptions.PERIODIC_MATERIALIZATION_INTERVAL;
+
+/** StateChangeIterator with local cache. */
+class StateChangeIteratorWithCache extends StateChangeIteratorImpl {
+    private static final Logger LOG = LoggerFactory.getLogger(StateChangeIteratorWithCache.class);
+
+    private static final String CACHE_FILE_PREFIX = "dstl-";
+
+    private final File cacheDir;
+    private final ConcurrentMap<Path, FileCache> cache = new ConcurrentHashMap<>();
+    private final ScheduledExecutorService cacheCleanScheduler;
+    private final ExecutorService downloadExecutor;
+    private final long cacheIdleMillis;
+
+    StateChangeIteratorWithCache(ExecutorService downloadExecutor, Configuration config) {
+        // TODO: 2022/5/31 add a new options for cache idle
+        long cacheIdleMillis = config.get(PERIODIC_MATERIALIZATION_INTERVAL).toMillis();
+        File cacheDir = ConfigurationUtils.getRandomTempDirectory(config);
+
+        this.cacheCleanScheduler =
+                SchedulerFactory.create(1, "ChangelogCacheFileCleanScheduler", LOG);
+        this.downloadExecutor = downloadExecutor;
+        this.cacheIdleMillis = cacheIdleMillis;
+        this.cacheDir = cacheDir;
+    }
+
+    @Override
+    public CloseableIterator<StateChange> read(StreamStateHandle handle, long offset)
+            throws IOException {
+
+        if (!(handle instanceof FileStateHandle)) {
+            return new StateChangeFormat().read(wrapAndSeek(handle.openInputStream(), offset));
+        }
+
+        FileStateHandle fileHandle = (FileStateHandle) handle;
+        DataInputStream input;
+
+        if (fileHandle.getFilePath().getFileSystem().isDistributedFS()) {
+
+            Path dfsPath = fileHandle.getFilePath();
+            FileCache fileCache =
+                    cache.computeIfAbsent(
+                            dfsPath,
+                            key -> {
+                                FileCache fCache = new FileCache(cacheDir);
+                                downloadExecutor.execute(() -> downloadFile(fileHandle, fCache));
+                                return fCache;
+                            });
+
+            FileInputStream fin = fileCache.openAndSeek(offset);
+
+            input =
+                    new DataInputStream(new BufferedInputStream(fin)) {
+                        @Override
+                        public void close() throws IOException {
+                            super.close();
+                            if (fileCache.getRefCount() == 0) {
+                                cacheCleanScheduler.schedule(
+                                        () -> cleanFileCache(dfsPath, fileCache),
+                                        cacheIdleMillis,
+                                        TimeUnit.MILLISECONDS);
+                            }
+                        }
+                    };
+        } else {
+            input = wrapAndSeek(handle.openInputStream(), offset);
+        }
+
+        return new StateChangeFormat().read(input);
+    }
+
+    private DataInputViewStreamWrapper wrapAndSeek(InputStream stream, long offset)
+            throws IOException {
+        DataInputViewStreamWrapper wrappedStream = wrap(stream);
+        if (offset != 0) {
+            LOG.debug("seek to {}", offset);
+            wrappedStream.skipBytesToRead((int) offset);
+        }
+        return wrappedStream;
+    }
+
+    private void downloadFile(FileStateHandle handle, FileCache fileCache) {
+        try {
+            IOUtils.copyBytes(
+                    wrap(handle.openInputStream()), fileCache.getOutputStreamForSaveCacheData());
+            LOG.debug(
+                    "download and decompress dstl file : {} to local cache file : {}",
+                    handle.getFilePath(),
+                    fileCache.getFilePath());
+
+        } catch (IOException e) {
+            fileCache.setSaveCacheDataException(e);
+        }
+    }
+
+    private void cleanFileCache(Path dfsPath, FileCache fileCache) {
+        if (fileCache.getRefCount() == 0) {
+            LOG.debug("clean local cache file : {}", fileCache.getFilePath());
+            cache.remove(dfsPath);
+            fileCache.discard();
+        }
+    }
+
+    static class FileCache {
+
+        private final File cacheDir;
+        private final AtomicLong writeInBytes;
+        private final AtomicBoolean writeComplete;
+        private final AtomicInteger refCount;
+        private final CountDownLatch readLatch;
+
+        private volatile File file;
+        private volatile FileOutputStream fo;
+        private volatile Exception saveCacheDataException;
+
+        FileCache(File cacheDir) {
+            this.cacheDir = cacheDir;
+            this.writeInBytes = new AtomicLong(0);
+            this.writeComplete = new AtomicBoolean(false);
+            this.refCount = new AtomicInteger(0);
+            this.readLatch = new CountDownLatch(1);
+        }
+
+        String getFilePath() {
+            return this.file.getAbsolutePath();
+        }
+
+        OutputStream getOutputStreamForSaveCacheData() throws IOException {
+            synchronized (this) {
+                if (fo == null) {
+                    file = File.createTempFile(CACHE_FILE_PREFIX, null, cacheDir);
+                    fo = new FileOutputStream(file);
+                    readLatch.countDown();
+                } else {
+                    throw new IllegalStateException("only can get OutputStream once !");
+                }
+            }
+
+            return new OutputStream() {
+                @Override
+                public void write(int b) throws IOException {
+                    fo.write(b);
+                    writeInBytes.incrementAndGet();
+                }
+
+                @Override
+                public void write(byte[] b, int off, int len) throws IOException {
+                    fo.write(b, off, len);
+                    writeInBytes.addAndGet(len);
+                }
+
+                @Override
+                public void close() throws IOException {
+                    fo.close();
+                    writeComplete.set(true);
+                }
+            };
+        }
+
+        void setSaveCacheDataException(Exception e) {
+            this.saveCacheDataException = e;
+        }
+
+        int getRefCount() {
+            return refCount.get();
+        }
+
+        private void handoverException() throws IOException {
+            if (saveCacheDataException != null) {
+                throw new IOException(
+                        "there is a exception when save data to cache file : ",
+                        saveCacheDataException);
+            }
+        }
+
+        FileInputStream open() throws IOException {
+            return open0();
+        }
+
+        FileInputStream openAndSeek(long offset) throws IOException {
+            FileInputStream fin = open0();
+            if (offset != 0) {
+                LOG.debug("seek to {}", offset);
+                fin.getChannel().position(offset);
+            }
+            return fin;
+        }
+
+        private FileInputStream open0() throws IOException {
+            try {
+                readLatch.await();
+            } catch (InterruptedException e) {
+                Thread.currentThread().interrupt();
+            }
+
+            refCount.incrementAndGet();
+
+            return new FileInputStream(file) {
+                private final String id = UUID.randomUUID().toString();
+                private final AtomicBoolean closed = new AtomicBoolean(false);
+
+                @Override
+                public int read() throws IOException {
+                    handoverException();
+                    if (writeComplete.get()) {
+                        return super.read();
+                    } else {
+                        int data = super.read();
+                        if (data != -1) {
+                            return data;
+                        } else {
+                            waitWrite();
+                            return read();
+                        }
+                    }
+                }
+
+                @Override
+                public int read(byte[] b) throws IOException {
+                    handoverException();
+                    if (writeComplete.get()) {
+                        return super.read(b);
+                    } else {
+                        int count = super.read(b);
+                        if (count == -1) {
+                            return count;
+                        } else {
+                            waitWrite();
+                            return read(b);
+                        }
+                    }
+                }
+
+                @Override
+                public int read(byte[] b, int off, int len) throws IOException {
+                    handoverException();
+                    if (writeComplete.get()) {
+                        return super.read(b, off, len);
+                    } else {
+                        int count = super.read(b, off, len);
+                        if (count != -1) {
+                            return count;
+                        } else {
+                            waitWrite();
+                            return read(b, off, len);
+                        }
+                    }
+                }
+
+                @Override
+                public long skip(long n) throws IOException {
+                    handoverException();
+                    if (writeComplete.get()) {
+                        return super.skip(n);
+                    } else {
+                        long skips = super.skip(n);
+                        if (skips == n) {
+                            return skips;
+                        } else {
+                            waitWrite();
+                            return skip(n - skips);
+                        }
+                    }
+                }
+
+                @Override
+                public int available() throws IOException {
+                    handoverException();
+                    if (writeComplete.get()) {
+                        return super.available();
+                    } else {
+                        int count = super.available();
+                        if (count != 0) {
+                            return count;
+                        } else {
+                            return 1;
+                        }
+                    }
+                }
+
+                private void waitWrite() {
+                    long writeInBytes0 = writeInBytes.get();
+                    while (writeInBytes0 == writeInBytes.get() && !writeComplete.get()) {
+                        try {
+                            synchronized (fo) {
+                                fo.wait(10);
+                            }
+                        } catch (InterruptedException e) {
+                            Thread.currentThread().interrupt();

Review Comment:
   Shouldn't it be rethrown (in addition to setting the interrupt flag)?



##########
flink-state-backends/flink-statebackend-changelog/src/main/java/org/apache/flink/state/changelog/ChangelogStateBackend.java:
##########
@@ -87,9 +87,14 @@ protected <K> CheckpointableKeyedStateBackend<K> restore(
         String subtaskName = env.getTaskInfo().getTaskNameWithSubtasks();
         ExecutionConfig executionConfig = env.getExecutionConfig();
 
+        env.getAsyncOperationsThreadPool();
+
         ChangelogStateFactory changelogStateFactory = new ChangelogStateFactory();
         CheckpointableKeyedStateBackend<K> keyedStateBackend =
                 ChangelogBackendRestoreOperation.restore(
+                        env.getJobID(),
+                        env.getAsyncOperationsThreadPool(),
+                        env.getTaskManagerInfo().getConfiguration(),

Review Comment:
   I think this should be `env.getJobConfiguration()` rather than `env.getTaskManagerInfo().getConfiguration()`
   
   cc: @fredia 



##########
flink-runtime/src/main/java/org/apache/flink/runtime/state/changelog/StateChangelogStorageLoader.java:
##########
@@ -104,21 +114,35 @@ public static StateChangelogStorage<?> load(
 
     @Nonnull
     public static StateChangelogStorageView<?> loadFromStateHandle(
-            ChangelogStateHandle changelogStateHandle) throws IOException {
+            JobID jobID,
+            ExecutorService asyncExecutor,
+            Configuration configuration,
+            ChangelogStateHandle changelogStateHandle)
+            throws IOException {
         StateChangelogStorageFactory factory =
                 STATE_CHANGELOG_STORAGE_FACTORIES.get(changelogStateHandle.getStorageIdentifier());
+
         if (factory == null) {
             throw new FlinkRuntimeException(
                     String.format(
                             "Cannot find a factory for changelog storage with name '%s' to restore from '%s'.",
                             changelogStateHandle.getStorageIdentifier(),
                             changelogStateHandle.getClass().getSimpleName()));
-        } else {
-            LOG.info(
-                    "Creating a changelog storage with name '{}' to restore from '{}'.",
-                    changelogStateHandle.getStorageIdentifier(),
-                    changelogStateHandle.getClass().getSimpleName());
-            return factory.createStorageView();
         }
+
+        if (!changelogStorageViewsByJobId.containsKey(jobID)) {
+            synchronized (lock) {
+                if (!changelogStorageViewsByJobId.containsKey(jobID)) {

Review Comment:
   Can this be replaced with `computeIfAbsent`?



##########
flink-state-backends/flink-statebackend-changelog/src/main/java/org/apache/flink/state/changelog/ChangelogStateBackend.java:
##########
@@ -87,9 +87,14 @@ protected <K> CheckpointableKeyedStateBackend<K> restore(
         String subtaskName = env.getTaskInfo().getTaskNameWithSubtasks();
         ExecutionConfig executionConfig = env.getExecutionConfig();
 
+        env.getAsyncOperationsThreadPool();
+

Review Comment:
   This seems unnecessary.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/state/changelog/StateChangelogStorageLoader.java:
##########
@@ -50,6 +54,12 @@ public class StateChangelogStorageLoader {
     private static final HashMap<String, StateChangelogStorageFactory>
             STATE_CHANGELOG_STORAGE_FACTORIES = new HashMap<>();
 
+    private static final Object lock = new Object();
+
+    @GuardedBy("lock")
+    private static final ConcurrentHashMap<JobID, StateChangelogStorageView<?>>
+            changelogStorageViewsByJobId = new ConcurrentHashMap<>();

Review Comment:
   I couldn't find the cleanup of this map.
   
   I think that `GuardedBy` could be removed when addressed the comment below.



##########
flink-dstl/flink-dstl-dfs/src/main/java/org/apache/flink/changelog/fs/StateChangeIteratorWithCache.java:
##########
@@ -0,0 +1,367 @@
+/*
+ * 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.Path;
+import org.apache.flink.core.memory.DataInputViewStreamWrapper;
+import org.apache.flink.runtime.state.StreamStateHandle;
+import org.apache.flink.runtime.state.changelog.StateChange;
+import org.apache.flink.runtime.state.filesystem.FileStateHandle;
+import org.apache.flink.util.CloseableIterator;
+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.io.InputStream;
+import java.io.OutputStream;
+import java.nio.file.Files;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+
+import static org.apache.flink.configuration.StateChangelogOptions.PERIODIC_MATERIALIZATION_INTERVAL;
+
+/** StateChangeIterator with local cache. */
+class StateChangeIteratorWithCache extends StateChangeIteratorImpl {
+    private static final Logger LOG = LoggerFactory.getLogger(StateChangeIteratorWithCache.class);
+
+    private static final String CACHE_FILE_PREFIX = "dstl-";
+
+    private final File cacheDir;
+    private final ConcurrentMap<Path, FileCache> cache = new ConcurrentHashMap<>();
+    private final ScheduledExecutorService cacheCleanScheduler;
+    private final ExecutorService downloadExecutor;
+    private final long cacheIdleMillis;
+
+    StateChangeIteratorWithCache(ExecutorService downloadExecutor, Configuration config) {
+        // TODO: 2022/5/31 add a new options for cache idle
+        long cacheIdleMillis = config.get(PERIODIC_MATERIALIZATION_INTERVAL).toMillis();
+        File cacheDir = ConfigurationUtils.getRandomTempDirectory(config);
+
+        this.cacheCleanScheduler =
+                SchedulerFactory.create(1, "ChangelogCacheFileCleanScheduler", LOG);
+        this.downloadExecutor = downloadExecutor;
+        this.cacheIdleMillis = cacheIdleMillis;
+        this.cacheDir = cacheDir;
+    }
+
+    @Override
+    public CloseableIterator<StateChange> read(StreamStateHandle handle, long offset)
+            throws IOException {
+
+        if (!(handle instanceof FileStateHandle)) {
+            return new StateChangeFormat().read(wrapAndSeek(handle.openInputStream(), offset));
+        }
+
+        FileStateHandle fileHandle = (FileStateHandle) handle;
+        DataInputStream input;
+
+        if (fileHandle.getFilePath().getFileSystem().isDistributedFS()) {
+
+            Path dfsPath = fileHandle.getFilePath();
+            FileCache fileCache =
+                    cache.computeIfAbsent(
+                            dfsPath,
+                            key -> {
+                                FileCache fCache = new FileCache(cacheDir);
+                                downloadExecutor.execute(() -> downloadFile(fileHandle, fCache));
+                                return fCache;
+                            });
+
+            FileInputStream fin = fileCache.openAndSeek(offset);
+
+            input =
+                    new DataInputStream(new BufferedInputStream(fin)) {
+                        @Override
+                        public void close() throws IOException {
+                            super.close();
+                            if (fileCache.getRefCount() == 0) {
+                                cacheCleanScheduler.schedule(
+                                        () -> cleanFileCache(dfsPath, fileCache),
+                                        cacheIdleMillis,
+                                        TimeUnit.MILLISECONDS);
+                            }
+                        }
+                    };
+        } else {
+            input = wrapAndSeek(handle.openInputStream(), offset);
+        }
+
+        return new StateChangeFormat().read(input);
+    }
+
+    private DataInputViewStreamWrapper wrapAndSeek(InputStream stream, long offset)
+            throws IOException {
+        DataInputViewStreamWrapper wrappedStream = wrap(stream);
+        if (offset != 0) {
+            LOG.debug("seek to {}", offset);
+            wrappedStream.skipBytesToRead((int) offset);
+        }
+        return wrappedStream;
+    }
+
+    private void downloadFile(FileStateHandle handle, FileCache fileCache) {
+        try {
+            IOUtils.copyBytes(
+                    wrap(handle.openInputStream()), fileCache.getOutputStreamForSaveCacheData());
+            LOG.debug(
+                    "download and decompress dstl file : {} to local cache file : {}",
+                    handle.getFilePath(),
+                    fileCache.getFilePath());
+
+        } catch (IOException e) {
+            fileCache.setSaveCacheDataException(e);
+        }
+    }
+
+    private void cleanFileCache(Path dfsPath, FileCache fileCache) {
+        if (fileCache.getRefCount() == 0) {
+            LOG.debug("clean local cache file : {}", fileCache.getFilePath());
+            cache.remove(dfsPath);
+            fileCache.discard();
+        }
+    }
+
+    static class FileCache {
+
+        private final File cacheDir;
+        private final AtomicLong writeInBytes;
+        private final AtomicBoolean writeComplete;
+        private final AtomicInteger refCount;
+        private final CountDownLatch readLatch;
+
+        private volatile File file;
+        private volatile FileOutputStream fo;
+        private volatile Exception saveCacheDataException;
+
+        FileCache(File cacheDir) {
+            this.cacheDir = cacheDir;
+            this.writeInBytes = new AtomicLong(0);
+            this.writeComplete = new AtomicBoolean(false);
+            this.refCount = new AtomicInteger(0);
+            this.readLatch = new CountDownLatch(1);
+        }
+
+        String getFilePath() {
+            return this.file.getAbsolutePath();
+        }
+
+        OutputStream getOutputStreamForSaveCacheData() throws IOException {
+            synchronized (this) {
+                if (fo == null) {
+                    file = File.createTempFile(CACHE_FILE_PREFIX, null, cacheDir);
+                    fo = new FileOutputStream(file);
+                    readLatch.countDown();
+                } else {
+                    throw new IllegalStateException("only can get OutputStream once !");
+                }
+            }
+
+            return new OutputStream() {
+                @Override
+                public void write(int b) throws IOException {
+                    fo.write(b);
+                    writeInBytes.incrementAndGet();
+                }
+
+                @Override
+                public void write(byte[] b, int off, int len) throws IOException {
+                    fo.write(b, off, len);
+                    writeInBytes.addAndGet(len);
+                }
+
+                @Override
+                public void close() throws IOException {
+                    fo.close();
+                    writeComplete.set(true);
+                }
+            };
+        }
+
+        void setSaveCacheDataException(Exception e) {
+            this.saveCacheDataException = e;
+        }
+
+        int getRefCount() {
+            return refCount.get();
+        }
+
+        private void handoverException() throws IOException {
+            if (saveCacheDataException != null) {
+                throw new IOException(
+                        "there is a exception when save data to cache file : ",
+                        saveCacheDataException);
+            }
+        }
+
+        FileInputStream open() throws IOException {
+            return open0();
+        }
+
+        FileInputStream openAndSeek(long offset) throws IOException {
+            FileInputStream fin = open0();
+            if (offset != 0) {
+                LOG.debug("seek to {}", offset);
+                fin.getChannel().position(offset);
+            }
+            return fin;
+        }
+
+        private FileInputStream open0() throws IOException {
+            try {
+                readLatch.await();
+            } catch (InterruptedException e) {
+                Thread.currentThread().interrupt();
+            }
+
+            refCount.incrementAndGet();
+
+            return new FileInputStream(file) {
+                private final String id = UUID.randomUUID().toString();
+                private final AtomicBoolean closed = new AtomicBoolean(false);
+
+                @Override
+                public int read() throws IOException {
+                    handoverException();
+                    if (writeComplete.get()) {
+                        return super.read();
+                    } else {
+                        int data = super.read();
+                        if (data != -1) {
+                            return data;
+                        } else {
+                            waitWrite();
+                            return read();
+                        }
+                    }
+                }
+
+                @Override
+                public int read(byte[] b) throws IOException {
+                    handoverException();
+                    if (writeComplete.get()) {
+                        return super.read(b);
+                    } else {
+                        int count = super.read(b);
+                        if (count == -1) {
+                            return count;
+                        } else {
+                            waitWrite();
+                            return read(b);
+                        }
+                    }
+                }
+
+                @Override
+                public int read(byte[] b, int off, int len) throws IOException {
+                    handoverException();
+                    if (writeComplete.get()) {
+                        return super.read(b, off, len);
+                    } else {
+                        int count = super.read(b, off, len);
+                        if (count != -1) {
+                            return count;
+                        } else {
+                            waitWrite();
+                            return read(b, off, len);
+                        }
+                    }
+                }
+
+                @Override
+                public long skip(long n) throws IOException {
+                    handoverException();
+                    if (writeComplete.get()) {
+                        return super.skip(n);
+                    } else {
+                        long skips = super.skip(n);
+                        if (skips == n) {
+                            return skips;
+                        } else {
+                            waitWrite();
+                            return skip(n - skips);
+                        }
+                    }
+                }
+
+                @Override
+                public int available() throws IOException {
+                    handoverException();
+                    if (writeComplete.get()) {
+                        return super.available();
+                    } else {
+                        int count = super.available();
+                        if (count != 0) {
+                            return count;
+                        } else {
+                            return 1;
+                        }
+                    }
+                }
+
+                private void waitWrite() {
+                    long writeInBytes0 = writeInBytes.get();
+                    while (writeInBytes0 == writeInBytes.get() && !writeComplete.get()) {
+                        try {
+                            synchronized (fo) {
+                                fo.wait(10);

Review Comment:
   I don't see the no corresponding `notifyAll`, so it should probably be just `sleep`
   (but see the comment above).



##########
flink-dstl/flink-dstl-dfs/src/main/java/org/apache/flink/changelog/fs/StateChangeIteratorWithCache.java:
##########
@@ -0,0 +1,367 @@
+/*
+ * 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.Path;
+import org.apache.flink.core.memory.DataInputViewStreamWrapper;
+import org.apache.flink.runtime.state.StreamStateHandle;
+import org.apache.flink.runtime.state.changelog.StateChange;
+import org.apache.flink.runtime.state.filesystem.FileStateHandle;
+import org.apache.flink.util.CloseableIterator;
+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.io.InputStream;
+import java.io.OutputStream;
+import java.nio.file.Files;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+
+import static org.apache.flink.configuration.StateChangelogOptions.PERIODIC_MATERIALIZATION_INTERVAL;
+
+/** StateChangeIterator with local cache. */
+class StateChangeIteratorWithCache extends StateChangeIteratorImpl {
+    private static final Logger LOG = LoggerFactory.getLogger(StateChangeIteratorWithCache.class);
+
+    private static final String CACHE_FILE_PREFIX = "dstl-";
+
+    private final File cacheDir;
+    private final ConcurrentMap<Path, FileCache> cache = new ConcurrentHashMap<>();
+    private final ScheduledExecutorService cacheCleanScheduler;
+    private final ExecutorService downloadExecutor;
+    private final long cacheIdleMillis;
+
+    StateChangeIteratorWithCache(ExecutorService downloadExecutor, Configuration config) {
+        // TODO: 2022/5/31 add a new options for cache idle
+        long cacheIdleMillis = config.get(PERIODIC_MATERIALIZATION_INTERVAL).toMillis();
+        File cacheDir = ConfigurationUtils.getRandomTempDirectory(config);
+
+        this.cacheCleanScheduler =
+                SchedulerFactory.create(1, "ChangelogCacheFileCleanScheduler", LOG);
+        this.downloadExecutor = downloadExecutor;
+        this.cacheIdleMillis = cacheIdleMillis;
+        this.cacheDir = cacheDir;
+    }
+
+    @Override
+    public CloseableIterator<StateChange> read(StreamStateHandle handle, long offset)
+            throws IOException {
+
+        if (!(handle instanceof FileStateHandle)) {
+            return new StateChangeFormat().read(wrapAndSeek(handle.openInputStream(), offset));
+        }
+
+        FileStateHandle fileHandle = (FileStateHandle) handle;
+        DataInputStream input;
+
+        if (fileHandle.getFilePath().getFileSystem().isDistributedFS()) {
+
+            Path dfsPath = fileHandle.getFilePath();
+            FileCache fileCache =
+                    cache.computeIfAbsent(
+                            dfsPath,
+                            key -> {
+                                FileCache fCache = new FileCache(cacheDir);
+                                downloadExecutor.execute(() -> downloadFile(fileHandle, fCache));
+                                return fCache;
+                            });
+
+            FileInputStream fin = fileCache.openAndSeek(offset);
+
+            input =
+                    new DataInputStream(new BufferedInputStream(fin)) {
+                        @Override
+                        public void close() throws IOException {
+                            super.close();
+                            if (fileCache.getRefCount() == 0) {
+                                cacheCleanScheduler.schedule(
+                                        () -> cleanFileCache(dfsPath, fileCache),
+                                        cacheIdleMillis,
+                                        TimeUnit.MILLISECONDS);
+                            }
+                        }
+                    };
+        } else {
+            input = wrapAndSeek(handle.openInputStream(), offset);
+        }
+
+        return new StateChangeFormat().read(input);
+    }
+
+    private DataInputViewStreamWrapper wrapAndSeek(InputStream stream, long offset)
+            throws IOException {
+        DataInputViewStreamWrapper wrappedStream = wrap(stream);
+        if (offset != 0) {
+            LOG.debug("seek to {}", offset);
+            wrappedStream.skipBytesToRead((int) offset);
+        }
+        return wrappedStream;
+    }
+
+    private void downloadFile(FileStateHandle handle, FileCache fileCache) {
+        try {
+            IOUtils.copyBytes(
+                    wrap(handle.openInputStream()), fileCache.getOutputStreamForSaveCacheData());
+            LOG.debug(
+                    "download and decompress dstl file : {} to local cache file : {}",
+                    handle.getFilePath(),
+                    fileCache.getFilePath());
+
+        } catch (IOException e) {
+            fileCache.setSaveCacheDataException(e);
+        }
+    }
+
+    private void cleanFileCache(Path dfsPath, FileCache fileCache) {
+        if (fileCache.getRefCount() == 0) {
+            LOG.debug("clean local cache file : {}", fileCache.getFilePath());
+            cache.remove(dfsPath);
+            fileCache.discard();
+        }
+    }
+
+    static class FileCache {
+
+        private final File cacheDir;
+        private final AtomicLong writeInBytes;
+        private final AtomicBoolean writeComplete;
+        private final AtomicInteger refCount;
+        private final CountDownLatch readLatch;
+
+        private volatile File file;
+        private volatile FileOutputStream fo;
+        private volatile Exception saveCacheDataException;
+
+        FileCache(File cacheDir) {
+            this.cacheDir = cacheDir;
+            this.writeInBytes = new AtomicLong(0);
+            this.writeComplete = new AtomicBoolean(false);
+            this.refCount = new AtomicInteger(0);
+            this.readLatch = new CountDownLatch(1);
+        }
+
+        String getFilePath() {
+            return this.file.getAbsolutePath();
+        }
+
+        OutputStream getOutputStreamForSaveCacheData() throws IOException {
+            synchronized (this) {
+                if (fo == null) {
+                    file = File.createTempFile(CACHE_FILE_PREFIX, null, cacheDir);
+                    fo = new FileOutputStream(file);
+                    readLatch.countDown();
+                } else {
+                    throw new IllegalStateException("only can get OutputStream once !");
+                }
+            }
+
+            return new OutputStream() {
+                @Override
+                public void write(int b) throws IOException {
+                    fo.write(b);
+                    writeInBytes.incrementAndGet();
+                }
+
+                @Override
+                public void write(byte[] b, int off, int len) throws IOException {
+                    fo.write(b, off, len);
+                    writeInBytes.addAndGet(len);
+                }
+
+                @Override
+                public void close() throws IOException {
+                    fo.close();
+                    writeComplete.set(true);
+                }
+            };
+        }
+
+        void setSaveCacheDataException(Exception e) {
+            this.saveCacheDataException = e;
+        }
+
+        int getRefCount() {
+            return refCount.get();
+        }
+
+        private void handoverException() throws IOException {
+            if (saveCacheDataException != null) {
+                throw new IOException(
+                        "there is a exception when save data to cache file : ",
+                        saveCacheDataException);
+            }
+        }
+
+        FileInputStream open() throws IOException {
+            return open0();
+        }
+
+        FileInputStream openAndSeek(long offset) throws IOException {
+            FileInputStream fin = open0();
+            if (offset != 0) {
+                LOG.debug("seek to {}", offset);
+                fin.getChannel().position(offset);
+            }
+            return fin;
+        }
+
+        private FileInputStream open0() throws IOException {
+            try {
+                readLatch.await();
+            } catch (InterruptedException e) {
+                Thread.currentThread().interrupt();
+            }
+
+            refCount.incrementAndGet();
+
+            return new FileInputStream(file) {
+                private final String id = UUID.randomUUID().toString();
+                private final AtomicBoolean closed = new AtomicBoolean(false);
+
+                @Override
+                public int read() throws IOException {
+                    handoverException();
+                    if (writeComplete.get()) {
+                        return super.read();
+                    } else {
+                        int data = super.read();
+                        if (data != -1) {
+                            return data;
+                        } else {
+                            waitWrite();
+                            return read();
+                        }
+                    }
+                }
+
+                @Override
+                public int read(byte[] b) throws IOException {
+                    handoverException();
+                    if (writeComplete.get()) {
+                        return super.read(b);
+                    } else {
+                        int count = super.read(b);
+                        if (count == -1) {
+                            return count;
+                        } else {
+                            waitWrite();
+                            return read(b);
+                        }
+                    }
+                }
+
+                @Override
+                public int read(byte[] b, int off, int len) throws IOException {
+                    handoverException();
+                    if (writeComplete.get()) {
+                        return super.read(b, off, len);
+                    } else {
+                        int count = super.read(b, off, len);
+                        if (count != -1) {
+                            return count;
+                        } else {
+                            waitWrite();
+                            return read(b, off, len);
+                        }
+                    }
+                }
+
+                @Override
+                public long skip(long n) throws IOException {
+                    handoverException();
+                    if (writeComplete.get()) {
+                        return super.skip(n);
+                    } else {
+                        long skips = super.skip(n);
+                        if (skips == n) {
+                            return skips;
+                        } else {
+                            waitWrite();
+                            return skip(n - skips);
+                        }
+                    }
+                }
+
+                @Override
+                public int available() throws IOException {
+                    handoverException();
+                    if (writeComplete.get()) {
+                        return super.available();
+                    } else {
+                        int count = super.available();
+                        if (count != 0) {
+                            return count;
+                        } else {
+                            return 1;
+                        }
+                    }
+                }
+
+                private void waitWrite() {
+                    long writeInBytes0 = writeInBytes.get();
+                    while (writeInBytes0 == writeInBytes.get() && !writeComplete.get()) {
+                        try {
+                            synchronized (fo) {

Review Comment:
   Do we really need to allow partial reads before the whole file is downloaded?
   I think the added complexity isn't worth it (and if it is, the optimization can be added later).
   
   Also, performance-wise it could be worse because of the CAS operation and potential context switches on every write.



##########
flink-dstl/flink-dstl-dfs/src/main/java/org/apache/flink/changelog/fs/FsStateChangelogStorageForRecovery.java:
##########
@@ -27,14 +28,31 @@
 
 import javax.annotation.concurrent.ThreadSafe;
 
+import java.util.concurrent.ExecutorService;
+
 /** Filesystem-based implementation of {@link StateChangelogStorage} just for recovery. */
 @Experimental
 @ThreadSafe
 public class FsStateChangelogStorageForRecovery
         implements StateChangelogStorageView<ChangelogStateHandleStreamImpl> {
 
+    private final StateChangeIteratorWithCache stateChangeIteratorWithCache;
+
+    public FsStateChangelogStorageForRecovery() {
+        this.stateChangeIteratorWithCache = null;

Review Comment:
   Can we move `new StateChangeIteratorImpl()` here to avoid nullable field and the correspondign check?



##########
flink-dstl/flink-dstl-dfs/src/main/java/org/apache/flink/changelog/fs/StateChangeIteratorWithCache.java:
##########
@@ -0,0 +1,367 @@
+/*
+ * 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.Path;
+import org.apache.flink.core.memory.DataInputViewStreamWrapper;
+import org.apache.flink.runtime.state.StreamStateHandle;
+import org.apache.flink.runtime.state.changelog.StateChange;
+import org.apache.flink.runtime.state.filesystem.FileStateHandle;
+import org.apache.flink.util.CloseableIterator;
+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.io.InputStream;
+import java.io.OutputStream;
+import java.nio.file.Files;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+
+import static org.apache.flink.configuration.StateChangelogOptions.PERIODIC_MATERIALIZATION_INTERVAL;
+
+/** StateChangeIterator with local cache. */
+class StateChangeIteratorWithCache extends StateChangeIteratorImpl {
+    private static final Logger LOG = LoggerFactory.getLogger(StateChangeIteratorWithCache.class);
+
+    private static final String CACHE_FILE_PREFIX = "dstl-";
+
+    private final File cacheDir;
+    private final ConcurrentMap<Path, FileCache> cache = new ConcurrentHashMap<>();
+    private final ScheduledExecutorService cacheCleanScheduler;
+    private final ExecutorService downloadExecutor;
+    private final long cacheIdleMillis;
+
+    StateChangeIteratorWithCache(ExecutorService downloadExecutor, Configuration config) {
+        // TODO: 2022/5/31 add a new options for cache idle
+        long cacheIdleMillis = config.get(PERIODIC_MATERIALIZATION_INTERVAL).toMillis();
+        File cacheDir = ConfigurationUtils.getRandomTempDirectory(config);
+
+        this.cacheCleanScheduler =
+                SchedulerFactory.create(1, "ChangelogCacheFileCleanScheduler", LOG);
+        this.downloadExecutor = downloadExecutor;
+        this.cacheIdleMillis = cacheIdleMillis;
+        this.cacheDir = cacheDir;
+    }
+
+    @Override
+    public CloseableIterator<StateChange> read(StreamStateHandle handle, long offset)
+            throws IOException {
+
+        if (!(handle instanceof FileStateHandle)) {
+            return new StateChangeFormat().read(wrapAndSeek(handle.openInputStream(), offset));
+        }
+
+        FileStateHandle fileHandle = (FileStateHandle) handle;
+        DataInputStream input;
+
+        if (fileHandle.getFilePath().getFileSystem().isDistributedFS()) {
+
+            Path dfsPath = fileHandle.getFilePath();
+            FileCache fileCache =
+                    cache.computeIfAbsent(
+                            dfsPath,
+                            key -> {
+                                FileCache fCache = new FileCache(cacheDir);
+                                downloadExecutor.execute(() -> downloadFile(fileHandle, fCache));
+                                return fCache;
+                            });
+
+            FileInputStream fin = fileCache.openAndSeek(offset);
+
+            input =
+                    new DataInputStream(new BufferedInputStream(fin)) {
+                        @Override
+                        public void close() throws IOException {
+                            super.close();
+                            if (fileCache.getRefCount() == 0) {
+                                cacheCleanScheduler.schedule(
+                                        () -> cleanFileCache(dfsPath, fileCache),
+                                        cacheIdleMillis,
+                                        TimeUnit.MILLISECONDS);
+                            }
+                        }
+                    };
+        } else {
+            input = wrapAndSeek(handle.openInputStream(), offset);
+        }
+
+        return new StateChangeFormat().read(input);
+    }
+
+    private DataInputViewStreamWrapper wrapAndSeek(InputStream stream, long offset)
+            throws IOException {
+        DataInputViewStreamWrapper wrappedStream = wrap(stream);
+        if (offset != 0) {
+            LOG.debug("seek to {}", offset);
+            wrappedStream.skipBytesToRead((int) offset);
+        }
+        return wrappedStream;
+    }
+
+    private void downloadFile(FileStateHandle handle, FileCache fileCache) {
+        try {
+            IOUtils.copyBytes(
+                    wrap(handle.openInputStream()), fileCache.getOutputStreamForSaveCacheData());
+            LOG.debug(
+                    "download and decompress dstl file : {} to local cache file : {}",
+                    handle.getFilePath(),
+                    fileCache.getFilePath());
+
+        } catch (IOException e) {
+            fileCache.setSaveCacheDataException(e);
+        }
+    }
+
+    private void cleanFileCache(Path dfsPath, FileCache fileCache) {
+        if (fileCache.getRefCount() == 0) {
+            LOG.debug("clean local cache file : {}", fileCache.getFilePath());
+            cache.remove(dfsPath);
+            fileCache.discard();
+        }
+    }
+
+    static class FileCache {
+
+        private final File cacheDir;
+        private final AtomicLong writeInBytes;
+        private final AtomicBoolean writeComplete;
+        private final AtomicInteger refCount;
+        private final CountDownLatch readLatch;
+
+        private volatile File file;
+        private volatile FileOutputStream fo;
+        private volatile Exception saveCacheDataException;
+
+        FileCache(File cacheDir) {
+            this.cacheDir = cacheDir;
+            this.writeInBytes = new AtomicLong(0);
+            this.writeComplete = new AtomicBoolean(false);
+            this.refCount = new AtomicInteger(0);
+            this.readLatch = new CountDownLatch(1);
+        }
+
+        String getFilePath() {
+            return this.file.getAbsolutePath();
+        }
+
+        OutputStream getOutputStreamForSaveCacheData() throws IOException {
+            synchronized (this) {
+                if (fo == null) {
+                    file = File.createTempFile(CACHE_FILE_PREFIX, null, cacheDir);
+                    fo = new FileOutputStream(file);
+                    readLatch.countDown();
+                } else {
+                    throw new IllegalStateException("only can get OutputStream once !");
+                }
+            }
+
+            return new OutputStream() {
+                @Override
+                public void write(int b) throws IOException {
+                    fo.write(b);
+                    writeInBytes.incrementAndGet();
+                }
+
+                @Override
+                public void write(byte[] b, int off, int len) throws IOException {
+                    fo.write(b, off, len);
+                    writeInBytes.addAndGet(len);
+                }
+
+                @Override
+                public void close() throws IOException {
+                    fo.close();
+                    writeComplete.set(true);
+                }
+            };
+        }
+
+        void setSaveCacheDataException(Exception e) {
+            this.saveCacheDataException = e;
+        }
+
+        int getRefCount() {
+            return refCount.get();
+        }
+
+        private void handoverException() throws IOException {
+            if (saveCacheDataException != null) {
+                throw new IOException(
+                        "there is a exception when save data to cache file : ",
+                        saveCacheDataException);
+            }
+        }
+
+        FileInputStream open() throws IOException {
+            return open0();
+        }
+
+        FileInputStream openAndSeek(long offset) throws IOException {
+            FileInputStream fin = open0();
+            if (offset != 0) {
+                LOG.debug("seek to {}", offset);
+                fin.getChannel().position(offset);
+            }
+            return fin;
+        }
+
+        private FileInputStream open0() throws IOException {
+            try {
+                readLatch.await();
+            } catch (InterruptedException e) {
+                Thread.currentThread().interrupt();
+            }
+
+            refCount.incrementAndGet();
+
+            return new FileInputStream(file) {
+                private final String id = UUID.randomUUID().toString();

Review Comment:
   Not used?



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] rkhachatryan commented on a diff in pull request #20152: [FLINK-27155][changelog] Reduce multiple reads to the same Changelog …

Posted by GitBox <gi...@apache.org>.
rkhachatryan commented on code in PR #20152:
URL: https://github.com/apache/flink/pull/20152#discussion_r928325966


##########
flink-state-backends/flink-statebackend-changelog/src/main/java/org/apache/flink/state/changelog/ChangelogStateBackend.java:
##########
@@ -87,9 +87,14 @@ protected <K> CheckpointableKeyedStateBackend<K> restore(
         String subtaskName = env.getTaskInfo().getTaskNameWithSubtasks();
         ExecutionConfig executionConfig = env.getExecutionConfig();
 
+        env.getAsyncOperationsThreadPool();
+
         ChangelogStateFactory changelogStateFactory = new ChangelogStateFactory();
         CheckpointableKeyedStateBackend<K> keyedStateBackend =
                 ChangelogBackendRestoreOperation.restore(
+                        env.getJobID(),
+                        env.getAsyncOperationsThreadPool(),
+                        env.getTaskManagerInfo().getConfiguration(),

Review Comment:
   Thanks for checking this @fredia .
   
   Do you mean changing `StateChangelogStorageFactory` interface and passing `ExecutionConfig` to `createStorageView` instead of `Configuration`?
   
   That would require any new configuration parameter to be placed in `ExecutionConfig`. 
   That would be problematic especially for non-bundled `StateChangelogStorageFactory` implementations.
   
   After, #20160 (FLINK-28286), it should be env.getJobConfiguration()` ideally merged with `env.getTaskManagerInfo().getConfiguration()`, right?
   
   If so, I think this merging can either be implemented in this PR or in FLINK-26372.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] rkhachatryan commented on a diff in pull request #20152: [FLINK-27155][changelog] Reduce multiple reads to the same Changelog …

Posted by GitBox <gi...@apache.org>.
rkhachatryan commented on code in PR #20152:
URL: https://github.com/apache/flink/pull/20152#discussion_r935352864


##########
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);
+

Review Comment:
   I'm not concerned with the deletion, but rather with conflicts inside the directory in case when a TM runs the tasks of more than one job.



##########
flink-dstl/flink-dstl-dfs/src/main/java/org/apache/flink/changelog/fs/ChangelogHandleReader.java:
##########
@@ -0,0 +1,35 @@
+/*
+ * 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.annotation.Internal;
+import org.apache.flink.runtime.state.StreamStateHandle;
+
+import java.io.DataInputStream;
+import java.io.IOException;
+
+/** Changelog handle reader to use by {@link StateChangeIteratorImpl}. */
+@Internal
+public interface ChangelogHandleReader extends AutoCloseable {
+
+    DataInputStream openAndSeek(StreamStateHandle handle, Long aLong) throws IOException;

Review Comment:
   1. There is already an interface `StateChangelogHandleReader`.
   How about calling this interface `ChangelogStreamHandleReader` to differentiate?
   
   2. Does it have to be public?
   
   3. `aLong` -> `offset` ?



##########
flink-runtime/src/main/java/org/apache/flink/runtime/state/changelog/StateChangelogStorageLoader.java:
##########
@@ -51,6 +55,9 @@ public class StateChangelogStorageLoader {
     private static final HashMap<String, StateChangelogStorageFactory>
             STATE_CHANGELOG_STORAGE_FACTORIES = new HashMap<>();
 
+    private static final ConcurrentHashMap<JobID, StateChangelogStorageView<?>>
+            changelogStorageViewsByJobId = new ConcurrentHashMap<>();

Review Comment:
   > Agree with move this map to TaskExecutorStateChangelogStoragesManager for consistent.
   
   I see the two maps are still in different classes ( `StateChangelogStorageLoader.changelogStorageViewsByJobId` and `TaskExecutorStateChangelogStoragesManager.changelogStoragesByJobId`).
   
   > The current implementation does not consider switching StateChangelogStorage implementations
   IIRC, it **is** possible, after FLINK-23252. Or do you see any obstacles preventing from switching the implementation?
   I don't think it's mandatory though, just want to be on the same page.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] rkhachatryan commented on a diff in pull request #20152: [FLINK-27155][changelog] Reduce multiple reads to the same Changelog …

Posted by GitBox <gi...@apache.org>.
rkhachatryan commented on code in PR #20152:
URL: https://github.com/apache/flink/pull/20152#discussion_r938507104


##########
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:
   Good idea, a short comment would be helpful.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] rkhachatryan commented on a diff in pull request #20152: [FLINK-27155][changelog] Reduce multiple reads to the same Changelog …

Posted by GitBox <gi...@apache.org>.
rkhachatryan commented on code in PR #20152:
URL: https://github.com/apache/flink/pull/20152#discussion_r940174368


##########
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:
   Great, thanks!



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] rkhachatryan commented on pull request #20152: [FLINK-27155][changelog] Reduce multiple reads to the same Changelog …

Posted by GitBox <gi...@apache.org>.
rkhachatryan commented on PR #20152:
URL: https://github.com/apache/flink/pull/20152#issuecomment-1208358024

   Hi @zoltar9264, do you mean modifying somehow the existing test? I think it should be fine.


-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] flinkbot commented on pull request #20152: [FLINK-27155][changelog] Reduce multiple reads to the same Changelog …

Posted by GitBox <gi...@apache.org>.
flinkbot commented on PR #20152:
URL: https://github.com/apache/flink/pull/20152#issuecomment-1173614502

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "1cc8f2f87e98e9dfd489b42179afefbfc1b8a2c1",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "1cc8f2f87e98e9dfd489b42179afefbfc1b8a2c1",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 1cc8f2f87e98e9dfd489b42179afefbfc1b8a2c1 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run azure` re-run the last Azure build
   </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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] zoltar9264 commented on a diff in pull request #20152: [FLINK-27155][changelog] Reduce multiple reads to the same Changelog …

Posted by GitBox <gi...@apache.org>.
zoltar9264 commented on code in PR #20152:
URL: https://github.com/apache/flink/pull/20152#discussion_r930567765


##########
flink-dstl/flink-dstl-dfs/src/main/java/org/apache/flink/changelog/fs/StateChangeIteratorWithCache.java:
##########
@@ -0,0 +1,367 @@
+/*
+ * 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.Path;
+import org.apache.flink.core.memory.DataInputViewStreamWrapper;
+import org.apache.flink.runtime.state.StreamStateHandle;
+import org.apache.flink.runtime.state.changelog.StateChange;
+import org.apache.flink.runtime.state.filesystem.FileStateHandle;
+import org.apache.flink.util.CloseableIterator;
+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.io.InputStream;
+import java.io.OutputStream;
+import java.nio.file.Files;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+
+import static org.apache.flink.configuration.StateChangelogOptions.PERIODIC_MATERIALIZATION_INTERVAL;
+
+/** StateChangeIterator with local cache. */
+class StateChangeIteratorWithCache extends StateChangeIteratorImpl {
+    private static final Logger LOG = LoggerFactory.getLogger(StateChangeIteratorWithCache.class);
+
+    private static final String CACHE_FILE_PREFIX = "dstl-";
+
+    private final File cacheDir;
+    private final ConcurrentMap<Path, FileCache> cache = new ConcurrentHashMap<>();
+    private final ScheduledExecutorService cacheCleanScheduler;
+    private final ExecutorService downloadExecutor;
+    private final long cacheIdleMillis;
+
+    StateChangeIteratorWithCache(ExecutorService downloadExecutor, Configuration config) {
+        // TODO: 2022/5/31 add a new options for cache idle
+        long cacheIdleMillis = config.get(PERIODIC_MATERIALIZATION_INTERVAL).toMillis();
+        File cacheDir = ConfigurationUtils.getRandomTempDirectory(config);
+
+        this.cacheCleanScheduler =
+                SchedulerFactory.create(1, "ChangelogCacheFileCleanScheduler", LOG);
+        this.downloadExecutor = downloadExecutor;
+        this.cacheIdleMillis = cacheIdleMillis;
+        this.cacheDir = cacheDir;
+    }
+
+    @Override
+    public CloseableIterator<StateChange> read(StreamStateHandle handle, long offset)
+            throws IOException {
+
+        if (!(handle instanceof FileStateHandle)) {
+            return new StateChangeFormat().read(wrapAndSeek(handle.openInputStream(), offset));
+        }
+
+        FileStateHandle fileHandle = (FileStateHandle) handle;
+        DataInputStream input;
+
+        if (fileHandle.getFilePath().getFileSystem().isDistributedFS()) {
+
+            Path dfsPath = fileHandle.getFilePath();
+            FileCache fileCache =
+                    cache.computeIfAbsent(
+                            dfsPath,
+                            key -> {
+                                FileCache fCache = new FileCache(cacheDir);
+                                downloadExecutor.execute(() -> downloadFile(fileHandle, fCache));
+                                return fCache;
+                            });
+
+            FileInputStream fin = fileCache.openAndSeek(offset);
+
+            input =
+                    new DataInputStream(new BufferedInputStream(fin)) {
+                        @Override
+                        public void close() throws IOException {
+                            super.close();
+                            if (fileCache.getRefCount() == 0) {
+                                cacheCleanScheduler.schedule(
+                                        () -> cleanFileCache(dfsPath, fileCache),
+                                        cacheIdleMillis,
+                                        TimeUnit.MILLISECONDS);
+                            }
+                        }
+                    };
+        } else {
+            input = wrapAndSeek(handle.openInputStream(), offset);
+        }
+
+        return new StateChangeFormat().read(input);
+    }
+
+    private DataInputViewStreamWrapper wrapAndSeek(InputStream stream, long offset)
+            throws IOException {
+        DataInputViewStreamWrapper wrappedStream = wrap(stream);
+        if (offset != 0) {
+            LOG.debug("seek to {}", offset);
+            wrappedStream.skipBytesToRead((int) offset);
+        }
+        return wrappedStream;
+    }
+
+    private void downloadFile(FileStateHandle handle, FileCache fileCache) {
+        try {
+            IOUtils.copyBytes(
+                    wrap(handle.openInputStream()), fileCache.getOutputStreamForSaveCacheData());
+            LOG.debug(
+                    "download and decompress dstl file : {} to local cache file : {}",
+                    handle.getFilePath(),
+                    fileCache.getFilePath());
+
+        } catch (IOException e) {
+            fileCache.setSaveCacheDataException(e);
+        }
+    }
+
+    private void cleanFileCache(Path dfsPath, FileCache fileCache) {
+        if (fileCache.getRefCount() == 0) {
+            LOG.debug("clean local cache file : {}", fileCache.getFilePath());
+            cache.remove(dfsPath);
+            fileCache.discard();
+        }
+    }
+
+    static class FileCache {
+
+        private final File cacheDir;
+        private final AtomicLong writeInBytes;
+        private final AtomicBoolean writeComplete;
+        private final AtomicInteger refCount;
+        private final CountDownLatch readLatch;
+
+        private volatile File file;
+        private volatile FileOutputStream fo;
+        private volatile Exception saveCacheDataException;
+
+        FileCache(File cacheDir) {
+            this.cacheDir = cacheDir;
+            this.writeInBytes = new AtomicLong(0);
+            this.writeComplete = new AtomicBoolean(false);
+            this.refCount = new AtomicInteger(0);
+            this.readLatch = new CountDownLatch(1);
+        }
+
+        String getFilePath() {
+            return this.file.getAbsolutePath();
+        }
+
+        OutputStream getOutputStreamForSaveCacheData() throws IOException {
+            synchronized (this) {
+                if (fo == null) {
+                    file = File.createTempFile(CACHE_FILE_PREFIX, null, cacheDir);
+                    fo = new FileOutputStream(file);
+                    readLatch.countDown();
+                } else {
+                    throw new IllegalStateException("only can get OutputStream once !");
+                }
+            }
+
+            return new OutputStream() {
+                @Override
+                public void write(int b) throws IOException {
+                    fo.write(b);
+                    writeInBytes.incrementAndGet();
+                }
+
+                @Override
+                public void write(byte[] b, int off, int len) throws IOException {
+                    fo.write(b, off, len);
+                    writeInBytes.addAndGet(len);
+                }
+
+                @Override
+                public void close() throws IOException {
+                    fo.close();
+                    writeComplete.set(true);
+                }
+            };
+        }
+
+        void setSaveCacheDataException(Exception e) {
+            this.saveCacheDataException = e;
+        }
+
+        int getRefCount() {
+            return refCount.get();
+        }
+
+        private void handoverException() throws IOException {
+            if (saveCacheDataException != null) {
+                throw new IOException(
+                        "there is a exception when save data to cache file : ",
+                        saveCacheDataException);
+            }
+        }
+
+        FileInputStream open() throws IOException {
+            return open0();
+        }
+
+        FileInputStream openAndSeek(long offset) throws IOException {
+            FileInputStream fin = open0();
+            if (offset != 0) {
+                LOG.debug("seek to {}", offset);
+                fin.getChannel().position(offset);
+            }
+            return fin;
+        }
+
+        private FileInputStream open0() throws IOException {
+            try {
+                readLatch.await();
+            } catch (InterruptedException e) {
+                Thread.currentThread().interrupt();
+            }
+
+            refCount.incrementAndGet();
+
+            return new FileInputStream(file) {

Review Comment:
   Thanks @rkhachatryan , If there are any problems with the new implementation, just let me know and I will fix it as soon as possible.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] zoltar9264 commented on a diff in pull request #20152: [FLINK-27155][changelog] Reduce multiple reads to the same Changelog …

Posted by GitBox <gi...@apache.org>.
zoltar9264 commented on code in PR #20152:
URL: https://github.com/apache/flink/pull/20152#discussion_r932170532


##########
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());

Review Comment:
   Thanks @rkhachatryan , I got it.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] zoltar9264 commented on a diff in pull request #20152: [FLINK-27155][changelog] Reduce multiple reads to the same Changelog …

Posted by GitBox <gi...@apache.org>.
zoltar9264 commented on code in PR #20152:
URL: https://github.com/apache/flink/pull/20152#discussion_r930584225


##########
flink-state-backends/flink-statebackend-changelog/src/main/java/org/apache/flink/state/changelog/ChangelogStateBackend.java:
##########
@@ -87,9 +87,14 @@ protected <K> CheckpointableKeyedStateBackend<K> restore(
         String subtaskName = env.getTaskInfo().getTaskNameWithSubtasks();
         ExecutionConfig executionConfig = env.getExecutionConfig();
 
+        env.getAsyncOperationsThreadPool();
+
         ChangelogStateFactory changelogStateFactory = new ChangelogStateFactory();
         CheckpointableKeyedStateBackend<K> keyedStateBackend =
                 ChangelogBackendRestoreOperation.restore(
+                        env.getJobID(),
+                        env.getAsyncOperationsThreadPool(),
+                        env.getTaskManagerInfo().getConfiguration(),

Review Comment:
   Hi @fredia , thanks for reply. I'm not suggest pass PERIODIC_MATERIALIZATION_INTERVAL directly. StateChangelogStorage may have different implementations, each one has different  options. I think an implementation-specific configuration should not be exposed in the interface.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] zoltar9264 commented on pull request #20152: [FLINK-27155][changelog] Reduce multiple reads to the same Changelog …

Posted by GitBox <gi...@apache.org>.
zoltar9264 commented on PR #20152:
URL: https://github.com/apache/flink/pull/20152#issuecomment-1208338661

   Hi @rkhachatryan , I found that it is possible to verify that cache is working properly in `ChangelogRescalingITCase`, how do you think about ?


-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] rkhachatryan commented on a diff in pull request #20152: [FLINK-27155][changelog] Reduce multiple reads to the same Changelog …

Posted by GitBox <gi...@apache.org>.
rkhachatryan commented on code in PR #20152:
URL: https://github.com/apache/flink/pull/20152#discussion_r935341718


##########
flink-runtime/src/main/java/org/apache/flink/runtime/state/changelog/StateChangelogStorageLoader.java:
##########
@@ -51,6 +55,9 @@ public class StateChangelogStorageLoader {
     private static final HashMap<String, StateChangelogStorageFactory>
             STATE_CHANGELOG_STORAGE_FACTORIES = new HashMap<>();
 
+    private static final ConcurrentHashMap<JobID, StateChangelogStorageView<?>>
+            changelogStorageViewsByJobId = new ConcurrentHashMap<>();

Review Comment:
   > Agree with move this map to TaskExecutorStateChangelogStoragesManager for consistent.
   
   I see the two maps are still in different classes ( `StateChangelogStorageLoader.changelogStorageViewsByJobId` and `TaskExecutorStateChangelogStoragesManager.changelogStoragesByJobId`).
   
   > The current implementation does not consider switching StateChangelogStorage implementations
   
   IIRC, it **is** possible, after FLINK-23252. Or do you see any obstacles preventing from switching the implementation?
   I don't think it's mandatory though, just want to be on the same page.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] zoltar9264 commented on a diff in pull request #20152: [FLINK-27155][changelog] Reduce multiple reads to the same Changelog …

Posted by GitBox <gi...@apache.org>.
zoltar9264 commented on code in PR #20152:
URL: https://github.com/apache/flink/pull/20152#discussion_r932152120


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

Review Comment:
   Thanks for the guidance @rkhachatryan !



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] fredia commented on a diff in pull request #20152: [FLINK-27155][changelog] Reduce multiple reads to the same Changelog …

Posted by GitBox <gi...@apache.org>.
fredia commented on code in PR #20152:
URL: https://github.com/apache/flink/pull/20152#discussion_r930579497


##########
flink-state-backends/flink-statebackend-changelog/src/main/java/org/apache/flink/state/changelog/ChangelogStateBackend.java:
##########
@@ -87,9 +87,14 @@ protected <K> CheckpointableKeyedStateBackend<K> restore(
         String subtaskName = env.getTaskInfo().getTaskNameWithSubtasks();
         ExecutionConfig executionConfig = env.getExecutionConfig();
 
+        env.getAsyncOperationsThreadPool();
+
         ChangelogStateFactory changelogStateFactory = new ChangelogStateFactory();
         CheckpointableKeyedStateBackend<K> keyedStateBackend =
                 ChangelogBackendRestoreOperation.restore(
+                        env.getJobID(),
+                        env.getAsyncOperationsThreadPool(),
+                        env.getTaskManagerInfo().getConfiguration(),

Review Comment:
   > Do you mean changing StateChangelogStorageFactory interface and passing ExecutionConfig to createStorageView instead of Configuration?
   
   I tend to put `PERIODIC_MATERIALIZATION_INTERVAL` directly as a parameter instead of `xxConfiguration`.
   
   > it should be env.getJobConfiguration()ideally merged withenv.getTaskManagerInfo().getConfiguration()`, right?
   
   I'm not sure.  Currently, most configuration is in `env.getTaskManagerInfo().getConfiguration()` or `env.getTaskConfiguration()`,  which is better to merge?
   



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] rkhachatryan commented on a diff in pull request #20152: [FLINK-27155][changelog] Reduce multiple reads to the same Changelog …

Posted by GitBox <gi...@apache.org>.
rkhachatryan commented on code in PR #20152:
URL: https://github.com/apache/flink/pull/20152#discussion_r935400344


##########
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) {

Review Comment:
   I see, thanks for the explanation. I think that's fine.



##########
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) {

Review Comment:
   I think that should be achieved by reference counting (which is backed by an atomic integer).



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] rkhachatryan commented on a diff in pull request #20152: [FLINK-27155][changelog] Reduce multiple reads to the same Changelog …

Posted by GitBox <gi...@apache.org>.
rkhachatryan commented on code in PR #20152:
URL: https://github.com/apache/flink/pull/20152#discussion_r935352864


##########
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);
+

Review Comment:
   ~I'm not concerned with the deletion, but rather with conflicts inside the directory in case when a TM runs the tasks of more than one job.~
   
   Okay, I see `createTempFile` is used which should prevent the conflicts.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] zoltar9264 commented on a diff in pull request #20152: [FLINK-27155][changelog] Reduce multiple reads to the same Changelog …

Posted by GitBox <gi...@apache.org>.
zoltar9264 commented on code in PR #20152:
URL: https://github.com/apache/flink/pull/20152#discussion_r939942900


##########
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:
   Hi @rkhachatryan , I think there are two reasons here can't use `ChangelogStreamHandleReaderWithCache.canBeCached()` :
   1. `flink-runtime` should not depend on `flink-dstl-dfs`
   2. This logic is slightly different from there, `ChangelogStreamHandleReaderWithCache.canBeCached()` only used to determine whether a `StreamStateHandle` can be cached; and this place is determining whether a `changelogStateHandle` is a `ChangelogStateHandleStreamImpl`. In fact, a `ChangelogStateHandleStreamImpl` may consist of multiple `StreamStateHandles`.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] zoltar9264 commented on a diff in pull request #20152: [FLINK-27155][changelog] Reduce multiple reads to the same Changelog …

Posted by GitBox <gi...@apache.org>.
zoltar9264 commented on code in PR #20152:
URL: https://github.com/apache/flink/pull/20152#discussion_r940047906


##########
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:
   Yes.
   But as long as the DSTL implementation is using `ChangelogStateHandleStreamImpl`, then the cache can be used.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] rkhachatryan commented on pull request #20152: [FLINK-27155][changelog] Reduce multiple reads to the same Changelog …

Posted by GitBox <gi...@apache.org>.
rkhachatryan commented on PR #20152:
URL: https://github.com/apache/flink/pull/20152#issuecomment-1208969905

   Thank you for the contribution @zoltar9264, great work!
   Glad to hear that and no worries :)
   


-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] zoltar9264 commented on a diff in pull request #20152: [FLINK-27155][changelog] Reduce multiple reads to the same Changelog …

Posted by GitBox <gi...@apache.org>.
zoltar9264 commented on code in PR #20152:
URL: https://github.com/apache/flink/pull/20152#discussion_r938498533


##########
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:
   Thanks @rkhachatryan !



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] fredia commented on a diff in pull request #20152: [FLINK-27155][changelog] Reduce multiple reads to the same Changelog …

Posted by GitBox <gi...@apache.org>.
fredia commented on code in PR #20152:
URL: https://github.com/apache/flink/pull/20152#discussion_r927673437


##########
flink-state-backends/flink-statebackend-changelog/src/main/java/org/apache/flink/state/changelog/ChangelogStateBackend.java:
##########
@@ -87,9 +87,14 @@ protected <K> CheckpointableKeyedStateBackend<K> restore(
         String subtaskName = env.getTaskInfo().getTaskNameWithSubtasks();
         ExecutionConfig executionConfig = env.getExecutionConfig();
 
+        env.getAsyncOperationsThreadPool();
+
         ChangelogStateFactory changelogStateFactory = new ChangelogStateFactory();
         CheckpointableKeyedStateBackend<K> keyedStateBackend =
                 ChangelogBackendRestoreOperation.restore(
+                        env.getJobID(),
+                        env.getAsyncOperationsThreadPool(),
+                        env.getTaskManagerInfo().getConfiguration(),

Review Comment:
   I debug this locally, there is only  `ENABLE_CHANGE_LOG_FOR_APPLICATION` in `JobConfiguration`, and the `PERIODIC_MATERIALIZATION_INTERVAL`  that `ChangelogHandleReaderWithCache` needs is not in `TaskManagerInfo` too. 
   
   I think this should be `env.getExecutionConfig().getPeriodicMaterializeIntervalMillis()`. 



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] zoltar9264 commented on pull request #20152: [FLINK-27155][changelog] Reduce multiple reads to the same Changelog …

Posted by GitBox <gi...@apache.org>.
zoltar9264 commented on PR #20152:
URL: https://github.com/apache/flink/pull/20152#issuecomment-1192121188

   Hi @rkhachatryan , sorry for the delay in this pr due to my busy work for the past two weeks, I will do it as soon as possible .


-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] zoltar9264 commented on pull request #20152: [FLINK-27155][changelog] Reduce multiple reads to the same Changelog …

Posted by GitBox <gi...@apache.org>.
zoltar9264 commented on PR #20152:
URL: https://github.com/apache/flink/pull/20152#issuecomment-1174527623

   @flinkbot run azure


-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] zoltar9264 commented on a diff in pull request #20152: [FLINK-27155][changelog] Reduce multiple reads to the same Changelog …

Posted by GitBox <gi...@apache.org>.
zoltar9264 commented on code in PR #20152:
URL: https://github.com/apache/flink/pull/20152#discussion_r928360002


##########
flink-dstl/flink-dstl-dfs/src/main/java/org/apache/flink/changelog/fs/StateChangeIteratorWithCache.java:
##########
@@ -0,0 +1,367 @@
+/*
+ * 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.Path;
+import org.apache.flink.core.memory.DataInputViewStreamWrapper;
+import org.apache.flink.runtime.state.StreamStateHandle;
+import org.apache.flink.runtime.state.changelog.StateChange;
+import org.apache.flink.runtime.state.filesystem.FileStateHandle;
+import org.apache.flink.util.CloseableIterator;
+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.io.InputStream;
+import java.io.OutputStream;
+import java.nio.file.Files;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+
+import static org.apache.flink.configuration.StateChangelogOptions.PERIODIC_MATERIALIZATION_INTERVAL;
+
+/** StateChangeIterator with local cache. */
+class StateChangeIteratorWithCache extends StateChangeIteratorImpl {
+    private static final Logger LOG = LoggerFactory.getLogger(StateChangeIteratorWithCache.class);
+
+    private static final String CACHE_FILE_PREFIX = "dstl-";
+
+    private final File cacheDir;
+    private final ConcurrentMap<Path, FileCache> cache = new ConcurrentHashMap<>();
+    private final ScheduledExecutorService cacheCleanScheduler;
+    private final ExecutorService downloadExecutor;
+    private final long cacheIdleMillis;
+
+    StateChangeIteratorWithCache(ExecutorService downloadExecutor, Configuration config) {
+        // TODO: 2022/5/31 add a new options for cache idle
+        long cacheIdleMillis = config.get(PERIODIC_MATERIALIZATION_INTERVAL).toMillis();
+        File cacheDir = ConfigurationUtils.getRandomTempDirectory(config);
+
+        this.cacheCleanScheduler =
+                SchedulerFactory.create(1, "ChangelogCacheFileCleanScheduler", LOG);
+        this.downloadExecutor = downloadExecutor;
+        this.cacheIdleMillis = cacheIdleMillis;
+        this.cacheDir = cacheDir;
+    }
+
+    @Override
+    public CloseableIterator<StateChange> read(StreamStateHandle handle, long offset)
+            throws IOException {
+
+        if (!(handle instanceof FileStateHandle)) {
+            return new StateChangeFormat().read(wrapAndSeek(handle.openInputStream(), offset));
+        }
+
+        FileStateHandle fileHandle = (FileStateHandle) handle;
+        DataInputStream input;
+
+        if (fileHandle.getFilePath().getFileSystem().isDistributedFS()) {
+
+            Path dfsPath = fileHandle.getFilePath();
+            FileCache fileCache =
+                    cache.computeIfAbsent(
+                            dfsPath,
+                            key -> {
+                                FileCache fCache = new FileCache(cacheDir);
+                                downloadExecutor.execute(() -> downloadFile(fileHandle, fCache));
+                                return fCache;
+                            });
+
+            FileInputStream fin = fileCache.openAndSeek(offset);
+
+            input =
+                    new DataInputStream(new BufferedInputStream(fin)) {
+                        @Override
+                        public void close() throws IOException {
+                            super.close();
+                            if (fileCache.getRefCount() == 0) {
+                                cacheCleanScheduler.schedule(
+                                        () -> cleanFileCache(dfsPath, fileCache),
+                                        cacheIdleMillis,
+                                        TimeUnit.MILLISECONDS);
+                            }
+                        }
+                    };
+        } else {
+            input = wrapAndSeek(handle.openInputStream(), offset);
+        }
+
+        return new StateChangeFormat().read(input);
+    }
+
+    private DataInputViewStreamWrapper wrapAndSeek(InputStream stream, long offset)
+            throws IOException {
+        DataInputViewStreamWrapper wrappedStream = wrap(stream);
+        if (offset != 0) {
+            LOG.debug("seek to {}", offset);
+            wrappedStream.skipBytesToRead((int) offset);
+        }
+        return wrappedStream;
+    }
+
+    private void downloadFile(FileStateHandle handle, FileCache fileCache) {
+        try {
+            IOUtils.copyBytes(
+                    wrap(handle.openInputStream()), fileCache.getOutputStreamForSaveCacheData());
+            LOG.debug(
+                    "download and decompress dstl file : {} to local cache file : {}",
+                    handle.getFilePath(),
+                    fileCache.getFilePath());
+
+        } catch (IOException e) {
+            fileCache.setSaveCacheDataException(e);
+        }
+    }
+
+    private void cleanFileCache(Path dfsPath, FileCache fileCache) {
+        if (fileCache.getRefCount() == 0) {
+            LOG.debug("clean local cache file : {}", fileCache.getFilePath());
+            cache.remove(dfsPath);
+            fileCache.discard();
+        }
+    }
+
+    static class FileCache {
+
+        private final File cacheDir;
+        private final AtomicLong writeInBytes;
+        private final AtomicBoolean writeComplete;
+        private final AtomicInteger refCount;
+        private final CountDownLatch readLatch;
+
+        private volatile File file;
+        private volatile FileOutputStream fo;
+        private volatile Exception saveCacheDataException;
+
+        FileCache(File cacheDir) {
+            this.cacheDir = cacheDir;
+            this.writeInBytes = new AtomicLong(0);
+            this.writeComplete = new AtomicBoolean(false);
+            this.refCount = new AtomicInteger(0);
+            this.readLatch = new CountDownLatch(1);
+        }
+
+        String getFilePath() {
+            return this.file.getAbsolutePath();
+        }
+
+        OutputStream getOutputStreamForSaveCacheData() throws IOException {
+            synchronized (this) {
+                if (fo == null) {
+                    file = File.createTempFile(CACHE_FILE_PREFIX, null, cacheDir);
+                    fo = new FileOutputStream(file);
+                    readLatch.countDown();
+                } else {
+                    throw new IllegalStateException("only can get OutputStream once !");
+                }
+            }
+
+            return new OutputStream() {
+                @Override
+                public void write(int b) throws IOException {
+                    fo.write(b);
+                    writeInBytes.incrementAndGet();
+                }
+
+                @Override
+                public void write(byte[] b, int off, int len) throws IOException {
+                    fo.write(b, off, len);
+                    writeInBytes.addAndGet(len);
+                }
+
+                @Override
+                public void close() throws IOException {
+                    fo.close();
+                    writeComplete.set(true);
+                }
+            };
+        }
+
+        void setSaveCacheDataException(Exception e) {
+            this.saveCacheDataException = e;
+        }
+
+        int getRefCount() {
+            return refCount.get();
+        }
+
+        private void handoverException() throws IOException {
+            if (saveCacheDataException != null) {
+                throw new IOException(
+                        "there is a exception when save data to cache file : ",
+                        saveCacheDataException);
+            }
+        }
+
+        FileInputStream open() throws IOException {
+            return open0();
+        }
+
+        FileInputStream openAndSeek(long offset) throws IOException {
+            FileInputStream fin = open0();
+            if (offset != 0) {
+                LOG.debug("seek to {}", offset);
+                fin.getChannel().position(offset);
+            }
+            return fin;
+        }
+
+        private FileInputStream open0() throws IOException {
+            try {
+                readLatch.await();
+            } catch (InterruptedException e) {
+                Thread.currentThread().interrupt();
+            }
+
+            refCount.incrementAndGet();
+
+            return new FileInputStream(file) {
+                private final String id = UUID.randomUUID().toString();
+                private final AtomicBoolean closed = new AtomicBoolean(false);
+
+                @Override
+                public int read() throws IOException {
+                    handoverException();
+                    if (writeComplete.get()) {
+                        return super.read();
+                    } else {
+                        int data = super.read();
+                        if (data != -1) {
+                            return data;
+                        } else {
+                            waitWrite();
+                            return read();
+                        }
+                    }
+                }
+
+                @Override
+                public int read(byte[] b) throws IOException {
+                    handoverException();
+                    if (writeComplete.get()) {
+                        return super.read(b);
+                    } else {
+                        int count = super.read(b);
+                        if (count == -1) {
+                            return count;
+                        } else {
+                            waitWrite();
+                            return read(b);
+                        }
+                    }
+                }
+
+                @Override
+                public int read(byte[] b, int off, int len) throws IOException {
+                    handoverException();
+                    if (writeComplete.get()) {
+                        return super.read(b, off, len);
+                    } else {
+                        int count = super.read(b, off, len);
+                        if (count != -1) {
+                            return count;
+                        } else {
+                            waitWrite();
+                            return read(b, off, len);
+                        }
+                    }
+                }
+
+                @Override
+                public long skip(long n) throws IOException {
+                    handoverException();
+                    if (writeComplete.get()) {
+                        return super.skip(n);
+                    } else {
+                        long skips = super.skip(n);
+                        if (skips == n) {
+                            return skips;
+                        } else {
+                            waitWrite();
+                            return skip(n - skips);
+                        }
+                    }
+                }
+
+                @Override
+                public int available() throws IOException {
+                    handoverException();
+                    if (writeComplete.get()) {
+                        return super.available();
+                    } else {
+                        int count = super.available();
+                        if (count != 0) {
+                            return count;
+                        } else {
+                            return 1;
+                        }
+                    }
+                }
+
+                private void waitWrite() {
+                    long writeInBytes0 = writeInBytes.get();
+                    while (writeInBytes0 == writeInBytes.get() && !writeComplete.get()) {
+                        try {
+                            synchronized (fo) {

Review Comment:
   I really didn't think about the performance issues that CAS operations and context switching could cause performance problem, learn a lot, thanks @rkhachatryan !



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] zoltar9264 commented on a diff in pull request #20152: [FLINK-27155][changelog] Reduce multiple reads to the same Changelog …

Posted by GitBox <gi...@apache.org>.
zoltar9264 commented on code in PR #20152:
URL: https://github.com/apache/flink/pull/20152#discussion_r928365236


##########
flink-dstl/flink-dstl-dfs/src/main/java/org/apache/flink/changelog/fs/StateChangeIteratorWithCache.java:
##########
@@ -0,0 +1,367 @@
+/*
+ * 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.Path;
+import org.apache.flink.core.memory.DataInputViewStreamWrapper;
+import org.apache.flink.runtime.state.StreamStateHandle;
+import org.apache.flink.runtime.state.changelog.StateChange;
+import org.apache.flink.runtime.state.filesystem.FileStateHandle;
+import org.apache.flink.util.CloseableIterator;
+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.io.InputStream;
+import java.io.OutputStream;
+import java.nio.file.Files;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+
+import static org.apache.flink.configuration.StateChangelogOptions.PERIODIC_MATERIALIZATION_INTERVAL;
+
+/** StateChangeIterator with local cache. */
+class StateChangeIteratorWithCache extends StateChangeIteratorImpl {
+    private static final Logger LOG = LoggerFactory.getLogger(StateChangeIteratorWithCache.class);
+
+    private static final String CACHE_FILE_PREFIX = "dstl-";
+
+    private final File cacheDir;
+    private final ConcurrentMap<Path, FileCache> cache = new ConcurrentHashMap<>();
+    private final ScheduledExecutorService cacheCleanScheduler;
+    private final ExecutorService downloadExecutor;
+    private final long cacheIdleMillis;
+
+    StateChangeIteratorWithCache(ExecutorService downloadExecutor, Configuration config) {
+        // TODO: 2022/5/31 add a new options for cache idle
+        long cacheIdleMillis = config.get(PERIODIC_MATERIALIZATION_INTERVAL).toMillis();
+        File cacheDir = ConfigurationUtils.getRandomTempDirectory(config);
+
+        this.cacheCleanScheduler =
+                SchedulerFactory.create(1, "ChangelogCacheFileCleanScheduler", LOG);
+        this.downloadExecutor = downloadExecutor;
+        this.cacheIdleMillis = cacheIdleMillis;
+        this.cacheDir = cacheDir;
+    }
+
+    @Override
+    public CloseableIterator<StateChange> read(StreamStateHandle handle, long offset)
+            throws IOException {
+
+        if (!(handle instanceof FileStateHandle)) {
+            return new StateChangeFormat().read(wrapAndSeek(handle.openInputStream(), offset));
+        }
+
+        FileStateHandle fileHandle = (FileStateHandle) handle;
+        DataInputStream input;
+
+        if (fileHandle.getFilePath().getFileSystem().isDistributedFS()) {
+
+            Path dfsPath = fileHandle.getFilePath();
+            FileCache fileCache =
+                    cache.computeIfAbsent(
+                            dfsPath,
+                            key -> {
+                                FileCache fCache = new FileCache(cacheDir);
+                                downloadExecutor.execute(() -> downloadFile(fileHandle, fCache));
+                                return fCache;
+                            });
+
+            FileInputStream fin = fileCache.openAndSeek(offset);
+
+            input =
+                    new DataInputStream(new BufferedInputStream(fin)) {
+                        @Override
+                        public void close() throws IOException {
+                            super.close();
+                            if (fileCache.getRefCount() == 0) {
+                                cacheCleanScheduler.schedule(
+                                        () -> cleanFileCache(dfsPath, fileCache),
+                                        cacheIdleMillis,
+                                        TimeUnit.MILLISECONDS);
+                            }
+                        }
+                    };
+        } else {
+            input = wrapAndSeek(handle.openInputStream(), offset);
+        }
+
+        return new StateChangeFormat().read(input);
+    }
+
+    private DataInputViewStreamWrapper wrapAndSeek(InputStream stream, long offset)
+            throws IOException {
+        DataInputViewStreamWrapper wrappedStream = wrap(stream);
+        if (offset != 0) {
+            LOG.debug("seek to {}", offset);
+            wrappedStream.skipBytesToRead((int) offset);
+        }
+        return wrappedStream;
+    }
+
+    private void downloadFile(FileStateHandle handle, FileCache fileCache) {
+        try {
+            IOUtils.copyBytes(
+                    wrap(handle.openInputStream()), fileCache.getOutputStreamForSaveCacheData());
+            LOG.debug(
+                    "download and decompress dstl file : {} to local cache file : {}",
+                    handle.getFilePath(),
+                    fileCache.getFilePath());
+
+        } catch (IOException e) {
+            fileCache.setSaveCacheDataException(e);
+        }
+    }
+
+    private void cleanFileCache(Path dfsPath, FileCache fileCache) {
+        if (fileCache.getRefCount() == 0) {
+            LOG.debug("clean local cache file : {}", fileCache.getFilePath());
+            cache.remove(dfsPath);
+            fileCache.discard();

Review Comment:
   You are right , It seems that the new version I pushed last Friday also has this problem, I will check it. Thanks again @rkhachatryan .



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] zoltar9264 commented on a diff in pull request #20152: [FLINK-27155][changelog] Reduce multiple reads to the same Changelog …

Posted by GitBox <gi...@apache.org>.
zoltar9264 commented on code in PR #20152:
URL: https://github.com/apache/flink/pull/20152#discussion_r931045321


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

Review Comment:
   Thanks @rkhachatryan , having a specific interface makes sense, I will modify it later as suggested.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] zoltar9264 commented on a diff in pull request #20152: [FLINK-27155][changelog] Reduce multiple reads to the same Changelog …

Posted by GitBox <gi...@apache.org>.
zoltar9264 commented on code in PR #20152:
URL: https://github.com/apache/flink/pull/20152#discussion_r932130526


##########
flink-runtime/src/main/java/org/apache/flink/runtime/state/changelog/StateChangelogStorageLoader.java:
##########
@@ -51,6 +55,9 @@ public class StateChangelogStorageLoader {
     private static final HashMap<String, StateChangelogStorageFactory>
             STATE_CHANGELOG_STORAGE_FACTORIES = new HashMap<>();
 
+    private static final ConcurrentHashMap<JobID, StateChangelogStorageView<?>>
+            changelogStorageViewsByJobId = new ConcurrentHashMap<>();

Review Comment:
   Thanks @rkhachatryan .
   
   Agree with move this map to `TaskExecutorStateChangelogStoragesManager` for consistent.
   
   The current implementation does not consider switching `StateChangelogStorage` implementations, I think we can only cache `StateChangelogStorageView` with `ChangelogStateHandleStreamImpl`, WDYT ?



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] zoltar9264 commented on a diff in pull request #20152: [FLINK-27155][changelog] Reduce multiple reads to the same Changelog …

Posted by GitBox <gi...@apache.org>.
zoltar9264 commented on code in PR #20152:
URL: https://github.com/apache/flink/pull/20152#discussion_r932154648


##########
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();

Review Comment:
   Yes, you are right.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] rkhachatryan commented on a diff in pull request #20152: [FLINK-27155][changelog] Reduce multiple reads to the same Changelog …

Posted by GitBox <gi...@apache.org>.
rkhachatryan commented on code in PR #20152:
URL: https://github.com/apache/flink/pull/20152#discussion_r932793015


##########
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();

Review Comment:
   Thanks @zoltar9264, sounds good to me.
   WDYT @curcur?



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] zoltar9264 commented on a diff in pull request #20152: [FLINK-27155][changelog] Reduce multiple reads to the same Changelog …

Posted by GitBox <gi...@apache.org>.
zoltar9264 commented on code in PR #20152:
URL: https://github.com/apache/flink/pull/20152#discussion_r938431052


##########
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:
   Yes, may be we should add some annotations ?



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] zoltar9264 commented on a diff in pull request #20152: [FLINK-27155][changelog] Reduce multiple reads to the same Changelog …

Posted by GitBox <gi...@apache.org>.
zoltar9264 commented on code in PR #20152:
URL: https://github.com/apache/flink/pull/20152#discussion_r938430639


##########
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:
   Hi @rkhachatryan , thank you for your patience !  I‘m continuing to update this pr as you suggested.
   
   1. As mentioned in the [design document](https://docs.google.com/document/d/1uTYNutW52cz8GUxzTfIJojctEWD_orucYXs2MU6oHOo/#heading=h.4n9l6huj81wl), the decompressed content is stored in the cache file. This eliminates the need for repeated decompression each time read, and you can seek directly to the desired position.
   2. you are right , how about move `inputStream` into try body ?



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] zoltar9264 commented on a diff in pull request #20152: [FLINK-27155][changelog] Reduce multiple reads to the same Changelog …

Posted by GitBox <gi...@apache.org>.
zoltar9264 commented on code in PR #20152:
URL: https://github.com/apache/flink/pull/20152#discussion_r934329576


##########
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();

Review Comment:
   Hi @rkhachatryan & @curcur , how about `dstl.dfs.download.local-cache.idle-timeout` ?



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] zoltar9264 commented on pull request #20152: [FLINK-27155][changelog] Reduce multiple reads to the same Changelog …

Posted by GitBox <gi...@apache.org>.
zoltar9264 commented on PR #20152:
URL: https://github.com/apache/flink/pull/20152#issuecomment-1202240276

   Hi @rkhachatryan , I updated the PR follow above comments. PTAL.


-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] zoltar9264 commented on a diff in pull request #20152: [FLINK-27155][changelog] Reduce multiple reads to the same Changelog …

Posted by GitBox <gi...@apache.org>.
zoltar9264 commented on code in PR #20152:
URL: https://github.com/apache/flink/pull/20152#discussion_r940035484


##########
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:
   Only cache `StateChangelogStorageView` for `ChangelogStateHandleStreamImpl` just for avoid the problem of switching dstl implementations.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] zoltar9264 commented on a diff in pull request #20152: [FLINK-27155][changelog] Reduce multiple reads to the same Changelog …

Posted by GitBox <gi...@apache.org>.
zoltar9264 commented on code in PR #20152:
URL: https://github.com/apache/flink/pull/20152#discussion_r940050889


##########
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:
   Sorry @rkhachatryan , I'm missing the specific compression flag bit in `StreamStateHanle`. If another DSTL implementation does not use the first bit as a compression flag, there will be problems.
   
   About switching DSTL implementation , do you have some suggestions ?



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org