Skip to content

Commit b5f5265

Browse files
KAFKA-18796: Added more information to error message when assertion fails for acquisition lock timeout (#19247)
This PR adds extra information in assertion failed messages for tests in SharePartitionTest revolving around acquisition lock timeouts functionality. Reviewers: Andrew Schofield <[email protected]>
1 parent 82de719 commit b5f5265

File tree

2 files changed

+51
-25
lines changed

2 files changed

+51
-25
lines changed

core/src/main/java/kafka/server/share/SharePartition.java

+4
Original file line numberDiff line numberDiff line change
@@ -187,6 +187,10 @@ public static RecordState forId(byte id) {
187187
throw new IllegalArgumentException("Unknown record state id: " + id);
188188
}
189189
}
190+
191+
public byte id() {
192+
return this.id;
193+
}
190194
}
191195

192196
/**

core/src/test/java/kafka/server/share/SharePartitionTest.java

+47-25
Original file line numberDiff line numberDiff line change
@@ -2912,7 +2912,7 @@ public void testAcquisitionLockForAcquiringSingleRecord() throws InterruptedExce
29122912
sharePartition.cachedState().get(0L).batchAcquisitionLockTimeoutTask() == null &&
29132913
sharePartition.timer().size() == 0,
29142914
DEFAULT_MAX_WAIT_ACQUISITION_LOCK_TIMEOUT_MS,
2915-
() -> ACQUISITION_LOCK_NEVER_GOT_RELEASED);
2915+
() -> assertionFailedMessage(sharePartition, Map.of(0L, List.of())));
29162916

29172917
assertEquals(1, sharePartitionMetrics.acquisitionLockTimeoutPerSec().count());
29182918
assertTrue(sharePartitionMetrics.acquisitionLockTimeoutPerSec().meanRate() > 0);
@@ -2938,7 +2938,7 @@ public void testAcquisitionLockForAcquiringMultipleRecords() throws InterruptedE
29382938
&& sharePartition.cachedState().get(10L).batchDeliveryCount() == 1
29392939
&& sharePartition.cachedState().get(10L).batchAcquisitionLockTimeoutTask() == null,
29402940
DEFAULT_MAX_WAIT_ACQUISITION_LOCK_TIMEOUT_MS,
2941-
() -> ACQUISITION_LOCK_NEVER_GOT_RELEASED);
2941+
() -> assertionFailedMessage(sharePartition, Map.of(10L, List.of())));
29422942

29432943
assertEquals(5, sharePartitionMetrics.acquisitionLockTimeoutPerSec().count());
29442944
assertTrue(sharePartitionMetrics.acquisitionLockTimeoutPerSec().meanRate() > 0);
@@ -2973,7 +2973,7 @@ public void testAcquisitionLockForAcquiringMultipleRecordsWithOverlapAndNewBatch
29732973
sharePartition.cachedState().get(0L).batchAcquisitionLockTimeoutTask() == null &&
29742974
sharePartition.cachedState().get(5L).batchAcquisitionLockTimeoutTask() == null,
29752975
DEFAULT_MAX_WAIT_ACQUISITION_LOCK_TIMEOUT_MS,
2976-
() -> ACQUISITION_LOCK_NEVER_GOT_RELEASED);
2976+
() -> assertionFailedMessage(sharePartition, Map.of(0L, List.of(), 5L, List.of())));
29772977

29782978
assertEquals(10, sharePartitionMetrics.acquisitionLockTimeoutPerSec().count());
29792979
assertTrue(sharePartitionMetrics.acquisitionLockTimeoutPerSec().meanRate() > 0);
@@ -2997,7 +2997,7 @@ public void testAcquisitionLockForAcquiringSameBatchAgain() throws InterruptedEx
29972997
sharePartition.nextFetchOffset() == 10 &&
29982998
sharePartition.cachedState().get(10L).batchState() == RecordState.AVAILABLE,
29992999
DEFAULT_MAX_WAIT_ACQUISITION_LOCK_TIMEOUT_MS,
3000-
() -> ACQUISITION_LOCK_NEVER_GOT_RELEASED);
3000+
() -> assertionFailedMessage(sharePartition, Map.of(10L, List.of())));
30013001

30023002
// Acquire the same batch again.
30033003
fetchAcquiredRecords(sharePartition, memoryRecords(5, 10), 5);
@@ -3036,7 +3036,7 @@ public void testAcquisitionLockOnAcknowledgingSingleRecordBatch() throws Interru
30363036
sharePartition.cachedState().get(0L).batchDeliveryCount() == 1 &&
30373037
sharePartition.cachedState().get(0L).batchAcquisitionLockTimeoutTask() == null,
30383038
DEFAULT_MAX_WAIT_ACQUISITION_LOCK_TIMEOUT_MS,
3039-
() -> ACQUISITION_LOCK_NEVER_GOT_RELEASED);
3039+
() -> assertionFailedMessage(sharePartition, Map.of(0L, List.of())));
30403040
}
30413041

30423042
@Test
@@ -3065,7 +3065,7 @@ public void testAcquisitionLockOnAcknowledgingMultipleRecordBatch() throws Inter
30653065
sharePartition.cachedState().get(5L).batchDeliveryCount() == 1 &&
30663066
sharePartition.cachedState().get(5L).batchAcquisitionLockTimeoutTask() == null,
30673067
DEFAULT_MAX_WAIT_ACQUISITION_LOCK_TIMEOUT_MS,
3068-
() -> ACQUISITION_LOCK_NEVER_GOT_RELEASED);
3068+
() -> assertionFailedMessage(sharePartition, Map.of(5L, List.of())));
30693069
}
30703070

30713071
@Test
@@ -3115,7 +3115,7 @@ public void testAcquisitionLockOnAcknowledgingMultipleRecordBatchWithGapOffsets(
31153115
sharePartition.cachedState().get(5L).batchAcquisitionLockTimeoutTask() == null &&
31163116
sharePartition.cachedState().get(10L).batchAcquisitionLockTimeoutTask() == null,
31173117
DEFAULT_MAX_WAIT_ACQUISITION_LOCK_TIMEOUT_MS,
3118-
() -> ACQUISITION_LOCK_NEVER_GOT_RELEASED);
3118+
() -> assertionFailedMessage(sharePartition, Map.of(1L, List.of(), 5L, List.of(), 10L, List.of())));
31193119

31203120
assertEquals(RecordState.AVAILABLE, sharePartition.cachedState().get(1L).batchState());
31213121
assertEquals(RecordState.ACKNOWLEDGED, sharePartition.cachedState().get(5L).batchState());
@@ -3142,7 +3142,7 @@ public void testAcquisitionLockForAcquiringSubsetBatchAgain() throws Interrupted
31423142
sharePartition.cachedState().get(10L).batchState() == RecordState.AVAILABLE &&
31433143
sharePartition.cachedState().get(10L).batchAcquisitionLockTimeoutTask() == null,
31443144
DEFAULT_MAX_WAIT_ACQUISITION_LOCK_TIMEOUT_MS,
3145-
() -> ACQUISITION_LOCK_NEVER_GOT_RELEASED);
3145+
() -> assertionFailedMessage(sharePartition, Map.of(10L, List.of())));
31463146

31473147
// Acquire subset of records again.
31483148
fetchAcquiredRecords(sharePartition, memoryRecords(3, 12), 3);
@@ -3176,7 +3176,7 @@ public void testAcquisitionLockForAcquiringSubsetBatchAgain() throws Interrupted
31763176
expectedOffsetStateMap.equals(sharePartition.cachedState().get(10L).offsetState());
31773177
},
31783178
DEFAULT_MAX_WAIT_ACQUISITION_LOCK_TIMEOUT_MS,
3179-
() -> ACQUISITION_LOCK_NEVER_GOT_RELEASED);
3179+
() -> assertionFailedMessage(sharePartition, Map.of(10L, List.of(10L, 11L, 12L, 13L, 14L, 15L, 16L, 17L))));
31803180
assertNull(sharePartition.cachedState().get(10L).offsetState().get(10L).acquisitionLockTimeoutTask());
31813181
assertNull(sharePartition.cachedState().get(10L).offsetState().get(11L).acquisitionLockTimeoutTask());
31823182
assertNull(sharePartition.cachedState().get(10L).offsetState().get(12L).acquisitionLockTimeoutTask());
@@ -3264,7 +3264,7 @@ public void testAcquisitionLockOnAcknowledgingMultipleSubsetRecordBatchWithGapOf
32643264
expectedOffsetStateMap2.equals(sharePartition.cachedState().get(10L).offsetState());
32653265
},
32663266
DEFAULT_MAX_WAIT_ACQUISITION_LOCK_TIMEOUT_MS,
3267-
() -> ACQUISITION_LOCK_NEVER_GOT_RELEASED);
3267+
() -> assertionFailedMessage(sharePartition, Map.of(5L, List.of(5L, 6L), 10L, List.of(10L, 11L, 12L, 13L, 14L, 15L, 16L, 17L, 18L, 19L, 20L))));
32683268

32693269
assertNull(sharePartition.cachedState().get(5L).offsetState().get(5L).acquisitionLockTimeoutTask());
32703270
assertNull(sharePartition.cachedState().get(5L).offsetState().get(6L).acquisitionLockTimeoutTask());
@@ -3308,7 +3308,7 @@ public void testAcquisitionLockTimeoutCauseMaxDeliveryCountExceed() throws Inter
33083308
sharePartition.cachedState().get(10L).batchDeliveryCount() == 1 &&
33093309
sharePartition.cachedState().get(10L).batchAcquisitionLockTimeoutTask() == null,
33103310
DEFAULT_MAX_WAIT_ACQUISITION_LOCK_TIMEOUT_MS,
3311-
() -> ACQUISITION_LOCK_NEVER_GOT_RELEASED);
3311+
() -> assertionFailedMessage(sharePartition, Map.of(10L, List.of())));
33123312

33133313
fetchAcquiredRecords(sharePartition, memoryRecords(10, 10), 10);
33143314

@@ -3326,7 +3326,7 @@ public void testAcquisitionLockTimeoutCauseMaxDeliveryCountExceed() throws Inter
33263326
sharePartition.cachedState().get(10L).batchDeliveryCount() == 2 &&
33273327
sharePartition.cachedState().get(10L).batchAcquisitionLockTimeoutTask() == null,
33283328
DEFAULT_MAX_WAIT_ACQUISITION_LOCK_TIMEOUT_MS,
3329-
() -> ACQUISITION_LOCK_NEVER_GOT_RELEASED);
3329+
() -> assertionFailedMessage(sharePartition, Map.of(10L, List.of())));
33303330
}
33313331

33323332
@Test
@@ -3350,7 +3350,7 @@ public void testAcquisitionLockTimeoutCauseSPSOMoveForward() throws InterruptedE
33503350
sharePartition.cachedState().get(0L).batchDeliveryCount() == 1 &&
33513351
sharePartition.cachedState().get(0L).batchAcquisitionLockTimeoutTask() == null,
33523352
DEFAULT_MAX_WAIT_ACQUISITION_LOCK_TIMEOUT_MS,
3353-
() -> ACQUISITION_LOCK_NEVER_GOT_RELEASED);
3353+
() -> assertionFailedMessage(sharePartition, Map.of(0L, List.of())));
33543354

33553355
fetchAcquiredRecords(sharePartition, memoryRecords(5, 0), 5);
33563356

@@ -3384,7 +3384,7 @@ public void testAcquisitionLockTimeoutCauseSPSOMoveForward() throws InterruptedE
33843384
expectedOffsetStateMap.equals(sharePartition.cachedState().get(0L).offsetState());
33853385
},
33863386
DEFAULT_MAX_WAIT_ACQUISITION_LOCK_TIMEOUT_MS,
3387-
() -> ACQUISITION_LOCK_NEVER_GOT_RELEASED);
3387+
() -> assertionFailedMessage(sharePartition, Map.of(0L, List.of(0L, 1L, 2L, 3L, 4L, 5L, 6L, 7L, 8L, 9L))));
33883388

33893389
assertNull(sharePartition.cachedState().get(0L).offsetState().get(0L).acquisitionLockTimeoutTask());
33903390
assertNull(sharePartition.cachedState().get(0L).offsetState().get(1L).acquisitionLockTimeoutTask());
@@ -3422,7 +3422,7 @@ public void testAcquisitionLockTimeoutCauseSPSOMoveForwardAndClearCachedState()
34223422
sharePartition.cachedState().get(0L).batchState() == RecordState.AVAILABLE &&
34233423
sharePartition.cachedState().get(0L).batchAcquisitionLockTimeoutTask() == null,
34243424
DEFAULT_MAX_WAIT_ACQUISITION_LOCK_TIMEOUT_MS,
3425-
() -> ACQUISITION_LOCK_NEVER_GOT_RELEASED);
3425+
() -> assertionFailedMessage(sharePartition, Map.of(0L, List.of())));
34263426

34273427
fetchAcquiredRecords(sharePartition, memoryRecords(10, 0), 10);
34283428

@@ -3437,7 +3437,7 @@ public void testAcquisitionLockTimeoutCauseSPSOMoveForwardAndClearCachedState()
34373437
sharePartition.cachedState().isEmpty() &&
34383438
sharePartition.nextFetchOffset() == 10,
34393439
DEFAULT_MAX_WAIT_ACQUISITION_LOCK_TIMEOUT_MS,
3440-
() -> ACQUISITION_LOCK_NEVER_GOT_RELEASED);
3440+
() -> assertionFailedMessage(sharePartition, Map.of()));
34413441
}
34423442

34433443
@Test
@@ -3459,7 +3459,7 @@ public void testAcknowledgeAfterAcquisitionLockTimeout() throws InterruptedExcep
34593459
sharePartition.cachedState().get(5L).batchState() == RecordState.AVAILABLE &&
34603460
sharePartition.cachedState().get(5L).batchAcquisitionLockTimeoutTask() == null,
34613461
DEFAULT_MAX_WAIT_ACQUISITION_LOCK_TIMEOUT_MS,
3462-
() -> ACQUISITION_LOCK_NEVER_GOT_RELEASED);
3462+
() -> assertionFailedMessage(sharePartition, Map.of(5L, List.of())));
34633463

34643464
// Acknowledge with ACCEPT type should throw InvalidRecordStateException since they've been released due to acquisition lock timeout.
34653465
CompletableFuture<Void> ackResult = sharePartition.acknowledge(MEMBER_ID,
@@ -3525,7 +3525,7 @@ public void testAcquisitionLockAfterDifferentAcknowledges() throws InterruptedEx
35253525
expectedOffsetStateMap.equals(sharePartition.cachedState().get(5L).offsetState());
35263526
},
35273527
DEFAULT_MAX_WAIT_ACQUISITION_LOCK_TIMEOUT_MS,
3528-
() -> ACQUISITION_LOCK_NEVER_GOT_RELEASED);
3528+
() -> assertionFailedMessage(sharePartition, Map.of(5L, List.of(5L, 6L, 7L, 8L, 9L))));
35293529

35303530
assertNull(sharePartition.cachedState().get(5L).offsetState().get(5L).acquisitionLockTimeoutTask());
35313531
assertNull(sharePartition.cachedState().get(5L).offsetState().get(6L).acquisitionLockTimeoutTask());
@@ -3563,7 +3563,7 @@ public void testAcquisitionLockOnBatchWithWriteShareGroupStateFailure() throws I
35633563
sharePartition.cachedState().get(5L).batchState() == RecordState.AVAILABLE &&
35643564
sharePartition.cachedState().get(5L).batchAcquisitionLockTimeoutTask() == null,
35653565
DEFAULT_MAX_WAIT_ACQUISITION_LOCK_TIMEOUT_MS,
3566-
() -> ACQUISITION_LOCK_NEVER_GOT_RELEASED);
3566+
() -> assertionFailedMessage(sharePartition, Map.of(5L, List.of())));
35673567
}
35683568

35693569
@Test
@@ -3609,7 +3609,7 @@ public void testAcquisitionLockOnOffsetWithWriteShareGroupStateFailure() throws
36093609
expectedOffsetStateMap.equals(sharePartition.cachedState().get(5L).offsetState());
36103610
},
36113611
DEFAULT_MAX_WAIT_ACQUISITION_LOCK_TIMEOUT_MS,
3612-
() -> ACQUISITION_LOCK_NEVER_GOT_RELEASED);
3612+
() -> assertionFailedMessage(sharePartition, Map.of(5L, List.of(5L, 6L, 7L, 8L, 9L, 10L))));
36133613

36143614
assertNull(sharePartition.cachedState().get(5L).offsetState().get(5L).acquisitionLockTimeoutTask());
36153615
assertNull(sharePartition.cachedState().get(5L).offsetState().get(6L).acquisitionLockTimeoutTask());
@@ -4939,7 +4939,7 @@ public void testAcquisitionLockTimeoutForBatchesPostStartOffsetMovement() throws
49394939
sharePartition.cachedState().get(35L).offsetState().equals(expectedOffsetStateMap3);
49404940
},
49414941
DEFAULT_MAX_WAIT_ACQUISITION_LOCK_TIMEOUT_MS,
4942-
() -> ACQUISITION_LOCK_NEVER_GOT_RELEASED);
4942+
() -> assertionFailedMessage(sharePartition, Map.of(5L, List.of(5L, 6L, 7L, 8L, 9L), 20L, List.of(20L, 21L, 22L, 23L, 24L), 25L, List.of(), 30L, List.of(), 35L, List.of(35L, 36L, 37L, 38L, 39L))));
49434943

49444944
assertEquals(EMPTY_MEMBER_ID, sharePartition.cachedState().get(10L).batchMemberId());
49454945
assertEquals(RecordState.ARCHIVED, sharePartition.cachedState().get(10L).batchState());
@@ -4973,7 +4973,7 @@ public void testAcquisitionLockTimeoutForBatchesPostStartOffsetMovementToStartOf
49734973
sharePartition.cachedState().get(10L).batchMemberId().equals(EMPTY_MEMBER_ID) &&
49744974
sharePartition.cachedState().get(10L).batchState() == RecordState.AVAILABLE,
49754975
DEFAULT_MAX_WAIT_ACQUISITION_LOCK_TIMEOUT_MS,
4976-
() -> ACQUISITION_LOCK_NEVER_GOT_RELEASED);
4976+
() -> assertionFailedMessage(sharePartition, Map.of(5L, List.of(), 10L, List.of())));
49774977
}
49784978

49794979
@Test
@@ -5008,7 +5008,7 @@ public void testAcquisitionLockTimeoutForBatchesPostStartOffsetMovementToMiddleO
50085008
sharePartition.cachedState().get(5L).batchState() == RecordState.ARCHIVED;
50095009
},
50105010
DEFAULT_MAX_WAIT_ACQUISITION_LOCK_TIMEOUT_MS,
5011-
() -> ACQUISITION_LOCK_NEVER_GOT_RELEASED);
5011+
() -> assertionFailedMessage(sharePartition, Map.of(5L, List.of(), 10L, List.of(10L, 11L, 12L, 13L, 14L))));
50125012
}
50135013

50145014
@Test
@@ -5193,7 +5193,7 @@ public void testLsoMovementThenAcquisitionLockTimeoutThenAcknowledge() throws In
51935193
() -> sharePartition.nextFetchOffset() == 7 && sharePartition.cachedState().isEmpty() &&
51945194
sharePartition.startOffset() == 7 && sharePartition.endOffset() == 7,
51955195
DEFAULT_MAX_WAIT_ACQUISITION_LOCK_TIMEOUT_MS,
5196-
() -> ACQUISITION_LOCK_NEVER_GOT_RELEASED);
5196+
() -> assertionFailedMessage(sharePartition, Map.of()));
51975197

51985198
fetchAcquiredRecords(sharePartition, memoryRecords(5, 10), 5);
51995199

@@ -5242,7 +5242,7 @@ public void testLsoMovementThenAcquisitionLockTimeoutThenAcknowledgeBatchLastOff
52425242
() -> sharePartition.nextFetchOffset() == 3 && sharePartition.cachedState().isEmpty() &&
52435243
sharePartition.startOffset() == 3 && sharePartition.endOffset() == 3,
52445244
DEFAULT_MAX_WAIT_ACQUISITION_LOCK_TIMEOUT_MS,
5245-
() -> ACQUISITION_LOCK_NEVER_GOT_RELEASED);
5245+
() -> assertionFailedMessage(sharePartition, Map.of()));
52465246

52475247
fetchAcquiredRecords(sharePartition, memoryRecords(2, 3), 2);
52485248
fetchAcquiredRecords(sharePartition, memoryRecords(3, 5), 3);
@@ -6648,6 +6648,28 @@ public void testAcquireWhenBatchesRemovedForFetchOffsetForSameCachedBatch() {
66486648
});
66496649
});
66506650
}
6651+
6652+
private String assertionFailedMessage(SharePartition sharePartition, Map<Long, List<Long>> offsets) {
6653+
StringBuilder errorMessage = new StringBuilder(ACQUISITION_LOCK_NEVER_GOT_RELEASED + String.format(
6654+
" timer size: %d, next fetch offset: %d\n",
6655+
sharePartition.timer().size(),
6656+
sharePartition.nextFetchOffset()));
6657+
for (Map.Entry<Long, List<Long>> entry : offsets.entrySet()) {
6658+
if (entry.getValue() != null && !entry.getValue().isEmpty()) {
6659+
errorMessage.append(String.format("batch start offset: %d\n", entry.getKey()));
6660+
for (Long offset : entry.getValue()) {
6661+
errorMessage.append(String.format("\toffset: %d, offset state: %s, offset acquisition lock timeout task present: %b\n",
6662+
offset, sharePartition.cachedState().get(entry.getKey()).offsetState().get(offset).state().id(),
6663+
sharePartition.cachedState().get(entry.getKey()).offsetState().get(offset).acquisitionLockTimeoutTask() == null));
6664+
}
6665+
} else {
6666+
errorMessage.append(String.format("batch start offset: %d, batch state: %s, batch acquisition lock timeout task present: %b\n",
6667+
entry.getKey(), sharePartition.cachedState().get(entry.getKey()).batchState().id(),
6668+
sharePartition.cachedState().get(entry.getKey()).batchAcquisitionLockTimeoutTask() == null));
6669+
}
6670+
}
6671+
return errorMessage.toString();
6672+
}
66516673

66526674
private FetchPartitionData fetchPartitionData(Records records) {
66536675
return fetchPartitionData(records, 0);

0 commit comments

Comments
 (0)