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

Deadlock on connector initialization

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Major
    • Resolution: Fixed
    • 2.6.3, 3.6.1
    • 3.8.0
    • connect
    • None

    Description

       

      Tested with Kafka 3.6.1 and 2.6.3.

      The only plugin installed is confluentinc-kafka-connect-jdbc-10.7.4.

      Stack trace for Kafka 3.6.1:

      Found one Java-level deadlock:
      =============================
      "pool-3-thread-1":
        waiting to lock monitor 0x00007fbc88006300 (object 0x0000000091002aa0, a org.apache.kafka.connect.runtime.standalone.StandaloneHerder),
        which is held by "Thread-9"
      "Thread-9":
        waiting to lock monitor 0x00007fbc88008800 (object 0x000000009101ccd8, a org.apache.kafka.connect.storage.MemoryConfigBackingStore),
        which is held by "pool-3-thread-1"Java stack information for the threads listed above:
      ===================================================
      "pool-3-thread-1":
          at org.apache.kafka.connect.runtime.standalone.StandaloneHerder$ConfigUpdateListener.onTaskConfigUpdate(StandaloneHerder.java:516)
          - waiting to lock <0x0000000091002aa0> (a org.apache.kafka.connect.runtime.standalone.StandaloneHerder)
          at org.apache.kafka.connect.storage.MemoryConfigBackingStore.putTaskConfigs(MemoryConfigBackingStore.java:137)
          - locked <0x000000009101ccd8> (a org.apache.kafka.connect.storage.MemoryConfigBackingStore)
          at org.apache.kafka.connect.runtime.standalone.StandaloneHerder.updateConnectorTasks(StandaloneHerder.java:483)
          at org.apache.kafka.connect.runtime.standalone.StandaloneHerder.lambda$null$2(StandaloneHerder.java:229)
          at org.apache.kafka.connect.runtime.standalone.StandaloneHerder$$Lambda$692/0x0000000840557440.run(Unknown Source)
          at java.util.concurrent.Executors$RunnableAdapter.call(java.base@11.0.21/Executors.java:515)
          at java.util.concurrent.FutureTask.run(java.base@11.0.21/FutureTask.java:264)
          at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(java.base@11.0.21/ScheduledThreadPoolExecutor.java:304)
          at java.util.concurrent.ThreadPoolExecutor.runWorker(java.base@11.0.21/ThreadPoolExecutor.java:1128)
          at java.util.concurrent.ThreadPoolExecutor$Worker.run(java.base@11.0.21/ThreadPoolExecutor.java:628)
          at java.lang.Thread.run(java.base@11.0.21/Thread.java:829)
      "Thread-9":
          at org.apache.kafka.connect.storage.MemoryConfigBackingStore.putTaskConfigs(MemoryConfigBackingStore.java:129)
          - waiting to lock <0x000000009101ccd8> (a org.apache.kafka.connect.storage.MemoryConfigBackingStore)
          at org.apache.kafka.connect.runtime.standalone.StandaloneHerder.updateConnectorTasks(StandaloneHerder.java:483)
          at org.apache.kafka.connect.runtime.standalone.StandaloneHerder.requestTaskReconfiguration(StandaloneHerder.java:255)
          - locked <0x0000000091002aa0> (a org.apache.kafka.connect.runtime.standalone.StandaloneHerder)
          at org.apache.kafka.connect.runtime.HerderConnectorContext.requestTaskReconfiguration(HerderConnectorContext.java:50)
          at org.apache.kafka.connect.runtime.WorkerConnector$WorkerConnectorContext.requestTaskReconfiguration(WorkerConnector.java:548)
          at io.confluent.connect.jdbc.source.TableMonitorThread.run(TableMonitorThread.java:86)
      
      Found 1 deadlock.
      

      The jdbc source connector is loading tables from the database and updates the configuration once the list is available. The deadlock is very consistent in my environment, probably because the database is on the same machine.

      Maybe it is possible to avoid this situation by always locking the herder first and the config backing store second. From what I see, updateConnectorTasks sometimes is called before locking on herder and other times it is not.

       

      Attachments

        Issue Links

          Activity

            People

              developster Octavian Ciubotaru
              developster Octavian Ciubotaru
              Votes:
              0 Vote for this issue
              Watchers:
              2 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: