[jira] [Commented] (FLINK-11205) Task Manager Metaspace Memory Leak
[ https://issues.apache.org/jira/browse/FLINK-11205?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17110329#comment-17110329 ] Till Rohrmann commented on FLINK-11205: --- I think the particular problem of an increased meta space usage due to rapid failures has been solved with FLINK-16408 which avoids re-loading classes by reusing the user code class loader across restarts. > Task Manager Metaspace Memory Leak > --- > > Key: FLINK-11205 > URL: https://issues.apache.org/jira/browse/FLINK-11205 > Project: Flink > Issue Type: Bug > Components: Runtime / Coordination >Affects Versions: 1.5.5, 1.6.2, 1.7.0 >Reporter: Nawaid Shamim >Priority: Critical > Attachments: Screenshot 2018-12-18 at 12.14.11.png, Screenshot > 2018-12-18 at 15.47.55.png > > > Job Restarts causes task manager to dynamically load duplicate classes. > Metaspace is unbounded and grows with every restart. YARN aggressively kill > such containers but this affect is immediately seems on different task > manager which results in death spiral. > Task Manager uses dynamic loader as described in > [https://ci.apache.org/projects/flink/flink-docs-stable/monitoring/debugging_classloading.html] > {quote} > *YARN* > YARN classloading differs between single job deployments and sessions: > * When submitting a Flink job/application directly to YARN (via {{bin/flink > run -m yarn-cluster ...}}), dedicated TaskManagers and JobManagers are > started for that job. Those JVMs have both Flink framework classes and user > code classes in the Java classpath. That means that there is _no dynamic > classloading_ involved in that case. > * When starting a YARN session, the JobManagers and TaskManagers are started > with the Flink framework classes in the classpath. The classes from all jobs > that are submitted against the session are loaded dynamically. > {quote} > The above is not entirely true specially when you set {{-yD > classloader.resolve-order=parent-first}} . We also above observed the above > behaviour when submitting a Flink job/application directly to YARN (via > {{bin/flink run -m yarn-cluster ...}}). > !Screenshot 2018-12-18 at 12.14.11.png! -- This message was sent by Atlassian Jira (v8.3.4#803005)
[jira] [Commented] (FLINK-11205) Task Manager Metaspace Memory Leak
[ https://issues.apache.org/jira/browse/FLINK-11205?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17048845#comment-17048845 ] Guowei Ma commented on FLINK-11205: --- [~fwiffo] I have a question about the LogFactory caching class loader leads to the class leak. As far as I know, Flink does not use the Apache Commons Logging. So I assume that the Apache Commons Log jar is from the application. For failover restart only a job # If the Apache Commons Log and user jar are loaded by the system class loader I think there might be not class leak because all class is loaded by the system class.(The user class loader object is a leak.) # If the Apache Commons Log and user jar are loaded by the user class loader I think there might be also no class leak. The GC would release all the class. # If the Apache Commons Log is loaded by the system class loader and the user jar is load by the user class loader. I think there might be class leaks if we do not call LogFactory.release when closing. Do you mean the third scenario? Why do you not choose the other two scenarios? Correct me If I miss understanding something. > Task Manager Metaspace Memory Leak > --- > > Key: FLINK-11205 > URL: https://issues.apache.org/jira/browse/FLINK-11205 > Project: Flink > Issue Type: Bug > Components: Runtime / Coordination >Affects Versions: 1.5.5, 1.6.2, 1.7.0 >Reporter: NS >Priority: Critical > Attachments: Screenshot 2018-12-18 at 12.14.11.png, Screenshot > 2018-12-18 at 15.47.55.png > > > Job Restarts causes task manager to dynamically load duplicate classes. > Metaspace is unbounded and grows with every restart. YARN aggressively kill > such containers but this affect is immediately seems on different task > manager which results in death spiral. > Task Manager uses dynamic loader as described in > [https://ci.apache.org/projects/flink/flink-docs-stable/monitoring/debugging_classloading.html] > {quote} > *YARN* > YARN classloading differs between single job deployments and sessions: > * When submitting a Flink job/application directly to YARN (via {{bin/flink > run -m yarn-cluster ...}}), dedicated TaskManagers and JobManagers are > started for that job. Those JVMs have both Flink framework classes and user > code classes in the Java classpath. That means that there is _no dynamic > classloading_ involved in that case. > * When starting a YARN session, the JobManagers and TaskManagers are started > with the Flink framework classes in the classpath. The classes from all jobs > that are submitted against the session are loaded dynamically. > {quote} > The above is not entirely true specially when you set {{-yD > classloader.resolve-order=parent-first}} . We also above observed the above > behaviour when submitting a Flink job/application directly to YARN (via > {{bin/flink run -m yarn-cluster ...}}). > !Screenshot 2018-12-18 at 12.14.11.png! -- This message was sent by Atlassian Jira (v8.3.4#803005)
[jira] [Commented] (FLINK-11205) Task Manager Metaspace Memory Leak
[ https://issues.apache.org/jira/browse/FLINK-11205?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17042986#comment-17042986 ] Stephan Ewen commented on FLINK-11205: -- Coming back to this issue (sorry for the delay). There are similar discussion in issue FLINK-16142, but the root cause seems to be a different one. [~nawaidshamim] and [~fwiffo] is this still an issue for you, or have you found a way to solve this? My current thoughts on hoe to solve this are: * FLIP-49 configures metaspace by default, because Metaspace cleanup does not seem to happen unless the metaspace size limit is reached * As a final safety-net, the TMs kill/restart themselves when the metaspace blows up FLINK-16225 * [~fwiffo] I think the object stream class cache uses weak references, so should not contribute to the class leak. But maybe that has not always been so in all Java versions. * [~fwiffo] A generic mechanism to prevent leaks through ClassLoader caching (as in Apache Commons Logging) would be FLINK-16245 (use a delegating class loader where we drop the reference to the real one when closing it). * There are other cases where libraries produce class leaks, we just identified the AWS SDK as a culprit with its metric admin beans: FLINK-16142 Also worth noting that this should by only relevant to "sessions" (clusters that accept dynamic jobs submission) and not to "single applications clusters" which do not use dynamic class loading at all. > Task Manager Metaspace Memory Leak > --- > > Key: FLINK-11205 > URL: https://issues.apache.org/jira/browse/FLINK-11205 > Project: Flink > Issue Type: Bug > Components: Runtime / Coordination >Affects Versions: 1.5.5, 1.6.2, 1.7.0 >Reporter: Nawaid Shamim >Priority: Critical > Attachments: Screenshot 2018-12-18 at 12.14.11.png, Screenshot > 2018-12-18 at 15.47.55.png > > > Job Restarts causes task manager to dynamically load duplicate classes. > Metaspace is unbounded and grows with every restart. YARN aggressively kill > such containers but this affect is immediately seems on different task > manager which results in death spiral. > Task Manager uses dynamic loader as described in > [https://ci.apache.org/projects/flink/flink-docs-stable/monitoring/debugging_classloading.html] > {quote} > *YARN* > YARN classloading differs between single job deployments and sessions: > * When submitting a Flink job/application directly to YARN (via {{bin/flink > run -m yarn-cluster ...}}), dedicated TaskManagers and JobManagers are > started for that job. Those JVMs have both Flink framework classes and user > code classes in the Java classpath. That means that there is _no dynamic > classloading_ involved in that case. > * When starting a YARN session, the JobManagers and TaskManagers are started > with the Flink framework classes in the classpath. The classes from all jobs > that are submitted against the session are loaded dynamically. > {quote} > The above is not entirely true specially when you set {{-yD > classloader.resolve-order=parent-first}} . We also above observed the above > behaviour when submitting a Flink job/application directly to YARN (via > {{bin/flink run -m yarn-cluster ...}}). > !Screenshot 2018-12-18 at 12.14.11.png! -- This message was sent by Atlassian Jira (v8.3.4#803005)
[jira] [Commented] (FLINK-11205) Task Manager Metaspace Memory Leak
[ https://issues.apache.org/jira/browse/FLINK-11205?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16895306#comment-16895306 ] William Leese commented on FLINK-11205: --- We're seeing some significant metaspace growth with flink 1.8.0 and job restarts. We can easily hit 1Gb metaspace usage within 15 minutes with rapid restarts. That said, we attempted to troubleshoot this issue by looking at all the classes in metaspace using `jcmd GC.class_stats`. Here we observed that after every job restart another entry is created for every class in our job. Where the old classes have InstBytes=0. So far so good, but moving to the Total column for these entries show that memory is still being used. Also, adding up all entries in the Total column indeed corresponds to our metaspace usage. So far we could only conclude that our job classes - none of them - were being unloaded. Then we stumbled upon this ticket. Now here are our results running the SocketWindowWordCount jar in a flink 1.8.0 cluster with one taskmanager. We achieve a class count by doing a jcmd 3052 GC.class_stats | grep -i org.apache.flink.streaming.examples.windowing.SessionWindowing | wc -l *First* run: Class Count: 1 Metaspace: 30695K After *800*~ runs: Class Count: 802 Metaspace: 39406K Interesting when we looked a bit later the class count slowly went down, slowly, step by step. If I had to guess it took about 15 minutes to go from 800~ to 170~, with metaspace dropping to 35358K. In a sense we've seen this behavior, but with much much larger increases in metaspace usage over far fewer job restarts. > Task Manager Metaspace Memory Leak > --- > > Key: FLINK-11205 > URL: https://issues.apache.org/jira/browse/FLINK-11205 > Project: Flink > Issue Type: Bug > Components: Runtime / Coordination >Affects Versions: 1.5.5, 1.6.2, 1.7.0 >Reporter: Nawaid Shamim >Priority: Major > Attachments: Screenshot 2018-12-18 at 12.14.11.png, Screenshot > 2018-12-18 at 15.47.55.png > > > Job Restarts causes task manager to dynamically load duplicate classes. > Metaspace is unbounded and grows with every restart. YARN aggressively kill > such containers but this affect is immediately seems on different task > manager which results in death spiral. > Task Manager uses dynamic loader as described in > [https://ci.apache.org/projects/flink/flink-docs-stable/monitoring/debugging_classloading.html] > {quote} > *YARN* > YARN classloading differs between single job deployments and sessions: > * When submitting a Flink job/application directly to YARN (via {{bin/flink > run -m yarn-cluster ...}}), dedicated TaskManagers and JobManagers are > started for that job. Those JVMs have both Flink framework classes and user > code classes in the Java classpath. That means that there is _no dynamic > classloading_ involved in that case. > * When starting a YARN session, the JobManagers and TaskManagers are started > with the Flink framework classes in the classpath. The classes from all jobs > that are submitted against the session are loaded dynamically. > {quote} > The above is not entirely true specially when you set {{-yD > classloader.resolve-order=parent-first}} . We also above observed the above > behaviour when submitting a Flink job/application directly to YARN (via > {{bin/flink run -m yarn-cluster ...}}). > !Screenshot 2018-12-18 at 12.14.11.png! -- This message was sent by Atlassian JIRA (v7.6.14#76016)
[jira] [Commented] (FLINK-11205) Task Manager Metaspace Memory Leak
[ https://issues.apache.org/jira/browse/FLINK-11205?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16888411#comment-16888411 ] Joey Echeverria commented on FLINK-11205: - I didn't get a chance to reproduce this using a sample job, but we found two causes for our jobs. (1) Apache Commons Logging was caching LogFactory instances. We added the following code to our close() methods to release those LogFactories: {code:java} ClassLoader contextLoader = Thread.currentThread().getContextClassLoader(); LogFactory.release(contextLoader); {code} (2) We saw a leak that seemed to have an upper limit in ObjectStreamClass's caches. This one was trickier as we had to use reflection to clear out the caches, again in the close() method: {code:java} public static void cleanUpLeakingObjects(ClassLoader contextLoader) { try { Class caches = Class.forName("java.io.ObjectStreamClass$Caches"); clearCache(caches, "localDescs", contextLoader); clearCache(caches, "reflectors", contextLoader); } catch (ReflectiveOperationException | SecurityException | ClassCastException ex) { // Clean-up failed logger.warn("Cleanup of ObjectStreamClass caches failed with exception {}: {}", ex.getClass().getSimpleName(), ex.getMessage()); logger.debug("Stack trace follows.", ex); } } private static void clearCache(Class caches, String mapName, ClassLoader contextLoader) throws ReflectiveOperationException, SecurityException, ClassCastException { Field field = caches.getDeclaredField(mapName); field.setAccessible(true); Map map = TypeUtils.coerce(field.get(null)); Iterator keys = map.keySet().iterator(); while (keys.hasNext()) { Object key = keys.next(); if (key instanceof Reference) { Object clazz = ((Reference) key).get(); if (clazz instanceof Class) { ClassLoader cl = ((Class) clazz).getClassLoader(); while (cl != null) { if (cl == contextLoader) { keys.remove(); break; } cl = cl.getParent(); } } } } } {code} > Task Manager Metaspace Memory Leak > --- > > Key: FLINK-11205 > URL: https://issues.apache.org/jira/browse/FLINK-11205 > Project: Flink > Issue Type: Bug > Components: Runtime / Coordination >Affects Versions: 1.5.5, 1.6.2, 1.7.0 >Reporter: Nawaid Shamim >Priority: Major > Attachments: Screenshot 2018-12-18 at 12.14.11.png, Screenshot > 2018-12-18 at 15.47.55.png > > > Job Restarts causes task manager to dynamically load duplicate classes. > Metaspace is unbounded and grows with every restart. YARN aggressively kill > such containers but this affect is immediately seems on different task > manager which results in death spiral. > Task Manager uses dynamic loader as described in > [https://ci.apache.org/projects/flink/flink-docs-stable/monitoring/debugging_classloading.html] > {quote} > *YARN* > YARN classloading differs between single job deployments and sessions: > * When submitting a Flink job/application directly to YARN (via {{bin/flink > run -m yarn-cluster ...}}), dedicated TaskManagers and JobManagers are > started for that job. Those JVMs have both Flink framework classes and user > code classes in the Java classpath. That means that there is _no dynamic > classloading_ involved in that case. > * When starting a YARN session, the JobManagers and TaskManagers are started > with the Flink framework classes in the classpath. The classes from all jobs > that are submitted against the session are loaded dynamically. > {quote} > The above is not entirely true specially when you set {{-yD > classloader.resolve-order=parent-first}} . We also above observed the above > behaviour when submitting a Flink job/application directly to YARN (via > {{bin/flink run -m yarn-cluster ...}}). > !Screenshot 2018-12-18 at 12.14.11.png! -- This message was sent by Atlassian JIRA (v7.6.14#76016)
[jira] [Commented] (FLINK-11205) Task Manager Metaspace Memory Leak
[ https://issues.apache.org/jira/browse/FLINK-11205?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16887338#comment-16887338 ] Joey Echeverria commented on FLINK-11205: - I'm seeing this same issue when running Flink jobs that are in a restart loop. The restart policy is set with a very large number of max attempts and the Flink job throws an exception on each record. I'll see if I can reproduce with one of the example jobs. > Task Manager Metaspace Memory Leak > --- > > Key: FLINK-11205 > URL: https://issues.apache.org/jira/browse/FLINK-11205 > Project: Flink > Issue Type: Bug > Components: Runtime / Coordination >Affects Versions: 1.5.5, 1.6.2, 1.7.0 >Reporter: Nawaid Shamim >Priority: Major > Attachments: Screenshot 2018-12-18 at 12.14.11.png, Screenshot > 2018-12-18 at 15.47.55.png > > > Job Restarts causes task manager to dynamically load duplicate classes. > Metaspace is unbounded and grows with every restart. YARN aggressively kill > such containers but this affect is immediately seems on different task > manager which results in death spiral. > Task Manager uses dynamic loader as described in > [https://ci.apache.org/projects/flink/flink-docs-stable/monitoring/debugging_classloading.html] > {quote} > *YARN* > YARN classloading differs between single job deployments and sessions: > * When submitting a Flink job/application directly to YARN (via {{bin/flink > run -m yarn-cluster ...}}), dedicated TaskManagers and JobManagers are > started for that job. Those JVMs have both Flink framework classes and user > code classes in the Java classpath. That means that there is _no dynamic > classloading_ involved in that case. > * When starting a YARN session, the JobManagers and TaskManagers are started > with the Flink framework classes in the classpath. The classes from all jobs > that are submitted against the session are loaded dynamically. > {quote} > The above is not entirely true specially when you set {{-yD > classloader.resolve-order=parent-first}} . We also above observed the above > behaviour when submitting a Flink job/application directly to YARN (via > {{bin/flink run -m yarn-cluster ...}}). > !Screenshot 2018-12-18 at 12.14.11.png! -- This message was sent by Atlassian JIRA (v7.6.14#76016)
[jira] [Commented] (FLINK-11205) Task Manager Metaspace Memory Leak
[ https://issues.apache.org/jira/browse/FLINK-11205?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16743807#comment-16743807 ] Till Rohrmann commented on FLINK-11205: --- Thanks for the update [~nawaidshamim]. Could you reproduce this problem with the simple {{WindowJoin}} example job or did you use a special job? > Task Manager Metaspace Memory Leak > --- > > Key: FLINK-11205 > URL: https://issues.apache.org/jira/browse/FLINK-11205 > Project: Flink > Issue Type: Bug >Affects Versions: 1.5.5, 1.6.2, 1.7.0 >Reporter: Nawaid Shamim >Priority: Major > Attachments: Screenshot 2018-12-18 at 12.14.11.png, Screenshot > 2018-12-18 at 15.47.55.png > > > Job Restarts causes task manager to dynamically load duplicate classes. > Metaspace is unbounded and grows with every restart. YARN aggressively kill > such containers but this affect is immediately seems on different task > manager which results in death spiral. > Task Manager uses dynamic loader as described in > [https://ci.apache.org/projects/flink/flink-docs-stable/monitoring/debugging_classloading.html] > {quote} > *YARN* > YARN classloading differs between single job deployments and sessions: > * When submitting a Flink job/application directly to YARN (via {{bin/flink > run -m yarn-cluster ...}}), dedicated TaskManagers and JobManagers are > started for that job. Those JVMs have both Flink framework classes and user > code classes in the Java classpath. That means that there is _no dynamic > classloading_ involved in that case. > * When starting a YARN session, the JobManagers and TaskManagers are started > with the Flink framework classes in the classpath. The classes from all jobs > that are submitted against the session are loaded dynamically. > {quote} > The above is not entirely true specially when you set {{-yD > classloader.resolve-order=parent-first}} . We also above observed the above > behaviour when submitting a Flink job/application directly to YARN (via > {{bin/flink run -m yarn-cluster ...}}). > !Screenshot 2018-12-18 at 12.14.11.png! -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Commented] (FLINK-11205) Task Manager Metaspace Memory Leak
[ https://issues.apache.org/jira/browse/FLINK-11205?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16725872#comment-16725872 ] Nawaid Shamim commented on FLINK-11205: --- [~till.rohrmann] I just replied on FLINK-10317. {quote} I guess the root cause is memory leak due to dynamic loading. Limiting Metaspace to a number or throwing more memory at it would simply delay OOM. Limiting metaspace still causes OutOfMemoryError: Metaspace exception but in this case task manager dies instead of YARN killing it. I was able to reproduce the above issue in relatively smaller setup - One Master and One Core. * Start 1 Job Manager (JM). * Start 2 Task Managers - TM1 and TM2. * Submit job with global parallelism value of two so that both job is scheduled on both TMs. * Wait for job to take first checkpoint. * For every 4 minutes: ** Take heap dump of JB, TM1, TM2. ** Restart TM2 process. On every restart, TM2's JVM / YARN container is restarted. JB issues restart and restore RPC. TM2 is new process while TM1 is old process and will reload duplicate classes (that's where metaspace is exploding). I think it has something to do with org.apache.flink.runtime.execution.librarycache.FlinkUserCodeClassLoaders$ParentFirstClassLoader#2 {quote} > Task Manager Metaspace Memory Leak > --- > > Key: FLINK-11205 > URL: https://issues.apache.org/jira/browse/FLINK-11205 > Project: Flink > Issue Type: Bug >Affects Versions: 1.5.5, 1.6.2, 1.7.0 >Reporter: Nawaid Shamim >Priority: Major > Attachments: Screenshot 2018-12-18 at 12.14.11.png, Screenshot > 2018-12-18 at 15.47.55.png > > > Job Restarts causes task manager to dynamically load duplicate classes. > Metaspace is unbounded and grows with every restart. YARN aggressively kill > such containers but this affect is immediately seems on different task > manager which results in death spiral. > Task Manager uses dynamic loader as described in > [https://ci.apache.org/projects/flink/flink-docs-stable/monitoring/debugging_classloading.html] > {quote} > *YARN* > YARN classloading differs between single job deployments and sessions: > * When submitting a Flink job/application directly to YARN (via {{bin/flink > run -m yarn-cluster ...}}), dedicated TaskManagers and JobManagers are > started for that job. Those JVMs have both Flink framework classes and user > code classes in the Java classpath. That means that there is _no dynamic > classloading_ involved in that case. > * When starting a YARN session, the JobManagers and TaskManagers are started > with the Flink framework classes in the classpath. The classes from all jobs > that are submitted against the session are loaded dynamically. > {quote} > The above is not entirely true specially when you set {{-yD > classloader.resolve-order=parent-first}} . We also above observed the above > behaviour when submitting a Flink job/application directly to YARN (via > {{bin/flink run -m yarn-cluster ...}}). > !Screenshot 2018-12-18 at 12.14.11.png! -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Commented] (FLINK-11205) Task Manager Metaspace Memory Leak
[ https://issues.apache.org/jira/browse/FLINK-11205?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16725696#comment-16725696 ] Till Rohrmann commented on FLINK-11205: --- Thanks for reporting this issue [~nawaidshamim]. I think the problem could be solved by limiting the meta space as proposed in FLINK-10317. Would that be enough to close this issue? > Task Manager Metaspace Memory Leak > --- > > Key: FLINK-11205 > URL: https://issues.apache.org/jira/browse/FLINK-11205 > Project: Flink > Issue Type: Bug >Affects Versions: 1.5.5, 1.6.2, 1.7.0 >Reporter: Nawaid Shamim >Priority: Major > Attachments: Screenshot 2018-12-18 at 12.14.11.png, Screenshot > 2018-12-18 at 15.47.55.png > > > Job Restarts causes task manager to dynamically load duplicate classes. > Metaspace is unbounded and grows with every restart. YARN aggressively kill > such containers but this affect is immediately seems on different task > manager which results in death spiral. > Task Manager uses dynamic loader as described in > [https://ci.apache.org/projects/flink/flink-docs-stable/monitoring/debugging_classloading.html] > {quote} > *YARN* > YARN classloading differs between single job deployments and sessions: > * When submitting a Flink job/application directly to YARN (via {{bin/flink > run -m yarn-cluster ...}}), dedicated TaskManagers and JobManagers are > started for that job. Those JVMs have both Flink framework classes and user > code classes in the Java classpath. That means that there is _no dynamic > classloading_ involved in that case. > * When starting a YARN session, the JobManagers and TaskManagers are started > with the Flink framework classes in the classpath. The classes from all jobs > that are submitted against the session are loaded dynamically. > {quote} > The above is not entirely true specially when you set {{-yD > classloader.resolve-order=parent-first}} . We also above observed the above > behaviour when submitting a Flink job/application directly to YARN (via > {{bin/flink run -m yarn-cluster ...}}). > !Screenshot 2018-12-18 at 12.14.11.png! -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Commented] (FLINK-11205) Task Manager Metaspace Memory Leak
[ https://issues.apache.org/jira/browse/FLINK-11205?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16725120#comment-16725120 ] Nawaid Shamim commented on FLINK-11205: --- *Workaround:* Copy Flink job JAR into /usr/lib/flink/lib and AppClassLoader will load all the required classes from classpath. !Screenshot 2018-12-18 at 15.47.55.png! > Task Manager Metaspace Memory Leak > --- > > Key: FLINK-11205 > URL: https://issues.apache.org/jira/browse/FLINK-11205 > Project: Flink > Issue Type: Bug >Affects Versions: 1.5.5, 1.6.2, 1.7.0 >Reporter: Nawaid Shamim >Priority: Major > Attachments: Screenshot 2018-12-18 at 12.14.11.png, Screenshot > 2018-12-18 at 15.47.55.png > > > Job Restarts causes task manager to dynamically load duplicate classes. > Metaspace is unbounded and grows with every restart. YARN aggressively kill > such containers but this affect is immediately seems on different task > manager which results in death spiral. > Task Manager uses dynamic loader as described in > [https://ci.apache.org/projects/flink/flink-docs-stable/monitoring/debugging_classloading.html] > {quote} > *YARN* > YARN classloading differs between single job deployments and sessions: > * When submitting a Flink job/application directly to YARN (via {{bin/flink > run -m yarn-cluster ...}}), dedicated TaskManagers and JobManagers are > started for that job. Those JVMs have both Flink framework classes and user > code classes in the Java classpath. That means that there is _no dynamic > classloading_ involved in that case. > * When starting a YARN session, the JobManagers and TaskManagers are started > with the Flink framework classes in the classpath. The classes from all jobs > that are submitted against the session are loaded dynamically. > {quote} > The above is not entirely true specially when you set {{-yD > classloader.resolve-order=parent-first}} . We also above observed the above > behaviour when submitting a Flink job/application directly to YARN (via > {{bin/flink run -m yarn-cluster ...}}). > !Screenshot 2018-12-18 at 12.14.11.png! -- This message was sent by Atlassian JIRA (v7.6.3#76005)