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

GEORGE LI edited comment on KAFKA-4084 at 2/3/20 9:26 PM:
----------------------------------------------------------

[~blodsbror] We have implemented KIP-491 internally.   

Using a dynamic config leader.deprioritized.list to set it for the cluster 
global level  ('<default>') so in case of controller failover, the new 
controller will inherit this dynamic config settings. 

{code}
bin/kafka-configs.sh --bootstrap-server localhost:9092 --entity-type brokers 
--entity-default --alter --add-config leader.deprioritized.list=10001

$ zkcli -h zk_host1 get /kafka_cluster_name/config/brokers/'<default>'
('{"version":1,"config":{"leader.deprioritized.list":"10001"}}', 
ZnodeStat(czxid=25790129667, mzxid=25791108491, ctime=1572677228451, 
mtime=1580117760492, version=21, cversion=0, aversion=0, ephemeralOwner=0, 
dataLength=59, numChildren=0, pzxid=25790129667))
{code}

This will put the broker 10001 to the lowest priority when controller is 
considering leadership for that partition,  regardless this broker is in the 
1st position of the assignment (namely : preferred leader),  if this is 
currently serving leadership,  the preferred leader election will move it to 
another broker in the ISR. 

We have also implemented another feature separate from KIP-491 that when an 
empty broker is starting up,  a dynamic config for that broker called 
"{{replica.start.offset.strategy}}" is set to "latest" (default is "earliest" 
like current upstream behavior),   just like a consumer, it will fetch from 
current leaders'  latest offsets instead of  earliest(start) offset.  So this 
makes the failed empty broker coming up very fast.    This feature is used 
together with KIP-491  {{leader.deprioritized.list}} to blacklist this broker 
to serve traffic (because it does not have enough data).  After it's in 
replication for sometime (retention of the broker/topic level),  this broker is 
completely caught-up, and the {{leader.deprioritized.list}}  is removed. and 
when preferred leader is run, this broker can serve traffic again.  We haven't 
proposed this in any KIP yet.  But I think this is also a good feature. 

maybe I will restart the KIP-491 discussion in the dev mailing list.



was (Author: sql_consulting):
[~blodsbror] We have implemented KIP-491 internally.   

Using a dynamic config leader.deprioritized.list to set it for the cluster 
global level  ('<default>') so in case of controller failover, the new 
controller will inherit this dynamic config settings. 

{code}
bin/kafka-configs.sh --bootstrap-server localhost:9092 --entity-type brokers 
--entity-default --alter --add-config leader.deprioritized.list=10001

$ zkcli -h zk_host1 get /kafka_cluster_name/config/brokers/'<default>'
('{"version":1,"config":{"leader.deprioritized.list":"10001"}}', 
ZnodeStat(czxid=25790129667, mzxid=25791108491, ctime=1572677228451, 
mtime=1580117760492, version=21, cversion=0, aversion=0, ephemeralOwner=0, 
dataLength=59, numChildren=0, pzxid=25790129667))
{code}

This will put the broker 10001 to the lowest priority when controller is 
considering leadership for that partition,  regardless this broker is in the 
1st position of the assignment (namely : preferred leader),  if this is 
currently serving leadership,  the preferred leader election will move it to 
another broker in the ISR. 

We have also implemented another feature separate from KIP-491 that when an 
empty broker is starting up,  a dynamic config for that broker called 
"{{replica.start.offset.strategy}}" is set to "latest" (default is "earliest" 
like current upstream behavior),   just like a consumer, it will fetch from 
current leaders'  latest offsets instead of  earliest(start) offset.  So this 
makes the failed empty broker coming up very fast.    This feature is used 
together with KIP-491  {{leader.deprioritized.list}} to blacklist this broker 
to serve traffic (because it does not have enough data).  After it's in 
replication for sometime (retention of the broker/topic level),  this broker is 
completely caught-up, and the {{leader.deprioritized.list}}  is removed. and 
when preferred leader is run, this broker can serve traffic again.  We haven't 
proposed this in any KIP yet.  But I think this is also a good features. 

maybe I will restart the KIP-491 discussion in the dev mailing list.


> 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