I would think that network problems between Flink and Zookeeper in HA mode 
could indeed lead to problems. Maybe Till (in CC) has a better idea of what is 
going on there).

> Am 19.01.2017 um 14:55 schrieb Andrew Ge Wu <andrew.ge...@eniro.com>:
> 
> Hi Stefan
> 
> Yes we are running in HA mode with dedicated zookeeper cluster. As far as I 
> can see it looks like a networking issue with zookeeper cluster.
> 2 out of 5 zookeeper reported something around the same time:
> 
> server1
> 2017-01-19 11:52:13,044 [myid:1] - WARN  
> [QuorumPeer[myid=1]/0:0:0:0:0:0:0:0:2181:Follower@89] - Exception when 
> following the leader
> java.net.SocketTimeoutException: Read timed out
>         at java.net.SocketInputStream.socketRead0(Native Method)
>         at java.net.SocketInputStream.read(SocketInputStream.java:150)
>         at java.net.SocketInputStream.read(SocketInputStream.java:121)
>         at java.io.BufferedInputStream.fill(BufferedInputStream.java:246)
>         at java.io.BufferedInputStream.read(BufferedInputStream.java:265)
>         at java.io.DataInputStream.readInt(DataInputStream.java:387)
>         at 
> org.apache.jute.BinaryInputArchive.readInt(BinaryInputArchive.java:63)
>         at 
> org.apache.zookeeper.server.quorum.QuorumPacket.deserialize(QuorumPacket.java:83)
>         at 
> org.apache.jute.BinaryInputArchive.readRecord(BinaryInputArchive.java:103)
>         at 
> org.apache.zookeeper.server.quorum.Learner.readPacket(Learner.java:153)
>         at 
> org.apache.zookeeper.server.quorum.Follower.followLeader(Follower.java:85)
>         at 
> org.apache.zookeeper.server.quorum.QuorumPeer.run(QuorumPeer.java:786)
> 2017-01-19 11:52:13,045 [myid:1] - INFO  
> [QuorumPeer[myid=1]/0:0:0:0:0:0:0:0:2181:Follower@166] - shutdown called
> java.lang.Exception: shutdown Follower
>         at 
> org.apache.zookeeper.server.quorum.Follower.shutdown(Follower.java:166)
>         at 
> org.apache.zookeeper.server.quorum.QuorumPeer.run(QuorumPeer.java:790)
> 2017-01-19 11:52:13,045 [myid:1] - INFO  
> [QuorumPeer[myid=1]/0:0:0:0:0:0:0:0:2181:NIOServerCnxn@1007] - Closed socket 
> connection for client /172.27.163.227:51800 which had sessionid 
> 0x159b505820a0009
> 2017-01-19 11:52:13,046 [myid:1] - INFO  
> [QuorumPeer[myid=1]/0:0:0:0:0:0:0:0:2181:NIOServerCnxn@1007] - Closed socket 
> connection for client /172.27.163.227:51798 which had sessionid 
> 0x159b505820a0008
> 2017-01-19 11:52:13,046 [myid:1] - INFO  
> [QuorumPeer[myid=1]/0:0:0:0:0:0:0:0:2181:NIOServerCnxn@1007] - Closed socket 
> connection for client /0:0:0:0:0:0:0:1:46891 which had sessionid 
> 0x1537b32bbe100ad
> 2017-01-19 11:52:13,046 [myid:1] - INFO  
> [QuorumPeer[myid=1]/0:0:0:0:0:0:0:0:2181:NIOServerCnxn@1007] - Closed socket 
> connection for client /172.27.165.64:50075 which had sessionid 
> 0x159b505820a000d
> 2017-01-19 11:52:13,046 [myid:1] - INFO  
> [QuorumPeer[myid=1]/0:0:0:0:0:0:0:0:2181:FollowerZooKeeperServer@139] - 
> Shutting down
> 2017-01-19 11:52:13,046 [myid:1] - INFO  
> [QuorumPeer[myid=1]/0:0:0:0:0:0:0:0:2181:ZooKeeperServer@441] - shutting down
> 
> 
> server2
> 2017-01-19 11:52:13,061 [myid:2] - INFO  
> [WorkerReceiver[myid=2]:FastLeaderElection@597] - Notification: 1 (message 
> format version), 1 (n.leader), 0x4000000cb (n.zxid), 0x4 (n.round), LOOKING 
> (n.state), 1 (n.sid), 0x4 (n.peerEpoch) FOLLOWING (my state)
> 2017-01-19 11:52:13,082 [myid:2] - INFO  
> [WorkerReceiver[myid=2]:FastLeaderElection@597] - Notification: 1 (message 
> format version), 4 (n.leader), 0x4000000cb (n.zxid), 0x4 (n.round), LOOKING 
> (n.state), 4 (n.sid), 0x4 (n.peerEpoch) FOLLOWING (my state)
> 2017-01-19 11:52:13,083 [myid:2] - INFO  
> [WorkerReceiver[myid=2]:FastLeaderElection@597] - Notification: 1 (message 
> format version), 4 (n.leader), 0x4000000cb (n.zxid), 0x4 (n.round), LOOKING 
> (n.state), 1 (n.sid), 0x4 (n.peerEpoch) FOLLOWING (my state)
> 2017-01-19 11:52:13,284 [myid:2] - INFO  
> [WorkerReceiver[myid=2]:FastLeaderElection@597] - Notification: 1 (message 
> format version), 4 (n.leader), 0x4000000cb (n.zxid), 0x4 (n.round), LOOKING 
> (n.state), 4 (n.sid), 0x4 (n.peerEpoch) FOLLOWING (my state)
> 2017-01-19 11:52:13,285 [myid:2] - INFO  
> [WorkerReceiver[myid=2]:FastLeaderElection@597] - Notification: 1 (message 
> format version), 4 (n.leader), 0x4000000cb (n.zxid), 0x4 (n.round), LOOKING 
> (n.state), 1 (n.sid), 0x4 (n.peerEpoch) FOLLOWING (my state)
> 2017-01-19 11:52:13,310 [myid:2] - INFO  
> [NIOServerCxn.Factory:0.0.0.0/0.0.0.0:2181:NIOServerCnxnFactory@197] - 
> Accepted socket connection from /172.27.163.227:39302
> 2017-01-19 11:52:13,311 [myid:2] - INFO  
> [NIOServerCxn.Factory:0.0.0.0/0.0.0.0:2181:ZooKeeperServer@861] - Client 
> attempting to renew session 0x159b505820a0009 at /172.27.163.227:39302
> 2017-01-19 11:52:13,312 [myid:2] - INFO  
> [NIOServerCxn.Factory:0.0.0.0/0.0.0.0:2181:Learner@108] - Revalidating 
> client: 0x159b505820a0009
> 2017-01-19 11:52:13,687 [myid:2] - INFO  
> [WorkerReceiver[myid=2]:FastLeaderElection@597] - Notification: 1 (message 
> format version), 4 (n.leader), 0x4000000cb (n.zxid), 0x4 (n.round), LOOKING 
> (n.state), 4 (n.sid), 0x4 (n.peerEpoch) FOLLOWING (my state)
> 2017-01-19 11:52:13,687 [myid:2] - INFO  
> [WorkerReceiver[myid=2]:FastLeaderElection@597] - Notification: 1 (message 
> format version), 4 (n.leader), 0x4000000cb (n.zxid), 0x4 (n.round), LOOKING 
> (n.state), 1 (n.sid), 0x4 (n.peerEpoch) FOLLOWING (my state)
> 2017-01-19 11:52:14,488 [myid:2] - INFO  
> [WorkerReceiver[myid=2]:FastLeaderElection@597] - Notification: 1 (message 
> format version), 4 (n.leader), 0x4000000cb (n.zxid), 0x4 (n.round), LOOKING 
> (n.state), 1 (n.sid), 0x4 (n.peerEpoch) FOLLOWING (my state)
> 2017-01-19 11:52:14,489 [myid:2] - INFO  
> [WorkerReceiver[myid=2]:FastLeaderElection@597] - Notification: 1 (message 
> format version), 4 (n.leader), 0x4000000cb (n.zxid), 0x4 (n.round), LOOKING 
> (n.state), 4 (n.sid), 0x4 (n.peerEpoch) FOLLOWING (my state)
> 2017-01-19 11:52:14,719 [myid:2] - INFO  
> [QuorumPeer[myid=2]/0:0:0:0:0:0:0:0:2181:ZooKeeperServer@617] - Established 
> session 0x159b505820a0009 with negotiated timeout 40000 for client 
> /172.27.163.227:39302
> 
> I can’t say for sure if data in zookeeper is corrupted at that time. I guess 
> Flink is kinda sensitive on that?
> 
> 
> Thanks
> 
> 
> 
> Andrew
> 
> 
> 
> 
> 
>> On 19 Jan 2017, at 14:19, Stefan Richter <s.rich...@data-artisans.com 
>> <mailto:s.rich...@data-artisans.com>> wrote:
>> 
>> Hi,
>> 
>> I think depending on your configuration of Flink (are you using high 
>> availability mode?) and the type of ZK glitches we are talking about, it can 
>> very well be that some of Flink’s meta data in ZK got corrupted and the 
>> system can not longer operate. But for a deeper analysis, we would need more 
>> details about your configuration and the ZK problem.
>> 
>> Best,
>> Stefan
>> 
>>> Am 19.01.2017 um 13:16 schrieb Andrew Ge Wu <andrew.ge...@eniro.com 
>>> <mailto:andrew.ge...@eniro.com>>:
>>> 
>>> Hi,
>>> 
>>> 
>>> We recently had several zookeeper glitch, when that happens it seems to 
>>> take flink cluster with it.
>>> 
>>> We are running on 1.03
>>> 
>>> It started like this:
>>> 
>>> 
>>> 2017-01-19 11:52:13,047 INFO  org.apache.zookeeper.ClientCnxn               
>>>                 - Unable to read additional data from server sessionid 
>>> 0x159b505820a0008, likely server has closed socket, closing socket 
>>> connection and attempting reconnect
>>> 2017-01-19 11:52:13,047 INFO  org.apache.zookeeper.ClientCnxn               
>>>                 - Unable to read additional data from server sessionid 
>>> 0x159b505820a0009, likely server has closed socket, closing socket 
>>> connection and attempting reconnect
>>> 2017-01-19 11:52:13,151 INFO  
>>> org.apache.flink.shaded.org.apache.curator.framework.state.ConnectionStateManager
>>>   - State change: SUSPENDED
>>> 2017-01-19 11:52:13,151 INFO  
>>> org.apache.flink.shaded.org.apache.curator.framework.state.ConnectionStateManager
>>>   - State change: SUSPENDED
>>> 2017-01-19 11:52:13,166 WARN  
>>> org.apache.flink.runtime.jobmanager.ZooKeeperSubmittedJobGraphStore  - 
>>> ZooKeeper connection SUSPENDED. Changes to the submitted job graphs are not 
>>> monitored (temporarily).
>>> 2017-01-19 11:52:13,169 INFO  
>>> org.apache.flink.runtime.jobmanager.JobManager                - JobManager 
>>> akka://flink/user/jobmanager#1976923422 
>>> <akka://flink/user/jobmanager#1976923422> was revoked leadership.
>>> 2017-01-19 11:52:13,179 INFO  
>>> org.apache.flink.runtime.executiongraph.ExecutionGraph        - op1 -> 
>>> (Map, Map -> op2) (18/24) (5336dd375eb12616c5a0e93c84f93465) switched from 
>>> RUNNING to FAILED
>>> 
>>> 
>>> 
>>> Then our web-ui stopped serving and job manager stuck in an exception loop 
>>> like this:
>>> 2017-01-19 13:05:13,521 WARN  
>>> org.apache.flink.runtime.jobmanager.JobManager                - Discard 
>>> message 
>>> LeaderSessionMessage(0318ecf5-7069-41b2-a793-2f24bdbaa287,01/19/2017 
>>> 13:05:13     Job execution switched to status RESTARTING.) because the 
>>> expected leader session I
>>> D None did not equal the received leader session ID 
>>> Some(0318ecf5-7069-41b2-a793-2f24bdbaa287).
>>> 2017-01-19 13:05:13,521 INFO  
>>> org.apache.flink.runtime.executiongraph.restart.FixedDelayRestartStrategy  
>>> - Delaying retry of job execution for xxxxx ms …
>>> 
>>> 
>>> Is it because we misconfigured anything? or this is expected behavior? When 
>>> this happens we have to restart the cluster to bring it back.
>>> 
>>> 
>>> Thanks!
>>> 
>>> 
>>> Andrew
>>> -- 
>>> Confidentiality Notice: This e-mail transmission may contain confidential 
>>> or legally privileged information that is intended only for the individual 
>>> or entity named in the e-mail address. If you are not the intended 
>>> recipient, you are hereby notified that any disclosure, copying, 
>>> distribution, or reliance upon the contents of this e-mail is strictly 
>>> prohibited and may be unlawful. If you have received this e-mail in error, 
>>> please notify the sender immediately by return e-mail and delete all copies 
>>> of this message.
>> 
> 
> 
> Confidentiality Notice: This e-mail transmission may contain confidential or 
> legally privileged information that is intended only for the individual or 
> entity named in the e-mail address. If you are not the intended recipient, 
> you are hereby notified that any disclosure, copying, distribution, or 
> reliance upon the contents of this e-mail is strictly prohibited and may be 
> unlawful. If you have received this e-mail in error, please notify the sender 
> immediately by return e-mail and delete all copies of this message.

Reply via email to