Hi Chia-Ping,
Thanks for your question.

I agree that a group-level configuration for max record locks would be nice, and
also the delivery count limit. I propose to write a small KIP for AK 4.3 to 
address
this.

Today, the set of in-flight records is a contiguous range of offsets between the
SPSO and SPEO. Eventually, I expect to make it a sparse set so the distance
between the SPSO and SPEO can be much larger, even though the maximum
number of locks is still controlled by this config.

Thanks,
Andrew

On 2025/11/13 20:34:37 Chia-Ping Tsai wrote:
> hi Andrew
> 
> I have a question about `group.share.partition.max.record.locks`, and please 
> correct me if I misunderstand the design
> 
> I think `group.share.partition.max.record.locks` should be a group-level 
> configuration rather than a broker-level configuration
> 
> My concern is that this setting appears to be critical for throughputs and 
> memory usage, as it controls the distance between SPSO and the SPEO. I can 
> foresee scenarios where we would need to find a suitable value for this 
> setting to avoid burning out the server or creating a slow share group. 
> However, since `group.share.partition.max.record.locks` is currently a static 
> configuration, tweaking its value requires a broker restart. Furthermore, 
> different groups might need separate limits to flexibly balance memory 
> consumption and throughput
>  
> What do you think?
> 
> Best,
> Chia-Ping
> 
> 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