I’m beginning to agree with you and find it rather surprising that this is
mentioned nowhere explicitly (maybe I missed?). It is possible to serialize
code to be executed in executors to various nodes. It also seems possible
to serialize the “driver” bits of code although I’m not sure how the
boundary would be defined. All code is in the jars we pass to Spark so
until now I did not question the docs.

I see no mention of a distinction between running a driver in spark-submit
vs being programmatically launched for any of the Spark Master types:
Standalone, Yarn, Mesos, k8s.

We are building a Machine Learning Server in OSS. It has pluggable Engines
for different algorithms. Some of these use Spark so it is highly desirable
to offload driver code to the cluster since we don’t want the diver
embedded in the Server process. The Driver portion of our training workflow
could be very large indeed and so could force the scaling of the server to
worst case.

I hope someone knows how to run “Driver” code on the cluster when our
server is launching the code. So deployMode = cluster, deploy method =
programatic launch.


From: Andrew Melo <andrew.m...@gmail.com> <andrew.m...@gmail.com>
Reply: Andrew Melo <andrew.m...@gmail.com> <andrew.m...@gmail.com>
Date: March 25, 2019 at 11:40:07 AM
To: Pat Ferrel <p...@occamsmachete.com> <p...@occamsmachete.com>
Cc: Akhil Das <ak...@hacked.work> <ak...@hacked.work>, user
<user@spark.apache.org> <user@spark.apache.org>
Subject:  Re: Where does the Driver run?

Hi Pat,

Indeed, I don't think that it's possible to use cluster mode w/o
spark-submit. All the docs I see appear to always describe needing to use
spark-submit for cluster mode -- it's not even compatible with spark-shell.
But it makes sense to me -- if you want Spark to run your application's
driver, you need to package it up and send it to the cluster manager. You
can't start spark one place and then later migrate it to the cluster. It's
also why you can't use spark-shell in cluster mode either, I think.

Cheers
Andrew

On Mon, Mar 25, 2019 at 11:22 AM Pat Ferrel <p...@occamsmachete.com> wrote:

> In the GUI while the job is running the app-id link brings up logs to both
> executors, The “name” link goes to 4040 of the machine that launched the
> job but is not resolvable right now so the page is not shown. I’ll try the
> netstat but the use of port 4040 was a good clue.
>
> By what you say below this indicates the Driver is running on the
> launching machine, the client to the Spark Cluster. This should be the case
> in deployMode = client.
>
> Can someone explain what us going on? The Evidence seems to say that
> deployMode = cluster *does not work* as described unless you use
> spark-submit (and I’m only guessing at that).
>
> Further; if we don’t use spark-submit we can’t use deployMode = cluster ???
>
>
> From: Akhil Das <ak...@hacked.work> <ak...@hacked.work>
> Reply: Akhil Das <ak...@hacked.work> <ak...@hacked.work>
> Date: March 24, 2019 at 7:45:07 PM
> To: Pat Ferrel <p...@occamsmachete.com> <p...@occamsmachete.com>
> Cc: user <user@spark.apache.org> <user@spark.apache.org>
> Subject:  Re: Where does the Driver run?
>
> There's also a driver ui (usually available on port 4040), after running
> your code, I assume you are running it on your machine, visit
> localhost:4040 and you will get the driver UI.
>
> If you think the driver is running on your master/executor nodes, login to
> those machines and do a
>
>    netstat -napt | grep -I listen
>
> You will see the driver listening on 404x there, this won't be the case
> mostly as you are not doing Spark-submit or using the deployMode=cluster.
>
> On Mon, 25 Mar 2019, 01:03 Pat Ferrel, <p...@occamsmachete.com> wrote:
>
>> Thanks, I have seen this many times in my research. Paraphrasing docs:
>> “in deployMode ‘cluster' the Driver runs on a Worker in the cluster”
>>
>> When I look at logs I see 2 executors on the 2 slaves (executor 0 and 1
>> with addresses that match slaves). When I look at memory usage while the
>> job runs I see virtually identical usage on the 2 Workers. This would
>> support your claim and contradict Spark docs for deployMode = cluster.
>>
>> The evidence seems to contradict the docs. I am now beginning to wonder
>> if the Driver only runs in the cluster if we use spark-submit????
>>
>>
>>
>> From: Akhil Das <ak...@hacked.work> <ak...@hacked.work>
>> Reply: Akhil Das <ak...@hacked.work> <ak...@hacked.work>
>> Date: March 23, 2019 at 9:26:50 PM
>> To: Pat Ferrel <p...@occamsmachete.com> <p...@occamsmachete.com>
>> Cc: user <user@spark.apache.org> <user@spark.apache.org>
>> Subject:  Re: Where does the Driver run?
>>
>> If you are starting your "my-app" on your local machine, that's where the
>> driver is running.
>>
>> [image: image.png]
>>
>> Hope this helps.
>> <https://spark.apache.org/docs/latest/cluster-overview.html>
>>
>> On Sun, Mar 24, 2019 at 4:13 AM Pat Ferrel <p...@occamsmachete.com> wrote:
>>
>>> I have researched this for a significant amount of time and find answers
>>> that seem to be for a slightly different question than mine.
>>>
>>> The Spark 2.3.3 cluster is running fine. I see the GUI on “
>>> http://master-address:8080";, there are 2 idle workers, as configured.
>>>
>>> I have a Scala application that creates a context and starts execution
>>> of a Job. I *do not use spark-submit*, I start the Job programmatically and
>>> this is where many explanations forks from my question.
>>>
>>> In "my-app" I create a new SparkConf, with the following code (slightly
>>> abbreviated):
>>>
>>>       conf.setAppName(“my-job")
>>>       conf.setMaster(“spark://master-address:7077”)
>>>       conf.set(“deployMode”, “cluster”)
>>>       // other settings like driver and executor memory requests
>>>       // the driver and executor memory requests are for all mem on the
>>> slaves, more than
>>>       // mem available on the launching machine with “my-app"
>>>       val jars = listJars(“/path/to/lib")
>>>       conf.setJars(jars)
>>>       …
>>>
>>> When I launch the job I see 2 executors running on the 2 workers/slaves.
>>> Everything seems to run fine and sometimes completes successfully. Frequent
>>> failures are the reason for this question.
>>>
>>> Where is the Driver running? I don’t see it in the GUI, I see 2
>>> Executors taking all cluster resources. With a Yarn cluster I would expect
>>> the “Driver" to run on/in the Yarn Master but I am using the Spark
>>> Standalone Master, where is the Drive part of the Job running?
>>>
>>> If is is running in the Master, we are in trouble because I start the
>>> Master on one of my 2 Workers sharing resources with one of the Executors.
>>> Executor mem + driver mem is > available mem on a Worker. I can change this
>>> but need so understand where the Driver part of the Spark Job runs. Is it
>>> in the Spark Master, or inside and Executor, or ???
>>>
>>> The “Driver” creates and broadcasts some large data structures so the
>>> need for an answer is more critical than with more typical tiny Drivers.
>>>
>>> Thanks for you help!
>>>
>>
>>
>> --
>> Cheers!
>>
>>

Reply via email to