AskTimeoutException when canceling job with savepoint on flink 1.6.0

classic Classic list List threaded Threaded
3 messages Options
Reply | Threaded
Open this post in threaded view
|

AskTimeoutException when canceling job with savepoint on flink 1.6.0

jelmer
I am trying to upgrade a job from flink 1.4.2 to 1.6.0

When we do a deploy we cancel the job with a savepoint then deploy the new version of the job from that savepoint. Because our jobs tend to have a lot of state it often takes multiple minutes for our savepoints to complete. 

On flink 1.4.2 we set akka.client.timeout to a high value to make sure the request did not timeout

However on flink 1.6.0 I get an AskTimeoutException  and increasing akka.client.timeout only works if i apply it to the running flink process.
Applying it to just the flink client does nothing.

I am reluctant to configure this on the container itself because afaik it applies to everything inside of flink's internal actor system not just to creating savepoints. 

What is the correct way to use cancel with savepoint for jobs with lots of state in flink 1.6.0 ?

I Attached the error.




error.txt (6K) Download Attachment
Reply | Threaded
Open this post in threaded view
|

Re: AskTimeoutException when canceling job with savepoint on flink 1.6.0

vino yang
Hi Jelmer,

Here's a similar question, and you can refer to the discussion options.[1]


Hi Till and Chesnay,

Recently, several users have encountered this problem in the past month. 
Maybe the community should give priority to the stability of this part or list the guidelines in the official document FAQ?

Thanks, vino.

jelmer <[hidden email]> 于2018年9月5日周三 下午8:48写道:
I am trying to upgrade a job from flink 1.4.2 to 1.6.0

When we do a deploy we cancel the job with a savepoint then deploy the new version of the job from that savepoint. Because our jobs tend to have a lot of state it often takes multiple minutes for our savepoints to complete. 

On flink 1.4.2 we set akka.client.timeout to a high value to make sure the request did not timeout

However on flink 1.6.0 I get an AskTimeoutException  and increasing akka.client.timeout only works if i apply it to the running flink process.
Applying it to just the flink client does nothing.

I am reluctant to configure this on the container itself because afaik it applies to everything inside of flink's internal actor system not just to creating savepoints. 

What is the correct way to use cancel with savepoint for jobs with lots of state in flink 1.6.0 ?

I Attached the error.



Reply | Threaded
Open this post in threaded view
|

Re: AskTimeoutException when canceling job with savepoint on flink 1.6.0

Gary Yao-2
Hi Jelmer,

I saw that you have already found the JIRA issue tracking this problem [1] but
I will still answer on the mailing list for transparency.

The timeout for "cancel with savepoint" should be RpcUtils.INF_TIMEOUT.
Unfortunately Flink is currently not respecting this timeout. A pull request
is already available, and is expected to be merged within the next days [2].

Best,
Gary

[1] https://issues.apache.org/jira/browse/FLINK-10193
[2] https://github.com/apache/flink/pull/6601

On Thu, Sep 6, 2018 at 4:24 AM, vino yang <[hidden email]> wrote:
Hi Jelmer,

Here's a similar question, and you can refer to the discussion options.[1]


Hi Till and Chesnay,

Recently, several users have encountered this problem in the past month. 
Maybe the community should give priority to the stability of this part or list the guidelines in the official document FAQ?

Thanks, vino.

jelmer <[hidden email]> 于2018年9月5日周三 下午8:48写道:
I am trying to upgrade a job from flink 1.4.2 to 1.6.0

When we do a deploy we cancel the job with a savepoint then deploy the new version of the job from that savepoint. Because our jobs tend to have a lot of state it often takes multiple minutes for our savepoints to complete. 

On flink 1.4.2 we set akka.client.timeout to a high value to make sure the request did not timeout

However on flink 1.6.0 I get an AskTimeoutException  and increasing akka.client.timeout only works if i apply it to the running flink process.
Applying it to just the flink client does nothing.

I am reluctant to configure this on the container itself because afaik it applies to everything inside of flink's internal actor system not just to creating savepoints. 

What is the correct way to use cancel with savepoint for jobs with lots of state in flink 1.6.0 ?

I Attached the error.