Versions Compared

Key

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

...

Right now broker handles consumer state in a two-phase protocol. To solely explain consumer rebalance, we only discuss 3 involving states here: RUNNING, PREPARE_REBALANCE and SYNC.

  • When a new member consumer joins the consumer group, if this is a new member or the group leader, the broker will move this group state from RUNNING to PREPARE_REBALANCE. The reason for triggering rebalance when leader joins rejoins is because there might be assignment protocol change (for example if the consumer group is using regex subscription and new matching topics show up). If an old normal member rejoins the group, the state will not change. 
  • When moved to PREPARE_REBALANCE state, the broker will mark first joined consumer as leader, and wait for all the members to rejoin the group. Once we collected enough number of consumers/ all current members' join group requests or reached rebalance timeout, we will reply the leader with current member information and move the state to SYNCCOMPLETING_REBALANCE. All current members are informed to send SyncGroupRequest to get the final assignment.
  • The leader consumer will decide the assignment and send it back to broker. As last step, broker will announce the new assignment by sending SyncGroupResponse to all the followers. Till now we finished one rebalance and the group generation is incremented by 1.

In the current architecture, during each rebalance consumer groups on broker side will assign new member id with a UUID randomly generated UUID each time. This is to make sure we have unique identity for each group member. During client restart, consumer will send a JoinGroupRequest with a special UNKNOWN_MEMBER id, which has no intention to be treated as an existing _ID, and broker will interpret it as a new member.  To make the this KIP work, we need to change both client side and server side logic to make sure we persist member identity by checking persisting `group.instance.id` (explained later) throughout restarts, which means we could reduce number of rebalances since we are able to apply the same assignment based on member identities. The idea is summarized as static membership, which in contrary to dynamic membership (the one our system currently uses), is prioritizing "state persistence" over "liveness". 

...

  • Static Membership: the membership protocol where the consumer group will not trigger rebalance unless 1. a unless 
    • A new member joins
    2. a
    • A leader rejoins.
    3. an
    •  
    • An existing member
    go
    • offline time is over session timeout. 
    • Broker receives a leave group request (details later)
  • Group instance id: the unique identifier defined by user to distinguish each client instance.

...

Code Block
languagejava
titleStreamsMetadata.java
public Set<String> mainConsumerGroupInstanceIdsconsumerGroupInstanceIds();

Server side changes

We shall increase the cap of session timeout to 30 min for relaxing static membership liveness tracking.

...

For fault-tolerance, we also include group `group.instance id .id` within the member metadata to backup in the offset __consumer_offsets topic.

Code Block
languagescala
titleGroupMetadataManager
private val MEMBER_METADATA_V3 = new Schema(
  new Field(MEMBER_ID_KEY, STRING),
  new Field(MEMBER_NAME_KEY, STRING), // new
  new Field(CLIENT_ID_KEY, STRING),
  new Field(CLIENT_HOST_KEY, STRING),
  new Field(REBALANCE_TIMEOUT_KEY, INT32),
  new Field(SESSION_TIMEOUT_KEY, INT32),
  new Field(SUBSCRIPTION_KEY, BYTES),
  new Field(ASSIGNMENT_KEY, BYTES))

...

In short, the proposed feature is enabled if 

  1. JoinGroupRequest V4 is Latest JoinGroupReq/Res and LeaveGroupReq/Res are supported on both client and broker
  2. `group.instance.id` is configured with non-empty string.

...

On client side, we add a new config called `group.instance.id` in ConsumerConfig. On consumer service init, if the `group.instance.id` config is set, we will put it in the initial join group request to identify itself as a static member (static membership). Note that it is user's responsibility to assign unique `group.instance.id` for each consumers. This could be in service discovery hostname, unique IP address, etc. We also have logic handling duplicate `group.instance.id` in case client configured it wrong.

For the effectiveness of the KIP, consumer with `group.instance.id` set will not send leave group request when they go offline, which means we shall only rely on session.timeout to trigger group rebalance. It is because the proposed rebalance protocol will trigger rebalance with this intermittent in-and-out which is not ideal. In static membership we leverage the consumer group health management to client application such as K8. Therefore, it is also advised to make the session timeout large enough so that broker side will not trigger rebalance too frequently due to member come and go.

Kafka Streams change

By having a handful admin tool, user could proactively remove members if session timeout is too long in runtime.

Kafka Streams change

KStream uses KStream uses stream thread as consumer unit. For a stream instance configured with `num.threads` = 16, there would be 16 main consumers running on a single instance. If user specifies the client id, the stream consumer client id will be like: User client id + "-StreamThread-" + thread id + "-consumer"If user client id is not set, then we will use process id. Our plan is to reuse the consumer client id to define `group.instance.id`, so effectively the KStream instance will be able to use static membership if end user defines unique `client.id` for stream instances.

For easy operation, we defined a new field in StreamsMetadata to expose all the `group.instance.id` given on each stream instance, so that user could use the use REST API to get instance ids first, and then use command line API to batch remove offline consumers without waiting for session timeout.

Server behavior changes

