[jira] [Created] (FLINK-11402) User code can fail with an UnsatisfiedLinkError in the presence of multiple classloaders

classic Classic list List threaded Threaded
1 message Options
Reply | Threaded
Open this post in threaded view
|

[jira] [Created] (FLINK-11402) User code can fail with an UnsatisfiedLinkError in the presence of multiple classloaders

Shang Yuanchun (Jira)
Ufuk Celebi created FLINK-11402:
-----------------------------------

             Summary: User code can fail with an UnsatisfiedLinkError in the presence of multiple classloaders
                 Key: FLINK-11402
                 URL: https://issues.apache.org/jira/browse/FLINK-11402
             Project: Flink
          Issue Type: Bug
          Components: Distributed Coordination
    Affects Versions: 1.7.0
            Reporter: Ufuk Celebi
         Attachments: hello-snappy-1.0-SNAPSHOT.jar, hello-snappy.tgz

As reported on the user mailing list thread "[`env.java.opts` not persisting after job canceled or failed and then restarted|https://lists.apache.org/thread.html/37cc1b628e16ca6c0bacced5e825de8057f88a8d601b90a355b6a291@%3Cuser.flink.apache.org%3E]", there can be issues with using native libraries and user code class loading.

h2. Steps to reproduce

I was able to reproduce the issue reported on the mailing list using [snappy-java|https://github.com/xerial/snappy-java] in a user program. Running the attached user program works fine on initial submission, but results in a failure when re-executed.

I'm using Flink 1.7.0 using a standalone cluster started via {{bin/start-cluster.sh}}.

0. Unpack attached Maven project and build using {{mvn clean package}} *or* directly use attached {{hello-snappy-1.0-SNAPSHOT.jar}}
1. Download [snappy-java-1.1.7.2.jar|http://central.maven.org/maven2/org/xerial/snappy/snappy-java/1.1.7.2/snappy-java-1.1.7.2.jar] and unpack libsnappyjava for your system:
{code}
jar tf snappy-java-1.1.7.2.jar | grep libsnappy
...
org/xerial/snappy/native/Linux/x86_64/libsnappyjava.so
...
org/xerial/snappy/native/Mac/x86_64/libsnappyjava.jnilib
...
{code}
2. Configure system library path to {{libsnappyjava}} in {{flink-conf.yaml}} (path needs to be adjusted for your system):
{code}
env.java.opts: -Djava.library.path=/.../org/xerial/snappy/native/Mac/x86_64
{code}
3. Run attached {{hello-snappy-1.0-SNAPSHOT.jar}}
{code}
bin/flink run hello-snappy-1.0-SNAPSHOT.jar
Starting execution of program
Program execution finished
Job with JobID ae815b918dd7bc64ac8959e4e224f2b4 has finished.
Job Runtime: 359 ms
{code}
4. Rerun attached {{hello-snappy-1.0-SNAPSHOT.jar}}
{code}
bin/flink run hello-snappy-1.0-SNAPSHOT.jar
Starting execution of program

------------------------------------------------------------
 The program finished with the following exception:

org.apache.flink.client.program.ProgramInvocationException: Job failed. (JobID: 7d69baca58f33180cb9251449ddcd396)
  at org.apache.flink.client.program.rest.RestClusterClient.submitJob(RestClusterClient.java:268)
  at org.apache.flink.client.program.ClusterClient.run(ClusterClient.java:487)
  at org.apache.flink.streaming.api.environment.StreamContextEnvironment.execute(StreamContextEnvironment.java:66)
  at com.github.uce.HelloSnappy.main(HelloSnappy.java:18)
  at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
  at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
  at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
  at java.lang.reflect.Method.invoke(Method.java:498)
  at org.apache.flink.client.program.PackagedProgram.callMainMethod(PackagedProgram.java:529)
  at org.apache.flink.client.program.PackagedProgram.invokeInteractiveModeForExecution(PackagedProgram.java:421)
  at org.apache.flink.client.program.ClusterClient.run(ClusterClient.java:427)
  at org.apache.flink.client.cli.CliFrontend.executeProgram(CliFrontend.java:813)
  at org.apache.flink.client.cli.CliFrontend.runProgram(CliFrontend.java:287)
  at org.apache.flink.client.cli.CliFrontend.run(CliFrontend.java:213)
  at org.apache.flink.client.cli.CliFrontend.parseParameters(CliFrontend.java:1050)
  at org.apache.flink.client.cli.CliFrontend.lambda$main$11(CliFrontend.java:1126)
  at org.apache.flink.runtime.security.NoOpSecurityContext.runSecured(NoOpSecurityContext.java:30)
  at org.apache.flink.client.cli.CliFrontend.main(CliFrontend.java:1126)
Caused by: org.apache.flink.runtime.client.JobExecutionException: Job execution failed.
  at org.apache.flink.runtime.jobmaster.JobResult.toJobExecutionResult(JobResult.java:146)
  at org.apache.flink.client.program.rest.RestClusterClient.submitJob(RestClusterClient.java:265)
  ... 17 more
Caused by: java.lang.UnsatisfiedLinkError: Native Library /.../org/xerial/snappy/native/Mac/x86_64/libsnappyjava.jnilib already loaded in another classloader
  at java.lang.ClassLoader.loadLibrary0(ClassLoader.java:1907)
  at java.lang.ClassLoader.loadLibrary(ClassLoader.java:1861)
  at java.lang.Runtime.loadLibrary0(Runtime.java:870)
  at java.lang.System.loadLibrary(System.java:1122)
  at org.xerial.snappy.SnappyLoader.loadNativeLibrary(SnappyLoader.java:182)
  at org.xerial.snappy.SnappyLoader.loadSnappyApi(SnappyLoader.java:154)
  at org.xerial.snappy.Snappy.<clinit>(Snappy.java:47)
  at com.github.uce.HelloSnappy.lambda$main$95f17bfa$1(HelloSnappy.java:13)
  at org.apache.flink.streaming.api.operators.StreamMap.processElement(StreamMap.java:41)
  at org.apache.flink.streaming.runtime.tasks.OperatorChain$CopyingChainingOutput.pushToOperator(OperatorChain.java:579)
  at org.apache.flink.streaming.runtime.tasks.OperatorChain$CopyingChainingOutput.collect(OperatorChain.java:554)
  at org.apache.flink.streaming.runtime.tasks.OperatorChain$CopyingChainingOutput.collect(OperatorChain.java:534)
  at org.apache.flink.streaming.api.operators.AbstractStreamOperator$CountingOutput.collect(AbstractStreamOperator.java:718)
  at org.apache.flink.streaming.api.operators.AbstractStreamOperator$CountingOutput.collect(AbstractStreamOperator.java:696)
  at org.apache.flink.streaming.api.operators.StreamSourceContexts$NonTimestampContext.collect(StreamSourceContexts.java:104)
  at org.apache.flink.streaming.api.functions.source.FromElementsFunction.run(FromElementsFunction.java:164)
  at org.apache.flink.streaming.api.operators.StreamSource.run(StreamSource.java:94)
  at org.apache.flink.streaming.api.operators.StreamSource.run(StreamSource.java:58)
  at org.apache.flink.streaming.runtime.tasks.SourceStreamTask.run(SourceStreamTask.java:99)
  at org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:300)
  at org.apache.flink.runtime.taskmanager.Task.run(Task.java:704)
  at java.lang.Thread.run(Thread.java:748)
{code}

*Note*: The attached user code configures Snappy to use {{libsnappyjava}} in the path specified by {{java.library.path}} (see {{org-xerial-snappy.properties}}). When bundling the native code in the user JAR, repeated execution works fine.




--
This message was sent by Atlassian JIRA
(v7.6.3#76005)