Hi Jimmy,
Thanks for the KIP. Please find some comments below:

AM1: The KIP mentions the current behaviour of soft limit but it would be
helpful to explain the reasoning as well in KIP. Else it seems like the
"strict" should always be the preferred fetch behaviour. However, that's
not true. The broker never reads the actual data records, rather sends back
the batch of records as produced. Hence, say in strict mode the MaxRecords
is set to 1 but the producer generates a single batch of 5 records on log
then only 1 record will be acquired but the whole batch of 5 records will
be sent to the client. This will have higher egress from the broker and
wasted memory on the client. The strict behaviour is helpful in some
scenarios but not always.

AM2: When we say "Strict max fetch records enables clients to achieve
predictable
throughput", can you please help explain what is meant by it? An example
could help here.

AM3: As mentioned in the KIP "In scenarios where record processing is
time-consuming" hence strict mode is advisable. The client connection shall
be disconnected post session timeout configuration. Hence it means that if
processing is taking longer than the session timeout then client sessions
will be dropped and held records will be released. Shall we propose to
handle the behaviour for such scenarios in the KIP as well?

AM4: Currently, other than max and min bytes, there are 2 other parameters
in ShareFetch request 1) MaxRecords 2) BatchSize. Both of these share fetch
params currently use max.poll.records client configuration. Which means
that a single batch of records will be fetched as per max.poll.records
client configuration. But the MaxRecords and BatchSize were added because
of following reasons a) Have some predictable number of records returned
from broker as the records are backed by acquisition lock timeout, in case
client takes more time in processing higher number of records b) Generate
batches so client can "pre-fetch" record batches which can be
acknowledged individually (batch) rather waiting for all records to be
processed by client. Pre-fetch needs additional handling in client and
broker to renew the lock timeout for acquired-waiting record batches in
client, which currently does not exist. Questions:

AM4-a: What would be the suggested behaviour with "strict" mode and
BatchSize i.e. shall always only a single batch be allowed to fetch in
"strict" mode? Or there could be any reason to fetch multiple batches even
in strict mode? I am assuming, and as KIP mentions as well, applications
will generally use strict mode when the processing time is higher on
clients for records, then does it make sense to allow multiple batches?

AM4-b: As defined in the KIP-1199
<https://cwiki.apache.org/confluence/display/KAFKA/KIP-1199%3A+Add+max+record+count+limit+to+FetchRequest>,
there might be a separate config fetch.max.message.count (preferably
fetch.max.records) which will be used for MaxRecords. Hence, should we
introduce the fetch.max.records configuration in this KIP for ShareFetch
and think about how prefetching will work? Or if we want to leave this for
a separate KIP then do we want to define behaviour for MaxRecords in strict
mode i.e. should MaxRecords be same as max.poll.records and pre-fetching
should not be supported?

AM5: AcquireMode is also used by clients so should the enum AcquireMode reside
in the server module or it should be in the clients module?

Regards,
Apoorv Mittal


On Thu, Aug 21, 2025 at 6:55 PM Wang Jimmy <wangzhiwang...@gmail.com> wrote:

> Hello all,
> I would like to start a discussion on KIP-1206: Strict max fetch records
> in share fetch.
> This KIP introduces the AcquireMode in ShareFetchRequest, which provides
> two options: Strict or loose.  When strict mode is selected, we should only
> acquire records till maxFetchRecords.
>
> KIP:
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-1206:+Strict+max+fetch+records+in+share+fetch
>
> Thanks,
> Jimmy Wang
>

Reply via email to