You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@flink.apache.org by Hao Sun <ha...@zendesk.com> on 2017/12/27 15:41:29 UTC

org.apache.zookeeper.ClientCnxn, Client session timed out

Hi I need some help to figure out the root cause of this error.
I am running flink 1.3.2 on K8S.

My cluster has been up and running for almost two weeks and all of a sudden
I see this familiar error again, my task manager is killed/lost. There are
many ways cause this error, I need help to figure out what is the root
cause this time.

From JM.log

*2017-12-26 14:57:08,624* INFO org.apache.zookeeper.ClientCnxn - Client
session timed out, have not heard from server in 85001ms for sessionid
0x25ddcdec0ef77af, closing socket connection and attempting reconnect
2017-12-26 14:57:23,621 WARN akka.remote.RemoteWatcher - Detected
unreachable: [akka.tcp://flink@fps-flink-taskmanager-960711320-vx0hj:39249]
2017-12-26 14:57:23,623 INFO org.apache.flink.runtime.jobmanager.JobManager
- Task manager akka.tcp://flink@fps-flink-taskmanager-960711320-vx0hj:39249/user/taskmanager
terminated.
2017-12-26 14:57:23,624 INFO
org.apache.flink.runtime.executiongraph.ExecutionGraph - Source:
KafkaSource(maxwell.users) -> MaxwellFilter->Maxwell(maxwell.users) ->
FixedDelayWatermark(maxwell.users) ->
MaxwellFPSEvent->InfluxDBData(maxwell.users) -> Sink:
influxdbSink(maxwell.users) (1/1) (ddca953ae90906daaae08791e1fde729)
switched from RUNNING to FAILED.
java.lang.Exception: TaskManager was lost/killed:
e14186b8fd22699210273f887570a172 @ fps-flink-taskmanager-960711320-vx0hj
(dataPort=37353)


From TM.log
*2017-12-26 14:56:26,019 INFO* org.apache.flink.runtime.taskmanager.Task -
Source: KafkaSource(maxwell.tickets) ->
MaxwellFilter->Maxwell(maxwell.tickets) ->
FixedDelayWatermark(maxwell.tickets) ->
MaxwellFPSEvent->InfluxDBData(maxwell.tickets) -> Sink:
influxdbSink(maxwell.tickets) (1/1) (44b4caf2010bb2b061b67d4f6c8dbc3f)
switched from RUNNING to FAILED.java.lang.Exception: TaskManager
akka://flink/user/taskmanager disconnects from JobManager
akka.tcp://flink@fps-flink-jobmanager:6123/user/jobmanager: *Old JobManager
lost its leadership.*

Somehow TM detected JM leadership loss from ZK and self disconnected?
And couple of seconds later, JM failed to connect to ZK?

After all the cluster recovered nicely by its own, but I am wondering does
this break the exactly-once semantics? If yes, what should I take care?

Thanks team!

Full log:
https://gist.github.com/zenhao/e2f9b929f4eaee32f99948d462db7359

Re: org.apache.zookeeper.ClientCnxn, Client session timed out

Posted by Hao Sun <ha...@zendesk.com>.
Ok, thanks for the clarification.

On Thu, Dec 28, 2017 at 1:05 AM Ufuk Celebi <uc...@apache.org> wrote:

> On Thu, Dec 28, 2017 at 12:11 AM, Hao Sun <ha...@zendesk.com> wrote:
> > Thanks! Great to know I do not have to worry duplicates inside Flink.
> >
> > One more question, why this happens? Because TM and JM both check
> leadership
> > in different interval?
>
> Yes, it's not deterministic how this happens. There will also be cases
> when the JM notices before the TM.
>
>

Re: org.apache.zookeeper.ClientCnxn, Client session timed out

Posted by Ufuk Celebi <uc...@apache.org>.
On Thu, Dec 28, 2017 at 12:11 AM, Hao Sun <ha...@zendesk.com> wrote:
> Thanks! Great to know I do not have to worry duplicates inside Flink.
>
> One more question, why this happens? Because TM and JM both check leadership
> in different interval?

Yes, it's not deterministic how this happens. There will also be cases
when the JM notices before the TM.

Re: org.apache.zookeeper.ClientCnxn, Client session timed out

Posted by Hao Sun <ha...@zendesk.com>.
Thanks! Great to know I do not have to worry duplicates inside Flink.

One more question, why this happens? Because TM and JM both check
leadership in different interval?
> The TM noticed the loss of leadership before the JM did.

On Wed, Dec 27, 2017, 13:52 Ufuk Celebi <uc...@apache.org> wrote:

> On Wed, Dec 27, 2017 at 4:41 PM, Hao Sun <ha...@zendesk.com> wrote:
>
>> Somehow TM detected JM leadership loss from ZK and self disconnected?
>> And couple of seconds later, JM failed to connect to ZK?
>>
>
> Yes, exactly as you describe. The TM noticed the loss of leadership before
> the JM did.
>
>
>> After all the cluster recovered nicely by its own, but I am wondering
>> does this break the exactly-once semantics? If yes, what should I take care?
>>
>
> Great :-) It does not break exactly-once guarantees *within* the Flink
> pipeline as the state of the latest completed checkpoint will be restored
> after recovery. This rewinds your job and might result in duplicate or
> changed output if you don't use an exactly once or idempotent sink.
>
> – Ufuk
>
>

Re: org.apache.zookeeper.ClientCnxn, Client session timed out

Posted by Ufuk Celebi <uc...@apache.org>.
On Wed, Dec 27, 2017 at 4:41 PM, Hao Sun <ha...@zendesk.com> wrote:

> Somehow TM detected JM leadership loss from ZK and self disconnected?
> And couple of seconds later, JM failed to connect to ZK?
>

Yes, exactly as you describe. The TM noticed the loss of leadership before
the JM did.


> After all the cluster recovered nicely by its own, but I am wondering does
> this break the exactly-once semantics? If yes, what should I take care?
>

Great :-) It does not break exactly-once guarantees *within* the Flink
pipeline as the state of the latest completed checkpoint will be restored
after recovery. This rewinds your job and might result in duplicate or
changed output if you don't use an exactly once or idempotent sink.

– Ufuk