mirror of https://github.com/apache/kafka.git
MINOR: Less restrictive assertion in flaky BufferPool test (#5799)
Decrease the lower bound for expected available memory, as thread scheduling entails that a variable amount of deallocation happens by the point of assertion. Also make minor clarifications to test logic and comments. The passing rate improved from 98% to 100% locally after these changes (100+ runs). Reviewers: Ismael Juma <ismael@juma.me.uk> ### Committer Checklist (excluded from commit message) - [ ] Verify design and implementation - [ ] Verify test coverage and CI build status - [ ] Verify documentation (including upgrade notes)
This commit is contained in:
parent
9e088eb120
commit
74f686d3c2
|
@ -154,7 +154,7 @@ public class BufferPoolTest {
|
|||
|
||||
/**
|
||||
* Test if Timeout exception is thrown when there is not enough memory to allocate and the elapsed time is greater than the max specified block time.
|
||||
* And verify that the allocation should finish soon after the maxBlockTimeMs.
|
||||
* And verify that the allocation attempt finishes soon after the maxBlockTimeMs.
|
||||
*/
|
||||
@Test
|
||||
public void testBlockTimeout() throws Exception {
|
||||
|
@ -162,10 +162,10 @@ public class BufferPoolTest {
|
|||
ByteBuffer buffer1 = pool.allocate(1, maxBlockTimeMs);
|
||||
ByteBuffer buffer2 = pool.allocate(1, maxBlockTimeMs);
|
||||
ByteBuffer buffer3 = pool.allocate(1, maxBlockTimeMs);
|
||||
// First two buffers will be de-allocated within maxBlockTimeMs since the most recent de-allocation
|
||||
// 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 de-allocation
|
||||
// The third buffer will be de-allocated after maxBlockTimeMs since the most recent allocation
|
||||
delayedDeallocate(pool, buffer3, maxBlockTimeMs / 2 * 5);
|
||||
|
||||
long beginTimeMs = Time.SYSTEM.milliseconds();
|
||||
|
@ -175,9 +175,11 @@ public class BufferPoolTest {
|
|||
} catch (TimeoutException e) {
|
||||
// this is good
|
||||
}
|
||||
assertTrue("available memory" + pool.availableMemory(), pool.availableMemory() >= 9 && pool.availableMemory() <= 10);
|
||||
long endTimeMs = Time.SYSTEM.milliseconds();
|
||||
assertTrue("Allocation should finish not much later than maxBlockTimeMs", endTimeMs - beginTimeMs < maxBlockTimeMs + 1000);
|
||||
// Thread scheduling sometimes means that deallocation varies by this point
|
||||
assertTrue("available memory " + pool.availableMemory(), pool.availableMemory() >= 8 && pool.availableMemory() <= 10);
|
||||
long durationMs = Time.SYSTEM.milliseconds() - beginTimeMs;
|
||||
assertTrue("TimeoutException should not throw before maxBlockTimeMs", durationMs >= maxBlockTimeMs);
|
||||
assertTrue("TimeoutException should throw soon after maxBlockTimeMs", durationMs < maxBlockTimeMs + 1000);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -193,7 +195,8 @@ public class BufferPoolTest {
|
|||
} catch (TimeoutException e) {
|
||||
// this is good
|
||||
}
|
||||
assertTrue(pool.queued() == 0);
|
||||
assertEquals(0, pool.queued());
|
||||
assertEquals(1, pool.availableMemory());
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
Loading…
Reference in New Issue