Versions Compared

Key

  • This line was added.
  • This line was removed.
  • Formatting was changed.

...

  • The Controller Context contains two kinds of information: cluster reachability and Topic State(Partition, Replica, Leaders, etc)
  • Two types of ZK listeners:
    • Responsible of updating cluster reachability by listening to broker path in zookeeper.
    • Responsible for create events and add them to Event Queue.
  • A controlled shutdown event will be generated when receive controlled shutdown from broker.
    • The controlled shutdown process will be changed to make state change and controlled shutdown approval occur in order. (This might involve broker side change as well)
  • Scheduled tasks (e.g. preferred leader election) will
  • On controller starting up or resignation, a ControllerStartUp/ControllerResignation event will be generated.
  • Event Executor Thread:
    • Change Topic State in Controller Context
    • Propagate the new state to each broker using o.a.k.clients.NetworkClient in non-blocking way.
  • Broker will only trigger Zookeeper data change when:
    1. Broker is down (or long GC)
    2. New topic automatically created

Event Types and Handling Process

To be added.

Discussion Required

  1. As stated in KAFKA-2029, current state change propagation has an issue that in an unbalanced cluster. State change on a heavily loaded broker will be much slower than a lightly loaded broker. This is because of the following two reasons:
    • Controller traffic is not prioritized on broker side. So controller messages needs to wait until some clients requests are handled which takes much longer on a heavily loaded broker.
    • Heavily loaded broker needs to take state change for more partitions while same amount state changes are distributed among several brokers as followers.
    Batching state change into a single request and prioritize the controller traffic on broker will solve the problem. But I wonder is there any specific reason we did not batch the state change in current controller?
  2. Ideally, we should try to let the brokers in consistent state if possible. That indicates that we should put synchronization barriers between two events. Which means we do not execute the next event until:
    1. Callback has been fired on each broker, or
    2. A broker is down and we will just skip waiting for its callback and move on.
    Does it make sense to do so? Implementation wise, it means that if a broker is down, we will stop sending message to all brokers until we know it's down. (And this leads to question 3)
  3. We actually have two ways to determine if a broker is alive or not: from Zookeeper or from NetworkClient. Currently we take zookeeper as source of truth, but when we send messages, NetworkClient connectivity is the one actually matters. Because Zookeeper timeout could take a while, so what should we do if Zookeeper says a broker is alive but NetworkClient shows it is disconnected. Should we block the current event processing? If there is a long GC, is it possible that NetworkClient says broker is connected but Zookeeper says it's dead? Back to question 2, it is about when we consider "a broker is down".

...