azagrebin commented on a change in pull request #7351: [FLINK-11008][State
Backends, Checkpointing]SpeedUp upload state files using multithread
URL: https://github.com/apache/flink/pull/7351#discussion_r246443750
##########
File path:
flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/snapshot/RocksIncrementalSnapshotStrategy.java
##########
@@ -410,74 +412,51 @@ private void uploadSstFiles(
// write state data
Preconditions.checkState(localBackupDirectory.exists());
+ Map<StateHandleID, Path> sstFilePaths = new HashMap<>();
+ Map<StateHandleID, Path> miscFilePaths = new
HashMap<>();
+
FileStatus[] fileStatuses =
localBackupDirectory.listStatus();
if (fileStatuses != null) {
- for (FileStatus fileStatus : fileStatuses) {
- final Path filePath =
fileStatus.getPath();
- final String fileName =
filePath.getName();
- final StateHandleID stateHandleID = new
StateHandleID(fileName);
-
- if (fileName.endsWith(SST_FILE_SUFFIX))
{
- final boolean existsAlready =
- baseSstFiles != null &&
baseSstFiles.contains(stateHandleID);
-
- if (existsAlready) {
- // we introduce a
placeholder state handle, that is replaced with the
- // original from the
shared state registry (created from a previous checkpoint)
- sstFiles.put(
- stateHandleID,
- new
PlaceholderStreamStateHandle());
- } else {
-
sstFiles.put(stateHandleID, uploadLocalFileToCheckpointFs(filePath));
- }
- } else {
- StreamStateHandle fileHandle =
uploadLocalFileToCheckpointFs(filePath);
- miscFiles.put(stateHandleID,
fileHandle);
- }
- }
+ createUploadFilePaths(fileStatuses, sstFiles,
sstFilePaths, miscFilePaths);
+
+ sstFiles.putAll(uploadFilesToCheckpointFs(
+ sstFilePaths,
+ numberOfRestoringThreads,
+ checkpointStreamFactory,
+ getSnapshotCloseableRegistry()));
+ miscFiles.putAll(uploadFilesToCheckpointFs(
+ miscFilePaths,
+ numberOfRestoringThreads,
+ checkpointStreamFactory,
+ getSnapshotCloseableRegistry()));
}
}
- private StreamStateHandle uploadLocalFileToCheckpointFs(Path
filePath) throws Exception {
- FSDataInputStream inputStream = null;
- CheckpointStreamFactory.CheckpointStateOutputStream
outputStream = null;
-
- try {
- final byte[] buffer = new
byte[READ_BUFFER_SIZE];
-
- FileSystem backupFileSystem =
localBackupDirectory.getFileSystem();
- inputStream = backupFileSystem.open(filePath);
- registerCloseableForCancellation(inputStream);
-
- outputStream = checkpointStreamFactory
-
.createCheckpointStateOutputStream(CheckpointedStateScope.SHARED);
- registerCloseableForCancellation(outputStream);
-
- while (true) {
- int numBytes = inputStream.read(buffer);
-
- if (numBytes == -1) {
- break;
+ private void createUploadFilePaths(
+ FileStatus[] fileStatuses,
+ Map<StateHandleID, StreamStateHandle> sstFiles,
+ Map<StateHandleID, Path> sstFilePaths,
+ Map<StateHandleID, Path> miscFilePaths) {
+ for (FileStatus fileStatus : fileStatuses) {
+ final Path filePath = fileStatus.getPath();
+ final String fileName = filePath.getName();
+ final StateHandleID stateHandleID = new
StateHandleID(fileName);
+
+ if (fileName.endsWith(SST_FILE_SUFFIX)) {
+ final boolean existsAlready =
+ baseSstFiles != null &&
baseSstFiles.contains(stateHandleID);
+
+ if (existsAlready) {
+ // we introduce a placeholder
state handle, that is replaced with the
+ // original from the shared
state registry (created from a previous checkpoint)
+ sstFiles.put(
Review comment:
let's keep short enough expressions as one line, also `final boolean
existsAlready...`
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
[email protected]
With regards,
Apache Git Services