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

Multi noded with kraft combined mode will fail shutdown

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Open
    • Major
    • Resolution: Unresolved
    • 3.3.1
    • None
    • kraft
    • None

    Description

      Multiple nodes with kraft combined mode (i.e. process.roles='broker,controller') can startup successfully. When shutdown in combined mode, we'll unfence broker first. When the remaining controller nodes are less than quorum size (i.e. N / 2 + 1), the unfence record will not get committed to metadata topic successfully. So the broker will keep waiting for the shutdown granting response and then timeout error:

       

      2022-10-11 18:01:14,341] ERROR [kafka-raft-io-thread]: Graceful shutdown of RaftClient failed (kafka.raft.KafkaRaftManager$RaftIoThread)
      java.util.concurrent.TimeoutException: Timeout expired before graceful shutdown completed
          at org.apache.kafka.raft.KafkaRaftClient$GracefulShutdown.failWithTimeout(KafkaRaftClient.java:2408)
          at org.apache.kafka.raft.KafkaRaftClient.maybeCompleteShutdown(KafkaRaftClient.java:2163)
          at org.apache.kafka.raft.KafkaRaftClient.poll(KafkaRaftClient.java:2230)
          at kafka.raft.KafkaRaftManager$RaftIoThread.doWork(RaftManager.scala:52)
          at kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:96)
       

       

       

      to reproduce:

      1. start up 2 kraft combines nodes, so we need 2 nodes get quorum
      2. shutdown any one node, in this time, it will shutdown successfully because when broker shutdown, the 2 controllers are all alive, so broker can be granted for shutdown
      3. shutdown 2nd node, this time, the shutdown will be pending because we only have 1 controller left, and in the end, timeout error.

      Attachments

        Activity

          People

            showuon Luke Chen
            showuon Luke Chen
            Votes:
            0 Vote for this issue
            Watchers:
            3 Start watching this issue

            Dates

              Created:
              Updated: