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 2016/02/16 15:35:18 UTC

[jira] [Commented] (FLINK-3410) setting setNumberOfExecutionRetries to 0 still leads to RESTARTs.

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

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

GitHub user tillrohrmann opened a pull request:

    https://github.com/apache/flink/pull/1643

    [FLINK-3410] [restart] Choose NoRestart strategy if the number of retries is set to 0

    This fixes the problem that when checkpointing is enabled and the number of execution retries is set to `0` that the automatic restarting should be deactivated. This is consistent with the semantics before the restart strategies where introduced.
    
    Be aware, though, that whenever you enable checkpointing for streaming jobs, the cluster wide default restart strategy which is set in the configuration will always be overwritten. Either by manually setting a restart strategy or by automatically setting a `FixedDelayRestartStrategy(Integer.MAX_VALUE, 10000)` strategy in the `StreamingJobGraphGenerator` if nothing was specified. That is consistent with the previous behaviour where all default execution retry attempts set in the configuration where overwritten in case of a checkpointed streaming job.
    
    Additionally, this PR sets the default retry delay to 10000 ms and disallows to set it to negative values. Before, the default execution retry delay would have been used if the delay in the `ExecutionConfig` was set to `-1`. However, with the new `RestartStrategies` there is no longer the possibility to set an explicit execution retry delay independent of the number of retries in the configuration file. Therefore it is no longer possible to set the number of execution retries in the configuration file and to specify the execution retry delay in the `ExecutionConfig` or vice versa. Both have to be defined either in the `ExecutionConfig` or the configuration file.

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/tillrohrmann/flink fixRestartStrategy

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/flink/pull/1643.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #1643
    
----
commit 36ef22bc3d5491c67165a3ee1d4e08663723260b
Author: Till Rohrmann <tr...@apache.org>
Date:   2016-02-16T00:15:39Z

    [FLINK-3410] [restart] Choose NoRestart strategy if the number of retries is set to 0
    
    Add test case

----


> setting setNumberOfExecutionRetries to 0 still leads to RESTARTs.
> -----------------------------------------------------------------
>
>                 Key: FLINK-3410
>                 URL: https://issues.apache.org/jira/browse/FLINK-3410
>             Project: Flink
>          Issue Type: Bug
>          Components: JobManager
>    Affects Versions: 1.0.0
>            Reporter: Robert Metzger
>            Assignee: Till Rohrmann
>
> While testing the RC0 for 1.0.0 I found the following issue:
> Setting the number of retries to 0 still leads to the job being restarted:
> {code}
> final StreamExecutionEnvironment see = StreamExecutionEnvironment.getExecutionEnvironment();
> see.setNumberOfExecutionRetries(0);
> {code}
> {code}
> 21:19:50,677 INFO  org.apache.flink.runtime.jobmanager.JobManager                - Status of job 0e78d0825da485167aabee7e63c8e913 (Data Generator) changed to RESTARTING.
> 21:19:50,678 INFO  org.apache.flink.runtime.executiongraph.restart.FixedDelayRestartStrategy  - Delaying retry of job execution for 10000 ms ...
> {code}
> While looking through the code, it seems that the execution config is returning {{null}} when the number of retries is set to 0. With {{null}} the jobManager picks the default restart strategy.



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