[ https://issues.apache.org/jira/browse/KAFKA-6798?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16442765#comment-16442765 ]
Riley Zimmerman commented on KAFKA-6798: ---------------------------------------- Looking at the verbosegc, I do see large mark/sweeps every few hours. However, they are not regular and are not matching up with the zookeeper timeouts. Worst time I see in PMAT is 1884ms "GC Completed". The Used Memory is going from ~1GB to ~430MB after the big mark/sweeps every few hours. {noformat} cat gc.out | grep mark <gc-op id="19824" type="mark" timems="32.393" contextid="19804" timestamp="2018-04-13T22:34:27.138"> <gc-op id="39990" type="mark" timems="44.978" contextid="39970" timestamp="2018-04-14T00:56:06.037"> <gc-op id="60719" type="mark" timems="52.865" contextid="60699" timestamp="2018-04-14T03:20:32.729"> <gc-op id="81340" type="mark" timems="89.813" contextid="81320" timestamp="2018-04-14T05:43:46.783"> <gc-op id="101817" type="mark" timems="44.231" contextid="101797" timestamp="2018-04-14T08:06:28.890"> <gc-op id="122606" type="mark" timems="20.678" contextid="122586" timestamp="2018-04-14T10:33:40.123"> <gc-op id="142795" type="mark" timems="81.406" contextid="142775" timestamp="2018-04-14T12:58:26.778"> <gc-op id="163080" type="mark" timems="63.961" contextid="163060" timestamp="2018-04-14T15:21:11.679"> <gc-op id="181421" type="mark" timems="50.807" contextid="181401" timestamp="2018-04-14T17:44:32.736"> <gc-op id="200434" type="mark" timems="58.002" contextid="200414" timestamp="2018-04-14T20:18:23.332"> <gc-op id="220599" type="mark" timems="35.358" contextid="220579" timestamp="2018-04-14T22:29:34.993"> <gc-op id="245144" type="mark" timems="134.003" contextid="245124" timestamp="2018-04-15T00:37:11.876"> <gc-op id="265513" type="mark" timems="151.646" contextid="265493" timestamp="2018-04-15T03:01:01.553"> <gc-op id="284694" type="mark" timems="26.608" contextid="284674" timestamp="2018-04-15T06:32:31.639"> <gc-op id="306276" type="mark" timems="96.790" contextid="306255" timestamp="2018-04-15T09:47:25.383"> <gc-op id="325725" type="mark" timems="1000.223" contextid="325713" timestamp="2018-04-15T12:28:39.883"> <gc-op id="346540" type="mark" timems="76.978" contextid="346507" timestamp="2018-04-15T14:56:10.935"> <gc-op id="368113" type="mark" timems="398.931" contextid="368101" timestamp="2018-04-15T17:33:42.271"> <gc-op id="389275" type="mark" timems="5.095" contextid="389267" timestamp="2018-04-15T20:12:03.497"> <gc-op id="410511" type="mark" timems="617.447" contextid="410488" timestamp="2018-04-15T22:41:54.406"> <gc-op id="431976" type="mark" timems="1525.245" contextid="431965" timestamp="2018-04-16T01:18:46.744"> <gc-op id="453534" type="mark" timems="197.172" contextid="453514" timestamp="2018-04-16T03:54:43.384"> <gc-op id="475475" type="mark" timems="124.130" contextid="475455" timestamp="2018-04-16T06:31:16.389"> <gc-op id="496771" type="mark" timems="741.591" contextid="496760" timestamp="2018-04-16T09:04:49.666"> <gc-op id="518928" type="mark" timems="1043.252" contextid="518925" timestamp="2018-04-16T11:41:54.770"> <gc-op id="539814" type="mark" timems="43.090" contextid="539794" timestamp="2018-04-16T14:15:38.129"> <gc-op id="560581" type="mark" timems="451.300" contextid="560578" timestamp="2018-04-16T16:49:23.011"> {noformat} > 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)