DUE TO SPAM, SIGN-UP IS DISABLED. Goto Selfserve wiki signup and request an account.
...
In shared fetch operations, the maxFetchRecords parameter currently functions as a soft limit, meaning the actual number of records returned to the client may exceed this threshold. This is because records are read and packaged into ShareFetchResponse in batches, and the actual number of messages encapsulated in ShareFetchResponse might exceed maxFetchRecords hence currently the last acquired offset always the batches last offset.
The current implementation offers certain benefits, but it also has drawbacks:
Pros:
- Records are consistently delivered in batches, and no records that haven't been acquired will be sent to clients. This ensures there is no extra overhead incurred on either the client or server side. If the client aims to achieve the maximum throughput while having no strict expected limit on the number of returned records, the current implementation is acceptable.
...
ShareAcquireMode
A new class org.apache.kafka.clients.consumer.internals.ShareAcquireMode is is added.
| Code Block | ||
|---|---|---|
| ||
public enum ShareAcquireMode {
BATCH_OPTIMIZED("batch_optimized"),
RECORD_LIMIT("record_limit");
public final String name;
ShareAcquireMode(final String name) {
this.name = name;
}
/**
* Case-insensitive acquire mode lookup by string name.
*/
public static ShareAcquireMode of(final String name) {
return ShareAcquireMode.valueOf(name.toUpperCase(Locale.ROOT));
}
} |
...
- The records produced onto a topic-partition each contain 5 records. Then the batches would have offsets 0-4 inclusive, 5-9 inclusive, 10-14 inclusive
- a share consumer with `max.poll.records=6, batch.size=2` fetching records from the topic
- Two whole batches of records are acquired, from offsets 0 to 5 inclusive, which is 10 records in total.
- Records acquired: ShareAcquireRecord[0] (offsets 0 to 4) and ShareAcquireRecord[1] (offsets 5 to 9).
record_limit:
- The records produced onto a topic-partition each contain 5 records. Then the batches would have offsets 0-4 inclusive, 5-9 inclusive, 10-14 inclusive
- A share consumer with `max.poll.records=6, batch.size=2` fetching records from the topic
- Return two entire batches 0-9 inclusive, but only acquire records 0-5 inclusive. In this case, the broker has returned record data which has not been acquired by this consumer(offset 6 to 9).
- Records acquired: ShareAcquireRecord[0](offsets 0 to 5).
In record_limit mode, records that have not been acquired will be sent to clients, resulting in overhead on either the client or server side. Therefore, this mode should be used in the following scenarios:
...

