Hi,

Thanks for the details! I can confirm this behaviour. flink-conf.yaml 
restart-strategy value is being completely ignored (regardless of it’s value) 
when user enables checkpointing:

env.enableCheckpointing(5000, CheckpointingMode.EXACTLY_ONCE);

I suspect this is a bug, but I have to confirm it.

Thanks, Piotrek

> On 5 Apr 2018, at 12:40, Alexander Smirnov <alexander.smirn...@gmail.com> 
> wrote:
> 
> jobmanager.log:
> 
> 2018-04-05 22:37:28,348 INFO  
> org.apache.flink.configuration.GlobalConfiguration            - Loading 
> configuration property: restart-strategy, none
> 2018-04-05 22:37:28,353 INFO  org.apache.flink.core.fs.FileSystem             
>               - Hadoop is not in the classpath/dependencies. The extended set 
> of supported File Systems via Hadoop is not available.
> 2018-04-05 22:37:28,506 INFO  org.apache.flink.runtime.jobmanager.JobManager  
>               - Starting JobManager without high-availability
> 2018-04-05 22:37:28,510 INFO  org.apache.flink.runtime.jobmanager.JobManager  
>               - Starting JobManager on localhost:6123 with execution mode 
> CLUSTER
> 2018-04-05 22:37:28,517 INFO  
> org.apache.flink.runtime.security.modules.HadoopModuleFactory  - Cannot 
> create Hadoop Security Module because Hadoop cannot be found in the Classpath.
> 2018-04-05 22:37:28,546 INFO  org.apache.flink.runtime.security.SecurityUtils 
>               - Cannot install HadoopSecurityContext because Hadoop cannot be 
> found in the Classpath.
> 2018-04-05 22:37:28,591 INFO  org.apache.flink.runtime.jobmanager.JobManager  
>               - Trying to start actor system at localhost:6123
> 2018-04-05 22:37:28,981 INFO  akka.event.slf4j.Slf4jLogger                    
>               - Slf4jLogger started
> 2018-04-05 22:37:29,027 INFO  akka.remote.Remoting                            
>               - Starting remoting
> 2018-04-05 22:37:29,129 INFO  akka.remote.Remoting                            
>               - Remoting started; listening on addresses 
> :[akka.tcp://flink@localhost:6123]
> 2018-04-05 22:37:29,135 INFO  org.apache.flink.runtime.jobmanager.JobManager  
>               - Actor system started at akka.tcp://flink@localhost:6123
> 2018-04-05 22:37:29,148 INFO  
> org.apache.flink.runtime.metrics.MetricRegistryImpl           - No metrics 
> reporter configured, no metrics will be exposed/reported.
> 2018-04-05 22:37:29,152 INFO  org.apache.flink.runtime.jobmanager.JobManager  
>               - Starting JobManager web frontend
> 2018-04-05 22:37:29,161 INFO  
> org.apache.flink.runtime.webmonitor.WebMonitorUtils           - Determined 
> location of JobManager log file: 
> /Users/asmirnov/flink-1.4.2/log/flink-jobmanager-0.log
> 2018-04-05 22:37:29,161 INFO  
> org.apache.flink.runtime.webmonitor.WebMonitorUtils           - Determined 
> location of JobManager stdout file: 
> /Users/asmirnov/flink-1.4.2/log/flink-jobmanager-0.out
> 2018-04-05 22:37:29,162 INFO  
> org.apache.flink.runtime.webmonitor.WebRuntimeMonitor         - Using 
> directory 
> /var/folders/5s/yj6g5wd90h158whcb_483hhhq7t4sw/T/flink-web-901a3fb7-d366-4f90-b75c-1e1f8038ed37
>  for the web interface files
> 2018-04-05 22:37:29,162 INFO  
> org.apache.flink.runtime.webmonitor.WebRuntimeMonitor         - Created 
> directory 
> /var/folders/5s/yj6g5wd90h158whcb_483hhhq7t4sw/T/flink-web-21e5d8a8-7967-40f0-97d7-a803d9bd5913
>  for web frontend JAR file uploads.
> 2018-04-05 22:37:29,447 INFO  
> org.apache.flink.runtime.webmonitor.WebRuntimeMonitor         - Web frontend 
> listening at localhost:8081
> 2018-04-05 22:37:29,447 INFO  org.apache.flink.runtime.jobmanager.JobManager  
>               - Starting JobManager actor
> 2018-04-05 22:37:29,452 INFO  org.apache.flink.runtime.blob.BlobServer        
>               - Created BLOB server storage directory 
> /var/folders/5s/yj6g5wd90h158whcb_483hhhq7t4sw/T/blobStore-6777e862-0c2c-4679-a42f-b1921baa5236
> 2018-04-05 22:37:29,453 INFO  org.apache.flink.runtime.blob.BlobServer        
>               - Started BLOB server at 0.0.0.0:60697 <http://0.0.0.0:60697/> 
> - max concurrent requests: 50 - max backlog: 1000
> 2018-04-05 22:37:29,533 INFO  
> org.apache.flink.runtime.jobmanager.MemoryArchivist           - Started 
> memory archivist akka://flink/user/archive
> 2018-04-05 22:37:29,533 INFO  org.apache.flink.runtime.jobmanager.JobManager  
>               - Starting JobManager at 
> akka.tcp://flink@localhost:6123/user/jobmanager.
> 2018-04-05 22:37:29,544 INFO  org.apache.flink.runtime.jobmanager.JobManager  
>               - JobManager akka.tcp://flink@localhost:6123/user/jobmanager 
> was granted leadership with leader session ID 
> Some(00000000-0000-0000-0000-000000000000).
> 2018-04-05 22:37:29,545 INFO  
> org.apache.flink.runtime.clusterframework.standalone.StandaloneResourceManager
>   - Trying to associate with JobManager leader 
> akka.tcp://flink@localhost:6123/user/jobmanager
> 2018-04-05 22:37:29,552 INFO  
> org.apache.flink.runtime.clusterframework.standalone.StandaloneResourceManager
>   - Resource Manager associating with leading JobManager 
> Actor[akka://flink/user/jobmanager#-853250886] - leader session 
> 00000000-0000-0000-0000-000000000000
> 2018-04-05 22:37:30,495 INFO  
> org.apache.flink.runtime.clusterframework.standalone.StandaloneResourceManager
>   - TaskManager f0b0370186ab3c865db63fe60ca68e08 has started.
> 2018-04-05 22:37:30,497 INFO  
> org.apache.flink.runtime.instance.InstanceManager             - Registered 
> TaskManager at 192.168.0.26 
> (akka.tcp://flink@mb-sr-asmirnov.local:60696/user/taskmanager) as 
> 2972a72a7223e63bb5a4fedd159c0b78. Current number of registered hosts is 1. 
> Current number of alive task slots is 1.
> 2018-04-05 22:38:29,355 INFO  org.apache.flink.runtime.client.JobClient       
>               - Checking and uploading JAR files
> 2018-04-05 22:38:29,639 INFO  org.apache.flink.runtime.jobmanager.JobManager  
>               - Submitting job 43ecfe9cb258b7f624aad9868d306edb (Failed job).
> 2018-04-05 22:38:29,643 INFO  org.apache.flink.runtime.jobmanager.JobManager  
>               - Using restart strategy 
> FixedDelayRestartStrategy(maxNumberRestartAttempts=2147483647, 
> delayBetweenRestartAttempts=10000) for 43ecfe9cb258b7f624aad9868d306edb.
> 2018-04-05 22:38:29,656 INFO  
> org.apache.flink.runtime.executiongraph.ExecutionGraph        - Job recovers 
> via failover strategy: full graph restart
> 
> 
> 
> On Thu, Apr 5, 2018 at 10:35 PM Alexander Smirnov 
> <alexander.smirn...@gmail.com <mailto:alexander.smirn...@gmail.com>> wrote:
> Hi Piotr,
> 
> I'm using Flink 1.4.2
> 
> it's a standard flink distribution downloaded and unpacked.
> 
> added the following lines to conf/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
> 
> created new java project as described at 
> https://ci.apache.org/projects/flink/flink-docs-release-1.4/quickstart/java_api_quickstart.html
>  
> <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");
>     }
> }
> 
> attaching screenshots, please let me know if more info is needed
> 
> Alex
> 
> 
>  
> 
> On Thu, Apr 5, 2018 at 5:35 PM Piotr Nowojski <pi...@data-artisans.com 
> <mailto:pi...@data-artisans.com>> wrote:
> Hi,
> 
> Can you provide more details, like post your configuration/log files/screen 
> shots from web UI and Flink version being used?
> 
> Piotrek
> 
> > On 5 Apr 2018, at 06:07, Alexander Smirnov <alexander.smirn...@gmail.com 
> > <mailto:alexander.smirn...@gmail.com>> wrote:
> >
> > Hello,
> >
> > I've defined restart strategy in flink-conf.yaml as none. WebUI / Job 
> > Manager section confirms that.
> > But looks like this setting is disregarded.
> >
> > When I go into job's configuration in the WebUI, in the Execution 
> > Configuration section I can see:
> >     Max. number of execution retries          Restart with fixed delay 
> > (10000 ms). #2147483647 <tel:(214)%20748-3647> restart attempts.
> >
> > Do you think it is a bug?
> >
> > Alex
> 

Reply via email to