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

Compare with Current View Page History

« Previous Version 99 Next »

Status

Current state[Under Discussion]

Discussion thread: TBD

JIRA:  

Key Summary T Created Updated Due Assignee Reporter P Status Resolution
Loading...
Refresh


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

Motivation

Recently Kafka community is promoting cooperative rebalancing to mitigate the pain points in the stop-the-world rebalancing protocol and an initiation for Kafka Connect already started as KIP-415. There are already exciting discussions around it, but for Kafka Streams, the delayed rebalance is not the complete solution. This KIP is trying to customize the cooperative rebalancing approach specifically for KStream application context,  based on the great design for KConnect.

Currently Kafka Streams uses consumer membership protocol to coordinate the stream task assignment. When we scale up the stream application, KStream group will attempt to revoke active tasks and let the newly spinned hosts take over them. New hosts need to restore assigned tasks' state before transiting to "running". For state heavy application, it is not ideal to give up the tasks immediately once the new player joins the party, instead we should buffer some time to let the new player accept a fair amount of restoring tasks, and finish state reconstruction first before officially taking over the active tasks. Ideally, we could realize no downtime transition during cluster scaling.

In short, the goals of this KIP are:

  • Reduce unnecessary downtime due to task restoration and global application revocation.
  • Better auto scaling experience for KStream applications.
  • Stretch goal: better workload balance across KStream instances.


Background

Consumer Rebalance Protocol: Stop-The-World Effect

As mentioned in motivation, we also want to mitigate the stop-the-world effect of current global rebalance protocol. A quick recap of current rebalance semantics on KStream: when rebalance starts, all stream threads would

  1. Join group with all currently assigned tasks revoked.

  2. Wait until group assignment finish to get assigned tasks and resume working.

  3. Replay the assigned tasks state.

  4. Once all replay jobs finish, stream thread transits to running mode.

The reason for revoking all ongoing tasks is because we need to guarantee each topic partition is assigned with exactly one consumer at any time. In this way, any topic partition could not be re-assigned before it is revoked.

Streams Rebalance Metadata: Remember the PrevTasks

Today Streams embed a full fledged Consumer client, which hard-code a ConsumerCoordinator inside. Streams then injects a StreamsPartitionAssignor to its plugable PartitionAssignor interface and inside the StreamsPartitionAssignor we also have a TaskAssignor interface whose default implementation is StickyPartitionAssignor. Streams partition assignor logic today sites in the latter two classes. Hence the hierarchy today is:

KafkaConsumer -> ConsumerCoordinator -> StreamsPartitionAssignor -> StickyTaskAssignor.


StreamsPartitionAssignor uses the subscription / assignment metadata byte array field to encode additional information for sticky partitions. More specifically on subscription:

KafkaConsumer:


Subscription => TopicList SubscriptionInfo
   TopicList               => List<String>
   SubscriptionInfo        => Bytes

------------------


StreamsPartitionAssignor:

SubscriptionInfo (encoded in version 4) => VersionId LatestSupportVersionId ClientUUID PrevTasks StandbyTasks EndPoint

   VersionId               => Int32
   LatestSupportVersionId  => Int32
   ClientUUID              => 128bit
   PrevTasks               => Set<TaskId>
   StandbyTasks            => Set<TaskId>
   EndPoint                => HostInfo


And on assignment: 

KafkaConsumer:

Assignment = AssignedPartitions AssignmentInfo
   AssignedPartitions      => List<TopicPartition>
   AssignmentInfo          => Bytes

------------------

StreamsPartitionAssignor:

AssignmentInfo (encoded in version 4) => VersionId, LatestSupportedVersionId, ActiveTasks, StandbyTasks, PartitionsByHost, ErrorCode
   VersionId               => Int32
   LatestSupportVersionId  => Int32
   ActiveTasks             => List<TaskId>
   StandbyTasks            => Map<TaskId, Set<TopicPartition>>
   PartitionsByHost        => Map<HostInfo, Set<TopicPartition>>
   ErrorCode               => Int32


Streams Sticky TaskAssignor: Stickiness over Balance

Streams' StickyTaskAssignor will honor stickiness over workload balance. More specifically:

  1. First we calculate the average num.tasks each host should get on average as its "capacity", by dividing the total number of num.tasks to the total number of consumers (i.e. num.threads) and then multiple by the number of consumers that host has.
  2. Then for each task:
    1. If it has a client who owns it as its PrevTask, and that client still have capacity assign to it;
    2. Otherwise if it has a client who owns it as its StandbyTask, and that client still have capacity assign to it;
  3. If there are still unassigned tasks after step 2), then we loop over them at the per-sub-topology granularity (for workload balance), and again for each task:
    1. Find the client with the least load, and if there are multiple ones, prefer the one previously owns it, over the one previously owns it as standbyTask, over the one who does not own it at all.

