Uploaded image for project: 'Kafka'
  1. Kafka
  2. KAFKA-2813

selector doesn't close socket connection on non-IOExceptions

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Blocker
    • Resolution: Fixed
    • None
    • 0.9.0.0
    • core
    • None

    Description

      When running a system test, we saw lots of entries like the following. The issue is that when the current leader switches to the follower, we will truncate the log in the follower. It's possible there is a concurrent fetch request being served at this moment. If this happens, we throw a KafkaException when trying to send the fetch response (in FileMessageSet). The exception will propagate through Selector.poll(). Selector catches IOException and closes the corresponding socket. However, KafkaException is not an IOException. Since the socket is not closed, Selector.poll() will keep accessing the socket and keep getting the same error.

      [2015-11-11 07:25:01,150] ERROR Processor got uncaught exception. (kafka.network.Processor)
      kafka.common.KafkaException: Size of FileMessageSet /mnt/kafka-data-logs/test_topic-0/00000000000000000000.log has been truncated during write: old size 16368, new size 0
      at kafka.log.FileMessageSet.writeTo(FileMessageSet.scala:158)
      at kafka.api.PartitionDataSend.writeTo(FetchResponse.scala:77)
      at org.apache.kafka.common.network.MultiSend.writeTo(MultiSend.java:81)
      at kafka.api.TopicDataSend.writeTo(FetchResponse.scala:148)
      at org.apache.kafka.common.network.MultiSend.writeTo(MultiSend.java:81)
      at kafka.api.FetchResponseSend.writeTo(FetchResponse.scala:291)
      at org.apache.kafka.common.network.KafkaChannel.send(KafkaChannel.java:165)
      at org.apache.kafka.common.network.KafkaChannel.write(KafkaChannel.java:152)
      at org.apache.kafka.common.network.Selector.poll(Selector.java:301)
      at kafka.network.Processor.run(SocketServer.scala:413)
      at java.lang.Thread.run(Thread.java:745)

      Attachments

        Activity

          People

            junrao Jun Rao
            junrao Jun Rao
            Votes:
            0 Vote for this issue
            Watchers:
            2 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: