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

RivenSun commented on KAFKA-19427:
----------------------------------

The above document has posted the client log and coordinator broker log when 
the problem occurs.
The coordinator broker log is posted here in text form. We can see that two OOM 
exceptions occurred:
{code:java}
[2025-06-23 02:50:03,315] INFO [GroupCoordinator id=3 topic=__consumer_offsets 
partition=36] Dynamic member with unknown member id joins group rivenTest6 in 
Empty state. Created a new member id 
consumer-rivenTest6-1-2863a8c0-c963-4a1e-96f0-c7036ff85f6d and requesting the 
member to rejoin with this id. 
(org.apache.kafka.coordinator.group.GroupMetadataManager)
[2025-06-23 02:50:03,738] WARN [GroupCoordinator id=3 topic=__consumer_offsets 
partition=36] Failed to write empty metadata for group rivenTest6: Java heap 
space (org.apache.kafka.coordinator.group.GroupMetadataManager)
[2025-06-23 02:50:03,750] INFO [GroupCoordinator id=3 topic=__consumer_offsets 
partition=36] Pending dynamic member with id 
consumer-rivenTest6-1-2863a8c0-c963-4a1e-96f0-c7036ff85f6d joins group 
rivenTest6 in Empty state. Adding to the group now. 
(org.apache.kafka.coordinator.group.GroupMetadataManager)
[2025-06-23 02:50:03,750] INFO [GroupCoordinator id=3 topic=__consumer_offsets 
partition=36] Preparing to rebalance group rivenTest6 in state 
PreparingRebalance with old generation 0 (reason: Adding new member 
consumer-rivenTest6-1-2863a8c0-c963-4a1e-96f0-c7036ff85f6d with group instance 
id null; client reason: need to re-join with the given member-id: 
consumer-rivenTest6-1-2863a8c0-c963-4a1e-96f0-c7036ff85f6d). 
(org.apache.kafka.coordinator.group.GroupMetadataManager)
[2025-06-23 02:50:06,751] INFO [GroupCoordinator id=3 topic=__consumer_offsets 
partition=36] Stabilized group rivenTest6 generation 1 with 1 members. 
(org.apache.kafka.coordinator.group.GroupMetadataManager)
[2025-06-23 02:50:07,056] INFO [GroupCoordinator id=3 topic=__consumer_offsets 
partition=36] Assignment received from leader 
consumer-rivenTest6-1-2863a8c0-c963-4a1e-96f0-c7036ff85f6d for group rivenTest6 
for generation 1. The group has 1 members, 0 of which are static. 
(org.apache.kafka.coordinator.group.GroupMetadataManager)
[2025-06-23 02:50:07,314] INFO [GroupCoordinator id=3 topic=__consumer_offsets 
partition=36] Preparing to rebalance group rivenTest6 in state 
PreparingRebalance with old generation 1 (reason: Error UNKNOWN_SERVER_ERROR 
when storing group assignmentduring SyncGroup (member: 
consumer-rivenTest6-1-2863a8c0-c963-4a1e-96f0-c7036ff85f6d).). 
(org.apache.kafka.coordinator.group.GroupMetadataManager)
[2025-06-23 02:50:07,707] INFO [GroupCoordinator id=3 topic=__consumer_offsets 
partition=36] [Group rivenTest6] Member 
consumer-rivenTest6-1-2863a8c0-c963-4a1e-96f0-c7036ff85f6d has left group 
through explicit `LeaveGroup` request; client reason: the consumer is being 
closed (org.apache.kafka.coordinator.group.GroupMetadataManager)
[2025-06-23 02:50:07,709] INFO [GroupCoordinator id=3 topic=__consumer_offsets 
partition=36] Group rivenTest6 with generation 2 is now empty. 
(org.apache.kafka.coordinator.group.GroupMetadataManager)
[2025-06-23 02:50:08,069] WARN [GroupCoordinator id=3 topic=__consumer_offsets 
partition=36] Failed to write empty metadata for group rivenTest6: The server 
experienced an unexpected error when processing the request. 
(org.apache.kafka.coordinator.group.GroupMetadataManager)
[2025-06-23 02:50:08,090] ERROR [GroupCoordinator id=3] Operation 
classic-group-leave with LeaveGroupRequestData(groupId='rivenTest6', 
memberId='', 
members=[MemberIdentity(memberId='consumer-rivenTest6-1-2863a8c0-c963-4a1e-96f0-c7036ff85f6d',
 groupInstanceId=null, reason='the consumer is being closed')]) hit an 