As one can see, we honor stickiness (step 2) over workload balance (step 3).



Proposed Changes

We want to separate the protocol improvement into the consumer and streams layer, since streams today embeds a full-fledged consumer instance that hard-code ConsumerCoordinator.

Part I: Incremental Consumer Rebalance Protocol

We will augment the consumer's rebalance protocol as proposed in Incremental Cooperative Rebalancing: Support and Policies with some tweaks compared to KIP-415. The key idea is that, instead of relying on the single rebalance's synchronization barrier to rebalance the group and hence enforce everyone to give up all the assigned partitions before joining the group as the new generation, we use consecutive rebalances where the end of the first rebalance will actually be used as the synchronization barrier.

Consumer Protocol

More specifically, we would first inject more metadata at the consumer-layer, as:

KafkaConsumer:

Subscription => TopicList AssignedPartitions SubscriptionInfo
   TopicList               => List<String>
   AssignedPartitions      => List<TopicPartition>   // can be null
   SubscriptionInfo        => Bytes   


Assignment = AssignedPartitions RevokedPartitions AssignmentInfo
   AssignedPartitions      => List<TopicPartition>
   RevokedPartitions       => List<TopicPartition>   // can be null
   ErrorCode               => Int16
   AssignmentInfo          => Bytes


Note that it is compatible to inject additional fields before the assignor-specific SubscriptionInfo / AssignmentInfo bytes, since on serialization we would first call assignor to encode the info bytes, and then re-allocate larger buffer to append consumer-specific bytes before, and vice-versa on deserialization. So as long as we always keep the metadata bytes at the end of the protocol it is naturally compatible with the plug-in assignor. However there are indeed some compatibility challenges for the consumer protocol upgrade itself, which we will tackle below.


Consumer Coordinator Algorithm

Rebalance behavior of the consumer (captured in the consumer coordinator layer) would be changed as follows.

  1. For every consumer: before sending the join-group request, do NOT revoke any partitions; instead just encode the current assigned partitions as part of the Subscription.
  2. For every consumer: after received the sync-group request, do the following:
    1. Check that the newly assigned-partitions is a superset of Minus(assigned-partitions, revoked-partitions). This is because under cooperative rebalance, no partitions should be migrated directly before being revoked first.
    2. Check the error code as well, and depending on it move forward to c) below, or trigger a rebalance immediately (for incompatible members, see below), or fail immediately if it is fatal.
    3. Update the newly assigned-partitions, and for those newly added  partitions, call the rebalance-listener — this is the same as the current logic.
    4. If revoked partitions is not empty, remove those partitions by calling the rebalance-listener. And then immediately send another join group request with the updated assigned partitions.
  3. For the leader: after getting the received subscription topics, as well as the assigned-partitions, do the following:
    1. Call the registered assignor of the selected protocol to generate the assignment; let's call it newly-assigned-partitions.
    2. Segment the total-partitions set of partitions inferred from the newly-assigned-partitions into two exclusive sub-sets: Intersection(total-partitions, assigned-partitions), and Minus(total-partitions, assigned-partitions).
      Note that the latter is possibly non-empty because a partition maybe revoked in previous rebalance and hence not in any assigned partitions, or it is a newly created partition due to add-partitions. Let's call the former prev-assigned-partitions and the latter not-assigned-partitions.
    3. For not-assigned-partitions, we can encode the owner from the newly-assigned-partitions directly since we know no one owns it before, either due to revocation or due to newly created partitions.
    4. For prev-assigned-partitions, check if the owner has changed, if yes, encode it to the old owner in revoked-partitions but NOT encode to the assigned-partitions to the new owner.


No changes required from the broker side, since this logic change is completely wrapped inside the consumer protocol / coordinator implementation itself, and to brokers it is just the same as previous version's rebalances.

Note that one minor difference compared with KIP-415 is that we do not introduce the scheduledDelay in the protocol, but instead the consumer will trigger rebalance immediately. This is because the consumer protocol would applies to all consumers (including streams) and hence should be kept simple, and also because KIP-345 is being developed in parallel which is aimed for tackling the scaling out / rolling bounce scenarios already.

We would omit the common scenarios description here since it is already covered in KIP-415, which is very similar to this KIP with the difference of the scheduledDelay above.

Compatibility and Upgrade Path

Since we are modifying the consumer protocol as above, we need to design the upgrade path to enable consumers upgrade to the new rebalance protocol in an online manner.

