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

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

GitHub user dawidwys opened a pull request:

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

    [FLINK-9143] Use cluster strategy if none was set on client side

    ## What is the purpose of the change
    
    The goal of this PR is to enable configuring default restart strategy from 
the server side's config.
    
    
    ## Brief change log
    
      * no strategy is set on the client side if none explicitly specified
      * on server side the strategy is resolved based on: client configuration, 
server side configuration, fallback to `FixedDelayStrategy` if none set on 
client side and `NoRestartStrategy` set on server side in case of checkpointing 
enabled
    
    ## Verifying this change
    
    This change added tests and can be verified as follows:
      - RestartStrategyResolvingTest.java
      - tests using cluster pass
    
    
    ## Does this pull request potentially affect one of the following parts:
    
      - Dependencies (does it add or upgrade a dependency): (yes / **no**)
      - The public API, i.e., is any changed class annotated with 
`@Public(Evolving)`: (yes / **no**)
      - The serializers: (yes / **no** / don't know)
      - The runtime per-record code paths (performance sensitive): (yes / 
**no** / don't know)
      - Anything that affects deployment or recovery: JobManager (and its 
components), Checkpointing, Yarn/Mesos, ZooKeeper: (**yes** / no / don't know)
      - The S3 file system connector: (yes / **no** / don't know)
    
    ## Documentation
    
      - Does this pull request introduce a new feature? (yes / **no**)
      - If yes, how is the feature documented? (**not applicable** / docs / 
JavaDocs / not documented)


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

    $ git pull https://github.com/dawidwys/flink FLINK-9143

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

    https://github.com/apache/flink/pull/6283.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 #6283
    
----
commit 8efded9c1e1a555edd7733b35d9f1f49f8cc7304
Author: Dawid Wysakowicz <dwysakowicz@...>
Date:   2018-07-05T11:48:23Z

    [FLINK-9143] Use cluster strategy if none was set on client side

----


> 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