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

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

Github user dawidwys commented on a diff in the pull request:

    https://github.com/apache/flink/pull/5846#discussion_r191150742
  
    --- Diff: 
flink-clients/src/main/java/org/apache/flink/client/program/ClusterClient.java 
---
    @@ -896,6 +898,10 @@ public static JobGraph getJobGraph(Configuration 
flinkConfig, FlinkPlan optPlan,
                        job = gen.compileJobGraph((OptimizedPlan) optPlan);
                }
     
    +           //  if we disable checkpoint and do not set restart strategy, 
Restart strategy will be set as in flink-conf.yaml
    +           //  in flip6, jobmaster do not set this conf, so we have set 
this conf here.
    --- End diff --
    
    I am pretty sure the behaviour is still there. In JobMaster.java, or am I 
wrong?
    
        final RestartStrategies.RestartStrategyConfiguration 
restartStrategyConfiguration =
                        jobGraph.getSerializedExecutionConfig()
                                        .deserializeValue(userCodeLoader)
                                        .getRestartStrategy();
    
        this.restartStrategy = (restartStrategyConfiguration != null) ?
                        
RestartStrategyFactory.createRestartStrategy(restartStrategyConfiguration) :
                        
jobManagerSharedServices.getRestartStrategyFactory().createRestartStrategy();


> 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 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