[ https://issues.apache.org/jira/browse/FLINK-2805?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14944835#comment-14944835 ]
ASF GitHub Bot commented on FLINK-2805: --------------------------------------- Github user tillrohrmann commented on the pull request: https://github.com/apache/flink/pull/1227#issuecomment-145810928 Good PR @uce. I think the proposal in your description to let the client-side (TMs) 1. check whether they have the file cached, 2. check the filesystem backend and 3. (this case should then never happen in recovery mode) ask the JM, in order to obtain the required Jars would be a good improvement. Currently, I've got the feeling that the file state backend is too tightly coupled with the `BlobServer`. IMHO, it would be better to add an abstraction so that the effective backend to distribute the Jars can be easily swapped. Furthermore, I couldn't find a check whether the user provided `STATE_BACKEND_FS_RECOVERY_PATH` path points actually to a distributed file system and is, thus, accessible by the TMs. Maybe we could add a check which, if false, will give a comprehensive warning. > Make user jars available for all job managers to recover > -------------------------------------------------------- > > Key: FLINK-2805 > URL: https://issues.apache.org/jira/browse/FLINK-2805 > Project: Flink > Issue Type: Bug > Components: BlobManager, JobManager > Reporter: Ufuk Celebi > Assignee: Ufuk Celebi > > This is a bug in https://github.com/apache/flink/pull/1153. > In case of multiple job managers, the user jars need to be accessible by all > job managers (including those who arrive later). > Since #1153 requires the file state backend to be configured, the simplest > solution is to make the blob server aware of the configured recovery mode and > put/get/delete the user jars from the file state backend as well. -- This message was sent by Atlassian JIRA (v6.3.4#6332)