On server side, broker will keep handling join group request <= v3 as before. The `member.id` generation and assignment is still coordinated by broker, and broker will maintain an in-memory mapping of {group.instance.id → member.id} to track member uniqueness. When receiving an a known member's (A.K.A `group.instance.id` known) rejoin request, broker will return the cached assignment back to the member, without doing any rebalance.

...

  • If the `member.id` uses UNKNOWN_MEMBER_NAMEID, we shall always generate a new member id and replace the one within current map. We also expect that after KIP-394, all the join group requests are requiring `member.id` to physically enter the consumer group, so the behavior of static member is consistent with that proposal.
  • we are requiring member.id (if not unknown) to match the value stored in cache, otherwise reply MEMBER_ID_MISMATCH. The edge case is that if we could have members with the same `group.instance.id` (for example mis-configured instances with a valid member.id but added a used `group.instance.id` on runtime). When `group.instance.id` has duplicates, we could refuse join request from members with an outdated `member.id` (, since we update the mapping upon each join group request). In an edge case where the client hits this exception in the response, it is suggesting that some other consumer takes its spot. The client should immediately fail itself to inform end user that there is a configuration bug which is generating duplicate consumers with same identity. For first version of this KIP, we just want to have straightforward handling to expose the error in early stage and reproduce bug cases easily.

For join group requests under dynamic membership (without `group.instance.id` set), the handling logic will remain unchanged. If the broker version is not the latest (< v4), the join group request shall be downgraded to v3 without setting the member Id.

Scale up

We will not plan to solve the scale up issue holistically within this KIP, since there is a parallel discussion about Incremental Cooperative Rebalancing, in which we will encode the "when to rebalance" logic at the application level, instead of at the protocol level. 

We also Just for recod, there is a plan to deprecate group.initial.rebalance.delay.ms since we no longer needs it once static membership is delivered and the incremental rebalancing work is done.

Rolling bounce

Currently there is broker accepts a config value called rebalance timeout which is configured provided by consumer max.poll.intervals. The reason we set it to poll interval is because consumer could only send request within the call of poll() and we want to wait sufficient time for the join group request. When reaching rebalance timeout, the group will move towards completingRebalance COMPLETING_REBALANCE stage and remove unjoined members. This is actually conflicting with the design of static membership, because those temporarily unavailable members will potentially reattempt the join group and trigger extra rebalances. Internally we would optimize this logic by having rebalance timeout only in charge of stopping prepare rebalance stage, without removing non-responsive members immediately. There would not be a full rebalance if the lagging consumer sent a JoinGroup request within the session timeout.

So in summary, the member will only be removed due to session timeout. We shall remove it from both in-memory static group instance id mapping `group.instance.id` map and member list.

Scale down

Currently the scale down is controlled by session timeout, which means if user removes the over-provisioned consumer members it waits until session timeout to trigger the rebalance. This is not ideal and motivates us to change LeaveGroupRequest to be able to include a list of `group.instance.id`s id` such that we could batch remove offline members and trigger rebalance immediately without them.

...

To make sure we could recover from broker failure/leader transition, an in-memory `group.instance.id` map is not enough. We would reuse the ` _consumer_offsets` offsets topic to store the static member map information. When another broker takes over the leadership, it will load the static mapping info together. 

...

RemoveMemberFromGroup (introduced above) will remove given instances and trigger one rebalance immediately on static membership, which is mainly used for fast scale down/host replacement cases (we detect consumer failure faster than the session timeout). This API will first send a FindCoordinatorRequest to locate the target broker, and initiate a LeaveGroupRequest to target broker hosting that coordinator, and the coordinator will decide whether to take this metadata change request based on its status at timeon runtime.

Error will be returned if

  1. the broker is on an old version.
  2. Consumer group does not exist.
  3. Operator is not authorized. (neither admin nor consumer group creater)
  4. if the group is not in a valid state to transit to rebalance. (use `canRebalance` function defined in GroupMetadata.scala to check)
  5. Some instance ids are not found. (, which means the request is not valid).

We need to enforce special access to these APIs to the end user who may not be in administrative role of Kafka Cluster. We shall The solution is to allow a similar access level to the join group request, so the consumer service owner could easily use this API.

Compatibility, Deprecation, and Migration Plan

...

Upgrade process

The recommended upgrade process is as follow:

...

That's it! We believe that the static membership logic is compatible with the current dynamic membership, which means it is allowed to have static members and dynamic members co-exist within the same consumer group. This assumption could be further verified when we do some modeling of the protocol (through TLA maybe) or dev test. and unit test.  

For KStream user set `client.id` for Stream instances in step 3 should do the work.

Downgrade process

The downgrade process is also straightforward. End user could just unset `group.instance.id` and do a rolling bounce to switch back to dynamic membershipFor KStream user, unset `client.id` should do the work. The static membership metadata stored on broker will not take any effect when `group.instanceeventually wipe out when the corresponding `member.id` is empty. After consumer offset topic retention, the old mapping messages will be gone completelysession timeout. 

One more tip is that, user may want to change the session timeout to a smaller value, since the liveness is more important now.

Non-goal

We do have some offline discussions on handling leader rejoin case, for example since the broker could also do the subscription monitoring work, we don't actually need to trigger rebalance on leader side blindly based on its rejoin request. However this is a separate topic and we will address it in another KIP. 

...