Status
Current state: UNDER DISCUSSION
Discussion thread:
JIRA: SAMZA-1293
Released: <Samza version>
Problem and Goal
The input system used by Samza may need to expand the number of partitions of the input streams for various reasons. For example, Kafka generally needs to limit the maximum size of partition to scale up its performance. Thus the number of partitions of a Kafka topic needs to be expanded to reduce the partition size if the average byte-in-rate or retention time of the Kafka topic has doubled. However, Samza currently is not able to handle partition expansion of the input streams. For example, if user is running a stateful job using GroupByPartition as grouper and the partition of the input stream has increased from 2 to 4, then for a period of time after partition expansion, it is possible (e.g. with Kafka) that messages with a given key exists in both partition 1 an 3. Because GroupByPartition will assign partition 1 and 3 to different tasks, messages with the same key may be handled by different task/container/process and their state will be stored in different changelog partition. Thus the output from Samza may be wrong becasue this violates the assumption used by Samza for stateful jobs.
The goal of this proposal is to enable partition expansion of the input streams.
Motivation
This design doc provides a solution to increase partition number of the input streams of a stateful Samza job while still ensuring the correctness of Samze job output. The solution should work for all input systems that satisfy the operation requirement described below, i.e. 1) hash algorithm should be implemented in the form of hash(key) % partitionNum
and 2) partitionNum
should be multiplied by power of two when we increase it. The motivation of increasing partition number of Kafka topic includes 1) limit the maximum size of a partition in order to improve broker performance and 2) increase throughput of Kafka consumer in the Samza container. Other input systems may have different reasons for partition expansion.
The current solution would keep task number unchanged and use a proper partition-to-task assignment to make sure the Samza output is correct after partition expansion. An alternative solution is to allow task number to increase after partition expansion and uses a proper task-to-container assignment to make sure the Samza output is correct. The second solution, which allows task expansion, is needed in order to scale up the performance of Samza. Note that this solution would also allow partition expansion for stateful job that doesn't use join operation. However, the second solution is much more complicated to design and implement than the first solution in this doc. And it doesn't enable the partition expansion for stateful Samza jobs that uses join operation (See Rejected Alternative section), which can be addressed by this proposal. Thus, these two solutions don't replace each other and can be designed independently. We plan to use the first solution described in this doc to enable partition expansion as a low hanging fruit. The feature of task expansion is out of the scope of this proposal and will be addressed in a future SEP.
Proposed Changes
Operational Requirement of the Input System
1) Partitioning algorithms in producer
The partitioning algorithm should be implemented in the form of hash(key) % partitionNum
. User is free to choose hash function but the hashed result must be mod by the partitionNum
.
Note that this requirement is satisfied by the default partitioning algorithms in Kafka. Samza needs to repartition the user’s stream if user uses a customized partitioning algorithm in their producer implementation that doesn’t meet this requirement.
2) Stream management
The partition number of any stream that is used as input stream of stateful job should always be multiplied by power of two when we increase it.
Note that this can be trivially supported in Kafka.
3) Order of message consumption
The underlying system should ensure that messages with the same key from the same producer are always consumed in the order they are produced before and after the partition expansion.
Note that this requirement is not currently supported in the Kafka. For example, say the partition is expanded from 2 to 4. Messages with a given key may be produced to partition 1 before expansion and to partition 3 after expansion. Because Kafka consumer does not guarantee order of message delivery across partitions, it is possible that Samza consumes messages with that key from partition 3 before partition 1 which violates the requirement. Thus additional development work is needed in Kafka to meet this requirement.
Samza Implementation Change
1) Store SystemStreamPartition-to-Task assignment in the coordinator stream
Create classes SystemStreamPartitionAssignmentManager
and SetSSPTaskMapping
to read and write SystemStreamPartition-to-Task assignment in the coordinator stream. This will be done similar to how ChangelogPartitionManager
and SetChangelogMapping
are used to read and write Task-to-ChangeLogPartition assignment in the coordinator stream. The assignment should be written to the coordinator stream every time the job model is initialized.
2) Change interface SystemStreamPartitionGrouper
With the proposal in this doc, we should deprecate the existing method group(Set<SystemStreamPartition> ssps)
of the interface SystemStreamPartitionGrouper
and replace it with the method group(Map<SystemStreamPartition, String> previousSystemStreamPartitionMapping, Set<SystemStreamPartition> ssps)
. The new method takes the SystemStreamPartition-to-Task assignment from the previous job model which can be read from the coordinator stream.
3) Create class GroupByPartitionFixedTaskNum and GroupBySystemStreamPartitionFixedTaskNum
We should create two new classes GroupByPartitionFixedTaskNum
and GroupBySystemStreamPartitionFixedTaskNum
which implements the interface SystemStreamPartitionGrouper
. GroupByPartitionFixedTaskNum
(or GroupBySystemStreamPartitionFixedTaskNum
) should group system-stream-partitions in the same way as GroupByPartition
(or GroupBySystemStreamPartition
) if previousSystemStreamPartitionMapping
is empty (i.e. the job is run for the first time) or if partition of those streams has not changed since the job is created. Otherwise, GroupByPartitionFixedTaskNum
should group partitions in such a way that 1) the number of tasks consuming from any given stream does not change before and after the partition expansion; and 2) messages with the same key in the same stream will be consumed by the same task before and after the expansion.
More specifically, GroupByPartitionFixedTaskNum
will map a given SystemStreamPartition
ssp
to the taskName which is determined using the following algorithm:
- If previousSystemStreamPartitionMapping
is empty, return GroupByPartition.group(ssps).get(ssp)
, where ssps
represents to entire set of SystemStreamPartition
to be grouped.
- Calculate from previous assignment previousSystemStreamPartitionMapping
the total number of tasks that are consuming from partitions of the stream ssp.getStream()
. Denote this as taskNumForStream
.
- return previousSystemStreamPartitionMapping.get(new SystemStreamPartition(ssp.getSystem(), ssp.getStream(), ssp.getPartition() % taskNumForStream))
Similarly, GroupBySystemStreamPartitionFixedTaskNum
will map a given SystemStreamPartition
ssp
to the taskName which is determined using the following algorithm:
- If previousSystemStreamPartitionMapping
is empty, return GroupBySystemStreamPartition.group(ssps).get(ssp)
, where ssps
represents to entire set of SystemStreamPartition
to be grouped.
- Calculate from previous assignment previousSystemStreamPartitionMapping
the total number of tasks that are consuming from partitions of the stream ssp.getStream()
. Denote this as taskNumForStream
.
- return previousSystemStreamPartitionMapping.get(new SystemStreamPartition(ssp.getSystem(), ssp.getStream(), ssp.getPartition() % taskNumForStream))
Stateful Samza job which is using GroupByPartition
(or GroupBySystemStreamPartition
) as grouper class should be configured to use GroupByPartitionFixedTaskNum
(or GroupBySystemStreamPartitionFixedTaskNum
) in order to allow partition expansion. Note that GroupByPartitionFixedTaskNum
(or GroupBySystemStreamPartitionFixedTaskNum
) is backward compatible with GroupByPartition
(or GroupBySystemStreamPartition
) because they return the same partition-to-task assignment if partition doesn't expand. Thus user's job should not need to bootstrap key/value store from the changelog topic.
To help understand this algorithm, the idea is to split partitions into disjoint buckets (or groups) of partitions where the union of those buckets equals the original set of partitions. The partition-to-bucket assignment ensure that messages with the same key will be produced to the partitions of the same bucket. Then partitions in the same bucket will be assigned to the same task to ensure that messages with the same key will go to the same task.
For example, suppose partition is increased from 2 to 4 and we use GroupByPartitionFixedTaskNum
as grouper, partitions 0 and 2 should be mapped to the same task and partitions 1 and 3 should be mapped to the same task. The figure below shows the relation between partitions, buckets and tasks after we increase partition from 2 to 4.
3) Handle partition expansion while tasks are running
JobCoordinator
is already monitoring partition expansion of input streams as of current Samza implementation. And it already has logic to restart container in case of container failure. All we need to do is to let JobCoordinator
re-calculate JobModel
and restart container using the new JobModel
when partition expansion is detected.
Public Interfaces
1) Deprecate the method Map<TaskName, Set<SystemStreamPartition>> group(Set<SystemStreamPartition> ssps)
of the interface SystemStreamPartitionGrouper
2) Add method Map<TaskName, Set<SystemStreamPartition>> group(Map<SystemStreamPartition, String> previousSystemStreamPartitionMapping, Set<SystemStreamPartition> ssps)
in the interface SystemStreamPartitionGrouper
.
3) Add class GroupByPartitionFixedTaskNum
which implements the interface SystemStreamPartitionGrouper
4) Add class GroupBySystemStreamPartitionFixedTaskNum
which implements the interface SystemStreamPartitionGrouper
Test Plan
To be added
Compatibility, Deprecation, and Migration Plan
- Deprecate the method Map<TaskName, Set<SystemStreamPartition>> group(Set<SystemStreamPartition> ssps)
of the interface SystemStreamPartitionGrouper
.
- Users need to implement the newly-added method if they have a custom implementation of the interface SystemStreamPartitionGrouper
.
- Users need to use the newly-added GroupByPartitionFixedTaskNum
as the grouper class in order to deal with possible partition number change of their input streams.
Rejected Alternatives
1. Allow task number to increase instead of creating a new grouper class.
Allowing task number to increase is useful since it increases the performance of a given Samza job. However, this feature alone does not solve the problem of allowing partition expansion. For example, say we have a job that joins two streams both of which have 3 partitions. If partition number of one stream increases from 3 to 6, we would still want the task number to remain 3 to make sure that messages with the same key from both streams will be handled by the same task. This needs to be done with the new grouper classes proposed in this doc.