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

Jun Rao commented on KAFKA-6469:
--------------------------------

[~ambroff], thanks for reporting this. Is the # of children in the isr_change 
path a problem? Currently, each broker batches the isr changes when writing to 
ZK. So, the number of child nodes under isr_change should be proportional to 
the # brokers. If one follows the best practice by waiting for all replicas to 
be in sync before restarting the next broker, there should only be one child 
node in the isr_change typically.

> ISR change notification queue can prevent controller from making progress
> -------------------------------------------------------------------------
>
>                 Key: KAFKA-6469
>                 URL: https://issues.apache.org/jira/browse/KAFKA-6469
>             Project: Kafka
>          Issue Type: Bug
>            Reporter: Kyle Ambroff-Kao
>            Assignee: Kyle Ambroff-Kao
>            Priority: Major
>
> When the writes /isr_change_notification in ZooKeeper (which is effectively a 
> queue of ISR change events for the controller) happen at a rate high enough 
> that the node with a watch can't dequeue them, the trouble starts.
> The watcher kafka.controller.IsrChangeNotificationListener is fired in the 
> controller when a new entry is written to /isr_change_notification, and the 
> zkclient library sends a GetChildrenRequest to zookeeper to fetch all child 
> znodes.
> We've failures in one of our test clusters as the partition count started to 
> climb north of 60k per broker. We had brokers writing child nodes under 
> /isr_change_notification that were larger than the jute.maxbuffer size in 
> ZooKeeper (1MB), causing the ZooKeeper server to drop the controller's 
> session, effectively bricking the cluster.
> This can be partially mitigated by chunking ISR notifications to increase the 
> maximum number of partitions a broker can host.
>  



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to