flink-user mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Maciek Próchniak <...@touk.pl>
Subject Re: Checkpoint was declined (tasks not ready)
Date Mon, 23 Oct 2017 12:04:16 GMT
it seems that one of operators is stuck during recovery:

prio=5 os_prio=0 tid=0x00007f634bb31000 nid=0xd5e in Object.wait() 
[0x00007f63f13cc000]
java.lang.Thread.State: WAITING (on object monitor)
at java.lang.Object.wait(Native Method)
at java.lang.Object.wait(Object.java:502 <http://object.java:502/>)
at 
org.apache.flink.streaming.api.operators.async.AsyncWaitOperator.addAsyncBufferEntry(AsyncWaitOperator.java:406

<http://asyncwaitoperator.java:406/>)
at 
org.apache.flink.streaming.api.operators.async.AsyncWaitOperator.processElement(AsyncWaitOperator.java:228

<http://asyncwaitoperator.java:228/>)
at 
org.apache.flink.streaming.api.operators.async.AsyncWaitOperator.open(AsyncWaitOperator.java:174

<http://asyncwaitoperator.java:174/>)
at 
org.apache.flink.streaming.runtime.tasks.StreamTask.openAllOperators(StreamTask.java:376 
<http://streamtask.java:376/>)
at 
org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:253 
<http://streamtask.java:253/>)
- locked <0x000000037ae51a38> (a java.lang.Object)
atorg.apache.flink.runtime.taskmanager.Task.run 
<http://org.apache.flink.runtime.taskmanager.task.run/>(Task.java:702)
atjava.lang.Thread.run <http://java.lang.thread.run/>(Thread.java:745)


On 23/10/2017 13:54, Maciek Próchniak wrote:
>
> we also have similar problem - it happens really often when we invoke 
> async operators (ordered one). But we also observe that job is not 
> starting properly - we don't process any data when such problems appear
>
> we'll keep you posted if we manage to find exact cause...
>
>
> thanks,
> maciek
>
> On 09/10/2017 12:10, Karthik Deivasigamani wrote:
>> Hi Stephan,
>>     Once the job restarts due to an async io operator timeout we 
>> notice that its checkpoints never succeed again.  But the job is 
>> running fine and is processing data.
>> ~
>> Karthik
>>
>>
>> On Mon, Oct 9, 2017 at 3:19 PM, Stephan Ewen <sewen@apache.org 
>> <mailto:sewen@apache.org>> wrote:
>>
>>     As long as this does not appear all the time, but only once in a
>>     while, it should not be a problem.
>>     It simply means that this particular checkpoint could not be
>>     triggered, because some sources were not ready yet.
>>
>>     It should try another checkpoint and then be okay.
>>
>>
>>     On Fri, Oct 6, 2017 at 4:53 PM, Karthik Deivasigamani
>>     <karthik.d@gmail.com <mailto:karthik.d@gmail.com>> wrote:
>>
>>         We are using Flink 1.3.1 in Standalone mode with a HA job
>>         manager setup.
>>         ~
>>         Karthik
>>
>>         On Fri, Oct 6, 2017 at 8:22 PM, Karthik Deivasigamani
>>         <karthik.d@gmail.com <mailto:karthik.d@gmail.com>> wrote:
>>
>>             Hi,
>>                 I'm noticing a weird issue with our flink streaming
>>             job. We use async io operator which makes a HTTP call and
>>             in certain cases when the async task times out, it throws
>>             an exception and causing the job to restart.
>>
>>             java.lang.Exception: An async function call terminated with an exception.
Failing the AsyncWaitOperator.
>>             	at org.apache.flink.streaming.api.operators.async.Emitter.output(Emitter.java:136)
>>             	at org.apache.flink.streaming.api.operators.async.Emitter.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.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 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
>>             else has experienced this issue before?
>>             ~
>>             Karthik
>>
>>
>>
>>
>


Mime
View raw message