[jira] [Commented] (FLINK-17771) "PyFlink end-to-end test" fails with "The output result: [] is not as expected: [2, 3, 4]!" on Java11
[ https://issues.apache.org/jira/browse/FLINK-17771?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17113707#comment-17113707 ] Dian Fu commented on FLINK-17771: - Merged via : master: 8b14cd807d165052da46df2fc0d9536eadc97fe7 release-1.11: c7243c001ba632f412add975a26fe3ae1caff7b2 > "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 >Assignee: Wei Zhong >Priority: Major > Labels: pull-request-available, 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-146fff88 > {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)
[jira] [Commented] (FLINK-17771) "PyFlink end-to-end test" fails with "The output result: [] is not as expected: [2, 3, 4]!" on Java11
[ https://issues.apache.org/jira/browse/FLINK-17771?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17113143#comment-17113143 ] Wei Zhong commented on FLINK-17771: --- To solve the memory leak issue of the ChildFirstClassLoader completely needs further effort and discussion. Before that we can fix the test case temporarily via adjusting the limit of the off-heap memory. What do you think? [~dian.fu] > "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-146fff88 > {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)
[jira] [Commented] (FLINK-17771) "PyFlink end-to-end test" fails with "The output result: [] is not as expected: [2, 3, 4]!" on Java11
[ 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-146fff88 > {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)
[jira] [Commented] (FLINK-17771) "PyFlink end-to-end test" fails with "The output result: [] is not as expected: [2, 3, 4]!" on Java11
[ https://issues.apache.org/jira/browse/FLINK-17771?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17109491#comment-17109491 ] Dian Fu commented on FLINK-17771: - When I tried to reproduce this problem locally with JDK 11, it will throw exceptions as following: {code} Exception in thread "grpc-default-executor-0" java.lang.OutOfMemoryError: Direct buffer memory at java.base/java.nio.Bits.reserveMemory(Bits.java:175) at java.base/java.nio.DirectByteBuffer.(DirectByteBuffer.java:118) at java.base/java.nio.ByteBuffer.allocateDirect(ByteBuffer.java:317) at org.apache.beam.vendor.grpc.v1p21p0.io.netty.buffer.PoolArena$DirectArena.allocateDirect(PoolArena.java:777) at org.apache.beam.vendor.grpc.v1p21p0.io.netty.buffer.PoolArena$DirectArena.newChunk(PoolArena.java:753) at org.apache.beam.vendor.grpc.v1p21p0.io.netty.buffer.PoolArena.allocateNormal(PoolArena.java:250) at org.apache.beam.vendor.grpc.v1p21p0.io.netty.buffer.PoolArena.allocate(PoolArena.java:220) at org.apache.beam.vendor.grpc.v1p21p0.io.netty.buffer.PoolArena.allocate(PoolArena.java:152) at org.apache.beam.vendor.grpc.v1p21p0.io.netty.buffer.PooledByteBufAllocator.newDirectBuffer(PooledByteBufAllocator.java:332) at org.apache.beam.vendor.grpc.v1p21p0.io.netty.buffer.AbstractByteBufAllocator.directBuffer(AbstractByteBufAllocator.java:187) at org.apache.beam.vendor.grpc.v1p21p0.io.netty.buffer.AbstractByteBufAllocator.buffer(AbstractByteBufAllocator.java:123) at org.apache.beam.vendor.grpc.v1p21p0.io.grpc.netty.NettyWritableBufferAllocator.allocate(NettyWritableBufferAllocator.java:51) at org.apache.beam.vendor.grpc.v1p21p0.io.grpc.internal.MessageFramer.writeKnownLengthUncompressed(MessageFramer.java:226) at org.apache.beam.vendor.grpc.v1p21p0.io.grpc.internal.MessageFramer.writeUncompressed(MessageFramer.java:167) at org.apache.beam.vendor.grpc.v1p21p0.io.grpc.internal.MessageFramer.writePayload(MessageFramer.java:140) at org.apache.beam.vendor.grpc.v1p21p0.io.grpc.internal.AbstractStream.writeMessage(AbstractStream.java:53) at org.apache.beam.vendor.grpc.v1p21p0.io.grpc.internal.ServerCallImpl.sendMessageInternal(ServerCallImpl.java:163) at org.apache.beam.vendor.grpc.v1p21p0.io.grpc.internal.ServerCallImpl.sendMessage(ServerCallImpl.java:145) at org.apache.beam.vendor.grpc.v1p21p0.io.grpc.stub.ServerCalls$ServerCallStreamObserverImpl.onNext(ServerCalls.java:349) at org.apache.beam.runners.fnexecution.provisioning.StaticGrpcProvisionService.getProvisionInfo(StaticGrpcProvisionService.java:48) at org.apache.beam.model.fnexecution.v1.ProvisionServiceGrpc$MethodHandlers.invoke(ProvisionServiceGrpc.java:246) at org.apache.beam.vendor.grpc.v1p21p0.io.grpc.stub.ServerCalls$UnaryServerCallHandler$UnaryServerCallListener.onHalfClose(ServerCalls.java:171) at org.apache.beam.vendor.grpc.v1p21p0.io.grpc.PartialForwardingServerCallListener.onHalfClose(PartialForwardingServerCallListener.java:35) at org.apache.beam.vendor.grpc.v1p21p0.io.grpc.ForwardingServerCallListener.onHalfClose(ForwardingServerCallListener.java:23) at org.apache.beam.vendor.grpc.v1p21p0.io.grpc.ForwardingServerCallListener$SimpleForwardingServerCallListener.onHalfClose(ForwardingServerCallListener.java:40) at org.apache.beam.vendor.grpc.v1p21p0.io.grpc.Contexts$ContextualizedServerCallListener.onHalfClose(Contexts.java:86) at org.apache.beam.vendor.grpc.v1p21p0.io.grpc.internal.ServerCallImpl$ServerStreamListenerImpl.halfClosed(ServerCallImpl.java:322) at org.apache.beam.vendor.grpc.v1p21p0.io.grpc.internal.ServerImpl$JumpToApplicationThreadServerStreamListener$1HalfClosed.runInContext(ServerImpl.java:762) at org.apache.beam.vendor.grpc.v1p21p0.io.grpc.internal.ContextRunnable.run(ContextRunnable.java:37) at org.apache.beam.vendor.grpc.v1p21p0.io.grpc.internal.SerializingExecutor.run(SerializingExecutor.java:123) at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128) at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628) at java.base/java.lang.Thread.run(Thread.java:834) {code} I guess this is the reason why the result is empty as it encounters OutOfMemoryError during running. > "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
[jira] [Commented] (FLINK-17771) "PyFlink end-to-end test" fails with "The output result: [] is not as expected: [2, 3, 4]!" on Java11
[ https://issues.apache.org/jira/browse/FLINK-17771?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17109490#comment-17109490 ] Dian Fu commented on FLINK-17771: - cc [~zhongwei] > "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 > > > 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-146fff88 > {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)