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

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

GitHub user yuqi1129 opened a pull request:

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

    [FLINK-9143] [client] Restart strategy defined in flink-conf.yaml is ignored

    
    ## What is the purpose of the change
    
    This change is to fix the bug that if restart strategy was not specifically 
set in user code, flink will not use default strategy set in flink-conf.yaml in 
flip6 mode. 
    ## Brief change log
    
      - In `ClusterClient.getJobGraph`, if no strategy was set in user code, we 
set strategy strategy just as flink-conf.yaml do 
    
    ## Verifying this change
    
    This change added tests and can be verified as follows:
    
      - Add `ClusterClientTest.testgetJobGraph` 
     -  Add `FailureRateRestartStrategyTest.testFailureRateRestartStrategyConf` 
and `FixedDelayRestartStrategyTest.testFixedRestartStrategyConf`
    
    ## Does this pull request potentially affect one of the following parts:
    
      - Dependencies (does it add or upgrade a dependency): (no)
      - The public API, i.e., is any changed class annotated with 
`@Public(Evolving)`: (no)
      - The serializers: (no)
      - The runtime per-record code paths (performance sensitive): (no)
      - Anything that affects deployment or recovery: JobManager (and its 
components), Checkpointing, Yarn/Mesos, ZooKeeper: (no)
      - The S3 file system connector: (no)
    
    ## Documentation
    
      - Does this pull request introduce a new feature? (no)

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

    $ git pull https://github.com/yuqi1129/flink confproblem

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

    https://github.com/apache/flink/pull/5846.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 #5846
    
----
commit 7c2a8045a310e2942f4fbec3b4a2696b73fae3af
Author: hzyuqi1 <hzyuqi1@...>
Date:   2018-04-14T05:45:06Z

    [FLINK-9143] [client] Restart strategy defined in flink-conf.yaml is ignored
    
    This closes #9143.

commit df23fbe809c2cbd025a923cd8607ec852d40b4ae
Author: hzyuqi1 <hzyuqi1@...>
Date:   2018-04-14T05:46:21Z

    Merge branch 'master' of https://github.com/apache/flink into confproblem

----


> 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