Hi, @Jiangjie Qin this is the related info from controller.log:
[2015-03-11 10:54:11,962] ERROR [Controller 0]: Error completing reassignment of partition [ad_click_sts,3] (kafka.controller.KafkaController) kafka.common.KafkaException: Partition [ad_click_sts,3] to be reassigned is already assigned to replicas 0,1. Ignoring request for partition reassignment at kafka.controller.KafkaController.initiateReassignReplicasForTopicPartition(KafkaController.scala:585) It seems like kafka ignore the kafka-reassign-partitions.sh command. the json file used in command is : {"version":1,"partitions":[{"topic":"ad_click_sts","partition":3,"replicas":[0,1]}]} The partition has lost sync replication in practice : Topic: ad_click_sts Partition: 3 Leader: 0 Replicas: 0,1 Isr: 0 Regards sy.pan > 在 2015年3月11日,12:32,Jiangjie Qin <j...@linkedin.com.INVALID> 写道: > > It looks that in your case it is because broker 1 somehow missed a > controller LeaderAndIsrRequest for [ad_click_sts,4]. So the zkVersion > would be different from the value stored in zookeeper from that on. > Therefore broker 1 failed to update ISR. In this case you have to bounce > broker to fix it. > From what you posted, it looks both broker 0 and broker 1 are having this > issue. So the question is how could both broker missed a controller > LeaderAndIsrRequest. Is there anything interesting in controller.log? > > Jiangjie (Becket) Qin > > On 3/10/15, 8:33 PM, "sy.pan" <shengyi....@gmail.com > <mailto:shengyi....@gmail.com>> wrote: > >> @tao xiao and Jiangjie Qin, Thank you very much >> >> I try to run kafka-reassign-partitions.sh, but the issue still exists… >> >> this the log info: >> >> [2015-03-11 11:00:40,086] ERROR Conditional update of path >> /brokers/topics/ad_click_sts/partitions/4/state with data >> {"controller_epoch":23,"leader":1,"version":1,"leader_epoch":35,"isr":[1,0 >> ]} and expected version 564 failed due to >> org.apache.zookeeper.KeeperException$BadVersionException: KeeperErrorCode >> = BadVersion for /brokers/topics/ad_click_sts/partitions/4/state >> (kafka.utils.ZkUtils$) >> >> [2015-03-11 11:00:40,086] INFO Partition [ad_click_sts,4] on broker 1: >> Cached zkVersion [564] not equal to that in zookeeper, skip updating ISR >> (kafka.cluster.Partition) >>>>>>>>>>>>>>>>>>>> >> >> finally, I had to restart the kafka node and the Isr problem is fixed, is >> there any better ways? >> >> Regards >> sy.pan >> >> >>> 在 2015年3月11日,03:34,Jiangjie Qin <j...@linkedin.com.INVALID> 写道: >>> >>> This looks like a leader broker somehow did not respond to a fetch >>> request >>> from the follower. It may be because the broker was too busy. If that is >>> the case, Xiao¹s approach could help - reassign partitions or reelect >>> leaders to balance the traffic among brokers. >>> >>> Jiangjie (Becket) Qin >>> >>> On 3/9/15, 8:31 PM, "sy.pan" <shengyi....@gmail.com >>> <mailto:shengyi....@gmail.com <mailto:shengyi....@gmail.com>>> wrote: >>> >>>> Hi, tao xiao and Jiangjie Qin >>>> >>>> I encounter with the same issue, my node had recovered from high load >>>> problem (caused by other application) >>>> >>>> this is the kafka-topic show: >>>> >>>> Topic:ad_click_sts PartitionCount:6 ReplicationFactor:2 Configs: >>>> Topic: ad_click_sts Partition: 0 Leader: 1 Replicas: 1,0 >>>> Isr: 1 >>>> Topic: ad_click_sts Partition: 1 Leader: 0 Replicas: 0,1 >>>> Isr: 0 >>>> Topic: ad_click_sts Partition: 2 Leader: 1 Replicas: 1,0 >>>> Isr: 1 >>>> Topic: ad_click_sts Partition: 3 Leader: 0 Replicas: 0,1 >>>> Isr: 0 >>>> Topic: ad_click_sts Partition: 4 Leader: 1 Replicas: 1,0 >>>> Isr: 1 >>>> Topic: ad_click_sts Partition: 5 Leader: 0 Replicas: 0,1 >>>> Isr: 0 >>>> >>>> ReplicaFetcherThread info extracted from kafka server.log : >>>> >>>> [2015-03-09 21:06:05,450] ERROR [ReplicaFetcherThread-0-0], Error in >>>> fetch Name: FetchRequest; Version: 0; CorrelationId: 7331; ClientId: >>>> ReplicaFetcherThread-0-0; ReplicaId: 1; MaxWait: 500 ms; MinBytes: 1 >>>> bytes; RequestInfo: [ad_click_sts,5] -> >>>> PartitionFetchInfo(6149699,1048576),[ad_click_sts,3] -> >>>> PartitionFetchInfo(6147835,1048576),[ad_click_sts,1] -> >>>> PartitionFetchInfo(6235071,1048576) (kafka.server.ReplicaFetcherThread) >>>> java.net.SocketTimeoutException >>>> at >>>> sun.nio.ch.SocketAdaptor$SocketInputStream.read(SocketAdaptor.java:201) >>>> at sun.nio.ch.ChannelInputStream.read(ChannelInputStream.java:86) >>>> ŠŠ.. >>>> at >>>> >>>> kafka.consumer.SimpleConsumer$$anonfun$fetch$1.apply$mcV$sp(SimpleConsum >>>> er >>>> .scala:108) >>>> at >>>> >>>> kafka.consumer.SimpleConsumer$$anonfun$fetch$1.apply(SimpleConsumer.scal >>>> a: >>>> 108) >>>> at >>>> >>>> kafka.consumer.SimpleConsumer$$anonfun$fetch$1.apply(SimpleConsumer.scal >>>> a: >>>> 108) >>>> at kafka.metrics.KafkaTimer.time(KafkaTimer.scala:33) >>>> at kafka.consumer.SimpleConsumer.fetch(SimpleConsumer.scala:107) >>>> at >>>> >>>> kafka.server.AbstractFetcherThread.processFetchRequest(AbstractFetcherTh >>>> re >>>> ad.scala:96) >>>> at >>>> >>>> kafka.server.AbstractFetcherThread.doWork(AbstractFetcherThread.scala:88 >>>> ) >>>> at >>>> kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:51) >>>> >>>> [2015-03-09 21:06:05,450] WARN Reconnect due to socket error: null >>>> (kafka.consumer.SimpleConsumer) >>>> >>>> [2015-03-09 21:05:57,116] INFO Partition [ad_click_sts,4] on broker 1: >>>> Cached zkVersion [556] not equal to that in zookeeper, skip updating >>>> ISR >>>> (kafka.cluster.Partition) >>>> >>>> [2015-03-09 21:06:05,772] INFO Partition [ad_click_sts,2] on broker 1: >>>> Shrinking ISR for partition [ad_click_sts,2] from 1,0 to 1 >>>> (kafka.cluster.Partition) >>>> >>>> >>>> How to fix this Isr problem ? Is there some command can be run ? >>>> >>>> Regards >>>> sy.pan