Kafka
  1. Kafka
  2. KAFKA-690

TopicMetadataRequest throws exception when no topics are specified

    Details

    • Type: Bug Bug
    • Status: Resolved
    • Priority: Major Major
    • Resolution: Fixed
    • Affects Version/s: 0.8.0
    • Fix Version/s: 0.8.0
    • Component/s: core
    • Labels:
      None

      Description

      If no topics are sent in a TopicMetadataRequest, `readFrom` throws an exception when trying to get the the head of the topic list for a debug statement.

      java.util.NoSuchElementException: head of empty list
      at scala.collection.immutable.Nil$.head(List.scala:386)
      at scala.collection.immutable.Nil$.head(List.scala:383)
      at kafka.api.TopicMetadataRequest$$anonfun$readFrom$2.apply(TopicMetadataRequest.scala:43)
      at kafka.api.TopicMetadataRequest$$anonfun$readFrom$2.apply(TopicMetadataRequest.scala:43)
      at kafka.utils.Logging$class.debug(Logging.scala:51)
      at kafka.api.TopicMetadataRequest$.debug(TopicMetadataRequest.scala:25)
      at kafka.api.TopicMetadataRequest$.readFrom(TopicMetadataRequest.scala:43)
      at kafka.api.RequestKeys$$anonfun$4.apply(RequestKeys.scala:37)
      at kafka.api.RequestKeys$$anonfun$4.apply(RequestKeys.scala:37)
      at kafka.network.RequestChannel$Request.<init>(RequestChannel.scala:47)
      at kafka.network.Processor.read(SocketServer.scala:320)
      at kafka.network.Processor.run(SocketServer.scala:231)
      at java.lang.Thread.run(Thread.java:680)

      1. KAFKA-690.patch
        5 kB
        David Arthur

        Issue Links

          Activity

          David Arthur created issue -
          Hide
          David Arthur added a comment -

          Removing the debug statement fixes the issue. However, it seems instead of returning all of the topic metadata - I get none.

          Here are some trace logs when I send a TopicMetadataRequest with no topics

          [2013-01-08 19:53:34,463] TRACE 26 bytes read from /127.0.0.1:61558 (kafka.network.Processor)
          [2013-01-08 19:53:34,465] TRACE Received request: TopicMetadataRequest(0,kafka-python,List(),0) (kafka.network.RequestChannel$)
          [2013-01-08 19:53:34,466] TRACE Recieved request, sending for processing by handler: Request(1,sun.nio.ch.SelectionKeyImpl@785e8d7d,java.nio.HeapByteBuffer[pos=0 lim=26 cap=26],1357692814463) (kafka.network.Processor)
          [2013-01-08 19:53:34,466] DEBUG [Kafka Request Handler 1 on Broker 0], handles request Request(1,sun.nio.ch.SelectionKeyImpl@785e8d7d,java.nio.HeapByteBuffer[pos=0 lim=26 cap=26],1357692814463) (kafka.server.KafkaRequestHandler)
          [2013-01-08 19:53:34,466] TRACE Handling topic metadata request TopicMetadataRequest(0,kafka-python,List(),0) (kafka.request.logger)
          [2013-01-08 19:53:34,466] TRACE [KafkaApi-0] Handling topic metadata request TopicMetadataRequest(0,kafka-python,List(),0) (kafka.server.KafkaApis)
          [2013-01-08 19:53:34,475] TRACE Socket server received response to send, registering for write: Response(1,Request(1,sun.nio.ch.SelectionKeyImpl@785e8d7d,java.nio.HeapByteBuffer[pos=0 lim=26 cap=26],1357692814463),kafka.network.BoundedByteBufferSend@2e82674b) (kafka.network.Processor)
          [2013-01-08 19:53:34,476] TRACE 16 bytes written to /127.0.0.1:61558 (kafka.network.Processor)
          [2013-01-08 19:53:34,476] TRACE Completed request: TopicMetadataRequest(0,kafka-python,List(),0) totalTime:13 queueTime:3 localTime:9 remoteTime:0 sendTime:1 (kafka.network.RequestChannel$)

          I would expect this to return info about the one existing topic "test".

          Here are the logs for when I send a TopicMetadataRequest with the "test" topic (this gives the expected response):

          [2013-01-08 19:55:26,247] TRACE 32 bytes read from /127.0.0.1:61719 (kafka.network.Processor)
          [2013-01-08 19:55:26,247] TRACE Received request: TopicMetadataRequest(0,kafka-python,List(test),0) (kafka.network.RequestChannel$)
          [2013-01-08 19:55:26,247] TRACE Recieved request, sending for processing by handler: Request(0,sun.nio.ch.SelectionKeyImpl@30d647d8,java.nio.HeapByteBuffer[pos=0 lim=32 cap=32],1357692926247) (kafka.network.Processor)
          [2013-01-08 19:55:26,247] DEBUG [Kafka Request Handler 0 on Broker 0], handles request Request(0,sun.nio.ch.SelectionKeyImpl@30d647d8,java.nio.HeapByteBuffer[pos=0 lim=32 cap=32],1357692926247) (kafka.server.KafkaRequestHandler)
          [2013-01-08 19:55:26,247] TRACE Handling topic metadata request TopicMetadataRequest(0,kafka-python,List(test),0) (kafka.request.logger)

          [2013-01-08 19:55:26,247] TRACE [KafkaApi-0] Handling topic metadata request TopicMetadataRequest(0,kafka-python,List(test),0) (kafka.server.KafkaApis)

          [2013-01-08 19:55:26,248] DEBUG Reading reply sessionid:0x13c15a1d848003f, packet:: clientPath:null serverPath:null finished:false header:: 30,3 replyHeader:: 30,354,0 request:: '/kafka/brokers/topics/test,F response:: s

          {320,320,1357689749289,1357689749289,0,1,0,0,14,1,323} (org.apache.zookeeper.ClientCnxn)
          [2013-01-08 19:55:26,249] DEBUG Reading reply sessionid:0x13c15a1d848003f, packet:: clientPath:null serverPath:null finished:false header:: 31,4 replyHeader:: 31,354,0 request:: '/kafka/brokers/topics/test,F response:: #7b202230223a205b2230225d207d,s{320,320,1357689749289,1357689749289,0,1,0,0,14,1,323}

          (org.apache.zookeeper.ClientCnxn)
          [2013-01-08 19:55:26,252] DEBUG Partition map for /brokers/topics/test is Map(0 -> List(0)) (kafka.utils.ZkUtils$)
          [2013-01-08 19:55:26,255] DEBUG Reading reply sessionid:0x13c15a1d848003f, packet:: clientPath:null serverPath:null finished:false header:: 32,4 replyHeader:: 32,354,0 request:: '/kafka/brokers/topics/test/partitions/0/leaderAndISR,F response:: #7b2022495352223a2230222c226c6561646572223a2230222c22636f6e74726f6c6c657245706f6368223a2232222c226c656164657245706f6368223a223022207d,s

          {325,325,1357689749406,1357689749406,0,0,0,0,66,0,325} (org.apache.zookeeper.ClientCnxn)
          [2013-01-08 19:55:26,266] DEBUG Reading reply sessionid:0x13c15a1d848003f, packet:: clientPath:null serverPath:null finished:false header:: 33,4 replyHeader:: 33,354,0 request:: '/kafka/brokers/topics/test/partitions/0/leaderAndISR,F response:: #7b2022495352223a2230222c226c6561646572223a2230222c22636f6e74726f6c6c657245706f6368223a2232222c226c656164657245706f6368223a223022207d,s{325,325,1357689749406,1357689749406,0,0,0,0,66,0,325}

          (org.apache.zookeeper.ClientCnxn)
          [2013-01-08 19:55:26,277] DEBUG replicas = List(0), in sync replicas = ArrayBuffer(0), leader = Some(0) (kafka.admin.AdminUtils$)
          [2013-01-08 19:55:26,278] DEBUG Reading reply sessionid:0x13c15a1d848003f, packet:: clientPath:null serverPath:null finished:false header:: 34,4 replyHeader:: 34,354,0 request:: '/kafka/brokers/ids/0,F response:: #3139322e3136382e322e313a393039323a39393939,s

          {352,352,1357692804787,1357692804787,0,0,0,88969877503082559,21,0,352}

          (org.apache.zookeeper.ClientCnxn)

          >>> The TopicMetadataResponse <<<
          [2013-01-08 19:55:26,282] TRACE [KafkaApi-0] Sending topic metadata TopicMetadata(test,List(PartitionMetadata(0,Some(id:0,host:192.168.2.1,port:9092),List(id:0,host:192.168.2.1,port:9092),ArrayBuffer(id:0,host:192.168.2.1,port:9092),0)),0) (kafka.server.KafkaApis)

          [2013-01-08 19:55:26,284] TRACE Socket server received response to send, registering for write: Response(0,Request(0,sun.nio.ch.SelectionKeyImpl@30d647d8,java.nio.HeapByteBuffer[pos=0 lim=32 cap=32],1357692926247),kafka.network.BoundedByteBufferSend@3ddfd90f) (kafka.network.Processor)
          [2013-01-08 19:55:26,284] TRACE 75 bytes written to /127.0.0.1:61719 (kafka.network.Processor)
          [2013-01-08 19:55:26,284] TRACE Completed request: TopicMetadataRequest(0,kafka-python,List(test),0) totalTime:37 queueTime:0 localTime:37 remoteTime:0 sendTime:0 (kafka.network.RequestChannel$)
          [2013-01-08 19:55:26,285] INFO Closing socket connection to /127.0.0.1. (kafka.network.Processor)
          [2013-01-08 19:55:26,285] DEBUG Closing connection from /127.0.0.1:61719 (kafka.network.Processor)

          Show
          David Arthur added a comment - Removing the debug statement fixes the issue. However, it seems instead of returning all of the topic metadata - I get none. Here are some trace logs when I send a TopicMetadataRequest with no topics [2013-01-08 19:53:34,463] TRACE 26 bytes read from /127.0.0.1:61558 (kafka.network.Processor) [2013-01-08 19:53:34,465] TRACE Received request: TopicMetadataRequest(0,kafka-python,List(),0) (kafka.network.RequestChannel$) [2013-01-08 19:53:34,466] TRACE Recieved request, sending for processing by handler: Request(1,sun.nio.ch.SelectionKeyImpl@785e8d7d,java.nio.HeapByteBuffer [pos=0 lim=26 cap=26] ,1357692814463) (kafka.network.Processor) [2013-01-08 19:53:34,466] DEBUG [Kafka Request Handler 1 on Broker 0] , handles request Request(1,sun.nio.ch.SelectionKeyImpl@785e8d7d,java.nio.HeapByteBuffer [pos=0 lim=26 cap=26] ,1357692814463) (kafka.server.KafkaRequestHandler) [2013-01-08 19:53:34,466] TRACE Handling topic metadata request TopicMetadataRequest(0,kafka-python,List(),0) (kafka.request.logger) [2013-01-08 19:53:34,466] TRACE [KafkaApi-0] Handling topic metadata request TopicMetadataRequest(0,kafka-python,List(),0) (kafka.server.KafkaApis) [2013-01-08 19:53:34,475] TRACE Socket server received response to send, registering for write: Response(1,Request(1,sun.nio.ch.SelectionKeyImpl@785e8d7d,java.nio.HeapByteBuffer [pos=0 lim=26 cap=26] ,1357692814463),kafka.network.BoundedByteBufferSend@2e82674b) (kafka.network.Processor) [2013-01-08 19:53:34,476] TRACE 16 bytes written to /127.0.0.1:61558 (kafka.network.Processor) [2013-01-08 19:53:34,476] TRACE Completed request: TopicMetadataRequest(0,kafka-python,List(),0) totalTime:13 queueTime:3 localTime:9 remoteTime:0 sendTime:1 (kafka.network.RequestChannel$) I would expect this to return info about the one existing topic "test". Here are the logs for when I send a TopicMetadataRequest with the "test" topic (this gives the expected response): [2013-01-08 19:55:26,247] TRACE 32 bytes read from /127.0.0.1:61719 (kafka.network.Processor) [2013-01-08 19:55:26,247] TRACE Received request: TopicMetadataRequest(0,kafka-python,List(test),0) (kafka.network.RequestChannel$) [2013-01-08 19:55:26,247] TRACE Recieved request, sending for processing by handler: Request(0,sun.nio.ch.SelectionKeyImpl@30d647d8,java.nio.HeapByteBuffer [pos=0 lim=32 cap=32] ,1357692926247) (kafka.network.Processor) [2013-01-08 19:55:26,247] DEBUG [Kafka Request Handler 0 on Broker 0] , handles request Request(0,sun.nio.ch.SelectionKeyImpl@30d647d8,java.nio.HeapByteBuffer [pos=0 lim=32 cap=32] ,1357692926247) (kafka.server.KafkaRequestHandler) [2013-01-08 19:55:26,247] TRACE Handling topic metadata request TopicMetadataRequest(0,kafka-python,List(test),0) (kafka.request.logger) [2013-01-08 19:55:26,247] TRACE [KafkaApi-0] Handling topic metadata request TopicMetadataRequest(0,kafka-python,List(test),0) (kafka.server.KafkaApis) [2013-01-08 19:55:26,248] DEBUG Reading reply sessionid:0x13c15a1d848003f, packet:: clientPath:null serverPath:null finished:false header:: 30,3 replyHeader:: 30,354,0 request:: '/kafka/brokers/topics/test,F response:: s {320,320,1357689749289,1357689749289,0,1,0,0,14,1,323} (org.apache.zookeeper.ClientCnxn) [2013-01-08 19:55:26,249] DEBUG Reading reply sessionid:0x13c15a1d848003f, packet:: clientPath:null serverPath:null finished:false header:: 31,4 replyHeader:: 31,354,0 request:: '/kafka/brokers/topics/test,F response:: #7b202230223a205b2230225d207d,s{320,320,1357689749289,1357689749289,0,1,0,0,14,1,323} (org.apache.zookeeper.ClientCnxn) [2013-01-08 19:55:26,252] DEBUG Partition map for /brokers/topics/test is Map(0 -> List(0)) (kafka.utils.ZkUtils$) [2013-01-08 19:55:26,255] DEBUG Reading reply sessionid:0x13c15a1d848003f, packet:: clientPath:null serverPath:null finished:false header:: 32,4 replyHeader:: 32,354,0 request:: '/kafka/brokers/topics/test/partitions/0/leaderAndISR,F response:: #7b2022495352223a2230222c226c6561646572223a2230222c22636f6e74726f6c6c657245706f6368223a2232222c226c656164657245706f6368223a223022207d,s {325,325,1357689749406,1357689749406,0,0,0,0,66,0,325} (org.apache.zookeeper.ClientCnxn) [2013-01-08 19:55:26,266] DEBUG Reading reply sessionid:0x13c15a1d848003f, packet:: clientPath:null serverPath:null finished:false header:: 33,4 replyHeader:: 33,354,0 request:: '/kafka/brokers/topics/test/partitions/0/leaderAndISR,F response:: #7b2022495352223a2230222c226c6561646572223a2230222c22636f6e74726f6c6c657245706f6368223a2232222c226c656164657245706f6368223a223022207d,s{325,325,1357689749406,1357689749406,0,0,0,0,66,0,325} (org.apache.zookeeper.ClientCnxn) [2013-01-08 19:55:26,277] DEBUG replicas = List(0), in sync replicas = ArrayBuffer(0), leader = Some(0) (kafka.admin.AdminUtils$) [2013-01-08 19:55:26,278] DEBUG Reading reply sessionid:0x13c15a1d848003f, packet:: clientPath:null serverPath:null finished:false header:: 34,4 replyHeader:: 34,354,0 request:: '/kafka/brokers/ids/0,F response:: #3139322e3136382e322e313a393039323a39393939,s {352,352,1357692804787,1357692804787,0,0,0,88969877503082559,21,0,352} (org.apache.zookeeper.ClientCnxn) >>> The TopicMetadataResponse <<< [2013-01-08 19:55:26,282] TRACE [KafkaApi-0] Sending topic metadata TopicMetadata(test,List(PartitionMetadata(0,Some(id:0,host:192.168.2.1,port:9092),List(id:0,host:192.168.2.1,port:9092),ArrayBuffer(id:0,host:192.168.2.1,port:9092),0)),0) (kafka.server.KafkaApis) [2013-01-08 19:55:26,284] TRACE Socket server received response to send, registering for write: Response(0,Request(0,sun.nio.ch.SelectionKeyImpl@30d647d8,java.nio.HeapByteBuffer [pos=0 lim=32 cap=32] ,1357692926247),kafka.network.BoundedByteBufferSend@3ddfd90f) (kafka.network.Processor) [2013-01-08 19:55:26,284] TRACE 75 bytes written to /127.0.0.1:61719 (kafka.network.Processor) [2013-01-08 19:55:26,284] TRACE Completed request: TopicMetadataRequest(0,kafka-python,List(test),0) totalTime:37 queueTime:0 localTime:37 remoteTime:0 sendTime:0 (kafka.network.RequestChannel$) [2013-01-08 19:55:26,285] INFO Closing socket connection to /127.0.0.1. (kafka.network.Processor) [2013-01-08 19:55:26,285] DEBUG Closing connection from /127.0.0.1:61719 (kafka.network.Processor)
          Hide
          David Arthur added a comment -

          This patch will return all topic metadata if none are specified in the TopicMetadataRequest. Also fixes that debug statement

          Show
          David Arthur added a comment - This patch will return all topic metadata if none are specified in the TopicMetadataRequest. Also fixes that debug statement
          David Arthur made changes -
          Field Original Value New Value
          Attachment KAFKA-690.patch [ 12563864 ]
          Hide
          Neha Narkhede added a comment -

          +1. Thanks for the patch !

          Show
          Neha Narkhede added a comment - +1. Thanks for the patch !
          Hide
          Maxime Brugidou added a comment -

          this would resolve KAFKA-653

          Show
          Maxime Brugidou added a comment - this would resolve KAFKA-653
          Maxime Brugidou made changes -
          Link This issue contains KAFKA-653 [ KAFKA-653 ]
          Hide
          David Arthur added a comment -

          Ah, I didn't see that JIRA. I was following https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-MetadataRequest and read "TopicName -> The topics to produce metadata for. If empty the request will yield metadata for all topics", so I assumed this was a bug instead of a TODO

          Show
          David Arthur added a comment - Ah, I didn't see that JIRA. I was following https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-MetadataRequest and read "TopicName -> The topics to produce metadata for. If empty the request will yield metadata for all topics", so I assumed this was a bug instead of a TODO
          David Arthur made changes -
          Link This issue duplicates KAFKA-653 [ KAFKA-653 ]
          David Arthur made changes -
          Link This issue duplicates KAFKA-653 [ KAFKA-653 ]
          Hide
          Jay Kreps added a comment -

          +1 Thanks David!

          Folks, objections to putting this on 0.8?

          Show
          Jay Kreps added a comment - +1 Thanks David! Folks, objections to putting this on 0.8?
          Hide
          Neha Narkhede added a comment -

          The protocol was meant to do this, so checked it on 0.8 branch.

          Show
          Neha Narkhede added a comment - The protocol was meant to do this, so checked it on 0.8 branch.
          Neha Narkhede made changes -
          Status Open [ 1 ] Resolved [ 5 ]
          Assignee David Arthur [ mumrah ]
          Resolution Fixed [ 1 ]

            People

            • Assignee:
              David Arthur
              Reporter:
              David Arthur
            • Votes:
              0 Vote for this issue
              Watchers:
              4 Start watching this issue

              Dates

              • Created:
                Updated:
                Resolved:

                Development