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

fetch requests can trigger repeated NPE when quota is enabled

    Details

    • Type: Bug
    • Status: Resolved
    • Priority: Blocker
    • Resolution: Fixed
    • Affects Version/s: 0.9.0.0, 0.9.0.1
    • Fix Version/s: 0.10.0.0
    • Component/s: None
    • Labels:
      None

      Description

      We saw the following NPE when consumer quota is enabled. NPE is triggered on every fetch request from the client.

      java.lang.NullPointerException
      at kafka.server.ClientQuotaManager.recordAndMaybeThrottle(ClientQuotaManager.scala:122)
      at kafka.server.KafkaApis.kafka$server$KafkaApis$$sendResponseCallback$3(KafkaApis.scala:419)
      at kafka.server.KafkaApis$$anonfun$handleFetchRequest$1.apply(KafkaApis.scala:436)
      at kafka.server.KafkaApis$$anonfun$handleFetchRequest$1.apply(KafkaApis.scala:436)
      at kafka.server.ReplicaManager.fetchMessages(ReplicaManager.scala:481)
      at kafka.server.KafkaApis.handleFetchRequest(KafkaApis.scala:431)
      at kafka.server.KafkaApis.handle(KafkaApis.scala:69)
      at kafka.server.KafkaRequestHandler.run(KafkaRequestHandler.scala:60)
      at java.lang.Thread.run(Thread.java:745)

      One possible cause of this is the logic of removing inactive sensors. Currently, in ClientQuotaManager, we create two sensors per clientId: a throttleTimeSensor and a quotaSensor. Each sensor expires if it's not actively updated for 1 hour. What can happen is that initially, the quota is not exceeded. So, quotaSensor is being updated actively, but throttleTimeSensor is not. At some point, throttleTimeSensor is removed by the expiring thread. Now, we are in a situation that quotaSensor is registered, but throttleTimeSensor is not. Later on, if the quota is exceeded, we will hit the above NPE when trying to update throttleTimeSensor.

        Attachments

          Activity

            People

            • Assignee:
              aauradkar Aditya Auradkar
              Reporter:
              junrao Jun Rao
              Reviewer:
              Jun Rao
            • Votes:
              0 Vote for this issue
              Watchers:
              6 Start watching this issue

              Dates

              • Created:
                Updated:
                Resolved: