JimmyWang6 commented on code in PR #20246:
URL: https://github.com/apache/kafka/pull/20246#discussion_r2367208823
##########
core/src/main/java/kafka/server/share/SharePartition.java:
##########
@@ -1553,12 +1555,15 @@ private ShareAcquiredRecords acquireNewBatchRecords(
// Check how many records can be acquired from the batch.
long lastAcquiredOffset = lastOffset;
+ long maxOffset = firstAcquiredOffset + maxFetchRecords - 1;
if (maxFetchRecords < lastAcquiredOffset - firstAcquiredOffset +
1) {
- // The max records to acquire is less than the complete
available batches hence
- // limit the acquired records. The last offset shall be the
batches last offset
- // which falls under the max records limit. As the max fetch
records is the soft
- // limit, the last offset can be higher than the max records.
- lastAcquiredOffset =
lastOffsetFromBatchWithRequestOffset(batches, firstAcquiredOffset +
maxFetchRecords - 1);
+ ShareAcquireMode mode = acquireMode(acquireMode);
+ lastAcquiredOffset = switch (mode) {
+ case STRICT ->
+ maxOffsetFromFirstBatch(batches, maxOffset);
Review Comment:
Hi @apoorvmittal10 , thanks for your coments!
Do you mean that the `lastAcquiredOffset` should still be aligned on
boundaries?
Although partial batches are created here, the actual records returned from
the server by invoking
`kafka.server.share.ShareFetchUtils#maybeSliceFetchRecords` are still aligned.
The lastAcquiredOffset here is used for two purposes:
1. Updating the endOffset for SharePartition.
2. Setting the lastOffset value for AcquiredRecords.
IMHO, I don't think the firstOffset and lastOffset should be strictly
aligned to batch boundaries. I have considered the following scenario:
1. A producer sends 1000 records in **a single batch**.
2. 10 share-consumers are started with `share.max.poll.records` or
`max.poll.records` set to 1.
The expected behavior should be that each consumer receives 1 record per
poll() call, which is the current behavior. WDYT?
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]