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

KafkaProducer.send() blocks and generates TimeoutException if topic name has illegal char

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Major
    • Resolution: Fixed
    • 0.10.2.0
    • 2.1.0
    • producer
    • None
    • Java client running against server using wurstmeister/kafka Docker image.

    Description

      The server is running with auto create enabled. If we try to publish to a topic with a forward slash in the name, the call blocks and we get a TimeoutException in the Callback. I would expect it to return immediately with an InvalidTopicException.

      There are other blocking issues that have been reported which may be related to some degree, but this particular cause seems unrelated.

      Sample code:

      import org.apache.kafka.clients.producer.*;
      import java.util.*;
      
      public class KafkaProducerUnexpectedBlockingAndTimeoutException {
      
        public static void main(String[] args) {
          Properties props = new Properties();
          props.put("bootstrap.servers", "kafka.example.com:9092");
          props.put("key.serializer", "org.apache.kafka.common.serialization.StringSerializer");
          props.put("value.serializer", "org.apache.kafka.common.serialization.StringSerializer");
          props.put("max.block.ms", 10000); // 10 seconds should illustrate our point
      
          String separator = "/";
          //String separator = "_";
      
          try (Producer<String, String> producer = new KafkaProducer<>(props)) {
      
            System.out.println("Calling KafkaProducer.send() at " + new Date());
            producer.send(
                new ProducerRecord<String, String>("abc" + separator + "someStreamName",
                    "Not expecting a TimeoutException here"),
                new Callback() {
                  @Override
                  public void onCompletion(RecordMetadata metadata, Exception e) {
                    if (e != null) {
                      System.out.println(e.toString());
                    }
                  }
                });
            System.out.println("KafkaProducer.send() completed at " + new Date());
          }
      
      
        }
      
      }
      

      Switching to the underscore separator in the above example works as expected.

      Mea culpa: We neglected to research allowed chars in a topic name, but the TimeoutException we encountered did not help point us in the right direction.

      Attachments

        Issue Links

          Activity

            People

              ahmeda Ahmed Al-Mehdi
              jlar310 Jeff Larsen
              Votes:
              0 Vote for this issue
              Watchers:
              7 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: