Hi Jun,
Thanks for responding...
I am using Kafka 2.9.2-0.8.1.1
I looked through the controller logs on a couple of nodes and did not find
any exceptions or error.
However in the state change log I see a bunch of the following exceptions -
[2014-10-13 14:39:12,475] TRACE Controller 3 epoch 116 started leader
election for partition [wordcount,1] (state.change.logger)
[2014-10-13 14:39:12,479] ERROR Controller 3 epoch 116 initiated state
change for partition [wordcount,1] from OfflinePartition to OnlinePartition
failed (state.change.logger)
kafka.common.NoReplicaOnlineException: No replica for partition
[wordcount,1] is alive. Live brokers are: [Set()], Assigned replicas are:
[List(8, 7, 1)]
at
kafka.controller.OfflinePartitionLeaderSelector.selectLeader(PartitionLeaderSelector.scala:61)
at
kafka.controller.PartitionStateMachine.electLeaderForPartition(PartitionStateMachine.scala:336)
at
kafka.controller.PartitionStateMachine.kafka$controller$PartitionStateMachine$$handleStateChange(PartitionStateMachine.scala:185)
at
kafka.controller.PartitionStateMachine$$anonfun$triggerOnlinePartitionStateChange$3.apply(PartitionStateMachine.scala:99)
at
kafka.controller.PartitionStateMachine$$anonfun$triggerOnlinePartitionStateChange$3.apply(PartitionStateMachine.scala:96)
at
scala.collection.TraversableLike$WithFilter$$anonfun$foreach$1.apply(TraversableLike.scala:743)
at
scala.collection.mutable.HashMap$$anonfun$foreach$1.apply(HashMap.scala:95)
at
scala.collection.mutable.HashMap$$anonfun$foreach$1.apply(HashMap.scala:95)
at scala.collection.Iterator$class.foreach(Iterator.scala:772)
at
scala.collection.mutable.HashTable$$anon$1.foreach(HashTable.scala:157)
at
scala.collection.mutable.HashTable$class.foreachEntry(HashTable.scala:190)
at scala.collection.mutable.HashMap.foreachEntry(HashMap.scala:45)
at scala.collection.mutable.HashMap.foreach(HashMap.scala:95)
at
scala.collection.TraversableLike$WithFilter.foreach(TraversableLike.scala:742)
at
kafka.controller.PartitionStateMachine.triggerOnlinePartitionStateChange(PartitionStateMachine.scala:96)
at
kafka.controller.PartitionStateMachine.startup(PartitionStateMachine.scala:68)
at
kafka.controller.KafkaController.onControllerFailover(KafkaController.scala:312)
at
kafka.controller.KafkaController$$anonfun$1.apply$mcV$sp(KafkaController.scala:162)
at
kafka.server.ZookeeperLeaderElector.elect(ZookeeperLeaderElector.scala:63)
at
kafka.server.ZookeeperLeaderElector$LeaderChangeListener$$anonfun$handleDataDeleted$1.apply$mcZ$sp(ZookeeperLeaderElector.scala:123)
at
kafka.server.ZookeeperLeaderElector$LeaderChangeListener$$anonfun$handleDataDeleted$1.apply(ZookeeperLeaderElector.scala:118)
at
kafka.server.ZookeeperLeaderElector$LeaderChangeListener$$anonfun$handleDataDeleted$1.apply(ZookeeperLeaderElector.scala:118)
at kafka.utils.Utils$.inLock(Utils.scala:538)
at
kafka.server.ZookeeperLeaderElector$LeaderChangeListener.handleDataDeleted(ZookeeperLeaderElector.scala:118)
at org.I0Itec.zkclient.ZkClient$6.run(ZkClient.java:549)
at org.I0Itec.zkclient.ZkEventThread.run(ZkEventThread.java:71)
Anyways, this morning after sending out the email, I set out to restart all
the brokers. I found that 3 brokers were in a hung state. I tried to use
the bin/kafka-server-stop.sh script (which is nothing but sending a SIGINT
signal), the java process running kafka would not terminate, I then issued
a 'kill -SIGTERM xxxxx' for the java process running Kafka, yet the process
would not terminate. This happened only on 3 nodes (1 node is running only
1 broker). For the other nodes kafka-server-stop.sh successfully bought
down the java process running Kafka.
For the three brokers that was not responding to either SIGINT and SIGTERM
signal I issued a SIGKILL instead and this, for sure brought down the
process.
I then restarted brokers on all nodes. After that I again ran the describe
topic script.
bin/kafka-topics.sh --describe --zookeeper tr-pan-hclstr-08.amers1b.
ciscloud:2181/kafka/kafka-clstr-01 --topic wordcount
Topic:wordcount PartitionCount:8 ReplicationFactor:3 Configs:
Topic: wordcount Partition: 0 Leader: 7 Replicas:
7,6,8 Isr: 6,7,8
Topic: wordcount Partition: 1 Leader: 8 Replicas:
8,7,1 Isr: 1,7,8
Topic: wordcount Partition: 2 Leader: 1 Replicas:
1,8,2 Isr: 1,2,8
Topic: wordcount Partition: 3 Leader: 2 Replicas:
2,1,3 Isr: 1,2,3
Topic: wordcount Partition: 4 Leader: 3 Replicas:
3,2,4 Isr: 2,3,4
Topic: wordcount Partition: 5 Leader: 4 Replicas:
4,3,5 Isr: 3,4,5
Topic: wordcount Partition: 6 Leader: 5 Replicas:
5,4,6 Isr: 4,5,6
Topic: wordcount Partition: 7 Leader: 6 Replicas:
6,5,7 Isr: 5,6,7
Since then it is been running fine.
My conclusion is that for some reason (which I don't really understand), 3
brokers were effectively in a hung state and probably caused the broken
cluster.
Regards,
-Jacob
On Tue, Oct 14, 2014 at 5:39 PM, Jun Rao <[email protected]> wrote:
> Also, which version of Kafka are you using?
>
> Thanks,
>
> Jun
>
> On Tue, Oct 14, 2014 at 5:31 PM, Jun Rao <[email protected]> wrote:
>
> > The following is a bit weird. It indicates no leader for partition 4,
> > which is inconsistent with what describe-topic shows.
> >
> > 2014-10-13 19:02:32,611 WARN [main] kafka.producer.BrokerPartitionInfo:
> > Error while fetching metadata partition 4 leader: none
> replicas: 3
> > (tr-pan-hclstr-13.amers1b.ciscloud:9092),2
> > (tr-pan-hclstr-12.amers1b.ciscloud:9092),4
> > (tr-pan-hclstr-14.amers1b.ciscloud:9092) isr: isUnderReplicated:
> > true for topic partition [wordcount,4]: [class
> > kafka.common.LeaderNotAvailableException]
> >
> > Any error in the controller and the state-change log? Do you see broker 3
> > marked as dead in the controller log? Also, could you check if the broker
> > registration in ZK (
> >
> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+data+structures+in+Zookeeper
> )
> > has the correct host/port?
> >
> > Thanks,
> >
> > Jun
> >
> > On Mon, Oct 13, 2014 at 5:35 PM, Abraham Jacob <[email protected]>
> > wrote:
> >
> >> Hi All,
> >>
> >> I have a 8 node Kafka cluster (broker.id - 1..8). On this cluster I
> have
> >> a
> >> topic "wordcount", which was 8 partitions with a replication factor of
> 3.
> >>
> >> So a describe of topic wordcount
> >> # bin/kafka-topics.sh --describe --zookeeper
> >> tr-pan-hclstr-08.amers1b.ciscloud:2181/kafka/kafka-clstr-01 --topic
> >> wordcount
> >>
> >>
> >> Topic:wordcount PartitionCount:8 ReplicationFactor:3 Configs:
> >> Topic: wordcount Partition: 0 Leader: 6 Replicas: 7,6,8
> >> Isr: 6,7,8
> >> Topic: wordcount Partition: 1 Leader: 7 Replicas: 8,7,1
> >> Isr: 7
> >> Topic: wordcount Partition: 2 Leader: 8 Replicas: 1,8,2
> >> Isr: 8
> >> Topic: wordcount Partition: 3 Leader: 3 Replicas: 2,1,3
> >> Isr: 3
> >> Topic: wordcount Partition: 4 Leader: 3 Replicas: 3,2,4
> >> Isr: 3,2,4
> >> Topic: wordcount Partition: 5 Leader: 3 Replicas: 4,3,5
> >> Isr: 3,5
> >> Topic: wordcount Partition: 6 Leader: 6 Replicas: 5,4,6
> >> Isr: 6,5
> >> Topic: wordcount Partition: 7 Leader: 6 Replicas: 6,5,7
> >> Isr: 6,5,7
> >>
> >> I wrote a simple producer to write to this topic. However when running I
> >> get these messages in the logs -
> >>
> >> 2014-10-13 19:02:32,459 INFO [main] kafka.client.ClientUtils$: Fetching
> >> metadata from broker
> id:0,host:tr-pan-hclstr-11.amers1b.ciscloud,port:9092
> >> with correlation id 0 for 1 topic(s) Set(wordcount)
> >> 2014-10-13 19:02:32,464 INFO [main] kafka.producer.SyncProducer:
> Connected
> >> to tr-pan-hclstr-11.amers1b.ciscloud:9092 for producing
> >> 2014-10-13 19:02:32,551 INFO [main] kafka.producer.SyncProducer:
> >> Disconnecting from tr-pan-hclstr-11.amers1b.ciscloud:9092
> >> 2014-10-13 19:02:32,611 WARN [main] kafka.producer.BrokerPartitionInfo:
> >> Error while fetching metadata partition 4 leader: none
> replicas:
> >> 3
> >> (tr-pan-hclstr-13.amers1b.ciscloud:9092),2
> >> (tr-pan-hclstr-12.amers1b.ciscloud:9092),4
> >> (tr-pan-hclstr-14.amers1b.ciscloud:9092) isr: isUnderReplicated:
> >> true for topic partition [wordcount,4]: [class
> >> kafka.common.LeaderNotAvailableException]
> >> 2014-10-13 19:02:33,505 INFO [main] kafka.producer.SyncProducer:
> Connected
> >> to tr-pan-hclstr-15.amers1b.ciscloud:9092 for producing
> >> 2014-10-13 19:02:33,543 WARN [main]
> >> kafka.producer.async.DefaultEventHandler: Produce request with
> correlation
> >> id 20611 failed due to [wordcount,5]:
> >> kafka.common.NotLeaderForPartitionException,[wordcount,6]:
> >> kafka.common.NotLeaderForPartitionException,[wordcount,7]:
> >> kafka.common.NotLeaderForPartitionException
> >> 2014-10-13 19:02:33,694 INFO [main] kafka.producer.SyncProducer:
> Connected
> >> to tr-pan-hclstr-18.amers1b.ciscloud:9092 for producing
> >> 2014-10-13 19:02:33,725 WARN [main]
> >> kafka.producer.async.DefaultEventHandler: Produce request with
> correlation
> >> id 20612 failed due to [wordcount,0]:
> >> kafka.common.NotLeaderForPartitionException
> >> 2014-10-13 19:02:33,861 INFO [main] kafka.producer.SyncProducer:
> Connected
> >> to tr-pan-hclstr-11.amers1b.ciscloud:9092 for producing
> >> 2014-10-13 19:02:33,983 WARN [main]
> >> kafka.producer.async.DefaultEventHandler: Failed to send data since
> >> partitions [wordcount,4] don't have a leader
> >>
> >>
> >> Obviously something is terribly wrong... I am quite new to Kafka, hence
> >> these messages don't make any sense to me, except for the fact that it
> is
> >> telling me that some of the partitions don't have any leader.
> >>
> >> Could somebody be kind enough to explain the above message?
> >>
> >> A few more questions -
> >>
> >> (1) How does one get into this state?
> >> (2) How can I get out of this state?
> >> (3) I have set auto.leader.rebalance.enable=true on all brokers.
> Shouldn't
> >> the partitions be balanced across all the brokers?
> >> (4) I can see that the Kafka service are running on all 8 nodes. (I used
> >> ps ax -o "pid pgid args" and I can see under the kafka Java process).
> >> (5) Is there a way I can force a re-balance?
> >>
> >>
> >>
> >> Regards,
> >> Jacob
> >>
> >>
> >>
> >> --
> >> ~
> >>
> >
> >
>
--
~