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_r223030317
########## 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<>(() -> { Review comment: I think it would be more maintainable if we extract these 3 methods: - `transferAllStateDataToDirectory` - `transferAllDataFromStateHandles` - `copyStateDataHandleData` into a separate class file, e.g. `RocksDbStateDataTransfer`. `RocksDBKeyedStateBackend` already contains a lot. ---------------------------------------------------------------- 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: us...@infra.apache.org With regards, Apache Git Services