Re: Classloader leak with Kafka Mbeans / JMX Reporter

2018-02-13 Thread Chesnay Schepler
I looked through the kafka JmxReporter class (and other kafka classes 
that we use) and couldn't spot anything
that would prevent the Mbeans from being unregistered. The obvious 
implication would be that we don't

properly clean up kafka classes, but that code looks good as well.

So at the moment I'm a bit stumped as to why this is happening.

For reproducing the problem, could you answer the following questions?

1. How do you kill the job?
2. Do you have any Flink reporter configured?
3. Does this only happen on Kubernetes?
4. Does this also happen with parent-first classloading?


On 09.02.2018 14:19, Edward wrote:

I applied the change in the pull request associated with that Kafka bug, and
unfortunately it didn't resolve anything. It doesn't unregister any
additional MBeans which are created by Kafka's JmxRepository -- it is just a
fix to the remove mbeans from Kafka's cache of mbeans (i.e. it is doing
cleanup within the job's  ChildFirst classloader, not the bootstrap/App
classloader where the strong reference exists).



--
Sent from: http://apache-flink-user-mailing-list-archive.2336050.n4.nabble.com/





Re: Classloader leak with Kafka Mbeans / JMX Reporter

2018-02-09 Thread Edward
I applied the change in the pull request associated with that Kafka bug, and
unfortunately it didn't resolve anything. It doesn't unregister any
additional MBeans which are created by Kafka's JmxRepository -- it is just a
fix to the remove mbeans from Kafka's cache of mbeans (i.e. it is doing
cleanup within the job's  ChildFirst classloader, not the bootstrap/App
classloader where the strong reference exists).



--
Sent from: http://apache-flink-user-mailing-list-archive.2336050.n4.nabble.com/


Re: Classloader leak with Kafka Mbeans / JMX Reporter

2018-02-07 Thread Chesnay Schepler
This could be a bug in Kafkas JmxReporter class: 
https://issues.apache.org/jira/browse/KAFKA-6307


On 07.02.2018 13:37, Edward wrote:

We are using FlinkKafkaConsumer011 and FlinkKafkaProducer011, but we also
experienced the same behavior with FlinkKafkaConsumer010 and
FlinkKafkaProducer010.



--
Sent from: http://apache-flink-user-mailing-list-archive.2336050.n4.nabble.com/





Re: Classloader leak with Kafka Mbeans / JMX Reporter

2018-02-07 Thread Edward
We are using FlinkKafkaConsumer011 and FlinkKafkaProducer011, but we also
experienced the same behavior with FlinkKafkaConsumer010 and
FlinkKafkaProducer010.



--
Sent from: http://apache-flink-user-mailing-list-archive.2336050.n4.nabble.com/


Re: Classloader leak with Kafka Mbeans / JMX Reporter

2018-02-06 Thread Chesnay Schepler
It would also be useful to know which source/sink you are using and 
which kafka version that is.


On 07.02.2018 08:58, Chesnay Schepler wrote:

Thanks for reporting this.

To narrow things down a bit, is your job using both kafka sources and 
sinks?


On 06.02.2018 21:30, Edward wrote:
I'm having an issue where off-heap memory is growing unchecked until 
I get

OOM exceptions.

I was hoping that upgrading to 1.4 would solve these, since the 
child-first

classloader is supposed to resolve issues with Avro classes cached in a
different classloader (which prevents the classloaders from being 
garbage

collected).

However, after upgrading, we are still seeing an off-heap memory leak. I
think I may have isolated the issue to the JmxReporter class used for
collecting Kafka metrics.

Here are the details of what I'm seeing:
Our cluster is running in kubernetes, using the latest flink:1.4 docker
image. We are using the default classloading order (child first).

If I resubmit my job repeatedly, the ClassLoaders from the previous job
submissions don't get cleaned up, and the non-heap memory slowly 
grows until

the task manager runs out of memory.

I can see all of the un-deleted classloaders if I run "sudo -u flink 
jmap
-clstats " (the output is below). This list of dead 
classloaders
continues to grow every time I kill and resubmit a new Flink job.  In 
all,
it lists 3200 dead class loaders. I'm only going to upload the ones 
which

show more than 2K of used memory.