Note that since we are injecting additional fields in the middle of the consumer protocol to keep the assignor-field "info" at the end, the new protocol would not be compatible with the old version. That means, an old-versioned consumer would not be able to deserialize a newer-versioned protocol data at all. Therefore when upgrading we need the new consumer byte-code to first still following the old versioned protocol for both metadata encoding, as well as the behavior (e.g. still revoking before send JoinGroup). And after everyone have upgraded to the new byte-code, we can allow them to start rebalancing with the new versioned protocol. Note that during the later rebalance, it is still possible that consumers will send join-group request with old version (but the key here is that they are all new-version aware), in which case consumer leader can freely adjust its logic based on the aggregated versions. More specifically, we introduce the following new config to Consumer:



Protocol Type
"rebalance.protocol":


type: Enum 
values: {eager, cooperative}
default: eager


When the config value is "eager", the consumer would still use V0 of the consumer protocol as well as the rebalance behavior; if the config value is "cooperative", the consumer will then use the new V1 protocol as well as the new algorithm. Note the difference with KIP-415 here, that since on consumer we cannot leverage on list of assignors to register multiple protocols and let leader to auto-switch to new versions, we need two rolling bounces instead of one rolling bounce to complete the upgrade, whereas Connect only need one rolling bounce (details below).


And we update the above algorithm on leader (i.e. bullet point 3) as, such that we will first check the versions of subscription of all the members:

  1. If all members are on V1, then follow the new algorithm.
  2. If all members are on V0, then follow the old algorithm.
  3. If there's no consensus, it means we are in a second rebalance then do the following:
    1. For those members in V1, send assignment in V1 as well by just giving back their existing assigned-partitions from the subscription metadata and leaving the revoked-partitions empty.
    2. For those members in V0, which means they have revoked their partitions but we do not know what are those partitions, we can only given them V0 assignment back with an empty assigned-partitions. 


So under case 3), those members in V0 will have a small window where no partitions are assigned, and some partitions are not assigned to anyone as well. Hence, we will add a new ConsumerCoordinatorMetrics, which client will record upon receiving the assignment with the error code

Protocol Type
"num.incompatible.rebalance": "total number of rebalances that have failed due to incompatible members joining group at the same time."


group: "consumer-coordinator-metrics"
tags: client-id=([-.\w]+)
type: Count

The client, upon receiving the assignment, would record it in a new


    1. metric called "incompatible group members"



As for the upgrade path, we would require users to do two rolling bounces, where:

  1. In the first rolling bounce, keep the rebalance.protocol as "eager" (no need to manually change anything though since it is the default value).
  2. After the first rolling bounce is completely done. Then do a second rolling bounce in which rebalance.protocol is updated to "cooperative". The above logic will make sure that eventually when everyone's sending the join-group request with V1.


Note that this proposal depends on user's correct behavior that everyone should be on the same "rebalance protocol" eventually, otherwise the we would fall into the case 3.b) forever where some partitions would not be assigned to anyone. In addition, this approach assumes that the leader would be V1-aware whenever some V1 subscription is received: again, if users follow the upgrade path above, it should be the case, but if users did not follow the guidance then it may cause consumers to crash badly due to deserialization failures. There's a few edge cases worth mentioning here:

Downgrading and Old-Versions New Member

If a consumer is downgraded after the above upgrade path is complete, it is treated as first leaving the group, and then rejoining the group as an new member with old V0. This situation can also be reflected when a new member with old version V0 is joining a team (probably mistakenly) that has been completely upgraded to V2. At this moment everyone else will still get their existing assigned-partitions and the new comer would not get anything. However if another member left the group as well, then its partitions would not be assigned to anyone due to the logic 3) above. We will add a consumer side metric after getting propagated back about this situation so that users would be notified in time.


-------------------------------


For KStream, we are going to take a trade-off between “revoking all” and “revoking none” solution: we shall only revoke tasks that are being learned since last round. So when we assign learner tasks to new members, we shall also mark active tasks as "being learned" on current owners. Every time when a rebalance begins, the task owners will revoke the being learned tasks and join group without affecting other ongoing tasks. Learned tasks could then immediately transfer ownership without attempting for a second round of rebalance upon readiness. Compared with KIP-415, we are optimizing for fewer rebalances, but increasing the metadata size and sacrificing partial availability of the learner tasks.


More specifically, we will push the following metadata bytes to the consumer's protocol:


Terminology

we shall define several terms for easy walkthrough of the algorithm.

  • Instance (A.K.A stream instance): the KStream instance serving as container of stream threads set. This could suggest a physical host or a k8s pod. The stream thread's capacity is essentially controlled by the instance relative size.
  • Learner task: a special standby task that gets assigned to one stream instance to restore a current active task and transits to active when the restoration is complete.

Learner Task Essential

