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

Patrik Kleindl commented on KAFKA-7660:
---------------------------------------

[~vvcephei] Well, at least that answers my next question why I couldn't find 
any references :P

And no, my code does not call any of these methods.

I'm attaching a heapdump from the last test.

The effect can mostly be seen on the KafkaMetric and MetricName, they start 
around 1900 right after the start.

Code is:
{code:java}
public class MemoryTest {
    public static void main(String[] args) {


        final String bootstrapServers = "broker0:9092";

        Properties streamsConfiguration = new Properties();
        streamsConfiguration.put(StreamsConfig.APPLICATION_ID_CONFIG, "test");
        streamsConfiguration.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, 
bootstrapServers);

        StreamsBuilder builder = new StreamsBuilder();

        KTable<String, String> table = builder.table("test");

        final KafkaStreams streams = new KafkaStreams(builder.build(), 
streamsConfiguration);

        streams.cleanUp();
        streams.start();
        Runtime.getRuntime().addShutdownHook(new Thread(() -> {
            try {
                streams.close();
            } catch (Exception e) {
                // ignored
            }
        }));
    }
}
{code}
and if I want to run it for some time with delays for the rebalances:
{code:java}
while (true) {
    final KafkaStreams streams = new KafkaStreams(builder.build(), 
streamsConfiguration);

    try {
        Thread.sleep(20000L);
        streams.start();
        Thread.sleep(5000L);
    } catch (InterruptedException e) {
        e.printStackTrace();
    } finally {
        streams.close();

    }

}
{code}
Topic:
{code:java}
./kafka-topics --zookeeper broker0:2181 --create --topic test --partitions 10 
--replication-factor 3
{code}

> Stream Metrics - Memory Analysis
> --------------------------------
>
>                 Key: KAFKA-7660
>                 URL: https://issues.apache.org/jira/browse/KAFKA-7660
>             Project: Kafka
>          Issue Type: Bug
>          Components: metrics, streams
>    Affects Versions: 2.0.0
>            Reporter: Patrik Kleindl
>            Priority: Minor
>         Attachments: Mem_Collections.jpeg, Mem_DuplicateStrings.jpeg, 
> Mem_DuplicateStrings2.jpeg, Mem_Hotspots.jpeg, Mem_KeepAliveSet.jpeg, 
> Mem_References.jpeg, heapdump-1543441898901.hprof
>
>
> During the analysis of JVM memory two possible issues were shown which I 
> would like to bring to your attention:
> 1) Duplicate strings
> Top findings: 
> string_content="stream-processor-node-metrics" count="534,277"
> string_content="processor-node-id" count="148,437"
> string_content="stream-rocksdb-state-metrics" count="41,832"
> string_content="punctuate-latency-avg" count="29,681" 
>  
> "stream-processor-node-metrics"  seems to be used in Sensors.java as a 
> literal and not interned.
>  
> 2) The HashMap parentSensors from 
> org.apache.kafka.streams.processor.internals.StreamThread$StreamsMetricsThreadImpl
>  was reported multiple times as suspicious for potentially keeping alive a 
> lot of objects. In our case the reported size was 40-50MB each.
> I haven't looked too deep in the code but noticed that the class Sensor.java 
> which is used as a key in the HashMap does not implement equals or hashCode 
> method. Not sure this is a problem though.
>  
> The analysis was done with Dynatrace 7.0
> We are running Confluent 5.0/Kafka2.0-cp1 (Brokers as well as Clients)
>  
> Screenshots are attached.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to