adixitconfluent commented on code in PR #18459: URL: https://github.com/apache/kafka/pull/18459#discussion_r1910299036
########## core/src/main/java/kafka/server/share/SharePartition.java: ########## @@ -1204,25 +1205,91 @@ private AcquiredRecords acquireNewBatchRecords( lastAcquiredOffset = lastOffsetFromBatchWithRequestOffset(batches, firstAcquiredOffset + maxFetchRecords - 1); } - // Schedule acquisition lock timeout for the batch. - AcquisitionLockTimerTask timerTask = scheduleAcquisitionLockTimeout(memberId, firstAcquiredOffset, lastAcquiredOffset); - // Add the new batch to the in-flight records along with the acquisition lock timeout task for the batch. - cachedState.put(firstAcquiredOffset, new InFlightBatch( - memberId, - firstAcquiredOffset, - lastAcquiredOffset, - RecordState.ACQUIRED, - 1, - timerTask)); + // Create batches of acquired records. + List<AcquiredRecords> acquiredRecords = createBatches(batches, firstAcquiredOffset, lastAcquiredOffset, memberId, batchSize); // if the cachedState was empty before acquiring the new batches then startOffset needs to be updated if (cachedState.firstKey() == firstAcquiredOffset) { startOffset = firstAcquiredOffset; } endOffset = lastAcquiredOffset; - return new AcquiredRecords() - .setFirstOffset(firstAcquiredOffset) - .setLastOffset(lastAcquiredOffset) - .setDeliveryCount((short) 1); + return new ShareAcquiredRecords(acquiredRecords, (int) (lastAcquiredOffset - firstAcquiredOffset + 1)); + } finally { + lock.writeLock().unlock(); + } + } + + private List<AcquiredRecords> createBatches( + Iterable<? extends RecordBatch> batches, + long firstAcquiredOffset, + long lastAcquiredOffset, + String memberId, + int batchSize + ) { + lock.writeLock().lock(); + try { + List<AcquiredRecords> result = new ArrayList<>(); + if (lastAcquiredOffset - firstAcquiredOffset + 1 <= batchSize) { + // No split of batches is required as the batch size is greater than records which + // can be acquired. + result.add(new AcquiredRecords() + .setFirstOffset(firstAcquiredOffset) + .setLastOffset(lastAcquiredOffset) + .setDeliveryCount((short) 1)); + } else { + // The batch is split into multiple batches considering batch size. + long currentFirstOffset = firstAcquiredOffset; Review Comment: shouldn't the variable name for `currentFirstOffset` be `previousBatchFirstOffset`, sounds more intuitive to me. Wdyt? ########## core/src/main/java/kafka/server/share/SharePartition.java: ########## @@ -589,6 +589,7 @@ public long nextFetchOffset() { @SuppressWarnings("cyclomaticcomplexity") // Consider refactoring to avoid suppression public ShareAcquiredRecords acquire( String memberId, + int batchSize, Review Comment: description of param `batchSize` is missing in the javadoc for this function -- 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: jira-unsubscr...@kafka.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org