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

KRaft Controller and ProcessExitingFaultHandler can deadlock shutdown

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Critical
    • Resolution: Fixed
    • 3.4.0
    • 3.5.0, 3.4.1
    • controller
    • None

    Description

      Problem

      When the kraft controller encounters an error that it cannot handle it calls ProcessExitingFaultHandler which calls Exit.exit which calls Runtime.exit.

      Based on the Runtime.exit documentation:

      All registered shutdown hooks, if any, are started in some unspecified order and allowed to run concurrently until they finish. Once this is done the virtual machine halts.

      One of the shutdown hooks registered by Kafka is Server.shutdown(). This shutdown hook eventually calls KafkaEventQueue.close. This last close method joins on the controller thread. Unfortunately, the controller thread also joined waiting for the shutdown hook thread to finish.

      Here are an sample thread stacks:

         "QuorumControllerEventHandler" #45 prio=5 os_prio=0 cpu=429352.87ms elapsed=620807.49s allocated=38544M defined_classes=353 tid=0x00007f5aeb31f800 nid=0x80c in Object.wait()  [0x00007f5a658fb000]
           java.lang.Thread.State: WAITING (on object monitor)                                                                                                                                                                                                            at java.lang.Object.wait(java.base@17.0.5/Native Method)
                - waiting on <no object reference available>
                at java.lang.Thread.join(java.base@17.0.5/Thread.java:1304)
                - locked <0x00000000a29241f8> (a org.apache.kafka.common.utils.KafkaThread)
                at java.lang.Thread.join(java.base@17.0.5/Thread.java:1372)
                at java.lang.ApplicationShutdownHooks.runHooks(java.base@17.0.5/ApplicationShutdownHooks.java:107)
                at java.lang.ApplicationShutdownHooks$1.run(java.base@17.0.5/ApplicationShutdownHooks.java:46)
                at java.lang.Shutdown.runHooks(java.base@17.0.5/Shutdown.java:130)
                at java.lang.Shutdown.exit(java.base@17.0.5/Shutdown.java:173)
                - locked <0x00000000ffe020b8> (a java.lang.Class for java.lang.Shutdown)
                at java.lang.Runtime.exit(java.base@17.0.5/Runtime.java:115)
                at java.lang.System.exit(java.base@17.0.5/System.java:1860)
                at org.apache.kafka.common.utils.Exit$2.execute(Exit.java:43)
                at org.apache.kafka.common.utils.Exit.exit(Exit.java:66)
                at org.apache.kafka.common.utils.Exit.exit(Exit.java:62)
                at org.apache.kafka.server.fault.ProcessExitingFaultHandler.handleFault(ProcessExitingFaultHandler.java:54)
                at org.apache.kafka.controller.QuorumController$ControllerWriteEvent$1.apply(QuorumController.java:891)
                at org.apache.kafka.controller.QuorumController$ControllerWriteEvent$1.apply(QuorumController.java:874)
                at org.apache.kafka.controller.QuorumController.appendRecords(QuorumController.java:969)

      and

        "kafka-shutdown-hook" #35 prio=5 os_prio=0 cpu=43.42ms elapsed=378593.04s allocated=4732K defined_classes=74 tid=0x00007f5a7c09d800 nid=0x4f37 in Object.wait()  [0x00007f5a47afd000]
           java.lang.Thread.State: WAITING (on object monitor)
                at java.lang.Object.wait(java.base@17.0.5/Native Method)
                - waiting on <no object reference available>
                at java.lang.Thread.join(java.base@17.0.5/Thread.java:1304)
                - locked <0x00000000a272bcb0> (a org.apache.kafka.common.utils.KafkaThread)
                at java.lang.Thread.join(java.base@17.0.5/Thread.java:1372)
                at org.apache.kafka.queue.KafkaEventQueue.close(KafkaEventQueue.java:509)
                at org.apache.kafka.controller.QuorumController.close(QuorumController.java:2553)
                at kafka.server.ControllerServer.shutdown(ControllerServer.scala:521)
                at kafka.server.KafkaRaftServer.shutdown(KafkaRaftServer.scala:184)
                at kafka.Kafka$.$anonfun$main$3(Kafka.scala:99)
                at kafka.Kafka$$$Lambda$406/0x0000000800fb9730.apply$mcV$sp(Unknown Source)
                at kafka.utils.Exit$.$anonfun$addShutdownHook$1(Exit.scala:38)
                at kafka.Kafka$$$Lambda$407/0x0000000800fb9a10.run(Unknown Source)
                at java.lang.Thread.run(java.base@17.0.5/Thread.java:833)
                at org.apache.kafka.common.utils.KafkaThread.run(KafkaThread.java:64) 

      Possible Solution

      A possible solution is to have the controller's unhandled fault handler call Runtime.halt instead of Runtime.exit.

      Attachments

        Issue Links

          Activity

            People

              jsancio José Armando García Sancio
              jsancio José Armando García Sancio
              Votes:
              0 Vote for this issue
              Watchers:
              1 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: