[
https://issues.apache.org/jira/browse/KAFKA-19912?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=18042944#comment-18042944
]
Kirk True commented on KAFKA-19912:
-----------------------------------
Excellent find [~kuoche1712003]!
I performed a similar test, first reverting the fix for KAFKA-19898, and
replacing {{TimeUnit.MINUTES.sleep(1000)}} with some logging of free memory and
the number of times {{NetworkClientDelegate.poll()}} was called:
{code:java}
@Test
public void testDelayInInitializeResources() {
assertInitializeResourcesError(
TimeoutException.class,
() -> {
long delayMs = initializationTimeoutMs * 2;
org.apache.kafka.common.utils.Utils.sleep(delayMs);
return networkClientDelegate;
}
);
while (true) {
org.apache.kafka.common.utils.Utils.sleep(1000);
long count = mockingDetails(networkClientDelegate)
.getInvocations()
.stream()
.filter(invocation ->
invocation.getMethod().getName().equals("poll"))
.count();
long freeMemory = Runtime.getRuntime().freeMemory() / 1024 / 1024;
System.out.println(Instant.now() + ": " + freeMemory + " MB heap after
" + count + " invocations");
}
}
{code}
The output of a test run is shown here:
{noformat}
2025-12-05T00:39:53.889595Z: 689 MB heap after 37700 invocations
2025-12-05T00:39:54.921442Z: 969 MB heap after 97051 invocations
2025-12-05T00:39:55.962888Z: 677 MB heap after 155901 invocations
2025-12-05T00:39:57.042017Z: 746 MB heap after 217061 invocations
2025-12-05T00:39:58.102429Z: 222 MB heap after 280630 invocations
2025-12-05T00:39:59.171355Z: 462 MB heap after 342655 invocations
2025-12-05T00:40:00.296148Z: 724 MB heap after 408091 invocations
2025-12-05T00:40:01.388197Z: 456 MB heap after 473660 invocations
2025-12-05T00:40:02.490445Z: 224 MB heap after 539325 invocations
2025-12-05T00:40:03.653549Z: 767 MB heap after 605296 invocations
2025-12-05T00:40:04.791374Z: 626 MB heap after 672688 invocations
2025-12-05T00:40:05.975627Z: 659 MB heap after 703050 invocations
2025-12-05T00:40:07.117901Z: 440 MB heap after 731810 invocations
2025-12-05T00:40:08.268261Z: 186 MB heap after 764113 invocations
2025-12-05T00:40:10.038818Z: 359 MB heap after 794285 invocations
2025-12-05T00:40:11.904331Z: 306 MB heap after 823599 invocations
2025-12-05T00:40:13.757628Z: 187 MB heap after 849425 invocations
2025-12-05T00:40:16.118026Z: 289 MB heap after 869557 invocations
2025-12-05T00:40:18.614948Z: 146 MB heap after 891120 invocations
2025-12-05T00:40:21.640328Z: 132 MB heap after 904641 invocations
2025-12-05T00:40:26.827973Z: 168 MB heap after 916033 invocations
{noformat}
The test starts off logging every second, but slows down under GC pressure, and
then dies with the message {{{}Out of memory. Java heap space{}}}.
After restoring the fix for KAFKA-19898, the logging shows that closing the
{{ConsumerNetworkThread}} does indeed prevent the runaway invocations of
{{NetworkClientDelegate.poll()}} from eating up all the available heap.
{noformat}
2025-12-05T00:46:58.348585Z: 932 MB heap after 0 invocations
2025-12-05T00:46:59.354655Z: 932 MB heap after 0 invocations
2025-12-05T00:47:00.361407Z: 932 MB heap after 0 invocations
2025-12-05T00:47:01.362990Z: 932 MB heap after 0 invocations
2025-12-05T00:47:02.369314Z: 932 MB heap after 0 invocations
2025-12-05T00:47:03.373500Z: 932 MB heap after 0 invocations
2025-12-05T00:47:04.380586Z: 932 MB heap after 0 invocations
2025-12-05T00:47:05.387682Z: 932 MB heap after 0 invocations
2025-12-05T00:47:06.392393Z: 932 MB heap after 0 invocations
2025-12-05T00:47:07.397600Z: 932 MB heap after 0 invocations
2025-12-05T00:47:08.404477Z: 932 MB heap after 0 invocations
2025-12-05T00:47:09.410993Z: 932 MB heap after 0 invocations
2025-12-05T00:47:10.415244Z: 932 MB heap after 0 invocations
2025-12-05T00:47:11.422493Z: 932 MB heap after 0 invocations
2025-12-05T00:47:12.428846Z: 932 MB heap after 0 invocations
2025-12-05T00:47:13.432934Z: 932 MB heap after 0 invocations
2025-12-05T00:47:14.439636Z: 932 MB heap after 0 invocations
2025-12-05T00:47:15.445884Z: 932 MB heap after 0 invocations
2025-12-05T00:47:16.458304Z: 932 MB heap after 0 invocations
2025-12-05T00:47:17.463392Z: 932 MB heap after 0 invocations{noformat}
At this point, there doesn't appear to be any further action needed.
> Investigate OOM error on builds due to leaked thread
> ----------------------------------------------------
>
> Key: KAFKA-19912
> URL: https://issues.apache.org/jira/browse/KAFKA-19912
> Project: Kafka
> Issue Type: Bug
> Components: clients, consumer, unit tests
> Reporter: Kirk True
> Assignee: Kirk True
> Priority: Major
>
> Builds of Kafka began failing in the GitHub Actions environment because of an
> OOM error. KAFKA-19898 was filed and one of the client unit tests was
> identified as the culprit. [A fix|https://github.com/apache/kafka/pull/20930]
> was pushed quickly to resolve the issue. However, since the test is only
> executed once per run, it's not clear how a small number of dangling threads
> could have caused the entire build to experience problems.
--
This message was sent by Atlassian Jira
(v8.20.10#820010)