apoorvmittal10 commented on code in PR #18459:
URL: https://github.com/apache/kafka/pull/18459#discussion_r1915462216


##########
core/src/main/java/kafka/server/share/SharePartition.java:
##########
@@ -1204,25 +1206,78 @@ 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(memberId, 
batches, firstAcquiredOffset, lastAcquiredOffset, 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)
+            return new ShareAcquiredRecords(acquiredRecords, (int) 
(lastAcquiredOffset - firstAcquiredOffset + 1));
+        } finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    private List<AcquiredRecords> createBatches(
+        String memberId,
+        Iterable<? extends RecordBatch> batches,
+        long firstAcquiredOffset,
+        long lastAcquiredOffset,
+        int batchSize
+    ) {
+        lock.writeLock().lock();
+        try {
+            List<AcquiredRecords> result = new ArrayList<>();
+            long currentFirstOffset = firstAcquiredOffset;
+            // No split of batches is required if the batch size is greater 
than records which
+            // can be acquired, else split the batch into multiple batches.
+            if (lastAcquiredOffset - firstAcquiredOffset + 1 > batchSize) {
+                // The batch is split into multiple batches considering batch 
size.
+                // Note: Try reading only the baseOffset of the batch and 
avoid reading the lastOffset
+                // as lastOffset call of RecordBatch is expensive (loads 
headers).
+                for (RecordBatch batch : batches) {
+                    long batchBaseOffset = batch.baseOffset();
+                    if (batchBaseOffset < firstAcquiredOffset) {

Review Comment:
   That is so right, I have added multiple test to validate skipping in 
different scenarios.



-- 
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

Reply via email to