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

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

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

    https://github.com/apache/flink/pull/6283#discussion_r202319384
  
    --- Diff: 
flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobGraph.java ---
    @@ -339,12 +339,28 @@ public void 
setSnapshotSettings(JobCheckpointingSettings settings) {
         * Gets the settings for asynchronous snapshots. This method returns 
null, when
         * checkpointing is not enabled.
         *
    -    * @return The snapshot settings, or null, if checkpointing is not 
enabled.
    +    * @return The snapshot settings
         */
        public JobCheckpointingSettings getCheckpointingSettings() {
                return snapshotSettings;
        }
     
    +   /**
    +    * Checks if the checkpointing was enabled for this job graph
    +    *
    +    * @return true if checkpointing enabled
    +    */
    +   public boolean isCheckpointingEnabled() {
    +
    +           if (snapshotSettings == null) {
    +                   return false;
    +           }
    +
    +           long checkpointInterval = 
snapshotSettings.getCheckpointCoordinatorConfiguration().getCheckpointInterval();
    +           return checkpointInterval > 0 &&
    +                   checkpointInterval < Long.MAX_VALUE;
    --- End diff --
    
    I don't think it is true (about the checkpoint enabling). I thought the 
same based on some javadocs, but it turned out that `snapshotSetting` is always 
set in 
`org.apache.flink.streaming.api.graph.StreamingJobGraphGenerator#configureCheckpointing`.
    That's why I added this method.
    
    The problem with the second method is that the `CheckpointCoordinator` is 
created while constructing `ExecutionGraph` which requires the restartstrategy. 
I thought adding this method was the least invasive one.


> 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
>              Labels: pull-request-available
>         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