You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@kafka.apache.org by onurkaraman <gi...@git.apache.org> on 2017/06/22 20:09:52 UTC

[GitHub] kafka pull request #3413: KAFKA-5502: read current brokers from zookeeper up...

GitHub user onurkaraman opened a pull request:

    https://github.com/apache/kafka/pull/3413

    KAFKA-5502: read current brokers from zookeeper upon processing broker change

    Dong Lin's testing of the 0.11.0 release revealed a controller-side performance regression in clusters with many brokers and many partitions when bringing up many brokers simultaneously.
    
    The regerssion is caused by KAFKA-5028: a Watcher receives WatchedEvent notifications from the raw ZooKeeper client EventThread. A WatchedEvent only contains the following information:
    - KeeperState
    - EventType
    - path
    
    Note that it does not actually contain the current data or current set of children associated with the data/child change notification. It is up to the user to do this lookup to see the current data or set of children.
    
    ZkClient is itself a Watcher. When it receives a WatchedEvent, it puts a ZkEvent into its own queue which its own ZkEventThread processes. Users of ZkClient interact with these notifications through listeners (IZkDataListener, IZkChildListener). IZkDataListener actually expects as input the current data of the watched znode, and likewise IZkChildListener actually expects as input the current set of children of the watched znode. In order to provide this information to the listeners, the ZkEventThread, when processing the ZkEvent in its queue, looks up the information (either the current data or current set of children) simultaneously sets up the next watch, and passes the result to the listener.
    
    The regression introduced in KAFKA-5028 is the time at which we lookup the information needed for the event processing.
    
    In the past, the lookup from the ZkEventThread during ZkEvent processing would be passed into the listener which is processed immediately after. For instance in ZkClient.fireChildChangedEvents:
    ```
    List<String> children = getChildren(path);
    listener.handleChildChange(path, children);
    ```
    Now, however, there are multiple listeners that pass information looked up by the ZkEventThread into a ControllerEvent which gets processed potentially much later. For instance in BrokerChangeListener:
    ```
    class BrokerChangeListener(controller: KafkaController) extends IZkChildListener with Logging {
      override def handleChildChange(parentPath: String, currentChilds: java.util.List[String]): Unit = {
        import JavaConverters._
        controller.addToControllerEventQueue(controller.BrokerChange(currentChilds.asScala))
      }
    }
    ```
    
    In terms of impact, this:
    - increases the odds of working with stale information by the time the ControllerEvent gets processed.
    - can cause the cluster to take a long time to stabilize if you bring up many brokers simultaneously.
    
    In terms of how to solve it:
    - (short term) just ignore the ZkClient's information lookup and repeat the lookup at the start of the ControllerEvent. This is the approach taken in this ticket.
    - (long term) try to remove a queue. This basically means getting rid of ZkClient. This is likely the approach that will be taken in KAFKA-5501.

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/onurkaraman/kafka KAFKA-5502

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/kafka/pull/3413.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #3413
    
----
commit 585defde3400e62faaa65c5175e6db9e82c8ad18
Author: Onur Karaman <ok...@linkedin.com>
Date:   2017-06-22T20:05:29Z

    KAFKA-5502: read current brokers from zookeeper upon processing broker change
    
    Dong Lin's testing of the 0.11.0 release revealed a controller-side performance regression in clusters with many brokers and many partitions when bringing up many brokers simultaneously.
    
    The regerssion is caused by KAFKA-5028: a Watcher receives WatchedEvent notifications from the raw ZooKeeper client EventThread. A WatchedEvent only contains the following information:
    - KeeperState
    - EventType
    - path
    
    Note that it does not actually contain the current data or current set of children associated with the data/child change notification. It is up to the user to do this lookup to see the current data or set of children.
    
    ZkClient is itself a Watcher. When it receives a WatchedEvent, it puts a ZkEvent into its own queue which its own ZkEventThread processes. Users of ZkClient interact with these notifications through listeners (IZkDataListener, IZkChildListener). IZkDataListener actually expects as input the current data of the watched znode, and likewise IZkChildListener actually expects as input the current set of children of the watched znode. In order to provide this information to the listeners, the ZkEventThread, when processing the ZkEvent in its queue, looks up the information (either the current data or current set of children) simultaneously sets up the next watch, and passes the result to the listener.
    
    The regression introduced in KAFKA-5028 is the time at which we lookup the information needed for the event processing.
    
    In the past, the lookup from the ZkEventThread during ZkEvent processing would be passed into the listener which is processed immediately after. For instance in ZkClient.fireChildChangedEvents:
    ```
    List<String> children = getChildren(path);
    listener.handleChildChange(path, children);
    ```
    Now, however, there are multiple listeners that pass information looked up by the ZkEventThread into a ControllerEvent which gets processed potentially much later. For instance in BrokerChangeListener:
    ```
    class BrokerChangeListener(controller: KafkaController) extends IZkChildListener with Logging {
      override def handleChildChange(parentPath: String, currentChilds: java.util.List[String]): Unit = {
        import JavaConverters._
        controller.addToControllerEventQueue(controller.BrokerChange(currentChilds.asScala))
      }
    }
    ```
    
    In terms of impact, this:
    - increases the odds of working with stale information by the time the ControllerEvent gets processed.
    - can cause the cluster to take a long time to stabilize if you bring up many brokers simultaneously.
    
    In terms of how to solve it:
    - (short term) just ignore the ZkClient's information lookup and repeat the lookup at the start of the ControllerEvent. This is the approach taken in this ticket.
    - (long term) try to remove a queue. This basically means getting rid of ZkClient. This is likely the approach that will be taken in KAFKA-5501.

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] kafka pull request #3413: KAFKA-5502: read current brokers from zookeeper up...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/kafka/pull/3413


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---