junrao commented on code in PR #18459: URL: https://github.com/apache/kafka/pull/18459#discussion_r1915367062
########## core/src/test/java/kafka/server/share/SharePartitionTest.java: ########## @@ -1104,6 +1116,120 @@ 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, 6, 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)), + 26 /* Gap of 3 records will also be added to first batch */); + + // Fetch expected records from 4 batches, but change the first expected record to include gap offsets. + List<AcquiredRecords> expectedAcquiredRecords = expectedAcquiredRecords(records, 1); + expectedAcquiredRecords.remove(0); + expectedAcquiredRecords.addAll(0, expectedAcquiredRecord(2, 9, 1)); + + assertArrayEquals(expectedAcquiredRecords.toArray(), acquiredRecordsList.toArray()); + assertEquals(28, sharePartition.nextFetchOffset()); + assertEquals(4, sharePartition.cachedState().size()); + assertTrue(sharePartition.cachedState().containsKey(2L)); + assertTrue(sharePartition.cachedState().containsKey(10L)); + assertTrue(sharePartition.cachedState().containsKey(15L)); + assertTrue(sharePartition.cachedState().containsKey(22L)); + assertEquals(RecordState.ACQUIRED, sharePartition.cachedState().get(2L).batchState()); + assertEquals(RecordState.ACQUIRED, sharePartition.cachedState().get(10L).batchState()); + assertEquals(RecordState.ACQUIRED, sharePartition.cachedState().get(15L).batchState()); + assertEquals(RecordState.ACQUIRED, sharePartition.cachedState().get(22L).batchState()); + assertEquals(MEMBER_ID, sharePartition.cachedState().get(2L).batchMemberId()); + assertEquals(MEMBER_ID, sharePartition.cachedState().get(10L).batchMemberId()); + assertEquals(MEMBER_ID, sharePartition.cachedState().get(15L).batchMemberId()); + assertEquals(MEMBER_ID, sharePartition.cachedState().get(22L).batchMemberId()); + assertEquals(1, sharePartition.cachedState().get(2L).batchDeliveryCount()); + assertEquals(1, sharePartition.cachedState().get(10L).batchDeliveryCount()); + assertEquals(1, sharePartition.cachedState().get(15L).batchDeliveryCount()); + assertEquals(1, sharePartition.cachedState().get(22L).batchDeliveryCount()); + assertNull(sharePartition.cachedState().get(2L).offsetState()); + assertNull(sharePartition.cachedState().get(10L).offsetState()); + assertNull(sharePartition.cachedState().get(15L).offsetState()); + assertNull(sharePartition.cachedState().get(22L).offsetState()); + } + + @Test + public void testAcquireWithBatchSizeAndMaxFetchRecords() { Review Comment: We don't have a test that combines multiple record batches into a single acquired inFlightBatch. Should we add one? ########## core/src/main/java/kafka/server/share/SharePartition.java: ########## @@ -580,6 +580,7 @@ public long nextFetchOffset() { * fetched from the leader. * * @param memberId The member id of the client that is fetching the record. + * @param batchSize The batch size of the individual acquired records batch. Review Comment: Could we add that the unit of batch is record? ########## core/src/test/java/kafka/server/share/SharePartitionTest.java: ########## @@ -1104,6 +1116,120 @@ 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. Review Comment: Created 4 batches instead of 3. ########## 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: It seems that we could eliminate this statement. -- 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