[ 
https://issues.apache.org/jira/browse/KAFKA-6798?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16442546#comment-16442546
 ] 

Riley Zimmerman commented on KAFKA-6798:
----------------------------------------

I have setup a script monitoring the liveness and readiness probe status from 
kubernetes for the 3 kafka and zookeeper.  At no time in the test did they go 
out of ready state or get restarted, including when the rebalance failures 
happened. 

I also have a script performing a `describe` on my main topic once every 5 
seconds running from each of the 3 kafka.  The command takes 3~6 seconds to 
complete, so I'm getting a view of the topic roughly every 10 seconds.  I can 
see when the ISR goes out of sync.  From the describe I can see that at ~Sat 
Apr 14 16:09:45 UTC the 6 partitions are all in ISR 0,1,2 (or 0,2,1...).  At 
~Sat Apr 14 16:09:52 UTC they are no longer all in sync based on the describe.  
Isr: 0,2
Inside the kafka server logs, the errors start at the beginning of minute Sat 
Apr 14 16:10, after the describe has shown they are out of sync.  
{noformat}
[2018-04-14 12:10:03,391] ERROR [KafkaApi-0] Error when handling request 
{controller_id=2,controller_epoch=1,partition_states=[{topic=metric.json,partition=7,controller_epoch=1,leader=0,leader_epoch=1,isr=[0,2],zk_version=1,replicas=[1,0,2]},
 ... 
,live_brokers=[{id=0,end_points=[{port=9092,host=10.1.42.8,listener_name=PLAINTEXT,security_protocol_type=0}],rack=null},{id=2,end_points=[{port=9092,host=10.1.104.11,listener_name=PLAINTEXT,security_protocol_type=0}],rack=null}]}
 (kafka.server.KafkaApis)
[2018-04-14 12:10:10,484] ERROR [ReplicaFetcherThread-0-1], Error for partition 
[metric.protobuf,0] to broker 
1:org.apache.kafka.common.errors.NotLeaderForPartitionException: This server is 
not the leader for that topic-partition. (kafka.server.ReplicaFetcherThread) 
[2018-04-14 12:10:12,108] ERROR [ReplicaFetcherThread-0-0], Current offset 
6201038 for partition [metric.protobuf,1] out of range; reset offset to 6200065 
(kafka.server.ReplicaFetcherThread){noformat}
 

I have a script running on my kubernetes master watching the 3 zookeepers by 
running `kubectl exec -it server-zookeeper-${1} /opt/stat.sh` each second.   
/opt/stat.sh is doing a `echo stat | nc <zookeeper>`.  My max zk clients is 60, 
and I'm never above 20 in the stats.  Not seeing anything odd there, the 
connection always succeeds each second and the stats return successfully.  

However, most importantly I see these messages for zookeeper:
{noformat}
| grep -i zoo | grep -i connect{noformat}
{noformat}
[2018-04-14 16:10:03,672] INFO zookeeper state changed (Disconnected) 
(org.I0Itec.zkclient.ZkClient)
[2018-04-14 16:10:03,974] INFO Opening socket connection to server 
server-zookeeper-1.server-zookeeper.default.svc.cluster.local/10.1.42.11:2181. 
Will not attempt to authenticate using SASL (unknown error) 
(org.apache.zookeeper.ClientCnxn)
[2018-04-14 16:10:03,982] INFO Socket connection established to 
server-zookeeper-1.server-zookeeper.default.svc.cluster.local/10.1.42.11:2181, 
initiating session (org.apache.zookeeper.ClientCnxn)
[2018-04-14 16:10:03,983] WARN Unable to reconnect to ZooKeeper service, 
session 0x262c0a0b1752762 has expired (org.apache.zookeeper.ClientCnxn)
[2018-04-14 16:10:03,983] INFO Unable to reconnect to ZooKeeper service, 
session 0x262c0a0b1752762 has expired, closing socket connection 
(org.apache.zookeeper.ClientCnxn)
[2018-04-14 16:10:03,983] INFO Initiating client connection, 
connectString=amserver-zookeeper:2181 sessionTimeout=6000 
watcher=org.I0Itec.zkclient.ZkClient@ac73a732 (org.apache.zookeeper.ZooKeeper)
[2018-04-14 16:10:04,166] INFO Opening socket connection to server 
amserver-zookeeper-1.amserver-zookeeper.default.svc.cluster.local/10.1.42.11:2181.
 Will not attempt to authenticate using SASL (unknown error) 
(org.apache.zookeeper.ClientCnxn)
[2018-04-14 16:10:04,166] INFO Socket connection established to 
server-zookeeper-1.server-zookeeper.default.svc.cluster.local/10.1.42.11:2181, 
initiating session (org.apache.zookeeper.ClientCnxn)
[2018-04-14 16:10:04,169] INFO zookeeper state changed (SyncConnected) 
(org.I0Itec.zkclient.ZkClient)
{noformat}
These appear to happen (basically) every 4 hours:
{noformat}
| grep 'INFO Client session timed out, have not heard from server in'
[2018-04-14 12:10:00,365] INFO Client session timed out, have not heard from 
server in 10038ms for sessionid 0x62c09e25d20000, closing socket connection and 
attempting reconnect (org.apache.zookeeper.ClientCnxn)
[2018-04-14 16:10:03,570] INFO Client session timed out, have not heard from 
server in 11751ms for sessionid 0x262c0a0b1752762, closing socket connection 
and attempting reconnect (org.apache.zookeeper.ClientCnxn)
[2018-04-14 20:10:07,232] INFO Client session timed out, have not heard from 
server in 13073ms for sessionid 0x162c09e25d93120, closing socket connection 
and attempting reconnect (org.apache.zookeeper.ClientCnxn)
[2018-04-15 00:09:57,298] INFO Client session timed out, have not heard from 
server in 13098ms for sessionid 0x162c09e25d93aef, closing socket connection 
and attempting reconnect (org.apache.zookeeper.ClientCnxn)
[2018-04-15 04:09:59,711] INFO Client session timed out, have not heard from 
server in 7026ms for sessionid 0x162c09e25d944b3, closing socket connection and 
attempting reconnect (org.apache.zookeeper.ClientCnxn)
[2018-04-15 08:11:28,844] INFO Client session timed out, have not heard from 
server in 18063ms for sessionid 0x162c09e25d94ed9, closing socket connection 
and attempting reconnect (org.apache.zookeeper.ClientCnxn)
[2018-04-15 12:11:29,643] INFO Client session timed out, have not heard from 
server in 7933ms for sessionid 0x62c09e25d25a09, closing socket connection and 
attempting reconnect (org.apache.zookeeper.ClientCnxn)
[2018-04-15 16:11:33,073] INFO Client session timed out, have not heard from 
server in 13294ms for sessionid 0x62c09e25d263c7, closing socket connection and 
attempting reconnect (org.apache.zookeeper.ClientCnxn)
[2018-04-15 20:11:36,895] INFO Client session timed out, have not heard from 
server in 8226ms for sessionid 0x262c0a0b1756cd9, closing socket connection and 
attempting reconnect (org.apache.zookeeper.ClientCnxn)
[2018-04-16 00:11:20,858] INFO Client session timed out, have not heard from 
server in 9589ms for sessionid 0x62c09e25d27798, closing socket connection and 
attempting reconnect (org.apache.zookeeper.ClientCnxn)
[2018-04-16 04:11:36,866] INFO Client session timed out, have not heard from 
server in 10360ms for sessionid 0x62c09e25d28136, closing socket connection and 
attempting reconnect (org.apache.zookeeper.ClientCnxn)
[2018-04-16 09:10:36,641] INFO Client session timed out, have not heard from 
server in 5684ms for sessionid 0x62c09e25d28b18, closing socket connection and 
attempting reconnect (org.apache.zookeeper.ClientCnxn)
[2018-04-16 12:11:37,382] INFO Client session timed out, have not heard from 
server in 6215ms for sessionid 0x62c09e25d28b18, closing socket connection and 
attempting reconnect (org.apache.zookeeper.ClientCnxn)
[2018-04-16 14:18:34,431] INFO Client session timed out, have not heard from 
server in 4122ms for sessionid 0x262c0a0b1759d92, closing socket connection and 
attempting reconnect (org.apache.zookeeper.ClientCnxn)
[2018-04-16 16:11:38,582] INFO Client session timed out, have not heard from 
server in 11997ms for sessionid 0x262c0a0b1759d92, closing socket connection 
and attempting reconnect (org.apache.zookeeper.ClientCnxn)
[2018-04-16 20:11:39,069] INFO Client session timed out, have not heard from 
server in 10402ms for sessionid 0x262c0a0b175a706, closing socket connection 
and attempting reconnect (org.apache.zookeeper.ClientCnxn)
[2018-04-17 00:11:37,266] INFO Client session timed out, have not heard from 
server in 8727ms for sessionid 0x262c0a0b175b027, closing socket connection and 
attempting reconnect (org.apache.zookeeper.ClientCnxn)
[2018-04-17 04:11:34,500] INFO Client session timed out, have not heard from 
server in 4287ms for sessionid 0x162c09e25d9bb9a, closing socket connection and 
attempting reconnect (org.apache.zookeeper.ClientCnxn)
[2018-04-17 08:11:30,872] INFO Client session timed out, have not heard from 
server in 6409ms for sessionid 0x162c09e25d9bb9a, closing socket connection and 
attempting reconnect (org.apache.zookeeper.ClientCnxn)
[2018-04-17 12:11:42,402] INFO Client session timed out, have not heard from 
server in 6820ms for sessionid 0x162c09e25d9bb9a, closing socket connection and 
attempting reconnect (org.apache.zookeeper.ClientCnxn)
[2018-04-17 16:11:40,462] INFO Client session timed out, have not heard from 
server in 7476ms for sessionid 0x162c09e25d9bb9a, closing socket connection and 
attempting reconnect (org.apache.zookeeper.ClientCnxn)
[2018-04-17 20:11:35,940] INFO Client session timed out, have not heard from 
server in 6862ms for sessionid 0x62c09e25d2e0c1, closing socket connection and 
attempting reconnect (org.apache.zookeeper.ClientCnxn)
[2018-04-18 00:11:47,066] INFO Client session timed out, have not heard from 
server in 6986ms for sessionid 0x62c09e25d2e0c1, closing socket connection and 
attempting reconnect (org.apache.zookeeper.ClientCnxn)
[2018-04-18 03:48:54,270] INFO Client session timed out, have not heard from 
server in 4705ms for sessionid 0x62c09e25d2e0c1, closing socket connection and 
attempting reconnect (org.apache.zookeeper.ClientCnxn)
[2018-04-18 04:11:46,785] INFO Client session timed out, have not heard from 
server in 5996ms for sessionid 0x162c09e25d9fc48, closing socket connection and 
attempting reconnect (org.apache.zookeeper.ClientCnxn)
[2018-04-18 08:11:42,749] INFO Client session timed out, have not heard from 
server in 7037ms for sessionid 0x162c09e25d9fc48, closing socket connection and 
attempting reconnect (org.apache.zookeeper.ClientCnxn)
{noformat}
As shown by my scripts querying the zookeeper statefulsets from the kubemaster, 
zookeeper is alive and able to be talked to.  Yet something during the 
rebalances is causing it to time out.  
I see almost zero CPU activity on the 3 zookeepers.  Disk, memory, etc also 
look good.  Total network bandwidth is ~30Mbps up and down, no where near the 
10Gbps link max. Is it possible that the kubernetes networking with Calico is 
timing things out?  That doesn't seem likely but with kubernetes and docker 
there are a lot of areas to look.  

It seems to me like zookeeper is having trouble handling that many rebalance 
requests at once, even if the cpu is basically idle.  Possibly some tuning is 
needed?  Any suggestions would be appreciated, thanks!

 

 

 

> Kafka leader rebalance failures
> -------------------------------
>
>                 Key: KAFKA-6798
>                 URL: https://issues.apache.org/jira/browse/KAFKA-6798
>             Project: Kafka
>          Issue Type: Bug
>    Affects Versions: 0.10.2.1, 1.0.1
>            Reporter: Riley Zimmerman
>            Priority: Critical
>
> I am running 3 Kafka (version 0.10.2.1 and more recently moved to 1.0.1) with 
> 3 Zookeeper (v3.4.9) as statefulsets in a kubernetes v1.9.1 deployment.  My 
> partitions are replication factor 3.  My main workload involves a kafka 
> streams consumer/producer (storing offsets in kafka) and a second kafka 
> consumer storing offsets in zookeeper (only commits every 30 seconds).  There 
> are ~200,000 kafka messages going through each per minute.  The log.retention 
> settings are all 4 hours.  I have auto.leader.rebalance.enabled.  
> I am randomly having failures during the rebalances.  The result is that 
> partitions for both topics and consumer_offsets go out of sync and the 
> partition leader becomes -1.  After 4 hours there is another (auto?) 
> rebalance and sometimes it sorts itself out.  Sometimes it runs for weeks 
> without problems, other times it it happens multiple times in a few days.  It 
> appears to happen earlier in test runs if it is going to happen.   
> {noformat}
> Topic:__consumer_offsets        PartitionCount:50       ReplicationFactor:3   
>   
> Configs:segment.bytes=104857600,cleanup.policy=compact,compression.type=producer
>         Topic: __consumer_offsets       Partition: 0    Leader: -1      
> Replicas: 2,0,1 Isr:
>         Topic: __consumer_offsets       Partition: 1    Leader: 0       
> Replicas: 0,1,2 Isr: 1,2,0
>         Topic: __consumer_offsets       Partition: 2    Leader: 1       
> Replicas: 1,2,0 Isr: 2,1,0
>         Topic: __consumer_offsets       Partition: 3    Leader: -1      
> Replicas: 2,1,0 Isr:
> {noformat}
> {noformat}
> [2018-03-20 12:42:32,180] WARN [Controller 2]: Partition [agent.metadata,5] 
> failed to complete preferred replica leader election. Leader is -1 
> (kafka.controller.KafkaController)
> {noformat}
> {noformat}
> [2018-03-20 11:02:32,099] TRACE Controller 2 epoch 27 started leader election 
> for partition [__consumer_offsets,30] (state.change.logger)
> [2018-03-20 11:02:32,101] ERROR Controller 2 epoch 27 encountered error while 
> electing leader for partition [__consumer_offsets,30] due to: Preferred 
> replica 2 for partition [__consumer_offsets,30] is either not alive or not in 
> the isr. Current leader and ISR: [{"leader":-1,"leader_epoch":59,"isr":[]}]. 
> (state.change.logger)
> [2018-03-20 11:02:32,101] ERROR Controller 2 epoch 27 initiated state change 
> for partition [__consumer_offsets,30] from OnlinePartition to OnlinePartition 
> failed (state.change.logger)
> kafka.common.StateChangeFailedException: encountered error while electing 
> leader for partition [__consumer_offsets,30] due to: Preferred replica 2 for 
> partition [__consumer_offsets,30] is either not alive or not in the isr. 
> Current leader and ISR: [{"leader":-1,"leader_epoch":59,"isr":[]}].
>       at 
> kafka.controller.PartitionStateMachine.electLeaderForPartition(PartitionStateMachine.scala:362)
>       at 
> kafka.controller.PartitionStateMachine.kafka$controller$PartitionStateMachine$$handleStateChange(PartitionStateMachine.scala:202)
>       at 
> kafka.controller.PartitionStateMachine$$anonfun$handleStateChanges$2.apply(PartitionStateMachine.scala:141)
>       at 
> kafka.controller.PartitionStateMachine$$anonfun$handleStateChanges$2.apply(PartitionStateMachine.scala:140)
>       at scala.collection.immutable.Set$Set1.foreach(Set.scala:94)
>       at 
> kafka.controller.PartitionStateMachine.handleStateChanges(PartitionStateMachine.scala:140)
>       at 
> kafka.controller.KafkaController.onPreferredReplicaElection(KafkaController.scala:662)
>       at 
> kafka.controller.KafkaController$$anonfun$kafka$controller$KafkaController$$checkAndTriggerPartitionRebalance$4$$anonfun$apply$16$$anonfun$apply$5.apply$mcV$sp(KafkaController.scala:1230)
>       at 
> kafka.controller.KafkaController$$anonfun$kafka$controller$KafkaController$$checkAndTriggerPartitionRebalance$4$$anonfun$apply$16$$anonfun$apply$5.apply(KafkaController.scala:1225)
>       at 
> kafka.controller.KafkaController$$anonfun$kafka$controller$KafkaController$$checkAndTriggerPartitionRebalance$4$$anonfun$apply$16$$anonfun$apply$5.apply(KafkaController.scala:1225)
>       at kafka.utils.CoreUtils$.inLock(CoreUtils.scala:213)
>       at 
> kafka.controller.KafkaController$$anonfun$kafka$controller$KafkaController$$checkAndTriggerPartitionRebalance$4$$anonfun$apply$16.apply(KafkaController.scala:1222)
>       at 
> kafka.controller.KafkaController$$anonfun$kafka$controller$KafkaController$$checkAndTriggerPartitionRebalance$4$$anonfun$apply$16.apply(KafkaController.scala:1221)
>       at 
> scala.collection.mutable.HashMap$$anon$1$$anonfun$foreach$2.apply(HashMap.scala:103)
>       at 
> scala.collection.mutable.HashMap$$anon$1$$anonfun$foreach$2.apply(HashMap.scala:103)
>       at 
> scala.collection.mutable.HashTable$class.foreachEntry(HashTable.scala:230)
>       at scala.collection.mutable.HashMap.foreachEntry(HashMap.scala:40)
>       at scala.collection.mutable.HashMap$$anon$1.foreach(HashMap.scala:103)
>       at 
> kafka.controller.KafkaController$$anonfun$kafka$controller$KafkaController$$checkAndTriggerPartitionRebalance$4.apply(KafkaController.scala:1221)
>       at 
> kafka.controller.KafkaController$$anonfun$kafka$controller$KafkaController$$checkAndTriggerPartitionRebalance$4.apply(KafkaController.scala:1203)
>       at scala.collection.immutable.Map$Map3.foreach(Map.scala:161)
>       at 
> kafka.controller.KafkaController.kafka$controller$KafkaController$$checkAndTriggerPartitionRebalance(KafkaController.scala:1203)
>       at 
> kafka.controller.KafkaController$$anonfun$onControllerFailover$1.apply$mcV$sp(KafkaController.scala:352)
>       at 
> kafka.utils.KafkaScheduler$$anonfun$1.apply$mcV$sp(KafkaScheduler.scala:110)
>       at kafka.utils.CoreUtils$$anon$1.run(CoreUtils.scala:57)
>       at 
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:522)
>       at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:319)
>       at 
> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:191)
>       at 
> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:305)
>       at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1160)
>       at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:635)
>       at java.lang.Thread.run(Thread.java:811)
> Caused by: kafka.common.StateChangeFailedException: Preferred replica 2 for 
> partition [__consumer_offsets,30] is either not alive or not in the isr. 
> Current leader and ISR: [{"leader":-1,"leader_epoch":59,"isr":[]}]
>       at 
> kafka.controller.PreferredReplicaPartitionLeaderSelector.selectLeader(PartitionLeaderSelector.scala:157)
>       at 
> kafka.controller.PartitionStateMachine.electLeaderForPartition(PartitionStateMachine.scala:339)
>       ... 31 more
> {noformat}
> There are these messages in the zookeeper logs, but they are happening all of 
> the time, not only when the failures happen:
> {noformat}
> 2018-03-29 04:46:43,495 [myid:0] - WARN  
> [NIOServerCxn.Factory:0.0.0.0/0.0.0.0:2181:NIOServerCnxn@368] - caught end of 
> stream exception
> EndOfStreamException: Unable to read additional data from client sessionid 
> 0x0, likely client has closed socket
>         at 
> org.apache.zookeeper.server.NIOServerCnxn.doIO(NIOServerCnxn.java:239)
>         at 
> org.apache.zookeeper.server.NIOServerCnxnFactory.run(NIOServerCnxnFactory.java:203)
>         at java.lang.Thread.run(Thread.java:811)
> {noformat}
> {noformat}
> 2018-03-29 08:56:46,195 [myid:1] - INFO  [ProcessThread(sid:1 
> cport:-1)::PrepRequestProcessor@648] - Got user-level KeeperException when 
> processing sessionid:0x62633bc4724c26 type:setData cxid:0x654465 
> zxid:0x100361191 txntype:-1 reqpath:n/a Error 
> Path:/brokers/topics/metric.json/partitions/1/state Error:KeeperErrorCode = 
> BadVersion for /brokers/topics/metric.json/partitions/1/state
> 2018-03-29 08:56:46,201 [myid:1] - INFO  [ProcessThread(sid:1 
> cport:-1)::PrepRequestProcessor@648] - Got user-level KeeperException when 
> processing sessionid:0x62633bc4724c26 type:setData cxid:0x654467 
> zxid:0x100361192 txntype:-1 reqpath:n/a Error 
> Path:/brokers/topics/metric.json/partitions/10/state Error:KeeperErrorCode = 
> BadVersion for /brokers/topics/metric.json/partitions/10/state
> {noformat}
> I saw https://issues.apache.org/jira/browse/KAFKA-4084 which involves major 
> changes to the rebalances.  I'm in the process of moving to kafka 1.1.0 to 
> see if it helps.  
>  Any advice on what else to look into would be appreciated.  
>  



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to