finding class loader instances ..done.
computing per loader stat ..done.
please wait.. computing liveness.liveness analysis may be inaccurate ...
class_loaderclassesbytesparent_loaderalive? type
0x807302a07522122130760x804c58c0 dead
sun/misc/Launcher$AppClassLoader@0x0001f070
0x8eb0369960215350x807302a0 dead
org/apache/flink/runtime/execution/librarycache/FlinkUserCodeClassLoaders$ChildFirstClassLoader@0x0001005cdc98 


0x94200190369360168070x807302a0 dead
org/apache/flink/runtime/execution/librarycache/FlinkUserCodeClassLoaders$ChildFirstClassLoader@0x0001005cdc98 


0x9e7bc6c8369660010810x807302a0 dead
org/apache/flink/runtime/execution/librarycache/FlinkUserCodeClassLoaders$ChildFirstClassLoader@0x0001005cdc98 


0xa9d80008358455304120x807302a0 dead
org/apache/flink/runtime/execution/librarycache/FlinkUserCodeClassLoaders$ChildFirstClassLoader@0x0001005cdc98 


0xf4103650358155273540x807302a0 dead
org/apache/flink/runtime/execution/librarycache/FlinkUserCodeClassLoaders$ChildFirstClassLoader@0x0001005cdc98 


0x901801f8358155273540x807302a0 dead
org/apache/flink/runtime/execution/librarycache/FlinkUserCodeClassLoaders$ChildFirstClassLoader@0x0001005cdc98 


0x942637c0323151211760x807302a0 dead
org/apache/flink/runtime/execution/librarycache/FlinkUserCodeClassLoaders$ChildFirstClassLoader@0x0001005cdc98 


0x96c2ec00323151196620x807302a0 dead
org/apache/flink/runtime/execution/librarycache/FlinkUserCodeClassLoaders$ChildFirstClassLoader@0x0001005cdc98 


0x8f60322551162410x807302a0 dead
org/apache/flink/runtime/execution/librarycache/FlinkUserCodeClassLoaders$ChildFirstClassLoader@0x0001005cdc98 


0x92700d48322851122700x807302a0 dead
org/apache/flink/runtime/execution/librarycache/FlinkUserCodeClassLoaders$ChildFirstClassLoader@0x0001005cdc98 


25484424440nulllive 
0x96b77190223436346020x807302a0 dead
org/apache/flink/runtime/execution/librarycache/FlinkUserCodeClassLoaders$ChildFirstClassLoader@0x0001005cdc98 



Next I took a heap dump:
sudo -u flink jmap -dump:format=b,file=/tmp/HeapDump.hprof 

Then, using Eclipse Memory Analyzer, I followed the steps from this blog
post:
http://java.jiderhamn.se/2011/12/11/classloader-leaks-i-how-to-find-classloader-leaks-with-eclipse-memory-analyser-mat/ 



The result of looking for strong references to classes in a dead 
classloader

is this tree:

Class Name
| Shallow Heap | Retained Heap
--- 

org.apache.flink.runtime.execution.librarycache.FlinkUserCodeClassLoaders$ChildFirstClassLoader 


@ 0x94200190|   88 |   616,992
'-  class
org.apache.kafka.common.metrics.JmxReporter$KafkaMbean @ 0x94250cb0
|0 | 0
'-  org.apache.kafka.common.metrics.JmxReporter$KafkaMbean @
0xbae537e8   |   24 | 328
   '- object com.sun.jmx.mbeanserver.NamedObject @ 0xbace01e0
|   24 |24
  '- value java.util.HashMap$Node @ 0xbace0110
|   32 |   232
  

Re: Classloader leak with Kafka Mbeans / JMX Reporter

2018-02-06 Thread Chesnay Schepler

Thanks for reporting this.

To narrow things down a bit, is your job using both kafka sources and sinks?

On 06.02.2018 21:30, Edward wrote:

I'm having an issue where off-heap memory is growing unchecked until I get
OOM exceptions.

I was hoping that upgrading to 1.4 would solve these, since the child-first
classloader is supposed to resolve issues with Avro classes cached in a
different classloader (which prevents the classloaders from being garbage
collected).

However, after upgrading, we are still seeing an off-heap memory leak. I
think I may have isolated the issue to the JmxReporter class used for
collecting Kafka metrics.

Here are the details of what I'm seeing:
Our cluster is running in kubernetes, using the latest flink:1.4 docker
image. We are using the default classloading order (child first).

