[ 
https://issues.apache.org/jira/browse/KAFKA-19427?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17986007#comment-17986007
 ] 

Luke Chen commented on KAFKA-19427:
-----------------------------------

So the root cause should be this config: `message.max.bytes=1073741824` (1GB). 
And like you said, you have 8 consumers, which might use 8GB of memory if all 8 
consumers are fetching from the same broker with 1GB message size each. This 
[config 
description|https://kafka.apache.org/documentation/#consumerconfigs_fetch.max.bytes]
 has clear explanation:

 

??The maximum amount of data the server should return for a fetch request. 
Records are fetched in batches by the consumer, and if the first record batch 
in the first non-empty partition of the fetch is larger than this value, the 
record batch will still be returned to ensure that the consumer can make 
progress. As such, this is not a absolute maximum. The maximum record batch 
size accepted by the broker is defined via {{message.max.bytes}} (broker 
config) or {{max.message.bytes}} (topic config). Note that the consumer 
performs multiple fetches in parallel.??

 

Even though you said this OOM won't happen in versions < 4.0, this is not a bug 
in kafka broker in my opinion. Let's see if David/ Chia-Ping has different 
opinions.

> Kafka 4.0 may have a memory leak, causing an OOM exception
> ----------------------------------------------------------
>
>                 Key: KAFKA-19427
>                 URL: https://issues.apache.org/jira/browse/KAFKA-19427
>             Project: Kafka
>          Issue Type: Bug
>          Components: consumer, group-coordinator
>    Affects Versions: 4.0.0
>            Reporter: RivenSun
>            Priority: Critical
>         Attachments: image-2025-06-23-14-16-00-554.png, 
> image-2025-06-23-14-17-34-767.png, image-2025-06-23-14-28-51-524.png, 
> image-2025-06-23-14-31-47-453.png, image-2025-06-23-15-01-32-074.png, 
> image-2025-06-23-15-04-15-708.png, image-2025-06-23-15-04-26-598.png, 
> image-2025-06-23-15-11-13-026.png, image-2025-06-23-15-33-06-851.png, 
> image-2025-06-23-15-33-26-209.png, image-2025-06-24-10-27-52-116.png, 
> image-2025-06-24-10-29-51-465.png, image-2025-06-24-10-41-40-027.png, 
> image-2025-06-24-10-43-46-826.png, image-2025-06-24-10-50-17-396.png, 
> image-2025-06-25-13-46-30-388.png
>
>
> h3. Kafka cluster configuration
> 1.Kafka version:4.0
> 2.The cluster specifications are: 3 brokers and 3 controllers
> 3.JVM startup parameters:
> !image-2025-06-23-14-16-00-554.png!
> 4.JDK version:
> !image-2025-06-23-14-17-34-767.png!
> h3. Steps to reproduce the problem
> 1.In this new cluster, create a test topic: {*}test{*},and this cluster will 
> eventually have *only this one topic* tested by external users.
> topic config : NewTopic newTopic = new NewTopic("test", 3, (short) 1);
> 2.Start the producer and send 1,000 messages
> 3.Start the consumer and use the earliest strategy for consumption. The 
> groupIds are rivenTest1/rivenTest2/.../rivenTest8
> 4.During the process of starting the consumer, it was found that some 
> consumer groups failed to start, and the coordinator brokers corresponding to 
> these groups also had OOM exceptions
> client error logs:
> {code:java}
> [main] INFO org.apache.kafka.common.telemetry.internals.KafkaMetricsCollector 
> - initializing Kafka metrics collector
> [main] INFO org.apache.kafka.common.security.authenticator.AbstractLogin - 
> Successfully logged in.
> [main] INFO org.apache.kafka.common.utils.AppInfoParser - Kafka version: 3.9.1
> [main] INFO org.apache.kafka.common.utils.AppInfoParser - Kafka commitId: 
> f745dfdcee2b9851
> [main] INFO org.apache.kafka.common.utils.AppInfoParser - Kafka startTimeMs: 
> 1750661985923
> [main] INFO org.apache.kafka.clients.consumer.internals.ClassicKafkaConsumer 
> - [Consumer clientId=consumer-rivenTest6-1, groupId=rivenTest6] Subscribed to 
> topic(s): test
> [main] INFO org.apache.kafka.clients.Metadata - [Consumer 
> clientId=consumer-rivenTest6-1, groupId=rivenTest6] Cluster ID: 
> 3esGOWhETi-zo2uHq7NsFg
> [main] INFO org.apache.kafka.clients.consumer.internals.ConsumerCoordinator - 
> [Consumer clientId=consumer-rivenTest6-1, groupId=rivenTest6] Discovered 
> group coordinator 18-97-25-88-k.mq.zoomdev.us:9889 (id: 2147483644 rack: null)
> [main] INFO org.apache.kafka.clients.consumer.internals.ConsumerCoordinator - 
> [Consumer clientId=consumer-rivenTest6-1, groupId=rivenTest6] (Re-)joining 
> group
> [main] INFO org.apache.kafka.clients.consumer.internals.ConsumerCoordinator - 
> [Consumer clientId=consumer-rivenTest6-1, groupId=rivenTest6] Request joining 
> group due to: need to re-join with the given member-id: 
> consumer-rivenTest6-1-38849218-32fa-430d-b14c-d3ce7ff402c4
> [main] INFO org.apache.kafka.clients.consumer.internals.ConsumerCoordinator - 
> [Consumer clientId=consumer-rivenTest6-1, groupId=rivenTest6] (Re-)joining 
> group
> [main] INFO org.apache.kafka.clients.consumer.internals.ConsumerCoordinator - 
> [Consumer clientId=consumer-rivenTest6-1, groupId=rivenTest6] Successfully 
> joined group with generation Generation{generationId=17, 
> memberId='consumer-rivenTest6-1-38849218-32fa-430d-b14c-d3ce7ff402c4', 
> protocol='roundrobin'}
> [main] INFO org.apache.kafka.clients.consumer.internals.ConsumerCoordinator - 
> [Consumer clientId=consumer-rivenTest6-1, groupId=rivenTest6] Finished 
> assignment for group at generation 17: 
> {consumer-rivenTest6-1-38849218-32fa-430d-b14c-d3ce7ff402c4=Assignment(partitions=[test-0,
>  test-1, test-2])}
> [main] INFO org.apache.kafka.clients.consumer.internals.ConsumerCoordinator - 
> [Consumer clientId=consumer-rivenTest6-1, groupId=rivenTest6] Request joining 
> group due to: rebalance failed due to 'Unexpected error from SyncGroup: The 
> server experienced an unexpected error when processing the request.' 
> (KafkaException)
> org.apache.kafka.common.KafkaException: Unexpected error from SyncGroup: The 
> server experienced an unexpected error when processing the request.
>     at 
> org.apache.kafka.clients.consumer.internals.AbstractCoordinator$SyncGroupResponseHandler.handle(AbstractCoordinator.java:893)
>     at 
> org.apache.kafka.clients.consumer.internals.AbstractCoordinator$SyncGroupResponseHandler.handle(AbstractCoordinator.java:812)
>     at 
> org.apache.kafka.clients.consumer.internals.AbstractCoordinator$CoordinatorResponseHandler.onSuccess(AbstractCoordinator.java:1311)
>     at 
> org.apache.kafka.clients.consumer.internals.AbstractCoordinator$CoordinatorResponseHandler.onSuccess(AbstractCoordinator.java:1286)
>     at 
> org.apache.kafka.clients.consumer.internals.RequestFuture$1.onSuccess(RequestFuture.java:206)
>     at 
> org.apache.kafka.clients.consumer.internals.RequestFuture.fireSuccess(RequestFuture.java:169)
>     at 
> org.apache.kafka.clients.consumer.internals.RequestFuture.complete(RequestFuture.java:129)
>     at 
> org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient$RequestFutureCompletionHandler.fireCompletion(ConsumerNetworkClient.java:617)
>     at 
> org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient.firePendingCompletedRequests(ConsumerNetworkClient.java:429)
>     at 
> org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient.poll(ConsumerNetworkClient.java:314)
>     at 
> org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient.poll(ConsumerNetworkClient.java:253)
>     at 
> org.apache.kafka.clients.consumer.internals.ClassicKafkaConsumer.pollForFetches(ClassicKafkaConsumer.java:692)
>     at 
> org.apache.kafka.clients.consumer.internals.ClassicKafkaConsumer.poll(ClassicKafkaConsumer.java:623)
>     at 
> org.apache.kafka.clients.consumer.internals.ClassicKafkaConsumer.poll(ClassicKafkaConsumer.java:596)
>     at 
> org.apache.kafka.clients.consumer.KafkaConsumer.poll(KafkaConsumer.java:874)
>     at us.zoom.mq.examples.ConsumerTest.startConsumer(ConsumerTest.java:233)
>     at us.zoom.mq.examples.ConsumerTest.main(ConsumerTest.java:149)
> [main] INFO org.apache.kafka.clients.consumer.internals.ConsumerCoordinator - 
> [Consumer clientId=consumer-rivenTest6-1, groupId=rivenTest6] Member 
> consumer-rivenTest6-1-38849218-32fa-430d-b14c-d3ce7ff402c4 sending LeaveGroup 
> request to coordinator 18-97-25-88-k.mq.zoomdev.us:9889 (id: 2147483644 rack: 
> null) due to the consumer is being closed
> [main] INFO org.apache.kafka.clients.consumer.internals.ConsumerCoordinator - 
> [Consumer clientId=consumer-rivenTest6-1, groupId=rivenTest6] Resetting 
> generation and member id due to: consumer pro-actively leaving the group
> [main] INFO org.apache.kafka.clients.consumer.internals.ConsumerCoordinator - 
> [Consumer clientId=consumer-rivenTest6-1, groupId=rivenTest6] Request joining 
> group due to: consumer pro-actively leaving the group
> [main] ERROR org.apache.kafka.clients.consumer.internals.ConsumerCoordinator 
> - [Consumer clientId=consumer-rivenTest6-1, groupId=rivenTest6] LeaveGroup 
> request with Generation{generationId=17, 
> memberId='consumer-rivenTest6-1-38849218-32fa-430d-b14c-d3ce7ff402c4', 
> protocol='roundrobin'} failed with error: The server experienced an 
> unexpected error when processing the request.
> [main] INFO org.apache.kafka.common.metrics.Metrics - Metrics scheduler closed
> [main] INFO org.apache.kafka.common.metrics.Metrics - Closing reporter 
> org.apache.kafka.common.metrics.JmxReporter
> [main] INFO org.apache.kafka.common.metrics.Metrics - Closing reporter 
> org.apache.kafka.common.telemetry.internals.ClientTelemetryReporter
> [main] INFO org.apache.kafka.common.metrics.Metrics - Metrics reporters closed
> [main] INFO org.apache.kafka.common.utils.AppInfoParser - App info 
> kafka.consumer for consumer-rivenTest6-1 unregistered {code}
> coordinator broker error logs:
> !image-2025-06-23-15-04-26-598.png!
>  
>  
> h3. Analysis:
> This is a brand new Kafka4.0 cluster with only one topic created;
> The JDK version is 17;
> Why does the broker encounter OOM so quickly when it is just sending and 
> consuming data? Is there a memory leak somewhere?
> 1 First, use the arthas tool to analyze the memory usage
> !image-2025-06-23-15-04-15-708.png!
> We can see that most of the heap memory is {*}occupied by the old 
> generation{*}, and it is likely that the program will directly experience OOM 
> of the heap memory when it needs to {color:#ff0000}*apply for a large object. 
> It should be noted that the maximum memory we allocate to the Kafka process 
> is actually 3G, and there is also a lot of space left in the heap memory. Why 
> does it directly trigger the Java heap space type OOM in this case?*{color}
> {color:#172b4d}2.Dump memory snapshots and use tools to analyze what is 
> currently occupying a large amount of memory in the program
> !image-2025-06-23-15-33-06-851.png!
> !image-2025-06-23-15-33-26-209.png!
> After analyzing the memory usage, I found that it was basically all the 
> *coordinators* objects in the *CoordinatorRuntime* class that occupied the 
> memory and did not release it; coordinators is a ConcurrentHashMap structure, 
> the key is the TopicPartition type, and the value is the CoordinatorContext 
> type.
> !image-2025-06-23-15-11-13-026.png!
> Why does a broker machine simply start a consumer, the topic has only three 
> partitions, and the consumer group uses no more than 10 partitions in total, 
> and the *coordinators* object in the broker process occupies such a large 
> amount of memory and does not release it?
> Is there a problem with the broker configuration or the JDK17 version or the 
> jvm startup parameters, or is there a memory leak in the kafka 4.0 version 
> code?{color}{color:#172b4d}Please help analyze and answer, looking forward to 
> your reply.
> Thank you very much!{color}



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

Reply via email to