From 571a6f0574e50e53cea403624ec3795cd03aa204 Mon Sep 17 00:00:00 2001 From: mcheah Date: Wed, 25 Jul 2018 11:08:41 -0700 Subject: [PATCH] [SPARK-23146][K8S] Support client mode. ## What changes were proposed in this pull request? Support client mode for the Kubernetes scheduler. Client mode works more or less identically to cluster mode. However, in client mode, the Spark Context needs to be manually bootstrapped with certain properties which would have otherwise been set up by spark-submit in cluster mode. Specifically: - If the user doesn't provide a driver pod name, we don't add an owner reference. This is for usage when the driver is not running in a pod in the cluster. In such a case, the driver can only provide a best effort to clean up the executors when the driver exits, but cleaning up the resources is not guaranteed. The executor JVMs should exit if the driver JVM exits, but the pods will still remain in the cluster in a COMPLETED or FAILED state. - The user must provide a host (spark.driver.host) and port (spark.driver.port) that the executors can connect to. When using spark-submit in cluster mode, spark-submit generates the headless service automatically; in client mode, the user is responsible for setting up their own connectivity. We also change the authentication configuration prefixes for client mode. ## How was this patch tested? Adding an integration test to exercise client mode support. Author: mcheah Closes #21748 from mccheah/k8s-client-mode. --- docs/running-on-kubernetes.md | 138 ++++++++++++++---- .../org/apache/spark/deploy/k8s/Config.scala | 3 +- .../spark/deploy/k8s/KubernetesConf.scala | 4 +- .../spark/deploy/k8s/KubernetesUtils.scala | 2 + .../features/BasicExecutorFeatureStep.scala | 17 ++- .../submit/KubernetesClientApplication.scala | 4 +- .../cluster/k8s/ExecutorPodsAllocator.scala | 13 +- .../k8s/KubernetesClusterManager.scala | 35 +++-- .../deploy/k8s/KubernetesConfSuite.scala | 8 +- .../BasicExecutorFeatureStepSuite.scala | 6 +- .../features/EnvSecretsFeatureStepSuite.scala | 2 +- .../MountSecretsFeatureStepSuite.scala | 2 +- .../k8s/ExecutorPodsAllocatorSuite.scala | 2 +- .../k8s/KubernetesExecutorBuilderSuite.scala | 6 +- .../src/main/dockerfiles/spark/Dockerfile | 2 +- .../ClientModeTestsSuite.scala | 111 ++++++++++++++ .../k8s/integrationtest/KubernetesSuite.scala | 11 +- 17 files changed, 290 insertions(+), 76 deletions(-) create mode 100644 resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/ClientModeTestsSuite.scala diff --git a/docs/running-on-kubernetes.md b/docs/running-on-kubernetes.md index 7149616e534aa..97c650d0f80aa 100644 --- a/docs/running-on-kubernetes.md +++ b/docs/running-on-kubernetes.md @@ -117,6 +117,45 @@ If the local proxy is running at localhost:8001, `--master k8s://http://127.0.0. spark-submit. Finally, notice that in the above example we specify a jar with a specific URI with a scheme of `local://`. This URI is the location of the example jar that is already in the Docker image. +## Client Mode + +Starting with Spark 2.4.0, it is possible to run Spark applications on Kubernetes in client mode. When your application +runs in client mode, the driver can run inside a pod or on a physical host. When running an application in client mode, +it is recommended to account for the following factors: + +### Client Mode Networking + +Spark executors must be able to connect to the Spark driver over a hostname and a port that is routable from the Spark +executors. The specific network configuration that will be required for Spark to work in client mode will vary per +setup. If you run your driver inside a Kubernetes pod, you can use a +[headless service](https://kubernetes.io/docs/concepts/services-networking/service/#headless-services) to allow your +driver pod to be routable from the executors by a stable hostname. When deploying your headless service, ensure that +the service's label selector will only match the driver pod and no other pods; it is recommended to assign your driver +pod a sufficiently unique label and to use that label in the label selector of the headless service. Specify the driver's +hostname via `spark.driver.host` and your spark driver's port to `spark.driver.port`. + +### Client Mode Executor Pod Garbage Collection + +If you run your Spark driver in a pod, it is highly recommended to set `spark.driver.pod.name` to the name of that pod. +When this property is set, the Spark scheduler will deploy the executor pods with an +[OwnerReference](https://kubernetes.io/docs/concepts/workloads/controllers/garbage-collection/), which in turn will +ensure that once the driver pod is deleted from the cluster, all of the application's executor pods will also be deleted. +The driver will look for a pod with the given name in the namespace specified by `spark.kubernetes.namespace`, and +an OwnerReference pointing to that pod will be added to each executor pod's OwnerReferences list. Be careful to avoid +setting the OwnerReference to a pod that is not actually that driver pod, or else the executors may be terminated +prematurely when the wrong pod is deleted. + +If your application is not running inside a pod, or if `spark.driver.pod.name` is not set when your application is +actually running in a pod, keep in mind that the executor pods may not be properly deleted from the cluster when the +application exits. The Spark scheduler attempts to delete these pods, but if the network request to the API server fails +for any reason, these pods will remain in the cluster. The executor processes should exit when they cannot reach the +driver, so the executor pods should not consume compute resources (cpu and memory) in the cluster after your application +exits. + +### Authentication Parameters + +Use the exact prefix `spark.kubernetes.authenticate` for Kubernetes authentication parameters in client mode. + ## Dependency Management If your application's dependencies are all hosted in remote locations like HDFS or HTTP servers, they may be referred to @@ -258,10 +297,6 @@ RBAC authorization and how to configure Kubernetes service accounts for pods, pl [Using RBAC Authorization](https://kubernetes.io/docs/admin/authorization/rbac/) and [Configure Service Accounts for Pods](https://kubernetes.io/docs/tasks/configure-pod-container/configure-service-account/). -## Client Mode - -Client mode is not currently supported. - ## Future Work There are several Spark on Kubernetes features that are currently being incubated in a fork - @@ -354,7 +389,7 @@ specific to Spark on Kubernetes. Path to the CA cert file for connecting to the Kubernetes API server over TLS when starting the driver. This file must be located on the submitting machine's disk. Specify this as a path as opposed to a URI (i.e. do not provide - a scheme). + a scheme). In client mode, use spark.kubernetes.authenticate.caCertFile instead. @@ -363,7 +398,7 @@ specific to Spark on Kubernetes. Path to the client key file for authenticating against the Kubernetes API server when starting the driver. This file must be located on the submitting machine's disk. Specify this as a path as opposed to a URI (i.e. do not provide - a scheme). + a scheme). In client mode, use spark.kubernetes.authenticate.clientKeyFile instead. @@ -372,7 +407,7 @@ specific to Spark on Kubernetes. Path to the client cert file for authenticating against the Kubernetes API server when starting the driver. This file must be located on the submitting machine's disk. Specify this as a path as opposed to a URI (i.e. do not - provide a scheme). + provide a scheme). In client mode, use spark.kubernetes.authenticate.clientCertFile instead. @@ -381,7 +416,7 @@ specific to Spark on Kubernetes. OAuth token to use when authenticating against the Kubernetes API server when starting the driver. Note that unlike the other authentication options, this is expected to be the exact string value of the token to use for - the authentication. + the authentication. In client mode, use spark.kubernetes.authenticate.oauthToken instead. @@ -390,7 +425,7 @@ specific to Spark on Kubernetes. Path to the OAuth token file containing the token to use when authenticating against the Kubernetes API server when starting the driver. This file must be located on the submitting machine's disk. Specify this as a path as opposed to a URI (i.e. do not - provide a scheme). + provide a scheme). In client mode, use spark.kubernetes.authenticate.oauthTokenFile instead. @@ -399,7 +434,8 @@ specific to Spark on Kubernetes. Path to the CA cert file for connecting to the Kubernetes API server over TLS from the driver pod when requesting executors. This file must be located on the submitting machine's disk, and will be uploaded to the driver pod. - Specify this as a path as opposed to a URI (i.e. do not provide a scheme). + Specify this as a path as opposed to a URI (i.e. do not provide a scheme). In client mode, use + spark.kubernetes.authenticate.caCertFile instead. @@ -407,10 +443,9 @@ specific to Spark on Kubernetes. (none) Path to the client key file for authenticating against the Kubernetes API server from the driver pod when requesting - executors. This file must be located on the submitting machine's disk, and will be uploaded to the driver pod. - Specify this as a path as opposed to a URI (i.e. do not provide a scheme). If this is specified, it is highly - recommended to set up TLS for the driver submission server, as this value is sensitive information that would be - passed to the driver pod in plaintext otherwise. + executors. This file must be located on the submitting machine's disk, and will be uploaded to the driver pod as + a Kubernetes secret. Specify this as a path as opposed to a URI (i.e. do not provide a scheme). + In client mode, use spark.kubernetes.authenticate.clientKeyFile instead. @@ -419,7 +454,8 @@ specific to Spark on Kubernetes. Path to the client cert file for authenticating against the Kubernetes API server from the driver pod when requesting executors. This file must be located on the submitting machine's disk, and will be uploaded to the - driver pod. Specify this as a path as opposed to a URI (i.e. do not provide a scheme). + driver pod as a Kubernetes secret. Specify this as a path as opposed to a URI (i.e. do not provide a scheme). + In client mode, use spark.kubernetes.authenticate.clientCertFile instead. @@ -428,9 +464,8 @@ specific to Spark on Kubernetes. OAuth token to use when authenticating against the Kubernetes API server from the driver pod when requesting executors. Note that unlike the other authentication options, this must be the exact string value of - the token to use for the authentication. This token value is uploaded to the driver pod. If this is specified, it is - highly recommended to set up TLS for the driver submission server, as this value is sensitive information that would - be passed to the driver pod in plaintext otherwise. + the token to use for the authentication. This token value is uploaded to the driver pod as a Kubernetes secret. + In client mode, use spark.kubernetes.authenticate.oauthToken instead. @@ -439,9 +474,8 @@ specific to Spark on Kubernetes. Path to the OAuth token file containing the token to use when authenticating against the Kubernetes API server from the driver pod when requesting executors. Note that unlike the other authentication options, this file must contain the exact string value of - the token to use for the authentication. This token value is uploaded to the driver pod. If this is specified, it is - highly recommended to set up TLS for the driver submission server, as this value is sensitive information that would - be passed to the driver pod in plaintext otherwise. + the token to use for the authentication. This token value is uploaded to the driver pod as a secret. In client mode, use + spark.kubernetes.authenticate.oauthTokenFile instead. @@ -450,7 +484,8 @@ specific to Spark on Kubernetes. Path to the CA cert file for connecting to the Kubernetes API server over TLS from the driver pod when requesting executors. This path must be accessible from the driver pod. - Specify this as a path as opposed to a URI (i.e. do not provide a scheme). + Specify this as a path as opposed to a URI (i.e. do not provide a scheme). In client mode, use + spark.kubernetes.authenticate.caCertFile instead. @@ -459,7 +494,8 @@ specific to Spark on Kubernetes. Path to the client key file for authenticating against the Kubernetes API server from the driver pod when requesting executors. This path must be accessible from the driver pod. - Specify this as a path as opposed to a URI (i.e. do not provide a scheme). + Specify this as a path as opposed to a URI (i.e. do not provide a scheme). In client mode, use + spark.kubernetes.authenticate.clientKeyFile instead. @@ -468,7 +504,8 @@ specific to Spark on Kubernetes. Path to the client cert file for authenticating against the Kubernetes API server from the driver pod when requesting executors. This path must be accessible from the driver pod. - Specify this as a path as opposed to a URI (i.e. do not provide a scheme). + Specify this as a path as opposed to a URI (i.e. do not provide a scheme). In client mode, use + spark.kubernetes.authenticate.clientCertFile instead. @@ -477,7 +514,8 @@ specific to Spark on Kubernetes. Path to the file containing the OAuth token to use when authenticating against the Kubernetes API server from the driver pod when requesting executors. This path must be accessible from the driver pod. - Note that unlike the other authentication options, this file must contain the exact string value of the token to use for the authentication. + Note that unlike the other authentication options, this file must contain the exact string value of the token to use + for the authentication. In client mode, use spark.kubernetes.authenticate.oauthTokenFile instead. @@ -486,7 +524,48 @@ specific to Spark on Kubernetes. Service account that is used when running the driver pod. The driver pod uses this service account when requesting executor pods from the API server. Note that this cannot be specified alongside a CA cert file, client key file, - client cert file, and/or OAuth token. + client cert file, and/or OAuth token. In client mode, use spark.kubernetes.authenticate.serviceAccountName instead. + + + + spark.kubernetes.authenticate.caCertFile + (none) + + In client mode, path to the CA cert file for connecting to the Kubernetes API server over TLS when + requesting executors. Specify this as a path as opposed to a URI (i.e. do not provide a scheme). + + + + spark.kubernetes.authenticate.clientKeyFile + (none) + + In client mode, path to the client key file for authenticating against the Kubernetes API server + when requesting executors. Specify this as a path as opposed to a URI (i.e. do not provide a scheme). + + + + spark.kubernetes.authenticate.clientCertFile + (none) + + In client mode, path to the client cert file for authenticating against the Kubernetes API server + when requesting executors. Specify this as a path as opposed to a URI (i.e. do not provide a scheme). + + + + spark.kubernetes.authenticate.oauthToken + (none) + + In client mode, the OAuth token to use when authenticating against the Kubernetes API server when + requesting executors. Note that unlike the other authentication options, this must be the exact string value of + the token to use for the authentication. + + + + spark.kubernetes.authenticate.oauthTokenFile + (none) + + In client mode, path to the file containing the OAuth token to use when authenticating against the Kubernetes API + server when requesting executors. @@ -529,8 +608,11 @@ specific to Spark on Kubernetes. spark.kubernetes.driver.pod.name (none) - Name of the driver pod. If not set, the driver pod name is set to "spark.app.name" suffixed by the current timestamp - to avoid name conflicts. + Name of the driver pod. In cluster mode, if this is not set, the driver pod name is set to "spark.app.name" + suffixed by the current timestamp to avoid name conflicts. In client mode, if your application is running + inside a pod, it is highly recommended to set this to the name of the pod your driver is running in. Setting this + value in client mode allows the driver to become the owner of its executor pods, which in turn allows the executor + pods to be garbage collected by the cluster. diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala index f9a77e71ad618..968679df60367 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala @@ -65,6 +65,7 @@ private[spark] object Config extends Logging { "spark.kubernetes.authenticate.driver" val KUBERNETES_AUTH_DRIVER_MOUNTED_CONF_PREFIX = "spark.kubernetes.authenticate.driver.mounted" + val KUBERNETES_AUTH_CLIENT_MODE_PREFIX = "spark.kubernetes.authenticate" val OAUTH_TOKEN_CONF_SUFFIX = "oauthToken" val OAUTH_TOKEN_FILE_CONF_SUFFIX = "oauthTokenFile" val CLIENT_KEY_FILE_CONF_SUFFIX = "clientKeyFile" @@ -90,7 +91,7 @@ private[spark] object Config extends Logging { ConfigBuilder("spark.kubernetes.submitInDriver") .internal() .booleanConf - .createOptional + .createWithDefault(false) val KUBERNETES_EXECUTOR_LIMIT_CORES = ConfigBuilder("spark.kubernetes.executor.limit.cores") diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesConf.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesConf.scala index 51d205fdb68d1..866ba3cbaa9c3 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesConf.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesConf.scala @@ -43,7 +43,7 @@ private[spark] case class KubernetesDriverSpecificConf( */ private[spark] case class KubernetesExecutorSpecificConf( executorId: String, - driverPod: Pod) + driverPod: Option[Pod]) extends KubernetesRoleSpecificConf /** @@ -186,7 +186,7 @@ private[spark] object KubernetesConf { sparkConf: SparkConf, executorId: String, appId: String, - driverPod: Pod): KubernetesConf[KubernetesExecutorSpecificConf] = { + driverPod: Option[Pod]): KubernetesConf[KubernetesExecutorSpecificConf] = { val executorCustomLabels = KubernetesUtils.parsePrefixedKeyValuePairs( sparkConf, KUBERNETES_EXECUTOR_LABEL_PREFIX) require( diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesUtils.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesUtils.scala index 66fff267545dc..588cd9d40f9a0 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesUtils.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesUtils.scala @@ -58,4 +58,6 @@ private[spark] object KubernetesUtils { case _ => uri } } + + def parseMasterUrl(url: String): String = url.substring("k8s://".length) } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStep.scala index abaeff0313a79..c37f713c56de1 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStep.scala @@ -152,19 +152,20 @@ private[spark] class BasicExecutorFeatureStep( .build() }.getOrElse(executorContainer) val driverPod = kubernetesConf.roleSpecificConf.driverPod + val ownerReference = driverPod.map(pod => + new OwnerReferenceBuilder() + .withController(true) + .withApiVersion(pod.getApiVersion) + .withKind(pod.getKind) + .withName(pod.getMetadata.getName) + .withUid(pod.getMetadata.getUid) + .build()) val executorPod = new PodBuilder(pod.pod) .editOrNewMetadata() .withName(name) .withLabels(kubernetesConf.roleLabels.asJava) .withAnnotations(kubernetesConf.roleAnnotations.asJava) - .withOwnerReferences() - .addNewOwnerReference() - .withController(true) - .withApiVersion(driverPod.getApiVersion) - .withKind(driverPod.getKind) - .withName(driverPod.getMetadata.getName) - .withUid(driverPod.getMetadata.getUid) - .endOwnerReference() + .addToOwnerReferences(ownerReference.toSeq: _*) .endMetadata() .editOrNewSpec() .withHostname(hostname) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesClientApplication.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesClientApplication.scala index eaff47205dbbc..9398faee2ea5c 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesClientApplication.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesClientApplication.scala @@ -27,7 +27,7 @@ import scala.util.control.NonFatal import org.apache.spark.SparkConf import org.apache.spark.deploy.SparkApplication -import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesDriverSpecificConf, SparkKubernetesClientFactory} +import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesDriverSpecificConf, KubernetesUtils, SparkKubernetesClientFactory} import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.internal.Logging @@ -228,7 +228,7 @@ private[spark] class KubernetesClientApplication extends SparkApplication { val namespace = kubernetesConf.namespace() // The master URL has been checked for validity already in SparkSubmit. // We just need to get rid of the "k8s://" prefix here. - val master = sparkConf.get("spark.master").substring("k8s://".length) + val master = KubernetesUtils.parseMasterUrl(sparkConf.get("spark.master")) val loggingInterval = if (waitForAppCompletion) Some(sparkConf.get(REPORT_INTERVAL)) else None val watcher = new LoggingPodStatusWatcherImpl(kubernetesAppId, loggingInterval) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala index 5a143ad3600fd..77bb9c3fcc9f4 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala @@ -46,13 +46,18 @@ private[spark] class ExecutorPodsAllocator( private val podCreationTimeout = math.max(podAllocationDelay * 5, 60000) + private val namespace = conf.get(KUBERNETES_NAMESPACE) + private val kubernetesDriverPodName = conf .get(KUBERNETES_DRIVER_POD_NAME) - .getOrElse(throw new SparkException("Must specify the driver pod name")) - private val driverPod = kubernetesClient.pods() - .withName(kubernetesDriverPodName) - .get() + private val driverPod = kubernetesDriverPodName + .map(name => Option(kubernetesClient.pods() + .withName(name) + .get()) + .getOrElse(throw new SparkException( + s"No pod was found named $kubernetesDriverPodName in the cluster in the " + + s"namespace $namespace (this was supposed to be the driver pod.)."))) // Executor IDs that have been requested from Kubernetes but have not been detected in any // snapshot yet. Mapped to the timestamp when they were created. diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala index de2a52bc7a0b8..9999c62c878df 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala @@ -22,7 +22,7 @@ import java.util.concurrent.TimeUnit import com.google.common.cache.CacheBuilder import io.fabric8.kubernetes.client.Config -import org.apache.spark.{SparkContext, SparkException} +import org.apache.spark.SparkContext import org.apache.spark.deploy.k8s.{KubernetesUtils, SparkKubernetesClientFactory} import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ @@ -35,12 +35,6 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit override def canCreate(masterURL: String): Boolean = masterURL.startsWith("k8s") override def createTaskScheduler(sc: SparkContext, masterURL: String): TaskScheduler = { - if (masterURL.startsWith("k8s") && - sc.deployMode == "client" && - !sc.conf.get(KUBERNETES_DRIVER_SUBMIT_CHECK).getOrElse(false)) { - throw new SparkException("Client mode is currently not supported for Kubernetes.") - } - new TaskSchedulerImpl(sc) } @@ -48,13 +42,32 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit sc: SparkContext, masterURL: String, scheduler: TaskScheduler): SchedulerBackend = { + val wasSparkSubmittedInClusterMode = sc.conf.get(KUBERNETES_DRIVER_SUBMIT_CHECK) + val (authConfPrefix, + apiServerUri, + defaultServiceAccountToken, + defaultServiceAccountCaCrt) = if (wasSparkSubmittedInClusterMode) { + require(sc.conf.get(KUBERNETES_DRIVER_POD_NAME).isDefined, + "If the application is deployed using spark-submit in cluster mode, the driver pod name " + + "must be provided.") + (KUBERNETES_AUTH_DRIVER_MOUNTED_CONF_PREFIX, + KUBERNETES_MASTER_INTERNAL_URL, + Some(new File(Config.KUBERNETES_SERVICE_ACCOUNT_TOKEN_PATH)), + Some(new File(Config.KUBERNETES_SERVICE_ACCOUNT_CA_CRT_PATH))) + } else { + (KUBERNETES_AUTH_CLIENT_MODE_PREFIX, + KubernetesUtils.parseMasterUrl(masterURL), + None, + None) + } + val kubernetesClient = SparkKubernetesClientFactory.createKubernetesClient( - KUBERNETES_MASTER_INTERNAL_URL, + apiServerUri, Some(sc.conf.get(KUBERNETES_NAMESPACE)), - KUBERNETES_AUTH_DRIVER_MOUNTED_CONF_PREFIX, + authConfPrefix, sc.conf, - Some(new File(Config.KUBERNETES_SERVICE_ACCOUNT_TOKEN_PATH)), - Some(new File(Config.KUBERNETES_SERVICE_ACCOUNT_CA_CRT_PATH))) + defaultServiceAccountToken, + defaultServiceAccountCaCrt) val requestExecutorsService = ThreadUtils.newDaemonCachedThreadPool( "kubernetes-executor-requests") diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/KubernetesConfSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/KubernetesConfSuite.scala index 661f942435921..ecdb71359c5bb 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/KubernetesConfSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/KubernetesConfSuite.scala @@ -184,9 +184,9 @@ class KubernetesConfSuite extends SparkFunSuite { new SparkConf(false), EXECUTOR_ID, APP_ID, - DRIVER_POD) + Some(DRIVER_POD)) assert(conf.roleSpecificConf.executorId === EXECUTOR_ID) - assert(conf.roleSpecificConf.driverPod === DRIVER_POD) + assert(conf.roleSpecificConf.driverPod.get === DRIVER_POD) } test("Image pull secrets.") { @@ -195,7 +195,7 @@ class KubernetesConfSuite extends SparkFunSuite { .set(IMAGE_PULL_SECRETS, "my-secret-1,my-secret-2 "), EXECUTOR_ID, APP_ID, - DRIVER_POD) + Some(DRIVER_POD)) assert(conf.imagePullSecrets() === Seq( new LocalObjectReferenceBuilder().withName("my-secret-1").build(), @@ -221,7 +221,7 @@ class KubernetesConfSuite extends SparkFunSuite { sparkConf, EXECUTOR_ID, APP_ID, - DRIVER_POD) + Some(DRIVER_POD)) assert(conf.roleLabels === Map( SPARK_EXECUTOR_ID_LABEL -> EXECUTOR_ID, SPARK_APP_ID_LABEL -> APP_ID, diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStepSuite.scala index a44fa1f2ffc63..95d373f791649 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStepSuite.scala @@ -81,7 +81,7 @@ class BasicExecutorFeatureStepSuite val step = new BasicExecutorFeatureStep( KubernetesConf( baseConf, - KubernetesExecutorSpecificConf("1", DRIVER_POD), + KubernetesExecutorSpecificConf("1", Some(DRIVER_POD)), RESOURCE_NAME_PREFIX, APP_ID, LABELS, @@ -121,7 +121,7 @@ class BasicExecutorFeatureStepSuite val step = new BasicExecutorFeatureStep( KubernetesConf( conf, - KubernetesExecutorSpecificConf("1", DRIVER_POD), + KubernetesExecutorSpecificConf("1", Some(DRIVER_POD)), longPodNamePrefix, APP_ID, LABELS, @@ -142,7 +142,7 @@ class BasicExecutorFeatureStepSuite val step = new BasicExecutorFeatureStep( KubernetesConf( conf, - KubernetesExecutorSpecificConf("1", DRIVER_POD), + KubernetesExecutorSpecificConf("1", Some(DRIVER_POD)), RESOURCE_NAME_PREFIX, APP_ID, LABELS, diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/EnvSecretsFeatureStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/EnvSecretsFeatureStepSuite.scala index 1c8d84b76c56b..85c6cb282d2b0 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/EnvSecretsFeatureStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/EnvSecretsFeatureStepSuite.scala @@ -37,7 +37,7 @@ class EnvSecretsFeatureStepSuite extends SparkFunSuite{ val sparkConf = new SparkConf(false) val kubernetesConf = KubernetesConf( sparkConf, - KubernetesExecutorSpecificConf("1", new PodBuilder().build()), + KubernetesExecutorSpecificConf("1", Some(new PodBuilder().build())), "resource-name-prefix", "app-id", Map.empty, diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/MountSecretsFeatureStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/MountSecretsFeatureStepSuite.scala index 2b49b72dfa569..dad610c443acc 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/MountSecretsFeatureStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/MountSecretsFeatureStepSuite.scala @@ -35,7 +35,7 @@ class MountSecretsFeatureStepSuite extends SparkFunSuite { val sparkConf = new SparkConf(false) val kubernetesConf = KubernetesConf( sparkConf, - KubernetesExecutorSpecificConf("1", new PodBuilder().build()), + KubernetesExecutorSpecificConf("1", Some(new PodBuilder().build())), "resource-name-prefix", "app-id", Map.empty, diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala index 0c19f5946b75f..e847f8590d353 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala @@ -166,7 +166,7 @@ class ExecutorPodsAllocatorSuite extends SparkFunSuite with BeforeAndAfter { conf, executorSpecificConf.executorId, TEST_SPARK_APP_ID, - driverPod) + Some(driverPod)) k8sConf.sparkConf.getAll.toMap == conf.getAll.toMap && // Since KubernetesConf.createExecutorConf clones the SparkConf object, force // deep equality comparison for the SparkConf object and use object equality diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesExecutorBuilderSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesExecutorBuilderSuite.scala index d0b4127065eb7..44fe4a24e1102 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesExecutorBuilderSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesExecutorBuilderSuite.scala @@ -51,7 +51,7 @@ class KubernetesExecutorBuilderSuite extends SparkFunSuite { val conf = KubernetesConf( new SparkConf(false), KubernetesExecutorSpecificConf( - "executor-id", new PodBuilder().build()), + "executor-id", Some(new PodBuilder().build())), "prefix", "appId", Map.empty, @@ -69,7 +69,7 @@ class KubernetesExecutorBuilderSuite extends SparkFunSuite { val conf = KubernetesConf( new SparkConf(false), KubernetesExecutorSpecificConf( - "executor-id", new PodBuilder().build()), + "executor-id", Some(new PodBuilder().build())), "prefix", "appId", Map.empty, @@ -96,7 +96,7 @@ class KubernetesExecutorBuilderSuite extends SparkFunSuite { val conf = KubernetesConf( new SparkConf(false), KubernetesExecutorSpecificConf( - "executor-id", new PodBuilder().build()), + "executor-id", Some(new PodBuilder().build())), "prefix", "appId", Map.empty, diff --git a/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/Dockerfile b/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/Dockerfile index 9badf8556afc3..42a670174eae1 100644 --- a/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/Dockerfile +++ b/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/Dockerfile @@ -31,7 +31,7 @@ RUN set -ex && \ apk upgrade --no-cache && \ apk add --no-cache bash tini libc6-compat && \ mkdir -p /opt/spark && \ - mkdir -p /opt/spark/work-dir \ + mkdir -p /opt/spark/work-dir && \ touch /opt/spark/RELEASE && \ rm /bin/sh && \ ln -sv /bin/bash /bin/sh && \ diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/ClientModeTestsSuite.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/ClientModeTestsSuite.scala new file mode 100644 index 0000000000000..0690db7f67cf0 --- /dev/null +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/ClientModeTestsSuite.scala @@ -0,0 +1,111 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.k8s.integrationtest + +import org.scalatest.concurrent.Eventually +import scala.collection.JavaConverters._ + +import org.apache.spark.deploy.k8s.integrationtest.KubernetesSuite.{k8sTestTag, INTERVAL, TIMEOUT} + +trait ClientModeTestsSuite { k8sSuite: KubernetesSuite => + + test("Run in client mode.", k8sTestTag) { + val labels = Map("spark-app-selector" -> driverPodName) + val driverPort = 7077 + val blockManagerPort = 10000 + val driverService = testBackend + .getKubernetesClient + .services() + .inNamespace(kubernetesTestComponents.namespace) + .createNew() + .withNewMetadata() + .withName(s"$driverPodName-svc") + .endMetadata() + .withNewSpec() + .withClusterIP("None") + .withSelector(labels.asJava) + .addNewPort() + .withName("driver-port") + .withPort(driverPort) + .withNewTargetPort(driverPort) + .endPort() + .addNewPort() + .withName("block-manager") + .withPort(blockManagerPort) + .withNewTargetPort(blockManagerPort) + .endPort() + .endSpec() + .done() + try { + val driverPod = testBackend + .getKubernetesClient + .pods() + .inNamespace(kubernetesTestComponents.namespace) + .createNew() + .withNewMetadata() + .withName(driverPodName) + .withLabels(labels.asJava) + .endMetadata() + .withNewSpec() + .withServiceAccountName("default") + .addNewContainer() + .withName("spark-example") + .withImage(image) + .withImagePullPolicy("IfNotPresent") + .withCommand("/opt/spark/bin/run-example") + .addToArgs("--master", s"k8s://https://kubernetes.default.svc") + .addToArgs("--deploy-mode", "client") + .addToArgs("--conf", s"spark.kubernetes.container.image=$image") + .addToArgs( + "--conf", + s"spark.kubernetes.namespace=${kubernetesTestComponents.namespace}") + .addToArgs("--conf", "spark.kubernetes.authenticate.oauthTokenFile=" + + "/var/run/secrets/kubernetes.io/serviceaccount/token") + .addToArgs("--conf", "spark.kubernetes.authenticate.caCertFile=" + + "/var/run/secrets/kubernetes.io/serviceaccount/ca.crt") + .addToArgs("--conf", s"spark.kubernetes.driver.pod.name=$driverPodName") + .addToArgs("--conf", "spark.executor.memory=500m") + .addToArgs("--conf", "spark.executor.cores=1") + .addToArgs("--conf", "spark.executor.instances=1") + .addToArgs("--conf", + s"spark.driver.host=" + + s"${driverService.getMetadata.getName}.${kubernetesTestComponents.namespace}.svc") + .addToArgs("--conf", s"spark.driver.port=$driverPort") + .addToArgs("--conf", s"spark.driver.blockManager.port=$blockManagerPort") + .addToArgs("SparkPi") + .addToArgs("10") + .endContainer() + .endSpec() + .done() + Eventually.eventually(TIMEOUT, INTERVAL) { + assert(kubernetesTestComponents.kubernetesClient + .pods() + .withName(driverPodName) + .getLog + .contains("Pi is roughly 3"), "The application did not complete.") + } + } finally { + // Have to delete the service manually since it doesn't have an owner reference + kubernetesTestComponents + .kubernetesClient + .services() + .inNamespace(kubernetesTestComponents.namespace) + .delete(driverService) + } + } + +} diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala index 95694aa93d5b5..0d829218cf774 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala @@ -38,12 +38,11 @@ private[spark] class KubernetesSuite extends SparkFunSuite import KubernetesSuite._ - private var testBackend: IntegrationTestBackend = _ - private var sparkHomeDir: Path = _ - private var image: String = _ - private var pyImage: String = _ - private var driverPodName: String = _ - + protected var testBackend: IntegrationTestBackend = _ + protected var sparkHomeDir: Path = _ + protected var image: String = _ + protected var pyImage: String = _ + protected var driverPodName: String = _ protected var kubernetesTestComponents: KubernetesTestComponents = _ protected var sparkAppConf: SparkAppConf = _ protected var containerLocalSparkDistroExamplesJar: String = _