You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@spark.apache.org by Gary Malouf <ma...@gmail.com> on 2014/04/14 17:12:20 UTC

Re: Akka problem when using scala command to launch Spark applications in the current 0.9.0-SNAPSHOT

Sorry to dig up an old issue.

We build an assembly against spark-0.9.0-RC3 to run on our Spark cluster on
top of Mesos.  When we upgraded to 0.9.0-RC3 from an earlier master cut
from November, we ran into Akka issues described above.

Is it supported to be able to deploy this jar using the Spark classpath
script and Java?  Putting Scala 2.10.3's libraries on the path seems to
break it at runtime.


On Tue, Dec 24, 2013 at 3:49 PM, Patrick Wendell <pa...@databricks.com>wrote:

> Even,
>
> This problem also exists for people who write their own applications that
> depend on/include Spark. E.g. they bundle up their app and then launch the
> driver with "scala -cp my-budle.jar"... I've seen this cause an issue in
> that setting.
>
> - Patrick
>
>
> On Tue, Dec 24, 2013 at 10:50 AM, Evan Chan <ev...@ooyala.com> wrote:
>
> > Hi Reynold,
> >
> > The default, documented methods of starting Spark all use the assembly
> > jar, and thus java, right?
> >
> > -Evan
> >
> >
> >
> > On Fri, Dec 20, 2013 at 11:36 PM, Reynold Xin <rx...@databricks.com>
> wrote:
> >
> >> It took me hours to debug a problem yesterday on the latest master
> branch
> >> (0.9.0-SNAPSHOT), and I would like to share with the dev list in case
> >> anybody runs into this Akka problem.
> >>
> >> A little background for those of you who haven't followed closely the
> >> development of Spark and YARN 2.2: YARN 2.2 uses protobuf 2.5, and Akka
> >> uses an older version of protobuf that is not binary compatible. In
> order
> >> to have a single build that is compatible for both YARN 2.2 and pre-2.2
> >> YARN/Hadoop, we published a special version of Akka that builds with
> >> protobuf shaded (i.e. using a different package name for the protobuf
> >> stuff).
> >>
> >> However, it turned out Scala 2.10 includes a version of Akka jar in its
> >> default classpath (look at the lib folder in Scala 2.10 binary
> >> distribution). If you use the scala command to launch any Spark
> >> application
> >> on the current master branch, there is a pretty high chance that you
> >> wouldn't be able to create the SparkContext (stack trace at the end of
> the
> >> email). The problem is that the Akka packaged with Scala 2.10 takes
> >> precedence in the classloader over the special Akka version Spark
> >> includes.
> >>
> >> Before we have a good solution for this, the workaround is to use java
> to
> >> launch the application instead of scala. All you need to do is to
> include
> >> the right Scala jars (scala-library and scala-compiler) in the
> classpath.
> >> Note that the scala command is really just a simple script that calls
> java
> >> with the right classpath.
> >>
> >>
> >> Stack trace:
> >>
> >> java.lang.NoSuchMethodException:
> >> akka.remote.RemoteActorRefProvider.<init>(java.lang.String,
> >> akka.actor.ActorSystem$Settings, akka.event.EventStream,
> >> akka.actor.Scheduler, akka.actor.DynamicAccess)
> >> at java.lang.Class.getConstructor0(Class.java:2763)
> >> at java.lang.Class.getDeclaredConstructor(Class.java:2021)
> >> at
> >>
> >>
> akka.actor.ReflectiveDynamicAccess$$anonfun$createInstanceFor$2.apply(DynamicAccess.scala:77)
> >> at scala.util.Try$.apply(Try.scala:161)
> >> at
> >>
> >>
> akka.actor.ReflectiveDynamicAccess.createInstanceFor(DynamicAccess.scala:74)
> >> at
> >>
> >>
> akka.actor.ReflectiveDynamicAccess$$anonfun$createInstanceFor$3.apply(DynamicAccess.scala:85)
> >> at
> >>
> >>
> akka.actor.ReflectiveDynamicAccess$$anonfun$createInstanceFor$3.apply(DynamicAccess.scala:85)
> >> at scala.util.Success.flatMap(Try.scala:200)
> >> at
> >>
> >>
> akka.actor.ReflectiveDynamicAccess.createInstanceFor(DynamicAccess.scala:85)
> >> at akka.actor.ActorSystemImpl.<init>(ActorSystem.scala:546)
> >> at akka.actor.ActorSystem$.apply(ActorSystem.scala:111)
> >> at akka.actor.ActorSystem$.apply(ActorSystem.scala:104)
> >> at
> org.apache.spark.util.AkkaUtils$.createActorSystem(AkkaUtils.scala:79)
> >> at
> >>
> org.apache.spark.SparkEnv$.createFromSystemProperties(SparkEnv.scala:120)
> >> at org.apache.spark.SparkContext.<init>(SparkContext.scala:106)
> >>
> >
> >
> >
> > --
> > --
> > Evan Chan
> > Staff Engineer
> > ev@ooyala.com  |
> >
> > <http://www.ooyala.com/> <http://www.facebook.com/ooyala><
> http://www.linkedin.com/company/ooyala><http://www.twitter.com/ooyala>
> >
> >
>