[ https://issues.apache.org/jira/browse/KAFKA-2319?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15872967#comment-15872967 ]
Michał Woś edited comment on KAFKA-2319 at 2/20/17 4:26 AM: ------------------------------------------------------------ I've got the same problem after sending SIGTERM to a broker. Moreover: each time I restart gracefully brokers I'm loosing data. I'm pretty sure not in the producer side, I see from producer metrics that there are retries during initializing of broker stop, but no records dropped recorded. Producer thinks that all records were stored. However they are missing on the server side. Moreover between "kafka.log.LogManager: Shutting down." and "INFO kafka.log.LogManager: Shutdown complete." there is 4 minute pause (nothing is printed in logs). So yes - stopping broker takes >4 minutes. I'm running kafka from cloudera which is based on 0.8.2.0 + (I belive) 127 commits. To be exact, Cloudera version is named: 0.8.2.0+kafka1.4.0+127. Each broker leads ~200 topics. Full log: {code} ... 2017-02-17 17:55:54,059 INFO kafka.server.ReplicaFetcherThread: [ReplicaFetcherThread-0-2032], Shutting down 2017-02-17 17:55:54,061 INFO kafka.server.ReplicaFetcherThread: [ReplicaFetcherThread-0-2032], Stopped 2017-02-17 17:55:54,061 INFO kafka.server.ReplicaFetcherThread: [ReplicaFetcherThread-0-2032], Shutdown completed 2017-02-17 17:55:54,062 INFO kafka.server.ReplicaFetcherManager: [ReplicaFetcherManager on broker 721] shutdown completed 2017-02-17 17:55:54,090 INFO kafka.server.ReplicaManager: [Replica Manager on Broker 721]: Shut down completely 2017-02-17 17:55:54,091 INFO kafka.log.LogManager: Shutting down. // note it is still not killed, 4 minutes break and finally: 2017-02-17 17:59:56,174 INFO kafka.log.LogManager: Shutdown complete. 2017-02-17 17:59:56,179 WARN kafka.utils.Utils$: Kafka scheduler has not been started java.lang.IllegalStateException: Kafka scheduler has not been started at kafka.utils.KafkaScheduler.ensureStarted(KafkaScheduler.scala:114) at kafka.utils.KafkaScheduler.shutdown(KafkaScheduler.scala:86) at kafka.controller.KafkaController.onControllerResignation(KafkaController.scala:354) at kafka.controller.KafkaController.shutdown(KafkaController.scala:677) at kafka.server.KafkaServer$$anonfun$shutdownapply$mcV$sp(KafkaServer.scala:285) at kafka.utils.Utils$.swallow(Utils.scala:172) at kafka.utils.Logging$class.swallowWarn(Logging.scala:92) at kafka.utils.Utils$.swallowWarn(Utils.scala:45) at kafka.utils.Logging$class.swallow(Logging.scala:94) at kafka.utils.Utils$.swallow(Utils.scala:45) at kafka.server.KafkaServer.shutdown(KafkaServer.scala:285) at kafka.server.KafkaServerStartable.shutdown(KafkaServerStartable.scala:42) at kafka.Kafka$$anonrun(Kafka.scala:42) 2017-02-17 17:59:56,179 INFO org.I0Itec.zkclient.ZkEventThread: Terminate ZkClient event thread. 2017-02-17 17:59:56,190 INFO org.apache.zookeeper.ZooKeeper: Session: 0x45a1d0f178a1267 closed 2017-02-17 17:59:56,190 INFO org.apache.zookeeper.ClientCnxn: EventThread shut down 2017-02-17 17:59:56,199 INFO kafka.server.KafkaServer: [Kafka Server 721], shut down completed {code} Can someone confirm that kafka 0.9.0.0 resolves that issue for sure? [~hachikuji] Did you upgrade? Did it help? was (Author: wosiu): I've got the same problem after seinding SIGTERM to a broker. Moreover: each time I restart gracefully brokers I'm loosing data. I'm pretty sure not in the producer side, I see from producer metrics that there are retries during initializing of broker stop, but no records dropped recorded. Producer thinks that all records were stored. However they are missing on the server side. Moreover between "kafka.log.LogManager: Shutting down." and "INFO kafka.log.LogManager: Shutdown complete." there is 4 minute pause (nothing is printed in logs). So yes - stopping broker takes >4 minutes. I'm running kafka from cloudera which is based on 0.8.2.0 + (I belive) 127 commits. To be exact, Cloudera version is named: 0.8.2.0+kafka1.4.0+127. Each broker leads ~200 topics. Full log: {code} ... 2017-02-17 17:55:54,059 INFO kafka.server.ReplicaFetcherThread: [ReplicaFetcherThread-0-2032], Shutting down 2017-02-17 17:55:54,061 INFO kafka.server.ReplicaFetcherThread: [ReplicaFetcherThread-0-2032], Stopped 2017-02-17 17:55:54,061 INFO kafka.server.ReplicaFetcherThread: [ReplicaFetcherThread-0-2032], Shutdown completed 2017-02-17 17:55:54,062 INFO kafka.server.ReplicaFetcherManager: [ReplicaFetcherManager on broker 721] shutdown completed 2017-02-17 17:55:54,090 INFO kafka.server.ReplicaManager: [Replica Manager on Broker 721]: Shut down completely 2017-02-17 17:55:54,091 INFO kafka.log.LogManager: Shutting down. // note it is still not killed, 4 minutes break and finally: 2017-02-17 17:59:56,174 INFO kafka.log.LogManager: Shutdown complete. 2017-02-17 17:59:56,179 WARN kafka.utils.Utils$: Kafka scheduler has not been started java.lang.IllegalStateException: Kafka scheduler has not been started at kafka.utils.KafkaScheduler.ensureStarted(KafkaScheduler.scala:114) at kafka.utils.KafkaScheduler.shutdown(KafkaScheduler.scala:86) at kafka.controller.KafkaController.onControllerResignation(KafkaController.scala:354) at kafka.controller.KafkaController.shutdown(KafkaController.scala:677) at kafka.server.KafkaServer$$anonfun$shutdownapply$mcV$sp(KafkaServer.scala:285) at kafka.utils.Utils$.swallow(Utils.scala:172) at kafka.utils.Logging$class.swallowWarn(Logging.scala:92) at kafka.utils.Utils$.swallowWarn(Utils.scala:45) at kafka.utils.Logging$class.swallow(Logging.scala:94) at kafka.utils.Utils$.swallow(Utils.scala:45) at kafka.server.KafkaServer.shutdown(KafkaServer.scala:285) at kafka.server.KafkaServerStartable.shutdown(KafkaServerStartable.scala:42) at kafka.Kafka$$anonrun(Kafka.scala:42) 2017-02-17 17:59:56,179 INFO org.I0Itec.zkclient.ZkEventThread: Terminate ZkClient event thread. 2017-02-17 17:59:56,190 INFO org.apache.zookeeper.ZooKeeper: Session: 0x45a1d0f178a1267 closed 2017-02-17 17:59:56,190 INFO org.apache.zookeeper.ClientCnxn: EventThread shut down 2017-02-17 17:59:56,199 INFO kafka.server.KafkaServer: [Kafka Server 721], shut down completed {code} Can someone confirm that kafka 0.9.0.0 resolves that issue for sure? [~hachikuji] Did you upgrade? Did it help? > After controlled shutdown: IllegalStateException: Kafka scheduler has not > been started > -------------------------------------------------------------------------------------- > > Key: KAFKA-2319 > URL: https://issues.apache.org/jira/browse/KAFKA-2319 > Project: Kafka > Issue Type: Bug > Reporter: Jason Rosenberg > > Running 0.8.2.1, just saw this today at the end of a controlled shutdown. It > doesn't happen every time, but I've seen it several times: > {code} > 2015-07-07 18:54:28,424 INFO [Thread-4] server.KafkaServer - [Kafka Server > 99], Controlled shutdown succeeded > 2015-07-07 18:54:28,425 INFO [Thread-4] network.SocketServer - [Socket > Server on Broker 99], Shutting down > 2015-07-07 18:54:28,435 INFO [Thread-4] network.SocketServer - [Socket > Server on Broker 99], Shutdown completed > 2015-07-07 18:54:28,435 INFO [Thread-4] server.KafkaRequestHandlerPool - > [Kafka Request Handler on Broker 99], shutting down > 2015-07-07 18:54:28,444 INFO [Thread-4] server.KafkaRequestHandlerPool - > [Kafka Request Handler on Broker 99], shut down completely > 2015-07-07 18:54:28,649 INFO [Thread-4] server.ReplicaManager - [Replica > Manager on Broker 99]: Shut down > 2015-07-07 18:54:28,649 INFO [Thread-4] server.ReplicaFetcherManager - > [ReplicaFetcherManager on broker 99] shutting down > 2015-07-07 18:54:28,650 INFO [Thread-4] server.ReplicaFetcherThread - > [ReplicaFetcherThread-0-95], Shutting down > 2015-07-07 18:54:28,750 INFO [Thread-4] server.ReplicaFetcherThread - > [ReplicaFetcherThread-0-95], Shutdown completed > 2015-07-07 18:54:28,750 INFO [ReplicaFetcherThread-0-95] > server.ReplicaFetcherThread - [ReplicaFetcherThread-0-95], Stopped > 2015-07-07 18:54:28,750 INFO [Thread-4] server.ReplicaFetcherThread - > [ReplicaFetcherThread-0-98], Shutting down > 2015-07-07 18:54:28,791 INFO [Thread-4] server.ReplicaFetcherThread - > [ReplicaFetcherThread-0-98], Shutdown completed > 2015-07-07 18:54:28,791 INFO [ReplicaFetcherThread-0-98] > server.ReplicaFetcherThread - [ReplicaFetcherThread-0-98], Stopped > 2015-07-07 18:54:28,791 INFO [Thread-4] server.ReplicaFetcherManager - > [ReplicaFetcherManager on broker 99] shutdown completed > 2015-07-07 18:54:28,819 INFO [Thread-4] server.ReplicaManager - [Replica > Manager on Broker 99]: Shut down completely > 2015-07-07 18:54:28,826 INFO [Thread-4] log.LogManager - Shutting down. > 2015-07-07 18:54:30,459 INFO [Thread-4] log.LogManager - Shutdown complete. > 2015-07-07 18:54:30,463 WARN [Thread-4] utils.Utils$ - Kafka scheduler has > not been started > java.lang.IllegalStateException: Kafka scheduler has not been started > at kafka.utils.KafkaScheduler.ensureStarted(KafkaScheduler.scala:114) > at kafka.utils.KafkaScheduler.shutdown(KafkaScheduler.scala:86) > at > kafka.controller.KafkaController.onControllerResignation(KafkaController.scala:350) > at > kafka.controller.KafkaController.shutdown(KafkaController.scala:664) > at > kafka.server.KafkaServer$$anonfun$shutdown$8.apply$mcV$sp(KafkaServer.scala:285) > at kafka.utils.Utils$.swallow(Utils.scala:172) > at kafka.utils.Logging$class.swallowWarn(Logging.scala:92) > at kafka.utils.Utils$.swallowWarn(Utils.scala:45) > at kafka.utils.Logging$class.swallow(Logging.scala:94) > at kafka.utils.Utils$.swallow(Utils.scala:45) > at kafka.server.KafkaServer.shutdown(KafkaServer.scala:285) > ... > {code} -- This message was sent by Atlassian JIRA (v6.3.15#6346)