[ https://issues.apache.org/jira/browse/FLINK-11813?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16815380#comment-16815380 ]
Zhu Zhu commented on FLINK-11813: --------------------------------- Thanks Till for the very detailed explanation. With clear definition of job identity, I agree DONE state is truly needed in this way. It will be great if we can leverage SubmittedJobGraphStore to do the RunningJobsRegistry work. One gap here is the job mode as you mentioned. RunningJobsRegistry is independent of whether it's session mode or job mode. But for SubmittedJobGraphStore, it's hard coded to use SingleJobSubmittedJobGraphStore in MiniDispatcher(job mode), which means the SubmittedJobGraphStore in job mode is not HA. I think there's need to unify the SubmittedJobGraphStore for session mode and job mode. . > Standby per job mode Dispatchers don't know job's JobSchedulingStatus > --------------------------------------------------------------------- > > Key: FLINK-11813 > URL: https://issues.apache.org/jira/browse/FLINK-11813 > Project: Flink > Issue Type: Bug > Components: Runtime / Coordination > Affects Versions: 1.6.4, 1.7.2, 1.8.0 > Reporter: Till Rohrmann > Priority: Major > > At the moment, it can happen that standby {{Dispatchers}} in per job mode > will restart a terminated job after they gained leadership. The problem is > that we currently clear the {{RunningJobsRegistry}} once a job has reached a > globally terminal state. After the leading {{Dispatcher}} terminates, a > standby {{Dispatcher}} will gain leadership. Without having the information > from the {{RunningJobsRegistry}} it cannot tell whether the job has been > executed or whether the {{Dispatcher}} needs to re-execute the job. At the > moment, the {{Dispatcher}} will assume that there was a fault and hence > re-execute the job. This can lead to duplicate results. > I think we need some way to tell standby {{Dispatchers}} that a certain job > has been successfully executed. One trivial solution could be to not clean up > the {{RunningJobsRegistry}} but then we will clutter ZooKeeper. -- This message was sent by Atlassian JIRA (v7.6.3#76005)