-
Notifications
You must be signed in to change notification settings - Fork 15.1k
KAFKA-3720: Change TimeoutException to BufferExhaustedException when no memory can be allocated for a record within max.block.ms #8399
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
Changes from all commits
e6fd25e
928b45d
714731e
8fba836
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -16,8 +16,8 @@ | |
| */ | ||
| package org.apache.kafka.clients.producer.internals; | ||
|
|
||
| import org.apache.kafka.clients.producer.BufferExhaustedException; | ||
| import org.apache.kafka.common.KafkaException; | ||
| import org.apache.kafka.common.errors.TimeoutException; | ||
| import org.apache.kafka.common.metrics.Metrics; | ||
| import org.apache.kafka.common.utils.MockTime; | ||
| import org.apache.kafka.common.utils.Time; | ||
|
|
@@ -152,8 +152,18 @@ private CountDownLatch asyncAllocate(final BufferPool pool, final int size) { | |
| } | ||
|
|
||
| /** | ||
| * 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 attempt finishes soon after the maxBlockTimeMs. | ||
| * Test if BufferExhausted exception is thrown when there is not enough memory to allocate and the elapsed | ||
| * time is greater than the max specified block time. | ||
| */ | ||
| @Test(expected = BufferExhaustedException.class) | ||
| public void testBufferExhaustedExceptionIsThrown() throws Exception { | ||
| BufferPool pool = new BufferPool(2, 1, metrics, time, metricGroup); | ||
| pool.allocate(1, maxBlockTimeMs); | ||
| pool.allocate(2, maxBlockTimeMs); | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This waits on real Timer. So waiting 2 secs in a unit test is too long. Perhaps try 10ms?
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Happy to do that of course. The maxBlockTimeMs affects the entire class of tests though, I'd like to move that to a separate PR to make the change easier to trace in case tests become unstable due to this. I ran a couple thousand tests with different values and for me testBlockTimeout became unstable with a value of 10ms - afaik can tell we are betting on a race condition in line 188 . We allocate three bytes, start delayed deallocations, wait a little and then hope that at least one deallocation took place by the time we check. Which worked for 2000 ms, but apparently breaks sometimes for 10 ms. Happy to discuss this further, but maybe we can first agree on if a new PR makes sense. I think it makes sense to separate this out tbh. |
||
| } | ||
|
|
||
| /** | ||
| * Verify that a failed allocation attempt due to not enough memory finishes soon after the maxBlockTimeMs. | ||
| */ | ||
| @Test | ||
| public void testBlockTimeout() throws Exception { | ||
|
|
@@ -171,14 +181,14 @@ public void testBlockTimeout() throws Exception { | |
| try { | ||
| pool.allocate(10, maxBlockTimeMs); | ||
| fail("The buffer allocated more memory than its maximum value 10"); | ||
| } catch (TimeoutException e) { | ||
| } catch (BufferExhaustedException e) { | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Similar to the above, perhaps reduce maxBlockTimeMs to 10ms?
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. thanks @junrao - I answered in your related comment above. |
||
| // this is good | ||
| } | ||
| // 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); | ||
| assertTrue("BufferExhaustedException should not throw before maxBlockTimeMs", durationMs >= maxBlockTimeMs); | ||
| assertTrue("BufferExhaustedException should throw soon after maxBlockTimeMs", durationMs < maxBlockTimeMs + 1000); | ||
| } | ||
|
|
||
| /** | ||
|
|
@@ -191,7 +201,7 @@ public void testCleanupMemoryAvailabilityWaiterOnBlockTimeout() throws Exception | |
| try { | ||
| pool.allocate(2, maxBlockTimeMs); | ||
| fail("The buffer allocated more memory than its maximum value 2"); | ||
| } catch (TimeoutException e) { | ||
| } catch (BufferExhaustedException e) { | ||
| // this is good | ||
| } | ||
| assertEquals(0, pool.queued()); | ||
|
|
@@ -266,7 +276,7 @@ public void run() { | |
| try { | ||
| pool.allocate(2, maxBlockTimeMs); | ||
| fail("The buffer allocated more memory than its maximum value 2"); | ||
| } catch (TimeoutException e) { | ||
| } catch (BufferExhaustedException e) { | ||
| // this is good | ||
| } catch (InterruptedException e) { | ||
| // this can be neglected | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Could you add comment for this inheritance? this change is for keeping compatibility.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Done
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Is this for backwards compatibility? It seems to me that part of the reason is to allow the users to handle all timeout related type of exceptions with one catch clause.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
the client code catching TimeoutException to handle memory issue will be broken if BufferExhaustedException does not extend TimeoutException.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
It is a bit hen and egg I guess. The main reason for adding it in this case was compatibility (at least to my mind), but what you say is a very welcome side effect.
Happy to add a clarification to the comment of course if we feel this makes sense.