One thing to note is that we do support controlled shutdown as part of the
regular shutdown hook in the broker. The wiki was not very clear w.r.t
this and I have updated it to convey this. You can turn on controlled
shutdown by setting "controlled.shutdown.enable" to true in kafka config.
This will ensure that on shutdown it moves the leadership over and then
gracefully shuts down. Two configs control the number of retries and back
off during the shutdown "controlled.shutdown.max.retries" and
"controlled.shutdown.retry.backoff.ms". Going forward we would be using
this approach to automate our shutdown.

On 7/10/13 11:05 PM, "Joel Koshy" <jjkosh...@gmail.com> wrote:

>It's not ideal - right now we use the JMX operation (which returns an
>empty set on a successful controlled shutdown). If not it returns a
>set containing the partitions still being led on the broker. We retry
>(with appropriate intervals) until it succeeds. After that we do a
>regular broker shutdown (SIGTERM). i.e., it is currently not automated
>and it does take a while (an hour or so) to do a rolling bounce across
>a 16 node cluster with a few hundred topics. We could also use the
>inbuilt controlled shutdown feature on the broker to do the same thing
>- which is also better because the JMX port is not always open to
>remote access in production environments.
>
>It is possible to automate it to some degree - and if controlled
>shutdown fails after 'n' retries the automation policy could be to
>proceed with the unclean shutdown or abort and wait for manual
>intervention. Another issue is that when a broker is taken down there
>will be underreplicated partitions in the cluster. When the broker
>comes back up we should (ideally) wait until the underreplicated
>partition count goes back down to zero before proceeding to the next
>broker - otherwise that broker could take longer to do its controlled
>shutdown (since it needs to move leadership of partitions it leads to
>other replicas which would not be possible if the other replica is the
>broker that just came up). We currently don't have an easy way to
>integrate this seamlessly with the deployment system at Linkedin.
>
>Joel
>
>On Wed, Jul 10, 2013 at 9:48 PM, Vadim Keylis <vkeylis2...@gmail.com>
>wrote:
>> Joel. How do you guys do kafka service shutdown and startup?
>>
>>
>> On Wed, Jul 10, 2013 at 5:32 PM, Joel Koshy <jjkosh...@gmail.com> wrote:
>>
>>> https://cwiki.apache.org/confluence/display/KAFKA/Replication+tools
>>> has more details. The ShutdownBroker tool does not return anything.
>>> i.e., it does not exit with a System.exit code to pass back to a
>>> shell. it only logs if controlled shutdown was complete or not. You
>>> will need to configure the number of shutdown retries and retry
>>> interval suitably for the controlled shutdown to complete. E.g., if
>>> you have a very large number of partitions led by a broker it may take
>>> more time to do a controlled shutdown on that broker (because leaders
>>> need to be moved). (This yet another reason why we recommend that you
>>> maintain even distribution of leaders - this can be accomplished by
>>> running the PreferredReplicaLeaderElection command.)
>>>
>>> On Wed, Jul 10, 2013 at 4:06 PM, Vadim Keylis <vkeylis2...@gmail.com>
>>> wrote:
>>> > We have deployed kafka 0.8 beta1. It was my understanding that
>>> > ShutdownBroker program needs be used to initiate proper shutdown of
>>>the
>>> > server. We are going to use this script in automated fashion. Does
>>>the
>>> > script return meaningful error code that can be capture by calling
>>>script
>>> > and act up on? What other proper ways to shutdown kafka 08?
>>> >
>>> > Thanks,
>>> > Vadim
>>>

Reply via email to