[ https://issues.apache.org/jira/browse/KAFKA-4084?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17032267#comment-17032267 ]
GEORGE LI edited comment on KAFKA-4084 at 2/7/20 10:02 AM: ----------------------------------------------------------- [~blodsbror] [~junrao] [~sriharsha] With `auto.leader.rebalance.enable=true`, even with checking the broker is in ISR, then rebalance the leadership to this broker. It might still impact the partition that this broker is serving leadership because the broker is still trying to catch up. [~blodsbror] Is your cluster having lossless setting like min.insync.replicas > 1 ? For lossless, we experience high producer latency during catchup or reassignments. The leader deprioritized list feature is convenient in this case. it will just put this broker catching up in the lowest priority when considering being the leader. Another useful case is when the current controlller is very busy with metadata request, "blacklist" it, and only serving as followers can give 10-15% CPU back to the controller (without bouncing it). In a cluster without any down brokers, no URP (Under Replicated Partitions), there is a workaround to run reassignments to move that broker to the end of the partition assignment, e.g. broker_id 100 is down. then partition assignment (100, 101, 102) => (101, 102, 100). the reassignment should complete fast because all replicas in ISR. then run preferred leader election will change the leader from 100 => 101. The downside is: its more work to rollback or rebalance again. KIP-491 is very easy to rollback, just unset the dynamic config. In the case of the cluster with URP, the reassignment approach to move the broker to the end of the assignment might not work, because the broker is not in ISR. the reassignments will be pending till it catches up and in ISR. So maybe we do have a good use-case for KIP-491 ? was (Author: sql_consulting): [~blodsbror] [~junrao][~sriharsha] With `auto.leader.rebalance.enable=true`, even with checking the broker is in ISR, then rebalance the leadership to this broker. It might still impact the partition that this broker is serving leadership because the broker is still trying to catch up. [~blodsbror] is your cluster having lossless setting like min.insync.replicas > 1 ? For lossless, we experience high producer latency during catchup or reassignments. The leader deprioritized list feature is convenient in this case. it will just put this broker catching up in the lowest priority when considering being the leader. Another useful case is when the current controlller is very busy with metadata request, "blacklist" it, and only serving as followers can give 10-15% CPU back to the controller (without bouncing it). In a cluster without any down brokers, no URP (Under Replicated Partitions), there is a workaround to run reassignments to move that broker to the end of the partition assignment, e.g. broker_id 100 is down. then partition assignment (100, 101, 102) => (101, 102, 100). the reassignment should complete fast because all replicas in ISR. then run preferred leader election will change the leader from 100 => 101. The downside is: its more work to rollback or rebalance again. KIP-491 is very easy to rollback, just unset the dynamic config. In the case of the cluster with URP, the reassignment approach to move the broker to the end of the assignment might not work, because the broker is not in ISR. the reassignments will be pending till it catches up and in ISR. So maybe we do have a good use-case for KIP-491 ? > 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)