From 872b4c20492d2fe52f24f7f69d66c3f3fba03136 Mon Sep 17 00:00:00 2001 From: Sahil Prasad Date: Wed, 23 Aug 2017 01:32:03 -0700 Subject: [PATCH 1/4] Enabling in-cluster client mode execution --- .../scala/org/apache/spark/deploy/SparkSubmit.scala | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index ed46adcbe9dfb..487a57e950008 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -344,8 +344,8 @@ object SparkSubmit extends CommandLineUtils { // The following modes are not supported or applicable (clusterManager, deployMode) match { - case (KUBERNETES, CLIENT) => - printErrorAndExit("Client mode is currently not supported for Kubernetes.") + case (KUBERNETES, CLIENT) if !inK8sCluster() => + printErrorAndExit("Kubernetes currently only supports in-cluster client mode.") case (KUBERNETES, CLUSTER) if args.isR => printErrorAndExit("Kubernetes does not currently support R applications.") case (STANDALONE, CLUSTER) if args.isPython => @@ -856,6 +856,14 @@ object SparkSubmit extends CommandLineUtils { res == SparkLauncher.NO_RESOURCE } + /** + * Return whether the submission environment is within a Kubernetes cluster + */ + private[deploy] def inK8sCluster(): Boolean = { + !sys.env.get("KUBERNETES_SERVICE_HOST").isEmpty && + !sys.env.get("KUBERNETES_SERVICE_PORT").isEmpty + } + /** * Merge a sequence of comma-separated file lists, some of which may be null to indicate * no files, into a single comma-separated string. From 2fba4868c49dc4a8aa54a74c3637407658497736 Mon Sep 17 00:00:00 2001 From: Sahil Prasad Date: Wed, 23 Aug 2017 14:09:41 -0700 Subject: [PATCH 2/4] Usage docs --- docs/running-on-kubernetes.md | 49 ++++++++++++++++++++++++++++------- 1 file changed, 40 insertions(+), 9 deletions(-) diff --git a/docs/running-on-kubernetes.md b/docs/running-on-kubernetes.md index 4286ab19eb3ad..185c3fbcb5318 100644 --- a/docs/running-on-kubernetes.md +++ b/docs/running-on-kubernetes.md @@ -65,7 +65,7 @@ For example, if the registry host is `registry-host` and the registry is listeni docker push registry-host:5000/spark-driver:latest docker push registry-host:5000/spark-executor:latest docker push registry-host:5000/spark-init:latest - + Note that `spark-base` is the base image for the other images. It must be built first before the other images, and then afterwards the other images can be built in any order. ## Submitting Applications to Kubernetes @@ -182,10 +182,10 @@ is currently supported. ### Running PySpark -Running PySpark on Kubernetes leverages the same spark-submit logic when launching on Yarn and Mesos. -Python files can be distributed by including, in the conf, `--py-files` +Running PySpark on Kubernetes leverages the same spark-submit logic when launching on Yarn and Mesos. +Python files can be distributed by including, in the conf, `--py-files` -Below is an example submission: +Below is an example submission: ``` @@ -240,6 +240,37 @@ the command may then look like the following: ## Advanced +### Running in-cluster client mode applications + +While Spark on Kubernetes does not officially support client mode applications, such as the PySpark shell, there is a workaround that +allows for execution of these apps from within an existing Kubernetes cluster. This _in-cluster_ client mode bypasses some of the networking and +dependency issues inherent to running a client from outside of a cluster while allowing much of the same functionality in terms of interactive use cases. + +In order to run in client mode, use `kubectl attach` to attach to an existing driver pod on the cluster, or the following to run a new driver: + + kubectl run -it --image= --restart=Never -- /bin/bash + +This will open up a shell into the specified driver pod from which you can run client mode applications. In order to appropriately configure +these in-cluster applications, be sure to set the following configuration value, which essentially tells the cluster manager to refer back to the current driver pod as the driver for any applications you submit: + + spark.kubernetes.driver.pod.name=$HOSTNAME + +With that set, you should be able to run the following example from within the pod: + + bin/spark-submit \ + --class org.apache.spark.examples.SparkPi \ + --master k8s://$KUBERNETES_SERVICE_HOST:$KUBERNETES_SERVICE_PORT \ + --kubernetes-namespace default \ + --conf spark.app.name=spark-pi \ + --conf spark.kubernetes.driver.pod.name=$HOSTNAME \ + --conf spark.kubernetes.driver.docker.image=kubespark/spark-driver:latest \ + --conf spark.kubernetes.executor.docker.image=kubespark/spark-executor:latest \ + --conf spark.dynamicAllocation.enabled=true \ + --conf spark.shuffle.service.enabled=true \ + --conf spark.kubernetes.shuffle.namespace=default \ + --conf spark.kubernetes.shuffle.labels="app=spark-shuffle-service,spark-version=2.1.0" \ + local:///opt/spark/examples/jars/spark_examples_2.11-2.2.0.jar 10 + ### Securing the Resource Staging Server with TLS The default configuration of the resource staging server is not secured with TLS. It is highly recommended to configure @@ -759,17 +790,17 @@ from the other deployment modes. See the [configuration page](configuration.html - spark.kubernetes.node.selector.[labelKey] + spark.kubernetes.node.selector.[labelKey] (none) - Adds to the node selector of the driver pod and executor pods, with key labelKey and the value as the + Adds to the node selector of the driver pod and executor pods, with key labelKey and the value as the configuration's value. For example, setting spark.kubernetes.node.selector.identifier to myIdentifier - will result in the driver pod and executors having a node selector with key identifier and value + will result in the driver pod and executors having a node selector with key identifier and value myIdentifier. Multiple node selector keys can be added by setting multiple configurations with this prefix. - spark.executorEnv.[EnvironmentVariableName] + spark.executorEnv.[EnvironmentVariableName] (none) Add the environment variable specified by EnvironmentVariableName to @@ -777,7 +808,7 @@ from the other deployment modes. See the [configuration page](configuration.html - spark.kubernetes.driverEnv.[EnvironmentVariableName] + spark.kubernetes.driverEnv.[EnvironmentVariableName] (none) Add the environment variable specified by EnvironmentVariableName to From acca24a00d87c9927247313584540a9e28ed5f11 Mon Sep 17 00:00:00 2001 From: Sahil Prasad Date: Thu, 24 Aug 2017 16:10:38 -0700 Subject: [PATCH 3/4] Rewording in-cluster client mode docs --- docs/running-on-kubernetes.md | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/docs/running-on-kubernetes.md b/docs/running-on-kubernetes.md index 185c3fbcb5318..5e0de5811cfc2 100644 --- a/docs/running-on-kubernetes.md +++ b/docs/running-on-kubernetes.md @@ -251,7 +251,8 @@ In order to run in client mode, use `kubectl attach` to attach to an existing dr kubectl run -it --image= --restart=Never -- /bin/bash This will open up a shell into the specified driver pod from which you can run client mode applications. In order to appropriately configure -these in-cluster applications, be sure to set the following configuration value, which essentially tells the cluster manager to refer back to the current driver pod as the driver for any applications you submit: +these in-cluster applications, be sure to set the following configuration value for all applications, as in the following `spark-submit` example, +which essentially tells the cluster manager to refer back to the current driver pod as the driver for any applications you submit: spark.kubernetes.driver.pod.name=$HOSTNAME From 08b5433e942c9adc733a7933f1f2d25f45f06961 Mon Sep 17 00:00:00 2001 From: Sahil Prasad Date: Mon, 28 Aug 2017 18:37:11 -0700 Subject: [PATCH 4/4] Incorporating feedback into in-cluster docs --- docs/running-on-kubernetes.md | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/docs/running-on-kubernetes.md b/docs/running-on-kubernetes.md index 4dbbccbf0b421..4dad93186956b 100644 --- a/docs/running-on-kubernetes.md +++ b/docs/running-on-kubernetes.md @@ -256,23 +256,22 @@ the command may then look like the following: ### Running in-cluster client mode applications -While Spark on Kubernetes does not officially support client mode applications, such as the PySpark shell, there is a workaround that -allows for execution of these apps from within an existing Kubernetes cluster. This _in-cluster_ client mode bypasses some of the networking and -dependency issues inherent to running a client from outside of a cluster while allowing much of the same functionality in terms of interactive use cases. +While Spark on Kubernetes does not support client mode applications, such as the PySpark shell, when launched from outside Kubernetes, Spark on Kubernetes does support client mode applications launched from within the cluster. This _in-cluster_ client mode bypasses some of the networking and dependency issues inherent to running a client from outside of a cluster while allowing much of the same functionality in terms of interactive use cases, such as the PySpark shell and Jupyter notebooks. In order to run in client mode, use `kubectl attach` to attach to an existing driver pod on the cluster, or the following to run a new driver: kubectl run -it --image= --restart=Never -- /bin/bash This will open up a shell into the specified driver pod from which you can run client mode applications. In order to appropriately configure -these in-cluster applications, be sure to set the following configuration value for all applications, as in the following `spark-submit` example, -which essentially tells the cluster manager to refer back to the current driver pod as the driver for any applications you submit: +these in-cluster applications, be sure to set the following configuration value for all applications, as in the following `spark-submit` example, +which tells the cluster manager to refer back to the current driver pod as the driver for any applications you submit: spark.kubernetes.driver.pod.name=$HOSTNAME With that set, you should be able to run the following example from within the pod: bin/spark-submit \ + --deploy-mode client \ --class org.apache.spark.examples.SparkPi \ --master k8s://$KUBERNETES_SERVICE_HOST:$KUBERNETES_SERVICE_PORT \ --kubernetes-namespace default \