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

Jun Rao commented on KAFKA-5642:
--------------------------------

With 25K partitions, a replication factor of 2 and 5 brokers, our test shows 
the logging change reduces the controlled shutdown time from 6.5 mins to 30 
secs. Using the async ZK api further reduces the time to only 4 secs. The 
controller failover time is also improved. With 5 brokers, 2K topics with 50 
partitions each, a replication factor of 1, the controller failover time went 
down from 28 secs in 1.0.0 to 14 secs in 1.1.0.



> Use async ZookeeperClient in Controller
> ---------------------------------------
>
>                 Key: KAFKA-5642
>                 URL: https://issues.apache.org/jira/browse/KAFKA-5642
>             Project: Kafka
>          Issue Type: Sub-task
>            Reporter: Onur Karaman
>            Assignee: Onur Karaman
>             Fix For: 1.1.0
>
>
> Synchronous zookeeper writes means that we wait an entire round trip before 
> doing the next write. These synchronous writes are happening at a 
> per-partition granularity in several places, so partition-heavy clusters 
> suffer from the controller doing many sequential round trips to zookeeper.
> * PartitionStateMachine.electLeaderForPartition updates leaderAndIsr in 
> zookeeper on transition to OnlinePartition. This gets triggered per-partition 
> sequentially with synchronous writes during controlled shutdown of the 
> shutting down broker's replicas for which it is the leader.
> * ReplicaStateMachine updates leaderAndIsr in zookeeper on transition to 
> OfflineReplica when calling KafkaController.removeReplicaFromIsr. This gets 
> triggered per-partition sequentially with synchronous writes for failed or 
> controlled shutdown brokers.
> KAFKA-5501 introduced an async ZookeeperClient that encourages pipelined 
> requests to zookeeper. We should replace ZkClient's usage with this client.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

Reply via email to