Versions Compared

Key

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

Table of Contents

The following is a draft design that uses a high-available consumer coordinator at the broker side to handle consumer rebalance. By migrating the rebalance logic from the consumer to the coordinator we can resolve the consumer split brain problem and help thinner the consumer client. With this design we hope to incorporates:

...

KAFKA-242: Subsequent calls of ConsumerConnector.createMessageStreams cause Consumer offset to be incorrect

1.

...

Motivation

The motivation of moving to a new set of consumer client APIs with broker side co-ordination is laid out here. Some back of the envelope calculations along with experience of building a prototype for this suggest that with With a single coordinator we should support up to 5K consumers (the bottleneck should be the number of socket channels a single machine can have), and by adding more coordinator (we can enable every broker to be a coordinator, so the maximum of coordinator is . Every broker will be the co-ordinator for a subset of consumer groups, thereby, increasing the ability to manage more consumer groups linearly with the size of the server cluster), the scalability of the consumer will increase linearly.

2. Rebalancing Algorithm Overview

Rebalancing is the process where a group of consumer instances (belonging to the same group) co-ordinate to own a mutually exclusive set of partitions of topics that the group has subscribed to. At the end of a successful rebalance operation for a consumer group, every partition for all subscribed topics will be owned by a single consumer instance. The way rebalancing works is as follows. One of the brokers is elected as the coordinator for all a subset of the consumer groups. It will be responsible for triggering rebalancing attempts for certain consumer groups on consumer group membership changes or interested subscribed topic partition changes. It will also be responsible for communicating the resulting partition-consumer ownership configuration to all consumers of the group under undergoing a rebalance operation. More specifically, it will try to do the following:

...

The coordinator keeps a watch on all topic partitions registered in a cluster, that are being consumed by at least one consumer group. When a new partition is added (possibly due to a new added broker) or an old partition is deleted (possibly due to a broker failure), it will trigger the rebalance procedure for every group that is consuming this partition.

Initiate rebalance operation and communicate new partition ownership to consumers

...

Most of the original ZK paths storage are kept, in addition to the coordinator path (stores the current coordinator info):

Code Block

/consumers/coordinator --> broker_id (ephemeral; created by coordinator)

Besides, some of the original ZK paths are removed, including:

Code Block

/consumers/groups/ids

And some of the ephemeral ZK paths are changed to persistent:

Code Block

/consumers/groups/owners

4. On Coordinator Startup

...

Whenever it elects to become the leader, it will trigger the startup procedure of the coordinator described below:

Code Block

coordinatorStartup :

1. Register session expiration listener

2. Read all the topics from ZK and initialize the subscribed groups for each topic.

3. Register listeners for topics and their partition changes

3.1 Subscribe TopicChangeListener to /brokers/topics

3.2 Subscribe TopicPartitionChangeListener to each /brokers/topics/[topic]

4. Start up the committer and the scheduler

5. Initialize the threadpool of rebalancing handlers

The scheduler for ping requests is a delayed scheduler, using a priority queue of timestamps to keep track of the outstanding list of PingRequest, whenever it sends a request it needs to initialize an expire watcher in the PingRequestPurgatory for the request.

Code Block

scheduler.run :

While isRunning

  1. Peek the head consumer from the priority queue

  2. If the consumer.scheduledTime <= current_time() try to send the PingRequest, otherwise sleep for (consumer.scheduledTime - current_time()) and then sends it

  2.1 Sends the PingRequest via the SocketServer of the broker (the corresponding processor Id and selection key is remembered in the consumer registry)

  2.2 Set the timeout watcher in PingRequestPurgatory for the consumer

  3. Remove the consumer from the head of the queue and put the consumer with consumer.scheduledTime += (consumer.session_timeout_ms * 1/3) back to the queue

...

Handle Topic Change
Code Block

TopicChangeListener.handleChildChange :

1. Get the newly added topic (since /brokers/topics are persistent nodes, no topics should be deleted even if there is no consumers any more inside the group)

