Uploaded image for project: 'Apache Storm'
  1. Apache Storm
  2. STORM-3102

Storm Kafka Client performance issues with Kafka Client v1.0.0

    XMLWordPrintableJSON

Details

    Description

      Recently I upgraded our storm topology to use the storm-kafka-client instead of storm-kafka.  After the upgrade in our production environment we saw a significant (2x) reduction in our processing throughput.

      We process ~20000 kafka messages per second, on a 10 machine kafka 1.0.0 server cluster.

      After some investigation, it looks like the issue only occurs when using kafka clients 0.11 or newer.

      In kafka 0.11, the kafka consumer method commited always blocks to make an external call o get the last commited offsets

      https://github.com/apache/kafka/blob/e18335dd953107a61d89451932de33d33c0fd207/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java#L1326-L1351

      In kafka 0.10.2 the kafka consumer only made the blocking remote call if the partition is not assigned to the consumer

      https://github.com/apache/kafka/blob/695596977c7f293513f255e07f5a4b0240a7595c/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java#L1274-L1311

       

      The impact of this is to require every tuple to make blocking remote calls before being emitted.  

      https://github.com/apache/storm/blob/2dc3d53a11aa3fea621666690d1e44fa8b621466/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpout.java#L464-L473

      Removing this check returns performance to expected levels.

      Looking through the storm-kafka-client code, it is not clear to me the impact of ignoring the check.  In our case we want at least once processing, but for other processing gurantees the call to kafkaConsumer.commited(tp) is not needed, as the value is only looked at if the processing mode is at least once.

      Attachments

        Activity

          People

            acseidel Andy Seidel
            acseidel Andy Seidel
            Votes:
            0 Vote for this issue
            Watchers:
            2 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved:

              Time Tracking

                Estimated:
                Original Estimate - Not Specified
                Not Specified
                Remaining:
                Remaining Estimate - 0h
                0h
                Logged:
                Time Spent - 1h 10m
                1h 10m