If I resubmit my job repeatedly, the ClassLoaders from the previous job
submissions don't get cleaned up, and the non-heap memory slowly grows until
the task manager runs out of memory.

I can see all of the un-deleted classloaders if I run "sudo -u flink jmap
-clstats " (the output is below). This list of dead classloaders
continues to grow every time I kill and resubmit a new Flink job.  In all,
it lists 3200 dead class loaders. I'm only going to upload the ones which
show more than 2K of used memory.

finding class loader instances ..done.
computing per loader stat ..done.
please wait.. computing liveness.liveness analysis may be inaccurate ...
class_loaderclasses bytes   parent_loader   alive?  type
0x807302a0  7522122130760x804c58c0  dead
sun/misc/Launcher$AppClassLoader@0x0001f070
0x8eb0  36996021535 0x807302a0  dead
org/apache/flink/runtime/execution/librarycache/FlinkUserCodeClassLoaders$ChildFirstClassLoader@0x0001005cdc98
0x94200190  36936016807 0x807302a0  dead
org/apache/flink/runtime/execution/librarycache/FlinkUserCodeClassLoaders$ChildFirstClassLoader@0x0001005cdc98
0x9e7bc6c8  36966001081 0x807302a0  dead
org/apache/flink/runtime/execution/librarycache/FlinkUserCodeClassLoaders$ChildFirstClassLoader@0x0001005cdc98
0xa9d80008  35845530412 0x807302a0  dead
org/apache/flink/runtime/execution/librarycache/FlinkUserCodeClassLoaders$ChildFirstClassLoader@0x0001005cdc98
0xf4103650  35815527354 0x807302a0  dead
org/apache/flink/runtime/execution/librarycache/FlinkUserCodeClassLoaders$ChildFirstClassLoader@0x0001005cdc98
0x901801f8  35815527354 0x807302a0  dead
org/apache/flink/runtime/execution/librarycache/FlinkUserCodeClassLoaders$ChildFirstClassLoader@0x0001005cdc98
0x942637c0  32315121176 0x807302a0  dead
org/apache/flink/runtime/execution/librarycache/FlinkUserCodeClassLoaders$ChildFirstClassLoader@0x0001005cdc98
0x96c2ec00  32315119662 0x807302a0  dead
org/apache/flink/runtime/execution/librarycache/FlinkUserCodeClassLoaders$ChildFirstClassLoader@0x0001005cdc98
0x8f60  32255116241 0x807302a0  dead
org/apache/flink/runtime/execution/librarycache/FlinkUserCodeClassLoaders$ChildFirstClassLoader@0x0001005cdc98
0x92700d48  32285112270 0x807302a0  dead
org/apache/flink/runtime/execution/librarycache/FlinkUserCodeClassLoaders$ChildFirstClassLoader@0x0001005cdc98
   25484424440 nulllive
0x96b77190  22343634602 0x807302a0  dead
org/apache/flink/runtime/execution/librarycache/FlinkUserCodeClassLoaders$ChildFirstClassLoader@0x0001005cdc98

Next I took a heap dump:
sudo -u flink jmap -dump:format=b,file=/tmp/HeapDump.hprof 

Then, using Eclipse Memory Analyzer, I followed the steps from this blog
post:
http://java.jiderhamn.se/2011/12/11/classloader-leaks-i-how-to-find-classloader-leaks-with-eclipse-memory-analyser-mat/

The result of looking for strong references to classes in a dead classloader
is this tree:

Class Name
| Shallow Heap | Retained Heap
---
org.apache.flink.runtime.execution.librarycache.FlinkUserCodeClassLoaders$ChildFirstClassLoader
@ 0x94200190|   88 |   616,992
'-  class
org.apache.kafka.common.metrics.JmxReporter$KafkaMbean @ 0x94250cb0
|0 | 0
'-  org.apache.kafka.common.metrics.JmxReporter$KafkaMbean @
0xbae537e8   |   24 |   328
   '- object com.sun.jmx.mbeanserver.NamedObject @ 0xbace01e0
|   24 |24
  '- value java.util.HashMap$Node @ 0xbace0110
|   32 |   232
 '- [247] java.util.HashMap$Node[512] @ 0xfa0d08c0
|2,064 |   120,104
'- table java.util.HashMap @ 0x806