2. For each newly added topic:

2.1 Subscribe TopicPartitionChangeListener to /brokers/topics/topic

2.2 For groups that have wildcard interests, check whether any group is interested in the newly added topics; if yes, add the group to the interested group list of this topic

2.3 Get the set of groups that are interested in this topic, and if the group's rebalance bit is not set, set it and try to rebalance the group by assigning a rebalance task to some rebalance handler's queue randomly.

* To avoid concurrent rebalancing of the same group by two different handler threads, we need to enforce assigning the task to the same handler's queue if it is already assigned to that handler. Details will be described in the implementation section.
Handle Topic Partition Change
Code Block

TopicPartitionChangeListener.handleChildChange :

1. Get the set of groups that are interested in this topic, and if the group's rebalance bit is not set, set it and try to rebalance the group

...

Upon creation, the consumer will get a list of

Code Block

  {brokerId : (host, port)}

...

After that, the consumer will try to keep reading new requests from the coordinator. Hence the consumer does not need to embed a socket server.

Code Block

consumerStartup (initBrokers : Map[Int, (String, String)]):

1. In a round robin fashion, pick a broker in the initialized cluster metadata, create a socket channel with that broker

1.1. If the socket channel cannot be established, it will log an error and try the next broker in the initBroker list

1.2. The consumer will keep retrying connection to the brokers in a round robin fashioned it is shut down.

2. Send a ClusterMetadataRequest request to the broker and get a ClusterMetadataResponse from the broker

3. From the response update its local memory of the current server cluster metadata and the id of the current coordinator

4. Set up a socket channel with the current coordinator, send a RegisterConsumerRequest and receive a RegisterConsumerResponse

5. If the RegisterConsumerResponse indicates the consumer registration is successful, it will try to keep reading rebalancing requests from the channel; otherwise go back to step 1

...

Note that in this design we are still considering the format of 0.7, although new wire format has already been used in 0.8. Our implementation will keeps this in mind and make it easy to adapt to new format.

Code Block

{
  size: int32                // the size of this request, not including the first 4 bytes for this field
  request_type_id: int16     // the request type id, distinguish Fetch/Produce/Offset/Consult/etc requests
                             // do not need any data for the cluster metadata request
}
ClusterMetadataResponse
Code Block

{
  size: int32                                             // the size of this response, not including the first 4 bytes for this field
  error_code: int16                                       // global error code for this request, if any
  coordinator_id: int32                                   // broker id of the coordinator
  num_brokers: int16                                      // number of brokers in the following list
  brokers_info: [<broker_struct>]                         // current broker list in the cluster
}

broker_struct =>
{
  id: int32
  host: string                                            // we adapt the protocol of 0.8 for strings, where the size of the string is stored at the beginning, followed by the bytes
  port: int32
}
RegisterConsumerRequest
Code Block

{
  size: int32                        // the size of this request
  request_type_id: int16             // the type of the request, currently only one type is allowed: ConnectConsumerRequest
  group_id: string                   // group that the consumer belongs to
  consumer_id: string                // consumer id
  topic_count: string                // interested topic and number of streams, can be wildcard
  auto_commit: boolean               // indicator of whether autocommit is enabled
  auto_offset_rest: string           // indicator of what to do if an offset is out of range, currently either smallest or largest
  session_timeout_ms: int32          // session timeout in milliseconds
}

...

If all the checks are successful, then the co-ordinator accepts the registration and sends a RegisterConsumerReponse with no error message; otherwise it includes an appropriate error code in the RegisterConsumerReponse. It does not close the connection but let the consumer who receives the error code to close the connection.

Code Block

{
  size: int32                        // the size of this response
  error_code: int16                  // global error code for this request, if any
                                     // no data needed
}

...

Handle New Group
Code Block

handleNewGroup :

1. Create a new group registry with zero consumer members in coordinator

...

Handle New Consumer
Code Block

handleAddedConsumer :

1. Add the ping request task for this consumer to the scheduler.

