You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by "ASF GitHub Bot (JIRA)" <ji...@apache.org> on 2016/02/29 16:05:18 UTC

[jira] [Commented] (FLINK-3534) Cancelling a running job can lead to restart instead of stopping

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

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

GitHub user uce opened a pull request:

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

    [FLINK-3534] [runtime] Prevent canceling Execution from failing

    @StephanEwen, can you have a look at this?
    
    This should handle cases where a cancelled execution is failed (as reported by @rmetzger), but still allow the `ExecutionGraph` to be failed (and not suffer from the point you raised in #1726).
    
    Robert would like this in the next RC, but in general I think that it's better to have more exposure for a change to the `ExecutionGraph`... I'm undecided right now. What do you think?

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

    $ git pull https://github.com/uce/flink 3534-prevent_restart

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

    https://github.com/apache/flink/pull/1735.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 #1735
    
----
commit 54c1b12a67ac928b8461865adef6794b82d16325
Author: Ufuk Celebi <uc...@apache.org>
Date:   2016-02-29T14:37:45Z

    [FLINK-3534] [runtime] Prevent canceling Execution from failing

----


> Cancelling a running job can lead to restart instead of stopping
> ----------------------------------------------------------------
>
>                 Key: FLINK-3534
>                 URL: https://issues.apache.org/jira/browse/FLINK-3534
>             Project: Flink
>          Issue Type: Bug
>    Affects Versions: 1.0.0
>            Reporter: Robert Metzger
>            Priority: Critical
>
> I just tried cancelling a regularly running job. Instead of the job stopping, it restarted.
> {code}
> 2016-02-29 10:39:28,415 INFO  org.apache.flink.yarn.YarnJobManager                          - Trying to cancel job with ID 5c0604694c8469cfbb89daaa990068df.
> 2016-02-29 10:39:28,416 INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph        - Source: Out of order data generator -> (Flat Map, Timestamps/Watermarks) (1/1) (e3b05555ab0e373defb925898de9f200) switched from RUNNING to CANCELING
> ....
> 2016-02-29 10:39:28,488 INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph        - TriggerWindow(TumblingTimeWindows(60000), FoldingStateDescriptor{name=window-contents, defaultValue=(0,9223372036854775807,0), serializer=null}, EventTimeTrigger(), WindowedStream.apply(WindowedStream.java:397)) (19/24) (c1be31b0be596d2521073b2d78ffa60a) switched from CANCELING to CANCELED
> 2016-02-29 10:40:08,468 INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph        - Source: Out of order data generator -> (Flat Map, Timestamps/Watermarks) (1/1) (e3b05555ab0e373defb925898de9f200) switched from CANCELING to FAILED
> 2016-02-29 10:40:08,468 INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph        - TriggerWindow(TumblingTimeWindows(60000), FoldingStateDescriptor{name=window-contents, defaultValue=(0,9223372036854775807,0), serializer=null}, EventTimeTrigger(), WindowedStream.apply(WindowedStream.java:397)) (1/24) (5ad172ec9932b24d5a98377a2c82b0b3) switched from CANCELING to FAILED
> 2016-02-29 10:40:08,472 INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph        - TriggerWindow(TumblingTimeWindows(60000), FoldingStateDescriptor{name=window-contents, defaultValue=(0,9223372036854775807,0), serializer=null}, EventTimeTrigger(), WindowedStream.apply(WindowedStream.java:397)) (2/24) (5404ca28ac7cf23b67dff30ef2309078) switched from CANCELING to FAILED
> 2016-02-29 10:40:08,473 INFO  org.apache.flink.yarn.YarnJobManager                          - Status of job 5c0604694c8469cfbb89daaa990068df (Event counter: {auto.offset.reset=earliest, rocksdb=hdfs:///user/robert/rocksdb, generateInPlace=soTrue, parallelism=24, bootstrap.servers=cdh544-worker-0:9092, topic=eventsGenerator, eventsPerKeyPerGenerator=2, numKeys=1000000000, zookeeper.connect=cdh544-worker-0:2181, timeSliceSize=60000, eventsKerPey=1, genPar=1}) changed to FAILING.
> java.lang.Exception: Task could not be canceled.
> 	at org.apache.flink.runtime.executiongraph.Execution$5.onComplete(Execution.java:902)
> 	at akka.dispatch.OnComplete.internal(Future.scala:246)
> 	at akka.dispatch.OnComplete.internal(Future.scala:244)
> 	at akka.dispatch.japi$CallbackBridge.apply(Future.scala:174)
> 	at akka.dispatch.japi$CallbackBridge.apply(Future.scala:171)
> 	at scala.concurrent.impl.CallbackRunnable.run(Promise.scala:32)
> 	at scala.concurrent.impl.ExecutionContextImpl$$anon$3.exec(ExecutionContextImpl.scala:107)
> 	at scala.concurrent.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260)
> 	at scala.concurrent.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1339)
> 	at scala.concurrent.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979)
> 	at scala.concurrent.forkjoin.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:107)
> Caused by: akka.pattern.AskTimeoutException: Ask timed out on [Actor[akka.tcp://flink@10.240.242.143:50119/user/taskmanager#640539146]] after [10000 ms]
> 	at akka.pattern.PromiseActorRef$$anonfun$1.apply$mcV$sp(AskSupport.scala:333)
> 	at akka.actor.Scheduler$$anon$7.run(Scheduler.scala:117)
> 	at scala.concurrent.Future$InternalCallbackExecutor$.scala$concurrent$Future$InternalCallbackExecutor$$unbatchedExecute(Future.scala:694)
> 	at scala.concurrent.Future$InternalCallbackExecutor$.execute(Future.scala:691)
> 	at akka.actor.LightArrayRevolverScheduler$TaskHolder.executeTask(Scheduler.scala:467)
> 	at akka.actor.LightArrayRevolverScheduler$$anon$8.executeBucket$1(Scheduler.scala:419)
> 	at akka.actor.LightArrayRevolverScheduler$$anon$8.nextTick(Scheduler.scala:423)
> 	at akka.actor.LightArrayRevolverScheduler$$anon$8.run(Scheduler.scala:375)
> 	at java.lang.Thread.run(Thread.java:745)
> 2016-02-29 10:40:08,477 INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph        - TriggerWindow(TumblingTimeWindows(60000), FoldingStateDescriptor{name=window-contents, defaultValue=(0,9223372036854775807,0), serializer=null}, EventTimeTrigger(), WindowedStream.apply(WindowedStream.java:397)) (3/24) (fc527d65ec8df3ccf68f882d968e776e) switched from CANCELING to FAILED
> 2016-02-29 10:40:08,487 INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph        - TriggerWindow(TumblingTimeWindows(60000), FoldingStateDescriptor{name=window-contents, defaultValue=(0,9223372036854775807,0), serializer=null}, EventTimeTrigger(), WindowedStream.apply(WindowedStream.java:397)) (4/24) (afb1aa3c2d8acdee0f138cf344238e4e) switched from CANCELING to FAILED
> 2016-02-29 10:40:08,488 INFO  org.apache.flink.runtime.executiongraph.restart.FixedDelayRestartStrategy  - Delaying retry of job execution for 3000 ms ...
> 2016-02-29 10:40:08,488 INFO  org.apache.flink.yarn.YarnJobManager                          - Status of job 5c0604694c8469cfbb89daaa990068df (Event counter: {auto.offset.reset=earliest, rocksdb=hdfs:///user/robert/rocksdb, generateInPlace=soTrue, parallelism=24, bootstrap.servers=cdh544-worker-0:9092, topic=eventsGenerator, eventsPerKeyPerGenerator=2, numKeys=1000000000, zookeeper.connect=cdh544-worker-0:2181, timeSliceSize=60000, eventsKerPey=1, genPar=1}) changed to RESTARTING.
> 2016-02-29 10:40:11,490 INFO  org.apache.flink.yarn.YarnJobManager                          - Status of job 5c0604694c8469cfbb89daaa990068df (Event counter: {auto.offset.reset=earliest, rocksdb=hdfs:///user/robert/rocksdb, generateInPlace=soTrue, parallelism=24, bootstrap.servers=cdh544-worker-0:9092, topic=eventsGenerator, eventsPerKeyPerGenerator=2, numKeys=1000000000, zookeeper.connect=cdh544-worker-0:2181, timeSliceSize=60000, eventsKerPey=1, genPar=1}) changed to CREATED.
> 2016-02-29 10:40:11,490 INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph        - Source: Out of order data generator -> (Flat Map, Timestamps/Watermarks) (1/1) (1319b2f44d78d99948ffde4350c052d9) switched from CREATED to SCHEDULED
> 2016-02-29 10:40:11,490 INFO  org.apache.flink.yarn.YarnJobManager                          - Status of job 5c0604694c8469cfbb89daaa990068df (Event counter: {auto.offset.reset=earliest, rocksdb=hdfs:///user/robert/rocksdb, generateInPlace=soTrue, parallelism=24, bootstrap.servers=cdh544-worker-0:9092, topic=eventsGenerator, eventsPerKeyPerGenerator=2, numKeys=1000000000, zookeeper.connect=cdh544-worker-0:2181, timeSliceSize=60000, eventsKerPey=1, genPar=1}) changed to RUNNING.
> {code}



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)