[ https://issues.apache.org/jira/browse/KAFKA-790?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]
Neha Narkhede resolved KAFKA-790. --------------------------------- Resolution: Won't Fix This happens in cases when the broker becomes follower faster than another broker becomes leader. The root cause is the become-follower state change design, where we don't wait until the other broker finishes becoming leader and is ready to serve the replica fetch requests. To get rid of this exception, the controller would have to send a "start-replica" state change request to the followers after it has heard from the leaders about the become-leader completion. This would reduce the occurrence of these fake UnknownTopicOrPartitionException significantly. However, even after the controller receives the response for become-leader and after it sends the start-replica to the followers, the leader can die and we can get the same errors in the logs. One can argue that this scenario is very rare and the fix will work in most cases. On the other hand, implementing the fix will be somewhat complex on the controller since now it has to keep track of the outstanding leader and isr requests and send the start replica state change accordingly. I think we can wait and see how annoying this error message gets and then thinking about fixing it. Thoughts ? > Kafka server throws UnknownTopicOrPartitionException right after becoming > leader for a new partition > ---------------------------------------------------------------------------------------------------- > > Key: KAFKA-790 > URL: https://issues.apache.org/jira/browse/KAFKA-790 > Project: Kafka > Issue Type: Bug > Components: replication > Affects Versions: 0.8 > Reporter: Neha Narkhede > Assignee: Neha Narkhede > Priority: Blocker > Labels: kafka-0.8, p1 > > 2013/03/05 01:24:01.207 INFO [ReplicaManager] [kafka-request-handler-5] > [kafka] [] Replica Manager on Broker 273: Completed leader and isr request > Name: LeaderAndIsrRequest; Version: 0; CorrelationId: 15 > 38; ClientId: ; AckTimeoutMs: 1000 ms; ControllerEpoch: 3; > PartitionStateInfo: (foo,7) -> > PartitionStateInfo(LeaderIsrAndControllerEpoch({ "ISR":"273,271", > "leader":"273", "leaderEpoch":"0" },3),2),(foo,4) -> > PartitionStateInfo(LeaderIsrAndControllerEpoch({ "ISR":"270,273", > "leader":"270", "leaderEpoch":"0" },3),2),(foo,1) -> > PartitionStateInfo(LeaderIsrAndControllerEpoch({ "ISR":"273,270", > "leader":"273", "leaderEpoch":"0" },3),2); Leaders: > id:270,host:host270,port:10251,id:273,host:host273 > ,port:10251 > 2013/03/05 01:24:01.473 INFO [Processor] [kafka-processor-10251-4] [kafka] [] > Closing socket connection to /172.20.72.66. > 2013/03/05 01:24:01.505 INFO [Processor] [kafka-processor-10251-4] [kafka] [] > Closing socket connection to /172.20.72.244. > 2013/03/05 01:24:01.521 INFO [Processor] [kafka-processor-10251-5] [kafka] [] > Closing socket connection to /172.20.72.243. > 2013/03/05 01:24:01.555 INFO [Processor] [kafka-processor-10251-3] [kafka] [] > Closing socket connection to /172.20.72.11. > 2013/03/05 01:24:01.568 INFO [Processor] [kafka-processor-10251-1] [kafka] [] > Closing socket connection to /172.20.72.66. > 2013/03/05 01:24:01.606 INFO [Processor] [kafka-processor-10251-0] [kafka] [] > Closing socket connection to /172.20.72.66. > 2013/03/05 01:24:01.758 INFO [Processor] [kafka-processor-10251-0] [kafka] [] > Closing socket connection to /172.20.72.66. > 2013/03/05 01:24:01.776 ERROR [KafkaApis] [kafka-request-handler-7] [kafka] > [] [KafkaApi-273] error when processing request (foo,1,0,1048576) > kafka.common.UnknownTopicOrPartitionException: Topic foo partition 1 doesn't > exist on 273 > at > kafka.server.ReplicaManager.getLeaderReplicaIfLocal(ReplicaManager.scala:170) > at > kafka.server.KafkaApis.kafka$server$KafkaApis$$readMessageSet(KafkaApis.scala:301) > at > kafka.server.KafkaApis$$anonfun$kafka$server$KafkaApis$$readMessageSets$1.apply(KafkaApis.scala:268) > at > kafka.server.KafkaApis$$anonfun$kafka$server$KafkaApis$$readMessageSets$1.apply(KafkaApis.scala:264) > at > scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:206) > at > scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:206) > at scala.collection.immutable.Map$Map2.foreach(Map.scala:127) > at > scala.collection.TraversableLike$class.map(TraversableLike.scala:206) > at scala.collection.immutable.Map$Map2.map(Map.scala:110) > at > kafka.server.KafkaApis.kafka$server$KafkaApis$$readMessageSets(KafkaApis.scala:264) > at kafka.server.KafkaApis.handleFetchRequest(KafkaApis.scala:231) > at kafka.server.KafkaApis.handle(KafkaApis.scala:61) > at kafka.server.KafkaRequestHandler.run(KafkaRequestHandler.scala:41) > at java.lang.Thread.run(Thread.java:619) -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira