You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by "ASF GitHub Bot (JIRA)" <ji...@apache.org> on 2017/12/22 01:38:00 UTC

[jira] [Commented] (KAFKA-3400) Topic stop working / can't describe topic

    [ https://issues.apache.org/jira/browse/KAFKA-3400?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16300827#comment-16300827 ] 

ASF GitHub Bot commented on KAFKA-3400:
---------------------------------------

ijuma closed pull request #1086: KAFKA-3400: Fix describe topic in case there are zero partitions.
URL: https://github.com/apache/kafka/pull/1086
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git a/core/src/main/scala/kafka/admin/TopicCommand.scala b/core/src/main/scala/kafka/admin/TopicCommand.scala
index e89e09d2a3b..19b57ad949b 100755
--- a/core/src/main/scala/kafka/admin/TopicCommand.scala
+++ b/core/src/main/scala/kafka/admin/TopicCommand.scala
@@ -205,7 +205,10 @@ object TopicCommand extends Logging {
             val configs = AdminUtils.fetchEntityConfig(zkUtils, ConfigType.Topic, topic)
             if (!reportOverriddenConfigs || configs.size() != 0) {
               val numPartitions = topicPartitionAssignment.size
-              val replicationFactor = topicPartitionAssignment.head._2.size
+              val replicationFactor =
+                if (numPartitions > 0)
+                  topicPartitionAssignment.head._2.size
+                else 0
               println("Topic:%s\tPartitionCount:%d\tReplicationFactor:%d\tConfigs:%s"
                 .format(topic, numPartitions, replicationFactor, configs.map(kv => kv._1 + "=" + kv._2).mkString(",")))
             }
@@ -216,7 +219,7 @@ object TopicCommand extends Logging {
               val leader = zkUtils.getLeaderForPartition(topic, partitionId)
               if ((!reportUnderReplicatedPartitions && !reportUnavailablePartitions) ||
                   (reportUnderReplicatedPartitions && inSyncReplicas.size < assignedReplicas.size) ||
-                  (reportUnavailablePartitions && (!leader.isDefined || !liveBrokers.contains(leader.get)))) {
+                  (reportUnavailablePartitions && (leader.isEmpty || !liveBrokers.contains(leader.get)))) {
                 print("\tTopic: " + topic)
                 print("\tPartition: " + partitionId)
                 print("\tLeader: " + (if(leader.isDefined) leader.get else "none"))


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


> Topic stop working / can't describe topic
> -----------------------------------------
>
>                 Key: KAFKA-3400
>                 URL: https://issues.apache.org/jira/browse/KAFKA-3400
>             Project: Kafka
>          Issue Type: Bug
>    Affects Versions: 0.9.0.1
>            Reporter: Tobias
>            Assignee: Ashish Singh
>
> we are seeing an issue were we intermittently (every couple of hours) get and error with certain topics. They stop working and producers give a LeaderNotFoundException.
> When we then try to use kafka-topics.sh to describe the topic we get the error below.
> Error while executing topic command : next on empty iterator
> {{
> [2016-03-15 17:30:26,231] ERROR java.util.NoSuchElementException: next on empty iterator
> 	at scala.collection.Iterator$$anon$2.next(Iterator.scala:39)
> 	at scala.collection.Iterator$$anon$2.next(Iterator.scala:37)
> 	at scala.collection.IterableLike$class.head(IterableLike.scala:91)
> 	at scala.collection.AbstractIterable.head(Iterable.scala:54)
> 	at kafka.admin.TopicCommand$$anonfun$describeTopic$1.apply(TopicCommand.scala:198)
> 	at kafka.admin.TopicCommand$$anonfun$describeTopic$1.apply(TopicCommand.scala:188)
> 	at scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59)
> 	at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:47)
> 	at kafka.admin.TopicCommand$.describeTopic(TopicCommand.scala:188)
> 	at kafka.admin.TopicCommand$.main(TopicCommand.scala:66)
> 	at kafka.admin.TopicCommand.main(TopicCommand.scala)
>  (kafka.admin.TopicCommand$)
> }}
> if we delete the topic, then it will start to work again for a while
> We can't see anything obvious in the logs but are happy to provide if needed



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)