2. Add the consumer registry to the group registry's member consumer registry list.

3. Read the subscribed topic/count of this consumer.

3.1 If it is a wildcard topic/count subscription, then add its group to the list of groups with wildcard topic subscriptions.

3.2 For each topic that it subscribed to, add its group to the list of groups that has subscribed to the topic.

4. Try to rebalance the group

...

Handle Consumer Failure
Code Block

handleConsumerFailure :

1. Remove its ping request task from the scheduler.

2. Close its corresponding socket from the broker's SocketServer.

3. Remove its registry from its group's member consumer registry list.

4. If there is an ongoing rebalance procedure by some of the rebalance handlers, let it fail directly.

5. If the group still has some member consumers, try to rebalance the group.

...

For each rebalance request for a specific group it triggers the following rebalance logic:

Code Block

rebalance (group) :

1. Get the topics that are interested by the group. For each topic:

1.1. Get the number of partitions by reading from ZK

1.2. Get the number of threads for each topic from the meta information of consumers in the group it kept in memory

1.3. Compute the new ownership assignment for the topic

3. Check if a rebalance is necessary by trying to get the current ownership from ZK for each topic.

3.1 If there is no registered ownership info in ZK, rebalance is necessary

3.2 If some partitions are not owned by any threads, rebalance is necessary

3.3 If some partitions registered in the ownership map do not exist any longer, rebalance is necessary

3.4 If ownership map do not match with the newly computed one, rebalance is necessary

3.5 Otherwise rebalance is not necessary

4. If rebalance is necessary, do the following:

4.1 For each consumer in the group, send the StopFetcherRequest to the socket server's corresponding processor

4.2 Then wait until socket server has reported that all the StopFetcherReponse have been received or a timeout has expired

4.3 Enforce committing the current consumed offsets information to Zookeeper

4.4 For each consumer in the group, send the StartFetcherRequest to the socket server's corresponding processor

4.5 Then wait until socket server has reported that all the StartFetcherReponse have been received or a timeout has expired

4.6 Update the new assignment of partitions to consumers in the Zookeeper

5. If a timeout signal is received in either 4.2 or 4.5 from the socket server, wait for config.rebalanceBackoffMs and retry rebalancing again.

...

When the dead server comes back, it's elector will atomically reconnect to ZK and trigger the handleNewSession function:

Code Block

handleNewSession :

1. Reset its state by clearing its in memory metadata.

2. Re-register the session expiration listener (this is because ZkClient does not re-register itself once fired).

3. Try to re-elect to be the coordinator by directly calling the elect function of its coordinatorElector.

...

Coordinator uses PingRequest to check if the consumer is still alive, and consumer need to respond with the current offset for each consuming partition if auto_commit is set.

Code Block

{
  size: int32                     // the size of this request
  request_type_id: int16          // the type of the request, distinguish Ping/Stop/StartFetcher requests
  consumer_id: string             // id string of the consumer
  num_retries: int32              // indicate this is the #th ping retry
}
PingResponse
Code Block

{
  size: int32                     // the size of this response
  request_type_id: int16          // the type of the request, distinguish Ping/Stop/StartFetcher requests
  error_code: int16               // global error code for this request, if any
  consumer_id: string             // id string of the consumer, this is only used by the purgatory
  num_partition: int16            // number of partitions that the consumer is consuming
  offset_info: [<offset_struct>]  // the offset info for each partition consumed by this consumer, its size should be exactly num_partition
                                  // if auto_commit is set to false the last two fields should not exist
}

offset_struct =>
{
  topic: string
  partition: string
  offset: int64
}
Handling PingRequest
Code Block

handlePingRequest (request: PingRequest):

1. If shutdown is initiated:

1.1 If autoCommit is not turned on, can shut down immediately after responding this PingRequest

1.2 Otherwise, need to wait for the next PingRequest to arrive after responding this PingRequest to make sure the previous PingResponse with the offset info has been received by the coordinator.

