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

Nicholas Telford commented on KAFKA-16089:
------------------------------------------

I built a basic memory leak test, which essentially runs:
{noformat}
while (System.currentTimeMillis() - started < 3600000) {
    final RocksDBStore store = new RocksDBStore("test", "test");
    try {
        store.init(context, store);
    } finally {
        store.close();
    }
}{noformat}
And then ran it with the JVM args {{-Xms200m -Xmx200m -XX:+AlwaysPreTouch}} to 
ensure the JVM heap doesn't grow over time.

Measuring RSS, shows the memory leak:

!unfix.png|width=374,height=280!

I tracked it down to {{{}ColumnFamilyHandle#getDescriptor(){}}}, called in the 
new {{mergeColumnFamilyHandleLists}} method. A {{ColumnFamilyDescriptor}} is 
*not* a {{{}RocksObject{}}}, which means it's not backed by any native memory. 
However, when calling {{{}ColumnFamilyHandle#getDescriptor(){}}}, an internal 
[rocksdb::db::ColumnFamilyDescriptor is 
allocated|https://github.com/facebook/rocksdb/blob/c5fbfd7ad807867f85fb992a61a228e4417b55ea/db/column_family.cc#L91C21-L91C21],
 copying the name of the column family and its options from the handle.

Since `ColumnFamilyDescriptor` is not a `RocksObject`, it's not possible to 
free this allocated {{{}rocksdb::db::ColumnFamilyDescriptor{}}}, which leaks.

Fortunately, since we're only interested in the name of the column family, we 
can simply avoid calling {{{}ColumnFamilyHandle#getDescriptor(){}}}, and 
instead just call {{{}ColumnFamilyHandle#getName(){}}}, which does not leak 
memory.

With this fixed, running the same test again demonstrates no memory leak:

!fix.png|width=367,height=275!

I have opened an issue with RocksDB to report this bug: 
[https://github.com/facebook/rocksdb/issues/12224,] but we are not dependent on 
it being resolved.

> Kafka Streams still leaking memory
> ----------------------------------
>
>                 Key: KAFKA-16089
>                 URL: https://issues.apache.org/jira/browse/KAFKA-16089
>             Project: Kafka
>          Issue Type: Bug
>          Components: streams
>    Affects Versions: 3.8.0
>            Reporter: Lucas Brutschy
>            Priority: Critical
>         Attachments: fix.png, graphviz (1).svg, unfix.png
>
>
> In 
> [https://github.com/apache/kafka/commit/58d6d2e5922df60cdc5c5c1bcd1c2d82dd2b71e2]
>  a leak was fixed in the release candidate for 3.7.
>  
> However, Kafka Streams still seems to be leaking memory (just slower) after 
> the fix.
>  
> Attached is the `jeprof` output right before a crash after ~11 hours.
>  
>  
>  



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

Reply via email to