You are viewing an old version of this page. View the current version.

Compare with Current View Page History

« Previous Version 62 Next »



Status

Current state: In review

Discussion thread: TBD

JIRA: here

Please keep the discussion on the mailing list rather than commenting on the wiki (wiki discussions get unwieldy fast).

Motivation

For stateful applications, one of the biggest performance bottleneck is the state shuffling. In Kafka consumer, there is a concept called "rebalance" which means that for given M partitions and N consumers in one consumer group, Kafka will try to balance the load between consumers and ideally have each consumer dealing with M/N partitions. Broker will also adjust the workload dynamically by monitoring consumers' health so that we could kick dead consumer out of the group, and handling new consumers' join group request. The intuition of this design is to avoid processing hot spot and maintain fairness plus liveness of the whole application. However, when the service state is heavy, a rebalance of one topic partition from instance A to B means huge amount of data transfer. If multiple rebalances are triggered, the whole service could take a very long time to recover due to data transfer. 

The idea of this KIP is to reduce number of rebalances by introducing a new concept: static membership. Core argument is: Heavy state applications should reduce state shuffling as much as possible

Public Interfaces

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 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 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 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/ reached rebalance timeout, we will reply the leader with current member information and move the state to SYNC. 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 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 member.  To make the KIP work, we need to change both client side and server side logic to make sure we persist member identity 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". Since for many stateful consumer/stream applications, the state shuffling is more painful than short time partial unavailability.

Proposed Changes

We will be introducing a bunch of new terms:

  • Static Membership: the membership protocol where the consumer group will not trigger rebalance unless 1. a new member joins 2. a leader rejoins. 3. an existing member go offline over certain timeout.
  • Member name: the unique identifier defined by user to distinguish each client instance.
  • Member registration timeout: the max time we could tolerate a static member to go offline.
  • Member expansion timeout: the max time we will wait since we receive a new static member join request.

Client behavior changes

On client side, we add a new config called MEMBER_NAME in ConsumerConfig. On consumer service init, if the MEMBER_NAME config is set, we will put it in the initial join group request to identify itself as a static member (static membership); otherwise, we will still send UNKNOWN_MEMBER_ID to ask broker for allocating a new random ID (dynamic membership). To distinguish between previous version of protocol, we will also increase the join group request version to v4 when MEMBER_NAME is set. Note that it is user's responsibility to assign unique member id for each consumers. This could be in service discovery hostname, unique IP address, etc. We also have logic handling duplicate member.name in case client configured it wrong.

ConsumerConfig.java
public static final STRING MEMBER_NAME = "member_A"; // default empty String

Changes will be applied to join group request, sync group request and offset commit request to include member name and other settings: 

JoinGroupRequest => GroupId SessionTimeout RebalanceTimeout MemberId MemberName ProtocolType GroupProtocols
  GroupId             => String
  SessionTimeout      => int32
  RebalanceTimeout	  => int32
  MemberId            => String
  MemberName   		  => String // new
  ProtocolType        => String
  GroupProtocols      => [Protocol MemberMetadata]
  Protocol            => String
  MemberMetadata      => bytes


SyncGroupRequest => GroupId GroupGenerationId MemberId MemberName
  GroupId           => String
  GroupGenerationId => int32
  GroupState        => [MemberId MemberState]
  MemberId          => String
  MemberName   		=> String // new
  MemberState       => bytes
  
OffsetCommitRequest => GroupId GenerationId MemberId MemberName Topics
 GroupId      		=> String
 GenerationId 		=> String
 MemberId    		=> String
 MemberName 		=> String // new
 Topics 			=> partition data


If the broker version is not the latest (< v4), the join group request shall be downgraded to v3 without setting the member Id.

We shall also bump the join group request/response version to v4.

JoinGroupRequest.java
public static Schema[] schemaVersions() {
    return new Schema[] {JOIN_GROUP_REQUEST_V0, JOIN_GROUP_REQUEST_V1, JOIN_GROUP_REQUEST_V2, JOIN_GROUP_REQUEST_V3, JOIN_GROUP_REQUEST_V4};
}


JoinGroupResponse.java
public static Schema[] schemaVersions() {
    return new Schema[] {JOIN_GROUP_RESPONSE_V0, JOIN_GROUP_RESPONSE_V1, JOIN_GROUP_RESPONSE_V2, JOIN_GROUP_RESPONSE_V3, JOIN_GROUP_RESPONSE_V4};
}


For the effectiveness of the KIP, consumer with member.name set will not send leave group request when they go offline. It is because the current rebalance protocol will trigger rebalance with this intermittent in-and-out which is not ideal. 

Server behavior changes

