What I meant to ask was, does it do any harm to keep calling cancel-with-savepoint until the job exits? If the job is already cancelling with savepoint, I would assume that another cancel-with-savepoint call is just ignored.
On Tue, Aug 21, 2018 at 1:18 PM Till Rohrmann <trohrm...@apache.org> wrote: > Just a small addition. Concurrent cancel call will interfere with the > cancel-with-savepoint command and directly cancel the job. So it is better > to use the cancel-with-savepoint call in order to take savepoint and then > cancel the job automatically. > > Cheers, > Till > > On Thu, Aug 9, 2018 at 9:53 AM vino yang <yanghua1...@gmail.com> wrote: > >> Hi Juho, >> >> We use REST client API : triggerSavepoint(), this API returns a >> CompletableFuture, then we call it's get() API. >> >> You can understand that I am waiting for it to complete in sync. >> Because cancelWithSavepoint is actually waiting for savepoint to complete >> synchronization, and then execute the cancel command. >> >> We do not use CLI. I think since you are through the CLI, you can observe >> whether the savepoint is complete by combining the log or the web UI. >> >> Thanks, vino. >> >> >> Juho Autio <juho.au...@rovio.com> 于2018年8月9日周四 下午3:07写道: >> >>> Thanks for the suggestion. Is the separate savepoint triggering async? >>> Would you then separately poll for the savepoint's completion before >>> executing cancel? If additional polling is needed, then I would say that >>> for my purpose it's still easier to call cancel with savepoint and simply >>> ignore the result of the call. I would assume that it won't do any harm if >>> I keep retrying cancel with savepoint until the job stops – I expect that >>> an overlapping cancel request is ignored if the job is already creating a >>> savepoint. Please correct if my assumption is wrong. >>> >>> On Thu, Aug 9, 2018 at 5:04 AM vino yang <yanghua1...@gmail.com> wrote: >>> >>>> Hi Juho, >>>> >>>> This problem does exist, I suggest you separate these two steps to >>>> temporarily deal with this problem: >>>> 1) Trigger Savepoint separately; >>>> 2) execute the cancel command; >>>> >>>> Hi Till, Chesnay: >>>> >>>> Our internal environment and multiple users on the mailing list have >>>> encountered similar problems. >>>> >>>> In our environment, it seems that JM shows that the save point is >>>> complete and JM has stopped itself, but the client will still connect to >>>> the old JM and report a timeout exception. >>>> >>>> Thanks, vino. >>>> >>>> >>>> Juho Autio <juho.au...@rovio.com> 于2018年8月8日周三 下午9:18写道: >>>> >>>>> I was trying to cancel a job with savepoint, but the CLI command >>>>> failed with "akka.pattern.AskTimeoutException: Ask timed out". >>>>> >>>>> The stack trace reveals that ask timeout is 10 seconds: >>>>> >>>>> Caused by: akka.pattern.AskTimeoutException: Ask timed out on >>>>> [Actor[akka://flink/user/jobmanager_0#106635280]] after [10000 ms]. >>>>> Sender[null] sent message of type >>>>> "org.apache.flink.runtime.rpc.messages.LocalFencedMessage". >>>>> >>>>> Indeed it's documented that the default value for akka.ask.timeout="10 >>>>> s" in >>>>> >>>>> https://ci.apache.org/projects/flink/flink-docs-stable/ops/config.html#distributed-coordination-via-akka >>>>> >>>>> Behind the scenes the savepoint creation & job cancellation succeeded, >>>>> that was to be expected, kind of. So my problem is just getting a proper >>>>> response back from the CLI call instead of timing out so eagerly. >>>>> >>>>> To be exact, what I ran was: >>>>> >>>>> flink-1.5.2/bin/flink cancel b7c7d19d25e16a952d3afa32841024e5 -m >>>>> yarn-cluster -yid application_1533676784032_0001 --withSavepoint >>>>> >>>>> Should I change the akka.ask.timeout to have a longer timeout? If yes, >>>>> can I override it just for the CLI call somehow? Maybe it might have >>>>> undesired side-effects if set globally for the actual flink jobs to use? >>>>> >>>>> What about akka.client.timeout? The default for it is also rather >>>>> low: "60 s". Should it also be increased accordingly if I want to accept >>>>> longer than 60 s for savepoint creation? >>>>> >>>>> Finally, that default timeout is so low that I would expect this to be >>>>> a common problem. I would say that Flink CLI should have higher default >>>>> timeout for cancel and savepoint creation ops. >>>>> >>>>> Thanks! >>>>> >>>> >>>