You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2018/12/26 15:36:31 UTC

[GitHub] azagrebin commented on a change in pull request #7351: [FLINK-11008][State Backends, Checkpointing]SpeedUp upload state files using multithread

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_r244009608
 
 

 ##########
 File path: flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/snapshot/RocksIncrementalSnapshotStrategy.java
 ##########
 @@ -403,85 +409,6 @@ private void cleanupIncompleteSnapshot(@Nonnull List<StateObject> statesToDiscar
 			}
 		}
 
-		private void uploadSstFiles(
-			@Nonnull Map<StateHandleID, StreamStateHandle> sstFiles,
-			@Nonnull Map<StateHandleID, StreamStateHandle> miscFiles) throws Exception {
-
-			// write state data
-			Preconditions.checkState(localBackupDirectory.exists());
-
-			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));
 
 Review comment:
   I think this logic of sst/placeholders/misc file separation belongs here.
   `RocksDbStateDataTransfer` should be responsible only for the upload.
   
   What if instead of calling `uploadLocalFileToCheckpointFs`, we fill `sstFilesToUpload` and `miscFilesToUpload` of `Map<StateHandleID, Path>` in this loop and then call `RocksDbStateDataTransfer.uploadStateFiles` for each map? This `uploadStateFiles` could return `Map<StateHandleID, StreamStateHandle>` which could be put into the final `sstFiles`/`miscFiles`.
   
   I think `ConcurrentHashMap` is not needed then.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services