Hello,

thank you for the proposal! A very interesting read.

I do have one question, though. When you subscribe to a topic using consumer 
groups, it might happen that one consumer has processed all messages from its 
partitions, while another one still has a lot of work to do (this might be due 
to unbalanced partitioning, long processing times etc.). In a message-queue 
approach, it would be great to solve this problem - so that a consumer that is 
free can steal work from other consumers. Is this somehow covered by share 
groups?

Maybe this is planned as "further work", as indicated here:

"
It manages the topic-partition assignments for the share-group members. An 
initial, trivial implementation would be to give each member the list of all 
topic-partitions which matches its subscriptions and then use the pull-based 
protocol to fetch records from all partitions. A more sophisticated 
implementation could use topic-partition load and lag metrics to distribute 
partitions among the consumers as a kind of autonomous, self-balancing 
partition assignment, steering more consumers to busier partitions, for 
example. Alternatively, a push-based fetching scheme could be used. Protocol 
details will follow later.
"

but I’m not sure if I understand this correctly. A fully-connected graph seems 
like a lot of connections, and I’m not sure if this would play well with 
streaming.

This also seems as one of the central problems - a key differentiator between 
share and consumer groups (the other one being persisting state of messages). 
And maybe the exact way we’d want to approach this would, to a certain degree, 
dictate the design of the queueing system?

Best,
Adam Warski

On 2023/05/15 11:55:14 Andrew Schofield wrote:
> Hi,
> I would like to start a discussion thread on KIP-932: Queues for Kafka. This 
> KIP proposes an alternative to consumer groups to enable cooperative 
> consumption by consumers without partition assignment. You end up with queue 
> semantics on top of regular Kafka topics, with per-message acknowledgement 
> and automatic handling of messages which repeatedly fail to be processed.
> 
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-932%3A+Queues+for+Kafka
> 
> Please take a look and let me know what you think.
> 
> Thanks.
> Andrew 

Reply via email to