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

Bug prone response from producer.send(ProducerRecord, Callback) if Kafka broker is not running

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Major
    • Resolution: Fixed
    • 2.0.0
    • 2.2.0
    • producer
    • None

    Description

      Hi there, I have probably found a bug in Java Kafka producer client.

      Scenario & current behavior:

      • Start Kafka broker, single instance.
      • Start application that produces messages to Kafka.
      • Let the application to load partitions for a topic to warm up the producer, e.g. send a message to Kafka. I'm not sure if this is necessary step, but our code does it.
      • Gracefully stop the Kafka broker.
      • Application logs now contains "org.apache.kafka.clients.NetworkClient: [Producer clientId=...] Connection to node 0 could not be established. Broker may not be available." so the client is aware about the Kafka unavailability.
      • Trigger the producer to send a message using KafkaProducer.send(ProducerRecord, Callback) method.
      • The callback that notifies business code receives non-null RecordMetadata and null Exception after request.timeout.ms. The metadata contains offset -1 which is value of ProduceResponse.INVALID_OFFSET.

      Expected behavior:

      • If the Kafka is not running and the message is not appended to the log, the callback should contain null RecordMetadata and non-null Exception. At least I subjectively understand the Javadoc this way, "exception on production error" in simple words.
      • Developer that is not aware of this behavior and that doesn't test for offset -1, may consider the message as successfully send and properly acked by the broker.

      Known workaround

      • Together with checking for non-null exception in the callback, add another condition for ProduceResponse.INVALID_OFFSET.
                      try {
                          producer.send(record, (metadata, exception) -> {
                              if (metadata != null) {
                                  if (metadata.offset() != ProduceResponse.INVALID_OFFSET) {
                                      // Success
                                  } else {
                                      // Failure
                                  }
                              } else {
                                  // Failure
                              }
                          });
                      } catch (Exception e) {
                          // Failure
                      }
      

      Used setup

      • Latest Kafka 2.0.0 for both broker and Java client.
      • Originally found with broker 0.11.0.1 and client 2.0.0.
      • Code is analogy of the one in Javadoc of KafkaProducer.send().
      • Used producer configuration (others use defaults).
          bootstrap.servers = "localhost:9092"
          client.id = "..."
          acks = "all"
          retries = 1
          linger.ms = "20"
          compression.type = "lz4"
          request.timeout.ms = 5000 # The same behavior is with default, this is to speed up the tests
      

      Attachments

        1. metadata_when_kafka_is_stopped.png
          32 kB
          Michal Turek
        2. both_metadata_and_exception.png
          65 kB
          Michal Turek

        Issue Links

          Activity

            People

              huxi_2b huxihx
              turek@avast.com Michal Turek
              Votes:
              0 Vote for this issue
              Watchers:
              5 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: