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

Connect incorrectly logs that task has failed when one takes too long to shutdown

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Major
    • Resolution: Fixed
    • 2.3.0, 2.4.0, 2.3.1, 2.2.3, 2.5.0, 2.3.2, 2.4.1, 2.4.2, 2.5.1
    • 3.0.0
    • connect

    Description

      If the OffsetStorageReader is closed while the task is trying to shutdown, and the task is trying to access the offsets from the OffsetStorageReader, then we see the following in the logs.

      [2020-05-05 05:28:58,937] ERROR WorkerSourceTask{id=connector-18} Task threw an uncaught and unrecoverable exception (org.apache.kafka.connect.runtime.WorkerTask)
      org.apache.kafka.connect.errors.ConnectException: Failed to fetch offsets.
              at org.apache.kafka.connect.storage.OffsetStorageReaderImpl.offsets(OffsetStorageReaderImpl.java:114)
              at org.apache.kafka.connect.storage.OffsetStorageReaderImpl.offset(OffsetStorageReaderImpl.java:63)
              at org.apache.kafka.connect.runtime.WorkerSourceTask.execute(WorkerSourceTask.java:205)
              at org.apache.kafka.connect.runtime.WorkerTask.doRun(WorkerTask.java:175)
              at org.apache.kafka.connect.runtime.WorkerTask.run(WorkerTask.java:219)
              at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
              at java.util.concurrent.FutureTask.run(FutureTask.java:266)
              at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
              at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
              at java.lang.Thread.run(Thread.java:745)
      Caused by: org.apache.kafka.connect.errors.ConnectException: Offset reader closed while attempting to read offsets. This is likely because the task was been scheduled to stop but has taken longer than the graceful shutdown period to do so.
              at org.apache.kafka.connect.storage.OffsetStorageReaderImpl.offsets(OffsetStorageReaderImpl.java:103)
              ... 14 more
      [2020-05-05 05:28:58,937] ERROR WorkerSourceTask{id=connector-18} Task is being killed and will not recover until manually restarted (org.apache.kafka.connect.runtime.WorkerTask)
      

      This is a bit misleading, because the task is already on its way of being shutdown, and doesn't actually need manual intervention to be restarted. We can see that as later on in the logs we see that it throws another unrecoverable exception.

      [2020-05-05 05:40:39,361] ERROR WorkerSourceTask{id=connector-18} Task threw an uncaught and unrecoverable exception (org.apache.kafka.connect.runtime.WorkerTask)
      

      If we know a task is on its way of shutting down, we should not throw a ConnectException and instead log a warning so that we don't log false negatives.

      Attachments

        Issue Links

          Activity

            People

              kpatelatwork Kalpesh Patel
              skaundinya Sanjana Kaundinya
              Votes:
              0 Vote for this issue
              Watchers:
              3 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: