-
Notifications
You must be signed in to change notification settings - Fork 118
Changing submission logic to accommodate Spark shell applications #402
Changing submission logic to accommodate Spark shell applications #402
Conversation
Can you post some example instructions (and the jupyter notebook image you're using) on this PR discussion? |
Can you expand on what you mean by "tested pyspark and scala shells"? |
@erikerlandson Here's a link to doc I put together a few days ago. Regarding the shells, I ran a small in-cluster job with the |
@sahilprasad does spark-shell also work from outside the cluster? |
@erikerlandson no, when I try running spark-shell from outside the cluster, I get errors regarding missing service account tokens and executors not being able to find the driver. |
My main concern is that enabling spark-shell doesn't distinguish between running inside the cluster, where it works, and outside, where it won't. However I like the jupyter notebook capability. A possible compromise is implement some kind of detection to distinguish these cases, so an informative error can be thrown if it is executed external to a cluster. |
@erikerlandson I agree that the compromise is the way to go. Would it be preferable to automatically detect in-cluster execution, perhaps by checking for the |
I'm thinking auto detection - if somebody tries to run it from the outside, it should automatically fail on them, and explain why |
I don't want to close this, but we are going to hold off from including it in the initial 2.2 release |
@erikerlandson Sounds good. I'll push something in the next few days for review. |
How does this resolve? Does the submitting shell still eventually call |
@mccheah It eventually calls |
If we're calling the submission client then that implies we're not running in client mode. Client mode runs the scheduler backend and the user application directly in the process that runs spark-submit. |
@mccheah Then is there any way to get around the submission client to achieve the sort of pseudo client mode that I need for Jupyter? The in-cluster use case is all I'm going for, so any advice you have on this front, or towards functionality closer to true client mode, would be awesome! |
Can the submission client just be run inside the cluster in cluster mode? I'm not entirely certain why the submission client has to be run inside a docker container. Another option is just to support client mode in a sense - that's just a matter of getting the Java/Scala code to instantiate a |
@sahilprasad, have you tried jupyterhub and https://github.com/jupyterhub/kubespawner? |
@mccheah I'm not aware of a way to run PySpark applications inside of a Jupyter notebook without client deploy mode. Running the submission client in-cluster is more of a workaround to avoid the sort of networking and dependency resolution issues a remote, or out-of-cluster, client mode would involve. It seems to me that if shell applications work in-cluster and as-is, this would be worth supporting. @foxish Yeah! Although I don't quite remember the end result, I experimented with those specifically with this project a few weeks ago, and was able to get a similar outcome in that I was able to run a PySpark application on a proxied Jupyter notebook being run on a k8s cluster. Successfully running it did involve allowing client mode through — exactly the changes of this PR. |
I think we want to modify
|
The basis of the client mode submission option is that there isn't an intermediate client that spawns another driver, but that the process running the |
@mccheah Got it. Would we need something equivalent to a |
It would be good to look into writing a |
@mccheah Sounds good. I'll model it after the YARN scheduler backends. Should we continue discussion and review on this PR, or close it off in favor of a more poignant PR? |
We should close this and open another one with the more canonical approach. |
Will open a WIP PR when ready. |
Allows client and cluster mode for shell applications. Specifically, this was done to allow executing a Jupyter notebook server that is able to interact with the k8s cluster.
With this change, I was able to use
kubectl run
with an image that runs a Jupyter notebook with thejupyter notebook
command. After port-forwarding to the container port that the server is running on, I can access and use the notebook as usual, and if I provide appropriate configuration values (spark.master
,spark.kubernetes.driver.docker.image
, etc.), Spark tasks interact as expected with the Kubernetes cluster, and dynamic allocation behaves as expected.Along with Jupyter, I've tested and confirmed that the PySpark and Scala shells work, and with slight modifications, so does in-cluster spark-submit.
This PR is less intended for merging and more for figuring out how best to facilitate this in-cluster workflow with Jupyter and client-mode applications. I know that there have been previous discussions around this, such as #211, and that there are limitations (executor IPs must be routable and other networking issues) but I would love to hear any thoughts around alternative approaches or solutions!