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

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

Github user uce commented on the issue:

    https://github.com/apache/flink/pull/2096
  
    Really good changes and great that you added the `JobStatus` graph 
transitions figure to the docs.
    
    I looked at all changes and tests. Furthermore, I've tested this with lost 
ZooKeeper connections and it works as expected. The job is suspended and 
recovered smoothly. 👍 
    
    Some general notes:
    - The only thing that might be slightly confusing for users is that until 
the web frontend is notified about the leader changes, it shows the job status 
as FAILED until it disappears. This is probably some web interface quirk, 
because it does not recognize the new job status. I understand that you remove 
suspended job graphs and don't archive them (which makes sense as they are 
recovered later), but giving that it takes some time until all information 
propagates to the web frontend, we might want to temporarily show the state as 
suspended. We can do this as a follow up.
    
    - Regarding the removal of restart strategy disabling, I thought about 
whether we should leave it in and get rid of the `SuppressRestartsException` 
instead. That seems to be cleaner than the custom exception, which is checked 
in the `ExecutionGraph`. If we want to get rid of it (as a follow up), we 
should leave the disable call in.
    
    - If you think that 
https://github.com/apache/flink/pull/2095#issuecomment-227731879 is a valid 
point, then we should apply it for the suspended state as well.
    
    These comments are not blockers and I think this is good to merge. Great 
job!



> ExecutionGraphs can become orphans
> ----------------------------------
>
>                 Key: FLINK-3800
>                 URL: https://issues.apache.org/jira/browse/FLINK-3800
>             Project: Flink
>          Issue Type: Bug
>          Components: JobManager
>    Affects Versions: 1.0.0, 1.1.0
>            Reporter: Till Rohrmann
>            Assignee: Till Rohrmann
>
> The {{JobManager.cancelAndClearEverything}} method fails all currently 
> executed jobs on the {{JobManager}} and then clears the list of 
> {{currentJobs}} kept in the JobManager. This can become problematic if the 
> user has set a restart strategy for a job, because the {{RestartStrategy}} 
> will try to restart the job. This can lead to unwanted re-deployments of the 
> job which consumes resources and thus will trouble the execution of other 
> jobs. If the restart strategy never stops, then this prevents that the 
> {{ExecutionGraph}} from ever being properly terminated.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to