Learner task shares the same semantics as standby task, which is utilized by the restore consumer to replicate active task state. When the restoration of learner task is complete, the stream instance will initiate a new JoinGroupRequest to call out another rebalance to do the task transfer. The goal of learner task is to delay the task migration when the destination host has not finished replaying the active task.

Next we are going to look at several typical scaling scenarios and edge scenarios to better understand the design of this algorithm.

Normal Scenarios

Scale Up Running Application

The newly joined stream threads will be assigned with learner tasks by the group leader and they will replay the corresponding changelogs on local first. By the end of first round of rebalance, there is no “real ownership transfer”. When new member finally finishes the replay task, it will re-attempt to join the group to indicate that it is “ready” to take on real active tasks. During second rebalance, the leader will eventually transfer the task ownership.

Scale-up
Cluster has 3 stream threads S1(leader), S2, S3, and they each own some tasks T1 ~ T5
Group stable state: S1[T1, T2], S2[T3, T4], S3[T5]

#First Rebalance 
New member S4 joins the group.
S1 performs task assignments:
	S1(assigned: [T1, T2], revoked: [], learning: [])
	S2(assigned: [T3, T4], revoked: [], learning: [])
	S3(assigned: [T5], revoked: [], learning: [])
	S4(assigned: [], revoked: [], learning: [T1])

#Second Rebalance 
New member S5 joins the group.
Member S1~S5 join with following metadata: (S4 is not ready yet)
	S1(assigned: [T2], revoked: [T1], learning: []) // T1 revoked because it's "being learned"
	S2(assigned: [T3, T4], revoked: [], learning: [])
	S3(assigned: [T5], revoked: [], learning: [])
	S4(assigned: [], revoked: [], learning: [T1])
	S5(assigned: [], revoked: [], learning: [])
S1 performs task assignments: 
	S1(assigned: [T1, T2], revoked: [], learning: [])
	S2(assigned: [T3, T4], revoked: [], learning: [])
	S3(assigned: [T5], revoked: [], learning: [])
	S4(assigned: [], revoked: [], learning: [T1])
	S5(assigned: [], revoked: [], learning: [T3])

#Third Rebalance 
Member S4 finishes its replay and becomes ready, re-attempting to join the group.
Member S1~S5 join with following status:(S5 is not ready yet)
	S1(assigned: [T2], revoked: [T1], learning: [])
	S2(assigned: [T4], revoked: [T3], learning: []) // T3 revoked because it's "being learned"
	S3(assigned: [T5], revoked: [], learning: [])
	S4(assigned: [], revoked: [], learning: [T1])
	S5(assigned: [], revoked: [], learning: [T3])
S1 performs task assignments:
	S1(assigned: [T2], revoked: [T1], learning: [])
	S2(assigned: [T3, T4], revoked: [], learning: [])
	S3(assigned: [T5], revoked: [], learning: [])
	S4(assigned: [T1], revoked: [], learning: [])
	S5(assigned: [], revoked: [], learning: [T3])

#Fourth Rebalance 
Member S5 is ready, re-attempt to join the group. 
Member S1~S5 join with following status:(S5 is not ready yet)
	S1(assigned: [T2], revoked: [], learning: [])
	S2(assigned: [T4], revoked: [T3], learning: []) // T3 revoked because it's "being learned"
	S3(assigned: [T5], revoked: [], learning: [])
	S4(assigned: [T1], revoked: [], learning: [])
	S5(assigned: [], revoked: [], learning: [T3])
S1 performs task assignments:
	S1(assigned: [T2], revoked: [], learning: [])
	S2(assigned: [T4], revoked: [T3], learning: [])
	S3(assigned: [T5], revoked: [], learning: [])
	S4(assigned: [T1], revoked: [], learning: [])
	S5(assigned: [T3], revoked: [], learning: [])
Now the group reaches balance with 5 members each owning one task.


Scale Up from Empty Group

Scaling up from scratch means all stream threads are new members. There is no need to start a learner stage because there is nothing to learn: we don’t even have a changelog topic to start with. We should be able to handle this case by identifying whether the given task is in the active task bucket for other members, if not we just transfer the ownership immediately.

After deprecating group.initial.rebalance.delay, we still expect the algorithm to work because every task assignment during rebalance will adhere to the rule "if given task is currently active, reassignment must happen only to stream threads who are declared ready to serve this task."

Scale-up from ground
Group empty state: unassigned tasks [T1, T2, T3, T4, T5]

#First Rebalance 
New member S1 joins the group
S1 performs task assignments:
S1(assigned: [T1, T2, T3, T4, T5], revoked: [], learning: []) // T1~5 not previously owned

#Second Rebalance 
New member S2, S3 joins the group
S1 performs task assignments:
S1(assigned: [T1, T2, T3, T4, T5], revoked: [], learning: []) 
S2(assigned: [], revoked: [], learning: [T3, T4])
S3(assigned: [], revoked: [], learning: [T5])

