This seems similar to behavior we’re seeing. At some point one of our brokers (id 1) just gives up and starts throwing those errors and kafka-topics no longer lists it as a ISR. However the logs for that broker say something very odd:
[2014-05-09 10:16:00,248] INFO Partition [callbackServiceTopic-High,8] on broker 1: Cached zkVersion [10] not equal to that in zookeeper, skip updating ISR (kafka.cluster.Partition) [2014-05-09 10:16:00,248] INFO Partition [callbackServiceTopic,3] on broker 1: Shrinking ISR for partition [callbackServiceTopic,3] from 1,2,3 to 1 (kafka.cluster.Partition) [2014-05-09 10:16:00,251] ERROR Conditional update of path /brokers/topics/callbackServiceTopic/partitions/3/state with data {"controller_epoch":4,"leader":1,"version":1,"leader_epoch":4,"isr":[1]} and expected version 9 failed due to org.apache.zookeeper.KeeperException$BadVersionException: KeeperErrorCode = BadVersion for /brokers/topics/callbackServiceTopic/partitions/3/state (kafka.utils.ZkUtils$) [2014-05-09 10:16:00,251] INFO Partition [callbackServiceTopic,3] on broker 1: Cached zkVersion [9] not equal to that in zookeeper, skip updating ISR (kafka.cluster.Partition) [2014-05-09 10:16:00,251] INFO Partition [callbackServiceTopic-High,31] on broker 1: Shrinking ISR for partition [callbackServiceTopic-High,31] from 1,2,3 to 1 (kafka.cluster.Partition) [2014-05-09 10:16:00,255] ERROR Conditional update of path /brokers/topics/callbackServiceTopic-High/partitions/31/state with data {"controller_epoch":4,"leader":1,"version":1,"leader_epoch":4,"isr":[1]} and expected version 9 failed due to org.apache.zookeeper.KeeperException$BadVersionException: KeeperErrorCode = BadVersion for /brokers/topics/callbackServiceTopic-High/partitions/31/state (kafka.utils.ZkUtils$) [2014-05-09 10:16:00,255] INFO Partition [callbackServiceTopic-High,31] on broker 1: Cached zkVersion [9] not equal to that in zookeeper, skip updating ISR (kafka.cluster.Partition) [2014-05-09 10:16:00,255] INFO Partition [callbackServiceTopic-Low,3] on broker 1: Shrinking ISR for partition [callbackServiceTopic-Low,3] from 1,2,3 to 1 (kafka.cluster.Partition) [2014-05-09 10:16:00,258] ERROR Conditional update of path /brokers/topics/callbackServiceTopic-Low/partitions/3/state with data {"controller_epoch":4,"leader":1,"version":1,"leader_epoch":4,"isr":[1]} and expected version 9 failed due to org.apache.zookeeper.KeeperException$BadVersionException: KeeperErrorCode = BadVersion for /brokers/topics/callbackServiceTopic-Low/partitions/3/state (kafka.utils.ZkUtils$) etc. And these errors continue every few seconds. kafka-topics.sh —describe output: Topic:callbackServiceTopic-High PartitionCount:50 ReplicationFactor:3 Configs: Topic: callbackServiceTopic-High Partition: 0 Leader: 2 Replicas: 3,1,2 Isr: 2,3 Topic: callbackServiceTopic-High Partition: 1 Leader: 2 Replicas: 1,2,3 Isr: 2,3 Topic: callbackServiceTopic-High Partition: 2 Leader: 2 Replicas: 2,3,1 Isr: 2,3 I went and looked at one of those ZNodes in the zkCLI and found this: [zk: localhost:2181(CONNECTED) 2] get /brokers/topics/callbackServiceTopic-High/partitions/31/state {"controller_epoch":5,"leader":2,"version":1,"leader_epoch":5,"isr":[2,3]} What does the version number there represent and how does it get out of sync? Should I restart broker 1? Is the fact that broker 1 is behind in leader_epoch significant? Still trying to figure out Kafka operations :( —Ian On Apr 24, 2014, at 9:26 PM, 陈小军 <chenxiao...@nhn.com> wrote: > I don't do any partition reassignment. > > When broker occure following error, this phenomenon will happen. > > [hadoop@nelo76 libs]$ [2014-03-14 12:11:44,310] INFO Partition > [nelo2-normal-logs,0] on broker 0: Shrinking ISR for partition > [nelo2-normal-logs,0] from 0,1 to 0 (kafka.cluster.Partition) > [2014-03-14 12:11:44,313] ERROR Conditional update of path > /brokers/topics/nelo2-normal-logs/partitions/0/state with data > {"controller_epoch":4,"leader":0,"version":1,"leader_epoch":5,"isr":[0]} and > expected version 7 failed due to > org.apache.zookeeper.KeeperException$BadVersionException: KeeperErrorCode = > BadVersion for /brokers/topics/nelo2-normal-logs/partitions/0/state > (kafka.utils.ZkUtils$) > [2014-03-14 12:11:44,313] INFO Partition [nelo2-normal-logs,0] on broker 0: > Cached zkVersion [7] not equal to that in zookeeper, skip updating ISR > (kafka.cluster.Partition) > [2014-03-14 12:11:44,313] INFO Partition [nelo2-symbolicated-logs,1] on > broker 0: Shrinking ISR for partition [nelo2-symbolicated-logs,1] from 0,2 to > 0 (kafka.cluster.Partition) > [2014-03-14 12:11:44,315] ERROR Conditional update of path > /brokers/topics/nelo2-symbolicated-logs/partitions/1/state with data > {"controller_epoch":4,"leader":0,"version":1,"leader_epoch":6,"isr":[0]} and > expected version 8 failed due to > org.apache.zookeeper.KeeperException$BadVersionException: KeeperErrorCode = > BadVersion for /brokers/topics/nelo2-symbolicated-logs/partitions/1/state > (kafka.utils.ZkUtils$) > [2014-03-14 12:11:44,315] INFO Partition [nelo2-symbolicated-logs,1] on > broker 0: Cached zkVersion [8] not equal to that in zookeeper, skip updating > ISR (kafka.cluster.Partition) > [2014-03-14 12:11:44,316] INFO Partition [nelo2-crash-logs,1] on broker 0: > Shrinking ISR for partition [nelo2-crash-logs,1] from 0,1 to 0 > (kafka.cluster.Partition) > [2014-03-14 12:11:44,318] ERROR Conditional update of path > /brokers/topics/nelo2-crash-logs/partitions/1/state with data > {"controller_epoch":4,"leader":0,"version":1,"leader_epoch":5,"isr":[0]} and > expected version 7 failed due to > org.apache.zookeeper.KeeperException$BadVersionException: KeeperErrorCode = > BadVersion for /brokers/topics/nelo2-crash-logs/partitions/1/state > (kafka.utils.ZkUtils$) > [2014-03-14 12:11:44,318] INFO Partition [nelo2-crash-logs,1] on broker 0: > Cached zkVersion [7] not equal to that in zookeeper, skip updating ISR > (kafka.cluster.Partit > Best Regards > Jerry > -----Original Message----- > From: "Jun Rao"<jun...@gmail.com> > To: "users@kafka.apache.org"<users@kafka.apache.org>; > "陈小军"<chenxiao...@nhn.com>; > Cc: > Sent: 2014-04-25 (星期五) 02:12:02 > Subject: Re: question about isr > > Interesting. Which version of Kafka are you using? Were you doing some > partition reassignment? Thanks, Jun > > > On Wed, Apr 23, 2014 at 11:14 PM, 陈小军 <chenxiao...@nhn.com> wrote: > > Hi Team, > > I found a strange phenomenon of isr list in my kafka cluster > > > > When I use the tool that kafka provide to get the topic information, and it > show isr list as following, seem it is ok > > > > [irt...@xseed171.kdev bin]$ ./kafka-topics.sh --describe --zookeeper > 10.96.250.215:10013,10.96.250.216:10013,10.96.250.217:10013/nelo2-kafka > > > > > Topic:nelo2-normal-logs PartitionCount:3 ReplicationFactor:2 > Configs: > > Topic: nelo2-normal-logs Partition: 0 Leader: 3 > Replicas: 3,0 Isr: 0,3 > > Topic: nelo2-normal-logs Partition: 1 Leader: 0 > Replicas: 0,1 Isr: 0,1 > > Topic: nelo2-normal-logs Partition: 2 Leader: 1 > Replicas: 1,3 Isr: 1,3 > > > > but when I use some sdk to get the meta info from broker, the isr is > different. > > metadata: { size: 246, > > correlationId: 0, > > brokerNum: -1, > > nodeId: 1, > > host: 'xseed171.kdev.nhnsystem.com', > > port: 9093, > > topicNum: 0, > > topicError: 0, > > topic: 'nelo2-normal-logs', > > partitionNum: 2, > > errorCode: 0, > > partition: 0, > > leader: 3, > > replicasNum: 2, > > replicas: [ 3, 0 ], > > isrNum: 2, > > isr: [ 0, 3 ] } > > metadata: { size: 246, > > correlationId: 0, > > brokerNum: -1, > > nodeId: 1, > > host: 'xseed171.kdev.nhnsystem.com', > > port: 9093, > > topicNum: 0, > > topicError: 0, > > topic: 'nelo2-normal-logs', > > partitionNum: 1, > > errorCode: 0, > > partition: 1, > > leader: 0, > > replicasNum: 2, > > replicas: [ 0, 1 ], > > isrNum: 2, > > isr: [ 0, 1 ] } > > metadata: { size: 246, > > correlationId: 0, > > brokerNum: -1, > > nodeId: 1, > > host: 'xseed171.kdev.nhnsystem.com', > > port: 9093, > > topicNum: 0, > > topicError: 0, > > topic: 'nelo2-normal-logs', > > partitionNum: 0, > > errorCode: 0, > > partition: 2, > > leader: 1, > > replicasNum: 2, > > replicas: [ 1, 3 ], > > isrNum: 1, > > isr: [ 1 ] } > > > > I also use other sdk, get the same result. I check the logs from kafka, it > seems the sdk result is right. the tool get the wrong result. why is it > happend? > > > > [2014-04-24 14:53:57,705] TRACE Broker 3 cached leader info > (LeaderAndIsrInfo:(Leader:0,ISR:0,1,LeaderEpoch:7,ControllerEpoch:9),ReplicationFactor:2),AllReplicas:0,1) > for partition [nelo2-normal-logs,1] in response to UpdateMetadata request > sent by controller 0 epoch 10 with correlation id 13 (state.change.logger) > > > [2014-04-24 14:53:57,705] TRACE Broker 3 cached leader info > (LeaderAndIsrInfo:(Leader:1,ISR:1,LeaderEpoch:9,ControllerEpoch:10),ReplicationFactor:2),AllReplicas:1,3) > for partition [nelo2-normal-logs,2] in response to UpdateMetadata request > sent by controller 0 epoch 10 with correlation id 13 (state.change.logger) > > > [2014-04-24 14:53:57,705] TRACE Broker 3 cached leader info > (LeaderAndIsrInfo:(Leader:3,ISR:0,3,LeaderEpoch:10,ControllerEpoch:10),ReplicationFactor:2),AllReplicas:3,0) > for partition [nelo2-normal-logs,0] in response to UpdateMetadata request > sent by controller 0 epoch 10 with correlation id 13 (state.change.logger) > > > > > Thanks~! > > > > Best Regards > > Jerry > > > >