unexpected exception: Java heap space. 
(org.apache.kafka.coordinator.group.GroupCoordinatorService)
java.lang.OutOfMemoryError: Java heap space {code}
Today I used rivenTest6 to start the consumer again and found that this group 
can start consumption normally. The following is the coordinatorBroker log 
today:
{code:java}
[2025-06-24 02:24:04,810] INFO [GroupCoordinator id=3 topic=__consumer_offsets 
partition=36] Dynamic member with unknown member id joins group rivenTest6 in 
Stable state. Created a new member id 
consumer-rivenTest6-1-4fbe3e3c-fa31-45ce-a917-3ea7ea2f8026 and requesting the 
member to rejoin with this id. 
(org.apache.kafka.coordinator.group.GroupMetadataManager)
[2025-06-24 02:24:05,096] INFO [GroupCoordinator id=3 topic=__consumer_offsets 
partition=36] Pending dynamic member with id 
consumer-rivenTest6-1-4fbe3e3c-fa31-45ce-a917-3ea7ea2f8026 joins group 
rivenTest6 in Stable state. Adding to the group now. 
(org.apache.kafka.coordinator.group.GroupMetadataManager)
[2025-06-24 02:24:05,096] INFO [GroupCoordinator id=3 topic=__consumer_offsets 
partition=36] Preparing to rebalance group rivenTest6 in state 
PreparingRebalance with old generation 1 (reason: Adding new member 
consumer-rivenTest6-1-4fbe3e3c-fa31-45ce-a917-3ea7ea2f8026 with group instance 
id null; client reason: need to re-join with the given member-id: 
consumer-rivenTest6-1-4fbe3e3c-fa31-45ce-a917-3ea7ea2f8026). 
(org.apache.kafka.coordinator.group.GroupMetadataManager)
[2025-06-24 02:24:27,863] INFO [GroupCoordinator id=3 topic=__consumer_offsets 
partition=36] Member consumer-rivenTest6-1-227a766a-90d2-4365-af17-117566bbbd58 
in group rivenTest6 has failed, removing it from the group. 
(org.apache.kafka.coordinator.group.GroupMetadataManager)
[2025-06-24 02:24:27,863] INFO [GroupCoordinator id=3 topic=__consumer_offsets 
partition=36] Stabilized group rivenTest6 generation 2 with 1 members. 
(org.apache.kafka.coordinator.group.GroupMetadataManager)
[2025-06-24 02:24:28,160] INFO [GroupCoordinator id=3 topic=__consumer_offsets 
partition=36] Assignment received from leader 
consumer-rivenTest6-1-4fbe3e3c-fa31-45ce-a917-3ea7ea2f8026 for group rivenTest6 
for generation 2. The group has 1 members, 0 of which are static. 
(org.apache.kafka.coordinator.group.GroupMetadataManager) {code}
Then I immediately looked at the broker's memory usage at this time, as follows:

!image-2025-06-24-10-29-51-465.png!

The kafka process was started on June 20th and has not been restarted so far.
Yesterday, the memory usage of g1_old_gen was 76.14%. When rivenTest6 groupId 
consumption was started, the coordinator broker would experience OOM.
Today, the memory usage of g1_old_gen was 42.62%. When rivenTest6 groupId 
consumption was started, the coordinator broker was normal and the client could 
consume data normally.
*So it was because the process had enough memory to support the use after the 
old generation heap memory was reclaimed by GC. But what I don't understand is 
why the coordinator broker needs to apply for such a large heap memory for a 
topic registration consumption with three partitions.*

 

*The partition status of __consumer_offsets is as follows:*
*!image-2025-06-24-10-43-46-826.png!*

[~squah-confluent] Looking forward to your reply. Thank you!

> 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
>
>
> 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