#Third Rebalance 
S2 and S3 are ready immediately after the assignment.
Member S1~S3 join with following status:
	S1(assigned: [T1, T2], revoked: [T3, T4, T5], learning: []) 
	S2(assigned: [], revoked: [], learning: [T3, T4])
	S3(assigned: [], revoked: [], learning: [T5])
S1 performs task assignments:
	S1(assigned: [T1, T2], revoked: [T3, T4, T5], learning: []) 
	S2(assigned: [T3, T4], revoked: [], learning: [])
	S3(assigned: [T5], revoked: [], learning: [])

Scale Down Running Application

When performing the scale down of stream group, it is also favorable to initiate learner tasks before actually shutting down the instances. Although standby tasks could help in this case, it requires user to pre-set num.standby.tasks which may not be available when administrator performs scaling down. Besides the standby tasks are not guaranteed up-to-date. The plan is to use command line tool to tell certain stream members that a shutdown is on the way to be executed. These informed members will send join group request to indicate that they are “leaving soon”. During assignment phase, leader will perform the learner assignment among members who are not leaving. And the leaving member will shut down itself once received the instruction to revoke all its active tasks.

For ease of operation, a new tool for scaling down the stream app shall be built. It will have access to the application instances, and ideally could do two types of scaling down:

  1. Percentage scaling. Compute targeting scaled down members while end user just needs to provide a %. For example, if the current cluster size is 40 and we choose to scale down to 80%, then the script will attempt to inform 8 of 40 hosts to “prepare leaving” the group.
  2. Name-based scaling. Name the stream instances that we want to shut down soon. This is built for online hot swapping and host replacement.

Scale-down stream applications
Group stable state: S1[T1, T2], S2[T3, T4], S3[T5]
Scaling down the application, S2 will be leaving.

#First Rebalance 
Member S2 joins the group and claims that it is leaving.
S1 performs task assignments:
	S1(assigned: [T1, T2], revoked: [], learning: [T3])
	S2(assigned: [T3, T4], revoked: [], learning: [])
	S3(assigned: [T5], revoked: [], learning: [T4])

#Second Rebalance 
S3 finishes replay first and trigger another rebalance
Member S1 ~ S3 join with following status:(S1 is not ready yet)
	S1(assigned: [T1, T2], revoked: [], learning: [T3])
	S2(assigned: [], revoked: [T3, T4], learning: []) 
	S3(assigned: [T5], revoked: [], learning: [T4])
S1 performs task assignments:
	S1(assigned: [T1, T2], revoked: [], learning: [T3])
	S2(assigned: [T3], revoked: [T4], learning: [])
	S3(assigned: [T4, T5], revoked: [], learning: [])

#Third Rebalance 
S1 finishes replay and trigger rebalance.
Member S1~S3 join with following status: 
	S1(assigned: [T1, T2], revoked: [], learning: [T3])
	S2(assigned: [], revoked: [T3], learning: []) 
	S3(assigned: [T4, T5], revoked: [], learning: [])
S1 performs task assignments:
	S1(assigned: [T1, T2, T3], revoked: [], learning: [])
	S2(assigned: [], revoked: [T3], learning: [])
	S3(assigned: [T4, T5], revoked: [], learning: [])
S2 will shutdown itself upon new assignment since there is no assigned task left.

Online Host Swapping (Scaling Up Then Down)

This is a typical use case where user wants to replace entire application's host type. Normally administrator will choose to do host swap one by one, which could cause endless KStream resource shuffling. The recommended approach under cooperative rebalancing is like:

  • Increase the capacity of the current stream job to 2X and boost up new type instances.
  • Mark existing stream instances as leaving.
  • Learner tasks finished on new hosts, shutting down old ones.
Online Swapping
Group stable state: S1[T1, T2], S2[T3, T4]
Swapping application instances, adding S3, S4 with new instance type.

#First Rebalance 
Member S3, S4 join the group.
S1 performs task assignments:
	S1(assigned: [T1, T2], revoked: [], learning: [])
	S2(assigned: [T3, T4], revoked: [], learning: [])
	S3(assigned: [], revoked: [], learning: [T2])
	S4(assigned: [], revoked: [], learning: [T4])

Use scaling tool to indicate S1 & S2 are leaving.
#Second Rebalance 
Member S1, S2 initiate rebalance to indicate state change (leaving)
Member S1~S4 join with following status: 
	S1(assigned: [T1], revoked: [T2], learning: [])
	S2(assigned: [T3], revoked: [T4], learning: []) 
	S3(assigned: [], revoked: [], learning: [T2])
	S4(assigned: [], revoked: [], learning: [T4])
