You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by "ASF GitHub Bot (JIRA)" <ji...@apache.org> on 2018/10/05 14:53:00 UTC

[jira] [Commented] (FLINK-10461) Speed up download file procedure when restore

    [ https://issues.apache.org/jira/browse/FLINK-10461?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16639912#comment-16639912 ] 

ASF GitHub Bot commented on FLINK-10461:
----------------------------------------

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_r223032334
 
 

 ##########
 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);
 
 Review comment:
   Cancelation of `copyStateDataHandleData` futures and `executorService.shutdownNow()` should be registered in `cancelStreamRegistry.registerCloseable()` and unregistered with `unregisterCloseable` before `executorService.shutdown()` in `finally`. Similar to streams in `copyStateDataHandleData`. This way we guarantee freeing of all resources in case of job abrupt shutdown.

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


> Speed up download file procedure when restore 
> ----------------------------------------------
>
>                 Key: FLINK-10461
>                 URL: https://issues.apache.org/jira/browse/FLINK-10461
>             Project: Flink
>          Issue Type: Improvement
>          Components: State Backends, Checkpointing
>            Reporter: Congxian Qiu
>            Assignee: Congxian Qiu
>            Priority: Major
>              Labels: pull-request-available
>
> In the current master branch, the restore will download file from DFS, the download procedure are single-thread, this could speed up by using multi-thread for downloading states from DFS.
>  
> In my company, the states will come to some terabytes, so the restore procedure will become a litter slow, after a bit digging, I find download states from DFS using single thread, this could using multi-thread for speed up.
> I test the time used for download states from DFS with ~2 terabytes states. With single thread it used 640+s, and 130+s when using 5 threads for download.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)