In my opinion, the fat jar solution is easier than having to copy the Gelly jar to all task managers. I would be in favor of including flink-gelly in the flink-examples module, but we can also simply document the current behavior nicely.
-V. On 29 February 2016 at 12:15, Till Rohrmann <trohrm...@apache.org> wrote: > Good catch :-) I mean we could also change the behaviour to include > flink-gelly in the flink-gelly-examples module. > > On Mon, Feb 29, 2016 at 12:13 PM, Vasiliki Kalavri < > vasilikikala...@gmail.com> wrote: > > > Thanks Till! Then, we'd better update the docs, too. Currently both links > > to examples and cluster execution instructions are broken. > > I'll create an issue. > > > > -V. > > > > On 29 February 2016 at 11:54, Till Rohrmann <trohrm...@apache.org> > wrote: > > > > > Hi Vasia, > > > > > > that is because you're missing the flink-gelly dependency. If you just > > > build flink-gelly-examples, then it won't contain flink-gelly because > it > > is > > > not a fat jar. You either have to install flink-gelly on your cluster > or > > > package it in the final user jar. > > > > > > Cheers, > > > Till > > > > > > On Sat, Feb 27, 2016 at 7:19 PM, Vasiliki Kalavri < > > > vasilikikala...@gmail.com > > > > wrote: > > > > > > > Hi squirrels, > > > > > > > > sorry I've been slow to respond to this, but I'm now testing RC1 and > > I'm > > > a > > > > bit confused with this change. > > > > > > > > So far, the easier way to run a Gelly example on a cluster was to > > package > > > > and submit the Gelly jar. > > > > Now, since the flink-gelly project doesn't contain the examples > > anymore, > > > I > > > > tried running a Gelly example by the packaging flink-gelly-examples > jar > > > > (mvn package). However, this gives me a ClassNotFoundException. > > > > e.g. for SingleSourceShortestPaths, the following: > > > > > > > > java.lang.RuntimeException: Could not look up the main(String[]) > method > > > > from the class > > org.apache.flink.graph.examples.SingleSourceShortestPaths: > > > > org/apache/flink/graph/spargel/MessagingFunction > > > > at > > > > > > > > > > > > > > org.apache.flink.client.program.PackagedProgram.hasMainMethod(PackagedProgram.java:478) > > > > at > > > > > > > > > > > > > > org.apache.flink.client.program.PackagedProgram.<init>(PackagedProgram.java:216) > > > > at > > org.apache.flink.client.CliFrontend.buildProgram(CliFrontend.java:922) > > > > at org.apache.flink.client.CliFrontend.run(CliFrontend.java:301) > > > > at > > > > > > > > > > org.apache.flink.client.CliFrontend.parseParameters(CliFrontend.java:1189) > > > > at org.apache.flink.client.CliFrontend.main(CliFrontend.java:1239) > > > > Caused by: java.lang.NoClassDefFoundError: > > > > org/apache/flink/graph/spargel/MessagingFunction > > > > at java.lang.Class.getDeclaredMethods0(Native Method) > > > > at java.lang.Class.privateGetDeclaredMethods(Class.java:2531) > > > > at java.lang.Class.getMethod0(Class.java:2774) > > > > at java.lang.Class.getMethod(Class.java:1663) > > > > at > > > > > > > > > > > > > > org.apache.flink.client.program.PackagedProgram.hasMainMethod(PackagedProgram.java:473) > > > > ... 5 more > > > > Caused by: java.lang.ClassNotFoundException: > > > > org.apache.flink.graph.spargel.MessagingFunction > > > > at java.net.URLClassLoader$1.run(URLClassLoader.java:366) > > > > at java.net.URLClassLoader$1.run(URLClassLoader.java:355) > > > > at java.security.AccessController.doPrivileged(Native Method) > > > > at java.net.URLClassLoader.findClass(URLClassLoader.java:354) > > > > at java.lang.ClassLoader.loadClass(ClassLoader.java:425) > > > > at java.lang.ClassLoader.loadClass(ClassLoader.java:358) > > > > ... 10 more > > > > > > > > > > > > What am I missing here? > > > > > > > > Thanks! > > > > -Vasia. > > > > > > > > On 26 February 2016 at 14:10, Márton Balassi < > balassi.mar...@gmail.com > > > > > > > wrote: > > > > > > > > > Thanks, +1. > > > > > > > > > > On Fri, Feb 26, 2016 at 12:35 PM, Stephan Ewen <se...@apache.org> > > > wrote: > > > > > > > > > > > Hi! > > > > > > > > > > > > I think it is a release blocker. > > > > > > > > > > > > That was a change to make it possible to use Flink with SBT (and > > make > > > > it > > > > > > easier to use it with Maven). > > > > > > Just had an offline chat with Till, and we suggest the following > > > > > solution: > > > > > > > > > > > > All libraries should be split into two projects: > > > > > > - library > > > > > > - library-examples > > > > > > > > > > > > The "library" project has the core flink dependencies as > "provided" > > > > > > The "library-examples" project has both the "library" and the > core > > > > flink > > > > > > dependencies with scope "compile" > > > > > > > > > > > > That way the example should run in the IDE out of the cox, and > > users > > > > that > > > > > > reference the libraries will still get the correct packaging > > (include > > > > the > > > > > > library in the user jar, but not additionally the core flink > jars). > > > > > > > > > > > > Greetings, > > > > > > Stephan > > > > > > > > > > > > > > > > > > On Thu, Feb 25, 2016 at 3:45 PM, Márton Balassi < > > > > > balassi.mar...@gmail.com> > > > > > > wrote: > > > > > > > > > > > > > Issued JIRA ticket 3511 to make it referable in other > > discussions. > > > > [1] > > > > > > > > > > > > > > [1] https://issues.apache.org/jira/browse/FLINK-3511 > > > > > > > > > > > > > > On Thu, Feb 25, 2016 at 3:36 PM, Márton Balassi < > > > > > > balassi.mar...@gmail.com> > > > > > > > wrote: > > > > > > > > > > > > > > > Recent changes to the build [1] where many libraries got > their > > > core > > > > > > > > dependencies (the ones included in the flink-dist fat jar) > > moved > > > to > > > > > the > > > > > > > > provided scope. > > > > > > > > > > > > > > > > The reasoning was that when submitting to the Flink cluster > the > > > > > > > > application already has these dependencies, while when a user > > > > writes > > > > > a > > > > > > > > program against these libraries she will include the core > > > > > dependencies > > > > > > > > explicitly anyway. > > > > > > > > > > > > > > > > There is one other case of usage however, namely when someone > > is > > > > > trying > > > > > > > to > > > > > > > > run an application defined in these libraries depending on > the > > > core > > > > > > jars. > > > > > > > > To give an example if you were to run the Gelly > > > ConnectedComponents > > > > > > > example > > > > > > > > [2] from an IDE after importing Flink (or running with java > > -jar > > > > > > without > > > > > > > > including the flink fat jar in the classpath) you would > receive > > > the > > > > > > > > following class not found exception as per the current > master: > > > > > > > > > > > > > > > > Exception in thread "main" java.lang.NoClassDefFoundError: > > > > > > > > org/apache/flink/api/common/ProgramDescription > > > > > > > > at java.lang.ClassLoader.defineClass1(Native Method) > > > > > > > > at java.lang.ClassLoader.defineClass(ClassLoader.java:760) > > > > > > > > at > > > > > > > > > > > > java.security.SecureClassLoader.defineClass(SecureClassLoader.java:142) > > > > > > > > at > java.net.URLClassLoader.defineClass(URLClassLoader.java:467) > > > > > > > > at java.net.URLClassLoader.access$100(URLClassLoader.java:73) > > > > > > > > at java.net.URLClassLoader$1.run(URLClassLoader.java:368) > > > > > > > > at java.net.URLClassLoader$1.run(URLClassLoader.java:362) > > > > > > > > at java.security.AccessController.doPrivileged(Native Method) > > > > > > > > at java.net.URLClassLoader.findClass(URLClassLoader.java:361) > > > > > > > > at java.lang.ClassLoader.loadClass(ClassLoader.java:424) > > > > > > > > at > > sun.misc.Launcher$AppClassLoader.loadClass(Launcher.java:331) > > > > > > > > at java.lang.ClassLoader.loadClass(ClassLoader.java:357) > > > > > > > > at java.lang.Class.forName0(Native Method) > > > > > > > > at java.lang.Class.forName(Class.java:264) > > > > > > > > at > > > > > > com.intellij.rt.execution.application.AppMain.main(AppMain.java:122) > > > > > > > > > > > > > > > > Where the class missing in runtime is in flink-core, a > > transitive > > > > > > > > dependency of the jars moved to the provided scope. > > > > > > > > > > > > > > > > Funny thing is we have tests in place to run our examples, > but > > > > those > > > > > > add > > > > > > > > test scope dependencies re-adding the missing classes, so it > is > > > > never > > > > > > > > discovered. > > > > > > > > > > > > > > > > I agree with the original purpose of PR #1683, but also think > > > that > > > > > the > > > > > > > > current state makes for very inconvenient user experience. > > > > > > > > > > > > > > > > I would like to open a discussion on how and when to resolve > > the > > > > > issue > > > > > > > > given the release of 1.0.0. > > > > > > > > > > > > > > > > 1. Is it a release blocker? > > > > > > > > 2. Should the change be reverted or is it sufficient to have > > > proper > > > > > > > > documentation around it? Maybe a maven profile for explicitly > > for > > > > > > > > developing Flink without the provided scope? > > > > > > > > > > > > > > > > Note that the issue was originally reported by Gábor Gévay. > > > > > > > > > > > > > > > > [1] https://github.com/apache/flink/pull/1683 > > > > > > > > [2] > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > https://github.com/apache/flink/blob/master/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/example/ConnectedComponents.java > > > > > > > > > > > > > > > > Best, > > > > > > > > > > > > > > > > Marton > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > >