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

Compare with Current View Page History

« Previous Version 16 Next »


Status

Current state: UNDER DISCUSSION

Discussion thread

JIRASAMZA-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 each 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 stateful jobs. 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 partition 3 may be assigned to a new task after partition expansion. Because the new task handling partition 3 does not have the previous state to resume the work, a key-based counter would wrongfully start from count 0 for a specific key that was reassigned from partition 1 to partition 3 after the expansion, instead of correctly starting from the previous count held by task 1.

The goal of this proposal is to enable partition expansion of the input streams while still allowing stateful jobs in Samza to produce the correct result.

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 solution proposed 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. We will provide in this doc the link to the KIP once it is available.

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) Add interface SystemStreamPartitionGrouperWithFixedTaskNum

The new interface should extend the existing interface SystemStreamPartitionGrouper. It should include a new 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 GroupByPartitionWithFixedTaskNum and GroupBySystemStreamPartitionWithFixedTaskNum

We should create two new classes GroupByPartitionWithFixedTaskNum and GroupBySystemStreamPartitionWithFixedTaskNum which implements the interface SystemStreamPartitionGrouperWithFixedTaskNumGroupByPartitionWithFixedTaskNum (or GroupBySystemStreamPartitionWithFixedTaskNum) 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, GroupByPartitionWithFixedTaskNum 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, GroupByPartitionWithFixedTaskNum 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, GroupBySystemStreamPartitionWithFixedTaskNum 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 GroupByPartitionWithFixedTaskNum (or GroupBySystemStreamPartitionWithFixedTaskNum ) in order to allow partition expansion. Note that GroupByPartitionWithFixedTaskNum (or GroupBySystemStreamPartitionWithFixedTaskNum ) 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 GroupByPartitionWithFixedTaskNum 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. When JobCoordinator detects partition expansion of any input stream, it should shutdown all containers using the off-the-shelf Yarn API, re-calculate JobModel and restart container using the new JobModel.

 

Public Interfaces

1) Add interface SystemStreamPartitionGrouperWithFixedTaskNum with the following definition:

public interface SystemStreamPartitionGrouperWithFixedTaskNum extends SystemStreamPartitionGrouper {
  Map<TaskName, Set<SystemStreamPartition>> group(Map<SystemStreamPartition, String> previousSystemStreamPartitionMapping, Set<SystemStreamPartition> ssps);
}


3) Add class GroupByPartitionWithFixedTaskNum which implements the interface SystemStreamPartitionGrouperWithFixedTaskNum

4) Add class GroupBySystemStreamPartitionWithFixedTaskNum which implements the interface SystemStreamPartitionGrouperWithFixedTaskNum


Test Plan

To be added

Compatibility, Deprecation, and Migration Plan

- The change made in this proposal is both source backward-compatible and binary backward compatible. Their code can compile and run correctly without change.
- For users who want to enable partition expansion for its input streams, they can do the following:
  - Set grouper class to GroupByPartitionWithFixedTaskNum if the job is using GroupByPartition as grouper class
  - Set grouper class to GroupBySystemStreamPartitionWithFixedTaskNum if the job is using GroupBySystemStreamPartition as grouper class 
  - Change their custom grouper class implementation to extend the new interface if the job is using a custom grouper class implementation.
  - Set job.coordinator.monitor-partition-change to true in the job configuration
  - Run ConfigManager


Rejected Alternatives

1. Allow task number to increase instead of creating a new grouper class.

Allow 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.

2. Add the new method to the existing interface SystemStreamPartitionGrouper instead of creating a new interface.

The advantage of this alternative solution is that it requires less interface class and shorter class hierarchy. But we choose to follow the existing pattern of interface extension in Samza as we do with BalancingTaskNameGrouper.

3. Add new config and class for user to specify the new-partition to old-partition mapping strategy based on the input system.

The current proposal relies on the input system to meet the specified operational requirements. While these requirements can be satisfied by Kafka, they may or may not be satisfied by other systems such as Kinesis. We can support partition expansion for more input systems than Kafka if user is able to express the new-partition to old-partition mapping strategy. However, since we currently don't know how user is going to use such config/class to do it, we choose to keep the current SEP simple and only add new config/class when we have specific use-case for them.

 

 

 

 

 



  • No labels