Not that I am some expert on this subject, but I can see that broker logs indicate the shutdown progress: https://github.com/tejasapatil/kafka/blob/0.8.0-beta1-candidate1/core/src/main/scala/kafka/server/KafkaServer.scala#L165
On Mon, Aug 19, 2013 at 10:19 PM, Vadim Keylis <[email protected]>wrote: > Neha. Thanks so much for explaining. That leaves only one open question. > How do you validate that shutdown was successful if you do not have remote > jmx access unless besides setting timeout reasonable high? > > Thanks so much again, > Vadim > > > On Mon, Aug 19, 2013 at 9:11 PM, Neha Narkhede <[email protected] > >wrote: > > > It depends on how much flexibility you need during the controlled > shutdown > > and whether you have remote jmx operations enabled in your production > Kafka > > cluster. The jmx controlled shutdown method will offer more flexibility > as > > your script will have the retry logic, you don't need to make config > > changes to Kafka brokers to change the timeout or the # of retries for > > controlled shutdown. On the other hand, the jmx controlled shutdown > method > > requires access to remote jmx on the broker. At LinkedIn, we do not have > > the ability to invoke jmx operations remotely on Kafka brokers in > > production. So we prefer using the controlled.shutdown.enable method. > > > > Thanks, > > Neha > > > > > > On Mon, Aug 19, 2013 at 12:34 PM, Vadim Keylis <[email protected] > > >wrote: > > > > > What is preferred method for control shutdown using admin tool or > setting > > > as flag "controlled.shutdown.enable" to true? What is the advantage of > > > using one verses the other? > > > > > > Thanks, > > > Vadim > > > > > > > > > On Sun, Aug 18, 2013 at 11:05 PM, Vadim Keylis <[email protected] > > > >wrote: > > > > > > > thanks so much. Greatly appreciated. > > > > > > > > > > > > On Sun, Aug 18, 2013 at 10:00 PM, Neha Narkhede < > > [email protected] > > > >wrote: > > > > > > > >> It is exposed on every leader through the > > > >> "kafka.server.UnderReplicatedPartitions" jmx bean. It is independent > > of > > > >> the > > > >> controlled shutdown functionality. > > > >> > > > >> Thanks, > > > >> Neha > > > >> > > > >> > > > >> On Sun, Aug 18, 2013 at 8:33 PM, Vadim Keylis < > [email protected]> > > > >> wrote: > > > >> > > > >> > Neha. Thanks so much for response. How can I get under replicated > > > >> partition > > > >> > count during control shutdown that is configured in the property > > file? > > > >> > > > > >> > Thanks, > > > >> > Vadim > > > >> > > > > >> > > > > >> > On Sun, Aug 18, 2013 at 6:11 PM, Neha Narkhede < > > > [email protected] > > > >> > >wrote: > > > >> > > > > >> > > Vadim, > > > >> > > > > > >> > > Controlled shutdown takes 2 parameters - number of retries and > > > >> shutdown > > > >> > > timeout. In every retry, controlled shutdown attempts to move > > > leaders > > > >> off > > > >> > > of the broker that needs to be shutdown. If the controlled > > shutdown > > > >> runs > > > >> > > out of retries, it proceeds to shutting down the broker even if > it > > > >> still > > > >> > > hosts a few leaders. At LinkedIn, the script to bounce Kafka > > brokers > > > >> > waits > > > >> > > for the under replicated partition count to drop to 0 before > > > invoking > > > >> > > controlled shutdown on the next broker. The aim is to avoid data > > > loss > > > >> > that > > > >> > > occurs if you shut down a broker that still has some leaders. If > > the > > > >> > under > > > >> > > replicated count never drops to 0, it indicates a bug in Kafka > > code > > > >> and > > > >> > the > > > >> > > script does not proceed to bouncing any more brokers in a > cluster. > > > We > > > >> > > measure the time it takes to move "n" leaders off of some > broker, > > > and > > > >> > > configure the shutdown timeout accordingly. We also configure > the > > > >> retries > > > >> > > to a small number (2 or 3). If the controlled shutdown fails the > > > >> retries, > > > >> > > the broker shuts itself down anyways. In general, you want to > > avoid > > > >> hard > > > >> > > killing (kill -9) a broker since that means the broker will run > a > > > long > > > >> > > running log recovery process on startup. That significantly > delays > > > the > > > >> > time > > > >> > > the broker takes to rejoin the cluster. > > > >> > > > > > >> > > Thanks, > > > >> > > Neha > > > >> > > > > > >> > > > > > >> > > On Sun, Aug 18, 2013 at 3:33 PM, Vadim Keylis < > > > [email protected]> > > > >> > > wrote: > > > >> > > > > > >> > > > Good afternoon. We are running kafka on centos linux. I > enabled > > > >> > > controlled > > > >> > > > shutdown in the property file. We are starting/stopping kafka > > > using > > > >> > init > > > >> > > > script. The init script will issue term signal first followed > 3 > > > >> seconds > > > >> > > > later by kill signal. Is that right process to shutdown kafka? > > > Which > > > >> > > > startup/shutdown/restart script you guys use? What shutdown > > > process > > > >> > > > linkedin uses? What side effects could be after kafka service > is > > > >> killed > > > >> > > > uncleanly using kill -9 signal? > > > >> > > > > > > >> > > > Thanks, > > > >> > > > Vadim > > > >> > > > > > > >> > > > > > >> > > > > >> > > > > > > > > > > > > > >
