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

2022-08-08 Thread GitBox


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

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



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

2022-08-08 Thread GitBox


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

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



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

2022-08-08 Thread GitBox


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

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



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

2022-08-08 Thread GitBox


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

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



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

2022-08-05 Thread GitBox


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

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

2022-08-05 Thread GitBox


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

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

2022-08-04 Thread GitBox


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

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

2022-08-02 Thread GitBox


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
 STATE_CHANGELOG_STORAGE_FACTORIES = new HashMap<>();
 
+private static final ConcurrentHashMap>
+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-unsubscr...@flink.apache.org

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



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

2022-08-02 Thread GitBox


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,
+AutoCloseable {
+private static final Logger LOG = 
LoggerFactory.getLogger(ChangelogHandleReaderWithCache.class);
+
+private static final String CACHE_FILE_SUB_DIR = "dstl-cache-file";
+
+private final FunctionWithException
+cacheFileCreator;
+private final ConcurrentMap 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();
+

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

2022-08-02 Thread GitBox


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,
+AutoCloseable {
+private static final Logger LOG = 
LoggerFactory.getLogger(ChangelogHandleReaderWithCache.class);
+
+private static final String CACHE_FILE_SUB_DIR = "dstl-cache-file";
+
+private final FunctionWithException
+cacheFileCreator;
+private final ConcurrentMap 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();
+

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

2022-08-02 Thread GitBox


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,
+AutoCloseable {
+private static final Logger LOG = 
LoggerFactory.getLogger(ChangelogHandleReaderWithCache.class);
+
+private static final String CACHE_FILE_SUB_DIR = "dstl-cache-file";
+
+private final FunctionWithException
+cacheFileCreator;
+private final ConcurrentMap 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();
+

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

2022-08-02 Thread GitBox


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,
+AutoCloseable {
+private static final Logger LOG = 
LoggerFactory.getLogger(ChangelogHandleReaderWithCache.class);
+
+private static final String CACHE_FILE_SUB_DIR = "dstl-cache-file";
+
+private final FunctionWithException
+cacheFileCreator;
+private final ConcurrentMap 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();
+

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

2022-08-02 Thread GitBox


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,
+AutoCloseable {
+private static final Logger LOG = 
LoggerFactory.getLogger(ChangelogHandleReaderWithCache.class);
+
+private static final String CACHE_FILE_SUB_DIR = "dstl-cache-file";
+
+private final FunctionWithException
+cacheFileCreator;
+private final ConcurrentMap 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-unsubscr...@flink.apache.org

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



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

2022-08-02 Thread GitBox


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,
+AutoCloseable {
+private static final Logger LOG = 
LoggerFactory.getLogger(ChangelogHandleReaderWithCache.class);
+
+private static final String CACHE_FILE_SUB_DIR = "dstl-cache-file";
+
+private final FunctionWithException
+cacheFileCreator;
+private final ConcurrentMap 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-unsubscr...@flink.apache.org

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



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

2022-08-02 Thread GitBox


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,
+AutoCloseable {
+private static final Logger LOG = 
LoggerFactory.getLogger(ChangelogHandleReaderWithCache.class);
+
+private static final String CACHE_FILE_SUB_DIR = "dstl-cache-file";
+
+private final FunctionWithException
+cacheFileCreator;
+private final ConcurrentMap 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 

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

2022-08-01 Thread GitBox


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,
+AutoCloseable {
+private static final Logger LOG = 
LoggerFactory.getLogger(ChangelogHandleReaderWithCache.class);
+
+private static final String CACHE_FILE_SUB_DIR = "dstl-cache-file";
+
+private final FunctionWithException
+cacheFileCreator;
+private final ConcurrentMap 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-unsubscr...@flink.apache.org

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



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

2022-07-28 Thread GitBox


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,
+AutoCloseable {
+private static final Logger LOG = 
LoggerFactory.getLogger(ChangelogHandleReaderWithCache.class);
+
+private static final String CACHE_FILE_SUB_DIR = "dstl-cache-file";
+
+private final FunctionWithException
+cacheFileCreator;
+private final ConcurrentMap 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-unsubscr...@flink.apache.org

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



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

2022-07-27 Thread GitBox


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
 STATE_CHANGELOG_STORAGE_FACTORIES = new HashMap<>();
 
+private static final ConcurrentHashMap>
+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,
+AutoCloseable {
+private static final Logger LOG = 
LoggerFactory.getLogger(ChangelogHandleReaderWithCache.class);
+
+private static final String CACHE_FILE_SUB_DIR = "dstl-cache-file";
+
+private final FunctionWithException
+cacheFileCreator;
+private final ConcurrentMap 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 

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

2022-07-26 Thread GitBox


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

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

2022-07-24 Thread GitBox


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  CheckpointableKeyedStateBackend restore(
 String subtaskName = env.getTaskInfo().getTaskNameWithSubtasks();
 ExecutionConfig executionConfig = env.getExecutionConfig();
 
+env.getAsyncOperationsThreadPool();
+
 ChangelogStateFactory changelogStateFactory = new 
ChangelogStateFactory();
 CheckpointableKeyedStateBackend 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-unsubscr...@flink.apache.org

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



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

2022-07-07 Thread GitBox


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;