2. If the fetchers of the consumer has stopped unexpectedly due to out-of-range error, close the current connection with the coordinator, stop fetchers, and try to reconnect to the coordinator in order to trigger rebalance.

3.1 If autoCommit is not enabled, just sends a PingResponse with no num_partition and offset_info

3.2 Otherwise read topicRegistry and sends a PingResponse with the created num_partitions and offset_info
Handling PingResponse
Code Block

handlePingResponse (response: PingResponse):

1. Get the offset information for each consumed partition, and record the offset information to the group's registry if possible.

2. Clear the corresponding expiration watcher in PingRequestPurgatory.

...

When the coordinator decides to rebalance a group, it will send StopFetcherRequest to every consumer in the group to let them stop their fetchers.

Code Block

{
  size: int32                              // the size of this request
  request_type_id: int16                   // the type of the request, distinguish Ping/Stop/StartFetcher requests
  consumer_id: string                      // id string of the consumer, this is only used by the purgatory
}
Handling StopFetcherRequest
Code Block

StopFetcherRequest (request: StopFetcherRequest):

1. Close all fetchers and clean their corresponding queues

2. Send the response, if autoCommit is turned on attached the current consumed offsets to the response also.

3. Clear the corresponding expiration watcher in StopFetcherRequestPurgatory, and if all the watchers have been cleared notify the rebalance handler.
StopFetcherResponse
Code Block

{
  size: int32                     // the size of this response
  request_type_id: int16          // the type of the request, distinguish Ping/Stop/StartFetcher requests
  error_code: int16               // global error code for this request, if any
  consumer_id: string             // id string of the consumer, this is only used by the purgatory
  num_partition: int16            // number of partitions that the consumer is consuming
  offset_info: [<offset_struct>]  // the offset info for each partition consumed by this consumer, its size should be exactly num_partition
                                  // if auto_commit is set to false the last two fields should not exist
}

...

When the coordinator have received the StopFetcherResponse from all the consumers, it will send the StartFetcherRequest to the consumers along with the starting offset for each partition got from the StopFetcherResponses. Note that the StartFetcherRequest also contains the current cluster metadata, which is used to update the consumer's server cluster metadata in order to avoid unnecessary rebalancing attempts.

Code Block

{
  size: int32                              // the size of this request
  request_type_id: int16                   // the type of the request, distinguish Ping/Start/StartFetcher requests
  consumer_id: string                      // id string of the consumer
  num_threads: int16                       // the number of threads that have assigned partitions
  ownership_info: [<ownership_struct>]     // the detailed info of the ownership
  num_brokers: int16                       // number of brokers in the following list
  brokers_info: [<broker_struct>]          // current broker list in the cluster}

ownership_struct =>
{
  consumer_thread_id: string               // consumer thread id
  num_partitions: int16                    // the number of partitions assigned to the consumer
  assigned_parts: [<partition_struct>]     // the detailed info of assigned partitions, along with the starting offset
}

partition_struct =>
{
  topic: string
  partition: string                        // note that this partition string already contains the broker id
  offset: int64
}
Handling StartFetcherRequest
Code Block

handleStartFetcherRequest (request: StartFetcherRequest):

1. If the consumer's topic interests are wildcard, re-construct topicThreadIdAndQueues and KafkaMessageAndMetadataStreams

2. Read the assigned partitions along with offsets from assigned_parts

3. Update topic registry with the newly assigned partition map

4. Start fetchers with the new topic registry

5. Send back the StartFetcherResponse
StartFetcherResponse
Code Block

{
  size: int32                     // the size of this response
  request_type_id: int16          // the type of the request, distinguish Ping/Stop/StartFetcher requests
  error_code: int16               // global error code for this request, if any
  consumer_id: string             // id string of the consumer, this is only used by the purgatory
}
Handling StartFetcherResponse
Code Block

handleStartRequest (response: StartFetcherResponse):

1. Clear the corresponding expiration watcher in StartFetcherRequestPurgatory, and if all the watchers have been cleared notify the rebalance handler.

...