[ https://issues.apache.org/jira/browse/FLINK-23972?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17423283#comment-17423283 ]
Till Rohrmann commented on FLINK-23972: --------------------------------------- Thanks for reporting this issue [~tashoyan]. Could you maybe use [Flink's application mode|https://nightlies.apache.org/flink/flink-docs-release-1.14/docs/deployment/resource-providers/yarn/#application-mode]? The application mode creates the user code on the cluster. Hence, it should have the same classpath as the cluster. Independent of this idea I think you are right that it would be helpful to be able to change the client's classpath. > NoClassDefFoundError: Jars from yarn.ship-files are missing in the classpath > of the client > ------------------------------------------------------------------------------------------ > > Key: FLINK-23972 > URL: https://issues.apache.org/jira/browse/FLINK-23972 > Project: Flink > Issue Type: Bug > Components: Client / Job Submission > Affects Versions: 1.12.4, 1.13.2 > Environment: Flink 1.12.4 or 1.13.2 on YARN > Reporter: Arseniy Tashoyan > Assignee: Arseniy Tashoyan > Priority: Major > > Flink provides a way to inject additional jars into the classpath of the Job > Manager and Task Manager: the configuration option > [yarn.ship-files|https://ci.apache.org/projects/flink/flink-docs-release-1.13/docs/deployment/config/#yarn-ship-files]. > However, an applications fails to submit to YARN if it uses a class from the > additional jars. The execution graph is built inside the client before the > application goes to the YARN cluster. Meanwhile, the client part (the > command-line tool *flink*) misses the additional jars in its classpath. > > *Example with Scalaz library.* > {code:scala} > package com.example > import ALenses.aLens > import org.apache.flink.streaming.api.scala.{StreamExecutionEnvironment, > createTypeInformation} > import scalaz.Lens > import scalaz.Lens.lensu > object DummyMain { > def main(args: Array[String]): Unit = { > val env = StreamExecutionEnvironment.getExecutionEnvironment > /* Way 1: Using Scalaz inside the client code. */ > val inputData = Seq(1, 2, 3).map { i => aLens.set(A(0), i) } > /* Way 2: Not using Scalaz inside the client code. */ > // val inputData = Seq(1, 2, 3).map(A) > val input = env.fromCollection(inputData) > val output = input > .map { a => > /* Using Scalaz inside Task Manager. */ > s"DUMMY: ${aLens.get(a)}" > } > output.print() > env.execute() > } > } > case class A( > i: Int > ) > object ALenses { > val aLens: Lens[A, Int] = lensu[A, Int]( > set = (a, i1) => a.copy(i = i1), > get = _.i > ) > } > {code} > Running the application as follows: > {code:bash} > # Find the additional jars under $lib_dir, scalaz-core_2.12-7.3.3.jar among > them. > # Join jars with semicolon > jars=$(mk_string \; $(find "$lib_dir" -type f -name \*.jar)) > # Provide the additional jars via yarn.ship-files > /opt/flink/bin/flink \ > run \ > --target yarn-per-job \ > --detached \ > --class com.example.DummyMain \ > -Dyarn.ship-files="$jars" \ > dummy-app-1.0.jar > {code} > > When using "Way 1: Using Scalaz inside the client code", the application > fails immediately - inside the client, before getting to YARN: > {code:none} > java.lang.NoClassDefFoundError: scalaz/Lens$ > at com.example.ALenses$.<init>(DummyMain.scala:38) > at com.example.ALenses$.<clinit>(DummyMain.scala) > at com.example.DummyMain$.$anonfun$main$1(DummyMain.scala:17) > at com.example.DummyMain$.$anonfun$main$1$adapted(DummyMain.scala:16) > at > scala.collection.TraversableLike.$anonfun$map$1(TraversableLike.scala:233) > at scala.collection.immutable.List.foreach(List.scala:388) > at scala.collection.TraversableLike.map(TraversableLike.scala:233) > at scala.collection.TraversableLike.map$(TraversableLike.scala:226) > at scala.collection.immutable.List.map(List.scala:294) > at com.example.DummyMain$.main(DummyMain.scala:16) > at com.example.DummyMain.main(DummyMain.scala) > 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:349) > at > org.apache.flink.client.program.PackagedProgram.invokeInteractiveModeForExecution(PackagedProgram.java:219) > at > org.apache.flink.client.ClientUtils.executeProgram(ClientUtils.java:114) > ... > {code} > When using "Way 2: Not using Scalaz inside the client code", the application > successfully runs on YARN. The subsequent usage of Scalaz inside Task > Managers does not cause a failure. > This behavior is a blocker - the setting yarn.ship-files is actually > non-working. > > *Possible solution* > Currently Flink provides no way to customize the classpath of the *flink* > client. The function *constructFlinkClassPath()* in > *FLINK_HOME/**bin/config.sh* does not provide any means for customization. > Instead this function could prepend the resulting classpath with a variable > *FLINK_CLIENT_ADD_CLASSPATH* settable by user. > Another approach: restore the command-line argument > [-yt|https://ci.apache.org/projects/flink/flink-docs-release-1.11/ops/cli.html#usage] > (removed in Flink 1.12). Use this argument to configure classpaths for all > components: client, Job Manager, Task Manager. > > There are 2 alternate ways to circumvent this problem - both are not suitable. > 1. Pack everything in a fat-jar. Fat-jars have numerous disadvantages, like > huge file size, many multiplied classes across different applications, > untraceable dependencies, classpath conflicts. > 2. Put application-level jars into *FLINK_HOME/lib*. We have the same Flink > installation for many different applications belonging to different teams. We > do not want to pollute the Flink installation with user's jars. *FLINK_HOME* > is located on a filesystem not writable for ordinary users. > > Discussions on StackOverflow: > [Flink on yarn: how to add external jars class path in local > Client|https://stackoverflow.com/questions/49423861/flink-on-yarn-how-to-add-external-jars-class-path-in-local-client] > [Flink: What should we use instead of command-line option > '-yt'?|https://stackoverflow.com/questions/68895883/flink-what-should-we-use-instead-of-command-line-option-yt] -- This message was sent by Atlassian Jira (v8.3.4#803005)