Re: Checkpoint was declined (tasks not ready)

2017-10-31 Thread bartektartanus
I've created an issue on Jira and prepared pull request, here's the link: https://github.com/apache/flink/pull/4924 Travis CI check is not passing but looks like it's not my fault :) -- Sent from: http://apache-flink-user-mailing-list-archive.2336050.n4.nabble.com/

Re: Checkpoint was declined (tasks not ready)

2017-10-27 Thread Till Rohrmann
Yes please open the PR against Flink's master branch. You can also ping me once you've opened the PR. Then we can hopefully quickly merge it :-) Cheers, Till On Thu, Oct 26, 2017 at 12:44 PM, bartektartanus wrote: > I think we could try with option number one, as it seems to be easier to > impl

Re: Checkpoint was declined (tasks not ready)

2017-10-26 Thread bartektartanus
I think we could try with option number one, as it seems to be easier to implement. Currently I'm cloning Flink repo to fix this and test that solution with our currently not working code. Unfortunately, it takes forever to download all the dependencies. Anyway, I hope that eventually will manage t

Re: Checkpoint was declined (tasks not ready)

2017-10-25 Thread Till Rohrmann
Hi Bartek, I think your explanation of the problem is correct. Thanks a lot for your investigation. What we could do to solve the problem is the following: Either) We start the emitter thread before we restore the elements in the open method. That way the open method won't block forever but only

Re: Checkpoint was declined (tasks not ready)

2017-10-23 Thread bartektartanus
Ok, looks like we've found the cause of this issue. The scenario looks like this: 1. The queue is full (let's assume that its capacity is N elements) 2. There is some pending element waiting, so the pendingStreamElementQueueEntry field in AsyncWaitOperator is not null and while-loop in addAsyncBuff

Re: Checkpoint was declined (tasks not ready)

2017-10-23 Thread Maciek Próchniak
ckpoints start failing continuously with this message : Checkpoint was declined (tasks not ready) Inline image 1 But we see the job is running, its processing data, the accumulators we have are getting incremented etc but

Re: Checkpoint was declined (tasks not ready)

2017-10-23 Thread Maciek Próchniak
time.concurrent.impl.FlinkFuture.get(FlinkFuture.java:110) After the job restarts(we have a fixed restart strategy) we notice that the checkpoints start failing continuously with this message : Checkpoint was declined (tasks not ready) Inline

Re: Checkpoint was declined (tasks not ready)

2017-10-09 Thread Karthik Deivasigamani
tter.run(Emitter.java:83) >>> at java.lang.Thread.run(Thread.java:745) >>> Caused by: java.util.concurrent.ExecutionException: >>> java.util.concurrent.TimeoutException: Async function call has timed out. >>> at >>> org.apache.flink.runtime.conc

Re: Checkpoint was declined (tasks not ready)

2017-10-09 Thread Stephan Ewen
a.util.concurrent.ExecutionException: >> java.util.concurrent.TimeoutException: Async function call has timed out. >> at >> org.apache.flink.runtime.concurrent.impl.FlinkFuture.get(FlinkFuture.java:110) >> >> >> After the job restarts(we have a fixed re

Re: Checkpoint was declined (tasks not ready)

2017-10-06 Thread Karthik Deivasigamani
: Async function call has timed out. > at > org.apache.flink.runtime.concurrent.impl.FlinkFuture.get(FlinkFuture.java:110) > > > After the job restarts(we have a fixed restart strategy) we notice that > the checkpoints start failing continuously with this message : > Checkpoint was declined (tasks not ready

Checkpoint was declined (tasks not ready)

2017-10-06 Thread Karthik Deivasigamani
with this message : Checkpoint was declined (tasks not ready) [image: Inline image 1] But we see the job is running, its processing data, the accumulators we have are getting incremented etc but checkpointing fails with tasks not ready message. Wanted to reach out to the community to see if anyone