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

Log broker shutdown reason during startup at the end of log output

    XMLWordPrintableJSON

Details

    • Improvement
    • Status: Resolved
    • Major
    • Resolution: Fixed
    • None
    • 3.3.0
    • core
    • None

    Description

      Currently, when the broker hit exception during startup, we will first log the exception, ex:

      ERROR [KafkaServer id=0] Fatal error during KafkaServer startup. Prepare to shutdown (kafka.server.KafkaServer)
      java.io.IOException: No space left on device

       

      And then go through the shutdown process, to close all the resources (i.e. threads, sockets, metrics...), and in the end, output:

      [KafkaServer id=0] shut down completed (kafka.server.KafkaServer)

      ERROR Exiting Kafka (kafka.Kafka$)

       

      Sometimes, during closing the resources, we will also hit some exceptions, or waiting for some other dependencies close, and output many more logs. When we tried to find out why the broker is shutdown from the log, we have to go through many lines of logs from the tail, and skip some unrelated exceptions, and then identify the specific reason why the broker shutting down.

      We could improve it by adding the reason why the broker is shutting down during startup at the end of log, something like this:

       

      [2022-04-07 18:19:33,005] ERROR [KafkaServer id=0] Fatal error during KafkaServer startup. Prepare to shutdown (kafka.server.KafkaServer)
      java.io.IOException: No space left on device
        at kafka.server.KafkaServer.startup(KafkaServer.scala:461)
        at kafka.Kafka$.main(Kafka.scala:110)
        at kafka.Kafka.main(Kafka.scala)
      [2022-04-07 18:19:33,007] INFO [KafkaServer id=0] shutting down (kafka.server.KafkaServer)
      [2022-04-07 18:19:33,008] INFO [KafkaServer id=0] Starting controlled shutdown (kafka.server.KafkaServer)
      [2022-04-07 18:19:33,016] INFO [KafkaServer id=0] Controlled shutdown request returned successfully after 6ms (kafka.server.KafkaServer)
      ....
      [2022-04-07 18:19:33,227] INFO Broker and topic stats closed (kafka.server.BrokerTopicStats)
      [2022-04-07 18:19:33,227] INFO App info kafka.server for 0 unregistered (org.apache.kafka.common.utils.AppInfoParser)
      [2022-04-07 18:19:33,227] INFO [KafkaServer id=0] shut down completed 
      
      // updated log
      [2022-04-11 15:06:27,405] ERROR Exiting Kafka due to fatal exception during startup (kafka.Kafka$)
      java.io.IOException: No space left on device
        at kafka.server.KafkaServer.startup(KafkaServer.scala:461)   
        at kafka.Kafka$.main(Kafka.scala:110)   
        at kafka.Kafka.main(Kafka.scala)

       

       

       

      Attachments

        Issue Links

          Activity

            People

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

              Dates

                Created:
                Updated:
                Resolved: