Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

KAFKA-17661: Fix flaky BufferPoolTest.testBlockTimeout #17319

Merged
merged 4 commits into from
Oct 11, 2024
Merged
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -129,14 +129,6 @@ private CountDownLatch asyncDeallocate(final BufferPool pool, final ByteBuffer b
return latch;
}

private void delayedDeallocate(final BufferPool pool, final ByteBuffer buffer, final long delayMs) {
Thread thread = new Thread(() -> {
Time.SYSTEM.sleep(delayMs);
pool.deallocate(buffer);
});
thread.start();
}

private CountDownLatch asyncAllocate(final BufferPool pool, final int size) {
final CountDownLatch completed = new CountDownLatch(1);
Thread thread = new Thread(() -> {
Expand Down Expand Up @@ -168,26 +160,13 @@ public void testBufferExhaustedExceptionIsThrown() throws Exception {
*/
@Test
public void testBlockTimeout() throws Exception {
BufferPool pool = new BufferPool(10, 1, metrics, Time.SYSTEM, metricGroup);
ByteBuffer buffer1 = pool.allocate(1, maxBlockTimeMs);
ByteBuffer buffer2 = pool.allocate(1, maxBlockTimeMs);
ByteBuffer buffer3 = pool.allocate(1, maxBlockTimeMs);
// The first two buffers will be de-allocated within maxBlockTimeMs since the most recent allocation
delayedDeallocate(pool, buffer1, maxBlockTimeMs / 2);
delayedDeallocate(pool, buffer2, maxBlockTimeMs);
// The third buffer will be de-allocated after maxBlockTimeMs since the most recent allocation
delayedDeallocate(pool, buffer3, maxBlockTimeMs / 2 * 5);
BufferPool pool = new BufferPool(2, 1, metrics, Time.SYSTEM, metricGroup);
pool.allocate(1, maxBlockTimeMs);

long beginTimeMs = Time.SYSTEM.milliseconds();
try {
pool.allocate(10, maxBlockTimeMs);
fail("The buffer allocated more memory than its maximum value 10");
} catch (BufferExhaustedException e) {
// this is good
}
// Thread scheduling sometimes means that deallocation varies by this point
assertTrue(pool.availableMemory() >= 7 && pool.availableMemory() <= 10, "available memory " + pool.availableMemory());
assertThrows(BufferExhaustedException.class, () -> pool.allocate(2, maxBlockTimeMs));
long durationMs = Time.SYSTEM.milliseconds() - beginTimeMs;

assertTrue(durationMs >= maxBlockTimeMs, "BufferExhaustedException should not throw before maxBlockTimeMs");
assertTrue(durationMs < maxBlockTimeMs + 1000, "BufferExhaustedException should throw soon after maxBlockTimeMs");
}
Expand Down