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

GEORGE LI commented on KAFKA-4084:
----------------------------------

[~blodsbror] 

If this turns out to be positive in the testing.   I can restart the discussion 
on the dev mailing list for KIP-491.    at least it works/helps with 
auto.leader.rebalance.enable=true. 

There are other use cases listed in  KIP-491.   e.g.   when controller is busy 
with metadata request,  can set this dynamic config for the controller, run 
PLE, and controller will give up all its leadership, just as a follower,  CPU 
usage  down. 10-15%, making it light-weighted doing its work,  no need to 
bounce the controller.   I know some company is working on  the feature of 
separating the controller to another set of machines.  


Our primary use case of this  `leader.deprioritized.list=<broker_Id>`  feature 
is  bundled together with another feature call  
replica.start.offlet.strategy=latest , which I have not filed for a KIP ,  
(default  is earliest like current  kafka behavior),  this is also a dynamic 
config.   can be set for broker level (or global cluster).  What it does is 
when a broker failed and lost all its local disk,  and replaced with an empty 
broker,  the empty broker will need to start replication from earliest offset 
by default,  for us,  this could be 20TB+ of data for a few hours and can cause 
outages if not throttled properly.     So just like the kafka consumer,  we 
introduce dynamic config replica.start.offlet.strategy=latest ,  to just 
replicate from each partition leader's  latest offset.   Once it's caught up 
(URP=> 0 for this broker) usually in 5-10minutes or sooner, then remove the 
dynamic config,   Because this broker does not have all the historical data,  
it should not be serving leaderships.  That's how the  KIP-491. 
`leader.deprioritized.list=<broker_Id>` is coming into play.    The automation 
software will calculate the  retention time at the broker and topic level, take 
the Max,  and once the broker is in replication for that amount of time (e.g.  
6 hours,  1 day,  3days, whatever,),  the automation software will remove the 
leader.deprioritized.list dynamic config for the broker.  and run PLE to change 
the leadership back to it. 


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

Reply via email to