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

Wei Zhong commented on FLINK-17771:
-----------------------------------

[~dian.fu] I have investigated the issue for several days. The empty result is 
indeed caused by the OOM. But the root cause is the memory leaking of the 
ChildFirstClassLoader of the PyFlink Task. Currently I have found 5 places 
which caused the memory leaking via the heapdump file:
 # The context class loader of the thread pools named "flink-file-cache-xxx".
 # The soft reference from the field "reflectors" and "localDescs" of the class 
"ObjectStreamClass$Caches". The OOM is cause by the DirectMemory which would 
not trigger the GC of the soft reference, so we need to consider the soft 
references here.
 # The context class loader of the process reaper thread.
 # The "classLoader" field of the "org.codehaus.janino.ClassLoaderIClassLoader" 
objects.
 # The soft reference of the object 
"org.apache.flink.table.runtime.generated.COMPILED_CACHE".

This is the screenshot:

!image-2020-05-21-20-11-07-626.png|width=754,height=151!

!image-2020-05-21-20-11-29-389.png|width=754,height=74!

!image-2020-05-21-20-11-48-220.png|width=744,height=157!

!image-2020-05-21-20-12-16-889.png|width=743,height=274!

 

Note that the issue also exists on jDK8. When running in JDK8, the shaded Netty 
in Beam uses the method "Unsafe.allocateMemory()" to allocate the Direct Memory 
instead of the method "ByteBuffer.allocateDirect()", which won't be limited by 
the param "MaxDirectMemorySize". So the OOM won't happen in JDK8 but the 
process memory is still increasing.

> "PyFlink end-to-end test" fails with "The output result: [] is not as 
> expected: [2, 3, 4]!" on Java11
> -----------------------------------------------------------------------------------------------------
>
>                 Key: FLINK-17771
>                 URL: https://issues.apache.org/jira/browse/FLINK-17771
>             Project: Flink
>          Issue Type: Bug
>          Components: API / Python, Tests
>    Affects Versions: 1.11.0
>            Reporter: Robert Metzger
>            Priority: Major
>              Labels: test-stability
>             Fix For: 1.11.0
>
>         Attachments: image-2020-05-21-20-11-07-626.png, 
> image-2020-05-21-20-11-29-389.png, image-2020-05-21-20-11-48-220.png, 
> image-2020-05-21-20-12-16-889.png
>
>
> Java 11 nightly profile: 
> https://dev.azure.com/apache-flink/apache-flink/_build/results?buildId=1579&view=logs&j=6caf31d6-847a-526e-9624-468e053467d6&t=679407b1-ea2c-5965-2c8d-1467777fff88
> {code}
> Job has been submitted with JobID ef78030becb3bfd6415d3de2e06420b4
> java.lang.AssertionError: The output result: [] is not as expected: [2, 3, 4]!
>       at 
> org.apache.flink.python.tests.FlinkStreamPythonUdfSqlJob.main(FlinkStreamPythonUdfSqlJob.java:55)
>       at 
> java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
>       at 
> java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
>       at 
> java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
>       at java.base/java.lang.reflect.Method.invoke(Method.java:566)
>       at 
> org.apache.flink.client.program.PackagedProgram.callMainMethod(PackagedProgram.java:288)
>       at 
> org.apache.flink.client.program.PackagedProgram.invokeInteractiveModeForExecution(PackagedProgram.java:198)
>       at 
> org.apache.flink.client.ClientUtils.executeProgram(ClientUtils.java:148)
>       at 
> org.apache.flink.client.cli.CliFrontend.executeProgram(CliFrontend.java:689)
>       at org.apache.flink.client.cli.CliFrontend.run(CliFrontend.java:227)
>       at 
> org.apache.flink.client.cli.CliFrontend.parseParameters(CliFrontend.java:906)
>       at 
> org.apache.flink.client.cli.CliFrontend.lambda$main$10(CliFrontend.java:982)
>       at 
> org.apache.flink.runtime.security.contexts.NoOpSecurityContext.runSecured(NoOpSecurityContext.java:30)
>       at org.apache.flink.client.cli.CliFrontend.main(CliFrontend.java:982)
> Stopping taskexecutor daemon (pid: 2705) on host fv-az670.
> {code}



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to