adixitconfluent commented on code in PR #20815:
URL: https://github.com/apache/kafka/pull/20815#discussion_r2490379738
##########
core/src/test/java/kafka/server/share/SharePartitionTest.java:
##########
@@ -9893,6 +9897,341 @@ public void
testRecordArchivedWithWriteStateRPCFailure() throws InterruptedExcep
assertEquals(5, sharePartition.deliveryCompleteCount());
}
+ @Test
+ public void testAckTypeToRecordStateMapping() {
+ // This test will help catch bugs if the map changes.
+ Map<Byte, RecordState> actualMap =
SharePartition.ackTypeToRecordStateMapping();
+ assertEquals(4, actualMap.size());
+
+ Map<Byte, RecordState> expected = Map.of(
+ (byte) 0, RecordState.ARCHIVED,
+ AcknowledgeType.ACCEPT.id, RecordState.ACKNOWLEDGED,
+ AcknowledgeType.RELEASE.id, RecordState.AVAILABLE,
+ AcknowledgeType.REJECT.id, RecordState.ARCHIVED
+ );
+
+ for (byte key : expected.keySet()) {
+ assertEquals(expected.get(key), actualMap.get(key));
+ }
+ }
+
+ @Test
+ public void testFetchAckTypeMapForBatch() {
+ ShareAcknowledgementBatch batch =
mock(ShareAcknowledgementBatch.class);
+ when(batch.acknowledgeTypes()).thenReturn(List.of((byte) -1));
+ assertThrows(IllegalArgumentException.class, () ->
SharePartition.fetchAckTypeMapForBatch(batch));
+ }
+
+ @Test
+ public void testRenewAcknowledgeWithCompleteBatchAck() throws
InterruptedException {
+ Persister persister = Mockito.mock(Persister.class);
+ SharePartition sharePartition = SharePartitionBuilder.builder()
+ .withState(SharePartitionState.ACTIVE)
+ .withDefaultAcquisitionLockTimeoutMs(ACQUISITION_LOCK_TIMEOUT_MS)
+ .withMaxDeliveryCount(2)
+ .withPersister(persister)
+ .build();
+
+ List<AcquiredRecords> records = fetchAcquiredRecords(sharePartition,
memoryRecords(0, 1), 1);
+ assertEquals(1, records.size());
+ assertEquals(records.get(0).firstOffset(),
records.get(0).lastOffset());
+ assertEquals(1, sharePartition.cachedState().size());
+ InFlightBatch batch = sharePartition.cachedState().get(0L);
+ AcquisitionLockTimerTask taskOrig =
batch.batchAcquisitionLockTimeoutTask();
+
+ sharePartition.acknowledge(MEMBER_ID, List.of(new
ShareAcknowledgementBatch(0, 0, List.of(AcknowledgeType.RENEW.id))));
+ assertTrue(taskOrig.isCancelled()); // Original acq lock cancelled.
+ assertNotEquals(taskOrig, batch.batchAcquisitionLockTimeoutTask()); //
Lock changes.
+ assertEquals(1, sharePartition.timer().size()); // Timer jobs
+ assertEquals(RecordState.ACQUIRED, batch.batchState());
+ Mockito.verify(persister, Mockito.times(0)).writeState(Mockito.any());
// No persister call.
+
+ // Expire timer
+ // On expiration state will transition to AVAILABLE resulting in
persister write RPC
+ WriteShareGroupStateResult writeShareGroupStateResult =
Mockito.mock(WriteShareGroupStateResult.class);
+
Mockito.when(writeShareGroupStateResult.topicsData()).thenReturn(List.of(
+ new TopicData<>(TOPIC_ID_PARTITION.topicId(), List.of(
+ PartitionFactory.newPartitionErrorData(0, Errors.NONE.code(),
Errors.NONE.message())))));
+
when(persister.writeState(Mockito.any())).thenReturn(CompletableFuture.completedFuture(writeShareGroupStateResult));
+
+ mockTimer.advanceClock(ACQUISITION_LOCK_TIMEOUT_MS + 1); // Trigger
expire
+
+ assertNull(batch.batchAcquisitionLockTimeoutTask());
+ assertEquals(RecordState.AVAILABLE, batch.batchState()); // Verify
batch record state
+ assertEquals(0, sharePartition.timer().size()); // Timer jobs
+ Mockito.verify(persister, Mockito.times(1)).writeState(Mockito.any());
// 1 persister call.
+ }
+
+ @Test
+ public void testRenewAcknowledgeOnExpiredBatch() throws
InterruptedException {
+ Persister persister = Mockito.mock(Persister.class);
+ SharePartition sharePartition = SharePartitionBuilder.builder()
+ .withState(SharePartitionState.ACTIVE)
+ .withDefaultAcquisitionLockTimeoutMs(ACQUISITION_LOCK_TIMEOUT_MS)
+ .withMaxDeliveryCount(2)
+ .withPersister(persister)
+ .build();
+
+ List<AcquiredRecords> records = fetchAcquiredRecords(sharePartition,
memoryRecords(0, 1), 1);
+ assertEquals(1, records.size());
+ assertEquals(records.get(0).firstOffset(),
records.get(0).lastOffset());
+ assertEquals(1, sharePartition.cachedState().size());
+ InFlightBatch batch = sharePartition.cachedState().get(0L);
+ AcquisitionLockTimerTask taskOrig =
batch.batchAcquisitionLockTimeoutTask();
+
+ // Expire acq lock timeout.
+ // Persister mocking for recordState transition.
+ WriteShareGroupStateResult writeShareGroupStateResult =
Mockito.mock(WriteShareGroupStateResult.class);
+
Mockito.when(writeShareGroupStateResult.topicsData()).thenReturn(List.of(
+ new TopicData<>(TOPIC_ID_PARTITION.topicId(), List.of(
+ PartitionFactory.newPartitionErrorData(0, Errors.NONE.code(),
Errors.NONE.message())))));
+
+
when(persister.writeState(Mockito.any())).thenReturn(CompletableFuture.completedFuture(writeShareGroupStateResult));
+
+ mockTimer.advanceClock(ACQUISITION_LOCK_TIMEOUT_MS + 1);
+ TestUtils.waitForCondition(() ->
batch.batchAcquisitionLockTimeoutTask() == null, "Acq lock timeout not
cancelled.");
+ CompletableFuture<Void> future = sharePartition.acknowledge(MEMBER_ID,
List.of(new ShareAcknowledgementBatch(0, 0,
List.of(AcknowledgeType.RENEW.id))));
+
+ assertTrue(future.isCompletedExceptionally());
+ try {
+ future.get();
+ fail("No exception thrown");
+ } catch (Exception e) {
+ assertNotNull(e);
+ assertInstanceOf(InvalidRecordStateException.class, e.getCause());
+ }
+ assertTrue(taskOrig.isCancelled()); // Original acq lock cancelled.
+ assertNotEquals(taskOrig, batch.batchAcquisitionLockTimeoutTask()); //
Lock changes.
+ assertEquals(0, sharePartition.timer().size()); // Timer jobs
+ assertEquals(RecordState.AVAILABLE, batch.batchState());
+ Mockito.verify(persister, Mockito.times(1)).writeState(Mockito.any());
// 1 persister call to update record state.
+ }
+
+ @Test
+ public void testRenewAcknowledgeWithPerOffsetAck() throws
InterruptedException {
+ Persister persister = Mockito.mock(Persister.class);
+ SharePartition sharePartition = SharePartitionBuilder.builder()
+ .withState(SharePartitionState.ACTIVE)
+ .withDefaultAcquisitionLockTimeoutMs(ACQUISITION_LOCK_TIMEOUT_MS)
+ .withMaxDeliveryCount(2)
+ .withPersister(persister)
+ .build();
+
+ List<AcquiredRecords> records = fetchAcquiredRecords(sharePartition,
memoryRecords(0, 2), 2);
+ assertEquals(1, records.size());
+ assertEquals(records.get(0).firstOffset() + 1,
records.get(0).lastOffset());
+ assertEquals(1, sharePartition.cachedState().size());
+ InFlightBatch batch = sharePartition.cachedState().get(0L);
+ assertEquals(RecordState.ACQUIRED, batch.batchState());
+ AcquisitionLockTimerTask taskOrig =
batch.batchAcquisitionLockTimeoutTask();
+
+ // For ACCEPT ack call.
+ WriteShareGroupStateResult writeShareGroupStateResult =
Mockito.mock(WriteShareGroupStateResult.class);
+
Mockito.when(writeShareGroupStateResult.topicsData()).thenReturn(List.of(
+ new TopicData<>(TOPIC_ID_PARTITION.topicId(), List.of(
+ PartitionFactory.newPartitionErrorData(0, Errors.NONE.code(),
Errors.NONE.message())))));
+
+
when(persister.writeState(Mockito.any())).thenReturn(CompletableFuture.completedFuture(writeShareGroupStateResult));
+
+ sharePartition.acknowledge(MEMBER_ID, List.of(new
ShareAcknowledgementBatch(0, 1,
+ List.of(AcknowledgeType.RENEW.id, AcknowledgeType.ACCEPT.id))));
+
+ assertTrue(taskOrig.isCancelled()); // Original acq lock cancelled.
+ assertNotEquals(taskOrig,
sharePartition.cachedState().get(0L).offsetState().get(0L).acquisitionLockTimeoutTask());
+ assertNotNull(sharePartition.cachedState().get(0L).offsetState());
+
+ InFlightState offset0 =
sharePartition.cachedState().get(0L).offsetState().get(0L);
+ InFlightState offset1 =
sharePartition.cachedState().get(0L).offsetState().get(1L);
+ assertEquals(RecordState.ACQUIRED, offset0.state());
+ assertNotNull(offset0.acquisitionLockTimeoutTask());
+ assertEquals(1, sharePartition.timer().size()); // Timer jobs
+
+ assertEquals(RecordState.ACKNOWLEDGED, offset1.state());
+ assertNull(offset1.acquisitionLockTimeoutTask());
+
+ Mockito.verify(persister, Mockito.times(1)).writeState(Mockito.any());
+
+ // Expire timer
+ mockTimer.advanceClock(ACQUISITION_LOCK_TIMEOUT_MS + 1); // Trigger
expire
+
+ assertNull(offset0.acquisitionLockTimeoutTask());
+ assertEquals(RecordState.AVAILABLE, offset0.state()); // Verify
batch record state
+ assertEquals(0, sharePartition.timer().size()); // Timer jobs
+ Mockito.verify(persister, Mockito.times(2)).writeState(Mockito.any());
// 1 more persister call.
+ }
+
+ @Test
+ public void testLsoMovementWithBatchRenewal() {
+ Persister persister = Mockito.mock(Persister.class);
+ SharePartition sharePartition = SharePartitionBuilder.builder()
+ .withState(SharePartitionState.ACTIVE)
+ .withDefaultAcquisitionLockTimeoutMs(ACQUISITION_LOCK_TIMEOUT_MS)
+ .withMaxDeliveryCount(2)
+ .withPersister(persister)
+ .build();
+
+ List<AcquiredRecords> records = fetchAcquiredRecords(sharePartition,
memoryRecords(0, 10), 10);
+ assertEquals(1, records.size());
+ assertNotEquals(records.get(0).firstOffset(),
records.get(0).lastOffset());
+ assertEquals(1, sharePartition.cachedState().size());
+ InFlightBatch batch = sharePartition.cachedState().get(0L);
+ AcquisitionLockTimerTask taskOrig =
batch.batchAcquisitionLockTimeoutTask();
+
+ sharePartition.acknowledge(MEMBER_ID, List.of(new
ShareAcknowledgementBatch(0, 9, List.of(AcknowledgeType.RENEW.id))));
+ sharePartition.updateCacheAndOffsets(5);
+
+ assertEquals(10, sharePartition.nextFetchOffset());
+ assertEquals(5, sharePartition.startOffset());
+ assertEquals(9, sharePartition.endOffset());
+ assertEquals(1, sharePartition.cachedState().size());
+
+ assertEquals(MEMBER_ID,
sharePartition.cachedState().get(0L).batchMemberId());
+ assertEquals(RecordState.ACQUIRED,
sharePartition.cachedState().get(0L).batchState());
+
+ assertTrue(taskOrig.isCancelled()); // Original acq lock cancelled.
+ assertNotEquals(taskOrig, batch.batchAcquisitionLockTimeoutTask()); //
Lock changes.
+ assertEquals(1, sharePartition.timer().size()); // Timer jobs
+ Mockito.verify(persister, Mockito.times(0)).writeState(Mockito.any());
// No persister call.
+ }
+
+ @Test
+ public void testLsoMovementWithPerOffsetRenewal() throws
InterruptedException {
+ Persister persister = Mockito.mock(Persister.class);
+ SharePartition sharePartition = SharePartitionBuilder.builder()
+ .withState(SharePartitionState.ACTIVE)
+ .withDefaultAcquisitionLockTimeoutMs(ACQUISITION_LOCK_TIMEOUT_MS)
+ .withMaxDeliveryCount(2)
+ .withPersister(persister)
+ .build();
+
+ List<AcquiredRecords> records = fetchAcquiredRecords(sharePartition,
memoryRecords(0, 5), 5);
+ assertEquals(1, records.size());
+ assertEquals(records.get(0).firstOffset() + 4,
records.get(0).lastOffset());
+ assertEquals(1, sharePartition.cachedState().size());
+ InFlightBatch batch = sharePartition.cachedState().get(0L);
+ assertEquals(RecordState.ACQUIRED, batch.batchState());
+ AcquisitionLockTimerTask taskOrig =
batch.batchAcquisitionLockTimeoutTask();
+
+ // For ACCEPT ack call.
+ WriteShareGroupStateResult writeShareGroupStateResult =
Mockito.mock(WriteShareGroupStateResult.class);
+
Mockito.when(writeShareGroupStateResult.topicsData()).thenReturn(List.of(
+ new TopicData<>(TOPIC_ID_PARTITION.topicId(), List.of(
+ PartitionFactory.newPartitionErrorData(0, Errors.NONE.code(),
Errors.NONE.message())))));
+
+
when(persister.writeState(Mockito.any())).thenReturn(CompletableFuture.completedFuture(writeShareGroupStateResult));
+
+ sharePartition.acknowledge(MEMBER_ID, List.of(new
ShareAcknowledgementBatch(0, 4,
+ List.of(AcknowledgeType.RENEW.id, AcknowledgeType.ACCEPT.id,
AcknowledgeType.RENEW.id, AcknowledgeType.ACCEPT.id,
AcknowledgeType.RENEW.id))));
+
+ sharePartition.updateCacheAndOffsets(3);
+
+ assertEquals(5, sharePartition.nextFetchOffset());
+ assertEquals(3, sharePartition.startOffset());
+ assertEquals(4, sharePartition.endOffset());
+ assertEquals(1, sharePartition.cachedState().size());
+
+ assertTrue(taskOrig.isCancelled()); // Original acq lock cancelled.
+ assertNotEquals(taskOrig,
sharePartition.cachedState().get(0L).offsetState().get(0L).acquisitionLockTimeoutTask());
+ assertNotNull(sharePartition.cachedState().get(0L).offsetState());
+
+ InFlightState offset0 =
sharePartition.cachedState().get(0L).offsetState().get(0L);
+ InFlightState offset1 =
sharePartition.cachedState().get(0L).offsetState().get(1L);
+ InFlightState offset2 =
sharePartition.cachedState().get(0L).offsetState().get(2L);
+ InFlightState offset3 =
sharePartition.cachedState().get(0L).offsetState().get(3L);
+ InFlightState offset4 =
sharePartition.cachedState().get(0L).offsetState().get(4L);
+
+ assertEquals(RecordState.ACQUIRED, offset0.state());
+ assertNotNull(offset0.acquisitionLockTimeoutTask());
+
+ assertEquals(RecordState.ACKNOWLEDGED, offset1.state());
+ assertNull(offset1.acquisitionLockTimeoutTask());
+
+ assertEquals(RecordState.ACQUIRED, offset2.state());
+ assertNotNull(offset2.acquisitionLockTimeoutTask());
+
+ assertEquals(RecordState.ACKNOWLEDGED, offset3.state());
+ assertNull(offset3.acquisitionLockTimeoutTask());
+
+ assertEquals(RecordState.ACQUIRED, offset4.state());
+ assertNotNull(offset4.acquisitionLockTimeoutTask());
+
+ assertEquals(3, sharePartition.timer().size()); // Timer jobs - 3
because the renewed offsets are non-contiguous.
+
+ // Expire timer
+ mockTimer.advanceClock(ACQUISITION_LOCK_TIMEOUT_MS + 1); // Trigger
expire
+ List<RecordState> expectedStates = List.of(RecordState.ARCHIVED,
RecordState.ACKNOWLEDGED, RecordState.AVAILABLE, RecordState.ACKNOWLEDGED,
RecordState.AVAILABLE);
Review Comment:
Although the test case `testLsoMovementWithPerOffsetRenewal` passes, this is
actually wrong. The state for the offset 2 should be `ARCHIVED` not
`AVAILABLE`. There is a bug in start offset movement within code for share
partition which is causing this issue. I'd suggest adding a comment `The state
for the offset 2 should be `ARCHIVED` not `AVAILABLE`. KAFKA-19859 will fix the
incorrect start offset movement leading to the correction of this test case`
--
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]