You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@spark.apache.org by Koert Kuipers <ko...@tresata.com> on 2014/06/19 19:22:59 UTC

trying to understand yarn-client mode

i am trying to understand how yarn-client mode works. i am not using
spark-submit, but instead launching a spark job from within my own
application.

i can see my application contacting yarn successfully, but then in yarn i
get an immediate error:

Application application_1403117970283_0014 failed 2 times due to AM
Container for appattempt_1403117970283_0014_000002 exited with exitCode:
-1000 due to: File file:/home/koert/test-assembly-0.1-SNAPSHOT.jar does not
exist
 .Failing this attempt.. Failing the application.

why is yarn trying to fetch my jar, and why as a local file? i would expect
the jar to be send to yarn over the wire upon job submission?

Re: trying to understand yarn-client mode

Posted by Marcelo Vanzin <va...@cloudera.com>.
On Fri, Jun 20, 2014 at 8:22 AM, Koert Kuipers <ko...@tresata.com> wrote:
> thanks! i will try that.
> i guess what i am most confused about is why the executors are trying to
> retrieve the jars directly using the info i provided to add jars to my spark
> context. i mean, thats bound to fail no? i could be on a different machine
> (so my file://) isnt going to work for them, or i could have the jars in a
> directory that is only readable by me.
>
> how come the jars are not just shipped to yarn as part of the job submittal?

They are if they are specified correctly. Check the guide:
http://spark.apache.org/docs/latest/submitting-applications.html

See the "Advanced Dependency Management" section.

Your default filesystem is probably hdfs, which means that if you
provide a path with no protocol, the executors will consider it as an
hdfs path, and it won't work if you're pointing at a file that exists
in your local fs.


-- 
Marcelo

Re: trying to understand yarn-client mode

Posted by Koert Kuipers <ko...@tresata.com>.
thanks! i will try that.
i guess what i am most confused about is why the executors are trying to
retrieve the jars directly using the info i provided to add jars to my
spark context. i mean, thats bound to fail no? i could be on a different
machine (so my file://) isnt going to work for them, or i could have the
jars in a directory that is only readable by me.

how come the jars are not just shipped to yarn as part of the job submittal?

i am worried i am supposed to put the jars in a "central" location and yarn
is going to fetch them from there, leading to jars in yet another place
such as on hdfs which i find pretty messy.


On Thu, Jun 19, 2014 at 2:54 PM, Marcelo Vanzin <va...@cloudera.com> wrote:

> Coincidentally, I just ran into the same exception. What's probably
> happening is that you're specifying some jar file in your job as an
> absolute local path (e.g. just
> "/home/koert/test-assembly-0.1-SNAPSHOT.jar"), but your Hadoop config
> has the default FS set to HDFS.
>
> So your driver does not know that it should tell executors to download
> that file from the driver.
>
> If you specify the jar with the "file:" scheme that should solve the
> problem.
>
> On Thu, Jun 19, 2014 at 10:22 AM, Koert Kuipers <ko...@tresata.com> wrote:
> > i am trying to understand how yarn-client mode works. i am not using
> > Application application_1403117970283_0014 failed 2 times due to AM
> > Container for appattempt_1403117970283_0014_000002 exited with exitCode:
> > -1000 due to: File file:/home/koert/test-assembly-0.1-SNAPSHOT.jar does
> not
> > exist
> > .Failing this attempt.. Failing the application.
>
>
> --
> Marcelo
>

Re: trying to understand yarn-client mode

Posted by Marcelo Vanzin <va...@cloudera.com>.
Coincidentally, I just ran into the same exception. What's probably
happening is that you're specifying some jar file in your job as an
absolute local path (e.g. just
"/home/koert/test-assembly-0.1-SNAPSHOT.jar"), but your Hadoop config
has the default FS set to HDFS.

So your driver does not know that it should tell executors to download
that file from the driver.

If you specify the jar with the "file:" scheme that should solve the problem.

On Thu, Jun 19, 2014 at 10:22 AM, Koert Kuipers <ko...@tresata.com> wrote:
> i am trying to understand how yarn-client mode works. i am not using
> Application application_1403117970283_0014 failed 2 times due to AM
> Container for appattempt_1403117970283_0014_000002 exited with exitCode:
> -1000 due to: File file:/home/koert/test-assembly-0.1-SNAPSHOT.jar does not
> exist
> .Failing this attempt.. Failing the application.


-- 
Marcelo

Re: trying to understand yarn-client mode

Posted by Koert Kuipers <ko...@tresata.com>.
okay. since for us the main purpose is to retrieve (small) data i guess i
will stick to yarn client mode. thx


On Thu, Jun 19, 2014 at 3:19 PM, DB Tsai <db...@stanford.edu> wrote:

> Currently, we save the result in HDFS, and read it back in our
> application. Since Clinet.run is blocking call, it's easy to do it in
> this way.
>
> We are now working on using akka to bring back the result to app
> without going through the HDFS, and we can use akka to track the log,
> and stack trace, etc.
>
> Sincerely,
>
> DB Tsai
> -------------------------------------------------------
> My Blog: https://www.dbtsai.com
> LinkedIn: https://www.linkedin.com/in/dbtsai
>
>
> On Thu, Jun 19, 2014 at 12:08 PM, Koert Kuipers <ko...@tresata.com> wrote:
> > db tsai,
> > if in yarn-cluster mode the driver runs inside yarn, how can you do a
> > rdd.collect and bring the results back to your application?
> >
> >
> > On Thu, Jun 19, 2014 at 2:33 PM, DB Tsai <db...@stanford.edu> wrote:
> >>
> >> We are submitting the spark job in our tomcat application using
> >> yarn-cluster mode with great success. As Kevin said, yarn-client mode
> >> runs driver in your local JVM, and it will have really bad network
> >> overhead when one do reduce action which will pull all the result from
> >> executor to your local JVM. Also, since you can only have one spark
> >> context object in one JVM, it will be tricky to run multiple spark
> >> jobs concurrently with yarn-clinet mode.
> >>
> >> This is how we submit spark job with yarn-cluster mode. Please use the
> >> current master code, otherwise, after the job is finished, spark will
> >> kill the JVM and exit your app.
> >>
> >> We setup the configuration of spark in a scala map.
> >>
> >>   def getArgsFromConf(conf: Map[String, String]): Array[String] = {
> >>     Array[String](
> >>       "--jar", conf.get("app.jar").getOrElse(""),
> >>       "--addJars", conf.get("spark.addJars").getOrElse(""),
> >>       "--class", conf.get("spark.mainClass").getOrElse(""),
> >>       "--num-executors", conf.get("spark.numWorkers").getOrElse("1"),
> >>       "--driver-memory", conf.get("spark.masterMemory").getOrElse("1g"),
> >>       "--executor-memory",
> conf.get("spark.workerMemory").getOrElse("1g"),
> >>       "--executor-cores", conf.get("spark.workerCores").getOrElse("1"))
> >>   }
> >>
> >>       System.setProperty("SPARK_YARN_MODE", "true")
> >>       val sparkConf = new SparkConf
> >>       val args = getArgsFromConf(conf)
> >>       new Client(new ClientArguments(args, sparkConf), hadoopConfig,
> >> sparkConf).run
> >>
> >> Sincerely,
> >>
> >> DB Tsai
> >> -------------------------------------------------------
> >> My Blog: https://www.dbtsai.com
> >> LinkedIn: https://www.linkedin.com/in/dbtsai
> >>
> >>
> >> On Thu, Jun 19, 2014 at 11:22 AM, Kevin Markey <kevin.markey@oracle.com
> >
> >> wrote:
> >> > Yarn client is much like Spark client mode, except that the executors
> >> > are
> >> > running in Yarn containers managed by the Yarn resource manager on the
> >> > cluster instead of as Spark workers managed by the Spark master.  The
> >> > driver
> >> > executes as a local client in your local JVM.  It communicates with
> the
> >> > workers on the cluster.  Transformations are scheduled on the cluster
> by
> >> > the
> >> > driver's logic.  Actions involve communication between local driver
> and
> >> > remote cluster executors.  So, there is some additional network
> >> > overhead,
> >> > especially if the driver is not co-located on the cluster.  In
> >> > yarn-cluster
> >> > mode -- in contrast, the driver is executed as a thread in a Yarn
> >> > application master on the cluster.
> >> >
> >> > In either case, the assembly JAR must be available to the application
> on
> >> > the
> >> > cluster.  Best to copy it to HDFS and specify its location by
> exporting
> >> > its
> >> > location as SPARK_JAR.
> >> >
> >> > Kevin Markey
> >> >
> >> >
> >> > On 06/19/2014 11:22 AM, Koert Kuipers wrote:
> >> >
> >> > i am trying to understand how yarn-client mode works. i am not using
> >> > spark-submit, but instead launching a spark job from within my own
> >> > application.
> >> >
> >> > i can see my application contacting yarn successfully, but then in
> yarn
> >> > i
> >> > get an immediate error:
> >> >
> >> > Application application_1403117970283_0014 failed 2 times due to AM
> >> > Container for appattempt_1403117970283_0014_000002 exited with
> exitCode:
> >> > -1000 due to: File file:/home/koert/test-assembly-0.1-SNAPSHOT.jar
> does
> >> > not
> >> > exist
> >> > .Failing this attempt.. Failing the application.
> >> >
> >> > why is yarn trying to fetch my jar, and why as a local file? i would
> >> > expect
> >> > the jar to be send to yarn over the wire upon job submission?
> >> >
> >> >
> >
> >
>

Re: trying to understand yarn-client mode

Posted by DB Tsai <db...@stanford.edu>.
Currently, we save the result in HDFS, and read it back in our
application. Since Clinet.run is blocking call, it's easy to do it in
this way.

We are now working on using akka to bring back the result to app
without going through the HDFS, and we can use akka to track the log,
and stack trace, etc.

Sincerely,

DB Tsai
-------------------------------------------------------
My Blog: https://www.dbtsai.com
LinkedIn: https://www.linkedin.com/in/dbtsai


On Thu, Jun 19, 2014 at 12:08 PM, Koert Kuipers <ko...@tresata.com> wrote:
> db tsai,
> if in yarn-cluster mode the driver runs inside yarn, how can you do a
> rdd.collect and bring the results back to your application?
>
>
> On Thu, Jun 19, 2014 at 2:33 PM, DB Tsai <db...@stanford.edu> wrote:
>>
>> We are submitting the spark job in our tomcat application using
>> yarn-cluster mode with great success. As Kevin said, yarn-client mode
>> runs driver in your local JVM, and it will have really bad network
>> overhead when one do reduce action which will pull all the result from
>> executor to your local JVM. Also, since you can only have one spark
>> context object in one JVM, it will be tricky to run multiple spark
>> jobs concurrently with yarn-clinet mode.
>>
>> This is how we submit spark job with yarn-cluster mode. Please use the
>> current master code, otherwise, after the job is finished, spark will
>> kill the JVM and exit your app.
>>
>> We setup the configuration of spark in a scala map.
>>
>>   def getArgsFromConf(conf: Map[String, String]): Array[String] = {
>>     Array[String](
>>       "--jar", conf.get("app.jar").getOrElse(""),
>>       "--addJars", conf.get("spark.addJars").getOrElse(""),
>>       "--class", conf.get("spark.mainClass").getOrElse(""),
>>       "--num-executors", conf.get("spark.numWorkers").getOrElse("1"),
>>       "--driver-memory", conf.get("spark.masterMemory").getOrElse("1g"),
>>       "--executor-memory", conf.get("spark.workerMemory").getOrElse("1g"),
>>       "--executor-cores", conf.get("spark.workerCores").getOrElse("1"))
>>   }
>>
>>       System.setProperty("SPARK_YARN_MODE", "true")
>>       val sparkConf = new SparkConf
>>       val args = getArgsFromConf(conf)
>>       new Client(new ClientArguments(args, sparkConf), hadoopConfig,
>> sparkConf).run
>>
>> Sincerely,
>>
>> DB Tsai
>> -------------------------------------------------------
>> My Blog: https://www.dbtsai.com
>> LinkedIn: https://www.linkedin.com/in/dbtsai
>>
>>
>> On Thu, Jun 19, 2014 at 11:22 AM, Kevin Markey <ke...@oracle.com>
>> wrote:
>> > Yarn client is much like Spark client mode, except that the executors
>> > are
>> > running in Yarn containers managed by the Yarn resource manager on the
>> > cluster instead of as Spark workers managed by the Spark master.  The
>> > driver
>> > executes as a local client in your local JVM.  It communicates with the
>> > workers on the cluster.  Transformations are scheduled on the cluster by
>> > the
>> > driver's logic.  Actions involve communication between local driver and
>> > remote cluster executors.  So, there is some additional network
>> > overhead,
>> > especially if the driver is not co-located on the cluster.  In
>> > yarn-cluster
>> > mode -- in contrast, the driver is executed as a thread in a Yarn
>> > application master on the cluster.
>> >
>> > In either case, the assembly JAR must be available to the application on
>> > the
>> > cluster.  Best to copy it to HDFS and specify its location by exporting
>> > its
>> > location as SPARK_JAR.
>> >
>> > Kevin Markey
>> >
>> >
>> > On 06/19/2014 11:22 AM, Koert Kuipers wrote:
>> >
>> > i am trying to understand how yarn-client mode works. i am not using
>> > spark-submit, but instead launching a spark job from within my own
>> > application.
>> >
>> > i can see my application contacting yarn successfully, but then in yarn
>> > i
>> > get an immediate error:
>> >
>> > Application application_1403117970283_0014 failed 2 times due to AM
>> > Container for appattempt_1403117970283_0014_000002 exited with exitCode:
>> > -1000 due to: File file:/home/koert/test-assembly-0.1-SNAPSHOT.jar does
>> > not
>> > exist
>> > .Failing this attempt.. Failing the application.
>> >
>> > why is yarn trying to fetch my jar, and why as a local file? i would
>> > expect
>> > the jar to be send to yarn over the wire upon job submission?
>> >
>> >
>
>

Re: trying to understand yarn-client mode

Posted by Koert Kuipers <ko...@tresata.com>.
db tsai,
if in yarn-cluster mode the driver runs inside yarn, how can you do a
rdd.collect and bring the results back to your application?


On Thu, Jun 19, 2014 at 2:33 PM, DB Tsai <db...@stanford.edu> wrote:

> We are submitting the spark job in our tomcat application using
> yarn-cluster mode with great success. As Kevin said, yarn-client mode
> runs driver in your local JVM, and it will have really bad network
> overhead when one do reduce action which will pull all the result from
> executor to your local JVM. Also, since you can only have one spark
> context object in one JVM, it will be tricky to run multiple spark
> jobs concurrently with yarn-clinet mode.
>
> This is how we submit spark job with yarn-cluster mode. Please use the
> current master code, otherwise, after the job is finished, spark will
> kill the JVM and exit your app.
>
> We setup the configuration of spark in a scala map.
>
>   def getArgsFromConf(conf: Map[String, String]): Array[String] = {
>     Array[String](
>       "--jar", conf.get("app.jar").getOrElse(""),
>       "--addJars", conf.get("spark.addJars").getOrElse(""),
>       "--class", conf.get("spark.mainClass").getOrElse(""),
>       "--num-executors", conf.get("spark.numWorkers").getOrElse("1"),
>       "--driver-memory", conf.get("spark.masterMemory").getOrElse("1g"),
>       "--executor-memory", conf.get("spark.workerMemory").getOrElse("1g"),
>       "--executor-cores", conf.get("spark.workerCores").getOrElse("1"))
>   }
>
>       System.setProperty("SPARK_YARN_MODE", "true")
>       val sparkConf = new SparkConf
>       val args = getArgsFromConf(conf)
>       new Client(new ClientArguments(args, sparkConf), hadoopConfig,
> sparkConf).run
>
> Sincerely,
>
> DB Tsai
> -------------------------------------------------------
> My Blog: https://www.dbtsai.com
> LinkedIn: https://www.linkedin.com/in/dbtsai
>
>
> On Thu, Jun 19, 2014 at 11:22 AM, Kevin Markey <ke...@oracle.com>
> wrote:
> > Yarn client is much like Spark client mode, except that the executors are
> > running in Yarn containers managed by the Yarn resource manager on the
> > cluster instead of as Spark workers managed by the Spark master.  The
> driver
> > executes as a local client in your local JVM.  It communicates with the
> > workers on the cluster.  Transformations are scheduled on the cluster by
> the
> > driver's logic.  Actions involve communication between local driver and
> > remote cluster executors.  So, there is some additional network overhead,
> > especially if the driver is not co-located on the cluster.  In
> yarn-cluster
> > mode -- in contrast, the driver is executed as a thread in a Yarn
> > application master on the cluster.
> >
> > In either case, the assembly JAR must be available to the application on
> the
> > cluster.  Best to copy it to HDFS and specify its location by exporting
> its
> > location as SPARK_JAR.
> >
> > Kevin Markey
> >
> >
> > On 06/19/2014 11:22 AM, Koert Kuipers wrote:
> >
> > i am trying to understand how yarn-client mode works. i am not using
> > spark-submit, but instead launching a spark job from within my own
> > application.
> >
> > i can see my application contacting yarn successfully, but then in yarn i
> > get an immediate error:
> >
> > Application application_1403117970283_0014 failed 2 times due to AM
> > Container for appattempt_1403117970283_0014_000002 exited with exitCode:
> > -1000 due to: File file:/home/koert/test-assembly-0.1-SNAPSHOT.jar does
> not
> > exist
> > .Failing this attempt.. Failing the application.
> >
> > why is yarn trying to fetch my jar, and why as a local file? i would
> expect
> > the jar to be send to yarn over the wire upon job submission?
> >
> >
>

Re: trying to understand yarn-client mode

Posted by nir <ni...@gmail.com>.
Hi DB Tsai-2,

I am trying to run singleton sparkcontext in my container (spring-boot
tomcat container). When my application bootstrap I used to create
sparkContext and keep the reference for future job submission. I got it
working with standalone spark perfectly but I am having trouble with yarn
modes specially yarn-cluster mode. 
What is "new Client(new ClientArguments(args, sparkConf), hadoopConfig,
sparkConf).run " API? 
How do submit subsequent request to spark after this?

I use Java API but I can use scala too.



--
View this message in context: http://apache-spark-user-list.1001560.n3.nabble.com/trying-to-understand-yarn-client-mode-tp7925p24145.html
Sent from the Apache Spark User List mailing list archive at Nabble.com.

---------------------------------------------------------------------
To unsubscribe, e-mail: user-unsubscribe@spark.apache.org
For additional commands, e-mail: user-help@spark.apache.org


Re: trying to understand yarn-client mode

Posted by DB Tsai <db...@stanford.edu>.
We are submitting the spark job in our tomcat application using
yarn-cluster mode with great success. As Kevin said, yarn-client mode
runs driver in your local JVM, and it will have really bad network
overhead when one do reduce action which will pull all the result from
executor to your local JVM. Also, since you can only have one spark
context object in one JVM, it will be tricky to run multiple spark
jobs concurrently with yarn-clinet mode.

This is how we submit spark job with yarn-cluster mode. Please use the
current master code, otherwise, after the job is finished, spark will
kill the JVM and exit your app.

We setup the configuration of spark in a scala map.

  def getArgsFromConf(conf: Map[String, String]): Array[String] = {
    Array[String](
      "--jar", conf.get("app.jar").getOrElse(""),
      "--addJars", conf.get("spark.addJars").getOrElse(""),
      "--class", conf.get("spark.mainClass").getOrElse(""),
      "--num-executors", conf.get("spark.numWorkers").getOrElse("1"),
      "--driver-memory", conf.get("spark.masterMemory").getOrElse("1g"),
      "--executor-memory", conf.get("spark.workerMemory").getOrElse("1g"),
      "--executor-cores", conf.get("spark.workerCores").getOrElse("1"))
  }

      System.setProperty("SPARK_YARN_MODE", "true")
      val sparkConf = new SparkConf
      val args = getArgsFromConf(conf)
      new Client(new ClientArguments(args, sparkConf), hadoopConfig,
sparkConf).run

Sincerely,

DB Tsai
-------------------------------------------------------
My Blog: https://www.dbtsai.com
LinkedIn: https://www.linkedin.com/in/dbtsai


On Thu, Jun 19, 2014 at 11:22 AM, Kevin Markey <ke...@oracle.com> wrote:
> Yarn client is much like Spark client mode, except that the executors are
> running in Yarn containers managed by the Yarn resource manager on the
> cluster instead of as Spark workers managed by the Spark master.  The driver
> executes as a local client in your local JVM.  It communicates with the
> workers on the cluster.  Transformations are scheduled on the cluster by the
> driver's logic.  Actions involve communication between local driver and
> remote cluster executors.  So, there is some additional network overhead,
> especially if the driver is not co-located on the cluster.  In yarn-cluster
> mode -- in contrast, the driver is executed as a thread in a Yarn
> application master on the cluster.
>
> In either case, the assembly JAR must be available to the application on the
> cluster.  Best to copy it to HDFS and specify its location by exporting its
> location as SPARK_JAR.
>
> Kevin Markey
>
>
> On 06/19/2014 11:22 AM, Koert Kuipers wrote:
>
> i am trying to understand how yarn-client mode works. i am not using
> spark-submit, but instead launching a spark job from within my own
> application.
>
> i can see my application contacting yarn successfully, but then in yarn i
> get an immediate error:
>
> Application application_1403117970283_0014 failed 2 times due to AM
> Container for appattempt_1403117970283_0014_000002 exited with exitCode:
> -1000 due to: File file:/home/koert/test-assembly-0.1-SNAPSHOT.jar does not
> exist
> .Failing this attempt.. Failing the application.
>
> why is yarn trying to fetch my jar, and why as a local file? i would expect
> the jar to be send to yarn over the wire upon job submission?
>
>

Re: trying to understand yarn-client mode

Posted by firemonk9 <dh...@gmail.com>.
I am able to fix it by adding the the jars(in the spark distribution) to the
classpath. In my sbt file I changed the scope to provided. 

Let me know if you need more details.



--
View this message in context: http://apache-spark-user-list.1001560.n3.nabble.com/trying-to-understand-yarn-client-mode-tp7925p20893.html
Sent from the Apache Spark User List mailing list archive at Nabble.com.

---------------------------------------------------------------------
To unsubscribe, e-mail: user-unsubscribe@spark.apache.org
For additional commands, e-mail: user-help@spark.apache.org


Re: trying to understand yarn-client mode

Posted by Fernando Otero <fo...@gmail.com>.
were you able to fix the issue? I'm facing a similar issue when trying to use
yarn client from spark-jobserver




--
View this message in context: http://apache-spark-user-list.1001560.n3.nabble.com/trying-to-understand-yarn-client-mode-tp7925p20888.html
Sent from the Apache Spark User List mailing list archive at Nabble.com.

---------------------------------------------------------------------
To unsubscribe, e-mail: user-unsubscribe@spark.apache.org
For additional commands, e-mail: user-help@spark.apache.org