On server side, broker will keep handling join group request <= v3 as before. If the protocol version is upgraded to v4 and the member name is set, the broker will use the member name specified in the join group request and respond with a unique "member id".  Broker will maintain an in-memory mapping of {member.name → member.id} to track member uniqueness.

For commit requests under static membership, we are requiring:

  • Both member.name and member.id must be set. Otherwise reply NO_STATIC_MEMBER_INFO_SET 
  • Member.name and member.id mapping are aligned with coordinator cache. Otherwise reply DUPLICATE_STATIC_MEMBER

so that when member name has duplicates, we could refuse commit request from members with an outdated member.id (since we update the mapping upon each join group request). Normally when hitting NO_STATIC_MEMBER_INFO_SET, it could potentially due to the consumer is doing rolling restart, where some members haven't updated their code with new member name. In an edge case where the client hits DUPLICATE_STATIC_MEMBER 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 corner cases easily.

When do we rebalance in static membership?

Rebalance happens rarely in static membership (unless scale up/down or leader rejoins). When receiving an existing member's rejoin request, broker will return the cached assignment back to the member, without doing any rebalance.

Scale down

Session timeout is the timeout we will trigger rebalance when a member goes offline for too long (not sending heartbeat request). To make static membership effective, we should increase the default max session timeout to 30 min so that end user could config it freely. Although in static membership we are leveraging client applications' self management (for example K8), session timeout could serve as the last line of defense for liveness.


KafkaConfig.scala
val GroupMaxSessionTimeoutMs = 1800000 // 30 min for max cap



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. 

Adding new static memberships should be straightforward. This operation should be happening fast enough (to make sure capacity could catch up quickly), we are defining another config called expansion timeout. In ideal case, we could actually introduce a new status called "learner" where the newly up hosts could try to catch up with the assigned task progress first before triggering the rebalance, from which we don't see a sudden dip on the progress. 

For scaling up from empty stage, we plan to deprecate group.initial.rebalance.delay.ms since 

Rolling bounce

Currently there is a config called rebalance timeout which is configured 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 stage and remove unjoined groups. This is actually conflicting with the design of static membership, because those temporarily unavailable members will reattempt the join group and trigger extra rebalances.

Fault-tolerance of static membership

To make sure we could recover from broker failure/leader transition, an in-memory member name map is not enough. We would reuse the `_consumer_offsets` topic to store the static member map information. When another broker takes over the leadership, we could transfer the mapping together. We shall also introduce new member metadata format:

GroupMetadataManager
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))


command line APIs for membership management

For the very first version, we hope to make membership transferring logic and human handling simple enough. We will define three command line APIs to help us better manage the groups:

AdminClient.java
public static MembershipChangeResult enableStaticMembership(String groupId, int registrationTimeout, int expansionTimeout)
public static MembershipChangeResult enableDynamicMembership(String groupId)
public static MembershipChangeResult forceStaticRebalance(String groupId)

enableStaticMembership will change the consumer group to static membership, along with changing registration timeout and expansion timeout. After that, all the joining members are required to set the member name field. Error will be returned if

  1. the broker is on an old version. 
  2. if the group is preparing rebalance/completing rebalance.
  3. Any member within the current group has not set `member.name`
  4. other potential failure cases.

Note that the client should already include member name field at this point. User could also use this API to change the timeout configs as they want, or leave it blank to use default value.

enableDynamicMembership will in the contrary just change the membership back to dynamic mode. Error will be returned if

  1. the broker is on an old version. 
  2. if the group is preparing rebalance/completing rebalance.
  3. group is already on dynamic membership.
  4. other potential failure cases.

forceStaticRebalance will trigger one rebalance immediately on static membership, which is mainly used for fast scale up/down cases. Error will be returned if 

  1. the broker is on an old version.
  2. if the group is preparing rebalance/completing rebalance.
  3. group is on dynamic membership.
  4. other potential failure cases.

We need to enforce special access to these APIs to the end user who may not be in administrative role of Kafka Cluster. This involves defining some customized Kafka principle.

Compatibility, Deprecation, and Migration Plan

  • The fallback logic has been discussed previously. Broker with a lower version would just downgrade static membership towards dynamic membership.

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. 

Rejected Alternatives

In this pull request, we did an experimental approach to materialize member id on the instance local disk. This approach could reduce the rebalances as expected, which is the experimental foundation of KIP-345. However, KIP-345 has a few advantages over it:

  1. It gives users more control of their member id string; this would help for debugging purposes.
  2. It is more cloud-/k8s-and-alike-friendly: when we move an instance from one container to another, we can copy the member id to the config files.
  3. It doe not require the consumer to be able to access another dir on the local disks (think your consumers are deployed on AWS with remote disks mounted).
  4. By allowing consumers to optionally specifying a member id, this rebalance benefit can be easily migrated to connect and streams as well which relies on consumers, even in a cloud environment.



  • No labels