Details
-
Bug
-
Status: Closed
-
Major
-
Resolution: Fixed
-
1.10.0
Description
Start a standalone cluster and after submit PyFlink UDF jobs multiple times, the TM will fail with the following exception:
Caused by: java.lang.OutOfMemoryError: Metaspace at java.lang.ClassLoader.defineClass1(Native Method) at java.lang.ClassLoader.defineClass(ClassLoader.java:788) at java.security.SecureClassLoader.defineClass(SecureClassLoader.java:142) at java.net.URLClassLoader.defineClass(URLClassLoader.java:467) at java.net.URLClassLoader.access$100(URLClassLoader.java:73) at java.net.URLClassLoader$1.run(URLClassLoader.java:368) at java.net.URLClassLoader$1.run(URLClassLoader.java:362) at java.security.AccessController.doPrivileged(Native Method) at java.net.URLClassLoader.findClass(URLClassLoader.java:361) at java.lang.ClassLoader.loadClass(ClassLoader.java:448) at org.apache.flink.util.ChildFirstClassLoader.loadClass(ChildFirstClassLoader.java:60) at java.lang.ClassLoader.loadClass(ClassLoader.java:380) at org.apache.flink.api.python.shaded.com.fasterxml.jackson.databind.ObjectMapper.<init>(ObjectMapper.java:628) at org.apache.flink.api.python.shaded.com.fasterxml.jackson.databind.ObjectMapper.<init>(ObjectMapper.java:531) at org.apache.beam.sdk.options.PipelineOptionsFactory.<clinit>(PipelineOptionsFactory.java:469) at org.apache.flink.python.AbstractPythonFunctionRunner.open(AbstractPythonFunctionRunner.java:173) at org.apache.flink.table.runtime.operators.python.AbstractPythonScalarFunctionOperator$ProjectUdfInputPythonScalarFunctionRunner.open(AbstractPythonScalarFunctionOperator.java:193) at org.apache.flink.streaming.api.operators.python.AbstractPythonFunctionOperator.open(AbstractPythonFunctionOperator.java:139) at org.apache.flink.table.runtime.operators.python.AbstractPythonScalarFunctionOperator.open(AbstractPythonScalarFunctionOperator.java:143) at org.apache.flink.table.runtime.operators.python.BaseRowPythonScalarFunctionOperator.open(BaseRowPythonScalarFunctionOperator.java:86) at org.apache.flink.streaming.runtime.tasks.StreamTask.initializeStateAndOpen(StreamTask.java:1018) at org.apache.flink.streaming.runtime.tasks.StreamTask.lambda$beforeInvoke$0(StreamTask.java:454) at org.apache.flink.streaming.runtime.tasks.StreamTask$$Lambda$125/800044563.run(Unknown Source) at org.apache.flink.streaming.runtime.tasks.StreamTaskActionExecutor$SynchronizedStreamTaskActionExecutor.runThrowing(StreamTaskActionExecutor.java:94) at org.apache.flink.streaming.runtime.tasks.StreamTask.beforeInvoke(StreamTask.java:449) at org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:461) at org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:702) at org.apache.flink.runtime.taskmanager.Task.run(Task.java:527) at java.lang.Thread.run(Thread.java:834)
Attachments
Issue Links
- relates to
-
BEAM-9006 Meta space memory leak caused by the shutdown hook of ProcessManager
- Resolved
- links to