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

Till Rohrmann commented on FLINK-9143:
--------------------------------------

The problem is that we don't necessarily have the cluster configuration when we 
submit the job. Consequently, it is a priori not possible to decide whether the 
cluster has a default restart strategy or not.

The underlying problem with the restart strategies is that the it is a cluster 
configuration as well as a job configuration option. I think it should only be 
a cluster configuration option. If we don't want break the behavior, then a 
solution could be to differentiate between an explicitly set job-specific 
restart strategy and the default job-specific restart strategy set when 
submitting a streaming job with checkpointing enabled.

> Restart strategy defined in flink-conf.yaml is ignored
> ------------------------------------------------------
>
>                 Key: FLINK-9143
>                 URL: https://issues.apache.org/jira/browse/FLINK-9143
>             Project: Flink
>          Issue Type: Bug
>          Components: Configuration
>    Affects Versions: 1.4.2
>            Reporter: Alex Smirnov
>            Assignee: yuqi
>            Priority: Major
>         Attachments: execution_config.png, jobmanager.log, jobmanager.png
>
>
> Restart strategy defined in flink-conf.yaml is disregarded, when user enables 
> checkpointing.
> Steps to reproduce:
> 1. Download flink distribution (1.4.2), update flink-conf.yaml:
>  
> restart-strategy: none
> state.backend: rocksdb
> state.backend.fs.checkpointdir: 
> file:///tmp/nfsrecovery/flink-checkpoints-metadata
> state.backend.rocksdb.checkpointdir: 
> file:///tmp/nfsrecovery/flink-checkpoints-rocksdb
>  
> 2. create new java project as described at 
> [https://ci.apache.org/projects/flink/flink-docs-release-1.4/quickstart/java_api_quickstart.html]
> here's the code:
> public class FailedJob
> {
>     static final Logger LOGGER = LoggerFactory.getLogger(FailedJob.class);
>     public static void main( String[] args ) throws Exception
>     {
>         final StreamExecutionEnvironment env = 
> StreamExecutionEnvironment.getExecutionEnvironment();
>         env.enableCheckpointing(5000, CheckpointingMode.EXACTLY_ONCE);
>         DataStream<String> stream = env.fromCollection(Arrays.asList("test"));
>         stream.map(new MapFunction<String, String>(){
>             @Override
>             public String map(String obj) {
>                 throw new NullPointerException("NPE");
>             } 
>         });
>         env.execute("Failed job");
>     }
> }
>  
> 3. Compile: mvn clean package; submit it to the cluster
>  
> 4. Go to Job Manager configuration in WebUI, ensure settings from 
> flink-conf.yaml is there (screenshot attached)
>  
> 5. Go to Job's configuration, see Execution Configuration section
>  
> *Expected result*: restart strategy as defined in flink-conf.yaml
>  
> *Actual result*: Restart with fixed delay (10000 ms). 
> #[2147483647|tel:(214)%20748-3647] restart attempts.
>  
>  
> see attached screenshots and jobmanager log (line 1 and 31)
>  



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

Reply via email to