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

Kafka Connect source task hangs when producing record with invalid topic name

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Open
    • Major
    • Resolution: Unresolved
    • 2.0.0
    • None
    • connect
    • None

    Description

      If a Kafka Connect source task returns a SourceRecord with an invalid topic name (e.g. "dbserver1.inventory.test@data"), that source task hangs (presumably indefinitely?) and doesn't continue it's polling loop. The log is flooded with this message:

      connect_1    | 2018-08-24 08:47:29,014 WARN   ||  [Producer clientId=producer-4] Error while fetching metadata with correlation id 833 : {dbserver1.inventory.test@data=INVALID_TOPIC_EXCEPTION}   [org.apache.kafka.clients.NetworkClient]
      

      The producer thread is stuck in the loop here:

      KafkaProducer<K,V>.waitOnMetadata(String, Integer, long) line: 938	
      KafkaProducer<K,V>.doSend(ProducerRecord<K,V>, Callback) line: 823	
      KafkaProducer<K,V>.send(ProducerRecord<K,V>, Callback) line: 803	
      WorkerSourceTask.sendRecords() line: 318	
      WorkerSourceTask.execute() line: 228	
      WorkerSourceTask(WorkerTask).doRun() line: 175	
      WorkerSourceTask(WorkerTask).run() line: 219	
      Executors$RunnableAdapter<T>.call() line: 511	
      FutureTask<V>.run() line: 266	
      ThreadPoolExecutor.runWorker(ThreadPoolExecutor$Worker) line: 1149	
      ThreadPoolExecutor$Worker.run() line: 624	
      Thread.run() line: 748	
      

      This causes the task to remain in RUNNING state, but no further invocations of poll() are done.

      Of course we'll work around this and make sure to not produce records with invalid topic names, but I think the source task should transition to FAILED state in this case.

      Attachments

        Activity

          People

            Unassigned Unassigned
            gunnar.morling Gunnar Morling
            Votes:
            0 Vote for this issue
            Watchers:
            2 Start watching this issue

            Dates

              Created:
              Updated: