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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to