Thank you Jan for your valuable feedback. Could it be that I should not use import shaded-jackson classes in my user code? For example import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper?
Bets, Flavio On Mon, Nov 16, 2020 at 3:15 PM Jan Lukavský <je...@seznam.cz> wrote: > Hi Flavio, > > when I encountered quite similar problem that you describe, it was related > to a static storage located in class that was loaded "parent-first". In my > case it was it was in java.lang.ClassValue, but it might (and probably will > be) different in your case. The problem is that if user-code registers > something in some (static) storage located in class loaded with parent > (TaskTracker) classloader, then its associated classes will never be GC'd > and Metaspace will grow. A good starting point would be not to focus on > biggest consumers of heap (in general), but to look at where the 15k > objects of type Class are referenced from. That might help you figure this > out. I'm not sure if there is something that can be done in general to > prevent this type of leaks. That would be probably question on dev@ > mailing list. > > Best, > > Jan > On 11/16/20 2:27 PM, Flavio Pompermaier wrote: > > Hello everybody, > I was writing this email when a similar thread on this mailing list > appeared.. > The difference is that the other problem seems to be related with Flink > 1.10 on YARN and does not output anything helpful in debugging the cause of > the problem. > > Indeed, in my use case I use Flink 1.11.0 and Flink on a standalone > session cluster (the job is submitted to the cluster using the CLI client). > The problem arises when I submit the same job for about 20 times (this > number unfortunately is not deterministic and can change a little bit). The > error reported by the Task Executor is related to the ever growing > Metaspace..the error seems to be pretty detailed [1]. > > I found the same issue in some previous threads on this mailing list and > I've tried to figure it out the cause of the problem. The issue is that > looking at the objects allocated I don't really get an idea of the source > of the problem because the type of objects that are consuming the memory > are of general purpose (i.e. Bytes, Integers and Strings)...these are my > "top" memory consumers if looking at the output of jmap -histo <PID>: > > At run 0: > > num #instances #bytes class name (module) > ------------------------------------------------------- > 1: 46238 13224056 [B (java.base@11.0.9.1) > 2: 3736 6536672 [I (java.base@11.0.9.1) > 3: 38081 913944 java.lang.String (java.base@11.0.9.1) > 4: 26 852384 [Lakka.dispatch.forkjoin.ForkJoinTask; > 5: 7146 844984 java.lang.Class (java.base@11.0.9.1) > > At run 1: > > 1: 77.608 25.317.496 [B (java.base@11.0.9.1) > 2: 7.004 9.088.360 [I (java.base@11.0.9.1) > 3: 15.814 1.887.256 java.lang.Class (java.base@11.0.9.1 > ) > 4: 67.381 1.617.144 java.lang.String ( > java.base@11.0.9.1) > 5: 3.906 1.422.960 [Ljava.util.HashMap$Node; ( > java.base@11.0.9.1) > > At run 6: > > 1: 81.408 25.375.400 [B (java.base@11.0.9.1) > 2: 12.479 7.249.392 [I (java.base@11.0.9.1) > 3: 29.090 3.496.168 java.lang.Class (java.base@11.0.9.1 > ) > 4: 4.347 2.813.416 [Ljava.util.HashMap$Node; ( > java.base@11.0.9.1) > 5: 71.584 1.718.016 java.lang.String ( > java.base@11.0.9.1) > > At run 8: > > 1: 985.979 127.193.256 [B (java.base@11.0.9.1) > 2: 35.400 13.702.112 [I (java.base@11.0.9.1) > 3: 260.387 6.249.288 java.lang.String ( > java.base@11.0.9.1) > 4: 148.836 5.953.440 java.util.HashMap$KeyIterator ( > java.base@11.0.9.1) > 5: 17.641 5.222.344 [Ljava.util.HashMap$Node; ( > java.base@11.0.9.1) > > Thanks in advance for any help, > Flavio > > [1] > -------------------------------------------------------------------------------------------------- > java.lang.OutOfMemoryError: Metaspace. The metaspace out-of-memory error > has occurred. This can mean two things: either the job requires a larger > size of JVM metaspace to load classes or there is a class loading leak. In > the first case 'taskmanager.memory.jvm-metaspace.size' configuration option > should be increased. If the error persists (usually in cluster after > several job (re-)submissions) then there is probably a class loading leak > in user code or some of its dependencies which has to be investigated and > fixed. The task executor has to be shutdown... > at java.lang.ClassLoader.defineClass1(Native Method) ~[?:?] > at java.lang.ClassLoader.defineClass(ClassLoader.java:1017) ~[?:?] > at > java.security.SecureClassLoader.defineClass(SecureClassLoader.java:174) > ~[?:?] > at java.net.URLClassLoader.defineClass(URLClassLoader.java:550) > ~[?:?] > at java.net.URLClassLoader$1.run(URLClassLoader.java:458) ~[?:?] > at java.net.URLClassLoader$1.run(URLClassLoader.java:452) ~[?:?] > at java.security.AccessController.doPrivileged(Native Method) > ~[?:?] > at java.net.URLClassLoader.findClass(URLClassLoader.java:451) > ~[?:?] > at > org.apache.flink.util.ChildFirstClassLoader.loadClassWithoutExceptionHandling(ChildFirstClassLoader.java:71) > ~[flink-dist_2.12-1.11.0.jar:1.11.0] > at > org.apache.flink.util.FlinkUserCodeClassLoader.loadClass(FlinkUserCodeClassLoader.java:48) > [flink-dist_2.12-1.11.0.jar:1.11.0] > at java.lang.ClassLoader.loadClass(ClassLoader.java:522) [?:?] > >