You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@kafka.apache.org by "Onur Karaman (JIRA)" <ji...@apache.org> on 2017/06/22 19:57:00 UTC

[jira] [Created] (KAFKA-5502) read current brokers from zookeeper upon processing broker change

Onur Karaman created KAFKA-5502:
-----------------------------------

             Summary: read current brokers from zookeeper upon processing broker change
                 Key: KAFKA-5502
                 URL: https://issues.apache.org/jira/browse/KAFKA-5502
             Project: Kafka
          Issue Type: Sub-task
            Reporter: Onur Karaman
            Assignee: Onur Karaman


[~lindong]'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:
{code}
List<String> children = getChildren(path);
listener.handleChildChange(path, children);
{code}
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:
{code}
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))
  }
}
{code}

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.



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