junrao commented on code in PR #18459: URL: https://github.com/apache/kafka/pull/18459#discussion_r1913952687
########## core/src/test/java/kafka/server/share/SharePartitionTest.java: ########## @@ -1104,6 +1116,109 @@ public void testAcquireWithEmptyFetchRecords() { assertEquals(0, sharePartition.nextFetchOffset()); } + @Test + public void testAcquireWithBatchSizeAndSingleBatch() { + SharePartition sharePartition = SharePartitionBuilder.builder().withState(SharePartitionState.ACTIVE).build(); + // Single batch has more records than batch size. Hence, only a single batch exceeding the batch size + // should be acquired. + MemoryRecords records = memoryRecords(5); + List<AcquiredRecords> acquiredRecordsList = fetchAcquiredRecords(sharePartition.acquire( + MEMBER_ID, + 2 /* Batch size */, + 10, + new FetchPartitionData(Errors.NONE, 20, 0, records, + Optional.empty(), OptionalLong.empty(), Optional.empty(), OptionalInt.empty(), false)), + 5); + + assertArrayEquals(expectedAcquiredRecord(0, 4, 1).toArray(), acquiredRecordsList.toArray()); + assertEquals(5, sharePartition.nextFetchOffset()); + assertEquals(1, sharePartition.cachedState().size()); + assertEquals(0, sharePartition.cachedState().get(0L).firstOffset()); + assertEquals(4, sharePartition.cachedState().get(0L).lastOffset()); + assertEquals(RecordState.ACQUIRED, sharePartition.cachedState().get(0L).batchState()); + assertEquals(MEMBER_ID, sharePartition.cachedState().get(0L).batchMemberId()); + assertEquals(1, sharePartition.cachedState().get(0L).batchDeliveryCount()); + assertNull(sharePartition.cachedState().get(0L).offsetState()); + } + + @Test + public void testAcquireWithBatchSizeAndMultipleBatches() { + SharePartition sharePartition = SharePartitionBuilder.builder().withState(SharePartitionState.ACTIVE).build(); + // Create 3 batches of records. + ByteBuffer buffer = ByteBuffer.allocate(4096); + memoryRecordsBuilder(buffer, 5, 2).close(); + memoryRecordsBuilder(buffer, 5, 10).close(); + memoryRecordsBuilder(buffer, 7, 15).close(); + memoryRecordsBuilder(buffer, 5, 22).close(); + buffer.flip(); + MemoryRecords records = MemoryRecords.readableRecords(buffer); + + List<AcquiredRecords> acquiredRecordsList = fetchAcquiredRecords(sharePartition.acquire( + MEMBER_ID, + 5 /* Batch size */, + 100, + new FetchPartitionData(Errors.NONE, 20, 0, records, + Optional.empty(), OptionalLong.empty(), Optional.empty(), OptionalInt.empty(), false)), + 25 /* Gap of 3 records will also be added to first batch */); + + // Fetch expected records from 3 batches, but change the first expected record to include gap offsets. Review Comment: There are 4 batches, instead of 3. ########## core/src/main/java/kafka/server/share/SharePartition.java: ########## @@ -1176,11 +1176,12 @@ private boolean initializedOrThrowException() { }; } - private AcquiredRecords acquireNewBatchRecords( + private ShareAcquiredRecords acquireNewBatchRecords( String memberId, Iterable<? extends RecordBatch> batches, long firstOffset, long lastOffset, + int batchSize, Review Comment: Currently, maxFetchRecords is the maxPollRecords. What benefits do we get by setting batchSize smaller than maxFetchRecords? To allow acks to be sent quicker? ########## 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: The javadoc for this new param is still missing. ########## core/src/main/java/kafka/server/share/SharePartition.java: ########## @@ -627,9 +628,8 @@ public ShareAcquiredRecords acquire( if (subMap.isEmpty()) { log.trace("No cached data exists for the share partition for requested fetch batch: {}-{}", groupId, topicIdPartition); - AcquiredRecords acquiredRecords = acquireNewBatchRecords(memberId, fetchPartitionData.records.batches(), - firstBatch.baseOffset(), lastBatch.lastOffset(), maxFetchRecords); - return ShareAcquiredRecords.fromAcquiredRecords(acquiredRecords); + return acquireNewBatchRecords(memberId, fetchPartitionData.records.batches(), Review Comment: This is an existing issue. In shareFetchResponse, we always set records to all fetch batches. However, some of the batches could be excluded during acquisition. So, it's wasteful to return excluded batches in shareFetchResponse. ########## 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; + RecordBatch lastBatch = null; + // 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) { + continue; + } + + // Track last batch for filling the final batch, if required. + lastBatch = batch; + // Check if the batch is already past the last acquired offset then break. + if (batchBaseOffset > lastAcquiredOffset) { + result.add(new AcquiredRecords() + .setFirstOffset(currentFirstOffset) + .setLastOffset(lastAcquiredOffset) + .setDeliveryCount((short) 1)); Review Comment: Could we just break here since the code outside the loop does the same thing? ########## core/src/main/java/kafka/server/share/SharePartitionManager.java: ########## @@ -244,13 +244,14 @@ public CompletableFuture<Map<TopicIdPartition, PartitionData>> fetchMessages( String groupId, String memberId, FetchParams fetchParams, + int batchSize, Review Comment: missing javadoc for the new param ########## 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 Review Comment: This is an existing issue. Will it be better for BatchSize to be a group level config instead of in each share fetch request? -- 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