[ https://issues.apache.org/jira/browse/KAFKA-4084?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17029058#comment-17029058 ]
Evan Williams commented on KAFKA-4084: -------------------------------------- We are seeing the same (or at least similar) issues. When terminating a broker in AWS, and assigning the same broker ID to it's replacement (that has a empty data dir). We have auto.leader.rebalance.enable=true and when the broker get's assigned as leader - we get 100% CPU usage (ReplicaFetcher threads), and many clients (streams) fail to connect, as the broker both tries to fetch and serve data. Am just thinking outside of the box a bit for a work around, but until KIP-491 is implemented (if at all), is it possible to implement some logic/automation to set auto.leader.rebalance.enable=false on the replacement node, so that as soon as the Kafka service is started, it will prevent it from becoming leader, until under replication partitions = 0, and then reset auto.leader.rebalance.enable back to true, and restart the kafka service ? Or must auto.leader.rebalance.enable be set on all brokers (or all other brokers), for it to be effective ? > automated leader rebalance causes replication downtime for clusters with too > many partitions > -------------------------------------------------------------------------------------------- > > Key: KAFKA-4084 > URL: https://issues.apache.org/jira/browse/KAFKA-4084 > Project: Kafka > Issue Type: Bug > Components: controller > Affects Versions: 0.8.2.2, 0.9.0.0, 0.9.0.1, 0.10.0.0, 0.10.0.1 > Reporter: Tom Crayford > Priority: Major > Labels: reliability > Fix For: 1.1.0 > > > If you enable {{auto.leader.rebalance.enable}} (which is on by default), and > you have a cluster with many partitions, there is a severe amount of > replication downtime following a restart. This causes > `UnderReplicatedPartitions` to fire, and replication is paused. > This is because the current automated leader rebalance mechanism changes > leaders for *all* imbalanced partitions at once, instead of doing it > gradually. This effectively stops all replica fetchers in the cluster > (assuming there are enough imbalanced partitions), and restarts them. This > can take minutes on busy clusters, during which no replication is happening > and user data is at risk. Clients with {{acks=-1}} also see issues at this > time, because replication is effectively stalled. > To quote Todd Palino from the mailing list: > bq. There is an admin CLI command to trigger the preferred replica election > manually. There is also a broker configuration “auto.leader.rebalance.enable” > which you can set to have the broker automatically perform the PLE when > needed. DO NOT USE THIS OPTION. There are serious performance issues when > doing so, especially on larger clusters. It needs some development work that > has not been fully identified yet. > This setting is extremely useful for smaller clusters, but with high > partition counts causes the huge issues stated above. > One potential fix could be adding a new configuration for the number of > partitions to do automated leader rebalancing for at once, and *stop* once > that number of leader rebalances are in flight, until they're done. There may > be better mechanisms, and I'd love to hear if anybody has any ideas. -- This message was sent by Atlassian Jira (v8.3.4#803005)