You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by "Jun Rao (Jira)" <ji...@apache.org> on 2020/06/25 18:59:00 UTC

[jira] [Commented] (KAFKA-4084) automated leader rebalance causes replication downtime for clusters with too many partitions

    [ https://issues.apache.org/jira/browse/KAFKA-4084?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17145752#comment-17145752 ] 

Jun Rao commented on KAFKA-4084:
--------------------------------

[~hai_lin] : (1) If we just need a tool to disallow a broker from taking over as the leader for some partitions, it seems the existing partitionReassignment tool can do that already. One just needs to move that broker away from the first replica in the assigned replica list. So, I am not sure if we need a separate tool proposed by KIP-491 for this. (2) For the common case when a broker is restarted, currently, we move the leader when some, but not all replicas are caught up. It could be a better behavior to wait until all replicas are caught up before changing the leaders. If this is the more desired behavior, maybe this can just be added in the auto leader balancing logic, instead of a separate tool.

> 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)