[ https://issues.apache.org/jira/browse/KAFKA-464?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13436793#comment-13436793 ]
Jun Rao commented on KAFKA-464: ------------------------------- A couple of more comments: 1. The following check in handleNewSession() should probably be done inside controllerRegisterOrFailover() since other callers of controllerRegisterOrFailover() should check isRunning too. if(isRunning) controllerRegisterOrFailover() 2. This is not really related to this jira, but it's in KafkaController. So, maybe we can fix it in this jira too. It seems that we can simplify the following code in controllerRegisterOrFailover() using filter() on a map. // If there are some partition with leader not initialized, init the leader for them val partitionReplicaAssignment = allPartitionReplicaAssignment.clone() for((topicPartition, replicas) <- partitionReplicaAssignment){ if (allLeaders.contains(topicPartition)){ partitionReplicaAssignment.remove(topicPartition) } } > KafkaController NPE in SessionExpireListener > -------------------------------------------- > > Key: KAFKA-464 > URL: https://issues.apache.org/jira/browse/KAFKA-464 > Project: Kafka > Issue Type: Bug > Components: core > Reporter: Jun Rao > Labels: bugs > Fix For: 0.8 > > Attachments: kafka_464.diff, kafka_464_v2.diff, kafka_464_v3.diff > > Original Estimate: 48h > Remaining Estimate: 48h > > Sometime see the following in LogRecoverTest. > [2012-08-15 09:06:01,845] ERROR Error handling event ZkEvent[New session > event sent to kafka.server.KafkaController$SessionExpireListener@e8ae59a] > (org.I0Itec.zkclient.ZkEventThread) > java.lang.NullPointerException > at > kafka.server.KafkaController$SessionExpireListener.handleNewSession(KafkaController.scala:284) > at org.I0Itec.zkclient.ZkClient$4.run(ZkClient.java:472) > at org.I0Itec.zkclient.ZkEventThread.run(ZkEventThread.java:71) -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira