Motivation
Summary of existing controller
Current Kafka controller is a multi-threaded controller that emulates a state machine. It works in the following way.
Maintained state:
- Partitions replicas on each machine.
- Leaders of partitions.
State change input source:
- Listeners Registered to Zookeeper.
- AddPartitionsListener
- BrokerChangeListener
- DeleteTopicListener
- PartitionReassignedListener(Admin)
- PreferredReplicaElectionListener(Admin)
- ReassignedPartitionsIsrChangeListener
- TopicChangeListener
- Channels to brokers (controlled shutdown)
- Internal scheduled tasks (preferred leader election)
State change execution:
- Listener threads, KafkaApi thread and internal scheduler thread makes state change concurrently.
State change propagation model:
- P2P blocking channel from controller to each broker.
- Dedicated message queue for each controller-to-broker connection.
- No synchronization on message sent to different brokers.
- No callback for sending messages except topic deletion.
Fail Over/Back model:
- Zookeeper based leader election
- Zookeeper as persistent state store for fault tolerance.
Problems of existing controller
- State change are executed by different listeners concurrently. Hence complicated synchronization is needed which is error prone and difficult to debug.
- State propagation is not synchronized. Brokers might be in different state for undetermined time. This leads to unnecessary extra data loss.
- During controlled shutdown process, two connections are used for controller to broker communication. This makes state change propagation and controlled shutdown approval out of order.
New controller design
Outline
We will keep maintained state and fail over/back model unchanged.
We are going to change the state change propagation model, state change execution and output of the state change input source. More specifically:
- Abstract the output of each state change input source to an event.
- Have a single execution thread to serially process events one at a time.
- The zk listeners are responsible for only context updating but not event execution.
- Use o.a.k.clients.NetworClient + callback for state change propagation.
We would also like to modify KafkaServer to use new NetworkClient and prioritize the controller-to-broker traffic.
Related tickets
KAFKA-2139, KAFKA-2029, KAFKA-1305 (and definitely some other tickets... Appreciate it if people can add it here.)
Architecture
- 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
- 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:
- Broker is down (or long GC)
- New topic automatically created