[ https://issues.apache.org/jira/browse/FLINK-18211?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17142724#comment-17142724 ]
Echo Lee commented on FLINK-18211: ---------------------------------- [~fly_in_gis] I just verified it in standalone mode, It seems that it will not be added to the classpath of the task, my verification method is as follows: I have standalone cluster of two nodes A and B, start jobmanager on node A, and start one taskmanager on node B, I execute the submit command on node A. By the way, pipeline.jars is flink-test-1.0-SNAPSHOT.jar, and user jar is flink-sql-connector-kafka_2.11-1.10.0-ad.jar and flink-json-1.10.0-ad.jar: ./bin/flink run -e remote -Drest.address=xxx -Drest.port=xxx -c xxx.CachedFileDemo -C file:///plugins/connector/flink-sql-connector-kafka_2.11-1.10.0-ad.jar -C file:///plugins/format/flink-json-1.10.0-ad.jar -Dpipeline.cached-files='name:file1,path:/plugins/connector/flink-sql-connector-kafka_2.11-1.10.0-ad.jar;name:file2,path:/plugins/format/flink-json-1.10.0-ad.jar' -Dyarn.application.queue=default flink-test-1.0-SNAPSHOT.jar But it will throw a ClassNotFoundException,the stackTrace is: Caused by: org.apache.flink.streaming.runtime.tasks.StreamTaskException: Cannot load user class: org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumerCaused by: org.apache.flink.streaming.runtime.tasks.StreamTaskException: Cannot load user class: org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumerClassLoader info: URL ClassLoader: file: '/tmp/blobStore-1d7f58ac-a2b4-4f3e-af70-5756de80651a/job_6ff0adec00cf56bed0e0a09a0888e66c/blob_p-1ad93d5770083ed166b02307da9cf50a0d334ea2-4feaba42fb9751da2f70af73f1f2de23' (valid JAR) file: '/plugins/connector/flink-sql-connector-kafka_2.11-1.10.0-ad.jar' (missing) file: '/plugins/format/flink-json-1.10.0-ad.jar' (missing)Class not resolvable through given classloader. at org.apache.flink.streaming.api.graph.StreamConfig.getStreamOperatorFactory(StreamConfig.java:266) at org.apache.flink.streaming.runtime.tasks.OperatorChain.<init>(OperatorChain.java:115) at org.apache.flink.streaming.runtime.tasks.StreamTask.beforeInvoke(StreamTask.java:433) at org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:461) at org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:707) at org.apache.flink.runtime.taskmanager.Task.run(Task.java:532) at java.base/java.lang.Thread.run(Thread.java:834)Caused by: java.lang.ClassNotFoundException: org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer at java.base/java.net.URLClassLoader.findClass(URLClassLoader.java:471) at java.base/java.lang.ClassLoader.loadClass(ClassLoader.java:588) at org.apache.flink.util.ChildFirstClassLoader.loadClass(ChildFirstClassLoader.java:60) at java.base/java.lang.ClassLoader.loadClass(ClassLoader.java:521) at java.base/java.lang.Class.forName0(Native Method) at java.base/java.lang.Class.forName(Class.java:398) at org.apache.flink.util.InstantiationUtil$ClassLoaderObjectInputStream.resolveClass(InstantiationUtil.java:78) at java.base/java.io.ObjectInputStream.readNonProxyDesc(ObjectInputStream.java:1886) at java.base/java.io.ObjectInputStream.readClassDesc(ObjectInputStream.java:1772) at java.base/java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream.java:2060) at java.base/java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1594) at java.base/java.io.ObjectInputStream.defaultReadFields(ObjectInputStream.java:2355) at java.base/java.io.ObjectInputStream.readSerialData(ObjectInputStream.java:2249) at java.base/java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream.java:2087) at java.base/java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1594) at java.base/java.io.ObjectInputStream.defaultReadFields(ObjectInputStream.java:2355) at java.base/java.io.ObjectInputStream.readSerialData(ObjectInputStream.java:2249) at java.base/java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream.java:2087) at java.base/java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1594) at java.base/java.io.ObjectInputStream.readObject(ObjectInputStream.java:430) at org.apache.flink.util.InstantiationUtil.deserializeObject(InstantiationUtil.java:576) at org.apache.flink.util.InstantiationUtil.deserializeObject(InstantiationUtil.java:562) at org.apache.flink.util.InstantiationUtil.deserializeObject(InstantiationUtil.java:550) at org.apache.flink.util.InstantiationUtil.readObjectFromConfig(InstantiationUtil.java:511) at org.apache.flink.streaming.api.graph.StreamConfig.getStreamOperatorFactory(StreamConfig.java:254) > Dynamic properties setting 'pipeline.jars' will be overwritten > -------------------------------------------------------------- > > Key: FLINK-18211 > URL: https://issues.apache.org/jira/browse/FLINK-18211 > Project: Flink > Issue Type: Bug > Components: Client / Job Submission > Affects Versions: 1.10.0, 1.11.0 > Reporter: Echo Lee > Assignee: Echo Lee > Priority: Major > Labels: pull-request-available > Fix For: 1.12.0 > > > When we submit the application through "flink run > -Dpipeline.jars='/user1.jar, user2.jar'..." command, configuration will > include 'pipeline.jars', But ExecutionConfigAccessor#fromProgramOptions will > be reset this property, So the property set by the user is invalid. -- This message was sent by Atlassian Jira (v8.3.4#803005)