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/10/05 14:52:49 UTC

[GitHub] azagrebin commented on a change in pull request #6777: [FLINK-10461] [State Backends, Checkpointing] Speed up download files when restore from DFS

azagrebin commented on a change in pull request #6777: [FLINK-10461] [State Backends, Checkpointing] Speed up download files when restore from DFS
URL: https://github.com/apache/flink/pull/6777#discussion_r223030850
 
 

 ##########
 File path: flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java
 ##########
 @@ -1286,10 +1289,38 @@ private void transferAllDataFromStateHandles(
 			Map<StateHandleID, StreamStateHandle> stateHandleMap,
 			Path restoreInstancePath) throws IOException {
 
+			ExecutorService executorService = Executors.newFixedThreadPool(5);
+			List<FutureTask<Void>> tasks = new ArrayList<>(stateHandleMap.size());
+
 			for (Map.Entry<StateHandleID, StreamStateHandle> entry : stateHandleMap.entrySet()) {
 				StateHandleID stateHandleID = entry.getKey();
 				StreamStateHandle remoteFileHandle = entry.getValue();
-				copyStateDataHandleData(new Path(restoreInstancePath, stateHandleID.toString()), remoteFileHandle);
+
+				FutureTask<Void> task = new FutureTask<>(() -> {
+					try {
+						copyStateDataHandleData(new Path(restoreInstancePath, stateHandleID.toString()), remoteFileHandle);
+					} catch (IOException e) {
+						LOG.error("Copy State Data error, stateHandleID [{}], remoteFileHandle[{}]", stateHandleID.toString(), remoteFileHandle);
+						throw e;
+					}
+					return null;
+				});
+				tasks.add(task);
+				executorService.submit(task);
 
 Review comment:
   I suggest to use `CompletableFuture.runAsync(Runnable, Executor)` instead of `FutureTask`. It allows to use `FutureUtils.waitForAll(CompletableFutures).get()`.

----------------------------------------------------------------
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