S1 performs task assignments:
	S1(assigned: [T1, T2], revoked: [], learning: [])
	S2(assigned: [T3, T4], revoked: [], learning: [])
	S3(assigned: [], revoked: [], learning: [T1, T2])
	S4(assigned: [], revoked: [], learning: [T3, T4])

#Third Rebalance 
S3 and S4 finishes replay T1 ~ T4 trigger rebalance.
Member S1~S4 join with following status: 
	S1(assigned: [], revoked: [T1, T2], learning: [])
	S2(assigned: [], revoked: [T3, T4], learning: [])
	S3(assigned: [], revoked: [], learning: [T1, T2])
	S4(assigned: [], revoked: [], learning: [T3, T4])
S1 performs task assignments:
	S1(assigned: [], revoked: [], learning: [])
	S2(assigned: [], revoked: [], learning: [])
	S3(assigned: [T1, T2], revoked: [], learning: [])
	S4(assigned: [T3, T4], revoked: [], learning: [])
S1~S2 will shutdown themselves upon new assignment since there is no assigned task left.

Edge Scenarios

Backing Up Information On Leader 

Since the incremental rebalancing requires certain historical information of last round assignment, the leader stream thread will need to maintain the knowledge of:

  1. Who participated in the last round of rebalance. This is required information to track new comers.
  2. Who will be leaving the consumer group. This is for scaling down support as the replay could take longer time than the scaling down timeout. Under static membership, since we don't send leave group information, we could leverage leader to explicitly trigger rebalance when the scale-down timeout reaches. Maintaining set of leaving members are critical in making the right task shuffle judgement.

These are essential group state knowledges leader wants to memorize. To avoid the severity of leader crash during scaling, we are avoiding backing up too much information on leader for now. The following edge cases are around leader incident during scaling.

Leader Transfer During Scaling 

Leader crash could cause a missing of historical assignment information. For the learners already assigned, however, each stream thread maintains its own assignment status, so when the learner task's id has no corresponding active task running, the transfer will happen immediately. Leader switch in this case is not a big concern. 

Leader crash during scaling
Cluster has 3 stream stream threads S1(leader), S2, S3, and they own tasks T1 ~ T5
Group stable state: S1[T1, T2], S2[T3, T4], S3[T5]

#First Rebalance 
New member S4 joins the group
S1 performs task assignments:
	S1(assigned: [T1, T2], revoked: [], learning: [])
	S2(assigned: [T3, T4], revoked: [], learning: [])
	S3(assigned: [T5], revoked: [], learning: [])
	S4(assigned: [], revoked: [], learning: [T1])

#Second Rebalance
S1 crashes/gets killed before S4 is ready, S2 takes over the leader.
Member S2~S4 join with following status: 
	S2(assigned: [T3, T4], revoked: [], learning: [])
	S3(assigned: [T5], revoked: [], learning: []) 
	S4(assigned: [], revoked: [], learning: [T1])
Note that T2 is unassigned, and S4 is learning T1 which has no current active task. We 
could rebalance T1, T2 immediately.	
S2 performs task assignments:
	S2(assigned: [T3, T4], revoked: [], learning: [])
	S3(assigned: [T5, T2], revoked: [], learning: [])
	S4(assigned: [T1], revoked: [], learning: [])
Now the group reaches balance.

Leader Transfer Before Scaling 

However, if the leader dies before new instances join, the potential risk is that leader could not differentiate which stream instance is "new", because it relies on the historical information. For version 1.0, final assignment is probably not ideal in this case if we only attempt to assign learner task to new comers. This also motivates us to figure out a better task coordination strategy for load balance in long term.

Leader crash before
Cluster has 3 stream threads S1(leader), S2 and they own tasks T1 ~ T5
Group stable state: S1[T1], S2[T2, T3, T4, T5]

#First Rebalance 
New member S4 joins the group, at the same time S1 crash.
S2 takes over the leader, while T1 is not assigned now
S2 ~ S4 join with following status
	S2(assigned: [T2, T3, T4, T5], revoked: [], learning: [])
	S3(assigned: [], revoked: [], learning: []) 
	S4(assigned: [], revoked: [], learning: [])
