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: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]