Uploaded image for project: 'Kafka'
  1. Kafka
  2. KAFKA-2729

Cached zkVersion not equal to that in zookeeper, broker not recovering.

    Details

    • Type: Bug
    • Status: Open
    • Priority: Major
    • Resolution: Unresolved
    • Affects Version/s: 0.8.2.1, 0.9.0.0, 0.10.0.0, 0.10.1.0, 0.11.0.0
    • Fix Version/s: None
    • Component/s: None
    • Labels:
      None

      Description

      After a small network wobble where zookeeper nodes couldn't reach each other, we started seeing a large number of undereplicated partitions. The zookeeper cluster recovered, however we continued to see a large number of undereplicated partitions. Two brokers in the kafka cluster were showing this in the logs:

      [2015-10-27 11:36:00,888] INFO Partition [__samza_checkpoint_event-creation_1,3] on broker 5: Shrinking ISR for partition [__samza_checkpoint_event-creation_1,3] from 6,5 to 5 (kafka.cluster.Partition)
      [2015-10-27 11:36:00,891] INFO Partition [__samza_checkpoint_event-creation_1,3] on broker 5: Cached zkVersion [66] not equal to that in zookeeper, skip updating ISR (kafka.cluster.Partition)
      

      For all of the topics on the effected brokers. Both brokers only recovered after a restart. Our own investigation yielded nothing, I was hoping you could shed some light on this issue. Possibly if it's related to: https://issues.apache.org/jira/browse/KAFKA-1382 , however we're using 0.8.2.1.

        Issue Links

          Activity

          Hide
          toidi Iskandarov Eduard added a comment -

          Our kafka cluster met the same issue:

          kafka2 1448388319:093 [2015-11-24 21:05:19,387] INFO Partition [dstat_wc_cpl_log,13] on broker 2: Shrinking ISR for partition [dstat_wc_cpl_log,13] from 2,1 to 2 (kafka.cluster.Partition)
          kafka2 1448388319:094 [2015-11-24 21:05:19,404] INFO Partition [dstat_wc_cpl_log,13] on broker 2: Cached zkVersion [332] not equal to that in zookeeper, skip updating ISR (kafka.cluster.Partition)
          

          We use confluent.io's kafka distribution.
          Kafka version is 0.8.2.2.

          Show
          toidi Iskandarov Eduard added a comment - Our kafka cluster met the same issue: kafka2 1448388319:093 [2015-11-24 21:05:19,387] INFO Partition [dstat_wc_cpl_log,13] on broker 2: Shrinking ISR for partition [dstat_wc_cpl_log,13] from 2,1 to 2 (kafka.cluster.Partition) kafka2 1448388319:094 [2015-11-24 21:05:19,404] INFO Partition [dstat_wc_cpl_log,13] on broker 2: Cached zkVersion [332] not equal to that in zookeeper, skip updating ISR (kafka.cluster.Partition) We use confluent.io's kafka distribution. Kafka version is 0.8.2.2.
          Hide
          agomez Andres Gomez Ferrer added a comment - - edited

          Our kafka cluster met the same issue too:

          [2016-01-12 01:16:15,907] INFO Partition [__consumer_offsets,10] on broker 0: Shrinking ISR for partition [__consumer_offsets,10] from 0,1 to 0 (kafka.cluster.Partition)
          [2016-01-12 01:16:15,909] INFO Partition [__consumer_offsets,10] on broker 0: Cached zkVersion [3240] not equal to that in zookeeper, skip updating ISR (kafka.cluster.Partition)
          [2016-01-12 01:16:15,909] INFO Partition [__consumer_offsets,45] on broker 0: Shrinking ISR for partition [__consumer_offsets,45] from 0,1 to 0 (kafka.cluster.Partition)
          [2016-01-12 01:16:15,911] INFO Partition [__consumer_offsets,45] on broker 0: Cached zkVersion [3192] not equal to that in zookeeper, skip updating ISR (kafka.cluster.Partition)
          [2016-01-12 01:16:15,911] INFO Partition [__consumer_offsets,24] on broker 0: Shrinking ISR for partition [__consumer_offsets,24] from 0,1 to 0 (kafka.cluster.Partition)
          [2016-01-12 01:16:15,912] INFO Partition [__consumer_offsets,24] on broker 0: Cached zkVersion [3233] not equal to that in zookeeper, skip updating ISR (kafka.cluster.Partition)
          

          Kafka version is 0.8.2.2

          Show
          agomez Andres Gomez Ferrer added a comment - - edited Our kafka cluster met the same issue too: [2016-01-12 01:16:15,907] INFO Partition [__consumer_offsets,10] on broker 0: Shrinking ISR for partition [__consumer_offsets,10] from 0,1 to 0 (kafka.cluster.Partition) [2016-01-12 01:16:15,909] INFO Partition [__consumer_offsets,10] on broker 0: Cached zkVersion [3240] not equal to that in zookeeper, skip updating ISR (kafka.cluster.Partition) [2016-01-12 01:16:15,909] INFO Partition [__consumer_offsets,45] on broker 0: Shrinking ISR for partition [__consumer_offsets,45] from 0,1 to 0 (kafka.cluster.Partition) [2016-01-12 01:16:15,911] INFO Partition [__consumer_offsets,45] on broker 0: Cached zkVersion [3192] not equal to that in zookeeper, skip updating ISR (kafka.cluster.Partition) [2016-01-12 01:16:15,911] INFO Partition [__consumer_offsets,24] on broker 0: Shrinking ISR for partition [__consumer_offsets,24] from 0,1 to 0 (kafka.cluster.Partition) [2016-01-12 01:16:15,912] INFO Partition [__consumer_offsets,24] on broker 0: Cached zkVersion [3233] not equal to that in zookeeper, skip updating ISR (kafka.cluster.Partition) Kafka version is 0.8.2.2
          Hide
          elevy Elias Levy added a comment - - edited

          Had the same issue happen here while testing a 5 node Kafka cluster with a 3 node ZK ensemble on Kubernetes on AWS. After running for a while broker 2 started showing the "Cached zkVersion [29] not equal to that in zookeeper, skip updating ISR" error message for al the partitions it leads. For those partition it is the only in sync replica. That has led to the Samza jobs I was running to stop.

          I should note that I am running 0.9.0.0.

          Show
          elevy Elias Levy added a comment - - edited Had the same issue happen here while testing a 5 node Kafka cluster with a 3 node ZK ensemble on Kubernetes on AWS. After running for a while broker 2 started showing the "Cached zkVersion [29] not equal to that in zookeeper, skip updating ISR" error message for al the partitions it leads. For those partition it is the only in sync replica. That has led to the Samza jobs I was running to stop. I should note that I am running 0.9.0.0.
          Hide
          wushujames James Cheng added a comment -

          We ran into the same issue today, when running 0.9.0.0.

          [2016-02-17 22:49:52,638] INFO Partition [the.topic.name,22] on broker 2: Cached zkVersion [5] not equal to that in zookeeper, skip updating ISR (kafka.cluster.Partition)

          Show
          wushujames James Cheng added a comment - We ran into the same issue today, when running 0.9.0.0. [2016-02-17 22:49:52,638] INFO Partition [the.topic.name,22] on broker 2: Cached zkVersion [5] not equal to that in zookeeper, skip updating ISR (kafka.cluster.Partition)
          Hide
          akheron Petri Lehtinen added a comment -

          This happened to me (again) a few days ago on 0.9.0.0 on a cluster of 2 kafka nodes.

          Show
          akheron Petri Lehtinen added a comment - This happened to me (again) a few days ago on 0.9.0.0 on a cluster of 2 kafka nodes.
          Hide
          michal.harish Michal Harish added a comment -

          Hit this on Kafka 0.8.2.2 as well

          Show
          michal.harish Michal Harish added a comment - Hit this on Kafka 0.8.2.2 as well
          Hide
          KaneK Kane Kim added a comment -

          Same problem with the same symptoms occurred on kafka 0.8.2.1. After network glitch brokers fall out of ISR set with
          Cached zkVersion [5] not equal to that in zookeeper, skip updating ISR

          Broker never recovers from this state until restart.

          Show
          KaneK Kane Kim added a comment - Same problem with the same symptoms occurred on kafka 0.8.2.1. After network glitch brokers fall out of ISR set with Cached zkVersion [5] not equal to that in zookeeper, skip updating ISR Broker never recovers from this state until restart.
          Hide
          Srdo Stig Rohde Døssing added a comment -

          We hit this on 0.9.0.1 today

          [2016-04-28 19:18:22,834] INFO Partition [dce-data,13] on broker 3: Shrinking ISR for partition [dce-data,13] from 3,2 to 3 (kafka.cluster.Partition)
          [2016-04-28 19:18:22,845] INFO Partition [dce-data,13] on broker 3: Cached zkVersion [304] not equal to that in zookeeper, skip updating ISR (kafka.cluster.Partition)
          [2016-04-28 19:18:32,785] INFO Partition [dce-data,16] on broker 3: Shrinking ISR for partition [dce-data,16] from 3,2 to 3 (kafka.cluster.Partition)
          [2016-04-28 19:18:32,803] INFO Partition [dce-data,16] on broker 3: Cached zkVersion [312] not equal to that in zookeeper, skip updating ISR (kafka.cluster.Partition)
          

          which continued until we rebooted broker 3. The ISR at this time in Zookeeper had only broker 2, and there was no leader for the affected partitions. I believe the preferred leader for these partitions was 3.

          Show
          Srdo Stig Rohde Døssing added a comment - We hit this on 0.9.0.1 today [2016-04-28 19:18:22,834] INFO Partition [dce-data,13] on broker 3: Shrinking ISR for partition [dce-data,13] from 3,2 to 3 (kafka.cluster.Partition) [2016-04-28 19:18:22,845] INFO Partition [dce-data,13] on broker 3: Cached zkVersion [304] not equal to that in zookeeper, skip updating ISR (kafka.cluster.Partition) [2016-04-28 19:18:32,785] INFO Partition [dce-data,16] on broker 3: Shrinking ISR for partition [dce-data,16] from 3,2 to 3 (kafka.cluster.Partition) [2016-04-28 19:18:32,803] INFO Partition [dce-data,16] on broker 3: Cached zkVersion [312] not equal to that in zookeeper, skip updating ISR (kafka.cluster.Partition) which continued until we rebooted broker 3. The ISR at this time in Zookeeper had only broker 2, and there was no leader for the affected partitions. I believe the preferred leader for these partitions was 3.
          Hide
          jpfaff Joel Pfaff added a comment -

          We have hit that as well on 0.9.0.1 today, same logs, and only a reboot of the faulty broker recovered the problem.

          Show
          jpfaff Joel Pfaff added a comment - We have hit that as well on 0.9.0.1 today, same logs, and only a reboot of the faulty broker recovered the problem.
          Hide
          crodier Chris Rodier added a comment -

          We also observed this identical issue, on 0.9.0.1 today. Restart of the failed broker resolved the issue without difficulty as a work around. This seems like a high priority issue where you could lose nodes, and/or lose a cluster fairly easily due to zookeeper instability / elections.

          Show
          crodier Chris Rodier added a comment - We also observed this identical issue, on 0.9.0.1 today. Restart of the failed broker resolved the issue without difficulty as a work around. This seems like a high priority issue where you could lose nodes, and/or lose a cluster fairly easily due to zookeeper instability / elections.
          Hide
          tbischel Tyler Bischel added a comment -

          We are also seeing this issue in 0.10.0.0 pretty much daily right now.

          [2016-07-13 21:30:50,170]  1292384 [kafka-scheduler-0] INFO  kafka.cluster.Partition  - Partition [events,580] on broker 10432234: Cached zkVersion [1267] not equal to that in zookeeper, skip updating ISR
          
          Show
          tbischel Tyler Bischel added a comment - We are also seeing this issue in 0.10.0.0 pretty much daily right now. [2016-07-13 21:30:50,170] 1292384 [kafka-scheduler-0] INFO kafka.cluster.Partition - Partition [events,580] on broker 10432234: Cached zkVersion [1267] not equal to that in zookeeper, skip updating ISR
          Hide
          williamyu William Yu added a comment -

          We are also seeing this in our production cluster: Running on Kafka: 0.9.0.1

          Is restarting the only solution?

          [2016-07-27 14:36:15,807] INFO Partition [tasks,265] on broker 4: Cached zkVersion [182] not equal to that in zookeeper, skip updating ISR (kafka.cluster.Partition)
          [2016-07-27 14:36:15,807] INFO Partition [tasks,150] on broker 4: Shrinking ISR for partition [tasks,150] from 6,4,7 to 4 (kafka.cluster.Partition)
          
          Show
          williamyu William Yu added a comment - We are also seeing this in our production cluster: Running on Kafka: 0.9.0.1 Is restarting the only solution? [2016-07-27 14:36:15,807] INFO Partition [tasks,265] on broker 4: Cached zkVersion [182] not equal to that in zookeeper, skip updating ISR (kafka.cluster.Partition) [2016-07-27 14:36:15,807] INFO Partition [tasks,150] on broker 4: Shrinking ISR for partition [tasks,150] from 6,4,7 to 4 (kafka.cluster.Partition)
          Hide
          kipz James Carnegie added a comment -

          That's our experience, though the only other thing we've tried is leaving it for a while.

          Show
          kipz James Carnegie added a comment - That's our experience, though the only other thing we've tried is leaving it for a while.
          Hide
          KaneK Kane Kim added a comment -

          This is hitting us badly on kafka 0.10.0.0, it doesn't recover by itself for several days at least.

          Show
          KaneK Kane Kim added a comment - This is hitting us badly on kafka 0.10.0.0, it doesn't recover by itself for several days at least.
          Hide
          kzadorozhny-tubemogul Konstantin Zadorozhny added a comment -

          Seeing the same issue in our staging and production environments on 0.9.0.1. Bouncing brokers helps, but still not ideal.

          Staging cluster were left to "recover" for a day. Didn't happen.

          Show
          kzadorozhny-tubemogul Konstantin Zadorozhny added a comment - Seeing the same issue in our staging and production environments on 0.9.0.1. Bouncing brokers helps, but still not ideal. Staging cluster were left to "recover" for a day. Didn't happen.
          Hide
          joshua.dickerson@dealer.com Joshua Dickerson added a comment -

          This has bit us twice in our live environment. 0.9.0.1
          Restarting the affected broker(s) is the only thing that seems to fix it.

          Show
          joshua.dickerson@dealer.com Joshua Dickerson added a comment - This has bit us twice in our live environment. 0.9.0.1 Restarting the affected broker(s) is the only thing that seems to fix it.
          Hide
          skidank Michael Sandrof added a comment -

          We seem to be having a similar problem running 0.10.0.0. However no amount of broker restarting corrects the problem. Once it happens, I see periodic "Cached zkVersion" messages along with complete instability in the ISRs. Continuous shrinking and expanding of the ISRs that makes the cluster unusable as we need 2 ISRs for our durability requirements.

          The only thing that fixes the problem is to delete all topics, recreate and reload. This isn't a practical approach for our production system in which we are using Kafka as a transactionally consistent replica of a relational database.

          Anyone have any clues about how to prevent this from happening?

          Show
          skidank Michael Sandrof added a comment - We seem to be having a similar problem running 0.10.0.0. However no amount of broker restarting corrects the problem. Once it happens, I see periodic "Cached zkVersion" messages along with complete instability in the ISRs. Continuous shrinking and expanding of the ISRs that makes the cluster unusable as we need 2 ISRs for our durability requirements. The only thing that fixes the problem is to delete all topics, recreate and reload. This isn't a practical approach for our production system in which we are using Kafka as a transactionally consistent replica of a relational database. Anyone have any clues about how to prevent this from happening?
          Hide
          KaneK Kane Kim added a comment -

          For us the reason was high percentage of lost packets to one of ZK nodes (from broker to ZK). After we fixed that, situation got a lot better.

          Show
          KaneK Kane Kim added a comment - For us the reason was high percentage of lost packets to one of ZK nodes (from broker to ZK). After we fixed that, situation got a lot better.
          Hide
          cmolter Charly Molter added a comment - - edited

          Hi,

          We had this issue on a test cluster running 0.10.0.0 so I took time to investigate some more.

          We had a bunch of disconnections to Zookeeper and we had 2 changes of controller in a short time.

          Broker 103 was controller with epoch 44
          Broker 104 was controller with epoch 45

          I looked at one specific partitions and found the following pattern:

          101 was the broker which thought was leader but kept failing shrink the ISR with:
          Partition [verifiable-test-topic,0] on broker 101: Shrinking ISR for partition [verifiable-test-topic,0] from 101,301,201 to 101,201
          Partition [verifiable-test-topic,0] on broker 101: Cached zkVersion [185] not equal to that in zookeeper, skip updating ISR

          Looking at ZK we have:
          get /brokers/topics/verifiable-test-topic/partitions/0/state

          {"controller_epoch":44,"leader":301,"version":1,"leader_epoch":96,"isr":[301]}

          And metadata (to a random broker) is saying:
          Topic: verifiable-test-topic Partition: 0 Leader: 301 Replicas: 101,201,301 Isr: 301

          Digging in the logs here’s what we think happened:

          1. 103 sends becomeFollower to 301 with epoch 44 and leaderEpoch 95
          2. 104 sends becomeLeader to 101 with epoch 45 and leaderEpoch 95 (after update zk!)
          3. 103 sends becomeLeader to 301 with epoch 44 and leaderEpoch 96 (after updating zk!)
          4. 104 sends becomeFollower to 301 with epoch 45 and leaderEpoch 95

          4) Is ignored by 301 as the leaderEpoch is older than the current one.

          We are missing a request: 103 sends becomeFollower to 101 with epoch 44 and leaderEpoch 95

          I believe this happened because when the controller steps down it empties its request queue so this request never left the controller: https://github.com/apache/kafka/blob/trunk/core/src/main/scala/kafka/controller/ControllerChannelManager.scala#L53-L57

          So we ended up in a case where 301 and 101 think they are both leaders. Obviously 101 wants to update the state in ZK to remove 301 as it’s not even fetching from 101.

          Does this seem correct to you?

          It seems impossible to avoid having no Controller overlap, which could make it quite hard to avoid having 2 leaders for a short time. Though there should be a way for this situation to get back to a good state.

          I believe the impact of this would be:

          • writes = -1 unavailability
          • writes != -1 possible log divergence (I’m unsure about this).

          Hope this helps. While I had to fix the cluster by bouncing a node I kept most of the logs so let me know if you need more info.

          Show
          cmolter Charly Molter added a comment - - edited Hi, We had this issue on a test cluster running 0.10.0.0 so I took time to investigate some more. We had a bunch of disconnections to Zookeeper and we had 2 changes of controller in a short time. Broker 103 was controller with epoch 44 Broker 104 was controller with epoch 45 I looked at one specific partitions and found the following pattern: 101 was the broker which thought was leader but kept failing shrink the ISR with: Partition [verifiable-test-topic,0] on broker 101: Shrinking ISR for partition [verifiable-test-topic,0] from 101,301,201 to 101,201 Partition [verifiable-test-topic,0] on broker 101: Cached zkVersion [185] not equal to that in zookeeper, skip updating ISR Looking at ZK we have: get /brokers/topics/verifiable-test-topic/partitions/0/state {"controller_epoch":44,"leader":301,"version":1,"leader_epoch":96,"isr":[301]} And metadata (to a random broker) is saying: Topic: verifiable-test-topic Partition: 0 Leader: 301 Replicas: 101,201,301 Isr: 301 Digging in the logs here’s what we think happened: 1. 103 sends becomeFollower to 301 with epoch 44 and leaderEpoch 95 2. 104 sends becomeLeader to 101 with epoch 45 and leaderEpoch 95 (after update zk!) 3. 103 sends becomeLeader to 301 with epoch 44 and leaderEpoch 96 (after updating zk!) 4. 104 sends becomeFollower to 301 with epoch 45 and leaderEpoch 95 4) Is ignored by 301 as the leaderEpoch is older than the current one. We are missing a request: 103 sends becomeFollower to 101 with epoch 44 and leaderEpoch 95 I believe this happened because when the controller steps down it empties its request queue so this request never left the controller: https://github.com/apache/kafka/blob/trunk/core/src/main/scala/kafka/controller/ControllerChannelManager.scala#L53-L57 So we ended up in a case where 301 and 101 think they are both leaders. Obviously 101 wants to update the state in ZK to remove 301 as it’s not even fetching from 101. Does this seem correct to you? It seems impossible to avoid having no Controller overlap, which could make it quite hard to avoid having 2 leaders for a short time. Though there should be a way for this situation to get back to a good state. I believe the impact of this would be: writes = -1 unavailability writes != -1 possible log divergence (I’m unsure about this). Hope this helps. While I had to fix the cluster by bouncing a node I kept most of the logs so let me know if you need more info.
          Hide
          derek@chen-becker.org derek added a comment -

          I'm on 0.10.1.0 and seeing the same thing. Maybe related to Charly Molter is saying above, what we see in the logs just prior to a broker becoming under-replicated is a flurry of

          org.apache.kafka.common.errors.NotLeaderForPartitionException: This server is not the leader for that topic-partition.
          

          messages. After that we see a bunch of activity around adding and removing fetchers, then it goes into the infinite ISR shrink loop. The only way we can recover is to restart.

          Show
          derek@chen-becker.org derek added a comment - I'm on 0.10.1.0 and seeing the same thing. Maybe related to Charly Molter is saying above, what we see in the logs just prior to a broker becoming under-replicated is a flurry of org.apache.kafka.common.errors.NotLeaderForPartitionException: This server is not the leader for that topic-partition. messages. After that we see a bunch of activity around adding and removing fetchers, then it goes into the infinite ISR shrink loop. The only way we can recover is to restart.
          Hide
          sini Sinóros-Szabó Péter added a comment -

          Same issue on 0.10.1.1. Do you need logs? I can collect them next time I see this.

          Show
          sini Sinóros-Szabó Péter added a comment - Same issue on 0.10.1.1. Do you need logs? I can collect them next time I see this.
          Hide
          peoplemerge Dave Thomas added a comment -

          Same with us, on 0.10.1.1 (following upgrade from 0.10.1.0 where we saw the same issue).

          Show
          peoplemerge Dave Thomas added a comment - Same with us, on 0.10.1.1 (following upgrade from 0.10.1.0 where we saw the same issue).
          Hide
          sini Sinóros-Szabó Péter added a comment -

          Do you have any plan to resolve this? Or is there a workaround for this issue?

          Show
          sini Sinóros-Szabó Péter added a comment - Do you have any plan to resolve this? Or is there a workaround for this issue?
          Hide
          prasincs Prasanna Gautam added a comment - - edited

          This is still replicable in Kafka 0.10.1.1 when Kafka brokers are partitioned from each other and zookeeper gets disconnected from the brokers briefly and comes back. This situation leads to brokers getting stuck in comparing Cached zkVersion and unable to expand the ISR.

          The code in Partition.scala does not seem to be handling enough error conditions other than the stale zkVersion. In addition to skipping in the current loop, I think it should reconnect to zookeeper to update the current state and version.

          Here's a suggestion to do this.. doing it asynchronously doesn't break the existing flow and you can update the state. ZkVersion may not be the only thing to update here.

          val newLeaderAndIsr = new LeaderAndIsr(localBrokerId, leaderEpoch, newIsr.map(r => r.brokerId).toList, zkVersion)
              val (updateSucceeded,newVersion) = ReplicationUtils.updateLeaderAndIsr(zkUtils, topic, partitionId,
                newLeaderAndIsr, controllerEpoch, zkVersion)
          
              if(updateSucceeded) {
                replicaManager.recordIsrChange(new TopicAndPartition(topic, partitionId))
                inSyncReplicas = newIsr
                zkVersion = newVersion
                trace("ISR updated to [%s] and zkVersion updated to [%d]".format(newIsr.mkString(","), zkVersion))
              } else {
                info("Cached zkVersion [%d] not equal to that in zookeeper, skip updating ISR".format(zkVersion))
                zkVersion = asyncUpdateTopicPartitionVersion(topic,partitionId)
              }
          
          Show
          prasincs Prasanna Gautam added a comment - - edited This is still replicable in Kafka 0.10.1.1 when Kafka brokers are partitioned from each other and zookeeper gets disconnected from the brokers briefly and comes back. This situation leads to brokers getting stuck in comparing Cached zkVersion and unable to expand the ISR. The code in Partition.scala does not seem to be handling enough error conditions other than the stale zkVersion. In addition to skipping in the current loop, I think it should reconnect to zookeeper to update the current state and version. Here's a suggestion to do this.. doing it asynchronously doesn't break the existing flow and you can update the state. ZkVersion may not be the only thing to update here. val newLeaderAndIsr = new LeaderAndIsr(localBrokerId, leaderEpoch, newIsr.map(r => r.brokerId).toList, zkVersion) val (updateSucceeded,newVersion) = ReplicationUtils.updateLeaderAndIsr(zkUtils, topic, partitionId, newLeaderAndIsr, controllerEpoch, zkVersion) if (updateSucceeded) { replicaManager.recordIsrChange( new TopicAndPartition(topic, partitionId)) inSyncReplicas = newIsr zkVersion = newVersion trace( "ISR updated to [%s] and zkVersion updated to [%d]" .format(newIsr.mkString( "," ), zkVersion)) } else { info( "Cached zkVersion [%d] not equal to that in zookeeper, skip updating ISR" .format(zkVersion)) zkVersion = asyncUpdateTopicPartitionVersion(topic,partitionId) }
          Hide
          elevy Elias Levy added a comment -

          Hit this again during testing with 0.10.0.1 on a 10 node broker cluster with a 3 node ZK ensemble. This should have priority Blocker instead of Major.

          Show
          elevy Elias Levy added a comment - Hit this again during testing with 0.10.0.1 on a 10 node broker cluster with a 3 node ZK ensemble. This should have priority Blocker instead of Major.
          Hide
          klesta490 Vladimír Kleštinec added a comment -

          Elias Levy Agree, we are experiencing same issue, this is real blocker and we are loosing trust in Kafka...

          Show
          klesta490 Vladimír Kleštinec added a comment - Elias Levy Agree, we are experiencing same issue, this is real blocker and we are loosing trust in Kafka...
          Hide
          prateekjaipuria Prateek Jaipuria added a comment -

          Having the same issue with 0.10.1.0 on a 8 node cluster. Restarting node also does not help, the problem just moves on to another node. This is becoming a deal breaker. Definitely losing trust in Kafka. Definitely a BLOCKER!

          Show
          prateekjaipuria Prateek Jaipuria added a comment - Having the same issue with 0.10.1.0 on a 8 node cluster. Restarting node also does not help, the problem just moves on to another node. This is becoming a deal breaker. Definitely losing trust in Kafka. Definitely a BLOCKER!
          Hide
          granthenke Grant Henke added a comment -

          I am curious if everyone on this Jira is actually seeing the reported issue. I have had multiple cases where someone presented my with an environment they thought was experiencing this issue. After researching the environment and logs, to date it has always been something else.

          The main culprits so far have been:

          • Long GC pauses causing zookeeper sessions to timeout
          • Slow or poorly configured zookeeper
          • Bad network configuration

          All of the above resulted in a soft reoccurring failure of brokers. That churn often caused addition load perpetuating the issue.

          If you are seeing this issue do you see the following pattern repeating in the logs?:

          INFO org.I0Itec.zkclient.ZkClient: zookeeper state changed (Disconnected)
          ...
          INFO org.I0Itec.zkclient.ZkClient: zookeeper state changed (Expired)
          INFO org.apache.zookeeper.ClientCnxn: Unable to reconnect to ZooKeeper service, session 0x153ab38abdbd360 has expired, closing socket connection
          ...
          INFO org.I0Itec.zkclient.ZkClient: zookeeper state changed (SyncConnected)
          INFO kafka.server.KafkaHealthcheck: re-registering broker info in ZK for broker 32
          INFO kafka.utils.ZKCheckedEphemeral: Creating /brokers/ids/32 (is it secure? false)
          INFO kafka.utils.ZKCheckedEphemeral: Result of znode creation is: OK
          

          If so, something is causing communication with zookeeper to take too long and the broker is unregistering itself. This will cause ISRs to shrink and expand over and over again.

          I don't think this will solve everyones issue here, but hopefully it will help solve some.

          Show
          granthenke Grant Henke added a comment - I am curious if everyone on this Jira is actually seeing the reported issue. I have had multiple cases where someone presented my with an environment they thought was experiencing this issue. After researching the environment and logs, to date it has always been something else. The main culprits so far have been: Long GC pauses causing zookeeper sessions to timeout Slow or poorly configured zookeeper Bad network configuration All of the above resulted in a soft reoccurring failure of brokers. That churn often caused addition load perpetuating the issue. If you are seeing this issue do you see the following pattern repeating in the logs?: INFO org.I0Itec.zkclient.ZkClient: zookeeper state changed (Disconnected) ... INFO org.I0Itec.zkclient.ZkClient: zookeeper state changed (Expired) INFO org.apache.zookeeper.ClientCnxn: Unable to reconnect to ZooKeeper service, session 0x153ab38abdbd360 has expired, closing socket connection ... INFO org.I0Itec.zkclient.ZkClient: zookeeper state changed (SyncConnected) INFO kafka.server.KafkaHealthcheck: re-registering broker info in ZK for broker 32 INFO kafka.utils.ZKCheckedEphemeral: Creating /brokers/ids/32 (is it secure? false) INFO kafka.utils.ZKCheckedEphemeral: Result of znode creation is: OK If so, something is causing communication with zookeeper to take too long and the broker is unregistering itself. This will cause ISRs to shrink and expand over and over again. I don't think this will solve everyones issue here, but hopefully it will help solve some.
          Hide
          peoplemerge Dave Thomas added a comment -

          Grant Henke We don't see brokers recovering. The message we see is:

          Cached zkVersion [xxx] not equal to that in zookeeper, skip updating ISR
          
          Show
          peoplemerge Dave Thomas added a comment - Grant Henke We don't see brokers recovering. The message we see is: Cached zkVersion [xxx] not equal to that in zookeeper, skip updating ISR
          Hide
          prateekjaipuria Prateek Jaipuria added a comment -

          Grant Henke We don't see any zookeeper disconnections.

          Just

          INFO Partition [topic,n] on broker m: Cached zkVersion [xxx] not equal to that in zookeeper, skip updating ISR (kafka.cluster.Partition)
          
          Show
          prateekjaipuria Prateek Jaipuria added a comment - Grant Henke We don't see any zookeeper disconnections. Just INFO Partition [topic,n] on broker m: Cached zkVersion [xxx] not equal to that in zookeeper, skip updating ISR (kafka.cluster.Partition)
          Hide
          KaneK Kane Kim added a comment -

          In my opinion it doesn't matter what's causing it (in our case that was indeed lost packets to zookeeper), the culprit is that brokers will not recover by itself until rolling restart. This is real problem and has to be fixed.

          Show
          KaneK Kane Kim added a comment - In my opinion it doesn't matter what's causing it (in our case that was indeed lost packets to zookeeper), the culprit is that brokers will not recover by itself until rolling restart. This is real problem and has to be fixed.
          Hide
          junrao Jun Rao added a comment -

          Sorry to hear about the impact to production. Grant mentioned ZK session expiration, which is indeed a potential cause of this issue. A related issue has been reported in KAFKA-3083. The issue is that when the controller's ZK session expires and loses its controller-ship, it's possible for this zombie controller to continue updating ZK and/or sending LeaderAndIsrRequests to the brokers for a short period of time. When this happens, the broker may not have the most up-to-date information about leader and isr, which can lead to subsequent ZK failure when isr needs to be updated.

          Fixing this issue requires us change the way how we use the ZK api and may take some time. In the interim, one suggestion is to make sure ZK session expiration never happens. This can be achieved by making sure that (1) ZK servers are performing, (2) the brokers don't have long GCs, (3) the ZK session expiration time is large enough.

          Show
          junrao Jun Rao added a comment - Sorry to hear about the impact to production. Grant mentioned ZK session expiration, which is indeed a potential cause of this issue. A related issue has been reported in KAFKA-3083 . The issue is that when the controller's ZK session expires and loses its controller-ship, it's possible for this zombie controller to continue updating ZK and/or sending LeaderAndIsrRequests to the brokers for a short period of time. When this happens, the broker may not have the most up-to-date information about leader and isr, which can lead to subsequent ZK failure when isr needs to be updated. Fixing this issue requires us change the way how we use the ZK api and may take some time. In the interim, one suggestion is to make sure ZK session expiration never happens. This can be achieved by making sure that (1) ZK servers are performing, (2) the brokers don't have long GCs, (3) the ZK session expiration time is large enough.
          Hide
          prasincs Prasanna Gautam added a comment -

          Jun Rao Thanks for looking into this. Do you mind elaborating on what you need to change in the ZK api and whether https://issues.apache.org/jira/browse/KAFKA-3083 is going to solve it. The issue here is that – in case of network partitions which are unrelated to the 3 points and can happen at any time, this can happen and leave the brokers in messed up state until a restart. Can this be fixed by handling the ZK Connection errors?

          If restarting the broker is the only fix, maybe the proper thing to do is to crash and let supervisor, etc. restart the service?

          Show
          prasincs Prasanna Gautam added a comment - Jun Rao Thanks for looking into this. Do you mind elaborating on what you need to change in the ZK api and whether https://issues.apache.org/jira/browse/KAFKA-3083 is going to solve it. The issue here is that – in case of network partitions which are unrelated to the 3 points and can happen at any time, this can happen and leave the brokers in messed up state until a restart. Can this be fixed by handling the ZK Connection errors? If restarting the broker is the only fix, maybe the proper thing to do is to crash and let supervisor, etc. restart the service?
          Hide
          junrao Jun Rao added a comment -

          Prasanna Gautam, if the controller is partitioned off other brokers and ZK, the expected flow is the following: (1) ZK server detects that the old controller's session expires; (2) the controller path is removed by ZK; (3) a new controller is elected and changes leaders/isrs; (4) network is back on the old controller; (5) old controller receives ZK session expiration event; (6) old controller stops doing the controller stuff and resign. Note that the old controller doesn't really know that it's no longer the controller until step (5). The gap we have now is that step (6) is not done in a timely fashion.

          Are you deploying Kafka in the same data center? What kind of network partitions are you seeing? Typically, we expect network partitions are rare within the same data center. If there are short network glitches, one temporary fix is to increase the ZK session timeout to accommodate for that until the network issue is fixed.

          Show
          junrao Jun Rao added a comment - Prasanna Gautam , if the controller is partitioned off other brokers and ZK, the expected flow is the following: (1) ZK server detects that the old controller's session expires; (2) the controller path is removed by ZK; (3) a new controller is elected and changes leaders/isrs; (4) network is back on the old controller; (5) old controller receives ZK session expiration event; (6) old controller stops doing the controller stuff and resign. Note that the old controller doesn't really know that it's no longer the controller until step (5). The gap we have now is that step (6) is not done in a timely fashion. Are you deploying Kafka in the same data center? What kind of network partitions are you seeing? Typically, we expect network partitions are rare within the same data center. If there are short network glitches, one temporary fix is to increase the ZK session timeout to accommodate for that until the network issue is fixed.
          Hide
          mjuarez mjuarez added a comment -

          We are also running into this problem in our staging cluster, running Kafka 0.10.0.1. Basically it looks like this happened yesterday:

          [2017-02-28 18:41:33,513] INFO Client session timed out, have not heard from server in 7799ms for sessionid 0x159d7893eab0088, closing socket connection and attempting reconnect (org.apache.zookeeper.ClientCnxn)
          

          I'm attributing that to a transient network issue, since we haven't seen any other issues. And less than a minute later, we started seeing these errors:

          [2017-02-28 18:42:45,739] INFO Partition [analyticsInfrastructure_KafkaAvroUserMessage,16] on broker 101: Shrinking ISR for partition [analyticsInfrastructure_KafkaAvroUserMessage,16] from 102,101,105 to 101 (kaf
          [2017-02-28 18:42:45,751] INFO Partition [analyticsInfrastructure_KafkaAvroUserMessage,16] on broker 101: Cached zkVersion [94] not equal to that in zookeeper, skip updating ISR (kafka.cluster.Partition)
          [2017-02-28 18:42:45,751] INFO Partition [qa_exporter11_slingshot_salesforce_invoice,6] on broker 101: Shrinking ISR for partition [qa_exporter11_slingshot_salesforce_invoice,6] from 101,105,104 to 101 (kafka.clu
          [2017-02-28 18:42:45,756] INFO Partition [qa_exporter11_slingshot_salesforce_invoice,6] on broker 101: Cached zkVersion [237] not equal to that in zookeeper, skip updating ISR (kafka.cluster.Partition)
          [2017-02-28 18:42:45,756] INFO Partition [GNRDEV_counters_singleCount,2] on broker 101: Shrinking ISR for partition [GNRDEV_counters_singleCount,2] from 101,105,104 to 101 (kafka.cluster.Partition)
          [2017-02-28 18:42:45,761] INFO Partition [GNRDEV_counters_singleCount,2] on broker 101: Cached zkVersion [334] not equal to that in zookeeper, skip updating ISR (kafka.cluster.Partition)
          [2017-02-28 18:42:45,761] INFO Partition [sod-spins-spark-local,1] on broker 101: Shrinking ISR for partition [sod-spins-spark-local,1] from 101,103,104 to 101 (kafka.cluster.Partition)
          [2017-02-28 18:42:45,764] INFO Partition [sod-spins-spark-local,1] on broker 101: Cached zkVersion [379] not equal to that in zookeeper, skip updating ISR (kafka.cluster.Partition)
          [2017-02-28 18:42:45,764] INFO Partition [sod-spins-spark-local,11] on broker 101: Shrinking ISR for partition [sod-spins-spark-local,11] from 102,101,105 to 101 (kafka.cluster.Partition)
          [2017-02-28 18:42:45,767] INFO Partition [sod-spins-spark-local,11] on broker 101: Cached zkVersion [237] not equal to that in zookeeper, skip updating ISR (kafka.cluster.Partition)
          

          The "current" server is 101. So it thinks it's the leader for basically every partition on that node, but it's refusing to update the ISRs, because the cached zkversion doesn't match the one in zookeeper. This is causing permanently under-replicated partitions, because server doesn't ever catch up, since it doesn't think there's a problem. Also, the metadata reported by the 101 server to consumers indicates it thinks it's part of the ISR, but every other broker doesn't think so.

          Let me know if more logs/details would be helpful. I'll try to fix this by restarting the node, and hopefully it fixes the issue.

          Show
          mjuarez mjuarez added a comment - We are also running into this problem in our staging cluster, running Kafka 0.10.0.1. Basically it looks like this happened yesterday: [2017-02-28 18:41:33,513] INFO Client session timed out, have not heard from server in 7799ms for sessionid 0x159d7893eab0088, closing socket connection and attempting reconnect (org.apache.zookeeper.ClientCnxn) I'm attributing that to a transient network issue, since we haven't seen any other issues. And less than a minute later, we started seeing these errors: [2017-02-28 18:42:45,739] INFO Partition [analyticsInfrastructure_KafkaAvroUserMessage,16] on broker 101: Shrinking ISR for partition [analyticsInfrastructure_KafkaAvroUserMessage,16] from 102,101,105 to 101 (kaf [2017-02-28 18:42:45,751] INFO Partition [analyticsInfrastructure_KafkaAvroUserMessage,16] on broker 101: Cached zkVersion [94] not equal to that in zookeeper, skip updating ISR (kafka.cluster.Partition) [2017-02-28 18:42:45,751] INFO Partition [qa_exporter11_slingshot_salesforce_invoice,6] on broker 101: Shrinking ISR for partition [qa_exporter11_slingshot_salesforce_invoice,6] from 101,105,104 to 101 (kafka.clu [2017-02-28 18:42:45,756] INFO Partition [qa_exporter11_slingshot_salesforce_invoice,6] on broker 101: Cached zkVersion [237] not equal to that in zookeeper, skip updating ISR (kafka.cluster.Partition) [2017-02-28 18:42:45,756] INFO Partition [GNRDEV_counters_singleCount,2] on broker 101: Shrinking ISR for partition [GNRDEV_counters_singleCount,2] from 101,105,104 to 101 (kafka.cluster.Partition) [2017-02-28 18:42:45,761] INFO Partition [GNRDEV_counters_singleCount,2] on broker 101: Cached zkVersion [334] not equal to that in zookeeper, skip updating ISR (kafka.cluster.Partition) [2017-02-28 18:42:45,761] INFO Partition [sod-spins-spark-local,1] on broker 101: Shrinking ISR for partition [sod-spins-spark-local,1] from 101,103,104 to 101 (kafka.cluster.Partition) [2017-02-28 18:42:45,764] INFO Partition [sod-spins-spark-local,1] on broker 101: Cached zkVersion [379] not equal to that in zookeeper, skip updating ISR (kafka.cluster.Partition) [2017-02-28 18:42:45,764] INFO Partition [sod-spins-spark-local,11] on broker 101: Shrinking ISR for partition [sod-spins-spark-local,11] from 102,101,105 to 101 (kafka.cluster.Partition) [2017-02-28 18:42:45,767] INFO Partition [sod-spins-spark-local,11] on broker 101: Cached zkVersion [237] not equal to that in zookeeper, skip updating ISR (kafka.cluster.Partition) The "current" server is 101. So it thinks it's the leader for basically every partition on that node, but it's refusing to update the ISRs, because the cached zkversion doesn't match the one in zookeeper. This is causing permanently under-replicated partitions, because server doesn't ever catch up, since it doesn't think there's a problem. Also, the metadata reported by the 101 server to consumers indicates it thinks it's part of the ISR, but every other broker doesn't think so. Let me know if more logs/details would be helpful. I'll try to fix this by restarting the node, and hopefully it fixes the issue.
          Hide
          junrao Jun Rao added a comment -

          mjuarez, did you see ZK session expiration in the server.log in the controller around that time? The log will look like the following.

          INFO zookeeper state changed (Expired) (org.I0Itec.zkclient.ZkClient)

          Show
          junrao Jun Rao added a comment - mjuarez , did you see ZK session expiration in the server.log in the controller around that time? The log will look like the following. INFO zookeeper state changed (Expired) (org.I0Itec.zkclient.ZkClient)
          Hide
          Ronghua Lin Ronghua Lin added a comment -

          Jun Rao, we also have this problem in a small cluster which has 3 brokers, running Kafka 0.10.1.1. When it happened, the logs of each broker look like this:

          broker 2
          [2017-03-20 01:03:48,903] INFO [Group Metadata Manager on Broker 2]: Removed 0 expired offsets in 0 milliseconds. (kafka.coordinator.GroupMetadataManager)
          [2017-03-20 01:13:27,283] INFO Creating /controller (is it secure? false) (kafka.utils.ZKCheckedEphemeral)
          [2017-03-20 01:13:27,293] INFO Result of znode creation is: OK (kafka.utils.ZKCheckedEphemeral)
          [2017-03-20 01:13:27,294] INFO 2 successfully elected as leader (kafka.server.ZookeeperLeaderElector)
          [2017-03-20 01:13:28,203] INFO re-registering broker info in ZK for broker 2 (kafka.server.KafkaHealthcheck$SessionExpireListener)
          [2017-03-20 01:13:28,205] INFO Creating /brokers/ids/2 (is it secure? false) (kafka.utils.ZKCheckedEphemeral)
          [2017-03-20 01:13:28,218] INFO Result of znode creation is: OK (kafka.utils.ZKCheckedEphemeral)
          [2017-03-20 01:13:28,219] INFO Registered broker 2 at path /brokers/ids/2 with addresses: PLAINTEXT -> EndPoint(xxxxx, xxxx,PLAINTEXT) (kafka.utils.ZkUtils)
          [2017-03-20 01:13:28,219] INFO done re-registering broker (kafka.server.KafkaHealthcheck$SessionExpireListener)
          [2017-03-20 01:13:28,220] INFO Subscribing to /brokers/topics path to watch for new topics (kafka.server.KafkaHealthcheck$SessionExpireListener)
          [2017-03-20 01:13:28,224] INFO New leader is 2 (kafka.server.ZookeeperLeaderElector$LeaderChangeListener)
          [2017-03-20 01:13:28,227] INFO New leader is 2 (kafka.server.ZookeeperLeaderElector$LeaderChangeListener)
          [2017-03-20 01:13:38,812] INFO Partition [topic1,1] on broker 2: Shrinking ISR for partition [topic1,1] from 0,2,1 to 2,1 (kafka.cluster.Partition)
          [2017-03-20 01:13:38,825] INFO Partition [topic1,1] on broker 2: Cached zkVersion [6] not equal to that in zookeeper, skip updating ISR (kafka.cluster.Partition)
          [2017-03-20 01:13:38,825] INFO Partition [topic2,1] on broker 2: Shrinking ISR for partition [topic2,1] from 0,2,1 to 2,1 (kafka.cluster.Partition)
          [2017-03-20 01:13:38,835] INFO Partition [topic2,1] on broker 2: Cached zkVersion [6] not equal to that in zookeeper, skip updating ISR (kafka.cluster.Partition)
          [2017-03-20 01:13:38,835] INFO Partition [topic3,0] on broker 2: Shrinking ISR for partition [topic3,0] from 0,2,1 to 2,1 (kafka.cluster.Partition)
          [2017-03-20 01:13:38,847] INFO Partition [topic3,0] on broker 2: Cached zkVersion [6] not equal to that in zookeeper, skip updating ISR (kafka.cluster.Partition)
          ....
          
          broker 1
          [2017-03-20 01:03:38,255] INFO [Group Metadata Manager on Broker 1]: Removed 0 expired offsets in 0 milliseconds. (kafka.coordinator.GroupMetadataManager)
          [2017-03-20 01:13:27,451] INFO New leader is 2 (kafka.server.ZookeeperLeaderElector$LeaderChangeListener)
          [2017-03-20 01:13:27,490] INFO re-registering broker info in ZK for broker 1 (kafka.server.KafkaHealthcheck$SessionExpireListener)
          [2017-03-20 01:13:27,491] INFO Creating /brokers/ids/1 (is it secure? false) (kafka.utils.ZKCheckedEphemeral)
          [2017-03-20 01:13:27,503] INFO Result of znode creation is: OK (kafka.utils.ZKCheckedEphemeral)
          [2017-03-20 01:13:27,503] INFO Registered broker 1 at path /brokers/ids/1 with addresses: PLAINTEXT -> EndPoint(xxxx,xxxx,PLAINTEXT) (kafka.utils.ZkUtils)
          [2017-03-20 01:13:27,504] INFO done re-registering broker (kafka.server.KafkaHealthcheck$SessionExpireListener)
          [2017-03-20 01:13:27,504] INFO Subscribing to /brokers/topics path to watch for new topics (kafka.server.KafkaHealthcheck$SessionExpireListener)
          [2017-03-20 01:13:27,508] INFO New leader is 2 (kafka.server.ZookeeperLeaderElector$LeaderChangeListener)
          [2017-03-20 01:13:38,134] INFO Partition [__consumer_offsets,40] on broker 1: Shrinking ISR for partition [__consumer_offsets,40] from 1,0 to 1 (kafka.cluster.Partition)
          [2017-03-20 01:13:38,155] INFO Partition [__consumer_offsets,40] on broker 1: Cached zkVersion [2] not equal to that in zookeeper, skip updating ISR (kafka.cluster.Partition)
          [2017-03-20 01:13:38,156] INFO Partition [__consumer_offsets,0] on broker 1: Shrinking ISR for partition [__consumer_offsets,0] from 1,0 to 1 (kafka.cluster.Partition)
          [2017-03-20 01:13:38,161] INFO Partition [__consumer_offsets,0] on broker 1: Cached zkVersion [2] not equal to that in zookeeper, skip updating ISR (kafka.cluster.Partition)
          [2017-03-20 01:13:38,162] INFO Partition [__consumer_offsets,12] on broker 1: Shrinking ISR for partition [__consumer_offsets,12] from 1,0 to 1 (kafka.cluster.Partition)
          [2017-03-20 01:13:38,170] INFO Partition [__consumer_offsets,12] on broker 1: Cached zkVersion [2] not equal to that in zookeeper, skip updating ISR (kafka.cluster.Partition)
          [2017-03-20 01:13:38,171] INFO Partition [__consumer_offsets,14] on broker 1: Shrinking ISR for partition [__consumer_offsets,14] from 1,0 to 1 (kafka.cluster.Partition)
          [2017-03-20 01:13:38,191] INFO Partition [__consumer_offsets,14] on broker 1: Cached zkVersion [2] not equal to that in zookeeper, skip updating ISR (kafka.cluster.Partition)
          [2017-03-20 01:13:38,191] INFO Partition [__consumer_offsets,24] on broker 1: Shrinking ISR for partition [__consumer_offsets,24] from 1,0 to 1 (kafka.cluster.Partition)
          [2017-03-20 01:13:38,200] INFO Partition [__consumer_offsets,24] on broker 1: Cached zkVersion [2] not equal to that in zookeeper, skip updating ISR (kafka.cluster.Partition)
          [2017-03-20 01:13:38,200] INFO Partition [__consumer_offsets,48] on broker 1: Shrinking ISR for partition [__consumer_offsets,48] from 1,0 to 1 (kafka.cluster.Partition)
          [2017-03-20 01:13:38,209] INFO Partition [__consumer_offsets,48] on broker 1: Cached zkVersion [2] not equal to that in zookeeper, skip updating ISR (kafka.cluster.Partition)
          [2017-03-20 01:13:38,209] INFO Partition [__consumer_offsets,2] on broker 1: Shrinking ISR for partition [__consumer_offsets,2] from 1,0 to 1 (kafka.cluster.Partition)
          [2017-03-20 01:13:38,215] INFO Partition [__consumer_offsets,2] on broker 1: Cached zkVersion [2] not equal to that in zookeeper, skip updating ISR (kafka.cluster.Partition)
          [2017-03-20 01:13:38,216] INFO Partition [__consumer_offsets,32] on broker 1: Shrinking ISR for partition [__consumer_offsets,32] from 1,0 to 1 (kafka.cluster.Partition)
          
          broker 0
          [2017-03-20 01:03:09,479] INFO [Group Metadata Manager on Broker 0]: Removed 0 expired offsets in 0 milliseconds. (kafka.coordinator.GroupMetadataManager)
          [2017-03-20 01:13:09,479] INFO [Group Metadata Manager on Broker 0]: Removed 0 expired offsets in 0 milliseconds. (kafka.coordinator.GroupMetadataManager)
          [2017-03-20 01:13:27,317] INFO re-registering broker info in ZK for broker 0 (kafka.server.KafkaHealthcheck$SessionExpireListener)
          [2017-03-20 01:13:27,320] INFO Creating /brokers/ids/0 (is it secure? false) (kafka.utils.ZKCheckedEphemeral)
          [2017-03-20 01:13:27,333] INFO Result of znode creation is: OK (kafka.utils.ZKCheckedEphemeral)
          [2017-03-20 01:13:27,333] INFO Registered broker 0 at path /brokers/ids/0 with addresses: PLAINTEXT -> EndPoint(xxxx,xxxx,PLAINTEXT) (kafka.utils.ZkUtils)
          [2017-03-20 01:13:27,333] INFO done re-registering broker (kafka.server.KafkaHealthcheck$SessionExpireListener)
          [2017-03-20 01:13:27,334] INFO Subscribing to /brokers/topics path to watch for new topics (kafka.server.KafkaHealthcheck$SessionExpireListener)
          [2017-03-20 01:13:27,342] INFO New leader is 2 (kafka.server.ZookeeperLeaderElector$LeaderChangeListener)
          [2017-03-20 01:13:27,362] INFO New leader is 2 (kafka.server.ZookeeperLeaderElector$LeaderChangeListener)
          [2017-03-20 01:13:28,128] INFO [ReplicaFetcherManager on broker 0] Removed fetcher for partitions xxxxx,xxxxx(all topics) (kafka.server.ReplicaFetcherManager)
          [2017-03-20 01:13:28,142] INFO [ReplicaFetcherThread-0-2], Shutting down (kafka.server.ReplicaFetcherThread)
          [2017-03-20 01:13:28,465] INFO [ReplicaFetcherThread-0-2], Stopped  (kafka.server.ReplicaFetcherThread)
          [2017-03-20 01:13:28,465] INFO [ReplicaFetcherThread-0-2], Shutdown completed (kafka.server.ReplicaFetcherThread)
          [2017-03-20 01:13:28,481] INFO [ReplicaFetcherThread-0-1], Shutting down (kafka.server.ReplicaFetcherThread)
          [2017-03-20 01:13:28,597] INFO [ReplicaFetcherThread-0-1], Stopped  (kafka.server.ReplicaFetcherThread)
          [2017-03-20 01:13:28,597] INFO [ReplicaFetcherThread-0-1], Shutdown completed (kafka.server.ReplicaFetcherThread)
          

          The broker 0 worked fine. But broker 1 and broker 2(leader) had the same problem. Notice that the topics in broker 1 and broker 2 which refused to update the ISRs are different. Not all the topic in Kafka cluster were refusing to update ISRs.

          Show
          Ronghua Lin Ronghua Lin added a comment - Jun Rao , we also have this problem in a small cluster which has 3 brokers, running Kafka 0.10.1.1. When it happened, the logs of each broker look like this: broker 2 [2017-03-20 01:03:48,903] INFO [Group Metadata Manager on Broker 2]: Removed 0 expired offsets in 0 milliseconds. (kafka.coordinator.GroupMetadataManager) [2017-03-20 01:13:27,283] INFO Creating /controller (is it secure? false ) (kafka.utils.ZKCheckedEphemeral) [2017-03-20 01:13:27,293] INFO Result of znode creation is: OK (kafka.utils.ZKCheckedEphemeral) [2017-03-20 01:13:27,294] INFO 2 successfully elected as leader (kafka.server.ZookeeperLeaderElector) [2017-03-20 01:13:28,203] INFO re-registering broker info in ZK for broker 2 (kafka.server.KafkaHealthcheck$SessionExpireListener) [2017-03-20 01:13:28,205] INFO Creating /brokers/ids/2 (is it secure? false ) (kafka.utils.ZKCheckedEphemeral) [2017-03-20 01:13:28,218] INFO Result of znode creation is: OK (kafka.utils.ZKCheckedEphemeral) [2017-03-20 01:13:28,219] INFO Registered broker 2 at path /brokers/ids/2 with addresses: PLAINTEXT -> EndPoint(xxxxx, xxxx,PLAINTEXT) (kafka.utils.ZkUtils) [2017-03-20 01:13:28,219] INFO done re-registering broker (kafka.server.KafkaHealthcheck$SessionExpireListener) [2017-03-20 01:13:28,220] INFO Subscribing to /brokers/topics path to watch for new topics (kafka.server.KafkaHealthcheck$SessionExpireListener) [2017-03-20 01:13:28,224] INFO New leader is 2 (kafka.server.ZookeeperLeaderElector$LeaderChangeListener) [2017-03-20 01:13:28,227] INFO New leader is 2 (kafka.server.ZookeeperLeaderElector$LeaderChangeListener) [2017-03-20 01:13:38,812] INFO Partition [topic1,1] on broker 2: Shrinking ISR for partition [topic1,1] from 0,2,1 to 2,1 (kafka.cluster.Partition) [2017-03-20 01:13:38,825] INFO Partition [topic1,1] on broker 2: Cached zkVersion [6] not equal to that in zookeeper, skip updating ISR (kafka.cluster.Partition) [2017-03-20 01:13:38,825] INFO Partition [topic2,1] on broker 2: Shrinking ISR for partition [topic2,1] from 0,2,1 to 2,1 (kafka.cluster.Partition) [2017-03-20 01:13:38,835] INFO Partition [topic2,1] on broker 2: Cached zkVersion [6] not equal to that in zookeeper, skip updating ISR (kafka.cluster.Partition) [2017-03-20 01:13:38,835] INFO Partition [topic3,0] on broker 2: Shrinking ISR for partition [topic3,0] from 0,2,1 to 2,1 (kafka.cluster.Partition) [2017-03-20 01:13:38,847] INFO Partition [topic3,0] on broker 2: Cached zkVersion [6] not equal to that in zookeeper, skip updating ISR (kafka.cluster.Partition) .... broker 1 [2017-03-20 01:03:38,255] INFO [Group Metadata Manager on Broker 1]: Removed 0 expired offsets in 0 milliseconds. (kafka.coordinator.GroupMetadataManager) [2017-03-20 01:13:27,451] INFO New leader is 2 (kafka.server.ZookeeperLeaderElector$LeaderChangeListener) [2017-03-20 01:13:27,490] INFO re-registering broker info in ZK for broker 1 (kafka.server.KafkaHealthcheck$SessionExpireListener) [2017-03-20 01:13:27,491] INFO Creating /brokers/ids/1 (is it secure? false ) (kafka.utils.ZKCheckedEphemeral) [2017-03-20 01:13:27,503] INFO Result of znode creation is: OK (kafka.utils.ZKCheckedEphemeral) [2017-03-20 01:13:27,503] INFO Registered broker 1 at path /brokers/ids/1 with addresses: PLAINTEXT -> EndPoint(xxxx,xxxx,PLAINTEXT) (kafka.utils.ZkUtils) [2017-03-20 01:13:27,504] INFO done re-registering broker (kafka.server.KafkaHealthcheck$SessionExpireListener) [2017-03-20 01:13:27,504] INFO Subscribing to /brokers/topics path to watch for new topics (kafka.server.KafkaHealthcheck$SessionExpireListener) [2017-03-20 01:13:27,508] INFO New leader is 2 (kafka.server.ZookeeperLeaderElector$LeaderChangeListener) [2017-03-20 01:13:38,134] INFO Partition [__consumer_offsets,40] on broker 1: Shrinking ISR for partition [__consumer_offsets,40] from 1,0 to 1 (kafka.cluster.Partition) [2017-03-20 01:13:38,155] INFO Partition [__consumer_offsets,40] on broker 1: Cached zkVersion [2] not equal to that in zookeeper, skip updating ISR (kafka.cluster.Partition) [2017-03-20 01:13:38,156] INFO Partition [__consumer_offsets,0] on broker 1: Shrinking ISR for partition [__consumer_offsets,0] from 1,0 to 1 (kafka.cluster.Partition) [2017-03-20 01:13:38,161] INFO Partition [__consumer_offsets,0] on broker 1: Cached zkVersion [2] not equal to that in zookeeper, skip updating ISR (kafka.cluster.Partition) [2017-03-20 01:13:38,162] INFO Partition [__consumer_offsets,12] on broker 1: Shrinking ISR for partition [__consumer_offsets,12] from 1,0 to 1 (kafka.cluster.Partition) [2017-03-20 01:13:38,170] INFO Partition [__consumer_offsets,12] on broker 1: Cached zkVersion [2] not equal to that in zookeeper, skip updating ISR (kafka.cluster.Partition) [2017-03-20 01:13:38,171] INFO Partition [__consumer_offsets,14] on broker 1: Shrinking ISR for partition [__consumer_offsets,14] from 1,0 to 1 (kafka.cluster.Partition) [2017-03-20 01:13:38,191] INFO Partition [__consumer_offsets,14] on broker 1: Cached zkVersion [2] not equal to that in zookeeper, skip updating ISR (kafka.cluster.Partition) [2017-03-20 01:13:38,191] INFO Partition [__consumer_offsets,24] on broker 1: Shrinking ISR for partition [__consumer_offsets,24] from 1,0 to 1 (kafka.cluster.Partition) [2017-03-20 01:13:38,200] INFO Partition [__consumer_offsets,24] on broker 1: Cached zkVersion [2] not equal to that in zookeeper, skip updating ISR (kafka.cluster.Partition) [2017-03-20 01:13:38,200] INFO Partition [__consumer_offsets,48] on broker 1: Shrinking ISR for partition [__consumer_offsets,48] from 1,0 to 1 (kafka.cluster.Partition) [2017-03-20 01:13:38,209] INFO Partition [__consumer_offsets,48] on broker 1: Cached zkVersion [2] not equal to that in zookeeper, skip updating ISR (kafka.cluster.Partition) [2017-03-20 01:13:38,209] INFO Partition [__consumer_offsets,2] on broker 1: Shrinking ISR for partition [__consumer_offsets,2] from 1,0 to 1 (kafka.cluster.Partition) [2017-03-20 01:13:38,215] INFO Partition [__consumer_offsets,2] on broker 1: Cached zkVersion [2] not equal to that in zookeeper, skip updating ISR (kafka.cluster.Partition) [2017-03-20 01:13:38,216] INFO Partition [__consumer_offsets,32] on broker 1: Shrinking ISR for partition [__consumer_offsets,32] from 1,0 to 1 (kafka.cluster.Partition) broker 0 [2017-03-20 01:03:09,479] INFO [Group Metadata Manager on Broker 0]: Removed 0 expired offsets in 0 milliseconds. (kafka.coordinator.GroupMetadataManager) [2017-03-20 01:13:09,479] INFO [Group Metadata Manager on Broker 0]: Removed 0 expired offsets in 0 milliseconds. (kafka.coordinator.GroupMetadataManager) [2017-03-20 01:13:27,317] INFO re-registering broker info in ZK for broker 0 (kafka.server.KafkaHealthcheck$SessionExpireListener) [2017-03-20 01:13:27,320] INFO Creating /brokers/ids/0 (is it secure? false ) (kafka.utils.ZKCheckedEphemeral) [2017-03-20 01:13:27,333] INFO Result of znode creation is: OK (kafka.utils.ZKCheckedEphemeral) [2017-03-20 01:13:27,333] INFO Registered broker 0 at path /brokers/ids/0 with addresses: PLAINTEXT -> EndPoint(xxxx,xxxx,PLAINTEXT) (kafka.utils.ZkUtils) [2017-03-20 01:13:27,333] INFO done re-registering broker (kafka.server.KafkaHealthcheck$SessionExpireListener) [2017-03-20 01:13:27,334] INFO Subscribing to /brokers/topics path to watch for new topics (kafka.server.KafkaHealthcheck$SessionExpireListener) [2017-03-20 01:13:27,342] INFO New leader is 2 (kafka.server.ZookeeperLeaderElector$LeaderChangeListener) [2017-03-20 01:13:27,362] INFO New leader is 2 (kafka.server.ZookeeperLeaderElector$LeaderChangeListener) [2017-03-20 01:13:28,128] INFO [ReplicaFetcherManager on broker 0] Removed fetcher for partitions xxxxx,xxxxx(all topics) (kafka.server.ReplicaFetcherManager) [2017-03-20 01:13:28,142] INFO [ReplicaFetcherThread-0-2], Shutting down (kafka.server.ReplicaFetcherThread) [2017-03-20 01:13:28,465] INFO [ReplicaFetcherThread-0-2], Stopped (kafka.server.ReplicaFetcherThread) [2017-03-20 01:13:28,465] INFO [ReplicaFetcherThread-0-2], Shutdown completed (kafka.server.ReplicaFetcherThread) [2017-03-20 01:13:28,481] INFO [ReplicaFetcherThread-0-1], Shutting down (kafka.server.ReplicaFetcherThread) [2017-03-20 01:13:28,597] INFO [ReplicaFetcherThread-0-1], Stopped (kafka.server.ReplicaFetcherThread) [2017-03-20 01:13:28,597] INFO [ReplicaFetcherThread-0-1], Shutdown completed (kafka.server.ReplicaFetcherThread) The broker 0 worked fine. But broker 1 and broker 2(leader) had the same problem. Notice that the topics in broker 1 and broker 2 which refused to update the ISRs are different. Not all the topic in Kafka cluster were refusing to update ISRs.
          Hide
          stephane.maarek@gmail.com Stephane Maarek added a comment -

          If I may add, this is a pretty bad issue, but it got worse. You not only have to recover Kafka, but also recover your Kafka Connect ClusterS. They got stuck for me in the following state:

          [2017-03-23 00:06:05,478] INFO Marking the coordinator kafka-1:9092 (id: 2147483626 rack: null) dead for group connect-MyConnector (org.apache.kafka.clients.consumer.internals.AbstractCoordinator)
          [2017-03-23 00:06:05,478] INFO Marking the coordinator kafka-1:9092 (id: 2147483626 rack: null) dead for group connect-MyConnector (org.apache.kafka.clients.consumer.internals.AbstractCoordinator)

          Show
          stephane.maarek@gmail.com Stephane Maarek added a comment - If I may add, this is a pretty bad issue, but it got worse. You not only have to recover Kafka, but also recover your Kafka Connect ClusterS. They got stuck for me in the following state: [2017-03-23 00:06:05,478] INFO Marking the coordinator kafka-1:9092 (id: 2147483626 rack: null) dead for group connect-MyConnector (org.apache.kafka.clients.consumer.internals.AbstractCoordinator) [2017-03-23 00:06:05,478] INFO Marking the coordinator kafka-1:9092 (id: 2147483626 rack: null) dead for group connect-MyConnector (org.apache.kafka.clients.consumer.internals.AbstractCoordinator)
          Hide
          sam.nguyen@sendgrid.com Sam Nguyen added a comment -

          We ran into this today on kafka_2.11-0.10.0.1.

          There is unexpected behavior with regards to partition availability. One out of 3 total brokers in our cluster entered this state (emitting "Cached zkVersion [140] not equal to that in zookeeper, skip updating ISR" errors).

          We have our producer "required acks" config set to wait for all (-1), and the min.insync.replicas set to 2. I would have expected to be able to still be able to produce to the topic, but our producer (sarama) was getting timeouts. After restarting the broken broker, we were able to continue producing.

          I confirmed that even after performing a graceful shutdown on 1 out of 3 brokers, we are still able to produce since we have 2 out of 3 brokers still alive to serve produce and acknowledge produce requests.

          Show
          sam.nguyen@sendgrid.com Sam Nguyen added a comment - We ran into this today on kafka_2.11-0.10.0.1. There is unexpected behavior with regards to partition availability. One out of 3 total brokers in our cluster entered this state (emitting "Cached zkVersion [140] not equal to that in zookeeper, skip updating ISR" errors). We have our producer "required acks" config set to wait for all (-1), and the min.insync.replicas set to 2. I would have expected to be able to still be able to produce to the topic, but our producer (sarama) was getting timeouts. After restarting the broken broker, we were able to continue producing. I confirmed that even after performing a graceful shutdown on 1 out of 3 brokers, we are still able to produce since we have 2 out of 3 brokers still alive to serve produce and acknowledge produce requests.
          Hide
          allenzhuyi allenzhuyi added a comment -

          we see the bug in kafka_2.12-0.10.2.0,when there is long lantency ping timeout.We have 3 brokers,2 Brokes find the error below
          [
          Partition [__consumer_offsets,9] on broker 3: Shrinking ISR for partition [__consumer_offsets,9] from 3,1,2 to 3,2 (kafka.cluster.Paition)
          Partition [__consumer_offsets,9] on broker 3: Cached zkVersion [89] not equal to that in zookeeper, skip updating ISR (kafka.cluster.Partition)
          ]
          but another one broker does'n realize the bug,the producer continuly report timeout exception and send message fail,Until finally the broker zk session expired and re-registering broker info in ZK for broker. The System is recovered.
          It is a serious bug,Please help us to solve it quickly.
          Thank you.

          Show
          allenzhuyi allenzhuyi added a comment - we see the bug in kafka_2.12-0.10.2.0,when there is long lantency ping timeout.We have 3 brokers,2 Brokes find the error below [ Partition [__consumer_offsets,9] on broker 3: Shrinking ISR for partition [__consumer_offsets,9] from 3,1,2 to 3,2 (kafka.cluster.Paition) Partition [__consumer_offsets,9] on broker 3: Cached zkVersion [89] not equal to that in zookeeper, skip updating ISR (kafka.cluster.Partition) ] but another one broker does'n realize the bug,the producer continuly report timeout exception and send message fail,Until finally the broker zk session expired and re-registering broker info in ZK for broker. The System is recovered. It is a serious bug,Please help us to solve it quickly. Thank you.
          Hide
          junrao Jun Rao added a comment -

          Thanks for the additional info. In both Ronghua Lin and allenzhuyi's case, it seems ZK session expiration had happened. As I mentioned earlier in the jira, there is a known issue reported in KAFKA-3083 that when the controller's ZK session expires and loses its controller-ship, it's possible for this zombie controller to continue updating ZK and/or sending LeaderAndIsrRequests to the brokers for a short period of time. When this happens, the broker may not have the most up-to-date information about leader and isr, which can lead to subsequent ZK failure when isr needs to be updated.

          It may take some time to have this issue fixed. In the interim, the workaround for this issue is to make sure ZK session expiration never happens. This first thing is to figure out what's causing the ZK session to expire. Two common causes are (1) long broker GC and (2) network glitches. For (1), one needs to tune the GC in the broker properly. For (2), one can look at the reported time that the ZK client can't hear from the ZK server and increase the ZK session expiration time according.

          Show
          junrao Jun Rao added a comment - Thanks for the additional info. In both Ronghua Lin and allenzhuyi 's case, it seems ZK session expiration had happened. As I mentioned earlier in the jira, there is a known issue reported in KAFKA-3083 that when the controller's ZK session expires and loses its controller-ship, it's possible for this zombie controller to continue updating ZK and/or sending LeaderAndIsrRequests to the brokers for a short period of time. When this happens, the broker may not have the most up-to-date information about leader and isr, which can lead to subsequent ZK failure when isr needs to be updated. It may take some time to have this issue fixed. In the interim, the workaround for this issue is to make sure ZK session expiration never happens. This first thing is to figure out what's causing the ZK session to expire. Two common causes are (1) long broker GC and (2) network glitches. For (1), one needs to tune the GC in the broker properly. For (2), one can look at the reported time that the ZK client can't hear from the ZK server and increase the ZK session expiration time according.
          Hide
          ecomar Edoardo Comar added a comment -

          FWIW - we saw the same message
          {{ Cached zkVersion [66] not equal to that in zookeeper, skip updating ISR (kafka.cluster.Partition) }}

          when redeploying kafka 0.10.0.1 in a cluster after we had run 0.10.2.0
          after having wiped kafka's storage, but having kept zookeeper's version (the one bundled with kafka 0.10.2) and its storage

          For us eventually the cluster recovered.
          HTH.

          Show
          ecomar Edoardo Comar added a comment - FWIW - we saw the same message {{ Cached zkVersion [66] not equal to that in zookeeper, skip updating ISR (kafka.cluster.Partition) }} when redeploying kafka 0.10.0.1 in a cluster after we had run 0.10.2.0 after having wiped kafka's storage, but having kept zookeeper's version (the one bundled with kafka 0.10.2) and its storage For us eventually the cluster recovered. HTH.
          Hide
          padilo Pablo added a comment -

          Guys, this issue is not only affecting to 0.8.2.1 as many people here are saying. We had this same problem on a 0.10.2 during an upgrade from 0.8.2, we workaround it increasing the zk session and connection timeouts and worked fine, but we don't feel very safe.

          I suggest to add all the affected versions people are saying.

          Show
          padilo Pablo added a comment - Guys, this issue is not only affecting to 0.8.2.1 as many people here are saying. We had this same problem on a 0.10.2 during an upgrade from 0.8.2, we workaround it increasing the zk session and connection timeouts and worked fine, but we don't feel very safe. I suggest to add all the affected versions people are saying.
          Hide
          timoha Andrey Elenskiy added a comment - - edited

          Seeing the same issue on 0.10.2.

          A node running zookeeper lost networking for split second and initiated an election which caused some sessions to expire with:

          2017-06-22 02:07:36,092 [myid:3] - WARN  [NIOServerCxn.Factory:0.0.0.0/0.0.0.0:2181:NIOServerCnxn@373] - Exception causing close of session 0x0 due to java.io.IOException: ZooKeeperServer not running
          

          which caused controller resignation:

          [2017-06-22 02:07:36,363] INFO [SessionExpirationListener on 158980], ZK expired; shut down all controller components and try to re-elect (kafka.controller.KafkaController$SessionExpirationListener)
          [2017-06-22 02:07:37,028] DEBUG [Controller 158980]: Controller resigning, broker id 158980 (kafka.controller.KafkaController)
          [2017-06-22 02:07:37,028] DEBUG [Controller 158980]: De-registering IsrChangeNotificationListener (kafka.controller.KafkaController)
          [2017-06-22 02:07:37,028] INFO [Partition state machine on Controller 158980]: Stopped partition state machine (kafka.controller.PartitionStateMachine)
          [2017-06-22 02:07:37,028] INFO [Replica state machine on controller 158980]: Stopped replica state machine (kafka.controller.ReplicaStateMachine)
          [2017-06-22 02:07:37,028] INFO [Controller 158980]: Broker 158980 resigned as the controller (kafka.controller.KafkaController)
          

          and after that just kept getting this in broker's server logs for next 8 hours until just restarting manually it:

          [2017-06-22 17:41:06,928] INFO Partition [A,5] on broker 158980: Shrinking ISR for partition [A,5] from 158980,133641,155394 to 158980 (kafka.cluster.Partition)
          [2017-06-22 17:41:06,935] INFO Partition [A,5] on broker 158980: Cached zkVersion [73] not equal to that in zookeeper, skip updating ISR (kafka.cluster.Partition)
          
          Show
          timoha Andrey Elenskiy added a comment - - edited Seeing the same issue on 0.10.2. A node running zookeeper lost networking for split second and initiated an election which caused some sessions to expire with: 2017-06-22 02:07:36,092 [myid:3] - WARN [NIOServerCxn.Factory:0.0.0.0/0.0.0.0:2181:NIOServerCnxn@373] - Exception causing close of session 0x0 due to java.io.IOException: ZooKeeperServer not running which caused controller resignation: [2017-06-22 02:07:36,363] INFO [SessionExpirationListener on 158980], ZK expired; shut down all controller components and try to re-elect (kafka.controller.KafkaController$SessionExpirationListener) [2017-06-22 02:07:37,028] DEBUG [Controller 158980]: Controller resigning, broker id 158980 (kafka.controller.KafkaController) [2017-06-22 02:07:37,028] DEBUG [Controller 158980]: De-registering IsrChangeNotificationListener (kafka.controller.KafkaController) [2017-06-22 02:07:37,028] INFO [Partition state machine on Controller 158980]: Stopped partition state machine (kafka.controller.PartitionStateMachine) [2017-06-22 02:07:37,028] INFO [Replica state machine on controller 158980]: Stopped replica state machine (kafka.controller.ReplicaStateMachine) [2017-06-22 02:07:37,028] INFO [Controller 158980]: Broker 158980 resigned as the controller (kafka.controller.KafkaController) and after that just kept getting this in broker's server logs for next 8 hours until just restarting manually it: [2017-06-22 17:41:06,928] INFO Partition [A,5] on broker 158980: Shrinking ISR for partition [A,5] from 158980,133641,155394 to 158980 (kafka.cluster.Partition) [2017-06-22 17:41:06,935] INFO Partition [A,5] on broker 158980: Cached zkVersion [73] not equal to that in zookeeper, skip updating ISR (kafka.cluster.Partition)
          Hide
          junrao Jun Rao added a comment -

          Andrey Elenskiy, we are trying to address the ZK session expiration issue in the controller improvement work under https://issues.apache.org/jira/browse/KAFKA-5027.

          Show
          junrao Jun Rao added a comment - Andrey Elenskiy , we are trying to address the ZK session expiration issue in the controller improvement work under https://issues.apache.org/jira/browse/KAFKA-5027 .
          Hide
          MiniMizer Dan added a comment -

          Happened in 0.11.0.0 as well. Had to restart the broker to bring it back to operational state.

          Show
          MiniMizer Dan added a comment - Happened in 0.11.0.0 as well. Had to restart the broker to bring it back to operational state.
          Hide
          padilo Pablo added a comment -

          Happened to us with timeout workaround:

          zookeeper.connection.timeout.ms=10000
          zookeeper.session.timeout.ms=10000
          

          On AWS eu-west-1 on saturday using a 0.10.2.0 cluster of 3 brokers and 3 zks with message format 0.8.2.0.

          Show
          padilo Pablo added a comment - Happened to us with timeout workaround: zookeeper.connection.timeout.ms=10000 zookeeper.session.timeout.ms=10000 On AWS eu-west-1 on saturday using a 0.10.2.0 cluster of 3 brokers and 3 zks with message format 0.8.2.0.
          Hide
          joseph.aliase07@gmail.com Joseph Aliase added a comment -

          Have happened to us twice in Prod. Restart seems to be a only solution.

          Show
          joseph.aliase07@gmail.com Joseph Aliase added a comment - Have happened to us twice in Prod. Restart seems to be a only solution.
          Hide
          davispw Peter Davis added a comment - - edited

          Jun Rao Per your previous comment, is this issue definitely covered under KAFKA-5027 then? It is not linked there.

          EDIT: fixed link

          Show
          davispw Peter Davis added a comment - - edited Jun Rao Per your previous comment , is this issue definitely covered under KAFKA-5027 then? It is not linked there. EDIT: fixed link
          Hide
          junrao Jun Rao added a comment -

          Peter Davis, the issue due to ZK session expiration will be addressed in KAFKA-5642.

          Show
          junrao Jun Rao added a comment - Peter Davis , the issue due to ZK session expiration will be addressed in KAFKA-5642 .
          Hide
          sumit.jain sumit jain added a comment -
          Show
          sumit.jain sumit jain added a comment - Facing the same issue.. here's the question I asked on stack overflow https://stackoverflow.com/questions/46644764/kafka-cached-zkversion-not-equal-to-that-in-zookeeper-broker-not-recovering
          Hide
          fravigotti Francesco vigotti added a comment -

          I'm having the same issue and definitely losing trust in kafka, every 2 months there is something that force me to reset the whole cluster, I'm searching for a good alternative for a distributed-persisted-fast-queue for a while.. yet to find something that give me a good vibe..

          anyway I'm facing this same issue with some small differences

          • restarting all brokers ( together and rolling-restart ) didn't fix it..

          all brokers in the cluster log such errors :
          — broker 5

          
          [2017-10-13 08:13:57,429] ERROR [ReplicaFetcherThread-0-2], Error for partition [__consumer_offsets,17] to broker 2:org.apache.kafka.common.errors.NotLeaderForPartitionException: This server is not the leader for that topic-partition. (kafka.server.ReplicaFetcherThread)
          [2017-10-13 08:13:57,429] ERROR [ReplicaFetcherThread-0-2], Error for partition [__consumer_offsets,23] to broker 2:org.apache.kafka.common.errors.NotLeaderForPartitionException: This server is not the leader for that topic-partition. (kafka.server.ReplicaFetcherThread)
          [2017-10-13 08:13:57,429] ERROR [ReplicaFetcherThread-0-2], Error for partition [__consumer_offsets,47] to broker 2:org.apache.kafka.common.errors.NotLeaderForPartitionException: This server is not the leader for that topic-partition. (kafka.server.ReplicaFetcherThread)
          [2017-10-13 08:13:57,429] ERROR [ReplicaFetcherThread-0-2], Error for partition [__consumer_offsets,29] to broker 2:org.apache.kafka.common.errors.NotLeaderForPartitionException: This server is not the leader for that topic-partition. (kafka.server.ReplicaFetcherThread)
          
          

          — broker3

          )

          
          [2017-10-13 08:13:58,547] INFO Partition [__consumer_offsets,20] on broker 3: Expanding ISR for partition __consumer_offsets-20 from 3,2 to 3,2,5 (kafka.cluster.Partition)
          [2017-10-13 08:13:58,551] INFO Partition [__consumer_offsets,44] on broker 3: Expanding ISR for partition __consumer_offsets-44 from 3,2 to 3,2,5 (kafka.cluster.Partition)
          [2017-10-13 08:13:58,554] INFO Partition [__consumer_offsets,5] on broker 3: Expanding ISR for partition __consumer_offsets-5 from 2,3 to 2,3,5 (kafka.cluster.Partition)
          [2017-10-13 08:13:58,557] INFO Partition [__consumer_offsets,26] on broker 3: Expanding ISR for partition __consumer_offsets-26 from 3,2 to 3,2,5 (kafka.cluster.Partition)
          [2017-10-13 08:13:58,563] INFO Partition [__consumer_offsets,29] on broker 3: Expanding ISR for partition __consumer_offsets-29 from 2,3 to 2,3,5 (kafka.cluster.Partition)
          [2017-10-13 08:13:58,566] INFO Partition [__consumer_offsets,32] on broker 3: Expanding ISR for partition __consumer_offsets-32 from 3,2 to 3,2,5 (kafka.cluster.Partition)
          [2017-10-13 08:13:58,570] INFO Partition [legacyJavaVarT,2] on broker 3: Expanding ISR for partition legacyJavaVarT-2 from 3 to 3,5 (kafka.cluster.Partition)
          [2017-10-13 08:13:58,573] INFO Partition [test4,3] on broker 3: Expanding ISR for partition test4-3 from 2,3 to 2,3,5 (kafka.cluster.Partition)
          [2017-10-13 08:13:58,577] INFO Partition [test4,0] on broker 3: Expanding ISR for partition test4-0 from 3,2 to 3,2,5 (kafka.cluster.Partition)
          [2017-10-13 08:13:58,582] INFO Partition [test3,5] on broker 3: Expanding ISR for partition test3-5 from 3 to 3,5 (kafka.cluster.Partition)
          
          

          — broker2

          
          [2017-10-13 08:13:36,289] INFO Partition [__consumer_offsets,11] on broker 2: Expanding ISR for partition __consumer_offsets-11 from 2,5 to 2,5,3 (kafka.cluster.Partition)
          [2017-10-13 08:13:36,293] INFO Partition [__consumer_offsets,41] on broker 2: Expanding ISR for partition __consumer_offsets-41 from 2,5 to 2,5,3 (kafka.cluster.Partition)
          [2017-10-13 08:13:36,296] INFO Partition [test3,2] on broker 2: Expanding ISR for partition test3-2 from 2 to 2,3 (kafka.cluster.Partition)
          [2017-10-13 08:13:36,300] INFO Partition [__consumer_offsets,23] on broker 2: Expanding ISR for partition __consumer_offsets-23 from 2,5 to 2,5,3 (kafka.cluster.Partition)
          [2017-10-13 08:13:36,304] INFO Partition [__consumer_offsets,5] on broker 2: Expanding ISR for partition __consumer_offsets-5 from 2,5 to 2,5,3 (kafka.cluster.Partition)
          [2017-10-13 08:13:36,337] INFO Partition [__consumer_offsets,35] on broker 2: Expanding ISR for partition __consumer_offsets-35 from 2,5 to 2,5,3 (kafka.cluster.Partition)
          [2017-10-13 08:13:36,372] INFO Partition [test_mainlog,24] on broker 2: Expanding ISR for partition test_mainlog-24 from 2 to 2,3 (kafka.cluster.Partition)
          [2017-10-13 08:13:36,375] INFO Partition [test_mainlog,6] on broker 2: Expanding ISR for partition test_mainlog-6 from 2 to 2,3 (kafka.cluster.Partition)
          [2017-10-13 08:13:36,379] INFO Partition [test_mainlog,18] on broker 2: Expanding ISR for partition test_mainlog-18 from 2 to 2,3 (kafka.cluster.Partition)
          [2017-10-13 08:13:36,384] INFO Partition [test_mainlog,0] on broker 2: Expanding ISR for partition test_mainlog-0 from 2 to 2,3 (kafka.cluster.Partition)
          [2017-10-13 08:13:36,388] INFO Partition [test_mainlog,12] on broker 2: Expanding ISR for partition test_mainlog-12 from 2 to 2,3 (kafka.cluster.Partition)
          [2017-10-13 08:13:40,367] INFO [ReplicaFetcherManager on broker 2] Removed fetcher for partitions __consumer_offsets-47 (kafka.server.ReplicaFetcherManager)
          [2017-10-13 08:13:40,367] INFO Truncating log __consumer_offsets-47 to offset 0. (kafka.log.Log)
          [2017-10-13 08:13:40,374] INFO [ReplicaFetcherThread-0-3], Starting  (kafka.server.ReplicaFetcherThread)
          [2017-10-13 08:13:40,374] INFO [ReplicaFetcherManager on broker 2] Added fetcher for partitions List([__consumer_offsets-47, initOffset 0 to broker BrokerEndPoint(3,--hidden----.73,9092)] ) (kafka.server.ReplicaFetcherManager)
          [2017-10-13 08:13:40,376] ERROR [ReplicaFetcherThread-0-3], Error for partition [__consumer_offsets,47] to broker 3:org.apache.kafka.common.errors.NotLeaderForPartitionException: This server is not the leader for that topic-partition. (kafka.server.ReplicaFetcherThread)
          [2017-10-13 08:13:40,393] INFO [ReplicaFetcherManager on broker 2] Removed fetcher for partitions __consumer_offsets-29 (kafka.server.ReplicaFetcherManager)
          [2017-10-13 08:13:40,393] INFO Truncating log __consumer_offsets-29 to offset 0. (kafka.log.Log)
          [2017-10-13 08:13:40,402] INFO [ReplicaFetcherManager on broker 2] Added fetcher for partitions List([__consumer_offsets-29, initOffset 0 to broker BrokerEndPoint(3,--hidden----.73,9092)] ) (kafka.server.ReplicaFetcherManager)
          [2017-10-13 08:13:40,403] ERROR [ReplicaFetcherThread-0-3], Error for partition [__consumer_offsets,29] to broker 3:org.apache.kafka.common.errors.NotLeaderForPartitionException: This server is not the leader for that topic-partition. (kafka.server.ReplicaFetcherThread)
          [2017-10-13 08:13:40,407] INFO [ReplicaFetcherManager on broker 2] Removed fetcher for partitions __consumer_offsets-41 (kafka.server.ReplicaFetcherManager)
          [2017-10-13 08:13:40,407] INFO Truncating log __consumer_offsets-41 to offset 0. (kafka.log.Log)
          [2017-10-13 08:13:40,413] INFO [ReplicaFetcherManager on broker 2] Added fetcher for partitions List([__consumer_offsets-41, initOffset 0 to broker BrokerEndPoint(3,--hidden----.73,9092)] ) (kafka.server.ReplicaFetcherManager)
          [2017-10-13 08:13:40,414] ERROR [ReplicaFetcherThread-0-3], Error for partition [__consumer_offsets,41] to broker 3:org.apache.kafka.common.errors.NotLeaderForPartitionException: This server is not the leader for that topic-partition. (kafka.server.ReplicaFetcherThread)
          [2017-10-13 08:13:40,419] INFO [ReplicaFetcherManager on broker 2] Removed fetcher for partitions test_mainlog-6 (kafka.server.ReplicaFetcherManager)
          [2017-10-13 08:13:40,419] INFO Truncating log test_mainlog-6 to offset 4997933406. (kafka.log.Log)
          [2017-10-13 08:13:40,425] INFO [ReplicaFetcherManager on broker 2] Added fetcher for partitions List([test_mainlog-6, initOffset 4997933406 to broker BrokerEndPoint(3,--hidden----.73,9092)] ) (kafka.server.ReplicaFetcherManager)
          [2017-10-13 08:13:40,432] INFO [ReplicaFetcherManager on broker 2] Removed fetcher for partitions __consumer_offsets-17 (kafka.server.ReplicaFetcherManager)
          [2017-10-13 08:13:40,432] INFO Truncating log __consumer_offsets-17 to offset 0. (kafka.log.Log)
          [2017-10-13 08:13:40,438] INFO [ReplicaFetcherManager on broker 2] Added fetcher for partitions List([__consumer_offsets-17, initOffset 0 to broker BrokerEndPoint(3,--hidden----.73,9092)] ) (kafka.server.ReplicaFetcherManager)
          [2017-10-13 08:13:40,443] INFO [ReplicaFetcherManager on broker 2] Removed fetcher for partitions test_mainlog-0 (kafka.server.ReplicaFetcherManager)
          [2017-10-13 08:13:40,443] INFO Truncating log test_mainlog-0 to offset 5704085814. (kafka.log.Log)
          [2017-10-13 08:13:40,449] INFO [ReplicaFetcherManager on broker 2] Added fetcher for partitions List([test_mainlog-0, initOffset 5704085814 to broker BrokerEndPoint(3,--hidden----.73,9092)] ) (kafka.server.ReplicaFetcherManager)
          [2017-10-13 08:13:40,464] INFO [ReplicaFetcherManager on broker 2] Removed fetcher for partitions __consumer_offsets-14 (kafka.server.ReplicaFetcherManager)
          [2017-10-13 08:13:40,464] INFO Truncating log __consumer_offsets-14 to offset 0. (kafka.log.Log)
          [2017-10-13 08:13:40,472] INFO [ReplicaFetcherManager on broker 2] Added fetcher for partitions List([__consumer_offsets-14, initOffset 0 to broker BrokerEndPoint(3,--hidden----.73,9092)] ) (kafka.server.ReplicaFetcherManager)
          
          

          those logs goes for hours and the cluster never recover, the only things that change something is when I repeatedly from zookeeper
          delete /controller # repeatedly untill it get assigned to kafka3 node

          and at this point all errors stop ( no more error logs ) , kafka seems working, kafkamanager show offsets for all partitions ( while some offset was missing ) , data ingestion /consumption works , the only things that presages something wrong is that on one topic with 30 partitions and replication 2 there is 1 broker skew ( 1 broker have 1 partitions more than normal and one broker have 1 partition less than normal )
          and the situation remain stable with this small anomaly for hours.. nodes delete indexes, delete segments , roll new segments..

          If i now delete the controller again, or restart the kafka3-node evreything goes to the previous situation again ( all errors logged ) and at this point I don't even know how to recover , the only "fix" I'm left to try is to wipe the whole cluster data and restart but what to do then if this happens again in future ?

          I don't know why two nodes seems to have a ("broken controller" ??) and the cluster remain in this in-consistent state forever..
          If you have any suggestion... on what to inspect / how to try to fix , those are very welcomed..

          Thank you,
          Francesco

          Show
          fravigotti Francesco vigotti added a comment - I'm having the same issue and definitely losing trust in kafka, every 2 months there is something that force me to reset the whole cluster, I'm searching for a good alternative for a distributed-persisted-fast-queue for a while.. yet to find something that give me a good vibe.. anyway I'm facing this same issue with some small differences restarting all brokers ( together and rolling-restart ) didn't fix it.. all brokers in the cluster log such errors : — broker 5 [2017-10-13 08:13:57,429] ERROR [ReplicaFetcherThread-0-2], Error for partition [__consumer_offsets,17] to broker 2:org.apache.kafka.common.errors.NotLeaderForPartitionException: This server is not the leader for that topic-partition. (kafka.server.ReplicaFetcherThread) [2017-10-13 08:13:57,429] ERROR [ReplicaFetcherThread-0-2], Error for partition [__consumer_offsets,23] to broker 2:org.apache.kafka.common.errors.NotLeaderForPartitionException: This server is not the leader for that topic-partition. (kafka.server.ReplicaFetcherThread) [2017-10-13 08:13:57,429] ERROR [ReplicaFetcherThread-0-2], Error for partition [__consumer_offsets,47] to broker 2:org.apache.kafka.common.errors.NotLeaderForPartitionException: This server is not the leader for that topic-partition. (kafka.server.ReplicaFetcherThread) [2017-10-13 08:13:57,429] ERROR [ReplicaFetcherThread-0-2], Error for partition [__consumer_offsets,29] to broker 2:org.apache.kafka.common.errors.NotLeaderForPartitionException: This server is not the leader for that topic-partition. (kafka.server.ReplicaFetcherThread) — broker3 ) [2017-10-13 08:13:58,547] INFO Partition [__consumer_offsets,20] on broker 3: Expanding ISR for partition __consumer_offsets-20 from 3,2 to 3,2,5 (kafka.cluster.Partition) [2017-10-13 08:13:58,551] INFO Partition [__consumer_offsets,44] on broker 3: Expanding ISR for partition __consumer_offsets-44 from 3,2 to 3,2,5 (kafka.cluster.Partition) [2017-10-13 08:13:58,554] INFO Partition [__consumer_offsets,5] on broker 3: Expanding ISR for partition __consumer_offsets-5 from 2,3 to 2,3,5 (kafka.cluster.Partition) [2017-10-13 08:13:58,557] INFO Partition [__consumer_offsets,26] on broker 3: Expanding ISR for partition __consumer_offsets-26 from 3,2 to 3,2,5 (kafka.cluster.Partition) [2017-10-13 08:13:58,563] INFO Partition [__consumer_offsets,29] on broker 3: Expanding ISR for partition __consumer_offsets-29 from 2,3 to 2,3,5 (kafka.cluster.Partition) [2017-10-13 08:13:58,566] INFO Partition [__consumer_offsets,32] on broker 3: Expanding ISR for partition __consumer_offsets-32 from 3,2 to 3,2,5 (kafka.cluster.Partition) [2017-10-13 08:13:58,570] INFO Partition [legacyJavaVarT,2] on broker 3: Expanding ISR for partition legacyJavaVarT-2 from 3 to 3,5 (kafka.cluster.Partition) [2017-10-13 08:13:58,573] INFO Partition [test4,3] on broker 3: Expanding ISR for partition test4-3 from 2,3 to 2,3,5 (kafka.cluster.Partition) [2017-10-13 08:13:58,577] INFO Partition [test4,0] on broker 3: Expanding ISR for partition test4-0 from 3,2 to 3,2,5 (kafka.cluster.Partition) [2017-10-13 08:13:58,582] INFO Partition [test3,5] on broker 3: Expanding ISR for partition test3-5 from 3 to 3,5 (kafka.cluster.Partition) — broker2 [2017-10-13 08:13:36,289] INFO Partition [__consumer_offsets,11] on broker 2: Expanding ISR for partition __consumer_offsets-11 from 2,5 to 2,5,3 (kafka.cluster.Partition) [2017-10-13 08:13:36,293] INFO Partition [__consumer_offsets,41] on broker 2: Expanding ISR for partition __consumer_offsets-41 from 2,5 to 2,5,3 (kafka.cluster.Partition) [2017-10-13 08:13:36,296] INFO Partition [test3,2] on broker 2: Expanding ISR for partition test3-2 from 2 to 2,3 (kafka.cluster.Partition) [2017-10-13 08:13:36,300] INFO Partition [__consumer_offsets,23] on broker 2: Expanding ISR for partition __consumer_offsets-23 from 2,5 to 2,5,3 (kafka.cluster.Partition) [2017-10-13 08:13:36,304] INFO Partition [__consumer_offsets,5] on broker 2: Expanding ISR for partition __consumer_offsets-5 from 2,5 to 2,5,3 (kafka.cluster.Partition) [2017-10-13 08:13:36,337] INFO Partition [__consumer_offsets,35] on broker 2: Expanding ISR for partition __consumer_offsets-35 from 2,5 to 2,5,3 (kafka.cluster.Partition) [2017-10-13 08:13:36,372] INFO Partition [test_mainlog,24] on broker 2: Expanding ISR for partition test_mainlog-24 from 2 to 2,3 (kafka.cluster.Partition) [2017-10-13 08:13:36,375] INFO Partition [test_mainlog,6] on broker 2: Expanding ISR for partition test_mainlog-6 from 2 to 2,3 (kafka.cluster.Partition) [2017-10-13 08:13:36,379] INFO Partition [test_mainlog,18] on broker 2: Expanding ISR for partition test_mainlog-18 from 2 to 2,3 (kafka.cluster.Partition) [2017-10-13 08:13:36,384] INFO Partition [test_mainlog,0] on broker 2: Expanding ISR for partition test_mainlog-0 from 2 to 2,3 (kafka.cluster.Partition) [2017-10-13 08:13:36,388] INFO Partition [test_mainlog,12] on broker 2: Expanding ISR for partition test_mainlog-12 from 2 to 2,3 (kafka.cluster.Partition) [2017-10-13 08:13:40,367] INFO [ReplicaFetcherManager on broker 2] Removed fetcher for partitions __consumer_offsets-47 (kafka.server.ReplicaFetcherManager) [2017-10-13 08:13:40,367] INFO Truncating log __consumer_offsets-47 to offset 0. (kafka.log.Log) [2017-10-13 08:13:40,374] INFO [ReplicaFetcherThread-0-3], Starting (kafka.server.ReplicaFetcherThread) [2017-10-13 08:13:40,374] INFO [ReplicaFetcherManager on broker 2] Added fetcher for partitions List([__consumer_offsets-47, initOffset 0 to broker BrokerEndPoint(3,--hidden----.73,9092)] ) (kafka.server.ReplicaFetcherManager) [2017-10-13 08:13:40,376] ERROR [ReplicaFetcherThread-0-3], Error for partition [__consumer_offsets,47] to broker 3:org.apache.kafka.common.errors.NotLeaderForPartitionException: This server is not the leader for that topic-partition. (kafka.server.ReplicaFetcherThread) [2017-10-13 08:13:40,393] INFO [ReplicaFetcherManager on broker 2] Removed fetcher for partitions __consumer_offsets-29 (kafka.server.ReplicaFetcherManager) [2017-10-13 08:13:40,393] INFO Truncating log __consumer_offsets-29 to offset 0. (kafka.log.Log) [2017-10-13 08:13:40,402] INFO [ReplicaFetcherManager on broker 2] Added fetcher for partitions List([__consumer_offsets-29, initOffset 0 to broker BrokerEndPoint(3,--hidden----.73,9092)] ) (kafka.server.ReplicaFetcherManager) [2017-10-13 08:13:40,403] ERROR [ReplicaFetcherThread-0-3], Error for partition [__consumer_offsets,29] to broker 3:org.apache.kafka.common.errors.NotLeaderForPartitionException: This server is not the leader for that topic-partition. (kafka.server.ReplicaFetcherThread) [2017-10-13 08:13:40,407] INFO [ReplicaFetcherManager on broker 2] Removed fetcher for partitions __consumer_offsets-41 (kafka.server.ReplicaFetcherManager) [2017-10-13 08:13:40,407] INFO Truncating log __consumer_offsets-41 to offset 0. (kafka.log.Log) [2017-10-13 08:13:40,413] INFO [ReplicaFetcherManager on broker 2] Added fetcher for partitions List([__consumer_offsets-41, initOffset 0 to broker BrokerEndPoint(3,--hidden----.73,9092)] ) (kafka.server.ReplicaFetcherManager) [2017-10-13 08:13:40,414] ERROR [ReplicaFetcherThread-0-3], Error for partition [__consumer_offsets,41] to broker 3:org.apache.kafka.common.errors.NotLeaderForPartitionException: This server is not the leader for that topic-partition. (kafka.server.ReplicaFetcherThread) [2017-10-13 08:13:40,419] INFO [ReplicaFetcherManager on broker 2] Removed fetcher for partitions test_mainlog-6 (kafka.server.ReplicaFetcherManager) [2017-10-13 08:13:40,419] INFO Truncating log test_mainlog-6 to offset 4997933406. (kafka.log.Log) [2017-10-13 08:13:40,425] INFO [ReplicaFetcherManager on broker 2] Added fetcher for partitions List([test_mainlog-6, initOffset 4997933406 to broker BrokerEndPoint(3,--hidden----.73,9092)] ) (kafka.server.ReplicaFetcherManager) [2017-10-13 08:13:40,432] INFO [ReplicaFetcherManager on broker 2] Removed fetcher for partitions __consumer_offsets-17 (kafka.server.ReplicaFetcherManager) [2017-10-13 08:13:40,432] INFO Truncating log __consumer_offsets-17 to offset 0. (kafka.log.Log) [2017-10-13 08:13:40,438] INFO [ReplicaFetcherManager on broker 2] Added fetcher for partitions List([__consumer_offsets-17, initOffset 0 to broker BrokerEndPoint(3,--hidden----.73,9092)] ) (kafka.server.ReplicaFetcherManager) [2017-10-13 08:13:40,443] INFO [ReplicaFetcherManager on broker 2] Removed fetcher for partitions test_mainlog-0 (kafka.server.ReplicaFetcherManager) [2017-10-13 08:13:40,443] INFO Truncating log test_mainlog-0 to offset 5704085814. (kafka.log.Log) [2017-10-13 08:13:40,449] INFO [ReplicaFetcherManager on broker 2] Added fetcher for partitions List([test_mainlog-0, initOffset 5704085814 to broker BrokerEndPoint(3,--hidden----.73,9092)] ) (kafka.server.ReplicaFetcherManager) [2017-10-13 08:13:40,464] INFO [ReplicaFetcherManager on broker 2] Removed fetcher for partitions __consumer_offsets-14 (kafka.server.ReplicaFetcherManager) [2017-10-13 08:13:40,464] INFO Truncating log __consumer_offsets-14 to offset 0. (kafka.log.Log) [2017-10-13 08:13:40,472] INFO [ReplicaFetcherManager on broker 2] Added fetcher for partitions List([__consumer_offsets-14, initOffset 0 to broker BrokerEndPoint(3,--hidden----.73,9092)] ) (kafka.server.ReplicaFetcherManager) those logs goes for hours and the cluster never recover, the only things that change something is when I repeatedly from zookeeper delete /controller # repeatedly untill it get assigned to kafka3 node and at this point all errors stop ( no more error logs ) , kafka seems working, kafkamanager show offsets for all partitions ( while some offset was missing ) , data ingestion /consumption works , the only things that presages something wrong is that on one topic with 30 partitions and replication 2 there is 1 broker skew ( 1 broker have 1 partitions more than normal and one broker have 1 partition less than normal ) and the situation remain stable with this small anomaly for hours.. nodes delete indexes, delete segments , roll new segments.. If i now delete the controller again, or restart the kafka3-node evreything goes to the previous situation again ( all errors logged ) and at this point I don't even know how to recover , the only "fix" I'm left to try is to wipe the whole cluster data and restart but what to do then if this happens again in future ? I don't know why two nodes seems to have a ("broken controller" ??) and the cluster remain in this in-consistent state forever.. If you have any suggestion... on what to inspect / how to try to fix , those are very welcomed.. Thank you, Francesco
          Hide
          ijuma Ismael Juma added a comment -

          Francesco vigotti, none of your log messages seems to be about the zkVersion issue, is it really the same issue as this one? If not, you should file a separate JIRA including the Kafka version.

          Show
          ijuma Ismael Juma added a comment - Francesco vigotti , none of your log messages seems to be about the zkVersion issue, is it really the same issue as this one? If not, you should file a separate JIRA including the Kafka version.
          Hide
          fravigotti Francesco vigotti added a comment -

          At the beginning of my cluster screw up I've got tons of zkVersion issue that's why I've posted here , but because seems that the problems for you goes away when you restarted your brokers maybe my problem is different..
          kafka version : 0.10.2.1

          Show
          fravigotti Francesco vigotti added a comment - At the beginning of my cluster screw up I've got tons of zkVersion issue that's why I've posted here , but because seems that the problems for you goes away when you restarted your brokers maybe my problem is different.. kafka version : 0.10.2.1
          Hide
          fravigotti Francesco vigotti added a comment -

          After having lost 2 days on this I've reset whole cluster, stopped all kafka brokers, stopped zookeeper cluster, delete all directories,stopped all consumer and producer ,then restarted everything , recreated topics and now guess what?

          one node reports...

          
          [2017-10-13 15:54:52,893] INFO Partition [__consumer_offsets,5] on broker 2: Expanding ISR for partition __consumer_offsets-5 from 10,13,2 to 10,13,2,5 (kafka.cluster.Partition)
          [2017-10-13 15:54:52,906] INFO Partition [__consumer_offsets,5] on broker 2: Cached zkVersion [13] not equal to that in zookeeper, skip updating ISR (kafka.cluster.Partition)
          [2017-10-13 15:54:52,908] INFO Partition [__consumer_offsets,25] on broker 2: Expanding ISR for partition __consumer_offsets-25 from 10,2,13 to 10,2,13,5 (kafka.cluster.Partition)
          [2017-10-13 15:54:52,915] INFO Partition [__consumer_offsets,25] on broker 2: Cached zkVersion [10] not equal to that in zookeeper, skip updating ISR (kafka.cluster.Partition)
          [2017-10-13 15:54:52,916] INFO Partition [__consumer_offsets,45] on broker 2: Expanding ISR for partition __consumer_offsets-45 from 10,13,2 to 10,13,2,5 (kafka.cluster.Partition)
          [2017-10-13 15:54:52,925] INFO Partition [__consumer_offsets,45] on broker 2: Cached zkVersion [15] not equal to that in zookeeper, skip updating ISR (kafka.cluster.Partition)
          [2017-10-13 15:54:52,926] INFO Partition [__consumer_offsets,5] on broker 2: Expanding ISR for partition __consumer_offsets-5 from 10,13,2 to 10,13,2,5 (kafka.cluster.Partition)
          [2017-10-13 15:54:52,936] INFO Partition [__consumer_offsets,5] on broker 2: Cached zkVersion [13] not equal to that in zookeeper, skip updating ISR (kafka.cluster.Partition)
          [2017-10-13 15:54:52,939] INFO Partition [__consumer_offsets,25] on broker 2: Expanding ISR for partition __consumer_offsets-25 from 10,2,13 to 10,2,13,5 (kafka.cluster.Partition)
          

          while others

          [2017-10-13 15:57:08,128] ERROR [ReplicaFetcherThread-0-2], Error for partition [__consumer_offsets,40] to broker 2:org.apache.kafka.common.errors.NotLeaderForPartitionException: This server is not the leader for that topic-partition. (kafka.server.ReplicaFetcherThread)
          [2017-10-13 15:57:09,129] ERROR [ReplicaFetcherThread-0-2], Error for partition [__consumer_offsets,40] to broker 2:org.apache.kafka.common.errors.NotLeaderForPartitionException: This server is not the leader for that topic-partition. (kafka.server.ReplicaFetcherThread)
          [2017-10-13 15:57:10,260] ERROR [ReplicaFetcherThread-0-2], Error for partition [__consumer_offsets,40] to broker 2:org.apache.kafka.common.errors.NotLeaderForPartitionException: This server is not the leader for that topic-partition. (kafka.server.ReplicaFetcherThread)
          [2017-10-13 15:57:11,262] ERROR [ReplicaFetcherThread-0-2], Error for partition [__consumer_offsets,40] to broker 2:org.apache.kafka.common.errors.NotLeaderForPartitionException: This server is not the leader for that topic-partition. (kafka.server.ReplicaFetcherThread)
          [2017-10-13 15:57:12,265] ERROR [ReplicaFetcherThread-0-2], Error for partition [__consumer_offsets,40] to broker 2:org.apache.kafka.common.errors.NotLeaderForPartitionException: This server is not the leader for that topic-partition. (kafka.server.ReplicaFetcherThread)
          [2017-10-13 15:57:13,289] ERROR [ReplicaFetcherThread-0-2], Error fo
          

          cluster still being inconsistent, I've also added 2 more nodes hoping in an increasing of stability but nothing, I don't know if something is wrong because if kafka do some kind of pre-flight checks during startup it does log nothing.. the only logs are those which have no sense because the leader should be re-elected when there are ISR available.. and there are
          I've started looking for an alternative software to use, I'm trying to use kafka is so frustrating

          Show
          fravigotti Francesco vigotti added a comment - After having lost 2 days on this I've reset whole cluster, stopped all kafka brokers, stopped zookeeper cluster, delete all directories,stopped all consumer and producer ,then restarted everything , recreated topics and now guess what? one node reports... [2017-10-13 15:54:52,893] INFO Partition [__consumer_offsets,5] on broker 2: Expanding ISR for partition __consumer_offsets-5 from 10,13,2 to 10,13,2,5 (kafka.cluster.Partition) [2017-10-13 15:54:52,906] INFO Partition [__consumer_offsets,5] on broker 2: Cached zkVersion [13] not equal to that in zookeeper, skip updating ISR (kafka.cluster.Partition) [2017-10-13 15:54:52,908] INFO Partition [__consumer_offsets,25] on broker 2: Expanding ISR for partition __consumer_offsets-25 from 10,2,13 to 10,2,13,5 (kafka.cluster.Partition) [2017-10-13 15:54:52,915] INFO Partition [__consumer_offsets,25] on broker 2: Cached zkVersion [10] not equal to that in zookeeper, skip updating ISR (kafka.cluster.Partition) [2017-10-13 15:54:52,916] INFO Partition [__consumer_offsets,45] on broker 2: Expanding ISR for partition __consumer_offsets-45 from 10,13,2 to 10,13,2,5 (kafka.cluster.Partition) [2017-10-13 15:54:52,925] INFO Partition [__consumer_offsets,45] on broker 2: Cached zkVersion [15] not equal to that in zookeeper, skip updating ISR (kafka.cluster.Partition) [2017-10-13 15:54:52,926] INFO Partition [__consumer_offsets,5] on broker 2: Expanding ISR for partition __consumer_offsets-5 from 10,13,2 to 10,13,2,5 (kafka.cluster.Partition) [2017-10-13 15:54:52,936] INFO Partition [__consumer_offsets,5] on broker 2: Cached zkVersion [13] not equal to that in zookeeper, skip updating ISR (kafka.cluster.Partition) [2017-10-13 15:54:52,939] INFO Partition [__consumer_offsets,25] on broker 2: Expanding ISR for partition __consumer_offsets-25 from 10,2,13 to 10,2,13,5 (kafka.cluster.Partition) while others [2017-10-13 15:57:08,128] ERROR [ReplicaFetcherThread-0-2], Error for partition [__consumer_offsets,40] to broker 2:org.apache.kafka.common.errors.NotLeaderForPartitionException: This server is not the leader for that topic-partition. (kafka.server.ReplicaFetcherThread) [2017-10-13 15:57:09,129] ERROR [ReplicaFetcherThread-0-2], Error for partition [__consumer_offsets,40] to broker 2:org.apache.kafka.common.errors.NotLeaderForPartitionException: This server is not the leader for that topic-partition. (kafka.server.ReplicaFetcherThread) [2017-10-13 15:57:10,260] ERROR [ReplicaFetcherThread-0-2], Error for partition [__consumer_offsets,40] to broker 2:org.apache.kafka.common.errors.NotLeaderForPartitionException: This server is not the leader for that topic-partition. (kafka.server.ReplicaFetcherThread) [2017-10-13 15:57:11,262] ERROR [ReplicaFetcherThread-0-2], Error for partition [__consumer_offsets,40] to broker 2:org.apache.kafka.common.errors.NotLeaderForPartitionException: This server is not the leader for that topic-partition. (kafka.server.ReplicaFetcherThread) [2017-10-13 15:57:12,265] ERROR [ReplicaFetcherThread-0-2], Error for partition [__consumer_offsets,40] to broker 2:org.apache.kafka.common.errors.NotLeaderForPartitionException: This server is not the leader for that topic-partition. (kafka.server.ReplicaFetcherThread) [2017-10-13 15:57:13,289] ERROR [ReplicaFetcherThread-0-2], Error fo cluster still being inconsistent, I've also added 2 more nodes hoping in an increasing of stability but nothing, I don't know if something is wrong because if kafka do some kind of pre-flight checks during startup it does log nothing.. the only logs are those which have no sense because the leader should be re-elected when there are ISR available.. and there are I've started looking for an alternative software to use, I'm trying to use kafka is so frustrating
          Hide
          ijuma Ismael Juma added a comment -

          If you're seeing the issue this often, then there's most likely a configuration issue. If you file a separate issue with all the logs (including GC logs) and configs (broker and ZK), maybe someone can help.

          Show
          ijuma Ismael Juma added a comment - If you're seeing the issue this often, then there's most likely a configuration issue. If you file a separate issue with all the logs (including GC logs) and configs (broker and ZK), maybe someone can help.
          Hide
          junrao Jun Rao added a comment -

          Francesco vigotti, sorry to hear that. A couple of quick suggestions.

          (1) Do you see any ZK session expiration in the log (e.g., INFO zookeeper state changed (Expired) (org.I0Itec.zkclient.ZkClient))? There are known bugs in Kafka in handling ZK session expiration. So, it would be useful to avoid it in the first place. Typical causes of ZK session expiration are long GC in the broker or network glitches. So you can either tune the broker or increase zookeeper.session.timeout.ms.

          (2) Do you have lots of partitions (say a few thousands) per broker? If so, you want to check if the controlled shutdown succeeds when shutting down a broker. If not, restarting the broker too soon could also lead the cluster to a weird state. To address this issue, you can increase request.timeout.ms on the broker.

          We are fixing the known issue in (1) and improving the performance with lots of partitions in (2) in KAFKA-5642 and we expect the fix to be included in the 1.1.0 release in Feb.

          Show
          junrao Jun Rao added a comment - Francesco vigotti , sorry to hear that. A couple of quick suggestions. (1) Do you see any ZK session expiration in the log (e.g., INFO zookeeper state changed (Expired) (org.I0Itec.zkclient.ZkClient))? There are known bugs in Kafka in handling ZK session expiration. So, it would be useful to avoid it in the first place. Typical causes of ZK session expiration are long GC in the broker or network glitches. So you can either tune the broker or increase zookeeper.session.timeout.ms. (2) Do you have lots of partitions (say a few thousands) per broker? If so, you want to check if the controlled shutdown succeeds when shutting down a broker. If not, restarting the broker too soon could also lead the cluster to a weird state. To address this issue, you can increase request.timeout.ms on the broker. We are fixing the known issue in (1) and improving the performance with lots of partitions in (2) in KAFKA-5642 and we expect the fix to be included in the 1.1.0 release in Feb.
          Hide
          fravigotti Francesco vigotti added a comment -

          I've maybe found the problem to my issue which maybe is not related to this topic because in my case simple broker restart didn't worked, I've create a dedicated issue then... https://issues.apache.org/jira/browse/KAFKA-6129

          Show
          fravigotti Francesco vigotti added a comment - I've maybe found the problem to my issue which maybe is not related to this topic because in my case simple broker restart didn't worked, I've create a dedicated issue then... https://issues.apache.org/jira/browse/KAFKA-6129

            People

            • Assignee:
              Unassigned
              Reporter:
              danil Danil Serdyuchenko
            • Votes:
              52 Vote for this issue
              Watchers:
              95 Start watching this issue

              Dates

              • Created:
                Updated:

                Development