This page is meant as a template for writing a KIP. To create a KIP choose Tools->Copy on this page and modify with your content and replace the heading with the next KIP number and a description of your issue. Replace anything in italics with your own description.
Status
Current state: [One of "Under Discussion", "Accepted", "Rejected"]
Discussion thread: here [Change the link from the KIP proposal email archive to your own email thread]
JIRA: here [Change the link from KAFKA-1 to your own ticket]
Please keep the discussion on the mailing list rather than commenting on the wiki (wiki discussions get unwieldy fast).
Motivation
Consumer semantics is very useful for distributed processing of the data, however the granularity of parallelism doesn’t satisfy the scaling need when number of topic partitions < number of consumers. Nowadays Kafka client users would do the capacity planning beforehand to allow 5X ~ 10X future traffic increase. This aims to avoid hitting the future scalability issue at the best effort, but still possible that eventually the traffic goes beyond the original planning, and user has to face the unfortunate online migration. One solution we have considered is to do online partition expanding. The proposal was not continuing to evolve due to its complexity. A second option which also painful is to switch input topic on the fly. As of today, the switch process is manual and cumbersome.
In the infra cost perspective, pre-define a higher number of partitions will definitely increase the network traffic as more metadata and replication will be needed. Besides extra money paid, the operation overhead increases while maintaining broker cluster in good shape with more topic partitions beyond necessity. It's been a known pain point for Kafka streaming processing scalability which is of great value to be resolved.
Further more take Kafka Streams as an example, the processing model honors the partition level ordering. However, most operations such as join, aggregation and so on are per-key level, so the relative order across different keys is not necessary, except for user customized operations.
The proposal here, is to decouple the consumption and physical partition count, by making consumers capable of collaborating on the same topic partition. There are a couple of benefits compared with existing model:
- Data consume and produce scales are no longer coupled. This means we could save money by configuring a reasonable input topic with decent amount of partitions.
- Better avoid partition level hotkeys. When a specific key is processing really slow, the decoupled key based consumption could bypass it and make progress on other keys.
No operation overhead for scaling out. Users just need to add more consumer/stream capacity to unblock even there are fewer consumers.
Proposed Changes
We want to clarify beforehand that this KIP would be a starting point of a transformational change on the Kafka client consumption semantics. It's not possible to have all the design details rolling out in one shot. Instead, the focus is to define a clear roadmap of what things need to be done, and better illustrate the long term plan while getting some concrete tasks in starting steps.
Use Case Scenario
As stated above, the scaling cap for consumer based application is the number of input partitions. In an extreme scenario when there is single input partition with two consumers, one must be idle. If the single box consumer could not keep up the speed of processing, there is no solution to add more computing capacity. It would be ideal we could co-process data within one partition with two consumers when the partition level order is not required, and we could add consumer instances as many as we want.
So this cooperative consumption model applies with following limitations:
- The bottleneck is on the application processing, not data consumption which could be caused by other roots such as network saturation of broker.
- The processing semantic does not require partition level order, otherwise only one consumer could work on the input sequentially without parallelism.
Future more for pt 2, there are also different requirements for stateless and stateful operations, such as whether key level ordering is required or not. Naturally speaking, with a requirement of key level ordering, the broker needs to allocate the same message key to the same consumer within one generation as a must. For stateless operation which doesn't care about the ordering at all, the design could be much simplified as round robin assignment.
Proposed Roadmap
We would start from supporting a new offset commit semantics as this unblocks the potential to process regardless of partition level ordering. The stateful operation support is of more value in nature, so concurrently we could add supports on key based filtering with Fetch calls. The stateless and transactional supports have to be built on top of the first two steps.
Stage name | Goal | Dependency |
---|---|---|
Individual commit | Create a generic offset commit model beyond current partition → offset mapping. | No |
Key filtering based fetch on topic level | Add capability to FetchRequest with specific hashed key range or specific keys | No |
Rebalance support for concurrent assignment | Add assignor support to allow splitting single topic partition with different hash range | Key based filtering |
Transactional support | Incorporate individual commit into the transaction processing model | Key based filtering |
Support cooperative fetch on broker level | Round robin assign data to cooperative consumer fetches when there is no key-range specified | Individual commit |
High Level Design
The design will be broken down, aligning with the roadmap defined above.
Individual Commit
Note that we will use term IC to refer to the individual commit for followup discussion.
The logic behind individual commit is very straightforward. For sequential commit each topic partition will only point to one offset number, while under individual commit it is possible to have "gaps" in between. The "stable offset" is a borrowed concept from transaction semantic which is just for illustration purpose, and the idea is that stable offset marks the position where all the messages before it are already committed. The `IC` blocks refer to the individual commits that are marking records that are processed.
In the individual commit mode, the offset metadata shall grow much quicker and harder to predict. To avoid messing up the stable offset, we propose to add another internal topic called `__individual_commit_offsets` which stores the individual commits specifically, and call the current __consumed_offsets topic the primary offset topic. This isolation should make the development more controllable by avoiding messing up primary and achieve at-least-once in worst case when we need to delete the corrupted IC offset topic.
The offset commit and offset fetch workflow will be slightly changed under individual commit mode.
Offset Commit
The committed individual offset will be appended into IC offset log as normal, however the broker shall maintain an in-memory state to keep track of the individual commits. To eventually compact this topic, each IC record will go through the coordinator memory to make double check whether there could be records deleted. In the illustration scenario,
- If there is an IC [48-49] then we no longer need to maintain IC ranges [45-47] and [50-50]. We shall append 2 null records to delete those and augment the original IC into [45-50]
- If there is an IC [43-44], we are good to move stable offset forward. The operation is like:
- Push stable offset to 47 in the primary offset topic
- Append a record to remove IC [45-47] in the IC offset topic
Offset Fetch
During a consumer startup in sequential commit mode, it will attempt to fetch the stored offsets on start up before resuming work. This would be the same step for IC mode, only the consumer will be aware of both the individual offsets and stable offsets now. The individual offsets serve as a reference when user calls #poll(), such that if there are any record that's already committed, it will be filtered and the polled records will only contain committed ones. When doing the offset commit, consumer will do a "gap amending" too.
Imagine a scenario when a consumer fetches based on below scenario:
Offset: stable: 40, IC: [43-45], [48-49]
Fetched range: [40-50]
Consumer will commit only the processed offsets:
[41-42], [46-47], [50-50]
which matches the offset commit semantic on always committing offsets that are processed within current batch.
In order to cleanup unnecessary IC offsets, the OffsetCommitResponse shall also contain the information about the latest stable offset, so that consumers could safely delete any IC offsets that fall behind stable offset.
Key Based Filtering
In order to allow sharing between the consumers, broker has to be able to distinguish data records by keys. For a generic purpose, the easy way to assign records by keys is to get a hashing range and allow range split when having multiple consumers talking to the same partition. This would be our starting point to support data sharing between consumers. Suppose we do a key hashing to a space of [0, Long.MAX] while two consumers are assigned to the same partition, they will each get key range [0, Long.MAX/2 - 1] and [Long.MAX/2, Long.MAX]. This hash range assignment decision will be made during rebalance phase.
With the key based hashing, brokers will load topic partitions data into the main memory and do the partitioning work based on the key hashed range. When a FetchRequest comes with specific key range, broker will only reply the ranges that meets the need. This workflow could be implemented as a generic server side filtering, and KIP-283 is already a good example where we also attempt to load data into memory for down-conversion.
There is obvious performance penalty compared with zero-copy mechanism for existing consumer fetching, however the value of unblocking new use cases motivates us to figure out more optimizations in the long run.
With key based filtering + IC offset support, user is already capable of doing standalone mode of partition data sharing. We shall discuss the consumer API changes to support this feature in the public interface section.
Rebalance support for concurrent assignment
To determine the hash range before each consumer generation stabilizes, in the group assignment phase the leader will evaluate the situation and decide whether to trigger the sharing mechanism. For the very first version, a very intuitive criteria is comparing the relative size of topic partitions vs number of consumers. For example if number of consumers m > number of partitions n, we would do the assignment based off partitions instead of they would be sharing part of partitions for consumption in a round robin fashion. For example, if 5 consumers subscribing to 1 topic with 3 partitions, the final assignment would be:
N = Long.MAX
M1: tp1[0, N/2 - 1],
M2: tp2[0, N/2 - 1],
M3: tp3,
M4: tp1[N/2, N]
M5: tp2[N/2, N]
The new assignment comes from the fact that partitions are playing a reverse mapping to consumers. So in partition perspective, our assignment looks like a round robin assignment based off partitions:
tp1: M1, M4
tp2: M2, M5
tp3: M3
The assigned ranges could then be used by the consumer to make their fetch more specific. This step unblocks the potential to allow a dynamic consumer group scaling beyond partition level capping.
Transactional Support
Eventually IC semantic has to be compatible with transaction support. Despite the ongoing discussion of any change to the transaction semantics, we are providing a rough plan on integrating with current transaction model in this KIP as well. To avoid over-complicating the semantic, we will introduce a new delayed queue or purgatory for transactional IC semantic, where the commit shall be blocked until it could hit the stable offset. Take Kafka Streams as an example, if we have a chunk of data [40-49] with stable offset at 39, with two stream threads A and B are turning on EOS at the same time in the sharing mode:
- Stream thread A did a fetch to get [40, 44]
- Stream thread B did a fetch to get [45, 49]
- Stream thread B finished processing and issue a transactional commit of [45, 49]
- As the offset range [40, 44] is held by someone at the moment, transactional commit request will be put in a purgatory to wait for stable offset advance
- Stream thread A finished processing and issue a transactional commit of [40, 44]
- We first advance the stable offset and reply stream thread A with successful commit
- Then we search purgatory to reply stream thread B and allow it to proceed
This high level workflow is blocking in nature, but easier to reason about its correctness. The design is still open to discussion and may change as the Kafka transaction model evolves.
Cooperative Fetch
Take a look back at the stateless operations like filter or map, there is no necessity to honor the consumer → key mappings during the processing. From KIP-283, we already know it's very costly and inefficient to copy data around. Based on client's need, broker could do a random assignment when receiving fetch requests without key level granularity. It will keep an advancing marker of who has been fetching to which position. This means we don't need to load any data into the main memory and the total of IC offsets will be significantly dropped.
For example if we have a chunk of data [40-49], where offsets 41, 43, 45, 47, 49 belong to key A and 40, 42, 44, 46, 48 belong to key B, consumers owning key A will commit 5 IC offsets, same as key B owner. If we don't maintain that mapping, for consumer fetching we will first return data range [40-44] and advance the in-memory marker, and then reply [45, 49] even potentially this consumer is trying to fetch starting from offset 40, with the hope that someone else will take care of [40-44] eventually.
Cooperative fetch is more like an optimization upon our stateless use case scenario, instead of a new feature. To make it right, we have to encode more metadata within the consumer fetch request such as consumer generation and consumer id, wisely refusing advancing the marker unless necessary when 1. the consumer is zombie, 2. the same consumer is retrying the fetch 3. session timeout. And there should also be a max wait time for commit gaps so that we don't lose the chance to process the data when consumer whoever did the fetch encounters hard failures already. It could be set the same as consumer configured session timeout.
Public Interfaces
The offset commit protocol will be changed to allow IC semantic. The committed offset will include a list of offset ranges:
OffsetCommitRequest => GroupId Offsets GenerationId MemberId GroupInstanceId GroupId => String Offsets => Map<TopicPartition, CommittedOffset> GenerationId => int32, default -1 MemberId => nullable String GroupInstanceId => nullable String CommittedOffset => offset, metadata, leaderEpoch, offsetRanges Offset => int64, default -1 Metadata => nullable String LeaderEpoch => Optional<int32> OffsetRanges => List<Tuple<int64, int64>> // NEW
When the offset range is not empty, broker will only handle offset ranges and ignore the plain offset field. Optionally broker will check if the original offset field is set to -1 to make sure there is no data corruption in the request.
The offset fetch request will also be augmented to incorporate the key hash range. Using a list instead of one hash range allows future extensibility.
FetchRequest => MaxWaitTime ReplicaId MinBytes IsolationLevel FetchSessionId FetchSessionEpoch [Topics] [RemovedTopics] KeyRanges MaxWaitTime => int32 ReplicaId => int32 MinBytes => int32 IsolationLevel => int8 FetchSessionId => int32 FetchSessionEpoch => int32 Topics => TopicName Partitions TopicName => String Partitions => [Partition FetchOffset StartOffset LeaderEpoch MaxBytes] Partition => int32 CurrentLeaderEpoch => int32 FetchOffset => int64 StartOffset => int64 MaxBytes => int32 RemovedTopics => RemovedTopicName [RemovedPartition] RemovedTopicName => String RemovedPartition => int32 KeyRanges => List<Tuple<int64, int64>> // NEW
Related Work
Pulsar has officially supported key share feature in 2.4, which suggests multiple consumers could share the same partition data.
The blocker for us to implement a similar feature are 3-folds:
- Our consumer model is pull based, which incurs random read if we let consumers ask for specific keyed records. Sequential read is the key performance sugar for Kafka, as otherwise we could not bypass memory copy of the data. (Intermediate service)
- Our broker doesn’t know anything about data distribution, all the metadata is encoded and at least it has to understand the message key. In reality, we could not afford letting consumers fetch with raw keys
- Consumer coordinator is at a centralized location, however we need to distribute keys in different partitions. The burden for broadcasting is high
Compatibility, Deprecation, and Migration Plan
The upgrade path for different stages may be slightly different, but as a rule of thumb brokers need to be upgraded to the latest version first, otherwise all the interacting components would be failing.
Rejected Alternatives
There are a couple of alternatives to this proposal here.
- KIP-253 proposed physical partition expansion, which is a fairly complex implementation and could be hard to reason about correctness.
- Some discussion around making Kafka Streams as a multi-threading model where consumers are completely decoupled from processing thread. This means we have to tcommitle the concurrent processing challenge and there could be more inherent work to redesign state store semantics too.