kafka-users mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Gwen Shapira <g...@confluent.io>
Subject Re: Kafka 0.9.0.1 failing on new leader election
Date Sat, 30 Jul 2016 01:35:47 GMT
you know, I ran into those null pointer exceptions when I accidentally
tested Kafka with mismatching version of zkclient.

Can you share the versions of both? And make sure you have only one
zkclient on your classpath?

On Tue, Jul 26, 2016 at 6:40 AM, Sean Morris (semorris)
<semorris@cisco.com> wrote:
> I have a setup with 2 brokers and it is going through leader re-election but seems to
fail to complete. The behavior I start to see is that some published succeed but others will
fail with NotLeader exceptions like this
>
>
> java.util.concurrent.ExecutionException: org.apache.kafka.common.errors.NotLeaderForPartitionException:
This server is not the leader for that topic-partition.
>
>         at org.apache.kafka.clients.producer.internals.FutureRecordMetadata.valueOrError(FutureRecordMetadata.java:56)
>
>         at org.apache.kafka.clients.producer.internals.FutureRecordMetadata.get(FutureRecordMetadata.java:43)
>
>         at org.apache.kafka.clients.producer.internals.FutureRecordMetadata.get(FutureRecordMetadata.java:25)
>
>
> My Kafka and zookeeper log file has errors like this
>
>
> [2016-07-26 02:01:12,842] ERROR [kafka.controller.ControllerBrokerRequestBatch] Haven't
been able to send metadata update requests, current state of the map is Map(2 -> Map(eox-1
-> (LeaderAndIsrInfo:(Leader:2,ISR:2,1,LeaderEpoch:46,ControllerEpoch:34),ReplicationFactor:2),AllReplicas:2,1),
notify-eportal-1 -> (LeaderAndIsrInfo:(Leader:2,ISR:2,1,LeaderEpoch:51,ControllerEpoch:34),ReplicationFactor:2),AllReplicas:2,1),
psirts-1 -> (LeaderAndIsrInfo:(Leader:2,ISR:2,1,LeaderEpoch:51,ControllerEpoch:34),ReplicationFactor:2),AllReplicas:2,1),
notify-pushNotif-low-1 -> (LeaderAndIsrInfo:(Leader:2,ISR:2,1,LeaderEpoch:51,ControllerEpoch:34),ReplicationFactor:2),AllReplicas:2,1)),
1 -> Map(eox-1 -> (LeaderAndIsrInfo:(Leader:2,ISR:2,1,LeaderEpoch:46,ControllerEpoch:34),ReplicationFactor:2),AllReplicas:2,1),
notify-eportal-1 -> (LeaderAndIsrInfo:(Leader:2,ISR:2,1,LeaderEpoch:51,ControllerEpoch:34),ReplicationFactor:2),AllReplicas:2,1),
psirts-1 -> (LeaderAndIsrInfo:(Leader:2,ISR:2,1,LeaderEpoch:51,ControllerEpoch:34),ReplicationFactor:2),AllReplicas:2,1),
notify-pushNotif-low-1 -> (LeaderAndIsrInfo:(Leader:2,ISR:2,1,LeaderEpoch:51,ControllerEpoch:34),ReplicationFactor:2),AllReplicas:2,1)))
>
> [2016-07-26 02:01:12,845] ERROR [kafka.controller.KafkaController] [Controller 1]: Forcing
the controller to resign
>
>
> Which is then followed by a null pointer exception
>
>
> [2016-07-26 02:01:13,021] ERROR [org.I0Itec.zkclient.ZkEventThread] Error handling event
ZkEvent[Children of /isr_change_notification changed sent to kafka.controller.IsrChangeNotificationListener@55ca3750]
>
> java.lang.IllegalStateException: java.lang.NullPointerException
>
>         at kafka.controller.ControllerBrokerRequestBatch.sendRequestsToBrokers(ControllerChannelManager.scala:434)
>
>         at kafka.controller.KafkaController.sendUpdateMetadataRequest(KafkaController.scala:1029)
>
>         at kafka.controller.IsrChangeNotificationListener.kafka$controller$IsrChangeNotificationListener$$processUpdateNotifications(KafkaController.scala:1372)
>
>         at kafka.controller.IsrChangeNotificationListener$$anonfun$handleChildChange$1.apply$mcV$sp(KafkaController.scala:1359)
>
>         at kafka.controller.IsrChangeNotificationListener$$anonfun$handleChildChange$1.apply(KafkaController.scala:1352)
>
>         at kafka.controller.IsrChangeNotificationListener$$anonfun$handleChildChange$1.apply(KafkaController.scala:1352)
>
>         at kafka.utils.CoreUtils$.inLock(CoreUtils.scala:262)
>
>         at kafka.controller.IsrChangeNotificationListener.handleChildChange(KafkaController.scala:1352)
>
>         at org.I0Itec.zkclient.ZkClient$10.run(ZkClient.java:842)
>
>         at org.I0Itec.zkclient.ZkEventThread.run(ZkEventThread.java:71)
>
> Caused by: java.lang.NullPointerException
>
>         at kafka.controller.KafkaController.sendRequest(KafkaController.scala:699)
>
>         at kafka.controller.ControllerBrokerRequestBatch$$anonfun$sendRequestsToBrokers$2.apply(ControllerChannelManager.scala:403)
>
>         at kafka.controller.ControllerBrokerRequestBatch$$anonfun$sendRequestsToBrokers$2.apply(ControllerChannelManager.scala:369)
>
>         at scala.collection.mutable.HashMap$$anonfun$foreach$1.apply(HashMap.scala:99)
>
>         at scala.collection.mutable.HashMap$$anonfun$foreach$1.apply(HashMap.scala:99)
>
>         at scala.collection.mutable.HashTable$class.foreachEntry(HashTable.scala:230)
>
>         at scala.collection.mutable.HashMap.foreachEntry(HashMap.scala:40)
>
>         at scala.collection.mutable.HashMap.foreach(HashMap.scala:99)
>
>         at kafka.controller.ControllerBrokerRequestBatch.sendRequestsToBrokers(ControllerChannelManager.scala:369)
>
>         ... 9 more
>
>
> I eventually restarted zookeeper and my brokers. This has happened twice in the last
week. Any ideas?
>
>
> Thanks,
>
> Sean
>

Mime
View raw message