S2 performs task assignments: (no learner assignment since S2 doesn't know S4 is new member)
	S2(assigned: [T2, T3, T4, T5], revoked: [], learning: [])
	S3(assigned: [T1], revoked: [], learning: [])
	S4(assigned: [], revoked: [], learning: [])

Now the group reaches balance, although the eventual load is skewed.

Assignment Algorithm

The above examples are focusing more on demonstrating expected behaviors with KStream incremental rebalancing "end picture". Next, we will present a holistic view of the new learner assignment algorithm during each actual rebalance.

The assignment will be broken down in the order of: active, learner and standby tasks.

Algorithm incremental-rebalancing

Input Set of Tasks,
	  Set of Instances,
      Set of Stream Threads,

      Where each stream thread contains:
		Set of active Tasks,
		Set of standby Tasks,
		owned by which instance

Main Function
	
	Assign active tasks: (if any)
		To instances with learner tasks that indicate "ready"
		To previous owners
		To unready learner tasks owners
  	 	To instances with standby tasks
		To instances who are not marked "leaving"	
		To resource available instances

	Keep existing learner tasks' assignment unchanged

 	Pick new learner tasks out of heaviest loaded instances
 
	Assign learner tasks: (if any)
		To new-coming instances with abundant resource
		To instances who are not marked "leaving"	
		To instances with corresponding standby tasks
	Prerequisite is that the instance version supports learner mechanism. 

	Assign standby tasks: (if any)
		To instances without matching active tasks
		To previous active task owners after learner transfer in this round
		To resource abundant instances
		To instances who are not marked "leaving"
	Based on num.standby.task config, standby task assignment could take multiple rounds

Output Finalized Task Assignment

Stream Task Tagging

To enable learner resource shuffling behavior, we need to have the following task status indicators to be provided:

Tag NameTask TypeExplanation
isStatefulbothIndicate whether given task has a state to restore.
isLearnerstandbyIndicate whether standby task is a learner task.
beingLearnedactiveIndicate whether active task is being learned by some other stream thread.
isReadystandbyIndicate whether standby task is ready to serve as active task.

Optimizations

Stateful vs Stateless Tasks

For stateless tasks the ownership transfer should happen immediately without the need of a learning stage, because there is nothing to restore. We should fallback the algorithm towards KIP-415 where the stateless tasks will only be revoked during second rebalance. This feature requires us to add a new tag towards a stream task, so that when we eventually consider the load balance of the stream applications, this could help us separate out tasks into two buckets and rebalance independently.

Eager Rebalance 

Sometimes the restoration time of learner tasks are not equivalent. When assigned with 1+ tasks to replay, the stream thread could require immediate rebalance as a subset of learning tasks are finished in order to speed up the load balance and resource waste of double task processing, with the sacrifice of global efficiency by introducing many more rebalances. We could supply user with a config to decide whether they want to take eager approach or stable approach eventually, with some follow-up benchmark tools of the rebalance efficiency. Example:

A stream thread S1 takes two learner tasks T1, T2, where restoring time time(T1) < time(T2). Under eager rebalance approach, the stream thread will call out rebalance immediately when T1 finishes replaying. While under conservative approach, stream thread will rejoin the group until it finishes replaying both T1 and T2.

Standby Task Utilization

Don’t forget the original purpose of standby task is to mitigate the issue during scaling down. When performing learner assignment, we shall prioritize stream threads which currently have standby tasks that match learner assignment. Therefore the group should rebalance pretty soon and let the leaving member shutdown themselves fairly quickly. 

Scale Down Timeout

User naturally wants to reach a sweet spot between ongoing task transfer and streaming resource free-up. So we want to take a similar approach as KIP-415, where we shall introduce a client config to make sure the scale down is time-bounded. If the time takes to migrate tasks outperforms this config, the leader will send out join group request and force removing active tasks on the leaving members and transfer those tasks to other staying members, so that leaving members will shut down themselves immediately after this round of rebalance.

Trade-offs

More Rebalances vs Global Efficiency

The new algorithm will invoke many more rebalances than the current protocol as one could perceive. As we have discussed in the overall incremental rebalancing design, it is not always bad to have multiple rebalances when we do it wisely, and after KIP-345 we have a future proposal to avoid scale up rebalances for static members. The goal is to pre-register the members that are planning to be added. The broker coordinator will augment the member list and wait for all the new members to join the group before rebalancing, since by default stream application’s rebalance timeout is infinity. The conclusion is that: it is server’s responsibility to avoid excessive rebalance, and client’s responsibility to make each rebalance more efficient.

Metadata Space vs Allocation Efficiency

Since we are carrying over more information during rebalance, we should be alerted on the metadata size increase. So far the hard limit is 1MB per metadata response, which means if we add-on too much information, the new protocol could hit hard failure. This is a common pain point for finding better encoding scheme for metadata if we are promoting incremental rebalancing KIPs like 415 and 429. Some thoughts from Guozhang have started in this JIRA and we will be planning to have a separate KIP discussing different encoding technologies and see which one could work.

Iteration Plan

For the smooth delivery of all the features discussed so far, the iteration is divided into four stages:

Version 1.0

Delivery goal: Scale up support, conservative rebalance

The goal of first version is to realize the foundation of learner algorithm for scaling up scenario. The leader stream thread will use previous round assignment to figure out which instances are new ones, and the learner tasks shall only be assigned to new instances onceThe reason for only implementing new instances logic is because there is a potential edge case that could break current naive learner assignment: when the number of tasks are much smaller than total cluster capacity, we could fall in endless resource shuffling. We plan to better address this issue in version 4.0 where we take eventual load balance into consideration. Some discussions have been initiated on marking task weight for a while. To me, it is unclear so far what kind of eventual balance model we are going to implement at current stage. In conclusion, we want to postpone the finalized design for eventual balance until last version.

Version 2.0

Delivery goal: Scale down support

We will focus on the delivery of scaling down support upon the success of version 1.0. We need to extend on the v1 protocol since we need existing instances to take the extra learning load. We shall break the statement in v1 which claims that "only new instances could take learner tasks". To make this happen, we need to deliver in following steps:

  1. Create new tooling for marking instances as ready to scale down.
  2. Tag the leaving information for targeted members.
  3. Scale down timeout support.

Version 3.0

Delivery goal: Eager rebalance

A detailed analysis and benchmark test need to be built before fully devoting effort to this feature. Intuitively most applications should be able to tolerate minor discrepancy of task replaying time, while the cost of extra rebalances and increased debugging complexity are definitely unfavorable. 

The version 3.0 is upon version 1.0 success, and could be done concurrently with version 2.0. We may choose to adopt or discard this change, depending on the benchmark result.

Version 4.0 (Stretch)

Delivery goal: Task state labeling, eventual workload balance

The 4.0 and the final version will take application eventual load balance into consideration. If we define a balancing factor x, the total number of tasks each instance owns should be within the range of +-x% of the expected number of tasks (according to relative instance capacity), which buffers some capacity in order to avoid imbalance. A stream.imbalance.percentage will be provided for the user to configure. The smaller this number sets to, the more strict the assignment protocol will behave. 

Eventual balance example
A group with 4 instances that have following capacities: 5, 10, 10, 15, and 80 tasks shall expect the perfect balances into:

10(5/40), 20(10/40), 20(10/40), 30(15/40) tasks.

In case we set imbalance factor to 20%
then an eventual assignment like
12, 18, 23, 27 should be stable, as all of them are not way-off the expected load assignment.


Some optimizations such as balancing the load separately for stateful tasks and stateless tasks could also be applied here. So far version 4.0 still has many unknowns and is slightly beyond the incremental rebalancing scope. Our plan is to keep iterating on the details or bake a separate KIP for balancing algorithm in the future.

Public Interfaces

We are going to add a new type of protocol called "stream" for the protocol type. 

Protocol Type
ProtocolTypes : {"consumer","connect","stream"}


Also adding new configs for user to better apply and customize the scaling change.

stream.rebalancing.mode

Default: incremental

Version 1.0

The setting to help ensure no downtime upgrade of online application.

Options : upgrading, incremental

scale.down.timeout.ms

Default: infinity

Version 2.0

Time in milliseconds to force terminate the stream thread when informed to be scaled down.

learner.partial.rebalance

Default : true

Version 3.0

If this config is set to true, new member will proactively trigger rebalance when it finishes restoring one learner task state each time, until it eventually finishes all the replaying. Otherwise, new stream thread will batch the ready call to ask for a single round of rebalance.

stream.imbalance.percentage

Default: 0.2 (20%)

Version 4.0

The tolerance of task imbalance factor between hosts to trigger rebalance.

Implementation Plan

To make sure the delivery is smooth with fundamental changes of KStream internals, we build a separate Google Doc here that could be sharable to outline the step of changes. Feel free to give your feedback on this plan while reviewing the algorithm, because some of the algorithm requirements are highly coupled with internal architecture reasoning.

Compatibility, Deprecation, and Migration Plan

Minimum Version Requirement

This change requires Kafka broker version >= 0.9, where broker will react with a rebalance when a normal consumer rejoin the encoded metadata. Client application needs to update to the earliest version which includes KIP-429 version 1.0 change.

Recommended Upgrade Procedure

As we have mentioned above, a new protocol type shall be created. To ensure smooth upgrade, we need to make sure the existing job won't fail. The procedure is like:

  • Set the `stream.rebalancing.mode` to `upgrading`, which will force the stream application to stay with protocol type "consumer".
  • Rolling restart the stream application and the change is automatically applied. This is safe because we are not changing protocol type.

In long term we are proposing a more smooth and elegant upgrade approach than the current one. However it requires broker upgrade which may not be trivial effort for the end user. So far, user could choose to take this much easier workaround.

Rejected Alternatives

N/A for the algorithm part. For implementation plan trade-off, please review the doc in implementation plan.

  • No labels