From 2ffed5916291077548e56ea733625dd715e1f0f3 Mon Sep 17 00:00:00 2001 From: mcheah Date: Tue, 6 Dec 2016 14:25:59 -0800 Subject: [PATCH 01/81] [SPARK-18278] Minimal support for submitting to Kubernetes. --- .../org/apache/spark/deploy/SparkSubmit.scala | 36 +- .../spark/deploy/SparkSubmitArguments.scala | 30 ++ kubernetes/core/pom.xml | 101 +++++ ...che.spark.scheduler.ExternalClusterManager | 1 + .../spark/deploy/kubernetes/Client.scala | 355 ++++++++++++++++++ .../kubernetes/KubernetesClientBuilder.scala | 53 +++ .../spark/deploy/kubernetes/Retry.scala | 42 +++ .../rest/KubernetesRestProtocolMessages.scala | 58 +++ .../rest/kubernetes/HttpClientUtil.scala | 57 +++ .../kubernetes/KubernetesSparkRestApi.scala | 39 ++ .../KubernetesSparkRestServer.scala | 274 ++++++++++++++ .../kubernetes/KubernetesClusterManager.scala | 42 +++ .../KubernetesClusterSchedulerBackend.scala | 264 +++++++++++++ kubernetes/docker-minimal-bundle/pom.xml | 137 +++++++ .../src/main/assembly/driver-assembly.xml | 84 +++++ .../src/main/assembly/executor-assembly.xml | 84 +++++ .../src/main/docker/driver/Dockerfile | 26 ++ .../src/main/docker/executor/Dockerfile | 26 ++ .../integration-tests-spark-jobs/pom.xml | 45 +++ .../jobs/SparkPiWithInfiniteWait.scala | 50 +++ kubernetes/integration-tests/pom.xml | 206 ++++++++++ .../integrationtest/KubernetesSuite.scala | 157 ++++++++ .../docker/SparkDockerImageBuilder.scala | 59 +++ .../integrationtest/minikube/Minikube.scala | 173 +++++++++ .../restapis/SparkRestApiV1.scala | 50 +++ .../launcher/SparkSubmitOptionParser.java | 10 + pom.xml | 49 +++ 27 files changed, 2505 insertions(+), 3 deletions(-) create mode 100644 kubernetes/core/pom.xml create mode 100644 kubernetes/core/src/main/resources/META-INF/services/org.apache.spark.scheduler.ExternalClusterManager create mode 100644 kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/Client.scala create mode 100644 kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/KubernetesClientBuilder.scala create mode 100644 kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/Retry.scala create mode 100644 kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/KubernetesRestProtocolMessages.scala create mode 100644 kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/HttpClientUtil.scala create mode 100644 kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/KubernetesSparkRestApi.scala create mode 100644 kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/KubernetesSparkRestServer.scala create mode 100644 kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterManager.scala create mode 100644 kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala create mode 100644 kubernetes/docker-minimal-bundle/pom.xml create mode 100644 kubernetes/docker-minimal-bundle/src/main/assembly/driver-assembly.xml create mode 100644 kubernetes/docker-minimal-bundle/src/main/assembly/executor-assembly.xml create mode 100644 kubernetes/docker-minimal-bundle/src/main/docker/driver/Dockerfile create mode 100644 kubernetes/docker-minimal-bundle/src/main/docker/executor/Dockerfile create mode 100644 kubernetes/integration-tests-spark-jobs/pom.xml create mode 100644 kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/kubernetes/integrationtest/jobs/SparkPiWithInfiniteWait.scala create mode 100644 kubernetes/integration-tests/pom.xml create mode 100644 kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KubernetesSuite.scala create mode 100644 kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/docker/SparkDockerImageBuilder.scala create mode 100644 kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/minikube/Minikube.scala create mode 100644 kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/restapis/SparkRestApiV1.scala 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 c70061bc5b5bc..598bafcab81dc 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -70,7 +70,8 @@ object SparkSubmit { private val STANDALONE = 2 private val MESOS = 4 private val LOCAL = 8 - private val ALL_CLUSTER_MGRS = YARN | STANDALONE | MESOS | LOCAL + private val KUBERNETES = 16 + private val ALL_CLUSTER_MGRS = YARN | STANDALONE | MESOS | KUBERNETES | LOCAL // Deploy modes private val CLIENT = 1 @@ -239,6 +240,7 @@ object SparkSubmit { YARN case m if m.startsWith("spark") => STANDALONE case m if m.startsWith("mesos") => MESOS + case m if m.startsWith("kubernetes") => KUBERNETES case m if m.startsWith("local") => LOCAL case _ => printErrorAndExit("Master must either be yarn or start with spark, mesos, local") @@ -284,6 +286,7 @@ object SparkSubmit { } val isYarnCluster = clusterManager == YARN && deployMode == CLUSTER val isMesosCluster = clusterManager == MESOS && deployMode == CLUSTER + val isKubernetesCluster = clusterManager == KUBERNETES && deployMode == CLUSTER // Resolve maven dependencies if there are any and add classpath to jars. Add them to py-files // too for packages that include Python code @@ -330,6 +333,10 @@ object SparkSubmit { // 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, CLUSTER) if args.isPython || args.isR => + printErrorAndExit("Kubernetes does not currently support python or R applications.") case (STANDALONE, CLUSTER) if args.isPython => printErrorAndExit("Cluster deploy mode is currently not supported for python " + "applications on standalone clusters.") @@ -463,7 +470,17 @@ object SparkSubmit { OptionAssigner(args.principal, YARN, ALL_DEPLOY_MODES, sysProp = "spark.yarn.principal"), OptionAssigner(args.keytab, YARN, ALL_DEPLOY_MODES, sysProp = "spark.yarn.keytab"), - // Other options + // Kubernetes only + OptionAssigner(args.kubernetesMaster, KUBERNETES, ALL_DEPLOY_MODES, + sysProp = "spark.kubernetes.master"), + OptionAssigner(args.kubernetesNamespace, KUBERNETES, ALL_DEPLOY_MODES, + sysProp = "spark.kubernetes.namespace"), + OptionAssigner(args.kubernetesUploadJars, KUBERNETES, CLUSTER, + sysProp = "spark.kubernetes.driver.uploads.jars"), + OptionAssigner(args.kubernetesUploadDriverExtraClasspath, KUBERNETES, CLUSTER, + sysProp = "spark.kubernetes.driver.uploads.driverExtraClasspath"), + + // Other options OptionAssigner(args.executorCores, STANDALONE | YARN, ALL_DEPLOY_MODES, sysProp = "spark.executor.cores"), OptionAssigner(args.executorMemory, STANDALONE | MESOS | YARN, ALL_DEPLOY_MODES, @@ -506,8 +523,9 @@ object SparkSubmit { // Add the application jar automatically so the user doesn't have to call sc.addJar // For YARN cluster mode, the jar is already distributed on each node as "app.jar" + // In Kubernetes cluster mode, the jar will be uploaded by the client separately. // For python and R files, the primary resource is already distributed as a regular file - if (!isYarnCluster && !args.isPython && !args.isR) { + if (!isYarnCluster && !isKubernetesCluster && !args.isPython && !args.isR) { var jars = sysProps.get("spark.jars").map(x => x.split(",").toSeq).getOrElse(Seq.empty) if (isUserJar(args.primaryResource)) { jars = jars ++ Seq(args.primaryResource) @@ -606,6 +624,13 @@ object SparkSubmit { } } + if (isKubernetesCluster) { + childMainClass = "org.apache.spark.deploy.kubernetes.Client" + childArgs += args.primaryResource + childArgs += args.mainClass + childArgs ++= args.childArgs + } + // Load any properties specified through --conf and the default properties file for ((k, v) <- args.sparkProperties) { sysProps.getOrElseUpdate(k, v) @@ -829,6 +854,7 @@ private[spark] object SparkSubmitUtils { /** * Represents a Maven Coordinate + * * @param groupId the groupId of the coordinate * @param artifactId the artifactId of the coordinate * @param version the version of the coordinate @@ -840,6 +866,7 @@ private[spark] object SparkSubmitUtils { /** * Extracts maven coordinates from a comma-delimited string. Coordinates should be provided * in the format `groupId:artifactId:version` or `groupId/artifactId:version`. + * * @param coordinates Comma-delimited string of maven coordinates * @return Sequence of Maven coordinates */ @@ -870,6 +897,7 @@ private[spark] object SparkSubmitUtils { /** * Extracts maven coordinates from a comma-delimited string + * * @param remoteRepos Comma-delimited string of remote repositories * @param ivySettings The Ivy settings for this session * @return A ChainResolver used by Ivy to search for and resolve dependencies. @@ -934,6 +962,7 @@ private[spark] object SparkSubmitUtils { /** * Output a comma-delimited list of paths for the downloaded jars to be added to the classpath * (will append to jars in SparkSubmit). + * * @param artifacts Sequence of dependencies that were resolved and retrieved * @param cacheDirectory directory where jars are cached * @return a comma-delimited list of paths for the dependencies @@ -990,6 +1019,7 @@ private[spark] object SparkSubmitUtils { /** * Resolves any dependencies that were supplied through maven coordinates + * * @param coordinates Comma-delimited string of maven coordinates * @param remoteRepos Comma-delimited string of remote repositories other than maven central * @param ivyPath The path to the local ivy repository diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala index f1761e7c1ec92..4244742aad14c 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -71,6 +71,12 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S var principal: String = null var keytab: String = null + // Kubernetes only + var kubernetesMaster: String = null + var kubernetesNamespace: String = null + var kubernetesUploadJars: String = null + var kubernetesUploadDriverExtraClasspath: String = null + // Standalone cluster mode only var supervise: Boolean = false var driverCores: String = null @@ -186,6 +192,18 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S .getOrElse(sparkProperties.get("spark.executor.instances").orNull) keytab = Option(keytab).orElse(sparkProperties.get("spark.yarn.keytab")).orNull principal = Option(principal).orElse(sparkProperties.get("spark.yarn.principal")).orNull + kubernetesMaster = Option(kubernetesMaster) + .orElse(sparkProperties.get("spark.kubernetes.master")) + .orNull + kubernetesNamespace = Option(kubernetesNamespace) + .orElse(sparkProperties.get("spark.kubernetes.namespace")) + .orNull + kubernetesUploadJars = Option(kubernetesUploadJars) + .orElse(sparkProperties.get("spark.kubernetes.driver.uploads.jars")) + .orNull + kubernetesUploadDriverExtraClasspath = Option(kubernetesUploadDriverExtraClasspath) + .orElse(sparkProperties.get("spark.kubernetes.driver.uploads.driverExtraClasspath")) + .orNull // Try to set main class from JAR if no --class argument is given if (mainClass == null && !isPython && !isR && primaryResource != null) { @@ -426,6 +444,18 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S case KEYTAB => keytab = value + case KUBERNETES_MASTER => + kubernetesMaster = value + + case KUBERNETES_NAMESPACE => + kubernetesNamespace = value + + case KUBERNETES_UPLOAD_JARS => + kubernetesUploadJars = value + + case KUBERNETES_UPLOAD_DRIVER_EXTRA_CLASSPATH => + kubernetesUploadDriverExtraClasspath = value + case HELP => printUsageAndExit(0) diff --git a/kubernetes/core/pom.xml b/kubernetes/core/pom.xml new file mode 100644 index 0000000000000..9c7eb52b2680a --- /dev/null +++ b/kubernetes/core/pom.xml @@ -0,0 +1,101 @@ + + + + 4.0.0 + + org.apache.spark + spark-parent_2.11 + 2.1.0-SNAPSHOT + ../../pom.xml + + + spark-kubernetes_2.11 + jar + Spark Project Kubernetes + + kubernetes + 1.4.17 + + + + + org.apache.spark + spark-core_${scala.binary.version} + ${project.version} + + + + org.apache.spark + spark-core_${scala.binary.version} + ${project.version} + test-jar + test + + + + io.fabric8 + kubernetes-client + ${kubernetes.client.version} + + + com.netflix.feign + feign-core + + + com.netflix.feign + feign-okhttp + + + com.netflix.feign + feign-jackson + + + com.netflix.feign + feign-jaxrs + + + javax.ws.rs + jsr311-api + + + + + com.fasterxml.jackson.jaxrs + jackson-jaxrs-json-provider + + + javax.ws.rs + javax.ws.rs-api + + + + com.google.guava + guava + + + + + + + + target/scala-${scala.binary.version}/classes + target/scala-${scala.binary.version}/test-classes + + + + diff --git a/kubernetes/core/src/main/resources/META-INF/services/org.apache.spark.scheduler.ExternalClusterManager b/kubernetes/core/src/main/resources/META-INF/services/org.apache.spark.scheduler.ExternalClusterManager new file mode 100644 index 0000000000000..55e7e38b28a08 --- /dev/null +++ b/kubernetes/core/src/main/resources/META-INF/services/org.apache.spark.scheduler.ExternalClusterManager @@ -0,0 +1 @@ +org.apache.spark.scheduler.cluster.kubernetes.KubernetesClusterManager diff --git a/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/Client.scala b/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/Client.scala new file mode 100644 index 0000000000000..4ee00e8802080 --- /dev/null +++ b/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/Client.scala @@ -0,0 +1,355 @@ +/* + * 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.kubernetes + +import java.io.File +import java.security.SecureRandom +import java.util.concurrent.{Executors, TimeUnit} +import javax.net.ssl.X509TrustManager + +import com.google.common.io.Files +import com.google.common.util.concurrent.{SettableFuture, ThreadFactoryBuilder} +import io.fabric8.kubernetes.api.model._ +import io.fabric8.kubernetes.client.{Config, ConfigBuilder, DefaultKubernetesClient, KubernetesClientException, Watch, Watcher} +import io.fabric8.kubernetes.client.Watcher.Action +import io.fabric8.kubernetes.client.internal.SSLUtils +import org.apache.commons.codec.binary.Base64 +import scala.collection.JavaConverters._ +import scala.collection.mutable.ArrayBuffer +import scala.concurrent.ExecutionContext +import scala.concurrent.duration.DurationInt +import scala.util.Success + +import org.apache.spark.SparkConf +import org.apache.spark.deploy.rest.{AppResource, KubernetesCreateSubmissionRequest, RemoteAppResource, UploadedAppResource} +import org.apache.spark.deploy.rest.kubernetes._ +import org.apache.spark.internal.Logging +import org.apache.spark.util.Utils + +private[spark] class Client( + sparkConf: SparkConf, + mainClass: String, + mainAppResource: String, + appArgs: Array[String]) extends Logging { + import Client._ + + private val namespace = sparkConf.getOption("spark.kubernetes.namespace").getOrElse( + throw new IllegalArgumentException("Namespace must be provided in spark.kubernetes.namespace")) + private val master = sparkConf + .getOption("spark.kubernetes.master") + .getOrElse("Master must be provided in spark.kubernetes.master") + + private val launchTime = System.currentTimeMillis + private val kubernetesAppId = sparkConf.getOption("spark.app.name") + .orElse(sparkConf.getOption("spark.app.id")) + .getOrElse(s"spark-$launchTime") + + private val secretName = s"spark-submission-server-secret-$kubernetesAppId" + private val driverLauncherSelectorValue = s"driver-launcher-$launchTime" + // TODO set precise version by default + private val driverDockerImage = sparkConf.get( + "spark.kubernetes.driver.docker.image", "spark-driver:latest") + private val uploadedDriverExtraClasspath = sparkConf + .getOption("spark.kubernetes.driver.uploads.driverExtraClasspath") + private val uploadedJars = sparkConf.getOption("spark.kubernetes.driver.uploads.jars") + + private val secretBytes = new Array[Byte](128) + SECURE_RANDOM.nextBytes(secretBytes) + private val secretBase64String = Base64.encodeBase64String(secretBytes) + + private implicit val retryableExecutionContext = ExecutionContext + .fromExecutorService( + Executors.newSingleThreadExecutor(new ThreadFactoryBuilder() + .setNameFormat("kubernetes-client-retryable-futures-%d") + .setDaemon(true) + .build())) + + def run(): Unit = { + var k8ConfBuilder = new ConfigBuilder() + .withApiVersion("v1") + .withMasterUrl(master) + .withNamespace(namespace) + sparkConf.getOption("spark.kubernetes.submit.caCertFile").foreach { + f => k8ConfBuilder = k8ConfBuilder.withCaCertFile(f) + } + sparkConf.getOption("spark.kubernetes.submit.clientKeyFile").foreach { + f => k8ConfBuilder = k8ConfBuilder.withClientKeyFile(f) + } + sparkConf.getOption("spark.kubernetes.submit.clientCertFile").foreach { + f => k8ConfBuilder = k8ConfBuilder.withClientCertFile(f) + } + + val k8ClientConfig = k8ConfBuilder.build + Utils.tryWithResource(new DefaultKubernetesClient(k8ClientConfig))(kubernetesClient => { + val secret = kubernetesClient.secrets().createNew() + .withNewMetadata() + .withName(secretName) + .endMetadata() + .withData(Map((SUBMISSION_SERVER_SECRET_NAME, secretBase64String)).asJava) + .withType("Opaque") + .done() + try { + val selectors = Map(DRIVER_LAUNCHER_SELECTOR_LABEL -> driverLauncherSelectorValue).asJava + val uiPort = sparkConf + .getOption("spark.ui.port") + .map(_.toInt) + .getOrElse(DEFAULT_UI_PORT) + val (servicePorts, containerPorts) = configurePorts(uiPort) + val service = kubernetesClient.services().createNew() + .withNewMetadata() + .withName(kubernetesAppId) + .endMetadata() + .withNewSpec() + .withSelector(selectors) + .withPorts(servicePorts.asJava) + .endSpec() + .done() + sparkConf.set("spark.kubernetes.driver.service.name", service.getMetadata.getName) + sparkConf.setIfMissing("spark.driver.port", DRIVER_PORT.toString) + sparkConf.setIfMissing("spark.blockmanager.port", BLOCKMANAGER_PORT.toString) + val submitRequest = buildSubmissionRequest() + val submitCompletedFuture = SettableFuture.create[Boolean] + val secretDirectory = s"/var/run/secrets/spark-submission/$kubernetesAppId" + + val podWatcher = new Watcher[Pod] { + override def eventReceived(action: Action, t: Pod): Unit = { + if ((action == Action.ADDED || action == Action.MODIFIED) + && t.getStatus.getPhase == "Running" + && !submitCompletedFuture.isDone) { + t.getStatus + .getContainerStatuses + .asScala + .find(status => + status.getName == DRIVER_LAUNCHER_CONTAINER_NAME && status.getReady) match { + case Some(status) => + try { + val driverLauncher = getDriverLauncherService( + k8ClientConfig, master) + val ping = Retry.retry(5, 5.seconds) { + driverLauncher.ping() + } + ping onFailure { + case t: Throwable => + if (!submitCompletedFuture.isDone) { + submitCompletedFuture.setException(t) + } + } + val submitComplete = ping andThen { + case Success(_) => + driverLauncher.create(submitRequest) + submitCompletedFuture.set(true) + } + submitComplete onFailure { + case t: Throwable => + if (!submitCompletedFuture.isDone) { + submitCompletedFuture.setException(t) + } + } + } catch { + case e: Throwable => + if (!submitCompletedFuture.isDone) { + submitCompletedFuture.setException(e) + throw e + } + } + case None => + } + } + } + + override def onClose(e: KubernetesClientException): Unit = { + if (!submitCompletedFuture.isDone) { + submitCompletedFuture.setException(e) + } + } + } + + def createDriverPod(unused: Watch): Unit = { + kubernetesClient.pods().createNew() + .withNewMetadata() + .withName(kubernetesAppId) + .withLabels(selectors) + .endMetadata() + .withNewSpec() + .withRestartPolicy("OnFailure") + .addNewVolume() + .withName(s"spark-submission-secret-volume") + .withNewSecret() + .withSecretName(secret.getMetadata.getName) + .endSecret() + .endVolume + .addNewContainer() + .withName(DRIVER_LAUNCHER_CONTAINER_NAME) + .withImage(driverDockerImage) + .withImagePullPolicy("IfNotPresent") + .addNewVolumeMount() + .withName("spark-submission-secret-volume") + .withMountPath(secretDirectory) + .withReadOnly(true) + .endVolumeMount() + .addNewEnv() + .withName("SPARK_SUBMISSION_SECRET_LOCATION") + .withValue(s"$secretDirectory/$SUBMISSION_SERVER_SECRET_NAME") + .endEnv() + .addNewEnv() + .withName("SPARK_DRIVER_LAUNCHER_SERVER_PORT") + .withValue(DRIVER_LAUNCHER_SERVICE_INTERNAL_PORT.toString) + .endEnv() + .withPorts(containerPorts.asJava) + .endContainer() + .endSpec() + .done() + submitCompletedFuture.get(30, TimeUnit.SECONDS) + } + + Utils.tryWithResource(kubernetesClient + .pods() + .withLabels(selectors) + .watch(podWatcher)) { createDriverPod } + } finally { + kubernetesClient.secrets().delete(secret) + } + }) + } + + private def configurePorts(uiPort: Int): (Seq[ServicePort], Seq[ContainerPort]) = { + val servicePorts = new ArrayBuffer[ServicePort] + val containerPorts = new ArrayBuffer[ContainerPort] + + def addPortToServiceAndContainer(portName: String, portValue: Int): Unit = { + servicePorts += new ServicePortBuilder() + .withName(portName) + .withPort(portValue) + .withNewTargetPort(portValue) + .build() + containerPorts += new ContainerPortBuilder() + .withContainerPort(portValue) + .build() + } + + addPortToServiceAndContainer( + DRIVER_LAUNCHER_SERVICE_PORT_NAME, + DRIVER_LAUNCHER_SERVICE_INTERNAL_PORT) + addPortToServiceAndContainer( + DRIVER_PORT_NAME, + sparkConf + .getOption("spark.driver.port") + .map(_.toInt) + .getOrElse(DRIVER_PORT)) + addPortToServiceAndContainer( + BLOCKMANAGER_PORT_NAME, + sparkConf + .getOption("spark.blockmanager.port") + .map(_.toInt) + .getOrElse(BLOCKMANAGER_PORT)) + + addPortToServiceAndContainer(UI_PORT_NAME, uiPort) + (servicePorts.toSeq, containerPorts.toSeq) + } + + private def buildSubmissionRequest(): KubernetesCreateSubmissionRequest = { + val appResourceUri = Utils.resolveURI(mainAppResource) + val resolvedAppResource: AppResource = appResourceUri.getScheme match { + case "file" | null => + val appFile = new File(appResourceUri.getPath) + if (!appFile.isFile) { + throw new IllegalStateException("Provided local file path does not exist" + + s" or is not a file: ${appFile.getAbsolutePath}") + } + val fileBytes = Files.toByteArray(appFile) + val fileBase64 = Base64.encodeBase64String(fileBytes) + UploadedAppResource(resourceBase64Contents = fileBase64, name = appFile.getName) + case other => RemoteAppResource(other) + } + + val uploadDriverExtraClasspathBase64Contents = getFileContents(uploadedDriverExtraClasspath) + val uploadJarsBase64Contents = getFileContents(uploadedJars) + KubernetesCreateSubmissionRequest( + appResource = resolvedAppResource, + mainClass = mainClass, + appArgs = appArgs, + secret = secretBase64String, + sparkProperties = sparkConf.getAll.toMap, + uploadedDriverExtraClasspathBase64Contents = uploadDriverExtraClasspathBase64Contents, + uploadedJarsBase64Contents = uploadJarsBase64Contents) + } + + def getFileContents(maybeFilePaths: Option[String]): Array[(String, String)] = { + maybeFilePaths + .map(_.split(",").map(filePath => { + val driverExtraClasspathFile = new File(filePath) + if (!driverExtraClasspathFile.isFile) { + throw new IllegalStateException("Provided file to upload for driver extra classpath" + + s" does not exist or is not a file: $filePath") + } else { + val fileBytes = Files.toByteArray(driverExtraClasspathFile) + val fileBase64 = Base64.encodeBase64String(fileBytes) + (driverExtraClasspathFile.getName, fileBase64) + } + })).getOrElse(Array.empty[(String, String)]) + } + + private def getDriverLauncherService( + k8ClientConfig: Config, + kubernetesMaster: String): KubernetesSparkRestApi = { + val url = s"${ + Array[String]( + kubernetesMaster, + "api", "v1", "proxy", + "namespaces", namespace, + "services", kubernetesAppId).mkString("/")}" + + s":$DRIVER_LAUNCHER_SERVICE_PORT_NAME/" + + val sslContext = SSLUtils.sslContext(k8ClientConfig) + val trustManager = SSLUtils.trustManagers( + k8ClientConfig)(0).asInstanceOf[X509TrustManager] + HttpClientUtil.createClient[KubernetesSparkRestApi]( + uri = url, + sslSocketFactory = sslContext.getSocketFactory, + trustContext = trustManager) + } +} + +private object Client { + + private val SUBMISSION_SERVER_SECRET_NAME = "spark-submission-server-secret" + private val DRIVER_LAUNCHER_SELECTOR_LABEL = "driver-launcher-selector" + private val DRIVER_LAUNCHER_SERVICE_INTERNAL_PORT = 7077 + private val DRIVER_PORT = 7078 + private val BLOCKMANAGER_PORT = 7079 + private val DEFAULT_UI_PORT = 4040 + private val UI_PORT_NAME = "spark-ui-port" + private val DRIVER_LAUNCHER_SERVICE_PORT_NAME = "driver-launcher-port" + private val DRIVER_PORT_NAME = "driver-port" + private val BLOCKMANAGER_PORT_NAME = "block-manager-port" + private val DRIVER_LAUNCHER_CONTAINER_NAME = "spark-kubernetes-driver-launcher" + private val SECURE_RANDOM = new SecureRandom() + + def main(args: Array[String]): Unit = { + require(args.length >= 2, s"Too few arguments. Usage: ${getClass.getName} " + + s" []") + val mainAppResource = args(0) + val mainClass = args(1) + val appArgs = args.drop(2) + val sparkConf = new SparkConf(true) + new Client( + mainAppResource = mainAppResource, + mainClass = mainClass, + sparkConf = sparkConf, + appArgs = appArgs).run() + } +} diff --git a/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/KubernetesClientBuilder.scala b/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/KubernetesClientBuilder.scala new file mode 100644 index 0000000000000..4c715c86cc7f9 --- /dev/null +++ b/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/KubernetesClientBuilder.scala @@ -0,0 +1,53 @@ +/* + * 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.kubernetes + +import java.io.File + +import com.google.common.base.Charsets +import com.google.common.io.Files +import io.fabric8.kubernetes.client.{ConfigBuilder, DefaultKubernetesClient} + +private[spark] object KubernetesClientBuilder { + private val API_SERVER_TOKEN = new File("/var/run/secrets/kubernetes.io/serviceaccount/token") + private val CA_CERT_FILE = new File("/var/run/secrets/kubernetes.io/serviceaccount/ca.crt") + + /** + * Creates a {@link KubernetesClient}, expecting to be from + * within the context of a pod. When doing so, credentials files + * are picked up from canonical locations, as they are injected + * into the pod's disk space. + */ + def buildFromWithinPod( + kubernetesMaster: String, + kubernetesNamespace: String): DefaultKubernetesClient = { + var clientConfigBuilder = new ConfigBuilder() + .withApiVersion("v1") + .withMasterUrl(kubernetesMaster) + .withNamespace(kubernetesNamespace) + + if (CA_CERT_FILE.isFile) { + clientConfigBuilder = clientConfigBuilder.withCaCertFile(CA_CERT_FILE.getAbsolutePath) + } + + if (API_SERVER_TOKEN.isFile) { + clientConfigBuilder = clientConfigBuilder.withOauthToken( + Files.toString(API_SERVER_TOKEN, Charsets.UTF_8)) + } + new DefaultKubernetesClient(clientConfigBuilder.build) + } +} diff --git a/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/Retry.scala b/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/Retry.scala new file mode 100644 index 0000000000000..e5ce0bcd606b2 --- /dev/null +++ b/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/Retry.scala @@ -0,0 +1,42 @@ +/* + * 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.kubernetes + +import scala.concurrent.{ExecutionContext, Future} +import scala.concurrent.duration.Duration + +private[spark] object Retry { + + private def retryableFuture[T] + (times: Int, interval: Duration) + (f: => Future[T]) + (implicit executionContext: ExecutionContext): Future[T] = { + f recoverWith { + case _ if times > 0 => { + Thread.sleep(interval.toMillis) + retryableFuture(times - 1, interval)(f) + } + } + } + + def retry[T] + (times: Int, interval: Duration) + (f: => T) + (implicit executionContext: ExecutionContext): Future[T] = { + retryableFuture(times, interval)(Future[T] { f }) + } +} diff --git a/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/KubernetesRestProtocolMessages.scala b/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/KubernetesRestProtocolMessages.scala new file mode 100644 index 0000000000000..4b7bb66083f29 --- /dev/null +++ b/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/KubernetesRestProtocolMessages.scala @@ -0,0 +1,58 @@ +/* + * 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.rest + +import com.fasterxml.jackson.annotation.{JsonSubTypes, JsonTypeInfo} + +import org.apache.spark.SPARK_VERSION + +// TODO: jars should probably be compressed. Shipping tarballs would be optimal. +case class KubernetesCreateSubmissionRequest( + val appResource: AppResource, + val mainClass: String, + val appArgs: Array[String], + val sparkProperties: Map[String, String], + val secret: String, + val uploadedDriverExtraClasspathBase64Contents: Array[(String, String)] + = Array.empty[(String, String)], + val uploadedJarsBase64Contents: Array[(String, String)] + = Array.empty[(String, String)]) extends SubmitRestProtocolRequest { + message = "create" + clientSparkVersion = SPARK_VERSION +} + +@JsonTypeInfo( + use = JsonTypeInfo.Id.NAME, + include = JsonTypeInfo.As.PROPERTY, + property = "type") +@JsonSubTypes(value = Array( + new JsonSubTypes.Type(value = classOf[UploadedAppResource], name = "UploadedAppResource"), + new JsonSubTypes.Type(value = classOf[RemoteAppResource], name = "RemoteAppResource"))) +abstract class AppResource + +case class UploadedAppResource( + resourceBase64Contents: String, + name: String = "spark-app-resource") extends AppResource + +case class RemoteAppResource(resource: String) extends AppResource + +class PingResponse extends SubmitRestProtocolResponse { + val text = "pong" + message = "pong" + serverSparkVersion = SPARK_VERSION +} + diff --git a/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/HttpClientUtil.scala b/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/HttpClientUtil.scala new file mode 100644 index 0000000000000..eb7d411700829 --- /dev/null +++ b/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/HttpClientUtil.scala @@ -0,0 +1,57 @@ +/* + * 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.rest.kubernetes + +import javax.net.ssl.{SSLContext, SSLSocketFactory, X509TrustManager} + +import com.fasterxml.jackson.databind.{DeserializationFeature, ObjectMapper} +import com.fasterxml.jackson.module.scala.DefaultScalaModule +import feign.Feign +import feign.Request.Options +import feign.jackson.{JacksonDecoder, JacksonEncoder} +import feign.jaxrs.JAXRSContract +import okhttp3.OkHttpClient +import scala.reflect.ClassTag + +import org.apache.spark.status.api.v1.JacksonMessageWriter + +private[spark] object HttpClientUtil { + + def createClient[T: ClassTag]( + uri: String, + sslSocketFactory: SSLSocketFactory = SSLContext.getDefault.getSocketFactory, + trustContext: X509TrustManager = null, + readTimeoutMillis: Int = 20000, + connectTimeoutMillis: Int = 20000): T = { + var httpClientBuilder = new OkHttpClient.Builder() + Option.apply(trustContext).foreach(context => { + httpClientBuilder = httpClientBuilder.sslSocketFactory(sslSocketFactory, context) + }) + val objectMapper = new ObjectMapper() + .registerModule(new DefaultScalaModule) + .setDateFormat(JacksonMessageWriter.makeISODateFormat) + objectMapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false) + val clazz = implicitly[ClassTag[T]].runtimeClass.asInstanceOf[Class[T]] + Feign.builder() + .client(new feign.okhttp.OkHttpClient(httpClientBuilder.build())) + .contract(new JAXRSContract) + .encoder(new JacksonEncoder(objectMapper)) + .decoder(new JacksonDecoder(objectMapper)) + .options(new Options(connectTimeoutMillis, readTimeoutMillis)) + .target(clazz, uri) + } +} diff --git a/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/KubernetesSparkRestApi.scala b/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/KubernetesSparkRestApi.scala new file mode 100644 index 0000000000000..3cbcb16293b1d --- /dev/null +++ b/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/KubernetesSparkRestApi.scala @@ -0,0 +1,39 @@ +/* + * 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.rest.kubernetes + +import javax.ws.rs.{Consumes, GET, Path, POST, Produces} +import javax.ws.rs.core.MediaType + +import org.apache.spark.deploy.rest.{CreateSubmissionResponse, KubernetesCreateSubmissionRequest, PingResponse} + +@Path("/v1/submissions/") +trait KubernetesSparkRestApi { + + @POST + @Consumes(Array(MediaType.APPLICATION_JSON)) + @Produces(Array(MediaType.APPLICATION_JSON)) + @Path("/create") + def create(request: KubernetesCreateSubmissionRequest): CreateSubmissionResponse + + @GET + @Consumes(Array(MediaType.APPLICATION_JSON)) + @Produces(Array(MediaType.APPLICATION_JSON)) + @Path("/ping") + def ping(): PingResponse + +} diff --git a/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/KubernetesSparkRestServer.scala b/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/KubernetesSparkRestServer.scala new file mode 100644 index 0000000000000..0a2e8176394ab --- /dev/null +++ b/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/KubernetesSparkRestServer.scala @@ -0,0 +1,274 @@ +/* + * 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.rest.kubernetes + +import java.io.File +import java.net.URI +import java.nio.file.Paths +import java.util.concurrent.CountDownLatch +import javax.servlet.http.{HttpServletRequest, HttpServletResponse} + +import com.google.common.io.Files +import org.apache.commons.codec.binary.Base64 +import scala.collection.mutable +import scala.collection.mutable.ArrayBuffer + +import org.apache.spark.{SecurityManager, SPARK_VERSION, SparkConf} +import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.deploy.rest._ +import org.apache.spark.util.{ShutdownHookManager, Utils} + +private case class KubernetesSparkRestServerArguments( + val host: Option[String] = None, + val port: Option[Int] = None, + val secretFile: Option[String] = None) { + def validate(): KubernetesSparkRestServerArguments = { + require(host.isDefined, "Hostname not set via --hostname.") + require(port.isDefined, "Port not set via --port") + require(secretFile.isDefined, "Secret file not set via --secret-file") + this + } +} + +private object KubernetesSparkRestServerArguments { + def fromArgsArray(inputArgs: Array[String]): KubernetesSparkRestServerArguments = { + var args = inputArgs.toList + var resolvedArguments = KubernetesSparkRestServerArguments() + while (args.nonEmpty) { + resolvedArguments = args match { + case "--hostname" :: value :: tail => + args = tail + resolvedArguments.copy(host = Some(value)) + case "--port" :: value :: tail => + args = tail + resolvedArguments.copy(port = Some(value.toInt)) + case "--secret-file" :: value :: tail => + args = tail + resolvedArguments.copy(secretFile = Some(value)) + // TODO polish usage message + case Nil => resolvedArguments + case unknown => throw new IllegalStateException(s"Unknown argument(s) found: $unknown") + } + } + resolvedArguments.validate() + } +} + +private[spark] class KubernetesSparkRestServer( + host: String, + port: Int, + conf: SparkConf, + expectedApplicationSecret: Array[Byte]) + extends RestSubmissionServer(host, port, conf) { + + private val javaExecutable = s"${System.getenv("JAVA_HOME")}/bin/java" + private val sparkHome = System.getenv("SPARK_HOME") + private val securityManager = new SecurityManager(conf) + override protected lazy val contextToServlet = Map[String, RestServlet]( + s"$baseContext/create/*" -> submitRequestServlet, + s"$baseContext/ping/*" -> pingServlet) + + private val pingServlet = new PingServlet + override protected val submitRequestServlet: SubmitRequestServlet + = new KubernetesSubmitRequestServlet + // TODO + override protected val statusRequestServlet: StatusRequestServlet = null + override protected val killRequestServlet: KillRequestServlet = null + + private class PingServlet extends RestServlet { + protected override def doGet( + request: HttpServletRequest, + response: HttpServletResponse): Unit = { + sendResponse(new PingResponse, response) + } + } + + private class KubernetesSubmitRequestServlet extends SubmitRequestServlet { + + // TODO validating the secret should be done as part of a header of the request. + // Instead here we have to specify the secret in the body. + override protected def handleSubmit( + requestMessageJson: String, + requestMessage: SubmitRestProtocolMessage, + responseServlet: HttpServletResponse): SubmitRestProtocolResponse = { + requestMessage match { + case KubernetesCreateSubmissionRequest( + appResource, + mainClass, + appArgs, + sparkProperties, + secret, + uploadedDriverExtraClasspath, + uploadedJars) => + val decodedSecret = Base64.decodeBase64(secret) + if (!expectedApplicationSecret.sameElements(decodedSecret)) { + responseServlet.setStatus(HttpServletResponse.SC_UNAUTHORIZED) + handleError("Unauthorized to submit application.") + } else { + val tempDir = Utils.createTempDir() + val appResourcePath = resolvedAppResource(appResource, tempDir) + val driverClasspathDirectory = new File(tempDir, "driver-extra-classpath") + if (!driverClasspathDirectory.mkdir) { + throw new IllegalStateException("Failed to create driver extra classpath" + + s" dir at ${driverClasspathDirectory.getAbsolutePath}") + } + val jarsDirectory = new File(tempDir, "jars") + if (!jarsDirectory.mkdir) { + throw new IllegalStateException("Failed to create jars dir at" + + s"${jarsDirectory.getAbsolutePath}") + } + val writtenDriverExtraClasspath = writeBase64ContentsToFiles( + uploadedDriverExtraClasspath, driverClasspathDirectory) + val writtenJars = writeBase64ContentsToFiles(uploadedJars, jarsDirectory) + val originalDriverExtraClasspath = sparkProperties.get("spark.driver.extraClassPath") + .map(_.split(",")) + .getOrElse(Array.empty[String]) + val resolvedDriverExtraClasspath = writtenDriverExtraClasspath ++ + originalDriverExtraClasspath + val originalJars = sparkProperties.get("spark.jars") + .map(_.split(",")) + .getOrElse(Array.empty[String]) + val resolvedJars = writtenJars ++ originalJars ++ Array(appResourcePath) + val sparkJars = new File(sparkHome, "jars").listFiles().map(_.getAbsolutePath) + val driverClasspath = resolvedDriverExtraClasspath ++ + resolvedJars ++ + sparkJars ++ + Array(appResourcePath) + val resolvedSparkProperties = new mutable.HashMap[String, String] + resolvedSparkProperties ++= sparkProperties + resolvedSparkProperties("spark.jars") = resolvedJars.mkString(",") + + val command = new ArrayBuffer[String] + command += javaExecutable + command += "-cp" + command += s"${driverClasspath.mkString(":")}" + for (prop <- resolvedSparkProperties) { + command += s"-D${prop._1}=${prop._2}" + } + val driverMemory = resolvedSparkProperties.getOrElse("spark.driver.memory", "1g") + command += s"-Xms$driverMemory" + command += s"-Xmx$driverMemory" + command += mainClass + command ++= appArgs + val pb = new ProcessBuilder(command: _*) + Paths.get(sparkHome, "logs").toFile.mkdirs + pb.redirectOutput(Paths.get(sparkHome, "logs", "stdout").toFile) + pb.redirectError(Paths.get(sparkHome, "logs", "stderr").toFile) + val process = pb.start() + ShutdownHookManager.addShutdownHook(() => { + logInfo("Received stop command, shutting down the running Spark application...") + process.destroy() + }) + val response = new CreateSubmissionResponse + response.success = true + response.submissionId = null + response.message = "success" + response.serverSparkVersion = SPARK_VERSION + response + } + case unexpected => + responseServlet.setStatus(HttpServletResponse.SC_BAD_REQUEST) + handleError(s"Received message of unexpected type ${unexpected.messageType}.") + } + } + + def resolvedAppResource(appResource: AppResource, tempDir: File): String = { + val appResourcePath = appResource match { + case UploadedAppResource(resourceContentsBase64, resourceName) => + val resourceFile = new File(tempDir, resourceName) + val resourceFilePath = resourceFile.getAbsolutePath + if (resourceFile.createNewFile()) { + val resourceContentsBytes = Base64.decodeBase64(resourceContentsBase64) + Files.write(resourceContentsBytes, resourceFile) + resourceFile.getAbsolutePath + } else { + throw new IllegalStateException(s"Failed to write main app resource file" + + s" to $resourceFilePath") + } + case RemoteAppResource(resource) => + Utils.fetchFile(resource, tempDir, conf, + securityManager, SparkHadoopUtil.get.newConfiguration(conf), + System.currentTimeMillis(), useCache = false) + val fileName = Utils.decodeFileNameInURI(URI.create(resource)) + val downloadedFile = new File(tempDir, fileName) + val downloadedFilePath = downloadedFile.getAbsolutePath + if (!downloadedFile.isFile) { + throw new IllegalStateException(s"Main app resource is not a file or" + + s" does not exist at $downloadedFilePath") + } + downloadedFilePath + } + appResourcePath + } + } + + private def writeBase64ContentsToFiles( + filesBase64Contents: Array[(String, String)], + rootDir: File): Seq[String] = { + val resolvedFileNames = new scala.collection.mutable.HashSet[String] + val resolvedFilePaths = new ArrayBuffer[String] + for (file <- filesBase64Contents) { + var currentFileName = file._1 + var deduplicationCounter = 1 + while (resolvedFileNames.contains(currentFileName)) { + // Prepend the deduplication counter so as to not mess with the extension + currentFileName = s"$deduplicationCounter-$currentFileName" + deduplicationCounter += 1 + } + val resolvedFile = new File(rootDir, currentFileName) + val resolvedFilePath = resolvedFile.getAbsolutePath + if (resolvedFile.createNewFile()) { + val fileContents = Base64.decodeBase64(file._2) + Files.write(fileContents, resolvedFile) + } else { + throw new IllegalStateException(s"Could not write jar file to $resolvedFilePath") + } + resolvedFileNames += currentFileName + resolvedFilePaths += resolvedFilePath + } + resolvedFilePaths.toSeq + } +} + +private[spark] object KubernetesSparkRestServer { + private val barrier = new CountDownLatch(1) + def main(args: Array[String]): Unit = { + val parsedArguments = KubernetesSparkRestServerArguments.fromArgsArray(args) + val secretFile = new File(parsedArguments.secretFile.get) + if (!secretFile.isFile) { + throw new IllegalArgumentException(s"Secret file specified by --secret-file" + + " is not a file, or does not exist.") + } + val secretBytes = Files.toByteArray(secretFile) + val sparkConf = new SparkConf(true) + val server = new KubernetesSparkRestServer( + parsedArguments.host.get, + parsedArguments.port.get, + sparkConf, + secretBytes) + server.start() + ShutdownHookManager.addShutdownHook(() => { + try { + server.stop() + } finally { + barrier.countDown() + } + }) + barrier.await() + } +} + diff --git a/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterManager.scala b/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterManager.scala new file mode 100644 index 0000000000000..0d3b97c636ca3 --- /dev/null +++ b/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterManager.scala @@ -0,0 +1,42 @@ +/* + * 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.scheduler.cluster.kubernetes + +import org.apache.spark.SparkContext +import org.apache.spark.scheduler.{ExternalClusterManager, SchedulerBackend, TaskScheduler, TaskSchedulerImpl} + +private[spark] class KubernetesClusterManager extends ExternalClusterManager { + + override def canCreate(masterURL: String): Boolean = masterURL.startsWith("kubernetes") + + override def createTaskScheduler(sc: SparkContext, masterURL: String): TaskScheduler = { + val scheduler = new TaskSchedulerImpl(sc) + sc.taskScheduler = scheduler + scheduler + } + + override def createSchedulerBackend(sc: SparkContext, masterURL: String, scheduler: TaskScheduler) + : SchedulerBackend = { + new KubernetesClusterSchedulerBackend(sc.taskScheduler.asInstanceOf[TaskSchedulerImpl], sc) + } + + override def initialize(scheduler: TaskScheduler, backend: SchedulerBackend): Unit = { + scheduler.asInstanceOf[TaskSchedulerImpl].initialize(backend) + } + +} + diff --git a/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala b/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala new file mode 100644 index 0000000000000..f37b97e4dd0dc --- /dev/null +++ b/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala @@ -0,0 +1,264 @@ +/* + * 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.scheduler.cluster.kubernetes + +import java.util.UUID +import java.util.concurrent.Executors +import java.util.concurrent.atomic.{AtomicInteger, AtomicLong} + +import com.google.common.util.concurrent.ThreadFactoryBuilder +import io.fabric8.kubernetes.api.model.{ContainerPort, ContainerPortBuilder, EnvVar, EnvVarBuilder, Pod, QuantityBuilder} +import scala.collection.JavaConverters._ +import scala.collection.mutable.ArrayBuffer +import scala.concurrent.{ExecutionContext, Future} + +import org.apache.spark.{SparkContext, SparkException} +import org.apache.spark.deploy.kubernetes.KubernetesClientBuilder +import org.apache.spark.rpc.RpcEndpointAddress +import org.apache.spark.scheduler.TaskSchedulerImpl +import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend +import org.apache.spark.util.Utils + +private[spark] class KubernetesClusterSchedulerBackend( + scheduler: TaskSchedulerImpl, + val sc: SparkContext) + extends CoarseGrainedSchedulerBackend(scheduler, sc.env.rpcEnv) { + + import KubernetesClusterSchedulerBackend._ + + private val EXECUTOR_MODIFICATION_LOCK = new Object + private val runningExecutorPods = new scala.collection.mutable.HashMap[String, Pod] + + private val kubernetesMaster = conf + .getOption("spark.kubernetes.master") + .getOrElse( + throw new SparkException("Kubernetes master must be specified in kubernetes mode.")) + + private val executorDockerImage = conf + .get("spark.kubernetes.executor.docker.image", s"spark-executor:${sc.version}") + + private val kubernetesNamespace = conf + .getOption("spark.kubernetes.namespace") + .getOrElse( + throw new SparkException("Kubernetes namespace must be specified in kubernetes mode.")) + + private val executorPort = conf.getInt("spark.executor.port", DEFAULT_STATIC_PORT) + + private val blockmanagerPort = conf + .getInt("spark.blockmanager.port", DEFAULT_BLOCKMANAGER_PORT) + + private val kubernetesDriverServiceName = conf + .getOption("spark.kubernetes.driver.service.name") + .getOrElse( + throw new SparkException("Must specify the service name the driver is running with")) + + private val executorMemory = conf.getOption("spark.executor.memory").getOrElse("1g") + private val executorMemoryBytes = Utils.byteStringAsBytes(executorMemory) + + private val memoryOverheadBytes = conf + .getOption("spark.kubernetes.executor.memoryOverhead") + .map(overhead => Utils.byteStringAsBytes(overhead)) + .getOrElse(math.max((MEMORY_OVERHEAD_FACTOR * executorMemoryBytes).toInt, + MEMORY_OVERHEAD_MIN)) + private val executorMemoryWithOverhead = executorMemoryBytes + memoryOverheadBytes + + private val executorCores = conf.getOption("spark.executor.cores").getOrElse("1") + + private implicit val requestExecutorContext = ExecutionContext.fromExecutorService( + Executors.newCachedThreadPool( + new ThreadFactoryBuilder() + .setDaemon(true) + .setNameFormat("kubernetes-executor-requests-%d") + .build)) + + private val kubernetesClient = KubernetesClientBuilder + .buildFromWithinPod(kubernetesMaster, kubernetesNamespace) + + override val minRegisteredRatio = + if (conf.getOption("spark.scheduler.minRegisteredResourcesRatio").isEmpty) { + 0.8 + } else { + super.minRegisteredRatio + } + + protected var totalExpectedExecutors = new AtomicInteger(0) + + private val driverUrl = RpcEndpointAddress( + System.getenv(s"${convertToEnvMode(kubernetesDriverServiceName)}_SERVICE_HOST"), + sc.getConf.getInt("spark.driver.port", DEFAULT_DRIVER_PORT), + CoarseGrainedSchedulerBackend.ENDPOINT_NAME).toString + + private def convertToEnvMode(value: String): String = + value.toUpperCase.map { c => if (c == '-') '_' else c } + + private val initialExecutors = getInitialTargetExecutorNumber(1) + + private def getInitialTargetExecutorNumber(defaultNumExecutors: Int = 1): Int = { + if (Utils.isDynamicAllocationEnabled(conf)) { + val minNumExecutors = conf.getInt("spark.dynamicAllocation.minExecutors", 0) + val initialNumExecutors = Utils.getDynamicAllocationInitialExecutors(conf) + val maxNumExecutors = conf.getInt("spark.dynamicAllocation.maxExecutors", 1) + require(initialNumExecutors >= minNumExecutors && initialNumExecutors <= maxNumExecutors, + s"initial executor number $initialNumExecutors must between min executor number " + + s"$minNumExecutors and max executor number $maxNumExecutors") + + initialNumExecutors + } else { + conf.getInt("spark.executor.instances", defaultNumExecutors) + } + } + + override def sufficientResourcesRegistered(): Boolean = { + totalRegisteredExecutors.get() >= initialExecutors * minRegisteredRatio + } + + override def start(): Unit = { + super.start() + if (!Utils.isDynamicAllocationEnabled(sc.conf)) { + doRequestTotalExecutors(initialExecutors) + } + } + + override def stop(): Unit = { + // TODO investigate why Utils.tryLogNonFatalError() doesn't work in this context. + // When using Utils.tryLogNonFatalError some of the code fails but without any logs or + // indication as to why. + try { + runningExecutorPods.values.foreach(kubernetesClient.pods().delete(_)) + } catch { + case e: Throwable => logError("Uncaught exception while shutting down controllers.", e) + } + try { + kubernetesClient.services().withName(kubernetesDriverServiceName).delete() + } catch { + case e: Throwable => logError("Uncaught exception while shutting down driver service.", e) + } + try { + kubernetesClient.close() + } catch { + case e: Throwable => logError("Uncaught exception closing Kubernetes client.", e) + } + super.stop() + } + + private def allocateNewExecutorPod(): (String, Pod) = { + val executorKubernetesId = UUID.randomUUID().toString.replaceAll("-", "") + val executorId = EXECUTOR_ID_COUNTER.incrementAndGet().toString + val name = s"exec$executorKubernetesId" + val selectors = Map(SPARK_EXECUTOR_SELECTOR -> executorId, + SPARK_APP_SELECTOR -> applicationId()).asJava + val executorMemoryQuantity = new QuantityBuilder(false) + .withAmount(executorMemoryBytes.toString) + .build() + val executorMemoryLimitQuantity = new QuantityBuilder(false) + .withAmount(executorMemoryWithOverhead.toString) + .build() + val requiredEnv = new ArrayBuffer[EnvVar] + requiredEnv += new EnvVarBuilder() + .withName("SPARK_EXECUTOR_PORT") + .withValue(executorPort.toString) + .build() + requiredEnv += new EnvVarBuilder() + .withName("SPARK_DRIVER_URL") + .withValue(driverUrl) + .build() + requiredEnv += new EnvVarBuilder() + .withName("SPARK_EXECUTOR_CORES") + .withValue(executorCores) + .build() + requiredEnv += new EnvVarBuilder() + .withName("SPARK_EXECUTOR_MEMORY") + .withValue(executorMemory) + .build() + requiredEnv += new EnvVarBuilder() + .withName("SPARK_APPLICATION_ID") + .withValue(applicationId()) + .build() + requiredEnv += new EnvVarBuilder() + .withName("SPARK_EXECUTOR_ID") + .withValue(executorId) + .build() + val requiredPorts = new ArrayBuffer[ContainerPort] + requiredPorts += new ContainerPortBuilder() + .withName(EXECUTOR_PORT_NAME) + .withContainerPort(executorPort) + .build() + requiredPorts += new ContainerPortBuilder() + .withName(BLOCK_MANAGER_PORT_NAME) + .withContainerPort(blockmanagerPort) + .build() + (executorKubernetesId, kubernetesClient.pods().createNew() + .withNewMetadata() + .withName(name) + .withLabels(selectors) + .endMetadata() + .withNewSpec() + .addNewContainer() + .withName(s"exec-${applicationId()}-container") + .withImage(executorDockerImage) + .withImagePullPolicy("IfNotPresent") + .withNewResources() + .addToRequests("memory", executorMemoryQuantity) + .addToLimits("memory", executorMemoryLimitQuantity) + .endResources() + .withEnv(requiredEnv.asJava) + .withPorts(requiredPorts.asJava) + .endContainer() + .endSpec() + .done()) + } + + override def doRequestTotalExecutors(requestedTotal: Int): Future[Boolean] = Future[Boolean] { + EXECUTOR_MODIFICATION_LOCK.synchronized { + if (requestedTotal > totalExpectedExecutors.get) { + logInfo(s"Requesting ${requestedTotal - totalExpectedExecutors.get}" + + s" additional executors, expecting total $requestedTotal and currently" + + s" expected ${totalExpectedExecutors.get}") + for (i <- 0 until (requestedTotal - totalExpectedExecutors.get)) { + runningExecutorPods += allocateNewExecutorPod() + } + } + totalExpectedExecutors.set(requestedTotal) + } + true + } + + override def doKillExecutors(executorIds: Seq[String]): Future[Boolean] = Future[Boolean] { + EXECUTOR_MODIFICATION_LOCK.synchronized { + for (executor <- executorIds) { + runningExecutorPods.remove(executor) match { + case Some(pod) => kubernetesClient.pods().delete(pod) + case None => logWarning(s"Unable to remove pod for unknown executor $executor") + } + } + } + true + } +} + +private object KubernetesClusterSchedulerBackend { + private val SPARK_EXECUTOR_SELECTOR = "spark-exec" + private val SPARK_APP_SELECTOR = "spark-app" + private val DEFAULT_STATIC_PORT = 10000 + private val DEFAULT_BLOCKMANAGER_PORT = 7079 + private val DEFAULT_DRIVER_PORT = 7078 + private val BLOCK_MANAGER_PORT_NAME = "blockmanager" + private val EXECUTOR_PORT_NAME = "executor" + private val MEMORY_OVERHEAD_FACTOR = 0.10 + private val MEMORY_OVERHEAD_MIN = 384L + private val EXECUTOR_ID_COUNTER = new AtomicLong(0L) +} diff --git a/kubernetes/docker-minimal-bundle/pom.xml b/kubernetes/docker-minimal-bundle/pom.xml new file mode 100644 index 0000000000000..3de939ea3978a --- /dev/null +++ b/kubernetes/docker-minimal-bundle/pom.xml @@ -0,0 +1,137 @@ + + + + + 4.0.0 + + org.apache.spark + spark-parent_2.11 + 2.1.0-SNAPSHOT + ../pom.xml + + + spark-docker-minimal-bundle_2.11 + Spark Project Docker Minimal Bundle + http://spark.apache.org/ + pom + + + docker-minimal-bundle + none + pre-integration-test + + + + + org.apache.spark + spark-assembly_${scala.binary.version} + ${project.version} + pom + + + + com.google.guava + guava + ${hadoop.deps.scope} + + + + + org.apache.spark + spark-kubernetes_${scala.binary.version} + ${project.version} + + + + + + org.apache.maven.plugins + maven-assembly-plugin + + + driver-docker-dist + pre-integration-test + + single + + + + src/main/assembly/driver-assembly.xml + + posix + + + + executor-docker-dist + pre-integration-test + + single + + + + src/main/assembly/executor-assembly.xml + + posix + + + + + + + + + + + hive + + + org.apache.spark + spark-hive_${scala.binary.version} + ${project.version} + + + + + hive-thriftserver + + + org.apache.spark + spark-hive-thriftserver_${scala.binary.version} + ${project.version} + + + + + spark-ganglia-lgpl + + + org.apache.spark + spark-ganglia-lgpl_${scala.binary.version} + ${project.version} + + + + + diff --git a/kubernetes/docker-minimal-bundle/src/main/assembly/driver-assembly.xml b/kubernetes/docker-minimal-bundle/src/main/assembly/driver-assembly.xml new file mode 100644 index 0000000000000..145244f34d1d9 --- /dev/null +++ b/kubernetes/docker-minimal-bundle/src/main/assembly/driver-assembly.xml @@ -0,0 +1,84 @@ + + + driver-docker-dist + + tar.gz + dir + + false + + + + ${project.parent.basedir}/core/src/main/resources/org/apache/spark/ui/static/ + + ui-resources/org/apache/spark/ui/static + + **/* + + + + + ${project.parent.basedir}/sbin/ + + sbin + + **/* + + + + + ${project.parent.basedir}/bin/ + + bin + + **/* + + + + + ${project.parent.basedir}/conf/ + + conf + + **/* + + + + + src/main/docker/driver + + + + **/* + + + + + + jars + true + false + runtime + false + + org.apache.spark:spark-assembly_${scala.binary.version}:pom + org.spark-project.spark:unused + + + + diff --git a/kubernetes/docker-minimal-bundle/src/main/assembly/executor-assembly.xml b/kubernetes/docker-minimal-bundle/src/main/assembly/executor-assembly.xml new file mode 100644 index 0000000000000..d97ba56562a12 --- /dev/null +++ b/kubernetes/docker-minimal-bundle/src/main/assembly/executor-assembly.xml @@ -0,0 +1,84 @@ + + + executor-docker-dist + + tar.gz + dir + + false + + + + ${project.parent.basedir}/core/src/main/resources/org/apache/spark/ui/static/ + + ui-resources/org/apache/spark/ui/static + + **/* + + + + + ${project.parent.basedir}/sbin/ + + sbin + + **/* + + + + + ${project.parent.basedir}/bin/ + + bin + + **/* + + + + + ${project.parent.basedir}/conf/ + + conf + + **/* + + + + + src/main/docker/executor + + + + **/* + + + + + + jars + true + false + runtime + false + + org.apache.spark:spark-assembly_${scala.binary.version}:pom + org.spark-project.spark:unused + + + + diff --git a/kubernetes/docker-minimal-bundle/src/main/docker/driver/Dockerfile b/kubernetes/docker-minimal-bundle/src/main/docker/driver/Dockerfile new file mode 100644 index 0000000000000..3bba38d8395ae --- /dev/null +++ b/kubernetes/docker-minimal-bundle/src/main/docker/driver/Dockerfile @@ -0,0 +1,26 @@ +FROM ubuntu:trusty + +# Upgrade package index +# install a few other useful packages plus Open Jdk 7 +# Remove unneeded /var/lib/apt/lists/* after install to reduce the +# docker image size (by ~30MB) +RUN apt-get update && \ + apt-get install -y less openjdk-7-jre-headless net-tools vim-tiny sudo openssh-server procps && \ + rm -rf /var/lib/apt/lists/* + +RUN mkdir -p /opt/spark +RUN mkdir -p /opt/spark/ui-resources/org/apache/spark/ui/static +RUN touch /opt/spark/RELEASE + +ADD jars /opt/spark/jars +ADD bin /opt/spark/bin +ADD sbin /opt/spark/sbin +ADD conf /opt/spark/conf + +ENV SPARK_HOME /opt/spark +ENV JAVA_HOME /usr/lib/jvm/java-7-openjdk-amd64/jre + +WORKDIR /opt/spark + +# This class will also require setting a secret via the SPARK_APP_SECRET environment variable +CMD exec bin/spark-class org.apache.spark.deploy.rest.kubernetes.KubernetesSparkRestServer --hostname $HOSTNAME --port $SPARK_DRIVER_LAUNCHER_SERVER_PORT --secret-file $SPARK_SUBMISSION_SECRET_LOCATION diff --git a/kubernetes/docker-minimal-bundle/src/main/docker/executor/Dockerfile b/kubernetes/docker-minimal-bundle/src/main/docker/executor/Dockerfile new file mode 100644 index 0000000000000..f68f1a3fb2694 --- /dev/null +++ b/kubernetes/docker-minimal-bundle/src/main/docker/executor/Dockerfile @@ -0,0 +1,26 @@ +FROM ubuntu:trusty + +# Upgrade package index +# install a few other useful packages plus Open Jdk 7 +# Remove unneeded /var/lib/apt/lists/* after install to reduce the +# docker image size (by ~30MB) +RUN apt-get update && \ + apt-get install -y less openjdk-7-jre-headless net-tools vim-tiny sudo openssh-server procps && \ + rm -rf /var/lib/apt/lists/* + +RUN mkdir -p /opt/spark +RUN mkdir -p /opt/spark/ui-resources/org/apache/spark/ui/static +RUN touch /opt/spark/RELEASE + +ADD jars /opt/spark/jars +ADD bin /opt/spark/bin +ADD sbin /opt/spark/sbin +ADD conf /opt/spark/conf + +ENV SPARK_HOME /opt/spark +ENV JAVA_HOME /usr/lib/jvm/java-7-openjdk-amd64/jre + +WORKDIR /opt/spark + +# TODO support spark.executor.extraClassPath +CMD exec ${JAVA_HOME}/bin/java -Dspark.executor.port=$SPARK_EXECUTOR_PORT -Xms$SPARK_EXECUTOR_MEMORY -Xmx$SPARK_EXECUTOR_MEMORY -cp ${SPARK_HOME}/jars/\* org.apache.spark.executor.CoarseGrainedExecutorBackend --driver-url $SPARK_DRIVER_URL --executor-id $SPARK_EXECUTOR_ID --cores $SPARK_EXECUTOR_CORES --app-id $SPARK_APPLICATION_ID --hostname $HOSTNAME diff --git a/kubernetes/integration-tests-spark-jobs/pom.xml b/kubernetes/integration-tests-spark-jobs/pom.xml new file mode 100644 index 0000000000000..17f1c4906214f --- /dev/null +++ b/kubernetes/integration-tests-spark-jobs/pom.xml @@ -0,0 +1,45 @@ + + + + 4.0.0 + + org.apache.spark + spark-parent_2.11 + 2.1.0-SNAPSHOT + ../../pom.xml + + + spark-kubernetes-integration-tests-spark-jobs_2.11 + jar + Spark Project Kubernetes Integration Tests Spark Jobs + + + + org.apache.spark + spark-core_${scala.binary.version} + ${project.version} + provided + + + org.apache.spark + spark-sql_${scala.binary.version} + ${project.version} + provided + + + diff --git a/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/kubernetes/integrationtest/jobs/SparkPiWithInfiniteWait.scala b/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/kubernetes/integrationtest/jobs/SparkPiWithInfiniteWait.scala new file mode 100644 index 0000000000000..6e4660b771305 --- /dev/null +++ b/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/kubernetes/integrationtest/jobs/SparkPiWithInfiniteWait.scala @@ -0,0 +1,50 @@ +/* + * 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.kubernetes.integrationtest.jobs + +import scala.math.random + +import org.apache.spark.sql.SparkSession + +// Equivalent to SparkPi except does not stop the Spark Context +// at the end and spins forever, so other things can inspect the +// Spark UI immediately after the fact. +private[spark] object SparkPiWithInfiniteWait { + + def main(args: Array[String]): Unit = { + val spark = SparkSession + .builder + .appName("Spark Pi") + .getOrCreate() + val slices = if (args.length > 0) args(0).toInt else 10 + val n = math.min(100000L * slices, Int.MaxValue).toInt // avoid overflow + val count = spark.sparkContext.parallelize(1 until n, slices).map { i => + val x = random * 2 - 1 + val y = random * 2 - 1 + if (x*x + y*y < 1) 1 else 0 + }.reduce(_ + _) + // scalastyle:off println + println("Pi is roughly " + 4.0 * count / (n - 1)) + // scalastyle:on println + + // Spin forever to keep the Spark UI active, so other things can inspect the job. + while (true) { + Thread.sleep(600000) + } + } + +} diff --git a/kubernetes/integration-tests/pom.xml b/kubernetes/integration-tests/pom.xml new file mode 100644 index 0000000000000..0568cb1e21826 --- /dev/null +++ b/kubernetes/integration-tests/pom.xml @@ -0,0 +1,206 @@ + + + + 4.0.0 + + org.apache.spark + spark-parent_2.11 + 2.1.0-SNAPSHOT + ../../pom.xml + + + spark-kubernetes-integration-tests_2.11 + jar + Spark Project Kubernetes Integration Tests + + + + org.apache.spark + spark-kubernetes_${scala.binary.version} + ${project.version} + test + + + org.apache.spark + spark-core_${scala.binary.version} + ${project.version} + test-jar + test + + + org.apache.spark + spark-kubernetes-integration-tests-spark-jobs_${scala.binary.version} + ${project.version} + test + + + org.apache.spark + spark-docker-minimal-bundle_${scala.binary.version} + ${project.version} + tar.gz + driver-docker-dist + test + + + * + * + + + + + com.google.guava + guava + test + + 18.0 + + + com.spotify + docker-client + test + + + + com.fasterxml.jackson.jaxrs + jackson-jaxrs-json-provider + + + com.fasterxml.jackson.core + jackson-databind + + + org.glassfish.jersey.core + jersey-client + + + org.glassfish.jersey.core + jersey-common + + + javax.ws.rs + jsr311-api + + + + + + + + + org.apache.maven.plugins + maven-dependency-plugin + + + copy-test-spark-jobs + pre-integration-test + + copy + + + + + org.apache.spark + spark-kubernetes-integration-tests-spark-jobs_${scala.binary.version} + ${project.version} + jar + ${project.build.directory}/integration-tests-spark-jobs + + + + + + unpack-docker-driver-bundle + pre-integration-test + + unpack + + + + + org.apache.spark + spark-docker-minimal-bundle_${scala.binary.version} + ${project.version} + driver-docker-dist + tar.gz + true + ${project.build.directory}/docker/driver + + + + + + unpack-docker-executor-bundle + pre-integration-test + + unpack + + + + + org.apache.spark + spark-docker-minimal-bundle_${scala.binary.version} + ${project.version} + executor-docker-dist + tar.gz + true + ${project.build.directory}/docker/executor + + + + + + + + com.googlecode.maven-download-plugin + download-maven-plugin + 1.3.0 + + + download-minikube-linux + pre-integration-test + + wget + + + https://storage.googleapis.com/minikube/releases/v0.12.2/minikube-linux-amd64 + ${project.build.directory}/minikube-bin/linux-amd64 + minikube + + + + download-minikube-darwin + pre-integration-test + + wget + + + https://storage.googleapis.com/minikube/releases/v0.12.2/minikube-darwin-amd64 + ${project.build.directory}/minikube-bin/darwin-amd64 + minikube + + + + + + + + + diff --git a/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KubernetesSuite.scala b/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KubernetesSuite.scala new file mode 100644 index 0000000000000..d79c75e484af5 --- /dev/null +++ b/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KubernetesSuite.scala @@ -0,0 +1,157 @@ +/* + * 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.kubernetes.integrationtest + +import java.nio.file.Paths +import java.util.UUID + +import com.google.common.collect.ImmutableList +import io.fabric8.kubernetes.client.{Config, KubernetesClient} +import org.scalatest.BeforeAndAfter +import org.scalatest.concurrent.{Eventually, PatienceConfiguration} +import org.scalatest.time.{Minutes, Seconds, Span} +import scala.collection.JavaConverters._ + +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.deploy.SparkSubmit +import org.apache.spark.deploy.kubernetes.Client +import org.apache.spark.deploy.kubernetes.integrationtest.docker.SparkDockerImageBuilder +import org.apache.spark.deploy.kubernetes.integrationtest.minikube.Minikube +import org.apache.spark.deploy.kubernetes.integrationtest.restapis.SparkRestApiV1 +import org.apache.spark.status.api.v1.{ApplicationStatus, StageStatus} + +private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { + + private val EXAMPLES_JAR = Paths.get("target", "integration-tests-spark-jobs") + .toFile + .listFiles()(0) + .getAbsolutePath + + private val TIMEOUT = PatienceConfiguration.Timeout(Span(2, Minutes)) + private val INTERVAL = PatienceConfiguration.Interval(Span(2, Seconds)) + private val MAIN_CLASS = "org.apache.spark.deploy.kubernetes" + + ".integrationtest.jobs.SparkPiWithInfiniteWait" + private val NAMESPACE = UUID.randomUUID().toString.replaceAll("-", "") + private var minikubeKubernetesClient: KubernetesClient = _ + private var clientConfig: Config = _ + + override def beforeAll(): Unit = { + Minikube.startMinikube() + new SparkDockerImageBuilder(Minikube.getDockerEnv).buildSparkDockerImages() + Minikube.getKubernetesClient.namespaces.createNew() + .withNewMetadata() + .withName(NAMESPACE) + .endMetadata() + .done() + minikubeKubernetesClient = Minikube.getKubernetesClient.inNamespace(NAMESPACE) + clientConfig = minikubeKubernetesClient.getConfiguration + } + + before { + Eventually.eventually(TIMEOUT, INTERVAL) { + assert(minikubeKubernetesClient.pods().list().getItems.isEmpty) + assert(minikubeKubernetesClient.services().list().getItems.isEmpty) + } + } + + after { + val pods = minikubeKubernetesClient.pods().list().getItems.asScala + pods.par.foreach(pod => { + minikubeKubernetesClient + .pods() + .withName(pod.getMetadata.getName) + .withGracePeriod(60) + .delete + }) + } + + override def afterAll(): Unit = { + if (!System.getProperty("spark.docker.test.persistMinikube", "false").toBoolean) { + Minikube.deleteMinikube() + } + } + + private def expectationsForStaticAllocation(sparkMetricsService: SparkRestApiV1): Unit = { + val apps = Eventually.eventually(TIMEOUT, INTERVAL) { + val result = sparkMetricsService + .getApplications(ImmutableList.of(ApplicationStatus.RUNNING, ApplicationStatus.COMPLETED)) + assert(result.size == 1 + && !result.head.id.equalsIgnoreCase("appid") + && !result.head.id.equalsIgnoreCase("{appId}")) + result + } + Eventually.eventually(TIMEOUT, INTERVAL) { + val result = sparkMetricsService.getExecutors(apps.head.id) + assert(result.size == 2) + assert(result.count(exec => exec.id != "driver") == 1) + result + } + Eventually.eventually(TIMEOUT, INTERVAL) { + val result = sparkMetricsService.getStages( + apps.head.id, Seq(StageStatus.COMPLETE).asJava) + assert(result.size == 1) + result + } + } + + test("Run a simple example") { + val sparkConf = new SparkConf(true) + .setMaster("kubernetes") + .set("spark.kubernetes.master", s"https://${Minikube.getMinikubeIp}:8443") + .set("spark.kubernetes.submit.caCertFile", clientConfig.getCaCertFile) + .set("spark.kubernetes.submit.clientKeyFile", clientConfig.getClientKeyFile) + .set("spark.kubernetes.submit.clientCertFile", clientConfig.getClientCertFile) + .set("spark.kubernetes.namespace", NAMESPACE) + .set("spark.kubernetes.executor.docker.image", "spark-executor:latest") + .set("spark.executor.memory", "500m") + .set("spark.executor.cores", "1") + .set("spark.executors.instances", "1") + .set("spark.app.id", "spark-pi") + val mainAppResource = s"file://$EXAMPLES_JAR" + + new Client( + sparkConf = sparkConf, + mainClass = MAIN_CLASS, + mainAppResource = mainAppResource, + appArgs = Array.empty[String]).run() + val sparkMetricsService = Minikube.getService[SparkRestApiV1]( + "spark-pi", NAMESPACE, "spark-ui-port") + expectationsForStaticAllocation(sparkMetricsService) + } + + test("Run using spark-submit") { + val args = Array( + "--master", "kubernetes", + "--deploy-mode", "cluster", + "--kubernetes-master", s"https://${Minikube.getMinikubeIp}:8443", + "--kubernetes-namespace", NAMESPACE, + "--name", "spark-pi", + "--executor-memory", "512m", + "--executor-cores", "1", + "--num-executors", "1", + "--class", MAIN_CLASS, + "--conf", s"spark.kubernetes.submit.caCertFile=${clientConfig.getCaCertFile}", + "--conf", s"spark.kubernetes.submit.clientKeyFile=${clientConfig.getClientKeyFile}", + "--conf", s"spark.kubernetes.submit.clientCertFile=${clientConfig.getClientCertFile}", + "--conf", "spark.kubernetes.executor.docker.image=spark-executor:latest", + EXAMPLES_JAR) + SparkSubmit.main(args) + val sparkMetricsService = Minikube.getService[SparkRestApiV1]( + "spark-pi", NAMESPACE, "spark-ui-port") + expectationsForStaticAllocation(sparkMetricsService) + } +} diff --git a/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/docker/SparkDockerImageBuilder.scala b/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/docker/SparkDockerImageBuilder.scala new file mode 100644 index 0000000000000..22d78142508c1 --- /dev/null +++ b/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/docker/SparkDockerImageBuilder.scala @@ -0,0 +1,59 @@ +/* + * 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.kubernetes.integrationtest.docker + +import java.net.URI +import java.nio.file.Paths + +import com.fasterxml.jackson.jaxrs.json.JacksonJaxbJsonProvider +import com.spotify.docker.client.{DefaultDockerClient, DockerCertificates} +import org.apache.http.client.utils.URIBuilder +import org.scalatest.concurrent.{Eventually, PatienceConfiguration} +import org.scalatest.time.{Minutes, Seconds, Span} + +private[spark] class SparkDockerImageBuilder(private val dockerEnv: Map[String, String]) { + + private val TIMEOUT = PatienceConfiguration.Timeout(Span(2, Minutes)) + private val INTERVAL = PatienceConfiguration.Interval(Span(2, Seconds)) + private val dockerHost = dockerEnv.getOrElse("DOCKER_HOST", + throw new IllegalStateException("DOCKER_HOST env not found.")) + + private val originalDockerUri = URI.create(dockerHost) + private val httpsDockerUri = new URIBuilder() + .setHost(originalDockerUri.getHost) + .setPort(originalDockerUri.getPort) + .setScheme("https") + .build() + + private val dockerCerts = dockerEnv.getOrElse("DOCKER_CERT_PATH", + throw new IllegalStateException("DOCKER_CERT_PATH env not found.")) + + private val dockerClient = new DefaultDockerClient.Builder() + .uri(httpsDockerUri) + .dockerCertificates(DockerCertificates + .builder() + .dockerCertPath(Paths.get(dockerCerts)) + .build().get()) + .build() + + def buildSparkDockerImages(): Unit = { + Eventually.eventually(TIMEOUT, INTERVAL) { dockerClient.ping() } + dockerClient.build(Paths.get("target", "docker", "driver"), "spark-driver") + dockerClient.build(Paths.get("target", "docker", "executor"), "spark-executor") + } + +} \ No newline at end of file diff --git a/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/minikube/Minikube.scala b/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/minikube/Minikube.scala new file mode 100644 index 0000000000000..92b809a4c7c59 --- /dev/null +++ b/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/minikube/Minikube.scala @@ -0,0 +1,173 @@ +/* + * 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.kubernetes.integrationtest.minikube + +import java.io.{BufferedReader, InputStreamReader} +import java.nio.file.Paths +import java.util.concurrent.TimeUnit +import javax.net.ssl.X509TrustManager + +import io.fabric8.kubernetes.client.internal.SSLUtils +import io.fabric8.kubernetes.client.{ConfigBuilder, DefaultKubernetesClient} +import scala.collection.mutable.ArrayBuffer +import scala.reflect.ClassTag + +import org.apache.spark.deploy.rest.kubernetes.HttpClientUtil +import org.apache.spark.internal.Logging +import org.apache.spark.util.Utils + +// TODO support windows +private[spark] object Minikube extends Logging { + private val MINIKUBE_EXECUTABLE_DEST = if (Utils.isMac) { + Paths.get("target", "minikube-bin", "darwin-amd64", "minikube").toFile + } else if (Utils.isWindows) { + throw new IllegalStateException("Executing Minikube based integration tests not yet " + + " available on Windows.") + } else { + Paths.get("target", "minikube-bin", "linux-amd64", "minikube").toFile + } + + private val EXPECTED_DOWNLOADED_MINIKUBE_MESSAGE = "Minikube is not downloaded, expected at " + + s"${MINIKUBE_EXECUTABLE_DEST.getAbsolutePath}" + + private val MINIKUBE_STARTUP_TIMEOUT_SECONDS = 60 + + def startMinikube(): Unit = synchronized { + assert(MINIKUBE_EXECUTABLE_DEST.exists(), EXPECTED_DOWNLOADED_MINIKUBE_MESSAGE) + if (getMinikubeStatus != MinikubeStatus.RUNNING) { + executeMinikube("start", "--memory", "6000", "--cpus", "8") + } else { + logInfo("Minikube is already started.") + } + } + + def getMinikubeIp: String = synchronized { + assert(MINIKUBE_EXECUTABLE_DEST.exists(), EXPECTED_DOWNLOADED_MINIKUBE_MESSAGE) + val outputs = executeMinikube("ip") + assert(outputs.size == 1, "Unexpected amount of output from minikube ip") + outputs.head + } + + def getMinikubeStatus: MinikubeStatus.Value = synchronized { + assert(MINIKUBE_EXECUTABLE_DEST.exists(), EXPECTED_DOWNLOADED_MINIKUBE_MESSAGE) + val statusString = executeMinikube("status").head.replaceFirst("minikubeVM: ", "") + MinikubeStatus.unapply(statusString) + .getOrElse(throw new IllegalStateException(s"Unknown status $statusString")) + } + + def getDockerEnv: Map[String, String] = synchronized { + assert(MINIKUBE_EXECUTABLE_DEST.exists(), EXPECTED_DOWNLOADED_MINIKUBE_MESSAGE) + executeMinikube("docker-env") + .filter(_.startsWith("export")) + .map(_.replaceFirst("export ", "").split('=')) + .map(arr => (arr(0), arr(1).replaceAllLiterally("\"", ""))) + .toMap + } + + def deleteMinikube(): Unit = synchronized { + assert(MINIKUBE_EXECUTABLE_DEST.exists, EXPECTED_DOWNLOADED_MINIKUBE_MESSAGE) + if (getMinikubeStatus != MinikubeStatus.DOES_NOT_EXIST) { + executeMinikube("delete") + } else { + logInfo("Minikube was already not running.") + } + } + + def getKubernetesClient: DefaultKubernetesClient = synchronized { + val kubernetesMaster = s"https://$getMinikubeIp:8443" + val userHome = System.getProperty("user.home") + val kubernetesConf = new ConfigBuilder() + .withApiVersion("v1") + .withMasterUrl(kubernetesMaster) + .withCaCertFile(Paths.get(userHome, ".minikube", "ca.crt").toFile.getAbsolutePath) + .withClientCertFile(Paths.get(userHome, ".minikube", "apiserver.crt").toFile.getAbsolutePath) + .withClientKeyFile(Paths.get(userHome, ".minikube", "apiserver.key").toFile.getAbsolutePath) + .build() + new DefaultKubernetesClient(kubernetesConf) + } + + def getService[T: ClassTag]( + serviceName: String, + namespace: String, + servicePortName: String, + servicePath: String = ""): T = synchronized { + val kubernetesMaster = s"https://$getMinikubeIp:8443" + val url = s"${ + Array[String]( + kubernetesMaster, + "api", "v1", "proxy", + "namespaces", namespace, + "services", serviceName).mkString("/")}" + + s":$servicePortName$servicePath" + val userHome = System.getProperty("user.home") + val kubernetesConf = new ConfigBuilder() + .withApiVersion("v1") + .withMasterUrl(kubernetesMaster) + .withCaCertFile(Paths.get(userHome, ".minikube", "ca.crt").toFile.getAbsolutePath) + .withClientCertFile(Paths.get(userHome, ".minikube", "apiserver.crt").toFile.getAbsolutePath) + .withClientKeyFile(Paths.get(userHome, ".minikube", "apiserver.key").toFile.getAbsolutePath) + .build() + val sslContext = SSLUtils.sslContext(kubernetesConf) + val trustManager = SSLUtils.trustManagers(kubernetesConf)(0).asInstanceOf[X509TrustManager] + HttpClientUtil.createClient[T](url, sslContext.getSocketFactory, trustManager) + } + + def executeMinikubeSsh(command: String): Unit = { + executeMinikube("ssh", command) + } + + private def executeMinikube(action: String, args: String*): Seq[String] = { + if (!MINIKUBE_EXECUTABLE_DEST.canExecute) { + if (!MINIKUBE_EXECUTABLE_DEST.setExecutable(true)) { + throw new IllegalStateException("Failed to make the Minikube binary executable.") + } + } + val fullCommand = Array(MINIKUBE_EXECUTABLE_DEST.getAbsolutePath, action) ++ args + val pb = new ProcessBuilder().command(fullCommand: _*) + pb.redirectErrorStream(true) + val proc = pb.start() + val outputLines = new ArrayBuffer[String] + + Utils.tryWithResource(new InputStreamReader(proc.getInputStream)) { procOutput => + Utils.tryWithResource(new BufferedReader(procOutput)) { (bufferedOutput: BufferedReader) => + var line: String = null + do { + line = bufferedOutput.readLine() + if (line != null) { + logInfo(line) + outputLines += line + } + } while (line != null) + } + } + assert(proc.waitFor(MINIKUBE_STARTUP_TIMEOUT_SECONDS, TimeUnit.SECONDS), + s"Timed out while executing $action on minikube.") + assert(proc.exitValue == 0, s"Failed to execute minikube $action ${args.mkString(" ")}") + outputLines.toSeq + } +} + +private[spark] object MinikubeStatus extends Enumeration { + + val RUNNING = status("Running") + val STOPPED = status("Stopped") + val DOES_NOT_EXIST = status("Does Not Exist") + val SAVED = status("Saved") + + def status(value: String): Value = new Val(nextId, value) + def unapply(s: String): Option[Value] = values.find(s == _.toString) +} diff --git a/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/restapis/SparkRestApiV1.scala b/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/restapis/SparkRestApiV1.scala new file mode 100644 index 0000000000000..7a3b06b1b5e58 --- /dev/null +++ b/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/restapis/SparkRestApiV1.scala @@ -0,0 +1,50 @@ +/* + * 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.kubernetes.integrationtest.restapis + +import java.util.{List => JList} +import javax.ws.rs._ +import javax.ws.rs.core.MediaType + +import org.apache.spark.status.api.v1._ + +@Path("/api/v1") +@Consumes(Array(MediaType.APPLICATION_JSON)) +@Produces(Array(MediaType.APPLICATION_JSON)) +trait SparkRestApiV1 { + + @GET + @Path("/applications") + @Consumes(Array(MediaType.APPLICATION_JSON)) + @Produces(Array(MediaType.APPLICATION_JSON)) + def getApplications( + @QueryParam("status") applicationStatuses: JList[ApplicationStatus]): Seq[ApplicationInfo] + + @GET + @Path("applications/{appId}/stages") + @Consumes(Array(MediaType.APPLICATION_JSON)) + @Produces(Array(MediaType.APPLICATION_JSON)) + def getStages( + @PathParam("appId") appId: String, + @QueryParam("status") statuses: JList[StageStatus]): Seq[StageData] + + @GET + @Path("applications/{appId}/executors") + @Consumes(Array(MediaType.APPLICATION_JSON)) + @Produces(Array(MediaType.APPLICATION_JSON)) + def getExecutors(@PathParam("appId") appId: String): Seq[ExecutorSummary] +} diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitOptionParser.java b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitOptionParser.java index 6767cc5079649..94f9bc319b6a2 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitOptionParser.java +++ b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitOptionParser.java @@ -76,6 +76,12 @@ class SparkSubmitOptionParser { protected final String PRINCIPAL = "--principal"; protected final String QUEUE = "--queue"; + // Kubernetes-only options. + protected final String KUBERNETES_MASTER = "--kubernetes-master"; + protected final String KUBERNETES_NAMESPACE = "--kubernetes-namespace"; + protected final String KUBERNETES_UPLOAD_JARS = "--upload-jars"; + protected final String KUBERNETES_UPLOAD_DRIVER_EXTRA_CLASSPATH = "--upload-driver-extra-classpath"; + /** * This is the canonical list of spark-submit options. Each entry in the array contains the * different aliases for the same option; the first element of each entry is the "official" @@ -115,6 +121,10 @@ class SparkSubmitOptionParser { { REPOSITORIES }, { STATUS }, { TOTAL_EXECUTOR_CORES }, + { KUBERNETES_MASTER }, + { KUBERNETES_NAMESPACE }, + { KUBERNETES_UPLOAD_JARS }, + { KUBERNETES_UPLOAD_DRIVER_EXTRA_CLASSPATH } }; /** diff --git a/pom.xml b/pom.xml index 49f12703c04df..d923941748ca2 100644 --- a/pom.xml +++ b/pom.xml @@ -136,6 +136,7 @@ 10.12.1.1 1.8.1 1.6.0 + 8.18.0 9.2.16.v20160414 3.1.0 0.8.0 @@ -303,6 +304,33 @@ chill-java ${chill.version} + + + com.netflix.feign + feign-core + ${feign.version} + + + com.netflix.feign + feign-okhttp + ${feign.version} + + + com.netflix.feign + feign-jackson + ${feign.version} + + + com.netflix.feign + feign-jaxrs + ${feign.version} + + + com.squareup.okhttp3 + okhttp + 3.4.1 + + @@ -617,6 +645,11 @@ jackson-module-jaxb-annotations ${fasterxml.jackson.version} + + com.fasterxml.jackson.jaxrs + jackson-jaxrs-json-provider + ${fasterxml.jackson.version} + org.glassfish.jersey.core jersey-server @@ -2592,6 +2625,22 @@ + + kubernetes + + kubernetes/core + + + + + kubernetes-integration-tests + + kubernetes/docker-minimal-bundle + kubernetes/integration-tests + kubernetes/integration-tests-spark-jobs + + + hive-thriftserver From 00e545f6c2df74e53ac3446737d960bc86e54f64 Mon Sep 17 00:00:00 2001 From: mcheah Date: Tue, 6 Dec 2016 14:36:57 -0800 Subject: [PATCH 02/81] Fix style --- .../src/main/scala/org/apache/spark/deploy/SparkSubmit.scala | 5 ----- 1 file changed, 5 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 598bafcab81dc..6d37b093a0b6b 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -854,7 +854,6 @@ private[spark] object SparkSubmitUtils { /** * Represents a Maven Coordinate - * * @param groupId the groupId of the coordinate * @param artifactId the artifactId of the coordinate * @param version the version of the coordinate @@ -866,7 +865,6 @@ private[spark] object SparkSubmitUtils { /** * Extracts maven coordinates from a comma-delimited string. Coordinates should be provided * in the format `groupId:artifactId:version` or `groupId/artifactId:version`. - * * @param coordinates Comma-delimited string of maven coordinates * @return Sequence of Maven coordinates */ @@ -897,7 +895,6 @@ private[spark] object SparkSubmitUtils { /** * Extracts maven coordinates from a comma-delimited string - * * @param remoteRepos Comma-delimited string of remote repositories * @param ivySettings The Ivy settings for this session * @return A ChainResolver used by Ivy to search for and resolve dependencies. @@ -962,7 +959,6 @@ private[spark] object SparkSubmitUtils { /** * Output a comma-delimited list of paths for the downloaded jars to be added to the classpath * (will append to jars in SparkSubmit). - * * @param artifacts Sequence of dependencies that were resolved and retrieved * @param cacheDirectory directory where jars are cached * @return a comma-delimited list of paths for the dependencies @@ -1019,7 +1015,6 @@ private[spark] object SparkSubmitUtils { /** * Resolves any dependencies that were supplied through maven coordinates - * * @param coordinates Comma-delimited string of maven coordinates * @param remoteRepos Comma-delimited string of remote repositories other than maven central * @param ivyPath The path to the local ivy repository From cdbd9bb9f96a86470589e137b58a42217837b869 Mon Sep 17 00:00:00 2001 From: mcheah Date: Tue, 6 Dec 2016 17:23:24 -0800 Subject: [PATCH 03/81] Make naming more consistent --- dev/scalastyle | 2 ++ .../org/apache/spark/deploy/kubernetes/Client.scala | 13 ++++++------- .../integrationtest/KubernetesSuite.scala | 2 ++ 3 files changed, 10 insertions(+), 7 deletions(-) diff --git a/dev/scalastyle b/dev/scalastyle index f3dec833636c6..de7423913fad9 100755 --- a/dev/scalastyle +++ b/dev/scalastyle @@ -26,6 +26,8 @@ ERRORS=$(echo -e "q\n" \ -Pyarn \ -Phive \ -Phive-thriftserver \ + -Pkubernetes \ + -Pkubernetes-integration-tests \ scalastyle test:scalastyle \ | awk '{if($1~/error/)print}' \ ) diff --git a/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/Client.scala b/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/Client.scala index 4ee00e8802080..f402b6df82fc4 100644 --- a/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/Client.scala +++ b/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/Client.scala @@ -34,7 +34,7 @@ import scala.concurrent.ExecutionContext import scala.concurrent.duration.DurationInt import scala.util.Success -import org.apache.spark.SparkConf +import org.apache.spark.{SPARK_VERSION, SparkConf} import org.apache.spark.deploy.rest.{AppResource, KubernetesCreateSubmissionRequest, RemoteAppResource, UploadedAppResource} import org.apache.spark.deploy.rest.kubernetes._ import org.apache.spark.internal.Logging @@ -60,9 +60,8 @@ private[spark] class Client( private val secretName = s"spark-submission-server-secret-$kubernetesAppId" private val driverLauncherSelectorValue = s"driver-launcher-$launchTime" - // TODO set precise version by default private val driverDockerImage = sparkConf.get( - "spark.kubernetes.driver.docker.image", "spark-driver:latest") + "spark.kubernetes.driver.docker.image", s"spark-driver:$SPARK_VERSION") private val uploadedDriverExtraClasspath = sparkConf .getOption("spark.kubernetes.driver.uploads.driverExtraClasspath") private val uploadedJars = sparkConf.getOption("spark.kubernetes.driver.uploads.jars") @@ -291,14 +290,14 @@ private[spark] class Client( def getFileContents(maybeFilePaths: Option[String]): Array[(String, String)] = { maybeFilePaths .map(_.split(",").map(filePath => { - val driverExtraClasspathFile = new File(filePath) - if (!driverExtraClasspathFile.isFile) { + val fileToUpload = new File(filePath) + if (!fileToUpload.isFile) { throw new IllegalStateException("Provided file to upload for driver extra classpath" + s" does not exist or is not a file: $filePath") } else { - val fileBytes = Files.toByteArray(driverExtraClasspathFile) + val fileBytes = Files.toByteArray(fileToUpload) val fileBase64 = Base64.encodeBase64String(fileBytes) - (driverExtraClasspathFile.getName, fileBase64) + (fileToUpload.getName, fileBase64) } })).getOrElse(Array.empty[(String, String)]) } diff --git a/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KubernetesSuite.scala b/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KubernetesSuite.scala index d79c75e484af5..3f3d2e609ea4d 100644 --- a/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KubernetesSuite.scala +++ b/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KubernetesSuite.scala @@ -116,6 +116,7 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { .set("spark.kubernetes.submit.clientKeyFile", clientConfig.getClientKeyFile) .set("spark.kubernetes.submit.clientCertFile", clientConfig.getClientCertFile) .set("spark.kubernetes.namespace", NAMESPACE) + .set("spark.kubernetes.driver.docker.image", "spark-driver:latest") .set("spark.kubernetes.executor.docker.image", "spark-executor:latest") .set("spark.executor.memory", "500m") .set("spark.executor.cores", "1") @@ -148,6 +149,7 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { "--conf", s"spark.kubernetes.submit.clientKeyFile=${clientConfig.getClientKeyFile}", "--conf", s"spark.kubernetes.submit.clientCertFile=${clientConfig.getClientCertFile}", "--conf", "spark.kubernetes.executor.docker.image=spark-executor:latest", + "--conf", "spark.kubernetes.driver.docker.image=spark-driver:latest", EXAMPLES_JAR) SparkSubmit.main(args) val sparkMetricsService = Minikube.getService[SparkRestApiV1]( From 8f69fc0bda56f898ebf97d32d49e7c3ec2e34d04 Mon Sep 17 00:00:00 2001 From: mcheah Date: Fri, 9 Dec 2016 14:55:13 -0800 Subject: [PATCH 04/81] Fix building assembly with Kubernetes. --- assembly/pom.xml | 10 ++++++++++ .../kubernetes/integrationtest/minikube/Minikube.scala | 2 +- 2 files changed, 11 insertions(+), 1 deletion(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index aebfd12227751..1819fe404a1d9 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -148,6 +148,16 @@ + + kubernetes + + + org.apache.spark + spark-kubernetes_${scala.binary.version} + ${project.version} + + + hive diff --git a/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/minikube/Minikube.scala b/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/minikube/Minikube.scala index 92b809a4c7c59..60c6564579a6e 100644 --- a/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/minikube/Minikube.scala +++ b/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/minikube/Minikube.scala @@ -21,8 +21,8 @@ import java.nio.file.Paths import java.util.concurrent.TimeUnit import javax.net.ssl.X509TrustManager -import io.fabric8.kubernetes.client.internal.SSLUtils import io.fabric8.kubernetes.client.{ConfigBuilder, DefaultKubernetesClient} +import io.fabric8.kubernetes.client.internal.SSLUtils import scala.collection.mutable.ArrayBuffer import scala.reflect.ClassTag From 75c6086aa52f0a98345b4fa63fde654740ddd28b Mon Sep 17 00:00:00 2001 From: mcheah Date: Fri, 9 Dec 2016 16:18:11 -0800 Subject: [PATCH 05/81] Service account support, use constants from fabric8 library. --- .../scala/org/apache/spark/deploy/kubernetes/Client.scala | 7 +++++-- .../spark/deploy/kubernetes/KubernetesClientBuilder.scala | 6 +++--- .../kubernetes/integrationtest/KubernetesSuite.scala | 6 +++--- 3 files changed, 11 insertions(+), 8 deletions(-) diff --git a/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/Client.scala b/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/Client.scala index f402b6df82fc4..cea90a51386b5 100644 --- a/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/Client.scala +++ b/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/Client.scala @@ -55,8 +55,8 @@ private[spark] class Client( private val launchTime = System.currentTimeMillis private val kubernetesAppId = sparkConf.getOption("spark.app.name") - .orElse(sparkConf.getOption("spark.app.id")) - .getOrElse(s"spark-$launchTime") + .orElse(sparkConf.getOption("spark.app.id")) + .getOrElse(s"spark-$launchTime") private val secretName = s"spark-submission-server-secret-$kubernetesAppId" private val driverLauncherSelectorValue = s"driver-launcher-$launchTime" @@ -69,6 +69,8 @@ private[spark] class Client( private val secretBytes = new Array[Byte](128) SECURE_RANDOM.nextBytes(secretBytes) private val secretBase64String = Base64.encodeBase64String(secretBytes) + private val serviceAccount = sparkConf.get("spark.kubernetes.submit.serviceAccountName", + "default") private implicit val retryableExecutionContext = ExecutionContext .fromExecutorService( @@ -191,6 +193,7 @@ private[spark] class Client( .withSecretName(secret.getMetadata.getName) .endSecret() .endVolume + .withServiceAccount(serviceAccount) .addNewContainer() .withName(DRIVER_LAUNCHER_CONTAINER_NAME) .withImage(driverDockerImage) diff --git a/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/KubernetesClientBuilder.scala b/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/KubernetesClientBuilder.scala index 4c715c86cc7f9..61a13dc7274d7 100644 --- a/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/KubernetesClientBuilder.scala +++ b/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/KubernetesClientBuilder.scala @@ -20,11 +20,11 @@ import java.io.File import com.google.common.base.Charsets import com.google.common.io.Files -import io.fabric8.kubernetes.client.{ConfigBuilder, DefaultKubernetesClient} +import io.fabric8.kubernetes.client.{Config, ConfigBuilder, DefaultKubernetesClient} private[spark] object KubernetesClientBuilder { - private val API_SERVER_TOKEN = new File("/var/run/secrets/kubernetes.io/serviceaccount/token") - private val CA_CERT_FILE = new File("/var/run/secrets/kubernetes.io/serviceaccount/ca.crt") + private val API_SERVER_TOKEN = new File(Config.KUBERNETES_SERVICE_ACCOUNT_TOKEN_PATH) + private val CA_CERT_FILE = new File(Config.KUBERNETES_SERVICE_ACCOUNT_CA_CRT_PATH) /** * Creates a {@link KubernetesClient}, expecting to be from diff --git a/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KubernetesSuite.scala b/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KubernetesSuite.scala index 3f3d2e609ea4d..902631b874539 100644 --- a/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KubernetesSuite.scala +++ b/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KubernetesSuite.scala @@ -37,9 +37,9 @@ import org.apache.spark.status.api.v1.{ApplicationStatus, StageStatus} private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { private val EXAMPLES_JAR = Paths.get("target", "integration-tests-spark-jobs") - .toFile - .listFiles()(0) - .getAbsolutePath + .toFile + .listFiles()(0) + .getAbsolutePath private val TIMEOUT = PatienceConfiguration.Timeout(Span(2, Minutes)) private val INTERVAL = PatienceConfiguration.Interval(Span(2, Seconds)) From 93b75cea65e8c5d98e4d9d78a541659d927fceea Mon Sep 17 00:00:00 2001 From: mcheah Date: Fri, 6 Jan 2017 16:15:35 -0800 Subject: [PATCH 06/81] Some small changes - Don't hold the raw secret bytes - Add CPU limits and requests --- .../spark/deploy/kubernetes/Client.scala | 39 +++++++++++-------- .../KubernetesClusterSchedulerBackend.scala | 9 ++++- 2 files changed, 29 insertions(+), 19 deletions(-) diff --git a/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/Client.scala b/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/Client.scala index cea90a51386b5..21c83dbf40e21 100644 --- a/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/Client.scala +++ b/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/Client.scala @@ -66,9 +66,12 @@ private[spark] class Client( .getOption("spark.kubernetes.driver.uploads.driverExtraClasspath") private val uploadedJars = sparkConf.getOption("spark.kubernetes.driver.uploads.jars") - private val secretBytes = new Array[Byte](128) - SECURE_RANDOM.nextBytes(secretBytes) - private val secretBase64String = Base64.encodeBase64String(secretBytes) + private val secretBase64String = { + val secretBytes = new Array[Byte](128) + SECURE_RANDOM.nextBytes(secretBytes) + Base64.encodeBase64String(secretBytes) + } + private val serviceAccount = sparkConf.get("spark.kubernetes.submit.serviceAccountName", "default") @@ -105,11 +108,7 @@ private[spark] class Client( .done() try { val selectors = Map(DRIVER_LAUNCHER_SELECTOR_LABEL -> driverLauncherSelectorValue).asJava - val uiPort = sparkConf - .getOption("spark.ui.port") - .map(_.toInt) - .getOrElse(DEFAULT_UI_PORT) - val (servicePorts, containerPorts) = configurePorts(uiPort) + val (servicePorts, containerPorts) = configurePorts() val service = kubernetesClient.services().createNew() .withNewMetadata() .withName(kubernetesAppId) @@ -120,11 +119,11 @@ private[spark] class Client( .endSpec() .done() sparkConf.set("spark.kubernetes.driver.service.name", service.getMetadata.getName) - sparkConf.setIfMissing("spark.driver.port", DRIVER_PORT.toString) - sparkConf.setIfMissing("spark.blockmanager.port", BLOCKMANAGER_PORT.toString) + sparkConf.setIfMissing("spark.driver.port", DEFAULT_DRIVER_PORT.toString) + sparkConf.setIfMissing("spark.blockmanager.port", DEFAULT_BLOCKMANAGER_PORT.toString) val submitRequest = buildSubmissionRequest() val submitCompletedFuture = SettableFuture.create[Boolean] - val secretDirectory = s"/var/run/secrets/spark-submission/$kubernetesAppId" + val secretDirectory = s"$SPARK_SUBMISSION_SECRET_BASE_DIR/$kubernetesAppId" val podWatcher = new Watcher[Pod] { override def eventReceived(action: Action, t: Pod): Unit = { @@ -228,7 +227,7 @@ private[spark] class Client( }) } - private def configurePorts(uiPort: Int): (Seq[ServicePort], Seq[ContainerPort]) = { + private def configurePorts(): (Seq[ServicePort], Seq[ContainerPort]) = { val servicePorts = new ArrayBuffer[ServicePort] val containerPorts = new ArrayBuffer[ContainerPort] @@ -251,15 +250,20 @@ private[spark] class Client( sparkConf .getOption("spark.driver.port") .map(_.toInt) - .getOrElse(DRIVER_PORT)) + .getOrElse(DEFAULT_DRIVER_PORT)) addPortToServiceAndContainer( BLOCKMANAGER_PORT_NAME, sparkConf .getOption("spark.blockmanager.port") .map(_.toInt) - .getOrElse(BLOCKMANAGER_PORT)) + .getOrElse(DEFAULT_BLOCKMANAGER_PORT)) - addPortToServiceAndContainer(UI_PORT_NAME, uiPort) + addPortToServiceAndContainer( + UI_PORT_NAME, + sparkConf + .getOption("spark.ui.port") + .map(_.toInt) + .getOrElse(DEFAULT_UI_PORT)) (servicePorts.toSeq, containerPorts.toSeq) } @@ -331,8 +335,8 @@ private object Client { private val SUBMISSION_SERVER_SECRET_NAME = "spark-submission-server-secret" private val DRIVER_LAUNCHER_SELECTOR_LABEL = "driver-launcher-selector" private val DRIVER_LAUNCHER_SERVICE_INTERNAL_PORT = 7077 - private val DRIVER_PORT = 7078 - private val BLOCKMANAGER_PORT = 7079 + private val DEFAULT_DRIVER_PORT = 7078 + private val DEFAULT_BLOCKMANAGER_PORT = 7079 private val DEFAULT_UI_PORT = 4040 private val UI_PORT_NAME = "spark-ui-port" private val DRIVER_LAUNCHER_SERVICE_PORT_NAME = "driver-launcher-port" @@ -340,6 +344,7 @@ private object Client { private val BLOCKMANAGER_PORT_NAME = "block-manager-port" private val DRIVER_LAUNCHER_CONTAINER_NAME = "spark-kubernetes-driver-launcher" private val SECURE_RANDOM = new SecureRandom() + private val SPARK_SUBMISSION_SECRET_BASE_DIR = "/var/run/secrets/spark-submission" def main(args: Array[String]): Unit = { require(args.length >= 2, s"Too few arguments. Usage: ${getClass.getName} " + diff --git a/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala b/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala index f37b97e4dd0dc..bbc95d4f4b7e3 100644 --- a/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala +++ b/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala @@ -34,8 +34,8 @@ import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend import org.apache.spark.util.Utils private[spark] class KubernetesClusterSchedulerBackend( - scheduler: TaskSchedulerImpl, - val sc: SparkContext) + scheduler: TaskSchedulerImpl, + val sc: SparkContext) extends CoarseGrainedSchedulerBackend(scheduler, sc.env.rpcEnv) { import KubernetesClusterSchedulerBackend._ @@ -167,6 +167,9 @@ private[spark] class KubernetesClusterSchedulerBackend( val executorMemoryLimitQuantity = new QuantityBuilder(false) .withAmount(executorMemoryWithOverhead.toString) .build() + val executorCpuQuantity = new QuantityBuilder(false) + .withAmount(executorCores) + .build() val requiredEnv = new ArrayBuffer[EnvVar] requiredEnv += new EnvVarBuilder() .withName("SPARK_EXECUTOR_PORT") @@ -214,6 +217,8 @@ private[spark] class KubernetesClusterSchedulerBackend( .withNewResources() .addToRequests("memory", executorMemoryQuantity) .addToLimits("memory", executorMemoryLimitQuantity) + .addToRequests("cpu", executorCpuQuantity) + .addToLimits("cpu", executorCpuQuantity) .endResources() .withEnv(requiredEnv.asJava) .withPorts(requiredPorts.asJava) From e7397e814f98570a9bfefa66c34a1dd67f53865b Mon Sep 17 00:00:00 2001 From: mcheah Date: Mon, 9 Jan 2017 14:25:49 -0800 Subject: [PATCH 07/81] Use k8s:// formatted URL instead of separate setting. --- .../main/scala/org/apache/spark/deploy/SparkSubmit.scala | 5 +---- .../org/apache/spark/deploy/SparkSubmitArguments.scala | 7 ------- .../scala/org/apache/spark/deploy/kubernetes/Client.scala | 8 +++++--- .../cluster/kubernetes/KubernetesClusterManager.scala | 2 +- .../kubernetes/KubernetesClusterSchedulerBackend.scala | 5 +---- .../kubernetes/integrationtest/KubernetesSuite.scala | 6 ++---- 6 files changed, 10 insertions(+), 23 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 6d37b093a0b6b..bd249ea377b65 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -240,7 +240,7 @@ object SparkSubmit { YARN case m if m.startsWith("spark") => STANDALONE case m if m.startsWith("mesos") => MESOS - case m if m.startsWith("kubernetes") => KUBERNETES + case m if m.startsWith("k8s") => KUBERNETES case m if m.startsWith("local") => LOCAL case _ => printErrorAndExit("Master must either be yarn or start with spark, mesos, local") @@ -470,9 +470,6 @@ object SparkSubmit { OptionAssigner(args.principal, YARN, ALL_DEPLOY_MODES, sysProp = "spark.yarn.principal"), OptionAssigner(args.keytab, YARN, ALL_DEPLOY_MODES, sysProp = "spark.yarn.keytab"), - // Kubernetes only - OptionAssigner(args.kubernetesMaster, KUBERNETES, ALL_DEPLOY_MODES, - sysProp = "spark.kubernetes.master"), OptionAssigner(args.kubernetesNamespace, KUBERNETES, ALL_DEPLOY_MODES, sysProp = "spark.kubernetes.namespace"), OptionAssigner(args.kubernetesUploadJars, KUBERNETES, CLUSTER, diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala index 4244742aad14c..d80f79332111f 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -72,7 +72,6 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S var keytab: String = null // Kubernetes only - var kubernetesMaster: String = null var kubernetesNamespace: String = null var kubernetesUploadJars: String = null var kubernetesUploadDriverExtraClasspath: String = null @@ -192,9 +191,6 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S .getOrElse(sparkProperties.get("spark.executor.instances").orNull) keytab = Option(keytab).orElse(sparkProperties.get("spark.yarn.keytab")).orNull principal = Option(principal).orElse(sparkProperties.get("spark.yarn.principal")).orNull - kubernetesMaster = Option(kubernetesMaster) - .orElse(sparkProperties.get("spark.kubernetes.master")) - .orNull kubernetesNamespace = Option(kubernetesNamespace) .orElse(sparkProperties.get("spark.kubernetes.namespace")) .orNull @@ -444,9 +440,6 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S case KEYTAB => keytab = value - case KUBERNETES_MASTER => - kubernetesMaster = value - case KUBERNETES_NAMESPACE => kubernetesNamespace = value diff --git a/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/Client.scala b/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/Client.scala index 21c83dbf40e21..0715c84495a2c 100644 --- a/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/Client.scala +++ b/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/Client.scala @@ -49,9 +49,11 @@ private[spark] class Client( private val namespace = sparkConf.getOption("spark.kubernetes.namespace").getOrElse( throw new IllegalArgumentException("Namespace must be provided in spark.kubernetes.namespace")) - private val master = sparkConf - .getOption("spark.kubernetes.master") - .getOrElse("Master must be provided in spark.kubernetes.master") + private val rawMaster = sparkConf.get("spark.master") + if (!rawMaster.startsWith("k8s://")) { + throw new IllegalArgumentException("Master should be a URL with scheme k8s://") + } + private val master = rawMaster.replaceFirst("k8s://", "") private val launchTime = System.currentTimeMillis private val kubernetesAppId = sparkConf.getOption("spark.app.name") diff --git a/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterManager.scala b/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterManager.scala index 0d3b97c636ca3..36f7149a832c3 100644 --- a/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterManager.scala +++ b/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterManager.scala @@ -21,7 +21,7 @@ import org.apache.spark.scheduler.{ExternalClusterManager, SchedulerBackend, Tas private[spark] class KubernetesClusterManager extends ExternalClusterManager { - override def canCreate(masterURL: String): Boolean = masterURL.startsWith("kubernetes") + override def canCreate(masterURL: String): Boolean = masterURL.startsWith("k8s") override def createTaskScheduler(sc: SparkContext, masterURL: String): TaskScheduler = { val scheduler = new TaskSchedulerImpl(sc) diff --git a/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala b/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala index bbc95d4f4b7e3..4e099cea3198b 100644 --- a/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala +++ b/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala @@ -43,10 +43,7 @@ private[spark] class KubernetesClusterSchedulerBackend( private val EXECUTOR_MODIFICATION_LOCK = new Object private val runningExecutorPods = new scala.collection.mutable.HashMap[String, Pod] - private val kubernetesMaster = conf - .getOption("spark.kubernetes.master") - .getOrElse( - throw new SparkException("Kubernetes master must be specified in kubernetes mode.")) + private val kubernetesMaster = sc.master.replaceFirst("k8s://", "") private val executorDockerImage = conf .get("spark.kubernetes.executor.docker.image", s"spark-executor:${sc.version}") diff --git a/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KubernetesSuite.scala b/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KubernetesSuite.scala index 902631b874539..183f666994d38 100644 --- a/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KubernetesSuite.scala +++ b/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KubernetesSuite.scala @@ -110,8 +110,7 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { test("Run a simple example") { val sparkConf = new SparkConf(true) - .setMaster("kubernetes") - .set("spark.kubernetes.master", s"https://${Minikube.getMinikubeIp}:8443") + .setMaster(s"k8s://https://${Minikube.getMinikubeIp}:8443") .set("spark.kubernetes.submit.caCertFile", clientConfig.getCaCertFile) .set("spark.kubernetes.submit.clientKeyFile", clientConfig.getClientKeyFile) .set("spark.kubernetes.submit.clientCertFile", clientConfig.getClientCertFile) @@ -136,9 +135,8 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { test("Run using spark-submit") { val args = Array( - "--master", "kubernetes", + "--master", s"k8s://https://${Minikube.getMinikubeIp}:8443", "--deploy-mode", "cluster", - "--kubernetes-master", s"https://${Minikube.getMinikubeIp}:8443", "--kubernetes-namespace", NAMESPACE, "--name", "spark-pi", "--executor-memory", "512m", From ed65428d64eabc3da71b8ea5f5be6ba5dac913f6 Mon Sep 17 00:00:00 2001 From: Anirudh Ramanathan Date: Mon, 9 Jan 2017 14:33:41 -0800 Subject: [PATCH 08/81] Reindent comment to conforn to JavaDoc style The build process fails ScalaStyle checks otherwise. --- .../deploy/kubernetes/KubernetesClientBuilder.scala | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/KubernetesClientBuilder.scala b/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/KubernetesClientBuilder.scala index 61a13dc7274d7..61d3ac17ac34a 100644 --- a/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/KubernetesClientBuilder.scala +++ b/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/KubernetesClientBuilder.scala @@ -27,11 +27,11 @@ private[spark] object KubernetesClientBuilder { private val CA_CERT_FILE = new File(Config.KUBERNETES_SERVICE_ACCOUNT_CA_CRT_PATH) /** - * Creates a {@link KubernetesClient}, expecting to be from - * within the context of a pod. When doing so, credentials files - * are picked up from canonical locations, as they are injected - * into the pod's disk space. - */ + * Creates a {@link KubernetesClient}, expecting to be from + * within the context of a pod. When doing so, credentials files + * are picked up from canonical locations, as they are injected + * into the pod's disk space. + */ def buildFromWithinPod( kubernetesMaster: String, kubernetesNamespace: String): DefaultKubernetesClient = { From f9ddb633d56561bb4272cd8888b64905362c5379 Mon Sep 17 00:00:00 2001 From: mcheah Date: Mon, 9 Jan 2017 15:30:02 -0800 Subject: [PATCH 09/81] Move kubernetes under resource-managers folder. --- .../main/scala/org/apache/spark/deploy/SparkSubmit.scala | 2 +- pom.xml | 8 ++++---- {kubernetes => resource-managers/kubernetes}/core/pom.xml | 4 ++-- .../org.apache.spark.scheduler.ExternalClusterManager | 0 .../scala/org/apache/spark/deploy/kubernetes/Client.scala | 0 .../spark/deploy/kubernetes/KubernetesClientBuilder.scala | 0 .../scala/org/apache/spark/deploy/kubernetes/Retry.scala | 0 .../deploy/rest/KubernetesRestProtocolMessages.scala | 0 .../spark/deploy/rest/kubernetes/HttpClientUtil.scala | 0 .../deploy/rest/kubernetes/KubernetesSparkRestApi.scala | 0 .../rest/kubernetes/KubernetesSparkRestServer.scala | 0 .../cluster/kubernetes/KubernetesClusterManager.scala | 0 .../kubernetes/KubernetesClusterSchedulerBackend.scala | 0 .../kubernetes}/docker-minimal-bundle/pom.xml | 4 ++-- .../src/main/assembly/driver-assembly.xml | 0 .../src/main/assembly/executor-assembly.xml | 0 .../src/main/docker/driver/Dockerfile | 0 .../src/main/docker/executor/Dockerfile | 0 .../kubernetes}/integration-tests-spark-jobs/pom.xml | 4 ++-- .../integrationtest/jobs/SparkPiWithInfiniteWait.scala | 0 .../kubernetes}/integration-tests/pom.xml | 4 ++-- .../kubernetes/integrationtest/KubernetesSuite.scala | 0 .../integrationtest/docker/SparkDockerImageBuilder.scala | 0 .../kubernetes/integrationtest/minikube/Minikube.scala | 0 .../integrationtest/restapis/SparkRestApiV1.scala | 0 25 files changed, 13 insertions(+), 13 deletions(-) rename {kubernetes => resource-managers/kubernetes}/core/pom.xml (97%) rename {kubernetes => resource-managers/kubernetes}/core/src/main/resources/META-INF/services/org.apache.spark.scheduler.ExternalClusterManager (100%) rename {kubernetes => resource-managers/kubernetes}/core/src/main/scala/org/apache/spark/deploy/kubernetes/Client.scala (100%) rename {kubernetes => resource-managers/kubernetes}/core/src/main/scala/org/apache/spark/deploy/kubernetes/KubernetesClientBuilder.scala (100%) rename {kubernetes => resource-managers/kubernetes}/core/src/main/scala/org/apache/spark/deploy/kubernetes/Retry.scala (100%) rename {kubernetes => resource-managers/kubernetes}/core/src/main/scala/org/apache/spark/deploy/rest/KubernetesRestProtocolMessages.scala (100%) rename {kubernetes => resource-managers/kubernetes}/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/HttpClientUtil.scala (100%) rename {kubernetes => resource-managers/kubernetes}/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/KubernetesSparkRestApi.scala (100%) rename {kubernetes => resource-managers/kubernetes}/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/KubernetesSparkRestServer.scala (100%) rename {kubernetes => resource-managers/kubernetes}/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterManager.scala (100%) rename {kubernetes => resource-managers/kubernetes}/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala (100%) rename {kubernetes => resource-managers/kubernetes}/docker-minimal-bundle/pom.xml (98%) rename {kubernetes => resource-managers/kubernetes}/docker-minimal-bundle/src/main/assembly/driver-assembly.xml (100%) rename {kubernetes => resource-managers/kubernetes}/docker-minimal-bundle/src/main/assembly/executor-assembly.xml (100%) rename {kubernetes => resource-managers/kubernetes}/docker-minimal-bundle/src/main/docker/driver/Dockerfile (100%) rename {kubernetes => resource-managers/kubernetes}/docker-minimal-bundle/src/main/docker/executor/Dockerfile (100%) rename {kubernetes => resource-managers/kubernetes}/integration-tests-spark-jobs/pom.xml (95%) rename {kubernetes => resource-managers/kubernetes}/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/kubernetes/integrationtest/jobs/SparkPiWithInfiniteWait.scala (100%) rename {kubernetes => resource-managers/kubernetes}/integration-tests/pom.xml (98%) rename {kubernetes => resource-managers/kubernetes}/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KubernetesSuite.scala (100%) rename {kubernetes => resource-managers/kubernetes}/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/docker/SparkDockerImageBuilder.scala (100%) rename {kubernetes => resource-managers/kubernetes}/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/minikube/Minikube.scala (100%) rename {kubernetes => resource-managers/kubernetes}/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/restapis/SparkRestApiV1.scala (100%) 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 bd249ea377b65..5be2d8a52d84c 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -243,7 +243,7 @@ object SparkSubmit { case m if m.startsWith("k8s") => KUBERNETES case m if m.startsWith("local") => LOCAL case _ => - printErrorAndExit("Master must either be yarn or start with spark, mesos, local") + printErrorAndExit("Master must either be yarn or start with spark, mesos, k8s, or local") -1 } diff --git a/pom.xml b/pom.xml index d923941748ca2..9ba31b4d78016 100644 --- a/pom.xml +++ b/pom.xml @@ -2628,16 +2628,16 @@ kubernetes - kubernetes/core + resource-managers/kubernetes/core kubernetes-integration-tests - kubernetes/docker-minimal-bundle - kubernetes/integration-tests - kubernetes/integration-tests-spark-jobs + resource-managers/kubernetes/docker-minimal-bundle + resource-managers/kubernetes/integration-tests + resource-managers/kubernetes/integration-tests-spark-jobs diff --git a/kubernetes/core/pom.xml b/resource-managers/kubernetes/core/pom.xml similarity index 97% rename from kubernetes/core/pom.xml rename to resource-managers/kubernetes/core/pom.xml index 9c7eb52b2680a..388defd93465d 100644 --- a/kubernetes/core/pom.xml +++ b/resource-managers/kubernetes/core/pom.xml @@ -20,8 +20,8 @@ org.apache.spark spark-parent_2.11 - 2.1.0-SNAPSHOT - ../../pom.xml + 2.2.0-SNAPSHOT + ../../../pom.xml spark-kubernetes_2.11 diff --git a/kubernetes/core/src/main/resources/META-INF/services/org.apache.spark.scheduler.ExternalClusterManager b/resource-managers/kubernetes/core/src/main/resources/META-INF/services/org.apache.spark.scheduler.ExternalClusterManager similarity index 100% rename from kubernetes/core/src/main/resources/META-INF/services/org.apache.spark.scheduler.ExternalClusterManager rename to resource-managers/kubernetes/core/src/main/resources/META-INF/services/org.apache.spark.scheduler.ExternalClusterManager diff --git a/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/Client.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/Client.scala similarity index 100% rename from kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/Client.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/Client.scala diff --git a/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/KubernetesClientBuilder.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/KubernetesClientBuilder.scala similarity index 100% rename from kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/KubernetesClientBuilder.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/KubernetesClientBuilder.scala diff --git a/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/Retry.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/Retry.scala similarity index 100% rename from kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/Retry.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/Retry.scala diff --git a/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/KubernetesRestProtocolMessages.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/KubernetesRestProtocolMessages.scala similarity index 100% rename from kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/KubernetesRestProtocolMessages.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/KubernetesRestProtocolMessages.scala diff --git a/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/HttpClientUtil.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/HttpClientUtil.scala similarity index 100% rename from kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/HttpClientUtil.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/HttpClientUtil.scala diff --git a/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/KubernetesSparkRestApi.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/KubernetesSparkRestApi.scala similarity index 100% rename from kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/KubernetesSparkRestApi.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/KubernetesSparkRestApi.scala diff --git a/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/KubernetesSparkRestServer.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/KubernetesSparkRestServer.scala similarity index 100% rename from kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/KubernetesSparkRestServer.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/KubernetesSparkRestServer.scala diff --git a/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterManager.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterManager.scala similarity index 100% rename from kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterManager.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterManager.scala diff --git a/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala similarity index 100% rename from kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala diff --git a/kubernetes/docker-minimal-bundle/pom.xml b/resource-managers/kubernetes/docker-minimal-bundle/pom.xml similarity index 98% rename from kubernetes/docker-minimal-bundle/pom.xml rename to resource-managers/kubernetes/docker-minimal-bundle/pom.xml index 3de939ea3978a..c20e51c93e7c7 100644 --- a/kubernetes/docker-minimal-bundle/pom.xml +++ b/resource-managers/kubernetes/docker-minimal-bundle/pom.xml @@ -21,8 +21,8 @@ org.apache.spark spark-parent_2.11 - 2.1.0-SNAPSHOT - ../pom.xml + 2.2.0-SNAPSHOT + ../../../pom.xml spark-docker-minimal-bundle_2.11 diff --git a/kubernetes/docker-minimal-bundle/src/main/assembly/driver-assembly.xml b/resource-managers/kubernetes/docker-minimal-bundle/src/main/assembly/driver-assembly.xml similarity index 100% rename from kubernetes/docker-minimal-bundle/src/main/assembly/driver-assembly.xml rename to resource-managers/kubernetes/docker-minimal-bundle/src/main/assembly/driver-assembly.xml diff --git a/kubernetes/docker-minimal-bundle/src/main/assembly/executor-assembly.xml b/resource-managers/kubernetes/docker-minimal-bundle/src/main/assembly/executor-assembly.xml similarity index 100% rename from kubernetes/docker-minimal-bundle/src/main/assembly/executor-assembly.xml rename to resource-managers/kubernetes/docker-minimal-bundle/src/main/assembly/executor-assembly.xml diff --git a/kubernetes/docker-minimal-bundle/src/main/docker/driver/Dockerfile b/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/driver/Dockerfile similarity index 100% rename from kubernetes/docker-minimal-bundle/src/main/docker/driver/Dockerfile rename to resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/driver/Dockerfile diff --git a/kubernetes/docker-minimal-bundle/src/main/docker/executor/Dockerfile b/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/executor/Dockerfile similarity index 100% rename from kubernetes/docker-minimal-bundle/src/main/docker/executor/Dockerfile rename to resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/executor/Dockerfile diff --git a/kubernetes/integration-tests-spark-jobs/pom.xml b/resource-managers/kubernetes/integration-tests-spark-jobs/pom.xml similarity index 95% rename from kubernetes/integration-tests-spark-jobs/pom.xml rename to resource-managers/kubernetes/integration-tests-spark-jobs/pom.xml index 17f1c4906214f..12b0234ae71bd 100644 --- a/kubernetes/integration-tests-spark-jobs/pom.xml +++ b/resource-managers/kubernetes/integration-tests-spark-jobs/pom.xml @@ -20,8 +20,8 @@ org.apache.spark spark-parent_2.11 - 2.1.0-SNAPSHOT - ../../pom.xml + 2.2.0-SNAPSHOT + ../../../pom.xml spark-kubernetes-integration-tests-spark-jobs_2.11 diff --git a/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/kubernetes/integrationtest/jobs/SparkPiWithInfiniteWait.scala b/resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/kubernetes/integrationtest/jobs/SparkPiWithInfiniteWait.scala similarity index 100% rename from kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/kubernetes/integrationtest/jobs/SparkPiWithInfiniteWait.scala rename to resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/kubernetes/integrationtest/jobs/SparkPiWithInfiniteWait.scala diff --git a/kubernetes/integration-tests/pom.xml b/resource-managers/kubernetes/integration-tests/pom.xml similarity index 98% rename from kubernetes/integration-tests/pom.xml rename to resource-managers/kubernetes/integration-tests/pom.xml index 0568cb1e21826..1e7eb0e12e6df 100644 --- a/kubernetes/integration-tests/pom.xml +++ b/resource-managers/kubernetes/integration-tests/pom.xml @@ -20,8 +20,8 @@ org.apache.spark spark-parent_2.11 - 2.1.0-SNAPSHOT - ../../pom.xml + 2.2.0-SNAPSHOT + ../../../pom.xml spark-kubernetes-integration-tests_2.11 diff --git a/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KubernetesSuite.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KubernetesSuite.scala similarity index 100% rename from kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KubernetesSuite.scala rename to resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KubernetesSuite.scala diff --git a/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/docker/SparkDockerImageBuilder.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/docker/SparkDockerImageBuilder.scala similarity index 100% rename from kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/docker/SparkDockerImageBuilder.scala rename to resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/docker/SparkDockerImageBuilder.scala diff --git a/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/minikube/Minikube.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/minikube/Minikube.scala similarity index 100% rename from kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/minikube/Minikube.scala rename to resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/minikube/Minikube.scala diff --git a/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/restapis/SparkRestApiV1.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/restapis/SparkRestApiV1.scala similarity index 100% rename from kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/restapis/SparkRestApiV1.scala rename to resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/restapis/SparkRestApiV1.scala From 178abc1375d1c56d4d7801ea355a19a4704e5dcc Mon Sep 17 00:00:00 2001 From: mccheah Date: Wed, 11 Jan 2017 14:36:45 -0800 Subject: [PATCH 10/81] Use tar and gzip to compress+archive shipped jars (#2) * Use tar and gzip to archive shipped jars. * Address comments * Move files to resolve merge --- pom.xml | 1 + .../spark/deploy/kubernetes/Client.scala | 21 +-- .../rest/KubernetesRestProtocolMessages.scala | 13 +- .../rest/kubernetes/CompressionUtils.scala | 139 ++++++++++++++++++ .../KubernetesSparkRestServer.scala | 27 +--- .../pom.xml | 33 +++++ .../kubernetes/integrationtest/PiHelper.java | 33 +++++ .../integration-tests-spark-jobs/pom.xml | 6 + .../jobs/SparkPiWithInfiniteWait.scala | 9 +- .../kubernetes/integration-tests/pom.xml | 13 ++ .../integrationtest/KubernetesSuite.scala | 7 + 11 files changed, 254 insertions(+), 48 deletions(-) create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/CompressionUtils.scala create mode 100644 resource-managers/kubernetes/integration-tests-spark-jobs-helpers/pom.xml create mode 100644 resource-managers/kubernetes/integration-tests-spark-jobs-helpers/src/main/java/org/apache/spark/deploy/kubernetes/integrationtest/PiHelper.java diff --git a/pom.xml b/pom.xml index 9ba31b4d78016..d04d7623d0584 100644 --- a/pom.xml +++ b/pom.xml @@ -2638,6 +2638,7 @@ resource-managers/kubernetes/docker-minimal-bundle resource-managers/kubernetes/integration-tests resource-managers/kubernetes/integration-tests-spark-jobs + resource-managers/kubernetes/integration-tests-spark-jobs-helpers diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/Client.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/Client.scala index 0715c84495a2c..230598d63bed1 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/Client.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/Client.scala @@ -35,7 +35,7 @@ import scala.concurrent.duration.DurationInt import scala.util.Success import org.apache.spark.{SPARK_VERSION, SparkConf} -import org.apache.spark.deploy.rest.{AppResource, KubernetesCreateSubmissionRequest, RemoteAppResource, UploadedAppResource} +import org.apache.spark.deploy.rest.{AppResource, KubernetesCreateSubmissionRequest, RemoteAppResource, TarGzippedData, UploadedAppResource} import org.apache.spark.deploy.rest.kubernetes._ import org.apache.spark.internal.Logging import org.apache.spark.util.Utils @@ -284,8 +284,8 @@ private[spark] class Client( case other => RemoteAppResource(other) } - val uploadDriverExtraClasspathBase64Contents = getFileContents(uploadedDriverExtraClasspath) - val uploadJarsBase64Contents = getFileContents(uploadedJars) + val uploadDriverExtraClasspathBase64Contents = compressJars(uploadedDriverExtraClasspath) + val uploadJarsBase64Contents = compressJars(uploadedJars) KubernetesCreateSubmissionRequest( appResource = resolvedAppResource, mainClass = mainClass, @@ -296,19 +296,10 @@ private[spark] class Client( uploadedJarsBase64Contents = uploadJarsBase64Contents) } - def getFileContents(maybeFilePaths: Option[String]): Array[(String, String)] = { + def compressJars(maybeFilePaths: Option[String]): Option[TarGzippedData] = { maybeFilePaths - .map(_.split(",").map(filePath => { - val fileToUpload = new File(filePath) - if (!fileToUpload.isFile) { - throw new IllegalStateException("Provided file to upload for driver extra classpath" + - s" does not exist or is not a file: $filePath") - } else { - val fileBytes = Files.toByteArray(fileToUpload) - val fileBase64 = Base64.encodeBase64String(fileBytes) - (fileToUpload.getName, fileBase64) - } - })).getOrElse(Array.empty[(String, String)]) + .map(_.split(",")) + .map(CompressionUtils.createTarGzip(_)) } private def getDriverLauncherService( diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/KubernetesRestProtocolMessages.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/KubernetesRestProtocolMessages.scala index 4b7bb66083f29..6da1a848b25e7 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/KubernetesRestProtocolMessages.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/KubernetesRestProtocolMessages.scala @@ -27,14 +27,19 @@ case class KubernetesCreateSubmissionRequest( val appArgs: Array[String], val sparkProperties: Map[String, String], val secret: String, - val uploadedDriverExtraClasspathBase64Contents: Array[(String, String)] - = Array.empty[(String, String)], - val uploadedJarsBase64Contents: Array[(String, String)] - = Array.empty[(String, String)]) extends SubmitRestProtocolRequest { + val uploadedDriverExtraClasspathBase64Contents: Option[TarGzippedData], + val uploadedJarsBase64Contents: Option[TarGzippedData]) extends SubmitRestProtocolRequest { message = "create" clientSparkVersion = SPARK_VERSION } +case class TarGzippedData( + val dataBase64: String, + val blockSize: Int = 10240, + val recordSize: Int = 512, + val encoding: String +) + @JsonTypeInfo( use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.PROPERTY, diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/CompressionUtils.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/CompressionUtils.scala new file mode 100644 index 0000000000000..805a52bada219 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/CompressionUtils.scala @@ -0,0 +1,139 @@ +/* + * 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.rest.kubernetes + +import java.io.{ByteArrayInputStream, File, FileInputStream, FileOutputStream} +import java.util.zip.{GZIPInputStream, GZIPOutputStream} + +import com.google.common.io.Files +import org.apache.commons.codec.binary.Base64 +import org.apache.commons.compress.archivers.tar.{TarArchiveEntry, TarArchiveInputStream, TarArchiveOutputStream} +import org.apache.commons.compress.utils.CharsetNames +import org.apache.commons.io.IOUtils +import scala.collection.mutable + +import org.apache.spark.deploy.rest.TarGzippedData +import org.apache.spark.internal.Logging +import org.apache.spark.util.{ByteBufferOutputStream, Utils} + +private[spark] object CompressionUtils extends Logging { + // Defaults from TarArchiveOutputStream + private val BLOCK_SIZE = 10240 + private val RECORD_SIZE = 512 + private val ENCODING = CharsetNames.UTF_8 + + /** + * Compresses all of the given paths into a gzipped-tar archive, returning the compressed data in + * memory as an instance of {@link TarGzippedData}. The files are taken without consideration to their + * original folder structure, and are added to the tar archive in a flat hierarchy. Directories are + * not allowed, and duplicate file names are de-duplicated by appending a numeric suffix to the file name, + * before the file extension. For example, if paths a/b.txt and b/b.txt were provided, then the files added + * to the tar archive would be b.txt and b-1.txt. + * @param paths A list of file paths to be archived + * @return An in-memory representation of the compressed data. + */ + def createTarGzip(paths: Iterable[String]): TarGzippedData = { + val compressedBytesStream = Utils.tryWithResource(new ByteBufferOutputStream()) { raw => + Utils.tryWithResource(new GZIPOutputStream(raw)) { gzipping => + Utils.tryWithResource(new TarArchiveOutputStream( + gzipping, + BLOCK_SIZE, + RECORD_SIZE, + ENCODING)) { tarStream => + val usedFileNames = mutable.HashSet.empty[String] + for (path <- paths) { + val file = new File(path) + if (!file.isFile) { + throw new IllegalArgumentException(s"Cannot add $path to tarball; either does" + + s" not exist or is a directory.") + } + var resolvedFileName = file.getName + val extension = Files.getFileExtension(file.getName) + val nameWithoutExtension = Files.getNameWithoutExtension(file.getName) + var deduplicationCounter = 1 + while (usedFileNames.contains(resolvedFileName)) { + val oldResolvedFileName = resolvedFileName + resolvedFileName = s"$nameWithoutExtension-$deduplicationCounter.$extension" + logWarning(s"File with name $oldResolvedFileName already exists. Trying to add with" + + s" file name $resolvedFileName instead.") + deduplicationCounter += 1 + } + usedFileNames += resolvedFileName + val tarEntry = new TarArchiveEntry(file, resolvedFileName) + tarStream.putArchiveEntry(tarEntry) + Utils.tryWithResource(new FileInputStream(file)) { fileInput => + IOUtils.copy(fileInput, tarStream) + } + tarStream.closeArchiveEntry() + } + } + } + raw + } + val compressedAsBase64 = Base64.encodeBase64String(compressedBytesStream.toByteBuffer.array) + TarGzippedData( + dataBase64 = compressedAsBase64, + blockSize = BLOCK_SIZE, + recordSize = RECORD_SIZE, + encoding = ENCODING + ) + } + + /** + * Decompresses the provided tar archive to a directory. + * @param compressedData In-memory representation of the compressed data, ideally created via + * {@link createTarGzip}. + * @param rootOutputDir Directory to write the output files to. All files from the tarball + * are written here in a flat hierarchy. + * @return List of file paths for each file that was unpacked from the archive. + */ + def unpackAndWriteCompressedFiles( + compressedData: TarGzippedData, + rootOutputDir: File): Seq[String] = { + val paths = mutable.Buffer.empty[String] + val compressedBytes = Base64.decodeBase64(compressedData.dataBase64) + if (!rootOutputDir.exists) { + if (!rootOutputDir.mkdirs) { + throw new IllegalStateException(s"Failed to create output directory for unpacking" + + s" files at ${rootOutputDir.getAbsolutePath}") + } + } else if (rootOutputDir.isFile) { + throw new IllegalArgumentException(s"Root dir for writing decompressed files: " + + s"${rootOutputDir.getAbsolutePath} exists and is not a directory.") + } + Utils.tryWithResource(new ByteArrayInputStream(compressedBytes)) { compressedBytesStream => + Utils.tryWithResource(new GZIPInputStream(compressedBytesStream)) { gzipped => + Utils.tryWithResource(new TarArchiveInputStream( + gzipped, + compressedData.blockSize, + compressedData.recordSize, + compressedData.encoding)) { tarInputStream => + var nextTarEntry = tarInputStream.getNextTarEntry + while (nextTarEntry != null) { + val outputFile = new File(rootOutputDir, nextTarEntry.getName) + Utils.tryWithResource(new FileOutputStream(outputFile)) { fileOutputStream => + IOUtils.copy(tarInputStream, fileOutputStream) + } + paths += outputFile.getAbsolutePath + nextTarEntry = tarInputStream.getNextTarEntry + } + } + } + } + paths.toSeq + } +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/KubernetesSparkRestServer.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/KubernetesSparkRestServer.scala index 0a2e8176394ab..2ca3d4a8c0656 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/KubernetesSparkRestServer.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/KubernetesSparkRestServer.scala @@ -217,30 +217,11 @@ private[spark] class KubernetesSparkRestServer( } private def writeBase64ContentsToFiles( - filesBase64Contents: Array[(String, String)], + maybeCompressedFiles: Option[TarGzippedData], rootDir: File): Seq[String] = { - val resolvedFileNames = new scala.collection.mutable.HashSet[String] - val resolvedFilePaths = new ArrayBuffer[String] - for (file <- filesBase64Contents) { - var currentFileName = file._1 - var deduplicationCounter = 1 - while (resolvedFileNames.contains(currentFileName)) { - // Prepend the deduplication counter so as to not mess with the extension - currentFileName = s"$deduplicationCounter-$currentFileName" - deduplicationCounter += 1 - } - val resolvedFile = new File(rootDir, currentFileName) - val resolvedFilePath = resolvedFile.getAbsolutePath - if (resolvedFile.createNewFile()) { - val fileContents = Base64.decodeBase64(file._2) - Files.write(fileContents, resolvedFile) - } else { - throw new IllegalStateException(s"Could not write jar file to $resolvedFilePath") - } - resolvedFileNames += currentFileName - resolvedFilePaths += resolvedFilePath - } - resolvedFilePaths.toSeq + maybeCompressedFiles.map { compressedFiles => + CompressionUtils.unpackAndWriteCompressedFiles(compressedFiles, rootDir) + }.getOrElse(Seq.empty[String]) } } diff --git a/resource-managers/kubernetes/integration-tests-spark-jobs-helpers/pom.xml b/resource-managers/kubernetes/integration-tests-spark-jobs-helpers/pom.xml new file mode 100644 index 0000000000000..f99838636b349 --- /dev/null +++ b/resource-managers/kubernetes/integration-tests-spark-jobs-helpers/pom.xml @@ -0,0 +1,33 @@ + + + + 4.0.0 + + org.apache.spark + spark-parent_2.11 + 2.2.0-SNAPSHOT + ../../../pom.xml + + + spark-kubernetes-integration-tests-spark-jobs-helpers_2.11 + jar + Spark Project Kubernetes Integration Tests Spark Jobs Helpers + + + + diff --git a/resource-managers/kubernetes/integration-tests-spark-jobs-helpers/src/main/java/org/apache/spark/deploy/kubernetes/integrationtest/PiHelper.java b/resource-managers/kubernetes/integration-tests-spark-jobs-helpers/src/main/java/org/apache/spark/deploy/kubernetes/integrationtest/PiHelper.java new file mode 100644 index 0000000000000..99d982397bb6e --- /dev/null +++ b/resource-managers/kubernetes/integration-tests-spark-jobs-helpers/src/main/java/org/apache/spark/deploy/kubernetes/integrationtest/PiHelper.java @@ -0,0 +1,33 @@ +/* + * 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.kubernetes.integrationtest; + +/** + * Primarily extracted so that a separate jar can be added as a dependency for the + * test Spark job. + */ +public class PiHelper { + public static int helpPi() { + double x = Math.random() * 2 - 1; + double y = Math.random() * 2 - 1; + if (x*x + y*y < 1) { + return 1; + } else { + return 0; + } + } +} diff --git a/resource-managers/kubernetes/integration-tests-spark-jobs/pom.xml b/resource-managers/kubernetes/integration-tests-spark-jobs/pom.xml index 12b0234ae71bd..59e59aca5109b 100644 --- a/resource-managers/kubernetes/integration-tests-spark-jobs/pom.xml +++ b/resource-managers/kubernetes/integration-tests-spark-jobs/pom.xml @@ -29,6 +29,12 @@ Spark Project Kubernetes Integration Tests Spark Jobs + + org.apache.spark + spark-kubernetes-integration-tests-spark-jobs-helpers_${scala.binary.version} + ${project.version} + provided + org.apache.spark spark-core_${scala.binary.version} diff --git a/resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/kubernetes/integrationtest/jobs/SparkPiWithInfiniteWait.scala b/resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/kubernetes/integrationtest/jobs/SparkPiWithInfiniteWait.scala index 6e4660b771305..d3372749f999e 100644 --- a/resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/kubernetes/integrationtest/jobs/SparkPiWithInfiniteWait.scala +++ b/resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/kubernetes/integrationtest/jobs/SparkPiWithInfiniteWait.scala @@ -16,8 +16,7 @@ */ package org.apache.spark.deploy.kubernetes.integrationtest.jobs -import scala.math.random - +import org.apache.spark.deploy.kubernetes.integrationtest.PiHelper import org.apache.spark.sql.SparkSession // Equivalent to SparkPi except does not stop the Spark Context @@ -32,10 +31,8 @@ private[spark] object SparkPiWithInfiniteWait { .getOrCreate() val slices = if (args.length > 0) args(0).toInt else 10 val n = math.min(100000L * slices, Int.MaxValue).toInt // avoid overflow - val count = spark.sparkContext.parallelize(1 until n, slices).map { i => - val x = random * 2 - 1 - val y = random * 2 - 1 - if (x*x + y*y < 1) 1 else 0 + val count = spark.sparkContext.parallelize(1 until n, slices).map { _ => + PiHelper.helpPi() }.reduce(_ + _) // scalastyle:off println println("Pi is roughly " + 4.0 * count / (n - 1)) diff --git a/resource-managers/kubernetes/integration-tests/pom.xml b/resource-managers/kubernetes/integration-tests/pom.xml index 1e7eb0e12e6df..569527de8e300 100644 --- a/resource-managers/kubernetes/integration-tests/pom.xml +++ b/resource-managers/kubernetes/integration-tests/pom.xml @@ -48,6 +48,12 @@ ${project.version} test + + org.apache.spark + spark-kubernetes-integration-tests-spark-jobs-helpers_${scala.binary.version} + ${project.version} + test + org.apache.spark spark-docker-minimal-bundle_${scala.binary.version} @@ -123,6 +129,13 @@ jar ${project.build.directory}/integration-tests-spark-jobs + + org.apache.spark + spark-kubernetes-integration-tests-spark-jobs-helpers_${scala.binary.version} + ${project.version} + jar + ${project.build.directory}/integration-tests-spark-jobs-helpers + diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KubernetesSuite.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KubernetesSuite.scala index 183f666994d38..6247a1674f8d6 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KubernetesSuite.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KubernetesSuite.scala @@ -41,6 +41,11 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { .listFiles()(0) .getAbsolutePath + private val HELPER_JAR = Paths.get("target", "integration-tests-spark-jobs-helpers") + .toFile + .listFiles()(0) + .getAbsolutePath + private val TIMEOUT = PatienceConfiguration.Timeout(Span(2, Minutes)) private val INTERVAL = PatienceConfiguration.Interval(Span(2, Seconds)) private val MAIN_CLASS = "org.apache.spark.deploy.kubernetes" + @@ -117,6 +122,7 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { .set("spark.kubernetes.namespace", NAMESPACE) .set("spark.kubernetes.driver.docker.image", "spark-driver:latest") .set("spark.kubernetes.executor.docker.image", "spark-executor:latest") + .set("spark.kubernetes.driver.uploads.jars", HELPER_JAR) .set("spark.executor.memory", "500m") .set("spark.executor.cores", "1") .set("spark.executors.instances", "1") @@ -142,6 +148,7 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { "--executor-memory", "512m", "--executor-cores", "1", "--num-executors", "1", + "--upload-jars", HELPER_JAR, "--class", MAIN_CLASS, "--conf", s"spark.kubernetes.submit.caCertFile=${clientConfig.getCaCertFile}", "--conf", s"spark.kubernetes.submit.clientKeyFile=${clientConfig.getClientKeyFile}", From e2787e8c1a45bda367246cc73178412206acbf33 Mon Sep 17 00:00:00 2001 From: mccheah Date: Wed, 11 Jan 2017 16:05:16 -0800 Subject: [PATCH 11/81] Use alpine and java 8 for docker images. (#10) * Use alpine and java 8 for docker images. * Remove installation of vim and redundant comment --- .../src/main/docker/driver/Dockerfile | 11 +---------- .../src/main/docker/executor/Dockerfile | 11 +---------- 2 files changed, 2 insertions(+), 20 deletions(-) diff --git a/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/driver/Dockerfile b/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/driver/Dockerfile index 3bba38d8395ae..7bbabc40c34fc 100644 --- a/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/driver/Dockerfile +++ b/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/driver/Dockerfile @@ -1,12 +1,4 @@ -FROM ubuntu:trusty - -# Upgrade package index -# install a few other useful packages plus Open Jdk 7 -# Remove unneeded /var/lib/apt/lists/* after install to reduce the -# docker image size (by ~30MB) -RUN apt-get update && \ - apt-get install -y less openjdk-7-jre-headless net-tools vim-tiny sudo openssh-server procps && \ - rm -rf /var/lib/apt/lists/* +FROM anapsix/alpine-java:8 RUN mkdir -p /opt/spark RUN mkdir -p /opt/spark/ui-resources/org/apache/spark/ui/static @@ -18,7 +10,6 @@ ADD sbin /opt/spark/sbin ADD conf /opt/spark/conf ENV SPARK_HOME /opt/spark -ENV JAVA_HOME /usr/lib/jvm/java-7-openjdk-amd64/jre WORKDIR /opt/spark diff --git a/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/executor/Dockerfile b/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/executor/Dockerfile index f68f1a3fb2694..f584525cdc5e9 100644 --- a/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/executor/Dockerfile +++ b/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/executor/Dockerfile @@ -1,12 +1,4 @@ -FROM ubuntu:trusty - -# Upgrade package index -# install a few other useful packages plus Open Jdk 7 -# Remove unneeded /var/lib/apt/lists/* after install to reduce the -# docker image size (by ~30MB) -RUN apt-get update && \ - apt-get install -y less openjdk-7-jre-headless net-tools vim-tiny sudo openssh-server procps && \ - rm -rf /var/lib/apt/lists/* +FROM anapsix/alpine-java:8 RUN mkdir -p /opt/spark RUN mkdir -p /opt/spark/ui-resources/org/apache/spark/ui/static @@ -18,7 +10,6 @@ ADD sbin /opt/spark/sbin ADD conf /opt/spark/conf ENV SPARK_HOME /opt/spark -ENV JAVA_HOME /usr/lib/jvm/java-7-openjdk-amd64/jre WORKDIR /opt/spark From acceb72d96c739918af3b83f02ebf8524727d1a1 Mon Sep 17 00:00:00 2001 From: mccheah Date: Wed, 11 Jan 2017 18:20:12 -0800 Subject: [PATCH 12/81] Copy the Dockerfiles from docker-minimal-bundle into the distribution. (#12) --- dev/make-distribution.sh | 7 +++++++ .../src/main/docker/driver/Dockerfile | 5 ++++- .../src/main/docker/executor/Dockerfile | 5 ++++- 3 files changed, 15 insertions(+), 2 deletions(-) diff --git a/dev/make-distribution.sh b/dev/make-distribution.sh index 6ea319e4362ab..62706b0fffedc 100755 --- a/dev/make-distribution.sh +++ b/dev/make-distribution.sh @@ -175,6 +175,13 @@ echo "Build flags: $@" >> "$DISTDIR/RELEASE" # Copy jars cp "$SPARK_HOME"/assembly/target/scala*/jars/* "$DISTDIR/jars/" +# Copy docker files +mkdir -p "$DISTDIR/dockerfiles/driver" +mkdir -p "$DISTDIR/dockerfiles/executor" +DOCKERFILES_SRC="$SPARK_HOME/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker" +cp "$DOCKERFILES_SRC/driver/Dockerfile" "$DISTDIR/dockerfiles/driver/Dockerfile" +cp "$DOCKERFILES_SRC/executor/Dockerfile" "$DISTDIR/dockerfiles/executor/Dockerfile" + # Only create the yarn directory if the yarn artifacts were build. if [ -f "$SPARK_HOME"/common/network-yarn/target/scala*/spark-*-yarn-shuffle.jar ]; then mkdir "$DISTDIR"/yarn diff --git a/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/driver/Dockerfile b/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/driver/Dockerfile index 7bbabc40c34fc..308bf392fb202 100644 --- a/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/driver/Dockerfile +++ b/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/driver/Dockerfile @@ -1,7 +1,10 @@ FROM anapsix/alpine-java:8 +# If this docker file is being used in the context of building your images from a Spark distribution, the docker build +# command should be invoked from the top level directory of the Spark distribution. E.g.: +# docker build -t spark-driver:latest -f dockerfiles/driver/Dockerfile . + RUN mkdir -p /opt/spark -RUN mkdir -p /opt/spark/ui-resources/org/apache/spark/ui/static RUN touch /opt/spark/RELEASE ADD jars /opt/spark/jars diff --git a/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/executor/Dockerfile b/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/executor/Dockerfile index f584525cdc5e9..164c0a4289cac 100644 --- a/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/executor/Dockerfile +++ b/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/executor/Dockerfile @@ -1,7 +1,10 @@ FROM anapsix/alpine-java:8 +# If this docker file is being used in the context of building your images from a Spark distribution, the docker build +# command should be invoked from the top level directory of the Spark distribution. E.g.: +# docker build -t spark-executor:latest -f dockerfiles/executor/Dockerfile . + RUN mkdir -p /opt/spark -RUN mkdir -p /opt/spark/ui-resources/org/apache/spark/ui/static RUN touch /opt/spark/RELEASE ADD jars /opt/spark/jars From 24f4bf02f64f0725c849ec1eab3d87b2fd77e594 Mon Sep 17 00:00:00 2001 From: Anirudh Ramanathan Date: Thu, 12 Jan 2017 14:11:02 -0800 Subject: [PATCH 13/81] inherit IO (#13) --- .../deploy/rest/kubernetes/KubernetesSparkRestServer.scala | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/KubernetesSparkRestServer.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/KubernetesSparkRestServer.scala index 2ca3d4a8c0656..837706ca9f5a8 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/KubernetesSparkRestServer.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/KubernetesSparkRestServer.scala @@ -164,10 +164,7 @@ private[spark] class KubernetesSparkRestServer( command += s"-Xmx$driverMemory" command += mainClass command ++= appArgs - val pb = new ProcessBuilder(command: _*) - Paths.get(sparkHome, "logs").toFile.mkdirs - pb.redirectOutput(Paths.get(sparkHome, "logs", "stdout").toFile) - pb.redirectError(Paths.get(sparkHome, "logs", "stderr").toFile) + val pb = new ProcessBuilder(command: _*).inheritIO() val process = pb.start() ShutdownHookManager.addShutdownHook(() => { logInfo("Received stop command, shutting down the running Spark application...") From adcc9062bbea2dcd226684afbba4791256c2b94c Mon Sep 17 00:00:00 2001 From: mccheah Date: Thu, 12 Jan 2017 17:59:11 -0800 Subject: [PATCH 14/81] Error messages when the driver container fails to start. (#11) * Error messages when the driver container fails to start. * Fix messages a bit * Use timeout constant * Delete the pod if it fails for any reason (not just timeout) * Actually set submit succeeded * Fix typo --- .../spark/deploy/kubernetes/Client.scala | 83 +++++++++++++++++-- 1 file changed, 77 insertions(+), 6 deletions(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/Client.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/Client.scala index 230598d63bed1..6d7de973a52c2 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/Client.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/Client.scala @@ -18,7 +18,7 @@ package org.apache.spark.deploy.kubernetes import java.io.File import java.security.SecureRandom -import java.util.concurrent.{Executors, TimeUnit} +import java.util.concurrent.{Executors, TimeoutException, TimeUnit} import javax.net.ssl.X509TrustManager import com.google.common.io.Files @@ -34,7 +34,7 @@ import scala.concurrent.ExecutionContext import scala.concurrent.duration.DurationInt import scala.util.Success -import org.apache.spark.{SPARK_VERSION, SparkConf} +import org.apache.spark.{SPARK_VERSION, SparkConf, SparkException} import org.apache.spark.deploy.rest.{AppResource, KubernetesCreateSubmissionRequest, RemoteAppResource, TarGzippedData, UploadedAppResource} import org.apache.spark.deploy.rest.kubernetes._ import org.apache.spark.internal.Logging @@ -130,8 +130,8 @@ private[spark] class Client( val podWatcher = new Watcher[Pod] { override def eventReceived(action: Action, t: Pod): Unit = { if ((action == Action.ADDED || action == Action.MODIFIED) - && t.getStatus.getPhase == "Running" - && !submitCompletedFuture.isDone) { + && t.getStatus.getPhase == "Running" + && !submitCompletedFuture.isDone) { t.getStatus .getContainerStatuses .asScala @@ -216,8 +216,78 @@ private[spark] class Client( .endContainer() .endSpec() .done() - submitCompletedFuture.get(30, TimeUnit.SECONDS) - } + var submitSucceeded = false + try { + submitCompletedFuture.get(LAUNCH_TIMEOUT_SECONDS, TimeUnit.SECONDS) + submitSucceeded = true + } catch { + case e: TimeoutException => + val driverPod = try { + kubernetesClient.pods().withName(kubernetesAppId).get() + } catch { + case throwable: Throwable => + logError(s"Timed out while waiting $LAUNCH_TIMEOUT_SECONDS seconds for the" + + " driver pod to start, but an error occurred while fetching the driver" + + " pod's details.", throwable) + throw new SparkException(s"Timed out while waiting $LAUNCH_TIMEOUT_SECONDS" + + " seconds for the driver pod to start. Unfortunately, in attempting to fetch" + + " the latest state of the pod, another error was thrown. Check the logs for" + + " the error that was thrown in looking up the driver pod.", e) + } + val topLevelMessage = s"The driver pod with name ${driverPod.getMetadata.getName}" + + s" in namespace ${driverPod.getMetadata.getNamespace} was not ready in" + + s" $LAUNCH_TIMEOUT_SECONDS seconds." + val podStatusPhase = if (driverPod.getStatus.getPhase != null) { + s"Latest phase from the pod is: ${driverPod.getStatus.getPhase}" + } else { + "The pod had no final phase." + } + val podStatusMessage = if (driverPod.getStatus.getMessage != null) { + s"Latest message from the pod is: ${driverPod.getStatus.getMessage}" + } else { + "The pod had no final message." + } + val failedDriverContainerStatusString = driverPod.getStatus + .getContainerStatuses + .asScala + .find(_.getName == DRIVER_LAUNCHER_CONTAINER_NAME) + .map(status => { + val lastState = status.getState + if (lastState.getRunning != null) { + "Driver container last state: Running\n" + + s"Driver container started at: ${lastState.getRunning.getStartedAt}" + } else if (lastState.getWaiting != null) { + "Driver container last state: Waiting\n" + + s"Driver container wait reason: ${lastState.getWaiting.getReason}\n" + + s"Driver container message: ${lastState.getWaiting.getMessage}\n" + } else if (lastState.getTerminated != null) { + "Driver container last state: Terminated\n" + + s"Driver container started at: ${lastState.getTerminated.getStartedAt}\n" + + s"Driver container finished at: ${lastState.getTerminated.getFinishedAt}\n" + + s"Driver container exit reason: ${lastState.getTerminated.getReason}\n" + + s"Driver container exit code: ${lastState.getTerminated.getExitCode}\n" + + s"Driver container message: ${lastState.getTerminated.getMessage}" + } else { + "Driver container last state: Unknown" + } + }).getOrElse("The driver container wasn't found in the pod; expected to find" + + s" container with name $DRIVER_LAUNCHER_CONTAINER_NAME") + val finalErrorMessage = s"$topLevelMessage\n" + + s"$podStatusPhase\n" + + s"$podStatusMessage\n\n$failedDriverContainerStatusString" + logError(finalErrorMessage, e) + throw new SparkException(finalErrorMessage, e) + } finally { + if (!submitSucceeded) { + try { + kubernetesClient.pods.withName(kubernetesAppId).delete + } catch { + case throwable: Throwable => + logError("Failed to delete driver pod after it failed to run.", throwable) + } + } + } + } Utils.tryWithResource(kubernetesClient .pods() @@ -338,6 +408,7 @@ private object Client { private val DRIVER_LAUNCHER_CONTAINER_NAME = "spark-kubernetes-driver-launcher" private val SECURE_RANDOM = new SecureRandom() private val SPARK_SUBMISSION_SECRET_BASE_DIR = "/var/run/secrets/spark-submission" + private val LAUNCH_TIMEOUT_SECONDS = 30 def main(args: Array[String]): Unit = { require(args.length >= 2, s"Too few arguments. Usage: ${getClass.getName} " + From 0b81dbf516b0c389db50d237f52f32747dda8056 Mon Sep 17 00:00:00 2001 From: Anirudh Ramanathan Date: Thu, 12 Jan 2017 19:27:44 -0800 Subject: [PATCH 15/81] Fix linter error to make CI happy (#18) --- .../org/apache/spark/launcher/SparkSubmitOptionParser.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitOptionParser.java b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitOptionParser.java index 94f9bc319b6a2..2b7290a12f8c1 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitOptionParser.java +++ b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitOptionParser.java @@ -80,7 +80,8 @@ class SparkSubmitOptionParser { protected final String KUBERNETES_MASTER = "--kubernetes-master"; protected final String KUBERNETES_NAMESPACE = "--kubernetes-namespace"; protected final String KUBERNETES_UPLOAD_JARS = "--upload-jars"; - protected final String KUBERNETES_UPLOAD_DRIVER_EXTRA_CLASSPATH = "--upload-driver-extra-classpath"; + protected final String KUBERNETES_UPLOAD_DRIVER_EXTRA_CLASSPATH = + "--upload-driver-extra-classpath"; /** * This is the canonical list of spark-submit options. Each entry in the array contains the From e70f427ec99d4dee6668b5eb7e3889288e3db505 Mon Sep 17 00:00:00 2001 From: mccheah Date: Fri, 13 Jan 2017 14:11:08 -0800 Subject: [PATCH 16/81] Documentation for the current state of the world (#16) * Documentation for the current state of the world. * Adding navigation links from other pages * Address comments, add TODO for things that should be fixed * Address comments, mostly making images section clearer * Virtual runtime -> container runtime --- docs/_layouts/global.html | 1 + docs/index.md | 1 + docs/running-on-kubernetes.md | 224 ++++++++++++++++++++++++++++++++++ 3 files changed, 226 insertions(+) create mode 100644 docs/running-on-kubernetes.md diff --git a/docs/_layouts/global.html b/docs/_layouts/global.html index c00d0db63cd10..3c786a6344066 100755 --- a/docs/_layouts/global.html +++ b/docs/_layouts/global.html @@ -99,6 +99,7 @@
  • Spark Standalone
  • Mesos
  • YARN
  • +
  • Kubernetes
  • diff --git a/docs/index.md b/docs/index.md index 57b9fa848f4a3..81d37aa5f63a1 100644 --- a/docs/index.md +++ b/docs/index.md @@ -113,6 +113,7 @@ options for deployment: * [Mesos](running-on-mesos.html): deploy a private cluster using [Apache Mesos](http://mesos.apache.org) * [YARN](running-on-yarn.html): deploy Spark on top of Hadoop NextGen (YARN) + * [Kubernetes](running-on-kubernetes.html): deploy Spark on top of Kubernetes **Other Documents:** diff --git a/docs/running-on-kubernetes.md b/docs/running-on-kubernetes.md new file mode 100644 index 0000000000000..5192d9d086618 --- /dev/null +++ b/docs/running-on-kubernetes.md @@ -0,0 +1,224 @@ +--- +layout: global +title: Running Spark on Kubernetes +--- + +Support for running on [Kubernetes](https://kubernetes.io/) is available in experimental status. The feature set is +currently limited and not well-tested. This should not be used in production environments. + +## Setting Up Docker Images + +Kubernetes requires users to supply images that can be deployed into containers within pods. The images are built to +be run in a container runtime environment that Kubernetes supports. Docker is a container runtime environment that is +frequently used with Kubernetes, so Spark provides some support for working with Docker to get started quickly. + +To use Spark on Kubernetes with Docker, images for the driver and the executors need to built and published to an +accessible Docker registry. Spark distributions include the Docker files for the driver and the executor at +`dockerfiles/driver/Dockerfile` and `docker/executor/Dockerfile`, respectively. Use these Docker files to build the +Docker images, and then tag them with the registry that the images should be sent to. Finally, push the images to the +registry. + +For example, if the registry host is `registry-host` and the registry is listening on port 5000: + + cd $SPARK_HOME + docker build -t registry-host:5000/spark-driver:latest -f dockerfiles/driver/Dockerfile . + docker build -t registry-host:5000/spark-executor:latest -f dockerfiles/executor/Dockerfile . + docker push registry-host:5000/spark-driver:latest + docker push registry-host:5000/spark-executor:latest + +## Submitting Applications to Kubernetes + +Kubernetes applications can be executed via `spark-submit`. For example, to compute the value of pi, assuming the images +are set up as described above: + + bin/spark-submit + --deploy-mode cluster + --class org.apache.spark.examples.SparkPi + --master k8s://https://: + --kubernetes-namespace default + --conf spark.executor.instances=5 + --conf spark.app.name=spark-pi + --conf spark.kubernetes.driver.docker.image=registry-host:5000/spark-driver:latest + --conf spark.kubernetes.executor.docker.image=registry-host:5000/spark-executor:latest + examples/jars/spark_2.11-2.2.0.jar + + +The Spark master, specified either via passing the `--master` command line argument to `spark-submit` or by setting +`spark.master` in the application's configuration, must be a URL with the format `k8s://`. Prefixing the +master string with `k8s://` will cause the Spark application to launch on the Kubernetes cluster, with the API server +being contacted at `api_server_url`. The HTTP protocol must also be specified. + +Note that applications can currently only be executed in cluster mode, where the driver and its executors are running on +the cluster. + +### Adding Other JARs + +Spark allows users to provide dependencies that are bundled into the driver's Docker image, or that are on the local +disk of the submitter's machine. These two types of dependencies are specified via different configuration options to +`spark-submit`: + +* Local jars provided by specifying the `--jars` command line argument to `spark-submit`, or by setting `spark.jars` in + the application's configuration, will be treated as jars that are located on the *disk of the driver Docker + container*. This only applies to jar paths that do not specify a scheme or that have the scheme `file://`. Paths with + other schemes are fetched from their appropriate locations. +* Local jars provided by specifying the `--upload-jars` command line argument to `spark-submit`, or by setting + `spark.kubernetes.driver.uploads.jars` in the application's configuration, will be treated as jars that are located on + the *disk of the submitting machine*. These jars are uploaded to the driver docker container before executing the + application. + +* A main application resource path that does not have a scheme or that has the scheme `file://` is assumed to be on the + *disk of the submitting machine*. This resource is uploaded to the driver docker container before executing the + application. A remote path can still be specified and the resource will be fetched from the appropriate location. + +In all of these cases, the jars are placed on the driver's classpath, and are also sent to the executors. Below are some +examples of providing application dependencies. + +To submit an application with both the main resource and two other jars living on the submitting user's machine: + + bin/spark-submit + --deploy-mode cluster + --class com.example.applications.SampleApplication + --master k8s://https://192.168.99.100 + --kubernetes-namespace default + --upload-jars /home/exampleuser/exampleapplication/dep1.jar,/home/exampleuser/exampleapplication/dep2.jar + --conf spark.kubernetes.driver.docker.image=registry-host:5000/spark-driver:latest + --conf spark.kubernetes.executor.docker.image=registry-host:5000/spark-executor:latest + /home/exampleuser/exampleapplication/main.jar + +Note that since passing the jars through the `--upload-jars` command line argument is equivalent to setting the +`spark.kubernetes.driver.uploads.jars` Spark property, the above will behave identically to this command: + + bin/spark-submit + --deploy-mode cluster + --class com.example.applications.SampleApplication + --master k8s://https://192.168.99.100 + --kubernetes-namespace default + --conf spark.kubernetes.driver.uploads.jars=/home/exampleuser/exampleapplication/dep1.jar,/home/exampleuser/exampleapplication/dep2.jar + --conf spark.kubernetes.driver.docker.image=registry-host:5000/spark-driver:latest + --conf spark.kubernetes.executor.docker.image=registry-host:5000/spark-executor:latest + /home/exampleuser/exampleapplication/main.jar + +To specify a main application resource that can be downloaded from an HTTP service, and if a plugin for that application +is located in the jar `/opt/spark-plugins/app-plugin.jar` on the docker image's disk: + + bin/spark-submit + --deploy-mode cluster + --class com.example.applications.PluggableApplication + --master k8s://https://192.168.99.100 + --kubernetes-namespace default + --jars /opt/spark-plugins/app-plugin.jar + --conf spark.kubernetes.driver.docker.image=registry-host:5000/spark-driver-custom:latest + --conf spark.kubernetes.executor.docker.image=registry-host:5000/spark-executor:latest + http://example.com:8080/applications/sparkpluggable/app.jar + +Note that since passing the jars through the `--jars` command line argument is equivalent to setting the `spark.jars` +Spark property, the above will behave identically to this command: + + bin/spark-submit + --deploy-mode cluster + --class com.example.applications.PluggableApplication + --master k8s://https://192.168.99.100 + --kubernetes-namespace default + --conf spark.jars=file:///opt/spark-plugins/app-plugin.jar + --conf spark.kubernetes.driver.docker.image=registry-host:5000/spark-driver-custom:latest + --conf spark.kubernetes.executor.docker.image=registry-host:5000/spark-executor:latest + http://example.com:8080/applications/sparkpluggable/app.jar + +### Spark Properties + +Below are some other common properties that are specific to Kubernetes. Most of the other configurations are the same +from the other deployment modes. See the [configuration page](configuration.html) for more information on those. + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    Property NameDefaultMeaning
    spark.kubernetes.namespace(none) + The namespace that will be used for running the driver and executor pods. Must be specified. When using + spark-submit in cluster mode, this can also be passed to spark-submit via the + --kubernetes-namespace command line argument. +
    spark.kubernetes.driver.docker.imagespark-driver:2.2.0 + Docker image to use for the driver. Specify this using the standard + Docker tag format. +
    spark.kubernetes.executor.docker.imagespark-executor:2.2.0 + Docker image to use for the executors. Specify this using the standard + Docker tag format. +
    spark.kubernetes.submit.caCertFile(none) + CA cert file for connecting to Kubernetes over SSL. This file should be located on the submitting machine's disk. +
    spark.kubernetes.submit.clientKeyFile(none) + Client key file for authenticating against the Kubernetes API server. This file should be located on the submitting + machine's disk. +
    spark.kubernetes.submit.clientCertFile(none) + Client cert file for authenticating against the Kubernetes API server. This file should be located on the submitting + machine's disk. +
    spark.kubernetes.submit.serviceAccountNamedefault + 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. +
    spark.kubernetes.driver.uploads.jars(none) + Comma-separated list of jars to sent to the driver and all executors when submitting the application in cluster + mode. Refer to adding other jars for more information. +
    spark.kubernetes.driver.uploads.driverExtraClasspath(none) + Comma-separated list of jars to be sent to the driver only when submitting the application in cluster mode. +
    spark.kubernetes.executor.memoryOverheadexecutorMemory * 0.10, with minimum of 384 + The amount of off-heap memory (in megabytes) to be allocated per executor. This is memory that accounts for things + like VM overheads, interned strings, other native overheads, etc. This tends to grow with the executor size + (typically 6-10%). +
    + +## Current Limitations + +Running Spark on Kubernetes is currently an experimental feature. Some restrictions on the current implementation that +should be lifted in the future include: +* Applications can only use a fixed number of executors. Dynamic allocation is not supported. +* Applications can only run in cluster mode. +* Only Scala and Java applications can be run. From b25bc8b712277a88f00b017a6a69a23585a1d19b Mon Sep 17 00:00:00 2001 From: mccheah Date: Fri, 13 Jan 2017 14:56:08 -0800 Subject: [PATCH 17/81] Development workflow documentation for the current state of the world. (#20) * Development workflow documentation for the current state of the world. * Address comments. * Clarified code change and added ticket link --- resource-managers/kubernetes/README.md | 56 ++++++++++++++++++++++++++ 1 file changed, 56 insertions(+) create mode 100644 resource-managers/kubernetes/README.md diff --git a/resource-managers/kubernetes/README.md b/resource-managers/kubernetes/README.md new file mode 100644 index 0000000000000..3c11efa38d5af --- /dev/null +++ b/resource-managers/kubernetes/README.md @@ -0,0 +1,56 @@ +--- +layout: global +title: Spark on Kubernetes Development +--- + +[Kubernetes](https://kubernetes.io/) is a framework for easily deploying, scaling, and managing containerized +applications. It would be useful for a user to run their Spark jobs on a Kubernetes cluster alongside their +other Kubernetes-managed applications. For more about the motivations for adding this feature, see the umbrella JIRA +ticket that tracks this project: [SPARK-18278](https://issues.apache.org/jira/browse/SPARK-18278). + +This submodule is an initial implementation of allowing Kubernetes to be a +supported cluster manager for Spark, along with Mesos, Hadoop YARN, and Standalone. This document provides a summary of +important matters to keep in mind when developing this feature. + +# Building Spark with Kubernetes Support + +To build Spark with Kubernetes support, use the `kubernetes` profile when invoking Maven. For example, to simply compile +the Kubernetes core implementation module along with its dependencies: + + build/mvn compile -Pkubernetes -pl resource-managers/kubernetes/core -am + +To build a distribution of Spark with Kubernetes support, use the `dev/make-distribution.sh` script, and add the +`kubernetes` profile as part of the build arguments. Any other build arguments can be specified as one would expect when +building Spark normally. For example, to build Spark against Hadoop 2.7 and Kubernetes: + + dev/make-distribution.sh --tgz -Phadoop2.7 -Pkubernetes + +# Kubernetes Code Modules + +Below is a list of the submodules for this cluster manager and what they do. + +* `core`: Implementation of the Kubernetes cluster manager support. +* `integration-tests`: Integration tests for the project. +* `docker-minimal-bundle`: Base Dockerfiles for the driver and the executors. The Dockerfiles are used for integration + tests as well as being provided in packaged distributions of Spark. +* `integration-tests-spark-jobs`: Spark jobs that are only used in integration tests. +* `integration-tests-spark-jobs-helpers`: Dependencies for the spark jobs used in integration tests. These dependencies + are separated out to facilitate testing the shipping of jars to drivers running on Kubernetes clusters. + +# Running the Kubernetes Integration Tests + +Note that the integration test framework is currently being heavily revised and is subject to change. + +Running any of the integration tests requires including `kubernetes-integration-tests` profile in the build command. In +order to prepare the environment for running the integration tests, the `pre-integration-test` step must be run in Maven +on the `resource-managers/kubernetes/integration-tests` module: + + build/mvn pre-integration-test -Pkubernetes -Pkubernetes-integration-tests -pl resource-managers/kubernetes/integration-tests -am + +Afterwards, the integration tests can be executed with Maven or your IDE. Note that when running tests from an IDE, the +`pre-integration-test` phase must be run every time the Spark main code changes. When running tests from the +command line, the `pre-integration-test` phase should automatically be invoked if the `integration-test` phase is run. + +# Usage Guide + +See the [usage guide](../../docs/running-on-kubernetes.md) for more information. From 761b3175c0cbe282dae1a23144669a5003f83e39 Mon Sep 17 00:00:00 2001 From: Anirudh Ramanathan Date: Fri, 13 Jan 2017 15:05:22 -0800 Subject: [PATCH 18/81] Added service name as prefix to executor pods (#14) * Added service name as prefix to executor pods to be able to tell them apart from kubectl output * Addressed comments --- .../cluster/kubernetes/KubernetesClusterSchedulerBackend.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala index 4e099cea3198b..2717d2f37d910 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala @@ -155,7 +155,7 @@ private[spark] class KubernetesClusterSchedulerBackend( private def allocateNewExecutorPod(): (String, Pod) = { val executorKubernetesId = UUID.randomUUID().toString.replaceAll("-", "") val executorId = EXECUTOR_ID_COUNTER.incrementAndGet().toString - val name = s"exec$executorKubernetesId" + val name = s"$kubernetesDriverServiceName-exec-$executorKubernetesId" val selectors = Map(SPARK_EXECUTOR_SELECTOR -> executorId, SPARK_APP_SELECTOR -> applicationId()).asJava val executorMemoryQuantity = new QuantityBuilder(false) From 8739b41db6ea93f4f7f3f3e982752366611ea8bf Mon Sep 17 00:00:00 2001 From: Kimoon Kim Date: Fri, 13 Jan 2017 20:44:56 -0800 Subject: [PATCH 19/81] Add kubernetes profile to travis CI yml file (#21) * Add kubernetes profile to travis yml file * Fix long lines in CompressionUtils.scala --- .travis.yml | 2 +- .../deploy/rest/kubernetes/CompressionUtils.scala | 10 +++++----- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/.travis.yml b/.travis.yml index 8739849a20798..a118421eb45e0 100644 --- a/.travis.yml +++ b/.travis.yml @@ -44,7 +44,7 @@ notifications: # 5. Run maven install before running lint-java. install: - export MAVEN_SKIP_RC=1 - - build/mvn -T 4 -q -DskipTests -Pmesos -Pyarn -Phadoop-2.3 -Pkinesis-asl -Phive -Phive-thriftserver install + - build/mvn -T 4 -q -DskipTests -Pmesos -Pyarn -Phadoop-2.3 -Pkubernetes -Pkinesis-asl -Phive -Phive-thriftserver install # 6. Run lint-java. script: diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/CompressionUtils.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/CompressionUtils.scala index 805a52bada219..1c95dacc7eb01 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/CompressionUtils.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/CompressionUtils.scala @@ -38,11 +38,11 @@ private[spark] object CompressionUtils extends Logging { /** * Compresses all of the given paths into a gzipped-tar archive, returning the compressed data in - * memory as an instance of {@link TarGzippedData}. The files are taken without consideration to their - * original folder structure, and are added to the tar archive in a flat hierarchy. Directories are - * not allowed, and duplicate file names are de-duplicated by appending a numeric suffix to the file name, - * before the file extension. For example, if paths a/b.txt and b/b.txt were provided, then the files added - * to the tar archive would be b.txt and b-1.txt. + * memory as an instance of {@link TarGzippedData}. The files are taken without consideration to + * their original folder structure, and are added to the tar archive in a flat hierarchy. + * Directories are not allowed, and duplicate file names are de-duplicated by appending a numeric + * suffix to the file name, before the file extension. For example, if paths a/b.txt and b/b.txt + * were provided, then the files added to the tar archive would be b.txt and b-1.txt. * @param paths A list of file paths to be archived * @return An in-memory representation of the compressed data. */ From 928e00eb19be53071f247e98bf9a74897417bc62 Mon Sep 17 00:00:00 2001 From: Shuai Lin Date: Tue, 17 Jan 2017 17:24:58 +0000 Subject: [PATCH 20/81] Improved the example commands in running-on-k8s document. (#25) * Improved the example commands in running-on-k8s document. * Fixed more example commands. * Fixed typo. --- docs/running-on-kubernetes.md | 84 +++++++++++++++++------------------ 1 file changed, 42 insertions(+), 42 deletions(-) diff --git a/docs/running-on-kubernetes.md b/docs/running-on-kubernetes.md index 5192d9d086618..234c9870548c7 100644 --- a/docs/running-on-kubernetes.md +++ b/docs/running-on-kubernetes.md @@ -31,16 +31,16 @@ For example, if the registry host is `registry-host` and the registry is listeni Kubernetes applications can be executed via `spark-submit`. For example, to compute the value of pi, assuming the images are set up as described above: - bin/spark-submit - --deploy-mode cluster - --class org.apache.spark.examples.SparkPi - --master k8s://https://: - --kubernetes-namespace default - --conf spark.executor.instances=5 - --conf spark.app.name=spark-pi - --conf spark.kubernetes.driver.docker.image=registry-host:5000/spark-driver:latest - --conf spark.kubernetes.executor.docker.image=registry-host:5000/spark-executor:latest - examples/jars/spark_2.11-2.2.0.jar + bin/spark-submit \ + --deploy-mode cluster \ + --class org.apache.spark.examples.SparkPi \ + --master k8s://https://: \ + --kubernetes-namespace default \ + --conf spark.executor.instances=5 \ + --conf spark.app.name=spark-pi \ + --conf spark.kubernetes.driver.docker.image=registry-host:5000/spark-driver:latest \ + --conf spark.kubernetes.executor.docker.image=registry-host:5000/spark-executor:latest \ + examples/jars/spark_examples_2.11-2.2.0.jar The Spark master, specified either via passing the `--master` command line argument to `spark-submit` or by setting @@ -75,53 +75,53 @@ examples of providing application dependencies. To submit an application with both the main resource and two other jars living on the submitting user's machine: - bin/spark-submit - --deploy-mode cluster - --class com.example.applications.SampleApplication - --master k8s://https://192.168.99.100 - --kubernetes-namespace default - --upload-jars /home/exampleuser/exampleapplication/dep1.jar,/home/exampleuser/exampleapplication/dep2.jar - --conf spark.kubernetes.driver.docker.image=registry-host:5000/spark-driver:latest - --conf spark.kubernetes.executor.docker.image=registry-host:5000/spark-executor:latest + bin/spark-submit \ + --deploy-mode cluster \ + --class com.example.applications.SampleApplication \ + --master k8s://https://192.168.99.100 \ + --kubernetes-namespace default \ + --upload-jars /home/exampleuser/exampleapplication/dep1.jar,/home/exampleuser/exampleapplication/dep2.jar \ + --conf spark.kubernetes.driver.docker.image=registry-host:5000/spark-driver:latest \ + --conf spark.kubernetes.executor.docker.image=registry-host:5000/spark-executor:latest \ /home/exampleuser/exampleapplication/main.jar Note that since passing the jars through the `--upload-jars` command line argument is equivalent to setting the `spark.kubernetes.driver.uploads.jars` Spark property, the above will behave identically to this command: - bin/spark-submit - --deploy-mode cluster - --class com.example.applications.SampleApplication - --master k8s://https://192.168.99.100 - --kubernetes-namespace default - --conf spark.kubernetes.driver.uploads.jars=/home/exampleuser/exampleapplication/dep1.jar,/home/exampleuser/exampleapplication/dep2.jar - --conf spark.kubernetes.driver.docker.image=registry-host:5000/spark-driver:latest - --conf spark.kubernetes.executor.docker.image=registry-host:5000/spark-executor:latest + bin/spark-submit \ + --deploy-mode cluster \ + --class com.example.applications.SampleApplication \ + --master k8s://https://192.168.99.100 \ + --kubernetes-namespace default \ + --conf spark.kubernetes.driver.uploads.jars=/home/exampleuser/exampleapplication/dep1.jar,/home/exampleuser/exampleapplication/dep2.jar \ + --conf spark.kubernetes.driver.docker.image=registry-host:5000/spark-driver:latest \ + --conf spark.kubernetes.executor.docker.image=registry-host:5000/spark-executor:latest \ /home/exampleuser/exampleapplication/main.jar To specify a main application resource that can be downloaded from an HTTP service, and if a plugin for that application is located in the jar `/opt/spark-plugins/app-plugin.jar` on the docker image's disk: - bin/spark-submit - --deploy-mode cluster - --class com.example.applications.PluggableApplication - --master k8s://https://192.168.99.100 - --kubernetes-namespace default - --jars /opt/spark-plugins/app-plugin.jar - --conf spark.kubernetes.driver.docker.image=registry-host:5000/spark-driver-custom:latest - --conf spark.kubernetes.executor.docker.image=registry-host:5000/spark-executor:latest + bin/spark-submit \ + --deploy-mode cluster \ + --class com.example.applications.PluggableApplication \ + --master k8s://https://192.168.99.100 \ + --kubernetes-namespace default \ + --jars /opt/spark-plugins/app-plugin.jar \ + --conf spark.kubernetes.driver.docker.image=registry-host:5000/spark-driver-custom:latest \ + --conf spark.kubernetes.executor.docker.image=registry-host:5000/spark-executor:latest \ http://example.com:8080/applications/sparkpluggable/app.jar Note that since passing the jars through the `--jars` command line argument is equivalent to setting the `spark.jars` Spark property, the above will behave identically to this command: - bin/spark-submit - --deploy-mode cluster - --class com.example.applications.PluggableApplication - --master k8s://https://192.168.99.100 - --kubernetes-namespace default - --conf spark.jars=file:///opt/spark-plugins/app-plugin.jar - --conf spark.kubernetes.driver.docker.image=registry-host:5000/spark-driver-custom:latest - --conf spark.kubernetes.executor.docker.image=registry-host:5000/spark-executor:latest + bin/spark-submit \ + --deploy-mode cluster \ + --class com.example.applications.PluggableApplication \ + --master k8s://https://192.168.99.100 \ + --kubernetes-namespace default \ + --conf spark.jars=file:///opt/spark-plugins/app-plugin.jar \ + --conf spark.kubernetes.driver.docker.image=registry-host:5000/spark-driver-custom:latest \ + --conf spark.kubernetes.executor.docker.image=registry-host:5000/spark-executor:latest \ http://example.com:8080/applications/sparkpluggable/app.jar ### Spark Properties From 3e3c4d4ac9450bff5ad27b7b20faba943623b86e Mon Sep 17 00:00:00 2001 From: Anirudh Ramanathan Date: Wed, 18 Jan 2017 11:24:43 -0800 Subject: [PATCH 21/81] Fix spacing for command highlighting (#31) --- resource-managers/kubernetes/README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/resource-managers/kubernetes/README.md b/resource-managers/kubernetes/README.md index 3c11efa38d5af..62764dcb2ca03 100644 --- a/resource-managers/kubernetes/README.md +++ b/resource-managers/kubernetes/README.md @@ -45,7 +45,7 @@ Running any of the integration tests requires including `kubernetes-integration- order to prepare the environment for running the integration tests, the `pre-integration-test` step must be run in Maven on the `resource-managers/kubernetes/integration-tests` module: - build/mvn pre-integration-test -Pkubernetes -Pkubernetes-integration-tests -pl resource-managers/kubernetes/integration-tests -am + build/mvn pre-integration-test -Pkubernetes -Pkubernetes-integration-tests -pl resource-managers/kubernetes/integration-tests -am Afterwards, the integration tests can be executed with Maven or your IDE. Note that when running tests from an IDE, the `pre-integration-test` phase must be run every time the Spark main code changes. When running tests from the From 36c4e949756a389d4502ccb6529b9af4541b9805 Mon Sep 17 00:00:00 2001 From: mccheah Date: Wed, 18 Jan 2017 17:30:09 -0800 Subject: [PATCH 22/81] Support custom labels on the driver pod. (#27) * Support custom labels on the driver pod. * Add integration test and fix logic. * Fix tests * Fix minor formatting mistake * Reduce unnecessary diff --- docs/running-on-kubernetes.md | 8 +++++ .../spark/deploy/kubernetes/Client.scala | 35 +++++++++++++++---- .../integrationtest/KubernetesSuite.scala | 34 ++++++++++++++++++ 3 files changed, 70 insertions(+), 7 deletions(-) diff --git a/docs/running-on-kubernetes.md b/docs/running-on-kubernetes.md index 234c9870548c7..14e2df4ed0702 100644 --- a/docs/running-on-kubernetes.md +++ b/docs/running-on-kubernetes.md @@ -213,6 +213,14 @@ from the other deployment modes. See the [configuration page](configuration.html (typically 6-10%). + + spark.kubernetes.driver.labels + (none) + + Custom labels that will be added to the driver pod. This should be a comma-separated list of label key-value pairs, + where each label is in the format key=value. + + ## Current Limitations diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/Client.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/Client.scala index 6d7de973a52c2..073afcbba7b52 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/Client.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/Client.scala @@ -77,6 +77,8 @@ private[spark] class Client( private val serviceAccount = sparkConf.get("spark.kubernetes.submit.serviceAccountName", "default") + private val customLabels = sparkConf.get("spark.kubernetes.driver.labels", "") + private implicit val retryableExecutionContext = ExecutionContext .fromExecutorService( Executors.newSingleThreadExecutor(new ThreadFactoryBuilder() @@ -85,6 +87,7 @@ private[spark] class Client( .build())) def run(): Unit = { + val parsedCustomLabels = parseCustomLabels(customLabels) var k8ConfBuilder = new ConfigBuilder() .withApiVersion("v1") .withMasterUrl(master) @@ -109,14 +112,15 @@ private[spark] class Client( .withType("Opaque") .done() try { - val selectors = Map(DRIVER_LAUNCHER_SELECTOR_LABEL -> driverLauncherSelectorValue).asJava + val resolvedSelectors = (Map(DRIVER_LAUNCHER_SELECTOR_LABEL -> driverLauncherSelectorValue) + ++ parsedCustomLabels).asJava val (servicePorts, containerPorts) = configurePorts() val service = kubernetesClient.services().createNew() .withNewMetadata() .withName(kubernetesAppId) .endMetadata() .withNewSpec() - .withSelector(selectors) + .withSelector(resolvedSelectors) .withPorts(servicePorts.asJava) .endSpec() .done() @@ -137,7 +141,7 @@ private[spark] class Client( .asScala .find(status => status.getName == DRIVER_LAUNCHER_CONTAINER_NAME && status.getReady) match { - case Some(status) => + case Some(_) => try { val driverLauncher = getDriverLauncherService( k8ClientConfig, master) @@ -184,7 +188,7 @@ private[spark] class Client( kubernetesClient.pods().createNew() .withNewMetadata() .withName(kubernetesAppId) - .withLabels(selectors) + .withLabels(resolvedSelectors) .endMetadata() .withNewSpec() .withRestartPolicy("OnFailure") @@ -291,7 +295,7 @@ private[spark] class Client( Utils.tryWithResource(kubernetesClient .pods() - .withLabels(selectors) + .withLabels(resolvedSelectors) .watch(podWatcher)) { createDriverPod } } finally { kubernetesClient.secrets().delete(secret) @@ -336,7 +340,7 @@ private[spark] class Client( .getOption("spark.ui.port") .map(_.toInt) .getOrElse(DEFAULT_UI_PORT)) - (servicePorts.toSeq, containerPorts.toSeq) + (servicePorts, containerPorts) } private def buildSubmissionRequest(): KubernetesCreateSubmissionRequest = { @@ -366,7 +370,7 @@ private[spark] class Client( uploadedJarsBase64Contents = uploadJarsBase64Contents) } - def compressJars(maybeFilePaths: Option[String]): Option[TarGzippedData] = { + private def compressJars(maybeFilePaths: Option[String]): Option[TarGzippedData] = { maybeFilePaths .map(_.split(",")) .map(CompressionUtils.createTarGzip(_)) @@ -391,6 +395,23 @@ private[spark] class Client( sslSocketFactory = sslContext.getSocketFactory, trustContext = trustManager) } + + private def parseCustomLabels(labels: String): Map[String, String] = { + labels.split(",").map(_.trim).filterNot(_.isEmpty).map(label => { + label.split("=", 2).toSeq match { + case Seq(k, v) => + require(k != DRIVER_LAUNCHER_SELECTOR_LABEL, "Label with key" + + s" $DRIVER_LAUNCHER_SELECTOR_LABEL cannot be used in" + + " spark.kubernetes.driver.labels, as it is reserved for Spark's" + + " internal configuration.") + (k, v) + case _ => + throw new SparkException("Custom labels set by spark.kubernetes.driver.labels" + + " must be a comma-separated list of key-value pairs, with format =." + + s" Got label: $label. All labels: $labels") + } + }).toMap + } } private object Client { diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KubernetesSuite.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KubernetesSuite.scala index 6247a1674f8d6..7b3c2b93b865b 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KubernetesSuite.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KubernetesSuite.scala @@ -161,4 +161,38 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { "spark-pi", NAMESPACE, "spark-ui-port") expectationsForStaticAllocation(sparkMetricsService) } + + test("Run with custom labels") { + val args = Array( + "--master", s"k8s://https://${Minikube.getMinikubeIp}:8443", + "--deploy-mode", "cluster", + "--kubernetes-namespace", NAMESPACE, + "--name", "spark-pi", + "--executor-memory", "512m", + "--executor-cores", "1", + "--num-executors", "1", + "--upload-jars", HELPER_JAR, + "--class", MAIN_CLASS, + "--conf", s"spark.kubernetes.submit.caCertFile=${clientConfig.getCaCertFile}", + "--conf", s"spark.kubernetes.submit.clientKeyFile=${clientConfig.getClientKeyFile}", + "--conf", s"spark.kubernetes.submit.clientCertFile=${clientConfig.getClientCertFile}", + "--conf", "spark.kubernetes.executor.docker.image=spark-executor:latest", + "--conf", "spark.kubernetes.driver.docker.image=spark-driver:latest", + "--conf", "spark.kubernetes.driver.labels=label1=label1value,label2=label2value", + EXAMPLES_JAR) + SparkSubmit.main(args) + val driverPodLabels = minikubeKubernetesClient + .pods + .withName("spark-pi") + .get + .getMetadata + .getLabels + // We can't match all of the selectors directly since one of the selectors is based on the + // launch time. + assert(driverPodLabels.size == 3, "Unexpected number of pod labels.") + assert(driverPodLabels.containsKey("driver-launcher-selector"), "Expected driver launcher" + + " selector label to be present.") + assert(driverPodLabels.get("label1") == "label1value", "Unexpected value for label1") + assert(driverPodLabels.get("label2") == "label2value", "Unexpected value for label2") + } } From b6c57c707a93b5d201ee381fac60c751bf068f9e Mon Sep 17 00:00:00 2001 From: Anirudh Ramanathan Date: Wed, 18 Jan 2017 17:34:02 -0800 Subject: [PATCH 23/81] Make pod name unique using the submission timestamp (#32) --- .../scala/org/apache/spark/deploy/kubernetes/Client.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/Client.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/Client.scala index 073afcbba7b52..30eaa6269cf47 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/Client.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/Client.scala @@ -56,10 +56,10 @@ private[spark] class Client( private val master = rawMaster.replaceFirst("k8s://", "") private val launchTime = System.currentTimeMillis - private val kubernetesAppId = sparkConf.getOption("spark.app.name") + private val appName = sparkConf.getOption("spark.app.name") .orElse(sparkConf.getOption("spark.app.id")) - .getOrElse(s"spark-$launchTime") - + .getOrElse("spark") + private val kubernetesAppId = s"$appName-$launchTime" private val secretName = s"spark-submission-server-secret-$kubernetesAppId" private val driverLauncherSelectorValue = s"driver-launcher-$launchTime" private val driverDockerImage = sparkConf.get( From 3fd9c62f4b9c23fe72901970d62687cc47a36c81 Mon Sep 17 00:00:00 2001 From: mccheah Date: Mon, 23 Jan 2017 18:02:45 -0800 Subject: [PATCH 24/81] A number of small tweaks to the MVP. (#23) * A number of small tweaks to the MVP. - Master protocol defaults to https if not specified - Removed upload driver extra classpath functionality - Added ability to specify main app resource with container:// URI - Updated docs to reflect all of the above - Add examples to Docker images, mostly for integration testing but could be useful for easily getting started without shipping anything * Add example to documentation. --- docs/running-on-kubernetes.md | 49 +++-- .../spark/deploy/kubernetes/Client.scala | 40 ++-- .../rest/KubernetesRestProtocolMessages.scala | 4 +- .../KubernetesSparkRestServer.scala | 183 ++++++++++-------- .../KubernetesClusterSchedulerBackend.scala | 9 +- .../kubernetes/docker-minimal-bundle/pom.xml | 7 + .../src/main/assembly/driver-assembly.xml | 20 +- .../src/main/assembly/executor-assembly.xml | 11 ++ .../src/main/docker/driver/Dockerfile | 1 + .../src/main/docker/executor/Dockerfile | 1 + .../integrationtest/KubernetesSuite.scala | 104 +++++++++- 11 files changed, 287 insertions(+), 142 deletions(-) diff --git a/docs/running-on-kubernetes.md b/docs/running-on-kubernetes.md index 14e2df4ed0702..5a73b1ad1ea29 100644 --- a/docs/running-on-kubernetes.md +++ b/docs/running-on-kubernetes.md @@ -42,11 +42,12 @@ are set up as described above: --conf spark.kubernetes.executor.docker.image=registry-host:5000/spark-executor:latest \ examples/jars/spark_examples_2.11-2.2.0.jar - The Spark master, specified either via passing the `--master` command line argument to `spark-submit` or by setting `spark.master` in the application's configuration, must be a URL with the format `k8s://`. Prefixing the master string with `k8s://` will cause the Spark application to launch on the Kubernetes cluster, with the API server -being contacted at `api_server_url`. The HTTP protocol must also be specified. +being contacted at `api_server_url`. If no HTTP protocol is specified in the URL, it defaults to `https`. For example, +setting the master to `k8s://example.com:443` is equivalent to setting it to `k8s://https://example.com:443`, but to +connect without SSL on a different port, the master would be set to `k8s://http://example.com:8443`. Note that applications can currently only be executed in cluster mode, where the driver and its executors are running on the cluster. @@ -58,17 +59,18 @@ disk of the submitter's machine. These two types of dependencies are specified v `spark-submit`: * Local jars provided by specifying the `--jars` command line argument to `spark-submit`, or by setting `spark.jars` in - the application's configuration, will be treated as jars that are located on the *disk of the driver Docker - container*. This only applies to jar paths that do not specify a scheme or that have the scheme `file://`. Paths with - other schemes are fetched from their appropriate locations. + the application's configuration, will be treated as jars that are located on the *disk of the driver container*. This + only applies to jar paths that do not specify a scheme or that have the scheme `file://`. Paths with other schemes are + fetched from their appropriate locations. * Local jars provided by specifying the `--upload-jars` command line argument to `spark-submit`, or by setting `spark.kubernetes.driver.uploads.jars` in the application's configuration, will be treated as jars that are located on the *disk of the submitting machine*. These jars are uploaded to the driver docker container before executing the application. - * A main application resource path that does not have a scheme or that has the scheme `file://` is assumed to be on the *disk of the submitting machine*. This resource is uploaded to the driver docker container before executing the application. A remote path can still be specified and the resource will be fetched from the appropriate location. +* A main application resource path that has the scheme `container://` is assumed to be on the *disk of the driver + container*. In all of these cases, the jars are placed on the driver's classpath, and are also sent to the executors. Below are some examples of providing application dependencies. @@ -78,8 +80,7 @@ To submit an application with both the main resource and two other jars living o bin/spark-submit \ --deploy-mode cluster \ --class com.example.applications.SampleApplication \ - --master k8s://https://192.168.99.100 \ - --kubernetes-namespace default \ + --master k8s://192.168.99.100 \ --upload-jars /home/exampleuser/exampleapplication/dep1.jar,/home/exampleuser/exampleapplication/dep2.jar \ --conf spark.kubernetes.driver.docker.image=registry-host:5000/spark-driver:latest \ --conf spark.kubernetes.executor.docker.image=registry-host:5000/spark-executor:latest \ @@ -91,8 +92,7 @@ Note that since passing the jars through the `--upload-jars` command line argume bin/spark-submit \ --deploy-mode cluster \ --class com.example.applications.SampleApplication \ - --master k8s://https://192.168.99.100 \ - --kubernetes-namespace default \ + --master k8s://192.168.99.100 \ --conf spark.kubernetes.driver.uploads.jars=/home/exampleuser/exampleapplication/dep1.jar,/home/exampleuser/exampleapplication/dep2.jar \ --conf spark.kubernetes.driver.docker.image=registry-host:5000/spark-driver:latest \ --conf spark.kubernetes.executor.docker.image=registry-host:5000/spark-executor:latest \ @@ -104,8 +104,7 @@ is located in the jar `/opt/spark-plugins/app-plugin.jar` on the docker image's bin/spark-submit \ --deploy-mode cluster \ --class com.example.applications.PluggableApplication \ - --master k8s://https://192.168.99.100 \ - --kubernetes-namespace default \ + --master k8s://192.168.99.100 \ --jars /opt/spark-plugins/app-plugin.jar \ --conf spark.kubernetes.driver.docker.image=registry-host:5000/spark-driver-custom:latest \ --conf spark.kubernetes.executor.docker.image=registry-host:5000/spark-executor:latest \ @@ -117,13 +116,22 @@ Spark property, the above will behave identically to this command: bin/spark-submit \ --deploy-mode cluster \ --class com.example.applications.PluggableApplication \ - --master k8s://https://192.168.99.100 \ - --kubernetes-namespace default \ + --master k8s://192.168.99.100 \ --conf spark.jars=file:///opt/spark-plugins/app-plugin.jar \ --conf spark.kubernetes.driver.docker.image=registry-host:5000/spark-driver-custom:latest \ --conf spark.kubernetes.executor.docker.image=registry-host:5000/spark-executor:latest \ http://example.com:8080/applications/sparkpluggable/app.jar +To specify a main application resource that is in the Docker image, and if it has no other dependencies: + + bin/spark-submit \ + --deploy-mode cluster \ + --class com.example.applications.PluggableApplication \ + --master k8s://192.168.99.100:8443 \ + --conf spark.kubernetes.driver.docker.image=registry-host:5000/spark-driver-custom:latest \ + --conf spark.kubernetes.executor.docker.image=registry-host:5000/spark-executor:latest \ + container:///home/applications/examples/example.jar + ### Spark Properties Below are some other common properties that are specific to Kubernetes. Most of the other configurations are the same @@ -133,10 +141,9 @@ from the other deployment modes. See the [configuration page](configuration.html Property NameDefaultMeaning spark.kubernetes.namespace - - (none) + default - The namespace that will be used for running the driver and executor pods. Must be specified. When using + The namespace that will be used for running the driver and executor pods. When using spark-submit in cluster mode, this can also be passed to spark-submit via the --kubernetes-namespace command line argument. @@ -196,14 +203,6 @@ from the other deployment modes. See the [configuration page](configuration.html mode. Refer to adding other jars for more information. - - - spark.kubernetes.driver.uploads.driverExtraClasspath - (none) - - Comma-separated list of jars to be sent to the driver only when submitting the application in cluster mode. - - spark.kubernetes.executor.memoryOverhead executorMemory * 0.10, with minimum of 384 diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/Client.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/Client.scala index 30eaa6269cf47..fe3256b9e12be 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/Client.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/Client.scala @@ -35,7 +35,7 @@ import scala.concurrent.duration.DurationInt import scala.util.Success import org.apache.spark.{SPARK_VERSION, SparkConf, SparkException} -import org.apache.spark.deploy.rest.{AppResource, KubernetesCreateSubmissionRequest, RemoteAppResource, TarGzippedData, UploadedAppResource} +import org.apache.spark.deploy.rest.{AppResource, ContainerAppResource, KubernetesCreateSubmissionRequest, RemoteAppResource, TarGzippedData, UploadedAppResource} import org.apache.spark.deploy.rest.kubernetes._ import org.apache.spark.internal.Logging import org.apache.spark.util.Utils @@ -47,13 +47,8 @@ private[spark] class Client( appArgs: Array[String]) extends Logging { import Client._ - private val namespace = sparkConf.getOption("spark.kubernetes.namespace").getOrElse( - throw new IllegalArgumentException("Namespace must be provided in spark.kubernetes.namespace")) - private val rawMaster = sparkConf.get("spark.master") - if (!rawMaster.startsWith("k8s://")) { - throw new IllegalArgumentException("Master should be a URL with scheme k8s://") - } - private val master = rawMaster.replaceFirst("k8s://", "") + private val namespace = sparkConf.get("spark.kubernetes.namespace", "default") + private val master = resolveK8sMaster(sparkConf.get("spark.master")) private val launchTime = System.currentTimeMillis private val appName = sparkConf.getOption("spark.app.name") @@ -64,8 +59,6 @@ private[spark] class Client( private val driverLauncherSelectorValue = s"driver-launcher-$launchTime" private val driverDockerImage = sparkConf.get( "spark.kubernetes.driver.docker.image", s"spark-driver:$SPARK_VERSION") - private val uploadedDriverExtraClasspath = sparkConf - .getOption("spark.kubernetes.driver.uploads.driverExtraClasspath") private val uploadedJars = sparkConf.getOption("spark.kubernetes.driver.uploads.jars") private val secretBase64String = { @@ -112,12 +105,15 @@ private[spark] class Client( .withType("Opaque") .done() try { - val resolvedSelectors = (Map(DRIVER_LAUNCHER_SELECTOR_LABEL -> driverLauncherSelectorValue) + val resolvedSelectors = (Map( + DRIVER_LAUNCHER_SELECTOR_LABEL -> driverLauncherSelectorValue, + SPARK_APP_NAME_LABEL -> appName) ++ parsedCustomLabels).asJava val (servicePorts, containerPorts) = configurePorts() val service = kubernetesClient.services().createNew() .withNewMetadata() .withName(kubernetesAppId) + .withLabels(Map(SPARK_APP_NAME_LABEL -> appName).asJava) .endMetadata() .withNewSpec() .withSelector(resolvedSelectors) @@ -355,10 +351,10 @@ private[spark] class Client( val fileBytes = Files.toByteArray(appFile) val fileBase64 = Base64.encodeBase64String(fileBytes) UploadedAppResource(resourceBase64Contents = fileBase64, name = appFile.getName) + case "container" => ContainerAppResource(appResourceUri.getPath) case other => RemoteAppResource(other) } - val uploadDriverExtraClasspathBase64Contents = compressJars(uploadedDriverExtraClasspath) val uploadJarsBase64Contents = compressJars(uploadedJars) KubernetesCreateSubmissionRequest( appResource = resolvedAppResource, @@ -366,7 +362,6 @@ private[spark] class Client( appArgs = appArgs, secret = secretBase64String, sparkProperties = sparkConf.getAll.toMap, - uploadedDriverExtraClasspathBase64Contents = uploadDriverExtraClasspathBase64Contents, uploadedJarsBase64Contents = uploadJarsBase64Contents) } @@ -414,7 +409,7 @@ private[spark] class Client( } } -private object Client { +private[spark] object Client extends Logging { private val SUBMISSION_SERVER_SECRET_NAME = "spark-submission-server-secret" private val DRIVER_LAUNCHER_SELECTOR_LABEL = "driver-launcher-selector" @@ -430,6 +425,7 @@ private object Client { private val SECURE_RANDOM = new SecureRandom() private val SPARK_SUBMISSION_SECRET_BASE_DIR = "/var/run/secrets/spark-submission" private val LAUNCH_TIMEOUT_SECONDS = 30 + private val SPARK_APP_NAME_LABEL = "spark-app-name" def main(args: Array[String]): Unit = { require(args.length >= 2, s"Too few arguments. Usage: ${getClass.getName} " + @@ -444,4 +440,20 @@ private object Client { sparkConf = sparkConf, appArgs = appArgs).run() } + + def resolveK8sMaster(rawMasterString: String): String = { + if (!rawMasterString.startsWith("k8s://")) { + throw new IllegalArgumentException("Master URL should start with k8s:// in Kubernetes mode.") + } + val masterWithoutK8sPrefix = rawMasterString.replaceFirst("k8s://", "") + if (masterWithoutK8sPrefix.startsWith("http://") + || masterWithoutK8sPrefix.startsWith("https://")) { + masterWithoutK8sPrefix + } else { + val resolvedURL = s"https://$masterWithoutK8sPrefix" + logDebug(s"No scheme specified for kubernetes master URL, so defaulting to https. Resolved" + + s" URL is $resolvedURL") + resolvedURL + } + } } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/KubernetesRestProtocolMessages.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/KubernetesRestProtocolMessages.scala index 6da1a848b25e7..813d070e0f876 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/KubernetesRestProtocolMessages.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/KubernetesRestProtocolMessages.scala @@ -27,7 +27,6 @@ case class KubernetesCreateSubmissionRequest( val appArgs: Array[String], val sparkProperties: Map[String, String], val secret: String, - val uploadedDriverExtraClasspathBase64Contents: Option[TarGzippedData], val uploadedJarsBase64Contents: Option[TarGzippedData]) extends SubmitRestProtocolRequest { message = "create" clientSparkVersion = SPARK_VERSION @@ -46,6 +45,7 @@ case class TarGzippedData( property = "type") @JsonSubTypes(value = Array( new JsonSubTypes.Type(value = classOf[UploadedAppResource], name = "UploadedAppResource"), + new JsonSubTypes.Type(value = classOf[ContainerAppResource], name = "ContainerLocalAppResource"), new JsonSubTypes.Type(value = classOf[RemoteAppResource], name = "RemoteAppResource"))) abstract class AppResource @@ -53,6 +53,8 @@ case class UploadedAppResource( resourceBase64Contents: String, name: String = "spark-app-resource") extends AppResource +case class ContainerAppResource(resourcePath: String) extends AppResource + case class RemoteAppResource(resource: String) extends AppResource class PingResponse extends SubmitRestProtocolResponse { diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/KubernetesSparkRestServer.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/KubernetesSparkRestServer.scala index 837706ca9f5a8..08ddbaf5e50dc 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/KubernetesSparkRestServer.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/KubernetesSparkRestServer.scala @@ -18,7 +18,6 @@ package org.apache.spark.deploy.rest.kubernetes import java.io.File import java.net.URI -import java.nio.file.Paths import java.util.concurrent.CountDownLatch import javax.servlet.http.{HttpServletRequest, HttpServletResponse} @@ -30,12 +29,12 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark.{SecurityManager, SPARK_VERSION, SparkConf} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.deploy.rest._ -import org.apache.spark.util.{ShutdownHookManager, Utils} +import org.apache.spark.util.{ShutdownHookManager, ThreadUtils, Utils} private case class KubernetesSparkRestServerArguments( - val host: Option[String] = None, - val port: Option[Int] = None, - val secretFile: Option[String] = None) { + val host: Option[String] = None, + val port: Option[Int] = None, + val secretFile: Option[String] = None) { def validate(): KubernetesSparkRestServerArguments = { require(host.isDefined, "Hostname not set via --hostname.") require(port.isDefined, "Port not set via --port") @@ -68,13 +67,21 @@ private object KubernetesSparkRestServerArguments { } } +/** + * Runs in the driver pod and receives a request to run an application. Note that + * unlike the submission rest server in standalone mode, this server is expected + * to be used to run one application only, and then shut down once that application + * is complete. + */ private[spark] class KubernetesSparkRestServer( host: String, port: Int, conf: SparkConf, - expectedApplicationSecret: Array[Byte]) + expectedApplicationSecret: Array[Byte], + shutdownLock: CountDownLatch) extends RestSubmissionServer(host, port, conf) { + private val SERVLET_LOCK = new Object private val javaExecutable = s"${System.getenv("JAVA_HOME")}/bin/java" private val sparkHome = System.getenv("SPARK_HOME") private val securityManager = new SecurityManager(conf) @@ -99,87 +106,105 @@ private[spark] class KubernetesSparkRestServer( private class KubernetesSubmitRequestServlet extends SubmitRequestServlet { + private val waitForProcessCompleteExecutor = ThreadUtils + .newDaemonSingleThreadExecutor("wait-for-spark-app-complete") + private var startedApplication = false + // TODO validating the secret should be done as part of a header of the request. // Instead here we have to specify the secret in the body. override protected def handleSubmit( - requestMessageJson: String, - requestMessage: SubmitRestProtocolMessage, - responseServlet: HttpServletResponse): SubmitRestProtocolResponse = { - requestMessage match { - case KubernetesCreateSubmissionRequest( + requestMessageJson: String, + requestMessage: SubmitRestProtocolMessage, + responseServlet: HttpServletResponse): SubmitRestProtocolResponse = { + SERVLET_LOCK.synchronized { + if (startedApplication) { + throw new IllegalStateException("Application has already been submitted.") + } else { + requestMessage match { + case KubernetesCreateSubmissionRequest( appResource, mainClass, appArgs, sparkProperties, secret, - uploadedDriverExtraClasspath, uploadedJars) => - val decodedSecret = Base64.decodeBase64(secret) - if (!expectedApplicationSecret.sameElements(decodedSecret)) { - responseServlet.setStatus(HttpServletResponse.SC_UNAUTHORIZED) - handleError("Unauthorized to submit application.") - } else { - val tempDir = Utils.createTempDir() - val appResourcePath = resolvedAppResource(appResource, tempDir) - val driverClasspathDirectory = new File(tempDir, "driver-extra-classpath") - if (!driverClasspathDirectory.mkdir) { - throw new IllegalStateException("Failed to create driver extra classpath" + - s" dir at ${driverClasspathDirectory.getAbsolutePath}") - } - val jarsDirectory = new File(tempDir, "jars") - if (!jarsDirectory.mkdir) { - throw new IllegalStateException("Failed to create jars dir at" + - s"${jarsDirectory.getAbsolutePath}") - } - val writtenDriverExtraClasspath = writeBase64ContentsToFiles( - uploadedDriverExtraClasspath, driverClasspathDirectory) - val writtenJars = writeBase64ContentsToFiles(uploadedJars, jarsDirectory) - val originalDriverExtraClasspath = sparkProperties.get("spark.driver.extraClassPath") - .map(_.split(",")) - .getOrElse(Array.empty[String]) - val resolvedDriverExtraClasspath = writtenDriverExtraClasspath ++ - originalDriverExtraClasspath - val originalJars = sparkProperties.get("spark.jars") - .map(_.split(",")) - .getOrElse(Array.empty[String]) - val resolvedJars = writtenJars ++ originalJars ++ Array(appResourcePath) - val sparkJars = new File(sparkHome, "jars").listFiles().map(_.getAbsolutePath) - val driverClasspath = resolvedDriverExtraClasspath ++ - resolvedJars ++ - sparkJars ++ - Array(appResourcePath) - val resolvedSparkProperties = new mutable.HashMap[String, String] - resolvedSparkProperties ++= sparkProperties - resolvedSparkProperties("spark.jars") = resolvedJars.mkString(",") - - val command = new ArrayBuffer[String] - command += javaExecutable - command += "-cp" - command += s"${driverClasspath.mkString(":")}" - for (prop <- resolvedSparkProperties) { - command += s"-D${prop._1}=${prop._2}" - } - val driverMemory = resolvedSparkProperties.getOrElse("spark.driver.memory", "1g") - command += s"-Xms$driverMemory" - command += s"-Xmx$driverMemory" - command += mainClass - command ++= appArgs - val pb = new ProcessBuilder(command: _*).inheritIO() - val process = pb.start() - ShutdownHookManager.addShutdownHook(() => { - logInfo("Received stop command, shutting down the running Spark application...") - process.destroy() - }) - val response = new CreateSubmissionResponse - response.success = true - response.submissionId = null - response.message = "success" - response.serverSparkVersion = SPARK_VERSION - response + val decodedSecret = Base64.decodeBase64(secret) + if (!expectedApplicationSecret.sameElements(decodedSecret)) { + responseServlet.setStatus(HttpServletResponse.SC_UNAUTHORIZED) + handleError("Unauthorized to submit application.") + } else { + val tempDir = Utils.createTempDir() + val appResourcePath = resolvedAppResource(appResource, tempDir) + val driverClasspathDirectory = new File(tempDir, "driver-extra-classpath") + if (!driverClasspathDirectory.mkdir) { + throw new IllegalStateException("Failed to create driver extra classpath" + + s" dir at ${driverClasspathDirectory.getAbsolutePath}") + } + val jarsDirectory = new File(tempDir, "jars") + if (!jarsDirectory.mkdir) { + throw new IllegalStateException("Failed to create jars dir at" + + s"${jarsDirectory.getAbsolutePath}") + } + val writtenJars = writeBase64ContentsToFiles(uploadedJars, jarsDirectory) + val driverExtraClasspath = sparkProperties + .get("spark.driver.extraClassPath") + .map(_.split(",")) + .getOrElse(Array.empty[String]) + val originalJars = sparkProperties.get("spark.jars") + .map(_.split(",")) + .getOrElse(Array.empty[String]) + val resolvedJars = writtenJars ++ originalJars ++ Array(appResourcePath) + val sparkJars = new File(sparkHome, "jars").listFiles().map(_.getAbsolutePath) + val driverClasspath = driverExtraClasspath ++ + resolvedJars ++ + sparkJars ++ + Array(appResourcePath) + val resolvedSparkProperties = new mutable.HashMap[String, String] + resolvedSparkProperties ++= sparkProperties + resolvedSparkProperties("spark.jars") = resolvedJars.mkString(",") + + val command = new ArrayBuffer[String] + command += javaExecutable + command += "-cp" + command += s"${driverClasspath.mkString(":")}" + for (prop <- resolvedSparkProperties) { + command += s"-D${prop._1}=${prop._2}" + } + val driverMemory = resolvedSparkProperties.getOrElse("spark.driver.memory", "1g") + command += s"-Xms$driverMemory" + command += s"-Xmx$driverMemory" + command += mainClass + command ++= appArgs + val pb = new ProcessBuilder(command: _*).inheritIO() + val process = pb.start() + ShutdownHookManager.addShutdownHook(() => { + logInfo("Received stop command, shutting down the running Spark application...") + process.destroy() + shutdownLock.countDown() + }) + waitForProcessCompleteExecutor.submit(new Runnable { + override def run(): Unit = { + process.waitFor + SERVLET_LOCK.synchronized { + logInfo("Spark application complete. Shutting down submission server...") + KubernetesSparkRestServer.this.stop + shutdownLock.countDown() + } + } + }) + startedApplication = true + val response = new CreateSubmissionResponse + response.success = true + response.submissionId = null + response.message = "success" + response.serverSparkVersion = SPARK_VERSION + response + } + case unexpected => + responseServlet.setStatus(HttpServletResponse.SC_BAD_REQUEST) + handleError(s"Received message of unexpected type ${unexpected.messageType}.") } - case unexpected => - responseServlet.setStatus(HttpServletResponse.SC_BAD_REQUEST) - handleError(s"Received message of unexpected type ${unexpected.messageType}.") + } } } @@ -196,6 +221,7 @@ private[spark] class KubernetesSparkRestServer( throw new IllegalStateException(s"Failed to write main app resource file" + s" to $resourceFilePath") } + case ContainerAppResource(resource) => resource case RemoteAppResource(resource) => Utils.fetchFile(resource, tempDir, conf, securityManager, SparkHadoopUtil.get.newConfiguration(conf), @@ -237,7 +263,8 @@ private[spark] object KubernetesSparkRestServer { parsedArguments.host.get, parsedArguments.port.get, sparkConf, - secretBytes) + secretBytes, + barrier) server.start() ShutdownHookManager.addShutdownHook(() => { try { diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala index 2717d2f37d910..b7110ba901842 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala @@ -27,7 +27,7 @@ import scala.collection.mutable.ArrayBuffer import scala.concurrent.{ExecutionContext, Future} import org.apache.spark.{SparkContext, SparkException} -import org.apache.spark.deploy.kubernetes.KubernetesClientBuilder +import org.apache.spark.deploy.kubernetes.{Client, KubernetesClientBuilder} import org.apache.spark.rpc.RpcEndpointAddress import org.apache.spark.scheduler.TaskSchedulerImpl import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend @@ -43,15 +43,12 @@ private[spark] class KubernetesClusterSchedulerBackend( private val EXECUTOR_MODIFICATION_LOCK = new Object private val runningExecutorPods = new scala.collection.mutable.HashMap[String, Pod] - private val kubernetesMaster = sc.master.replaceFirst("k8s://", "") + private val kubernetesMaster = Client.resolveK8sMaster(sc.master) private val executorDockerImage = conf .get("spark.kubernetes.executor.docker.image", s"spark-executor:${sc.version}") - private val kubernetesNamespace = conf - .getOption("spark.kubernetes.namespace") - .getOrElse( - throw new SparkException("Kubernetes namespace must be specified in kubernetes mode.")) + private val kubernetesNamespace = conf.get("spark.kubernetes.namespace", "default") private val executorPort = conf.getInt("spark.executor.port", DEFAULT_STATIC_PORT) diff --git a/resource-managers/kubernetes/docker-minimal-bundle/pom.xml b/resource-managers/kubernetes/docker-minimal-bundle/pom.xml index c20e51c93e7c7..0ec2f36075db3 100644 --- a/resource-managers/kubernetes/docker-minimal-bundle/pom.xml +++ b/resource-managers/kubernetes/docker-minimal-bundle/pom.xml @@ -43,6 +43,13 @@ ${project.version} pom
    + + + org.apache.spark + spark-examples_${scala.binary.version} + ${project.version} + provided + diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/Client.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/Client.scala index 77b7c793dc37e..07a45c7577bcd 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/Client.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/Client.scala @@ -16,25 +16,25 @@ */ package org.apache.spark.deploy.kubernetes -import java.io.File -import java.security.SecureRandom +import java.io.{File, FileInputStream} +import java.security.{KeyStore, SecureRandom} import java.util.concurrent.{Executors, TimeoutException, TimeUnit} -import javax.net.ssl.X509TrustManager +import java.util.concurrent.atomic.AtomicBoolean +import javax.net.ssl.{SSLContext, TrustManagerFactory, X509TrustManager} +import com.google.common.base.Charsets import com.google.common.io.Files import com.google.common.util.concurrent.{SettableFuture, ThreadFactoryBuilder} import io.fabric8.kubernetes.api.model._ -import io.fabric8.kubernetes.client.{Config, ConfigBuilder, DefaultKubernetesClient, KubernetesClientException, Watch, Watcher} +import io.fabric8.kubernetes.client.{ConfigBuilder, DefaultKubernetesClient, KubernetesClient, KubernetesClientException, Watcher} import io.fabric8.kubernetes.client.Watcher.Action -import io.fabric8.kubernetes.client.internal.SSLUtils import org.apache.commons.codec.binary.Base64 import scala.collection.JavaConverters._ -import scala.collection.mutable.ArrayBuffer -import scala.concurrent.ExecutionContext +import scala.collection.mutable +import scala.concurrent.{ExecutionContext, Future} import scala.concurrent.duration.DurationInt -import scala.util.Success -import org.apache.spark.{SPARK_VERSION, SparkConf, SparkException} +import org.apache.spark.{SecurityManager, SPARK_VERSION => sparkVersion, SparkConf, SparkException, SSLOptions} import org.apache.spark.deploy.rest.{AppResource, ContainerAppResource, KubernetesCreateSubmissionRequest, RemoteAppResource, TarGzippedData, UploadedAppResource} import org.apache.spark.deploy.rest.kubernetes._ import org.apache.spark.internal.Logging @@ -56,10 +56,14 @@ private[spark] class Client( .getOrElse("spark") private val kubernetesAppId = s"$appName-$launchTime".toLowerCase.replaceAll("\\.", "-") private val secretName = s"spark-submission-server-secret-$kubernetesAppId" + private val secretDirectory = s"$SPARK_SUBMISSION_SECRET_BASE_DIR/$kubernetesAppId" + private val sslSecretsDirectory = s"$SPARK_SUBMISSION_SECRET_BASE_DIR/$kubernetesAppId-ssl" + private val sslSecretsName = s"spark-submission-server-ssl-$kubernetesAppId" private val driverLauncherSelectorValue = s"driver-launcher-$launchTime" private val driverDockerImage = sparkConf.get( - "spark.kubernetes.driver.docker.image", s"spark-driver:$SPARK_VERSION") + "spark.kubernetes.driver.docker.image", s"spark-driver:$sparkVersion") private val uploadedJars = sparkConf.getOption("spark.kubernetes.driver.uploads.jars") + private val uiPort = sparkConf.getInt("spark.ui.port", DEFAULT_UI_PORT) private val driverLaunchTimeoutSecs = sparkConf.getTimeAsSeconds( "spark.kubernetes.driverLaunchTimeout", s"${DEFAULT_LAUNCH_TIMEOUT_SECONDS}s") @@ -82,6 +86,7 @@ private[spark] class Client( .build())) def run(): Unit = { + val (driverLaunchSslOptions, isKeyStoreLocalFile) = parseDriverLaunchSslOptions() val parsedCustomLabels = parseCustomLabels(customLabels) var k8ConfBuilder = new ConfigBuilder() .withApiVersion("v1") @@ -98,123 +103,50 @@ private[spark] class Client( } val k8ClientConfig = k8ConfBuilder.build - Utils.tryWithResource(new DefaultKubernetesClient(k8ClientConfig))(kubernetesClient => { - val secret = kubernetesClient.secrets().createNew() + Utils.tryWithResource(new DefaultKubernetesClient(k8ClientConfig)) { kubernetesClient => + val submitServerSecret = kubernetesClient.secrets().createNew() .withNewMetadata() - .withName(secretName) - .endMetadata() + .withName(secretName) + .endMetadata() .withData(Map((SUBMISSION_SERVER_SECRET_NAME, secretBase64String)).asJava) .withType("Opaque") .done() + val (sslEnvs, sslVolumes, sslVolumeMounts, sslSecrets) = configureSsl(kubernetesClient, + driverLaunchSslOptions, + isKeyStoreLocalFile) try { - val resolvedSelectors = (Map( + val driverKubernetesSelectors = (Map( DRIVER_LAUNCHER_SELECTOR_LABEL -> driverLauncherSelectorValue, SPARK_APP_NAME_LABEL -> appName) ++ parsedCustomLabels).asJava - val (servicePorts, containerPorts) = configurePorts() - val service = kubernetesClient.services().createNew() - .withNewMetadata() - .withName(kubernetesAppId) - .withLabels(Map(SPARK_APP_NAME_LABEL -> appName).asJava) - .endMetadata() - .withNewSpec() - .withSelector(resolvedSelectors) - .withPorts(servicePorts.asJava) - .endSpec() - .done() - sparkConf.set("spark.kubernetes.driver.service.name", service.getMetadata.getName) - sparkConf.set("spark.kubernetes.driver.pod.name", kubernetesAppId) - - sparkConf.setIfMissing("spark.driver.port", DEFAULT_DRIVER_PORT.toString) - sparkConf.setIfMissing("spark.blockmanager.port", DEFAULT_BLOCKMANAGER_PORT.toString) - val submitRequest = buildSubmissionRequest() + val containerPorts = buildContainerPorts() val submitCompletedFuture = SettableFuture.create[Boolean] - val secretDirectory = s"$SPARK_SUBMISSION_SECRET_BASE_DIR/$kubernetesAppId" - - val podWatcher = new Watcher[Pod] { - override def eventReceived(action: Action, t: Pod): Unit = { - if (action == Action.ADDED) { - val ownerRefs = new ArrayBuffer[OwnerReference] - ownerRefs += new OwnerReferenceBuilder() - .withApiVersion(t.getApiVersion) - .withController(true) - .withKind(t.getKind) - .withName(t.getMetadata.getName) - .withUid(t.getMetadata.getUid) - .build() - - secret.getMetadata().setOwnerReferences(ownerRefs.asJava) - kubernetesClient.secrets().createOrReplace(secret) - - service.getMetadata().setOwnerReferences(ownerRefs.asJava) - kubernetesClient.services().createOrReplace(service) - } - - if ((action == Action.ADDED || action == Action.MODIFIED) - && t.getStatus.getPhase == "Running" - && !submitCompletedFuture.isDone) { - t.getStatus - .getContainerStatuses - .asScala - .find(status => - status.getName == DRIVER_LAUNCHER_CONTAINER_NAME && status.getReady) match { - case Some(_) => - try { - val driverLauncher = getDriverLauncherService( - k8ClientConfig, master) - val ping = Retry.retry(5, 5.seconds) { - driverLauncher.ping() - } - ping onFailure { - case t: Throwable => - if (!submitCompletedFuture.isDone) { - submitCompletedFuture.setException(t) - } - } - val submitComplete = ping andThen { - case Success(_) => - driverLauncher.create(submitRequest) - submitCompletedFuture.set(true) - } - submitComplete onFailure { - case t: Throwable => - if (!submitCompletedFuture.isDone) { - submitCompletedFuture.setException(t) - } - } - } catch { - case e: Throwable => - if (!submitCompletedFuture.isDone) { - submitCompletedFuture.setException(e) - throw e - } - } - case None => - } - } - } - - override def onClose(e: KubernetesClientException): Unit = { - if (!submitCompletedFuture.isDone) { - submitCompletedFuture.setException(e) - } - } - } - - def createDriverPod(unused: Watch): Unit = { + val submitPending = new AtomicBoolean(false) + val podWatcher = new DriverPodWatcher( + submitCompletedFuture, + submitPending, + kubernetesClient, + driverLaunchSslOptions, + Array(submitServerSecret) ++ sslSecrets, + driverKubernetesSelectors) + Utils.tryWithResource(kubernetesClient + .pods() + .withLabels(driverKubernetesSelectors) + .watch(podWatcher)) { _ => kubernetesClient.pods().createNew() .withNewMetadata() .withName(kubernetesAppId) - .withLabels(resolvedSelectors) + .withLabels(driverKubernetesSelectors) .endMetadata() .withNewSpec() .withRestartPolicy("OnFailure") .addNewVolume() .withName(s"spark-submission-secret-volume") - .withNewSecret() - .withSecretName(secret.getMetadata.getName) + .withNewSecret() + .withSecretName(submitServerSecret.getMetadata.getName) .endSecret() .endVolume + .addToVolumes(sslVolumes: _*) .withServiceAccount(serviceAccount) .addNewContainer() .withName(DRIVER_LAUNCHER_CONTAINER_NAME) @@ -225,6 +157,7 @@ private[spark] class Client( .withMountPath(secretDirectory) .withReadOnly(true) .endVolumeMount() + .addToVolumeMounts(sslVolumeMounts: _*) .addNewEnv() .withName("SPARK_SUBMISSION_SECRET_LOCATION") .withValue(s"$secretDirectory/$SUBMISSION_SERVER_SECRET_NAME") @@ -233,6 +166,7 @@ private[spark] class Client( .withName("SPARK_DRIVER_LAUNCHER_SERVER_PORT") .withValue(DRIVER_LAUNCHER_SERVICE_INTERNAL_PORT.toString) .endEnv() + .addToEnv(sslEnvs: _*) .withPorts(containerPorts.asJava) .endContainer() .endSpec() @@ -243,121 +177,321 @@ private[spark] class Client( submitSucceeded = true } catch { case e: TimeoutException => - val driverPod = try { - kubernetesClient.pods().withName(kubernetesAppId).get() - } catch { - case throwable: Throwable => - logError(s"Timed out while waiting $driverLaunchTimeoutSecs seconds for the" + - " driver pod to start, but an error occurred while fetching the driver" + - " pod's details.", throwable) - throw new SparkException(s"Timed out while waiting $driverLaunchTimeoutSecs" + - " seconds for the driver pod to start. Unfortunately, in attempting to fetch" + - " the latest state of the pod, another error was thrown. Check the logs for" + - " the error that was thrown in looking up the driver pod.", e) - } - val topLevelMessage = s"The driver pod with name ${driverPod.getMetadata.getName}" + - s" in namespace ${driverPod.getMetadata.getNamespace} was not ready in" + - s" $driverLaunchTimeoutSecs seconds." - val podStatusPhase = if (driverPod.getStatus.getPhase != null) { - s"Latest phase from the pod is: ${driverPod.getStatus.getPhase}" - } else { - "The pod had no final phase." - } - val podStatusMessage = if (driverPod.getStatus.getMessage != null) { - s"Latest message from the pod is: ${driverPod.getStatus.getMessage}" - } else { - "The pod had no final message." - } - val failedDriverContainerStatusString = driverPod.getStatus - .getContainerStatuses - .asScala - .find(_.getName == DRIVER_LAUNCHER_CONTAINER_NAME) - .map(status => { - val lastState = status.getState - if (lastState.getRunning != null) { - "Driver container last state: Running\n" + - s"Driver container started at: ${lastState.getRunning.getStartedAt}" - } else if (lastState.getWaiting != null) { - "Driver container last state: Waiting\n" + - s"Driver container wait reason: ${lastState.getWaiting.getReason}\n" + - s"Driver container message: ${lastState.getWaiting.getMessage}\n" - } else if (lastState.getTerminated != null) { - "Driver container last state: Terminated\n" + - s"Driver container started at: ${lastState.getTerminated.getStartedAt}\n" + - s"Driver container finished at: ${lastState.getTerminated.getFinishedAt}\n" + - s"Driver container exit reason: ${lastState.getTerminated.getReason}\n" + - s"Driver container exit code: ${lastState.getTerminated.getExitCode}\n" + - s"Driver container message: ${lastState.getTerminated.getMessage}" - } else { - "Driver container last state: Unknown" - } - }).getOrElse("The driver container wasn't found in the pod; expected to find" + - s" container with name $DRIVER_LAUNCHER_CONTAINER_NAME") - val finalErrorMessage = s"$topLevelMessage\n" + - s"$podStatusPhase\n" + - s"$podStatusMessage\n\n$failedDriverContainerStatusString" + val finalErrorMessage: String = buildSubmitFailedErrorMessage(kubernetesClient, e) logError(finalErrorMessage, e) throw new SparkException(finalErrorMessage, e) - } finally { - if (!submitSucceeded) { - try { - kubernetesClient.pods.withName(kubernetesAppId).delete - } catch { - case throwable: Throwable => - logError("Failed to delete driver pod after it failed to run.", throwable) - } + } finally { + if (!submitSucceeded) { + Utils.tryLogNonFatalError { + kubernetesClient.pods.withName(kubernetesAppId).delete() } } } - - Utils.tryWithResource(kubernetesClient - .pods() - .withLabels(resolvedSelectors) - .watch(podWatcher)) { createDriverPod } + } } finally { - kubernetesClient.secrets().delete(secret) + Utils.tryLogNonFatalError { + kubernetesClient.secrets().delete(submitServerSecret) + } + Utils.tryLogNonFatalError { + kubernetesClient.secrets().delete(sslSecrets: _*) + } } - }) + } } - private def configurePorts(): (Seq[ServicePort], Seq[ContainerPort]) = { - val servicePorts = new ArrayBuffer[ServicePort] - val containerPorts = new ArrayBuffer[ContainerPort] + private def parseDriverLaunchSslOptions(): (SSLOptions, Boolean) = { + val maybeKeyStore = sparkConf.getOption("spark.ssl.kubernetes.driverlaunch.keyStore") + val resolvedSparkConf = sparkConf.clone() + val (isLocalKeyStore, resolvedKeyStore) = maybeKeyStore.map(keyStore => { + val keyStoreURI = Utils.resolveURI(keyStore) + val isProvidedKeyStoreLocal = keyStoreURI.getScheme match { + case "file" | null => true + case "container" => false + case _ => throw new SparkException(s"Invalid KeyStore URI $keyStore; keyStore URI" + + " for submit server must have scheme file:// or container:// (no scheme defaults" + + " to file://)") + } + (isProvidedKeyStoreLocal, Option.apply(keyStoreURI.getPath)) + }).getOrElse((true, Option.empty[String])) + resolvedKeyStore.foreach { + resolvedSparkConf.set("spark.ssl.kubernetes.driverlaunch.keyStore", _) + } + sparkConf.getOption("spark.ssl.kubernetes.driverlaunch.trustStore").foreach { trustStore => + val trustStoreURI = Utils.resolveURI(trustStore) + trustStoreURI.getScheme match { + case "file" | null => + resolvedSparkConf.set("spark.ssl.kubernetes.driverlaunch.trustStore", + trustStoreURI.getPath) + case _ => throw new SparkException(s"Invalid trustStore URI $trustStore; trustStore URI" + + " for submit server must have no scheme, or scheme file://") + } + } + val securityManager = new SecurityManager(resolvedSparkConf) + (securityManager.getSSLOptions("kubernetes.driverlaunch"), isLocalKeyStore) + } - def addPortToServiceAndContainer(portName: String, portValue: Int): Unit = { - servicePorts += new ServicePortBuilder() - .withName(portName) - .withPort(portValue) - .withNewTargetPort(portValue) + private def configureSsl(kubernetesClient: KubernetesClient, driverLaunchSslOptions: SSLOptions, + isKeyStoreLocalFile: Boolean): + (Array[EnvVar], Array[Volume], Array[VolumeMount], Array[Secret]) = { + if (driverLaunchSslOptions.enabled) { + val sslSecretsMap = mutable.HashMap[String, String]() + val sslEnvs = mutable.Buffer[EnvVar]() + val secrets = mutable.Buffer[Secret]() + driverLaunchSslOptions.keyStore.foreach(store => { + val resolvedKeyStoreFile = if (isKeyStoreLocalFile) { + if (!store.isFile) { + throw new SparkException(s"KeyStore specified at $store is not a file or" + + s" does not exist.") + } + val keyStoreBytes = Files.toByteArray(store) + val keyStoreBase64 = Base64.encodeBase64String(keyStoreBytes) + sslSecretsMap += (SSL_KEYSTORE_SECRET_NAME -> keyStoreBase64) + s"$sslSecretsDirectory/$SSL_KEYSTORE_SECRET_NAME" + } else { + store.getAbsolutePath + } + sslEnvs += new EnvVarBuilder() + .withName("SPARK_SUBMISSION_KEYSTORE_FILE") + .withValue(resolvedKeyStoreFile) + .build() + }) + driverLaunchSslOptions.keyStorePassword.foreach(password => { + val passwordBase64 = Base64.encodeBase64String(password.getBytes(Charsets.UTF_8)) + sslSecretsMap += (SSL_KEYSTORE_PASSWORD_SECRET_NAME -> passwordBase64) + sslEnvs += new EnvVarBuilder() + .withName("SPARK_SUBMISSION_KEYSTORE_PASSWORD_FILE") + .withValue(s"$sslSecretsDirectory/$SSL_KEYSTORE_PASSWORD_SECRET_NAME") + .build() + }) + driverLaunchSslOptions.keyPassword.foreach(password => { + val passwordBase64 = Base64.encodeBase64String(password.getBytes(Charsets.UTF_8)) + sslSecretsMap += (SSL_KEY_PASSWORD_SECRET_NAME -> passwordBase64) + sslEnvs += new EnvVarBuilder() + .withName("SPARK_SUBMISSION_KEYSTORE_KEY_PASSWORD_FILE") + .withValue(s"$sslSecretsDirectory/$SSL_KEY_PASSWORD_SECRET_NAME") + .build() + }) + driverLaunchSslOptions.keyStoreType.foreach(storeType => { + sslEnvs += new EnvVarBuilder() + .withName("SPARK_SUBMISSION_KEYSTORE_TYPE") + .withValue(storeType) + .build() + }) + sslEnvs += new EnvVarBuilder() + .withName("SPARK_SUBMISSION_USE_SSL") + .withValue("true") + .build() + val sslSecrets = kubernetesClient.secrets().createNew() + .withNewMetadata() + .withName(sslSecretsName) + .endMetadata() + .withData(sslSecretsMap.asJava) + .withType("Opaque") + .done() + secrets += sslSecrets + val sslVolume = new VolumeBuilder() + .withName("spark-submission-server-ssl-secrets") + .withNewSecret() + .withSecretName(sslSecrets.getMetadata.getName) + .endSecret() .build() - containerPorts += new ContainerPortBuilder() - .withContainerPort(portValue) + val sslVolumeMount = new VolumeMountBuilder() + .withName("spark-submission-server-ssl-secrets") + .withReadOnly(true) + .withMountPath(sslSecretsDirectory) .build() + (sslEnvs.toArray, Array(sslVolume), Array(sslVolumeMount), secrets.toArray) + } else { + (Array[EnvVar](), Array[Volume](), Array[VolumeMount](), Array[Secret]()) } + } + + private class DriverPodWatcher( + submitCompletedFuture: SettableFuture[Boolean], + submitPending: AtomicBoolean, + kubernetesClient: KubernetesClient, + driverLaunchSslOptions: SSLOptions, + applicationSecrets: Array[Secret], + driverKubernetesSelectors: java.util.Map[String, String]) extends Watcher[Pod] { + override def eventReceived(action: Action, pod: Pod): Unit = { + if ((action == Action.ADDED || action == Action.MODIFIED) + && pod.getStatus.getPhase == "Running" + && !submitCompletedFuture.isDone) { + if (!submitPending.getAndSet(true)) { + pod.getStatus + .getContainerStatuses + .asScala + .find(status => + status.getName == DRIVER_LAUNCHER_CONTAINER_NAME && status.getReady) match { + case Some(_) => + val ownerRefs = Seq(new OwnerReferenceBuilder() + .withName(pod.getMetadata.getName) + .withUid(pod.getMetadata.getUid) + .withApiVersion(pod.getApiVersion) + .withKind(pod.getKind) + .withController(true) + .build()) + + applicationSecrets.foreach(secret => { + secret.getMetadata.setOwnerReferences(ownerRefs.asJava) + kubernetesClient.secrets().createOrReplace(secret) + }) - addPortToServiceAndContainer( - DRIVER_LAUNCHER_SERVICE_PORT_NAME, - DRIVER_LAUNCHER_SERVICE_INTERNAL_PORT) - addPortToServiceAndContainer( - DRIVER_PORT_NAME, - sparkConf - .getOption("spark.driver.port") - .map(_.toInt) - .getOrElse(DEFAULT_DRIVER_PORT)) - addPortToServiceAndContainer( - BLOCKMANAGER_PORT_NAME, - sparkConf - .getOption("spark.blockmanager.port") - .map(_.toInt) - .getOrElse(DEFAULT_BLOCKMANAGER_PORT)) + val driverLauncherServicePort = new ServicePortBuilder() + .withName(DRIVER_LAUNCHER_SERVICE_PORT_NAME) + .withPort(DRIVER_LAUNCHER_SERVICE_INTERNAL_PORT) + .withNewTargetPort(DRIVER_LAUNCHER_SERVICE_INTERNAL_PORT) + .build() + val service = kubernetesClient.services().createNew() + .withNewMetadata() + .withName(kubernetesAppId) + .withLabels(driverKubernetesSelectors) + .withOwnerReferences(ownerRefs.asJava) + .endMetadata() + .withNewSpec() + .withType("NodePort") + .withSelector(driverKubernetesSelectors) + .withPorts(driverLauncherServicePort) + .endSpec() + .done() + try { + sparkConf.set("spark.kubernetes.driver.service.name", + service.getMetadata.getName) + sparkConf.set("spark.kubernetes.driver.pod.name", kubernetesAppId) + sparkConf.setIfMissing("spark.driver.port", DEFAULT_DRIVER_PORT.toString) + sparkConf.setIfMissing("spark.blockmanager.port", + DEFAULT_BLOCKMANAGER_PORT.toString) + val driverLauncher = buildDriverLauncherClient(kubernetesClient, service, + driverLaunchSslOptions) + val ping = Retry.retry(5, 5.seconds) { + driverLauncher.ping() + } + ping onFailure { + case t: Throwable => + submitCompletedFuture.setException(t) + kubernetesClient.services().delete(service) + } + val submitComplete = ping.flatMap { _ => + Future { + sparkConf.set("spark.driver.host", pod.getStatus.getPodIP) + val submitRequest = buildSubmissionRequest() + driverLauncher.create(submitRequest) + } + } + submitComplete onFailure { + case t: Throwable => + submitCompletedFuture.setException(t) + kubernetesClient.services().delete(service) + } + val adjustServicePort = submitComplete.flatMap { _ => + Future { + // After submitting, adjust the service to only expose the Spark UI + val uiServicePort = new ServicePortBuilder() + .withName(UI_PORT_NAME) + .withPort(uiPort) + .withNewTargetPort(uiPort) + .build() + kubernetesClient.services().withName(kubernetesAppId).edit() + .editSpec() + .withType("ClusterIP") + .withPorts(uiServicePort) + .endSpec() + .done + } + } + adjustServicePort onSuccess { + case _ => + submitCompletedFuture.set(true) + } + adjustServicePort onFailure { + case throwable: Throwable => + submitCompletedFuture.setException(throwable) + kubernetesClient.services().delete(service) + } + } catch { + case e: Throwable => + submitCompletedFuture.setException(e) + Utils.tryLogNonFatalError({ + kubernetesClient.services().delete(service) + }) + throw e + } + case None => + } + } + } + } - addPortToServiceAndContainer( - UI_PORT_NAME, - sparkConf - .getOption("spark.ui.port") - .map(_.toInt) - .getOrElse(DEFAULT_UI_PORT)) - (servicePorts, containerPorts) + override def onClose(e: KubernetesClientException): Unit = { + if (!submitCompletedFuture.isDone) { + submitCompletedFuture.setException(e) + } + } + } + + private def buildSubmitFailedErrorMessage( + kubernetesClient: DefaultKubernetesClient, + e: TimeoutException): String = { + val driverPod = try { + kubernetesClient.pods().withName(kubernetesAppId).get() + } catch { + case throwable: Throwable => + logError(s"Timed out while waiting $driverLaunchTimeoutSecs seconds for the" + + " driver pod to start, but an error occurred while fetching the driver" + + " pod's details.", throwable) + throw new SparkException(s"Timed out while waiting $driverLaunchTimeoutSecs" + + " seconds for the driver pod to start. Unfortunately, in attempting to fetch" + + " the latest state of the pod, another error was thrown. Check the logs for" + + " the error that was thrown in looking up the driver pod.", e) + } + val topLevelMessage = s"The driver pod with name ${driverPod.getMetadata.getName}" + + s" in namespace ${driverPod.getMetadata.getNamespace} was not ready in" + + s" $driverLaunchTimeoutSecs seconds." + val podStatusPhase = if (driverPod.getStatus.getPhase != null) { + s"Latest phase from the pod is: ${driverPod.getStatus.getPhase}" + } else { + "The pod had no final phase." + } + val podStatusMessage = if (driverPod.getStatus.getMessage != null) { + s"Latest message from the pod is: ${driverPod.getStatus.getMessage}" + } else { + "The pod had no final message." + } + val failedDriverContainerStatusString = driverPod.getStatus + .getContainerStatuses + .asScala + .find(_.getName == DRIVER_LAUNCHER_CONTAINER_NAME) + .map(status => { + val lastState = status.getState + if (lastState.getRunning != null) { + "Driver container last state: Running\n" + + s"Driver container started at: ${lastState.getRunning.getStartedAt}" + } else if (lastState.getWaiting != null) { + "Driver container last state: Waiting\n" + + s"Driver container wait reason: ${lastState.getWaiting.getReason}\n" + + s"Driver container message: ${lastState.getWaiting.getMessage}\n" + } else if (lastState.getTerminated != null) { + "Driver container last state: Terminated\n" + + s"Driver container started at: ${lastState.getTerminated.getStartedAt}\n" + + s"Driver container finished at: ${lastState.getTerminated.getFinishedAt}\n" + + s"Driver container exit reason: ${lastState.getTerminated.getReason}\n" + + s"Driver container exit code: ${lastState.getTerminated.getExitCode}\n" + + s"Driver container message: ${lastState.getTerminated.getMessage}" + } else { + "Driver container last state: Unknown" + } + }).getOrElse("The driver container wasn't found in the pod; expected to find" + + s" container with name $DRIVER_LAUNCHER_CONTAINER_NAME") + s"$topLevelMessage\n" + + s"$podStatusPhase\n" + + s"$podStatusMessage\n\n$failedDriverContainerStatusString" + } + + private def buildContainerPorts(): Seq[ContainerPort] = { + Seq(sparkConf.getInt("spark.driver.port", DEFAULT_DRIVER_PORT), + sparkConf.getInt("spark.blockManager.port", DEFAULT_BLOCKMANAGER_PORT), + DRIVER_LAUNCHER_SERVICE_INTERNAL_PORT, + uiPort).map(new ContainerPortBuilder().withContainerPort(_).build()) } private def buildSubmissionRequest(): KubernetesCreateSubmissionRequest = { @@ -392,26 +526,67 @@ private[spark] class Client( .map(CompressionUtils.createTarGzip(_)) } - private def getDriverLauncherService( - k8ClientConfig: Config, - kubernetesMaster: String): KubernetesSparkRestApi = { - val url = s"${ - Array[String]( - kubernetesMaster, - "api", "v1", "proxy", - "namespaces", namespace, - "services", kubernetesAppId).mkString("/")}" + - s":$DRIVER_LAUNCHER_SERVICE_PORT_NAME/" - - val sslContext = SSLUtils.sslContext(k8ClientConfig) - val trustManager = SSLUtils.trustManagers( - k8ClientConfig)(0).asInstanceOf[X509TrustManager] + private def buildDriverLauncherClient( + kubernetesClient: KubernetesClient, + service: Service, + driverLaunchSslOptions: SSLOptions): KubernetesSparkRestApi = { + val servicePort = service + .getSpec + .getPorts + .asScala + .filter(_.getName == DRIVER_LAUNCHER_SERVICE_PORT_NAME) + .head + .getNodePort + // NodePort is exposed on every node, so just pick one of them. + // TODO be resilient to node failures and try all of them + val node = kubernetesClient.nodes.list.getItems.asScala.head + val nodeAddress = node.getStatus.getAddresses.asScala.head.getAddress + val urlScheme = if (driverLaunchSslOptions.enabled) { + "https" + } else { + logWarning("Submitting application details, application secret, and local" + + " jars to the cluster over an insecure connection. You should configure SSL" + + " to secure this step.") + "http" + } + val (trustManager, sslContext): (X509TrustManager, SSLContext) = + if (driverLaunchSslOptions.enabled) { + buildSslConnectionConfiguration(driverLaunchSslOptions) + } else { + (null, SSLContext.getDefault) + } + val url = s"$urlScheme://$nodeAddress:$servicePort" HttpClientUtil.createClient[KubernetesSparkRestApi]( - uri = url, + url, sslSocketFactory = sslContext.getSocketFactory, trustContext = trustManager) } + private def buildSslConnectionConfiguration(driverLaunchSslOptions: SSLOptions) = { + driverLaunchSslOptions.trustStore.map(trustStoreFile => { + val trustManagerFactory = TrustManagerFactory.getInstance( + TrustManagerFactory.getDefaultAlgorithm) + val trustStore = KeyStore.getInstance( + driverLaunchSslOptions.trustStoreType.getOrElse(KeyStore.getDefaultType)) + if (!trustStoreFile.isFile) { + throw new SparkException(s"TrustStore file at ${trustStoreFile.getAbsolutePath}" + + s" does not exist or is not a file.") + } + Utils.tryWithResource(new FileInputStream(trustStoreFile)) { trustStoreStream => + driverLaunchSslOptions.trustStorePassword match { + case Some(password) => + trustStore.load(trustStoreStream, password.toCharArray) + case None => trustStore.load(trustStoreStream, null) + } + } + trustManagerFactory.init(trustStore) + val trustManagers = trustManagerFactory.getTrustManagers + val sslContext = SSLContext.getInstance("TLSv1.2") + sslContext.init(null, trustManagers, SECURE_RANDOM) + (trustManagers(0).asInstanceOf[X509TrustManager], sslContext) + }).getOrElse((null, SSLContext.getDefault)) + } + private def parseCustomLabels(labels: String): Map[String, String] = { labels.split(",").map(_.trim).filterNot(_.isEmpty).map(label => { label.split("=", 2).toSeq match { @@ -433,6 +608,9 @@ private[spark] class Client( private[spark] object Client extends Logging { private val SUBMISSION_SERVER_SECRET_NAME = "spark-submission-server-secret" + private val SSL_KEYSTORE_SECRET_NAME = "spark-submission-server-keystore" + private val SSL_KEYSTORE_PASSWORD_SECRET_NAME = "spark-submission-server-keystore-password" + private val SSL_KEY_PASSWORD_SECRET_NAME = "spark-submission-server-key-password" private val DRIVER_LAUNCHER_SELECTOR_LABEL = "driver-launcher-selector" private val DRIVER_LAUNCHER_SERVICE_INTERNAL_PORT = 7077 private val DEFAULT_DRIVER_PORT = 7078 diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/KubernetesSparkRestServer.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/KubernetesSparkRestServer.scala index 38fa4d1d3f0b2..451dc96dd65ed 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/KubernetesSparkRestServer.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/KubernetesSparkRestServer.scala @@ -21,20 +21,26 @@ import java.net.URI import java.util.concurrent.CountDownLatch import javax.servlet.http.{HttpServletRequest, HttpServletResponse} +import com.google.common.base.Charsets import com.google.common.io.Files import org.apache.commons.codec.binary.Base64 import scala.collection.mutable import scala.collection.mutable.ArrayBuffer -import org.apache.spark.{SecurityManager, SPARK_VERSION, SparkConf} +import org.apache.spark.{SecurityManager, SPARK_VERSION => sparkVersion, SparkConf, SparkException, SSLOptions} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.deploy.rest._ import org.apache.spark.util.{ShutdownHookManager, ThreadUtils, Utils} private case class KubernetesSparkRestServerArguments( - val host: Option[String] = None, - val port: Option[Int] = None, - val secretFile: Option[String] = None) { + host: Option[String] = None, + port: Option[Int] = None, + useSsl: Boolean = false, + secretFile: Option[String] = None, + keyStoreFile: Option[String] = None, + keyStorePasswordFile: Option[String] = None, + keyStoreType: Option[String] = None, + keyPasswordFile: Option[String] = None) { def validate(): KubernetesSparkRestServerArguments = { require(host.isDefined, "Hostname not set via --hostname.") require(port.isDefined, "Port not set via --port") @@ -58,6 +64,21 @@ private object KubernetesSparkRestServerArguments { case "--secret-file" :: value :: tail => args = tail resolvedArguments.copy(secretFile = Some(value)) + case "--use-ssl" :: value :: tail => + args = tail + resolvedArguments.copy(useSsl = value.toBoolean) + case "--keystore-file" :: value :: tail => + args = tail + resolvedArguments.copy(keyStoreFile = Some(value)) + case "--keystore-password-file" :: value :: tail => + args = tail + resolvedArguments.copy(keyStorePasswordFile = Some(value)) + case "--keystore-type" :: value :: tail => + args = tail + resolvedArguments.copy(keyStoreType = Some(value)) + case "--keystore-key-password-file" :: value :: tail => + args = tail + resolvedArguments.copy(keyPasswordFile = Some(value)) // TODO polish usage message case Nil => resolvedArguments case unknown => throw new IllegalStateException(s"Unknown argument(s) found: $unknown") @@ -78,8 +99,9 @@ private[spark] class KubernetesSparkRestServer( port: Int, conf: SparkConf, expectedApplicationSecret: Array[Byte], - shutdownLock: CountDownLatch) - extends RestSubmissionServer(host, port, conf) { + shutdownLock: CountDownLatch, + sslOptions: SSLOptions = new SSLOptions) + extends RestSubmissionServer(host, port, conf, sslOptions) { private val SERVLET_LOCK = new Object private val javaExecutable = s"${System.getenv("JAVA_HOME")}/bin/java" @@ -196,7 +218,7 @@ private[spark] class KubernetesSparkRestServer( response.success = true response.submissionId = null response.message = "success" - response.serverSparkVersion = SPARK_VERSION + response.serverSparkVersion = sparkVersion response } case unexpected => @@ -249,6 +271,7 @@ private[spark] class KubernetesSparkRestServer( private[spark] object KubernetesSparkRestServer { private val barrier = new CountDownLatch(1) + def main(args: Array[String]): Unit = { val parsedArguments = KubernetesSparkRestServerArguments.fromArgsArray(args) val secretFile = new File(parsedArguments.secretFile.get) @@ -256,6 +279,24 @@ private[spark] object KubernetesSparkRestServer { throw new IllegalArgumentException(s"Secret file specified by --secret-file" + " is not a file, or does not exist.") } + val sslOptions = if (parsedArguments.useSsl) { + val keyStorePassword = parsedArguments + .keyStorePasswordFile + .map(new File(_)) + .map(Files.toString(_, Charsets.UTF_8)) + val keyPassword = parsedArguments + .keyPasswordFile + .map(new File(_)) + .map(Files.toString(_, Charsets.UTF_8)) + new SSLOptions( + enabled = true, + keyStore = parsedArguments.keyStoreFile.map(new File(_)), + keyStoreType = parsedArguments.keyStoreType, + keyStorePassword = keyStorePassword, + keyPassword = keyPassword) + } else { + new SSLOptions + } val secretBytes = Files.toByteArray(secretFile) val sparkConf = new SparkConf(true) val server = new KubernetesSparkRestServer( @@ -263,7 +304,8 @@ private[spark] object KubernetesSparkRestServer { parsedArguments.port.get, sparkConf, secretBytes, - barrier) + barrier, + sslOptions) server.start() ShutdownHookManager.addShutdownHook(() => { try { diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala index f512c50a9a934..dae4b2714b4e4 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala @@ -106,13 +106,10 @@ private[spark] class KubernetesClusterSchedulerBackend( protected var totalExpectedExecutors = new AtomicInteger(0) private val driverUrl = RpcEndpointAddress( - System.getenv(s"${convertToEnvMode(kubernetesDriverServiceName)}_SERVICE_HOST"), + sc.getConf.get("spark.driver.host"), sc.getConf.getInt("spark.driver.port", DEFAULT_DRIVER_PORT), CoarseGrainedSchedulerBackend.ENDPOINT_NAME).toString - private def convertToEnvMode(value: String): String = - value.toUpperCase.map { c => if (c == '-') '_' else c } - private val initialExecutors = getInitialTargetExecutorNumber(1) private def getInitialTargetExecutorNumber(defaultNumExecutors: Int = 1): Int = { diff --git a/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/driver/Dockerfile b/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/driver/Dockerfile index 4d345158f356a..070008fce7410 100644 --- a/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/driver/Dockerfile +++ b/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/driver/Dockerfile @@ -19,5 +19,14 @@ ENV SPARK_HOME /opt/spark WORKDIR /opt/spark -# This class will also require setting a secret via the SPARK_APP_SECRET environment variable -CMD exec bin/spark-class org.apache.spark.deploy.rest.kubernetes.KubernetesSparkRestServer --hostname $HOSTNAME --port $SPARK_DRIVER_LAUNCHER_SERVER_PORT --secret-file $SPARK_SUBMISSION_SECRET_LOCATION +CMD SSL_ARGS="" && \ + if ! [ -z ${SPARK_SUBMISSION_USE_SSL+x} ]; then SSL_ARGS="$SSL_ARGS --use-ssl $SPARK_SUBMISSION_USE_SSL"; fi && \ + if ! [ -z ${SPARK_SUBMISSION_KEYSTORE_FILE+x} ]; then SSL_ARGS="$SSL_ARGS --keystore-file $SPARK_SUBMISSION_KEYSTORE_FILE"; fi && \ + if ! [ -z ${SPARK_SUBMISSION_KEYSTORE_TYPE+x} ]; then SSL_ARGS="$SSL_ARGS --keystore-type $SPARK_SUBMISSION_KEYSTORE_TYPE"; fi && \ + if ! [ -z ${SPARK_SUBMISSION_KEYSTORE_PASSWORD_FILE+x} ]; then SSL_ARGS="$SSL_ARGS --keystore-password-file $SPARK_SUBMISSION_KEYSTORE_PASSWORD_FILE"; fi && \ + if ! [ -z ${SPARK_SUBMISSION_KEYSTORE_KEY_PASSWORD_FILE+x} ]; then SSL_ARGS="$SSL_ARGS --keystore-key-password-file $SPARK_SUBMISSION_KEYSTORE_KEY_PASSWORD_FILE"; fi && \ + exec bin/spark-class org.apache.spark.deploy.rest.kubernetes.KubernetesSparkRestServer \ + --hostname $HOSTNAME \ + --port $SPARK_DRIVER_LAUNCHER_SERVER_PORT \ + --secret-file $SPARK_SUBMISSION_SECRET_LOCATION \ + ${SSL_ARGS} diff --git a/resource-managers/kubernetes/integration-tests/pom.xml b/resource-managers/kubernetes/integration-tests/pom.xml index 569527de8e300..f6a322f18cd75 100644 --- a/resource-managers/kubernetes/integration-tests/pom.xml +++ b/resource-managers/kubernetes/integration-tests/pom.xml @@ -106,6 +106,10 @@ + + org.bouncycastle + bcpkix-jdk15on +
    diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KubernetesSuite.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KubernetesSuite.scala index c4bb389f5ada2..13edea02dce9a 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KubernetesSuite.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KubernetesSuite.scala @@ -16,6 +16,7 @@ */ package org.apache.spark.deploy.kubernetes.integrationtest +import java.io.File import java.nio.file.Paths import java.util.UUID import java.util.concurrent.TimeUnit @@ -36,7 +37,7 @@ import org.apache.spark.deploy.kubernetes.Client import org.apache.spark.deploy.kubernetes.integrationtest.docker.SparkDockerImageBuilder import org.apache.spark.deploy.kubernetes.integrationtest.minikube.Minikube import org.apache.spark.deploy.kubernetes.integrationtest.restapis.SparkRestApiV1 -import org.apache.spark.internal.Logging +import org.apache.spark.deploy.kubernetes.integrationtest.sslutil.SSLUtils import org.apache.spark.status.api.v1.{ApplicationStatus, StageStatus} import org.apache.spark.util.Utils @@ -68,6 +69,8 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { private val NAMESPACE = UUID.randomUUID().toString.replaceAll("-", "") private var minikubeKubernetesClient: KubernetesClient = _ private var clientConfig: Config = _ + private var keyStoreFile: File = _ + private var trustStoreFile: File = _ override def beforeAll(): Unit = { Minikube.startMinikube() @@ -79,6 +82,13 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { .done() minikubeKubernetesClient = Minikube.getKubernetesClient.inNamespace(NAMESPACE) clientConfig = minikubeKubernetesClient.getConfiguration + val (keyStore, trustStore) = SSLUtils.generateKeyStoreTrustStorePair( + Minikube.getMinikubeIp, + "changeit", + "changeit", + "changeit") + keyStoreFile = keyStore + trustStoreFile = trustStore } before { @@ -296,4 +306,32 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { assert(driverPodLabels.get("label1") == "label1value", "Unexpected value for label1") assert(driverPodLabels.get("label2") == "label2value", "Unexpected value for label2") } + + test("Enable SSL on the driver submit server") { + val args = Array( + "--master", s"k8s://https://${Minikube.getMinikubeIp}:8443", + "--deploy-mode", "cluster", + "--kubernetes-namespace", NAMESPACE, + "--name", "spark-pi", + "--executor-memory", "512m", + "--executor-cores", "1", + "--num-executors", "1", + "--upload-jars", HELPER_JAR, + "--class", MAIN_CLASS, + "--conf", s"spark.kubernetes.submit.caCertFile=${clientConfig.getCaCertFile}", + "--conf", s"spark.kubernetes.submit.clientKeyFile=${clientConfig.getClientKeyFile}", + "--conf", s"spark.kubernetes.submit.clientCertFile=${clientConfig.getClientCertFile}", + "--conf", "spark.kubernetes.executor.docker.image=spark-executor:latest", + "--conf", "spark.kubernetes.driver.docker.image=spark-driver:latest", + "--conf", "spark.ssl.kubernetes.driverlaunch.enabled=true", + "--conf", "spark.ssl.kubernetes.driverlaunch.keyStore=" + + s"file://${keyStoreFile.getAbsolutePath}", + "--conf", "spark.ssl.kubernetes.driverlaunch.keyStorePassword=changeit", + "--conf", "spark.ssl.kubernetes.driverlaunch.keyPassword=changeit", + "--conf", "spark.ssl.kubernetes.driverlaunch.trustStore=" + + s"file://${trustStoreFile.getAbsolutePath}", + "--conf", s"spark.ssl.kubernetes.driverlaunch.trustStorePassword=changeit", + EXAMPLES_JAR) + SparkSubmit.main(args) + } } diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/sslutil/SSLUtils.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/sslutil/SSLUtils.scala new file mode 100644 index 0000000000000..bde7b43226660 --- /dev/null +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/sslutil/SSLUtils.scala @@ -0,0 +1,80 @@ +/* + * 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.kubernetes.integrationtest.sslutil + +import java.io.{File, FileOutputStream} +import java.math.BigInteger +import java.nio.file.Files +import java.security.{KeyPairGenerator, KeyStore, SecureRandom} +import java.util.{Calendar, Random} +import javax.security.auth.x500.X500Principal + +import org.bouncycastle.asn1.x509.{Extension, GeneralName, GeneralNames} +import org.bouncycastle.cert.jcajce.{JcaX509CertificateConverter, JcaX509v3CertificateBuilder} +import org.bouncycastle.operator.jcajce.JcaContentSignerBuilder + +import org.apache.spark.util.Utils + +private[spark] object SSLUtils { + + def generateKeyStoreTrustStorePair( + ipAddress: String, + keyStorePassword: String, + keyPassword: String, + trustStorePassword: String): (File, File) = { + val keyPairGenerator = KeyPairGenerator.getInstance("RSA") + keyPairGenerator.initialize(512) + val keyPair = keyPairGenerator.generateKeyPair() + val selfPrincipal = new X500Principal(s"cn=$ipAddress") + val currentDate = Calendar.getInstance + val validForOneHundredYears = Calendar.getInstance + validForOneHundredYears.add(Calendar.YEAR, 100) + val certificateBuilder = new JcaX509v3CertificateBuilder( + selfPrincipal, + new BigInteger(4096, new Random()), + currentDate.getTime, + validForOneHundredYears.getTime, + selfPrincipal, + keyPair.getPublic) + certificateBuilder.addExtension(Extension.subjectAlternativeName, false, + new GeneralNames(new GeneralName(GeneralName.iPAddress, ipAddress))) + val signer = new JcaContentSignerBuilder("SHA1WithRSA") + .setSecureRandom(new SecureRandom()) + .build(keyPair.getPrivate) + val bcCertificate = certificateBuilder.build(signer) + val jcaCertificate = new JcaX509CertificateConverter().getCertificate(bcCertificate) + val keyStore = KeyStore.getInstance("JKS") + keyStore.load(null, null) + keyStore.setKeyEntry("key", keyPair.getPrivate, + keyPassword.toCharArray, Array(jcaCertificate)) + val tempDir = Files.createTempDirectory("temp-ssl-stores").toFile() + tempDir.deleteOnExit() + val keyStoreFile = new File(tempDir, "keyStore.jks") + Utils.tryWithResource(new FileOutputStream(keyStoreFile)) { + keyStore.store(_, keyStorePassword.toCharArray) + } + val trustStore = KeyStore.getInstance("JKS") + trustStore.load(null, null) + trustStore.setCertificateEntry("key", jcaCertificate) + val trustStoreFile = new File(tempDir, "trustStore.jks") + Utils.tryWithResource(new FileOutputStream(trustStoreFile)) { + trustStore.store(_, trustStorePassword.toCharArray) + } + (keyStoreFile, trustStoreFile) + } + +} From b2e687715308a953c29ed9b597568819eabf9f65 Mon Sep 17 00:00:00 2001 From: mccheah Date: Tue, 31 Jan 2017 12:07:01 -0800 Subject: [PATCH 36/81] Extract constants and config into separate file. Launch => Submit. (#65) * Extract constants and config into separate file. Launch => Submit. * Address comments * A small shorthand * Refactor more ThreadUtils * Fix scalastyle, use cached thread pool * Tiny Scala style change --- docs/running-on-kubernetes.md | 16 +- .../spark/deploy/kubernetes/Client.scala | 251 +++++++++--------- .../spark/deploy/kubernetes/config.scala | 177 ++++++++++++ .../spark/deploy/kubernetes/constants.scala | 70 +++++ .../rest/KubernetesRestProtocolMessages.scala | 21 +- .../kubernetes/KubernetesSparkRestApi.scala | 3 +- .../KubernetesClusterSchedulerBackend.scala | 162 +++++------ .../src/main/docker/driver/Dockerfile | 2 +- .../integrationtest/KubernetesSuite.scala | 18 +- 9 files changed, 470 insertions(+), 250 deletions(-) create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/config.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/constants.scala diff --git a/docs/running-on-kubernetes.md b/docs/running-on-kubernetes.md index e25e189aa6d74..e256535fbbc9d 100644 --- a/docs/running-on-kubernetes.md +++ b/docs/running-on-kubernetes.md @@ -140,12 +140,12 @@ Spark supports using SSL to encrypt the traffic in this bootstrapping process. I whenever possible. See the [security page](security.html) and [configuration](configuration.html) sections for more information on -configuring SSL; use the prefix `spark.ssl.kubernetes.driverlaunch` in configuring the SSL-related fields in the context +configuring SSL; use the prefix `spark.ssl.kubernetes.submit` in configuring the SSL-related fields in the context of submitting to Kubernetes. For example, to set the trustStore used when the local machine communicates with the driver -pod in starting the application, set `spark.ssl.kubernetes.driverlaunch.trustStore`. +pod in starting the application, set `spark.ssl.kubernetes.submit.trustStore`. One note about the keyStore is that it can be specified as either a file on the client machine or a file in the -container image's disk. Thus `spark.ssl.kubernetes.driverlaunch.keyStore` can be a URI with a scheme of either `file:` +container image's disk. Thus `spark.ssl.kubernetes.submit.keyStore` can be a URI with a scheme of either `file:` or `container:`. A scheme of `file:` corresponds to the keyStore being located on the client machine; it is mounted onto the driver container as a [secret volume](https://kubernetes.io/docs/user-guide/secrets/). When the URI has the scheme `container:`, the file is assumed to already be on the container's disk at the appropriate path. @@ -235,7 +235,15 @@ from the other deployment modes. See the [configuration page](configuration.html (none) Custom labels that will be added to the driver pod. This should be a comma-separated list of label key-value pairs, - where each label is in the format key=value. + where each label is in the format key=value. Note that Spark also adds its own labels to the driver pod + for bookkeeping purposes. + + + + spark.kubernetes.driverSubmitTimeout + 60s + + Time to wait for the driver pod to start running before aborting its execution. diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/Client.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/Client.scala index 07a45c7577bcd..fed9334dbbab4 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/Client.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/Client.scala @@ -18,13 +18,13 @@ package org.apache.spark.deploy.kubernetes import java.io.{File, FileInputStream} import java.security.{KeyStore, SecureRandom} -import java.util.concurrent.{Executors, TimeoutException, TimeUnit} +import java.util.concurrent.{TimeoutException, TimeUnit} import java.util.concurrent.atomic.AtomicBoolean import javax.net.ssl.{SSLContext, TrustManagerFactory, X509TrustManager} import com.google.common.base.Charsets import com.google.common.io.Files -import com.google.common.util.concurrent.{SettableFuture, ThreadFactoryBuilder} +import com.google.common.util.concurrent.SettableFuture import io.fabric8.kubernetes.api.model._ import io.fabric8.kubernetes.client.{ConfigBuilder, DefaultKubernetesClient, KubernetesClient, KubernetesClientException, Watcher} import io.fabric8.kubernetes.client.Watcher.Action @@ -34,11 +34,13 @@ import scala.collection.mutable import scala.concurrent.{ExecutionContext, Future} import scala.concurrent.duration.DurationInt -import org.apache.spark.{SecurityManager, SPARK_VERSION => sparkVersion, SparkConf, SparkException, SSLOptions} +import org.apache.spark.{SecurityManager, SparkConf, SparkException, SSLOptions} +import org.apache.spark.deploy.kubernetes.config._ +import org.apache.spark.deploy.kubernetes.constants._ import org.apache.spark.deploy.rest.{AppResource, ContainerAppResource, KubernetesCreateSubmissionRequest, RemoteAppResource, TarGzippedData, UploadedAppResource} import org.apache.spark.deploy.rest.kubernetes._ import org.apache.spark.internal.Logging -import org.apache.spark.util.Utils +import org.apache.spark.util.{ThreadUtils, Utils} private[spark] class Client( sparkConf: SparkConf, @@ -47,25 +49,21 @@ private[spark] class Client( appArgs: Array[String]) extends Logging { import Client._ - private val namespace = sparkConf.get("spark.kubernetes.namespace", "default") + private val namespace = sparkConf.get(KUBERNETES_NAMESPACE) private val master = resolveK8sMaster(sparkConf.get("spark.master")) private val launchTime = System.currentTimeMillis private val appName = sparkConf.getOption("spark.app.name") - .orElse(sparkConf.getOption("spark.app.id")) .getOrElse("spark") private val kubernetesAppId = s"$appName-$launchTime".toLowerCase.replaceAll("\\.", "-") - private val secretName = s"spark-submission-server-secret-$kubernetesAppId" - private val secretDirectory = s"$SPARK_SUBMISSION_SECRET_BASE_DIR/$kubernetesAppId" - private val sslSecretsDirectory = s"$SPARK_SUBMISSION_SECRET_BASE_DIR/$kubernetesAppId-ssl" - private val sslSecretsName = s"spark-submission-server-ssl-$kubernetesAppId" - private val driverLauncherSelectorValue = s"driver-launcher-$launchTime" - private val driverDockerImage = sparkConf.get( - "spark.kubernetes.driver.docker.image", s"spark-driver:$sparkVersion") - private val uploadedJars = sparkConf.getOption("spark.kubernetes.driver.uploads.jars") + private val secretName = s"$SUBMISSION_APP_SECRET_PREFIX-$kubernetesAppId" + private val secretDirectory = s"$DRIVER_CONTAINER_SECRETS_BASE_DIR/$kubernetesAppId" + private val sslSecretsDirectory = s"$DRIVER_CONTAINER_SECRETS_BASE_DIR/$kubernetesAppId-ssl" + private val sslSecretsName = s"$SUBMISSION_SSL_SECRETS_PREFIX-$kubernetesAppId" + private val driverDockerImage = sparkConf.get(DRIVER_DOCKER_IMAGE) + private val uploadedJars = sparkConf.get(KUBERNETES_DRIVER_UPLOAD_JARS) private val uiPort = sparkConf.getInt("spark.ui.port", DEFAULT_UI_PORT) - private val driverLaunchTimeoutSecs = sparkConf.getTimeAsSeconds( - "spark.kubernetes.driverLaunchTimeout", s"${DEFAULT_LAUNCH_TIMEOUT_SECONDS}s") + private val driverSubmitTimeoutSecs = sparkConf.get(KUBERNETES_DRIVER_SUBMIT_TIMEOUT) private val secretBase64String = { val secretBytes = new Array[Byte](128) @@ -73,32 +71,27 @@ private[spark] class Client( Base64.encodeBase64String(secretBytes) } - private val serviceAccount = sparkConf.get("spark.kubernetes.submit.serviceAccountName", - "default") - - private val customLabels = sparkConf.get("spark.kubernetes.driver.labels", "") + private val serviceAccount = sparkConf.get(KUBERNETES_SERVICE_ACCOUNT_NAME) + private val customLabels = sparkConf.get(KUBERNETES_DRIVER_LABELS) private implicit val retryableExecutionContext = ExecutionContext .fromExecutorService( - Executors.newSingleThreadExecutor(new ThreadFactoryBuilder() - .setNameFormat("kubernetes-client-retryable-futures-%d") - .setDaemon(true) - .build())) + ThreadUtils.newDaemonSingleThreadExecutor("kubernetes-client-retryable-futures")) def run(): Unit = { - val (driverLaunchSslOptions, isKeyStoreLocalFile) = parseDriverLaunchSslOptions() + val (driverSubmitSslOptions, isKeyStoreLocalFile) = parseDriverSubmitSslOptions() val parsedCustomLabels = parseCustomLabels(customLabels) var k8ConfBuilder = new ConfigBuilder() .withApiVersion("v1") .withMasterUrl(master) .withNamespace(namespace) - sparkConf.getOption("spark.kubernetes.submit.caCertFile").foreach { + sparkConf.get(KUBERNETES_CA_CERT_FILE).foreach { f => k8ConfBuilder = k8ConfBuilder.withCaCertFile(f) } - sparkConf.getOption("spark.kubernetes.submit.clientKeyFile").foreach { + sparkConf.get(KUBERNETES_CLIENT_KEY_FILE).foreach { f => k8ConfBuilder = k8ConfBuilder.withClientKeyFile(f) } - sparkConf.getOption("spark.kubernetes.submit.clientCertFile").foreach { + sparkConf.get(KUBERNETES_CLIENT_CERT_FILE).foreach { f => k8ConfBuilder = k8ConfBuilder.withClientCertFile(f) } @@ -108,15 +101,16 @@ private[spark] class Client( .withNewMetadata() .withName(secretName) .endMetadata() - .withData(Map((SUBMISSION_SERVER_SECRET_NAME, secretBase64String)).asJava) + .withData(Map((SUBMISSION_APP_SECRET_NAME, secretBase64String)).asJava) .withType("Opaque") .done() val (sslEnvs, sslVolumes, sslVolumeMounts, sslSecrets) = configureSsl(kubernetesClient, - driverLaunchSslOptions, + driverSubmitSslOptions, isKeyStoreLocalFile) try { val driverKubernetesSelectors = (Map( - DRIVER_LAUNCHER_SELECTOR_LABEL -> driverLauncherSelectorValue, + SPARK_DRIVER_LABEL -> kubernetesAppId, + SPARK_APP_ID_LABEL -> kubernetesAppId, SPARK_APP_NAME_LABEL -> appName) ++ parsedCustomLabels).asJava val containerPorts = buildContainerPorts() @@ -126,7 +120,7 @@ private[spark] class Client( submitCompletedFuture, submitPending, kubernetesClient, - driverLaunchSslOptions, + driverSubmitSslOptions, Array(submitServerSecret) ++ sslSecrets, driverKubernetesSelectors) Utils.tryWithResource(kubernetesClient @@ -141,7 +135,7 @@ private[spark] class Client( .withNewSpec() .withRestartPolicy("OnFailure") .addNewVolume() - .withName(s"spark-submission-secret-volume") + .withName(SUBMISSION_APP_SECRET_VOLUME_NAME) .withNewSecret() .withSecretName(submitServerSecret.getMetadata.getName) .endSecret() @@ -149,22 +143,22 @@ private[spark] class Client( .addToVolumes(sslVolumes: _*) .withServiceAccount(serviceAccount) .addNewContainer() - .withName(DRIVER_LAUNCHER_CONTAINER_NAME) + .withName(DRIVER_CONTAINER_NAME) .withImage(driverDockerImage) .withImagePullPolicy("IfNotPresent") .addNewVolumeMount() - .withName("spark-submission-secret-volume") + .withName(SUBMISSION_APP_SECRET_VOLUME_NAME) .withMountPath(secretDirectory) .withReadOnly(true) .endVolumeMount() .addToVolumeMounts(sslVolumeMounts: _*) .addNewEnv() - .withName("SPARK_SUBMISSION_SECRET_LOCATION") - .withValue(s"$secretDirectory/$SUBMISSION_SERVER_SECRET_NAME") + .withName(ENV_SUBMISSION_SECRET_LOCATION) + .withValue(s"$secretDirectory/$SUBMISSION_APP_SECRET_NAME") .endEnv() .addNewEnv() - .withName("SPARK_DRIVER_LAUNCHER_SERVER_PORT") - .withValue(DRIVER_LAUNCHER_SERVICE_INTERNAL_PORT.toString) + .withName(ENV_SUBMISSION_SERVER_PORT) + .withValue(SUBMISSION_SERVER_PORT.toString) .endEnv() .addToEnv(sslEnvs: _*) .withPorts(containerPorts.asJava) @@ -173,7 +167,7 @@ private[spark] class Client( .done() var submitSucceeded = false try { - submitCompletedFuture.get(driverLaunchTimeoutSecs, TimeUnit.SECONDS) + submitCompletedFuture.get(driverSubmitTimeoutSecs, TimeUnit.SECONDS) submitSucceeded = true } catch { case e: TimeoutException => @@ -199,8 +193,8 @@ private[spark] class Client( } } - private def parseDriverLaunchSslOptions(): (SSLOptions, Boolean) = { - val maybeKeyStore = sparkConf.getOption("spark.ssl.kubernetes.driverlaunch.keyStore") + private def parseDriverSubmitSslOptions(): (SSLOptions, Boolean) = { + val maybeKeyStore = sparkConf.get(KUBERNETES_DRIVER_SUBMIT_KEYSTORE) val resolvedSparkConf = sparkConf.clone() val (isLocalKeyStore, resolvedKeyStore) = maybeKeyStore.map(keyStore => { val keyStoreURI = Utils.resolveURI(keyStore) @@ -214,30 +208,29 @@ private[spark] class Client( (isProvidedKeyStoreLocal, Option.apply(keyStoreURI.getPath)) }).getOrElse((true, Option.empty[String])) resolvedKeyStore.foreach { - resolvedSparkConf.set("spark.ssl.kubernetes.driverlaunch.keyStore", _) + resolvedSparkConf.set(KUBERNETES_DRIVER_SUBMIT_KEYSTORE, _) } - sparkConf.getOption("spark.ssl.kubernetes.driverlaunch.trustStore").foreach { trustStore => + sparkConf.get(KUBERNETES_DRIVER_SUBMIT_TRUSTSTORE).foreach { trustStore => val trustStoreURI = Utils.resolveURI(trustStore) trustStoreURI.getScheme match { case "file" | null => - resolvedSparkConf.set("spark.ssl.kubernetes.driverlaunch.trustStore", - trustStoreURI.getPath) + resolvedSparkConf.set(KUBERNETES_DRIVER_SUBMIT_TRUSTSTORE, trustStoreURI.getPath) case _ => throw new SparkException(s"Invalid trustStore URI $trustStore; trustStore URI" + " for submit server must have no scheme, or scheme file://") } } val securityManager = new SecurityManager(resolvedSparkConf) - (securityManager.getSSLOptions("kubernetes.driverlaunch"), isLocalKeyStore) + (securityManager.getSSLOptions(KUBERNETES_SUBMIT_SSL_NAMESPACE), isLocalKeyStore) } - private def configureSsl(kubernetesClient: KubernetesClient, driverLaunchSslOptions: SSLOptions, + private def configureSsl(kubernetesClient: KubernetesClient, driverSubmitSslOptions: SSLOptions, isKeyStoreLocalFile: Boolean): (Array[EnvVar], Array[Volume], Array[VolumeMount], Array[Secret]) = { - if (driverLaunchSslOptions.enabled) { + if (driverSubmitSslOptions.enabled) { val sslSecretsMap = mutable.HashMap[String, String]() val sslEnvs = mutable.Buffer[EnvVar]() val secrets = mutable.Buffer[Secret]() - driverLaunchSslOptions.keyStore.foreach(store => { + driverSubmitSslOptions.keyStore.foreach(store => { val resolvedKeyStoreFile = if (isKeyStoreLocalFile) { if (!store.isFile) { throw new SparkException(s"KeyStore specified at $store is not a file or" + @@ -245,40 +238,40 @@ private[spark] class Client( } val keyStoreBytes = Files.toByteArray(store) val keyStoreBase64 = Base64.encodeBase64String(keyStoreBytes) - sslSecretsMap += (SSL_KEYSTORE_SECRET_NAME -> keyStoreBase64) - s"$sslSecretsDirectory/$SSL_KEYSTORE_SECRET_NAME" + sslSecretsMap += (SUBMISSION_SSL_KEYSTORE_SECRET_NAME -> keyStoreBase64) + s"$sslSecretsDirectory/$SUBMISSION_SSL_KEYSTORE_SECRET_NAME" } else { store.getAbsolutePath } sslEnvs += new EnvVarBuilder() - .withName("SPARK_SUBMISSION_KEYSTORE_FILE") + .withName(ENV_SUBMISSION_KEYSTORE_FILE) .withValue(resolvedKeyStoreFile) .build() }) - driverLaunchSslOptions.keyStorePassword.foreach(password => { + driverSubmitSslOptions.keyStorePassword.foreach(password => { val passwordBase64 = Base64.encodeBase64String(password.getBytes(Charsets.UTF_8)) - sslSecretsMap += (SSL_KEYSTORE_PASSWORD_SECRET_NAME -> passwordBase64) + sslSecretsMap += (SUBMISSION_SSL_KEYSTORE_PASSWORD_SECRET_NAME -> passwordBase64) sslEnvs += new EnvVarBuilder() - .withName("SPARK_SUBMISSION_KEYSTORE_PASSWORD_FILE") - .withValue(s"$sslSecretsDirectory/$SSL_KEYSTORE_PASSWORD_SECRET_NAME") + .withName(ENV_SUBMISSION_KEYSTORE_PASSWORD_FILE) + .withValue(s"$sslSecretsDirectory/$SUBMISSION_SSL_KEYSTORE_PASSWORD_SECRET_NAME") .build() }) - driverLaunchSslOptions.keyPassword.foreach(password => { + driverSubmitSslOptions.keyPassword.foreach(password => { val passwordBase64 = Base64.encodeBase64String(password.getBytes(Charsets.UTF_8)) - sslSecretsMap += (SSL_KEY_PASSWORD_SECRET_NAME -> passwordBase64) + sslSecretsMap += (SUBMISSION_SSL_KEY_PASSWORD_SECRET_NAME -> passwordBase64) sslEnvs += new EnvVarBuilder() - .withName("SPARK_SUBMISSION_KEYSTORE_KEY_PASSWORD_FILE") - .withValue(s"$sslSecretsDirectory/$SSL_KEY_PASSWORD_SECRET_NAME") + .withName(ENV_SUBMISSION_KEYSTORE_KEY_PASSWORD_FILE) + .withValue(s"$sslSecretsDirectory/$SUBMISSION_SSL_KEY_PASSWORD_SECRET_NAME") .build() }) - driverLaunchSslOptions.keyStoreType.foreach(storeType => { + driverSubmitSslOptions.keyStoreType.foreach(storeType => { sslEnvs += new EnvVarBuilder() - .withName("SPARK_SUBMISSION_KEYSTORE_TYPE") + .withName(ENV_SUBMISSION_KEYSTORE_TYPE) .withValue(storeType) .build() }) sslEnvs += new EnvVarBuilder() - .withName("SPARK_SUBMISSION_USE_SSL") + .withName(ENV_SUBMISSION_USE_SSL) .withValue("true") .build() val sslSecrets = kubernetesClient.secrets().createNew() @@ -290,13 +283,13 @@ private[spark] class Client( .done() secrets += sslSecrets val sslVolume = new VolumeBuilder() - .withName("spark-submission-server-ssl-secrets") + .withName(SUBMISSION_SSL_SECRETS_VOLUME_NAME) .withNewSecret() .withSecretName(sslSecrets.getMetadata.getName) .endSecret() .build() val sslVolumeMount = new VolumeMountBuilder() - .withName("spark-submission-server-ssl-secrets") + .withName(SUBMISSION_SSL_SECRETS_VOLUME_NAME) .withReadOnly(true) .withMountPath(sslSecretsDirectory) .build() @@ -310,7 +303,7 @@ private[spark] class Client( submitCompletedFuture: SettableFuture[Boolean], submitPending: AtomicBoolean, kubernetesClient: KubernetesClient, - driverLaunchSslOptions: SSLOptions, + driverSubmitSslOptions: SSLOptions, applicationSecrets: Array[Secret], driverKubernetesSelectors: java.util.Map[String, String]) extends Watcher[Pod] { override def eventReceived(action: Action, pod: Pod): Unit = { @@ -322,7 +315,7 @@ private[spark] class Client( .getContainerStatuses .asScala .find(status => - status.getName == DRIVER_LAUNCHER_CONTAINER_NAME && status.getReady) match { + status.getName == DRIVER_CONTAINER_NAME && status.getReady) match { case Some(_) => val ownerRefs = Seq(new OwnerReferenceBuilder() .withName(pod.getMetadata.getName) @@ -337,10 +330,10 @@ private[spark] class Client( kubernetesClient.secrets().createOrReplace(secret) }) - val driverLauncherServicePort = new ServicePortBuilder() - .withName(DRIVER_LAUNCHER_SERVICE_PORT_NAME) - .withPort(DRIVER_LAUNCHER_SERVICE_INTERNAL_PORT) - .withNewTargetPort(DRIVER_LAUNCHER_SERVICE_INTERNAL_PORT) + val driverSubmissionServicePort = new ServicePortBuilder() + .withName(SUBMISSION_SERVER_PORT_NAME) + .withPort(SUBMISSION_SERVER_PORT) + .withNewTargetPort(SUBMISSION_SERVER_PORT) .build() val service = kubernetesClient.services().createNew() .withNewMetadata() @@ -351,20 +344,25 @@ private[spark] class Client( .withNewSpec() .withType("NodePort") .withSelector(driverKubernetesSelectors) - .withPorts(driverLauncherServicePort) + .withPorts(driverSubmissionServicePort) .endSpec() .done() try { - sparkConf.set("spark.kubernetes.driver.service.name", - service.getMetadata.getName) - sparkConf.set("spark.kubernetes.driver.pod.name", kubernetesAppId) + sparkConf.getOption("spark.app.id").foreach { id => + logWarning(s"Warning: Provided app id in spark.app.id as $id will be" + + s" overridden as $kubernetesAppId") + } + sparkConf.set(KUBERNETES_DRIVER_POD_NAME, kubernetesAppId) + sparkConf.set(KUBERNETES_DRIVER_SERVICE_NAME, service.getMetadata.getName) + sparkConf.set("spark.app.id", kubernetesAppId) + sparkConf.setIfMissing("spark.app.name", appName) sparkConf.setIfMissing("spark.driver.port", DEFAULT_DRIVER_PORT.toString) sparkConf.setIfMissing("spark.blockmanager.port", DEFAULT_BLOCKMANAGER_PORT.toString) - val driverLauncher = buildDriverLauncherClient(kubernetesClient, service, - driverLaunchSslOptions) + val driverSubmitter = buildDriverSubmissionClient(kubernetesClient, service, + driverSubmitSslOptions) val ping = Retry.retry(5, 5.seconds) { - driverLauncher.ping() + driverSubmitter.ping() } ping onFailure { case t: Throwable => @@ -375,7 +373,7 @@ private[spark] class Client( Future { sparkConf.set("spark.driver.host", pod.getStatus.getPodIP) val submitRequest = buildSubmissionRequest() - driverLauncher.create(submitRequest) + driverSubmitter.submitApplication(submitRequest) } } submitComplete onFailure { @@ -436,17 +434,17 @@ private[spark] class Client( kubernetesClient.pods().withName(kubernetesAppId).get() } catch { case throwable: Throwable => - logError(s"Timed out while waiting $driverLaunchTimeoutSecs seconds for the" + + logError(s"Timed out while waiting $driverSubmitTimeoutSecs seconds for the" + " driver pod to start, but an error occurred while fetching the driver" + " pod's details.", throwable) - throw new SparkException(s"Timed out while waiting $driverLaunchTimeoutSecs" + + throw new SparkException(s"Timed out while waiting $driverSubmitTimeoutSecs" + " seconds for the driver pod to start. Unfortunately, in attempting to fetch" + " the latest state of the pod, another error was thrown. Check the logs for" + " the error that was thrown in looking up the driver pod.", e) } val topLevelMessage = s"The driver pod with name ${driverPod.getMetadata.getName}" + s" in namespace ${driverPod.getMetadata.getNamespace} was not ready in" + - s" $driverLaunchTimeoutSecs seconds." + s" $driverSubmitTimeoutSecs seconds." val podStatusPhase = if (driverPod.getStatus.getPhase != null) { s"Latest phase from the pod is: ${driverPod.getStatus.getPhase}" } else { @@ -460,7 +458,7 @@ private[spark] class Client( val failedDriverContainerStatusString = driverPod.getStatus .getContainerStatuses .asScala - .find(_.getName == DRIVER_LAUNCHER_CONTAINER_NAME) + .find(_.getName == DRIVER_CONTAINER_NAME) .map(status => { val lastState = status.getState if (lastState.getRunning != null) { @@ -481,17 +479,21 @@ private[spark] class Client( "Driver container last state: Unknown" } }).getOrElse("The driver container wasn't found in the pod; expected to find" + - s" container with name $DRIVER_LAUNCHER_CONTAINER_NAME") + s" container with name $DRIVER_CONTAINER_NAME") s"$topLevelMessage\n" + s"$podStatusPhase\n" + s"$podStatusMessage\n\n$failedDriverContainerStatusString" } private def buildContainerPorts(): Seq[ContainerPort] = { - Seq(sparkConf.getInt("spark.driver.port", DEFAULT_DRIVER_PORT), - sparkConf.getInt("spark.blockManager.port", DEFAULT_BLOCKMANAGER_PORT), - DRIVER_LAUNCHER_SERVICE_INTERNAL_PORT, - uiPort).map(new ContainerPortBuilder().withContainerPort(_).build()) + Seq((DRIVER_PORT_NAME, sparkConf.getInt("spark.driver.port", DEFAULT_DRIVER_PORT)), + (BLOCK_MANAGER_PORT_NAME, + sparkConf.getInt("spark.blockManager.port", DEFAULT_BLOCKMANAGER_PORT)), + (SUBMISSION_SERVER_PORT_NAME, SUBMISSION_SERVER_PORT), + (UI_PORT_NAME, uiPort)).map(port => new ContainerPortBuilder() + .withName(port._1) + .withContainerPort(port._2) + .build()) } private def buildSubmissionRequest(): KubernetesCreateSubmissionRequest = { @@ -526,22 +528,22 @@ private[spark] class Client( .map(CompressionUtils.createTarGzip(_)) } - private def buildDriverLauncherClient( + private def buildDriverSubmissionClient( kubernetesClient: KubernetesClient, service: Service, - driverLaunchSslOptions: SSLOptions): KubernetesSparkRestApi = { + driverSubmitSslOptions: SSLOptions): KubernetesSparkRestApi = { val servicePort = service .getSpec .getPorts .asScala - .filter(_.getName == DRIVER_LAUNCHER_SERVICE_PORT_NAME) + .filter(_.getName == SUBMISSION_SERVER_PORT_NAME) .head .getNodePort // NodePort is exposed on every node, so just pick one of them. // TODO be resilient to node failures and try all of them val node = kubernetesClient.nodes.list.getItems.asScala.head val nodeAddress = node.getStatus.getAddresses.asScala.head.getAddress - val urlScheme = if (driverLaunchSslOptions.enabled) { + val urlScheme = if (driverSubmitSslOptions.enabled) { "https" } else { logWarning("Submitting application details, application secret, and local" + @@ -550,8 +552,8 @@ private[spark] class Client( "http" } val (trustManager, sslContext): (X509TrustManager, SSLContext) = - if (driverLaunchSslOptions.enabled) { - buildSslConnectionConfiguration(driverLaunchSslOptions) + if (driverSubmitSslOptions.enabled) { + buildSslConnectionConfiguration(driverSubmitSslOptions) } else { (null, SSLContext.getDefault) } @@ -562,18 +564,18 @@ private[spark] class Client( trustContext = trustManager) } - private def buildSslConnectionConfiguration(driverLaunchSslOptions: SSLOptions) = { - driverLaunchSslOptions.trustStore.map(trustStoreFile => { + private def buildSslConnectionConfiguration(driverSubmitSslOptions: SSLOptions) = { + driverSubmitSslOptions.trustStore.map(trustStoreFile => { val trustManagerFactory = TrustManagerFactory.getInstance( TrustManagerFactory.getDefaultAlgorithm) val trustStore = KeyStore.getInstance( - driverLaunchSslOptions.trustStoreType.getOrElse(KeyStore.getDefaultType)) + driverSubmitSslOptions.trustStoreType.getOrElse(KeyStore.getDefaultType)) if (!trustStoreFile.isFile) { throw new SparkException(s"TrustStore file at ${trustStoreFile.getAbsolutePath}" + s" does not exist or is not a file.") } Utils.tryWithResource(new FileInputStream(trustStoreFile)) { trustStoreStream => - driverLaunchSslOptions.trustStorePassword match { + driverSubmitSslOptions.trustStorePassword match { case Some(password) => trustStore.load(trustStoreStream, password.toCharArray) case None => trustStore.load(trustStoreStream, null) @@ -587,44 +589,29 @@ private[spark] class Client( }).getOrElse((null, SSLContext.getDefault)) } - private def parseCustomLabels(labels: String): Map[String, String] = { - labels.split(",").map(_.trim).filterNot(_.isEmpty).map(label => { - label.split("=", 2).toSeq match { - case Seq(k, v) => - require(k != DRIVER_LAUNCHER_SELECTOR_LABEL, "Label with key" + - s" $DRIVER_LAUNCHER_SELECTOR_LABEL cannot be used in" + - " spark.kubernetes.driver.labels, as it is reserved for Spark's" + - " internal configuration.") - (k, v) - case _ => - throw new SparkException("Custom labels set by spark.kubernetes.driver.labels" + - " must be a comma-separated list of key-value pairs, with format =." + - s" Got label: $label. All labels: $labels") - } - }).toMap + private def parseCustomLabels(maybeLabels: Option[String]): Map[String, String] = { + maybeLabels.map(labels => { + labels.split(",").map(_.trim).filterNot(_.isEmpty).map(label => { + label.split("=", 2).toSeq match { + case Seq(k, v) => + require(k != SPARK_APP_ID_LABEL, "Label with key" + + s" $SPARK_APP_ID_LABEL cannot be used in" + + " spark.kubernetes.driver.labels, as it is reserved for Spark's" + + " internal configuration.") + (k, v) + case _ => + throw new SparkException("Custom labels set by spark.kubernetes.driver.labels" + + " must be a comma-separated list of key-value pairs, with format =." + + s" Got label: $label. All labels: $labels") + } + }).toMap + }).getOrElse(Map.empty[String, String]) } } private[spark] object Client extends Logging { - private val SUBMISSION_SERVER_SECRET_NAME = "spark-submission-server-secret" - private val SSL_KEYSTORE_SECRET_NAME = "spark-submission-server-keystore" - private val SSL_KEYSTORE_PASSWORD_SECRET_NAME = "spark-submission-server-keystore-password" - private val SSL_KEY_PASSWORD_SECRET_NAME = "spark-submission-server-key-password" - private val DRIVER_LAUNCHER_SELECTOR_LABEL = "driver-launcher-selector" - private val DRIVER_LAUNCHER_SERVICE_INTERNAL_PORT = 7077 - private val DEFAULT_DRIVER_PORT = 7078 - private val DEFAULT_BLOCKMANAGER_PORT = 7079 - private val DEFAULT_UI_PORT = 4040 - private val UI_PORT_NAME = "spark-ui-port" - private val DRIVER_LAUNCHER_SERVICE_PORT_NAME = "driver-launcher-port" - private val DRIVER_PORT_NAME = "driver-port" - private val BLOCKMANAGER_PORT_NAME = "block-manager-port" - private val DRIVER_LAUNCHER_CONTAINER_NAME = "spark-kubernetes-driver-launcher" - private val SECURE_RANDOM = new SecureRandom() - private val SPARK_SUBMISSION_SECRET_BASE_DIR = "/var/run/secrets/spark-submission" - private val DEFAULT_LAUNCH_TIMEOUT_SECONDS = 60 - private val SPARK_APP_NAME_LABEL = "spark-app-name" + private[spark] val SECURE_RANDOM = new SecureRandom() def main(args: Array[String]): Unit = { require(args.length >= 2, s"Too few arguments. Usage: ${getClass.getName} " + diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/config.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/config.scala new file mode 100644 index 0000000000000..9b145370f87d6 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/config.scala @@ -0,0 +1,177 @@ +/* + * 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.kubernetes + +import java.util.concurrent.TimeUnit + +import org.apache.spark.{SPARK_VERSION => sparkVersion} +import org.apache.spark.internal.config.ConfigBuilder + +package object config { + + private[spark] val KUBERNETES_NAMESPACE = + ConfigBuilder("spark.kubernetes.namespace") + .doc(""" + | The namespace that will be used for running the driver and + | executor pods. When using spark-submit in cluster mode, + | this can also be passed to spark-submit via the + | --kubernetes-namespace command line argument. + """.stripMargin) + .stringConf + .createWithDefault("default") + + private[spark] val DRIVER_DOCKER_IMAGE = + ConfigBuilder("spark.kubernetes.driver.docker.image") + .doc(""" + | Docker image to use for the driver. Specify this using the + | standard Docker tag format. + """.stripMargin) + .stringConf + .createWithDefault(s"spark-driver:$sparkVersion") + + private[spark] val EXECUTOR_DOCKER_IMAGE = + ConfigBuilder("spark.kubernetes.executor.docker.image") + .doc(""" + | Docker image to use for the executors. Specify this using + | the standard Docker tag format. + """.stripMargin) + .stringConf + .createWithDefault(s"spark-executor:$sparkVersion") + + private[spark] val KUBERNETES_CA_CERT_FILE = + ConfigBuilder("spark.kubernetes.submit.caCertFile") + .doc(""" + | CA cert file for connecting to Kubernetes over SSL. This + | file should be located on the submitting machine's disk. + """.stripMargin) + .stringConf + .createOptional + + private[spark] val KUBERNETES_CLIENT_KEY_FILE = + ConfigBuilder("spark.kubernetes.submit.clientKeyFile") + .doc(""" + | Client key file for authenticating against the Kubernetes + | API server. This file should be located on the submitting + | machine's disk. + """.stripMargin) + .stringConf + .createOptional + + private[spark] val KUBERNETES_CLIENT_CERT_FILE = + ConfigBuilder("spark.kubernetes.submit.clientCertFile") + .doc(""" + | Client cert file for authenticating against the + | Kubernetes API server. This file should be located on + | the submitting machine's disk. + """.stripMargin) + .stringConf + .createOptional + + private[spark] val KUBERNETES_SERVICE_ACCOUNT_NAME = + ConfigBuilder("spark.kubernetes.submit.serviceAccountName") + .doc(""" + | 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. + """.stripMargin) + .stringConf + .createWithDefault("default") + + private[spark] val KUBERNETES_DRIVER_UPLOAD_JARS = + ConfigBuilder("spark.kubernetes.driver.uploads.jars") + .doc(""" + | Comma-separated list of jars to sent to the driver and + | all executors when submitting the application in cluster + | mode. + """.stripMargin) + .stringConf + .createOptional + + // Note that while we set a default for this when we start up the + // scheduler, the specific default value is dynamically determined + // based on the executor memory. + private[spark] val KUBERNETES_EXECUTOR_MEMORY_OVERHEAD = + ConfigBuilder("spark.kubernetes.executor.memoryOverhead") + .doc(""" + | The amount of off-heap memory (in megabytes) to be + | allocated per executor. This is memory that accounts for + | things like VM overheads, interned strings, other native + | overheads, etc. This tends to grow with the executor size + | (typically 6-10%). + """.stripMargin) + .stringConf + .createOptional + + private[spark] val KUBERNETES_DRIVER_LABELS = + ConfigBuilder("spark.kubernetes.driver.labels") + .doc(""" + | Custom labels that will be added to the driver pod. + | This should be a comma-separated list of label key-value + | pairs, where each label is in the format key=value. Note + | that Spark also adds its own labels to the driver pod + | for bookkeeping purposes. + """.stripMargin) + .stringConf + .createOptional + + private[spark] val KUBERNETES_DRIVER_SUBMIT_TIMEOUT = + ConfigBuilder("spark.kubernetes.driverSubmitTimeout") + .doc(""" + | Time to wait for the driver process to start running + | before aborting its execution. + """.stripMargin) + .timeConf(TimeUnit.SECONDS) + .createWithDefault(60L) + + private[spark] val KUBERNETES_DRIVER_SUBMIT_KEYSTORE = + ConfigBuilder("spark.ssl.kubernetes.submit.keyStore") + .doc(""" + | KeyStore file for the driver submission server listening + | on SSL. Can be pre-mounted on the driver container + | or uploaded from the submitting client. + """.stripMargin) + .stringConf + .createOptional + + private[spark] val KUBERNETES_DRIVER_SUBMIT_TRUSTSTORE = + ConfigBuilder("spark.ssl.kubernetes.submit.trustStore") + .doc(""" + | TrustStore containing certificates for communicating + | to the driver submission server over SSL. + """.stripMargin) + .stringConf + .createOptional + + private[spark] val KUBERNETES_DRIVER_SERVICE_NAME = + ConfigBuilder("spark.kubernetes.driver.service.name") + .doc(""" + | Kubernetes service that exposes the driver pod + | for external access. + """.stripMargin) + .internal() + .stringConf + .createOptional + + private[spark] val KUBERNETES_DRIVER_POD_NAME = + ConfigBuilder("spark.kubernetes.driver.pod.name") + .doc(""" + | Name of the driver pod. + """.stripMargin) + .internal() + .stringConf + .createOptional +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/constants.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/constants.scala new file mode 100644 index 0000000000000..027cc3c022b4e --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/constants.scala @@ -0,0 +1,70 @@ +/* + * 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.kubernetes + +package object constants { + // Labels + private[spark] val SPARK_DRIVER_LABEL = "spark-driver" + private[spark] val SPARK_APP_ID_LABEL = "spark-app-id" + private[spark] val SPARK_APP_NAME_LABEL = "spark-app-name" + private[spark] val SPARK_EXECUTOR_ID_LABEL = "spark-exec-id" + + // Secrets + private[spark] val DRIVER_CONTAINER_SECRETS_BASE_DIR = "/var/run/secrets/spark-submission" + private[spark] val SUBMISSION_APP_SECRET_NAME = "spark-submission-server-secret" + private[spark] val SUBMISSION_APP_SECRET_PREFIX = "spark-submission-server-secret" + private[spark] val SUBMISSION_APP_SECRET_VOLUME_NAME = "spark-submission-secret-volume" + private[spark] val SUBMISSION_SSL_KEY_PASSWORD_SECRET_NAME = + "spark-submission-server-key-password" + private[spark] val SUBMISSION_SSL_KEYSTORE_PASSWORD_SECRET_NAME = + "spark-submission-server-keystore-password" + private[spark] val SUBMISSION_SSL_KEYSTORE_SECRET_NAME = "spark-submission-server-keystore" + private[spark] val SUBMISSION_SSL_SECRETS_PREFIX = "spark-submission-server-ssl" + private[spark] val SUBMISSION_SSL_SECRETS_VOLUME_NAME = "spark-submission-server-ssl-secrets" + + // Default and fixed ports + private[spark] val SUBMISSION_SERVER_PORT = 7077 + private[spark] val DEFAULT_DRIVER_PORT = 7078 + private[spark] val DEFAULT_BLOCKMANAGER_PORT = 7079 + private[spark] val DEFAULT_UI_PORT = 4040 + private[spark] val UI_PORT_NAME = "spark-ui-port" + private[spark] val SUBMISSION_SERVER_PORT_NAME = "submit-server" + private[spark] val BLOCK_MANAGER_PORT_NAME = "blockmanager" + private[spark] val DRIVER_PORT_NAME = "driver" + private[spark] val EXECUTOR_PORT_NAME = "executor" + + // Environment Variables + private[spark] val ENV_SUBMISSION_SECRET_LOCATION = "SPARK_SUBMISSION_SECRET_LOCATION" + private[spark] val ENV_SUBMISSION_SERVER_PORT = "SPARK_SUBMISSION_SERVER_PORT" + private[spark] val ENV_SUBMISSION_KEYSTORE_FILE = "SPARK_SUBMISSION_KEYSTORE_FILE" + private[spark] val ENV_SUBMISSION_KEYSTORE_PASSWORD_FILE = + "SPARK_SUBMISSION_KEYSTORE_PASSWORD_FILE" + private[spark] val ENV_SUBMISSION_KEYSTORE_KEY_PASSWORD_FILE = + "SPARK_SUBMISSION_KEYSTORE_KEY_PASSWORD_FILE" + private[spark] val ENV_SUBMISSION_KEYSTORE_TYPE = "SPARK_SUBMISSION_KEYSTORE_TYPE" + private[spark] val ENV_SUBMISSION_USE_SSL = "SPARK_SUBMISSION_USE_SSL" + private[spark] val ENV_EXECUTOR_PORT = "SPARK_EXECUTOR_PORT" + private[spark] val ENV_DRIVER_URL = "SPARK_DRIVER_URL" + private[spark] val ENV_EXECUTOR_CORES = "SPARK_EXECUTOR_CORES" + private[spark] val ENV_EXECUTOR_MEMORY = "SPARK_EXECUTOR_MEMORY" + private[spark] val ENV_APPLICATION_ID = "SPARK_APPLICATION_ID" + private[spark] val ENV_EXECUTOR_ID = "SPARK_EXECUTOR_ID" + + // Miscellaneous + private[spark] val DRIVER_CONTAINER_NAME = "spark-kubernetes-driver" + private[spark] val KUBERNETES_SUBMIT_SSL_NAMESPACE = "kubernetes.submit" +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/KubernetesRestProtocolMessages.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/KubernetesRestProtocolMessages.scala index 813d070e0f876..8beba23bc8e11 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/KubernetesRestProtocolMessages.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/KubernetesRestProtocolMessages.scala @@ -20,23 +20,22 @@ import com.fasterxml.jackson.annotation.{JsonSubTypes, JsonTypeInfo} import org.apache.spark.SPARK_VERSION -// TODO: jars should probably be compressed. Shipping tarballs would be optimal. case class KubernetesCreateSubmissionRequest( - val appResource: AppResource, - val mainClass: String, - val appArgs: Array[String], - val sparkProperties: Map[String, String], - val secret: String, - val uploadedJarsBase64Contents: Option[TarGzippedData]) extends SubmitRestProtocolRequest { + appResource: AppResource, + mainClass: String, + appArgs: Array[String], + sparkProperties: Map[String, String], + secret: String, + uploadedJarsBase64Contents: Option[TarGzippedData]) extends SubmitRestProtocolRequest { message = "create" clientSparkVersion = SPARK_VERSION } case class TarGzippedData( - val dataBase64: String, - val blockSize: Int = 10240, - val recordSize: Int = 512, - val encoding: String + dataBase64: String, + blockSize: Int = 10240, + recordSize: Int = 512, + encoding: String ) @JsonTypeInfo( diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/KubernetesSparkRestApi.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/KubernetesSparkRestApi.scala index 3cbcb16293b1d..18eb9b7a12ca6 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/KubernetesSparkRestApi.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/KubernetesSparkRestApi.scala @@ -28,12 +28,11 @@ trait KubernetesSparkRestApi { @Consumes(Array(MediaType.APPLICATION_JSON)) @Produces(Array(MediaType.APPLICATION_JSON)) @Path("/create") - def create(request: KubernetesCreateSubmissionRequest): CreateSubmissionResponse + def submitApplication(request: KubernetesCreateSubmissionRequest): CreateSubmissionResponse @GET @Consumes(Array(MediaType.APPLICATION_JSON)) @Produces(Array(MediaType.APPLICATION_JSON)) @Path("/ping") def ping(): PingResponse - } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala index dae4b2714b4e4..550ddd113fa42 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala @@ -21,17 +21,18 @@ import java.util.concurrent.Executors import java.util.concurrent.atomic.{AtomicInteger, AtomicLong} import com.google.common.util.concurrent.ThreadFactoryBuilder -import io.fabric8.kubernetes.api.model.{ContainerPort, ContainerPortBuilder, EnvVar, EnvVarBuilder, Pod, QuantityBuilder} +import io.fabric8.kubernetes.api.model.{ContainerPortBuilder, EnvVarBuilder, Pod, QuantityBuilder} import scala.collection.JavaConverters._ -import scala.collection.mutable.ArrayBuffer import scala.concurrent.{ExecutionContext, Future} import org.apache.spark.{SparkContext, SparkException} import org.apache.spark.deploy.kubernetes.{Client, KubernetesClientBuilder} +import org.apache.spark.deploy.kubernetes.config._ +import org.apache.spark.deploy.kubernetes.constants._ import org.apache.spark.rpc.RpcEndpointAddress import org.apache.spark.scheduler.TaskSchedulerImpl import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend -import org.apache.spark.util.Utils +import org.apache.spark.util.{ThreadUtils, Utils} private[spark] class KubernetesClusterSchedulerBackend( scheduler: TaskSchedulerImpl, @@ -44,24 +45,19 @@ private[spark] class KubernetesClusterSchedulerBackend( private val runningExecutorPods = new scala.collection.mutable.HashMap[String, Pod] private val kubernetesMaster = Client.resolveK8sMaster(sc.master) - - private val executorDockerImage = conf - .get("spark.kubernetes.executor.docker.image", s"spark-executor:${sc.version}") - - private val kubernetesNamespace = conf.get("spark.kubernetes.namespace", "default") - + private val executorDockerImage = conf.get(EXECUTOR_DOCKER_IMAGE) + private val kubernetesNamespace = conf.get(KUBERNETES_NAMESPACE) private val executorPort = conf.getInt("spark.executor.port", DEFAULT_STATIC_PORT) - private val blockmanagerPort = conf .getInt("spark.blockmanager.port", DEFAULT_BLOCKMANAGER_PORT) private val kubernetesDriverServiceName = conf - .getOption("spark.kubernetes.driver.service.name") + .get(KUBERNETES_DRIVER_SERVICE_NAME) .getOrElse( throw new SparkException("Must specify the service name the driver is running with")) private val kubernetesDriverPodName = conf - .getOption("spark.kubernetes.driver.pod.name") + .get(KUBERNETES_DRIVER_POD_NAME) .getOrElse( throw new SparkException("Must specify the driver pod name")) @@ -69,7 +65,7 @@ private[spark] class KubernetesClusterSchedulerBackend( private val executorMemoryBytes = Utils.byteStringAsBytes(executorMemory) private val memoryOverheadBytes = conf - .getOption("spark.kubernetes.executor.memoryOverhead") + .get(KUBERNETES_EXECUTOR_MEMORY_OVERHEAD) .map(overhead => Utils.byteStringAsBytes(overhead)) .getOrElse(math.max((MEMORY_OVERHEAD_FACTOR * executorMemoryBytes).toInt, MEMORY_OVERHEAD_MIN)) @@ -78,16 +74,12 @@ private[spark] class KubernetesClusterSchedulerBackend( private val executorCores = conf.getOption("spark.executor.cores").getOrElse("1") private implicit val requestExecutorContext = ExecutionContext.fromExecutorService( - Executors.newCachedThreadPool( - new ThreadFactoryBuilder() - .setDaemon(true) - .setNameFormat("kubernetes-executor-requests-%d") - .build)) + ThreadUtils.newDaemonCachedThreadPool("kubernetes-executor-requests")) private val kubernetesClient = KubernetesClientBuilder .buildFromWithinPod(kubernetesMaster, kubernetesNamespace) - val driverPod = try { + private val driverPod = try { kubernetesClient.pods().inNamespace(kubernetesNamespace). withName(kubernetesDriverPodName).get() } catch { @@ -127,6 +119,8 @@ private[spark] class KubernetesClusterSchedulerBackend( } } + override def applicationId(): String = conf.get("spark.app.id", super.applicationId()) + override def sufficientResourcesRegistered(): Boolean = { totalRegisteredExecutors.get() >= initialExecutors * minRegisteredRatio } @@ -163,9 +157,9 @@ private[spark] class KubernetesClusterSchedulerBackend( private def allocateNewExecutorPod(): (String, Pod) = { val executorKubernetesId = UUID.randomUUID().toString.replaceAll("-", "") val executorId = EXECUTOR_ID_COUNTER.incrementAndGet().toString - val name = s"$kubernetesDriverServiceName-exec-$executorKubernetesId" - val selectors = Map(SPARK_EXECUTOR_SELECTOR -> executorId, - SPARK_APP_SELECTOR -> applicationId()).asJava + val name = s"${applicationId()}-exec-$executorKubernetesId" + val selectors = Map(SPARK_EXECUTOR_ID_LABEL -> executorId, + SPARK_APP_ID_LABEL -> applicationId()).asJava val executorMemoryQuantity = new QuantityBuilder(false) .withAmount(executorMemoryBytes.toString) .build() @@ -175,69 +169,61 @@ private[spark] class KubernetesClusterSchedulerBackend( val executorCpuQuantity = new QuantityBuilder(false) .withAmount(executorCores) .build() - val requiredEnv = new ArrayBuffer[EnvVar] - requiredEnv += new EnvVarBuilder() - .withName("SPARK_EXECUTOR_PORT") - .withValue(executorPort.toString) - .build() - requiredEnv += new EnvVarBuilder() - .withName("SPARK_DRIVER_URL") - .withValue(driverUrl) - .build() - requiredEnv += new EnvVarBuilder() - .withName("SPARK_EXECUTOR_CORES") - .withValue(executorCores) - .build() - requiredEnv += new EnvVarBuilder() - .withName("SPARK_EXECUTOR_MEMORY") - .withValue(executorMemory) - .build() - requiredEnv += new EnvVarBuilder() - .withName("SPARK_APPLICATION_ID") - .withValue(applicationId()) - .build() - requiredEnv += new EnvVarBuilder() - .withName("SPARK_EXECUTOR_ID") - .withValue(executorId) - .build() - val requiredPorts = new ArrayBuffer[ContainerPort] - requiredPorts += new ContainerPortBuilder() - .withName(EXECUTOR_PORT_NAME) - .withContainerPort(executorPort) - .build() - requiredPorts += new ContainerPortBuilder() - .withName(BLOCK_MANAGER_PORT_NAME) - .withContainerPort(blockmanagerPort) - .build() - (executorKubernetesId, kubernetesClient.pods().createNew() - .withNewMetadata() - .withName(name) - .withLabels(selectors) - .withOwnerReferences() - .addNewOwnerReference() - .withController(true) - .withApiVersion(driverPod.getApiVersion) - .withKind(driverPod.getKind) - .withName(driverPod.getMetadata.getName) - .withUid(driverPod.getMetadata.getUid) - .endOwnerReference() - .endMetadata() - .withNewSpec() - .addNewContainer() - .withName(s"exec-${applicationId()}-container") - .withImage(executorDockerImage) - .withImagePullPolicy("IfNotPresent") - .withNewResources() - .addToRequests("memory", executorMemoryQuantity) - .addToLimits("memory", executorMemoryLimitQuantity) - .addToRequests("cpu", executorCpuQuantity) - .addToLimits("cpu", executorCpuQuantity) - .endResources() - .withEnv(requiredEnv.asJava) - .withPorts(requiredPorts.asJava) - .endContainer() - .endSpec() - .done()) + val requiredEnv = Seq( + (ENV_EXECUTOR_PORT, executorPort.toString), + (ENV_DRIVER_URL, driverUrl), + (ENV_EXECUTOR_CORES, executorCores), + (ENV_EXECUTOR_MEMORY, executorMemory), + (ENV_APPLICATION_ID, applicationId()), + (ENV_EXECUTOR_ID, executorId) + ).map(env => new EnvVarBuilder() + .withName(env._1) + .withValue(env._2) + .build()) + val requiredPorts = Seq( + (EXECUTOR_PORT_NAME, executorPort), + (BLOCK_MANAGER_PORT_NAME, blockmanagerPort)) + .map(port => { + new ContainerPortBuilder() + .withName(port._1) + .withContainerPort(port._2) + .build() + }) + try { + (executorKubernetesId, kubernetesClient.pods().createNew() + .withNewMetadata() + .withName(name) + .withLabels(selectors) + .withOwnerReferences() + .addNewOwnerReference() + .withController(true) + .withApiVersion(driverPod.getApiVersion) + .withKind(driverPod.getKind) + .withName(driverPod.getMetadata.getName) + .withUid(driverPod.getMetadata.getUid) + .endOwnerReference() + .endMetadata() + .withNewSpec() + .addNewContainer() + .withName(s"executor") + .withImage(executorDockerImage) + .withImagePullPolicy("IfNotPresent") + .withNewResources() + .addToRequests("memory", executorMemoryQuantity) + .addToLimits("memory", executorMemoryLimitQuantity) + .addToRequests("cpu", executorCpuQuantity) + .addToLimits("cpu", executorCpuQuantity) + .endResources() + .withEnv(requiredEnv.asJava) + .withPorts(requiredPorts.asJava) + .endContainer() + .endSpec() + .done()) + } catch { + case throwable: Throwable => + logError("Failed to allocate executor pod.", throwable) + throw throwable + } } override def doRequestTotalExecutors(requestedTotal: Int): Future[Boolean] = Future[Boolean] { @@ -269,13 +255,7 @@ private[spark] class KubernetesClusterSchedulerBackend( } private object KubernetesClusterSchedulerBackend { - private val SPARK_EXECUTOR_SELECTOR = "spark-exec" - private val SPARK_APP_SELECTOR = "spark-app" private val DEFAULT_STATIC_PORT = 10000 - private val DEFAULT_BLOCKMANAGER_PORT = 7079 - private val DEFAULT_DRIVER_PORT = 7078 - private val BLOCK_MANAGER_PORT_NAME = "blockmanager" - private val EXECUTOR_PORT_NAME = "executor" private val MEMORY_OVERHEAD_FACTOR = 0.10 private val MEMORY_OVERHEAD_MIN = 384L private val EXECUTOR_ID_COUNTER = new AtomicLong(0L) diff --git a/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/driver/Dockerfile b/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/driver/Dockerfile index 070008fce7410..92fdfb8ac5f41 100644 --- a/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/driver/Dockerfile +++ b/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/driver/Dockerfile @@ -27,6 +27,6 @@ CMD SSL_ARGS="" && \ if ! [ -z ${SPARK_SUBMISSION_KEYSTORE_KEY_PASSWORD_FILE+x} ]; then SSL_ARGS="$SSL_ARGS --keystore-key-password-file $SPARK_SUBMISSION_KEYSTORE_KEY_PASSWORD_FILE"; fi && \ exec bin/spark-class org.apache.spark.deploy.rest.kubernetes.KubernetesSparkRestServer \ --hostname $HOSTNAME \ - --port $SPARK_DRIVER_LAUNCHER_SERVER_PORT \ + --port $SPARK_SUBMISSION_SERVER_PORT \ --secret-file $SPARK_SUBMISSION_SECRET_LOCATION \ ${SSL_ARGS} diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KubernetesSuite.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KubernetesSuite.scala index 13edea02dce9a..16de71118dec4 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KubernetesSuite.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KubernetesSuite.scala @@ -172,7 +172,7 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { .set("spark.executor.memory", "500m") .set("spark.executor.cores", "1") .set("spark.executors.instances", "1") - .set("spark.app.id", "spark-pi") + .set("spark.app.name", "spark-pi") .set("spark.ui.enabled", "true") .set("spark.testing", "false") val mainAppResource = s"file://$EXAMPLES_JAR" @@ -298,11 +298,11 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { .getLabels // We can't match all of the selectors directly since one of the selectors is based on the // launch time. - assert(driverPodLabels.size == 4, "Unexpected number of pod labels.") - assert(driverPodLabels.containsKey("driver-launcher-selector"), "Expected driver launcher" + - " selector label to be present.") + assert(driverPodLabels.size == 5, "Unexpected number of pod labels.") assert(driverPodLabels.get("spark-app-name") == "spark-pi", "Unexpected value for" + " spark-app-name label.") + assert(driverPodLabels.get("spark-app-id").startsWith("spark-pi"), "Unexpected value for" + + " spark-app-id label (should be prefixed with the app name).") assert(driverPodLabels.get("label1") == "label1value", "Unexpected value for label1") assert(driverPodLabels.get("label2") == "label2value", "Unexpected value for label2") } @@ -323,12 +323,12 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { "--conf", s"spark.kubernetes.submit.clientCertFile=${clientConfig.getClientCertFile}", "--conf", "spark.kubernetes.executor.docker.image=spark-executor:latest", "--conf", "spark.kubernetes.driver.docker.image=spark-driver:latest", - "--conf", "spark.ssl.kubernetes.driverlaunch.enabled=true", - "--conf", "spark.ssl.kubernetes.driverlaunch.keyStore=" + + "--conf", "spark.ssl.kubernetes.submit.enabled=true", + "--conf", "spark.ssl.kubernetes.submit.keyStore=" + s"file://${keyStoreFile.getAbsolutePath}", - "--conf", "spark.ssl.kubernetes.driverlaunch.keyStorePassword=changeit", - "--conf", "spark.ssl.kubernetes.driverlaunch.keyPassword=changeit", - "--conf", "spark.ssl.kubernetes.driverlaunch.trustStore=" + + "--conf", "spark.ssl.kubernetes.submit.keyStorePassword=changeit", + "--conf", "spark.ssl.kubernetes.submit.keyPassword=changeit", + "--conf", "spark.ssl.kubernetes.submit.trustStore=" + s"file://${trustStoreFile.getAbsolutePath}", "--conf", s"spark.ssl.kubernetes.driverlaunch.trustStorePassword=changeit", EXAMPLES_JAR) From 6ee3be5d0497131374f1ee4edab1f5071414892a Mon Sep 17 00:00:00 2001 From: mccheah Date: Thu, 2 Feb 2017 10:58:15 -0800 Subject: [PATCH 37/81] Retry the submit-application request to multiple nodes (#69) * Retry the submit-application request to multiple nodes. * Fix doc style comment * Check node unschedulable, log retry failures --- .../spark/deploy/kubernetes/Client.scala | 27 ++++---- .../spark/deploy/kubernetes/Retry.scala | 28 +++++--- .../rest/kubernetes/HttpClientUtil.scala | 21 ++++-- .../kubernetes/MultiServerFeignTarget.scala | 67 +++++++++++++++++++ .../integrationtest/minikube/Minikube.scala | 2 +- 5 files changed, 117 insertions(+), 28 deletions(-) create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/MultiServerFeignTarget.scala diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/Client.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/Client.scala index fed9334dbbab4..715df54e573c3 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/Client.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/Client.scala @@ -361,11 +361,13 @@ private[spark] class Client( DEFAULT_BLOCKMANAGER_PORT.toString) val driverSubmitter = buildDriverSubmissionClient(kubernetesClient, service, driverSubmitSslOptions) - val ping = Retry.retry(5, 5.seconds) { + val ping = Retry.retry(5, 5.seconds, + Some("Failed to contact the driver server")) { driverSubmitter.ping() } ping onFailure { case t: Throwable => + logError("Ping failed to the driver server", t) submitCompletedFuture.setException(t) kubernetesClient.services().delete(service) } @@ -532,17 +534,6 @@ private[spark] class Client( kubernetesClient: KubernetesClient, service: Service, driverSubmitSslOptions: SSLOptions): KubernetesSparkRestApi = { - val servicePort = service - .getSpec - .getPorts - .asScala - .filter(_.getName == SUBMISSION_SERVER_PORT_NAME) - .head - .getNodePort - // NodePort is exposed on every node, so just pick one of them. - // TODO be resilient to node failures and try all of them - val node = kubernetesClient.nodes.list.getItems.asScala.head - val nodeAddress = node.getStatus.getAddresses.asScala.head.getAddress val urlScheme = if (driverSubmitSslOptions.enabled) { "https" } else { @@ -551,15 +542,23 @@ private[spark] class Client( " to secure this step.") "http" } + val servicePort = service.getSpec.getPorts.asScala + .filter(_.getName == SUBMISSION_SERVER_PORT_NAME) + .head.getNodePort + val nodeUrls = kubernetesClient.nodes.list.getItems.asScala + .filterNot(_.getSpec.getUnschedulable) + .flatMap(_.getStatus.getAddresses.asScala.map(address => { + s"$urlScheme://${address.getAddress}:$servicePort" + })).toArray + require(nodeUrls.nonEmpty, "No nodes found to contact the driver!") val (trustManager, sslContext): (X509TrustManager, SSLContext) = if (driverSubmitSslOptions.enabled) { buildSslConnectionConfiguration(driverSubmitSslOptions) } else { (null, SSLContext.getDefault) } - val url = s"$urlScheme://$nodeAddress:$servicePort" HttpClientUtil.createClient[KubernetesSparkRestApi]( - url, + uris = nodeUrls, sslSocketFactory = sslContext.getSocketFactory, trustContext = trustManager) } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/Retry.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/Retry.scala index e5ce0bcd606b2..378583b29c547 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/Retry.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/Retry.scala @@ -19,24 +19,36 @@ package org.apache.spark.deploy.kubernetes import scala.concurrent.{ExecutionContext, Future} import scala.concurrent.duration.Duration -private[spark] object Retry { +import org.apache.spark.SparkException +import org.apache.spark.internal.Logging + +private[spark] object Retry extends Logging { private def retryableFuture[T] - (times: Int, interval: Duration) + (attempt: Int, maxAttempts: Int, interval: Duration, retryMessage: Option[String]) (f: => Future[T]) (implicit executionContext: ExecutionContext): Future[T] = { f recoverWith { - case _ if times > 0 => { - Thread.sleep(interval.toMillis) - retryableFuture(times - 1, interval)(f) - } + case error: Throwable => + if (attempt <= maxAttempts) { + retryMessage.foreach { message => + logWarning(s"$message - attempt $attempt of $maxAttempts", error) + } + Thread.sleep(interval.toMillis) + retryableFuture(attempt + 1, maxAttempts, interval, retryMessage)(f) + } else { + Future.failed(retryMessage.map(message => + new SparkException(s"$message - reached $maxAttempts attempts," + + s" and aborting task.", error) + ).getOrElse(error)) + } } } def retry[T] - (times: Int, interval: Duration) + (times: Int, interval: Duration, retryMessage: Option[String] = None) (f: => T) (implicit executionContext: ExecutionContext): Future[T] = { - retryableFuture(times, interval)(Future[T] { f }) + retryableFuture(1, times, interval, retryMessage)(Future[T] { f }) } } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/HttpClientUtil.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/HttpClientUtil.scala index eb7d411700829..1cabfbad656eb 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/HttpClientUtil.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/HttpClientUtil.scala @@ -20,7 +20,7 @@ import javax.net.ssl.{SSLContext, SSLSocketFactory, X509TrustManager} import com.fasterxml.jackson.databind.{DeserializationFeature, ObjectMapper} import com.fasterxml.jackson.module.scala.DefaultScalaModule -import feign.Feign +import feign.{Client, Feign, Request, Response} import feign.Request.Options import feign.jackson.{JacksonDecoder, JacksonEncoder} import feign.jaxrs.JAXRSContract @@ -32,7 +32,7 @@ import org.apache.spark.status.api.v1.JacksonMessageWriter private[spark] object HttpClientUtil { def createClient[T: ClassTag]( - uri: String, + uris: Array[String], sslSocketFactory: SSLSocketFactory = SSLContext.getDefault.getSocketFactory, trustContext: X509TrustManager = null, readTimeoutMillis: Int = 20000, @@ -45,13 +45,24 @@ private[spark] object HttpClientUtil { .registerModule(new DefaultScalaModule) .setDateFormat(JacksonMessageWriter.makeISODateFormat) objectMapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false) - val clazz = implicitly[ClassTag[T]].runtimeClass.asInstanceOf[Class[T]] + val target = new MultiServerFeignTarget[T](uris) + val baseHttpClient = new feign.okhttp.OkHttpClient(httpClientBuilder.build()) + val resetTargetHttpClient = new Client { + override def execute(request: Request, options: Options): Response = { + val response = baseHttpClient.execute(request, options) + if (response.status() >= 200 && response.status() < 300) { + target.reset() + } + response + } + } Feign.builder() - .client(new feign.okhttp.OkHttpClient(httpClientBuilder.build())) + .client(resetTargetHttpClient) .contract(new JAXRSContract) .encoder(new JacksonEncoder(objectMapper)) .decoder(new JacksonDecoder(objectMapper)) .options(new Options(connectTimeoutMillis, readTimeoutMillis)) - .target(clazz, uri) + .retryer(target) + .target(target) } } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/MultiServerFeignTarget.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/MultiServerFeignTarget.scala new file mode 100644 index 0000000000000..fea7f057cfa1b --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/MultiServerFeignTarget.scala @@ -0,0 +1,67 @@ +/* + * 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.rest.kubernetes + +import feign.{Request, RequestTemplate, RetryableException, Retryer, Target} +import scala.reflect.ClassTag +import scala.util.Random + +private[kubernetes] class MultiServerFeignTarget[T : ClassTag]( + private val servers: Seq[String]) extends Target[T] with Retryer { + require(servers.nonEmpty, "Must provide at least one server URI.") + + private val threadLocalShuffledServers = new ThreadLocal[Seq[String]] { + override def initialValue(): Seq[String] = Random.shuffle(servers) + } + + override def `type`(): Class[T] = { + implicitly[ClassTag[T]].runtimeClass.asInstanceOf[Class[T]] + } + + override def url(): String = threadLocalShuffledServers.get.head + + /** + * Cloning the target is done on every request, for use on the current + * thread - thus it's important that clone returns a "fresh" target. + */ + override def clone(): Retryer = { + reset() + this + } + + override def name(): String = { + s"${getClass.getSimpleName} with servers [${servers.mkString(",")}]" + } + + override def apply(requestTemplate: RequestTemplate): Request = { + if (!requestTemplate.url().startsWith("http")) { + requestTemplate.insert(0, url()) + } + requestTemplate.request() + } + + override def continueOrPropagate(e: RetryableException): Unit = { + threadLocalShuffledServers.set(threadLocalShuffledServers.get.drop(1)) + if (threadLocalShuffledServers.get.isEmpty) { + throw e + } + } + + def reset(): Unit = { + threadLocalShuffledServers.set(Random.shuffle(servers)) + } +} diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/minikube/Minikube.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/minikube/Minikube.scala index 60c6564579a6e..b42f97952394e 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/minikube/Minikube.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/minikube/Minikube.scala @@ -123,7 +123,7 @@ private[spark] object Minikube extends Logging { .build() val sslContext = SSLUtils.sslContext(kubernetesConf) val trustManager = SSLUtils.trustManagers(kubernetesConf)(0).asInstanceOf[X509TrustManager] - HttpClientUtil.createClient[T](url, sslContext.getSocketFactory, trustManager) + HttpClientUtil.createClient[T](Array(url), sslContext.getSocketFactory, trustManager) } def executeMinikubeSsh(command: String): Unit = { From d0f95dbec40fb686b42b8371ce532f12d69662c0 Mon Sep 17 00:00:00 2001 From: mccheah Date: Thu, 2 Feb 2017 12:22:54 -0800 Subject: [PATCH 38/81] Allow adding arbitrary files (#71) * Allow adding arbitrary files * Address comments and add documentation --- .../org/apache/spark/deploy/SparkSubmit.scala | 2 + .../spark/deploy/SparkSubmitArguments.scala | 7 ++ docs/running-on-kubernetes.md | 12 ++- .../launcher/SparkSubmitOptionParser.java | 4 +- .../spark/deploy/kubernetes/Client.scala | 34 ++++++-- .../spark/deploy/kubernetes/config.scala | 16 +++- .../rest/KubernetesRestProtocolMessages.scala | 3 +- .../rest/kubernetes/CompressionUtils.scala | 4 +- .../KubernetesSparkRestServer.scala | 53 ++++++++---- .../jobs/FileExistenceTest.scala | 54 ++++++++++++ .../integrationtest/KubernetesSuite.scala | 85 +++++++++++++++++-- .../integration-tests/test-data/input.txt | 1 + 12 files changed, 243 insertions(+), 32 deletions(-) create mode 100644 resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/kubernetes/integrationtest/jobs/FileExistenceTest.scala create mode 100644 resource-managers/kubernetes/integration-tests/test-data/input.txt 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 db6ec22ca919f..51eb23560defe 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -474,6 +474,8 @@ object SparkSubmit { sysProp = "spark.kubernetes.namespace"), OptionAssigner(args.kubernetesUploadJars, KUBERNETES, CLUSTER, sysProp = "spark.kubernetes.driver.uploads.jars"), + OptionAssigner(args.kubernetesUploadFiles, KUBERNETES, CLUSTER, + sysProp = "spark.kubernetes.driver.uploads.files"), // Other options OptionAssigner(args.executorCores, STANDALONE | YARN, ALL_DEPLOY_MODES, diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala index ae1bee7ee4d14..f771755244f31 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -74,6 +74,7 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S // Kubernetes only var kubernetesNamespace: String = null var kubernetesUploadJars: String = null + var kubernetesUploadFiles: String = null // Standalone cluster mode only var supervise: Boolean = false @@ -196,6 +197,9 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S kubernetesUploadJars = Option(kubernetesUploadJars) .orElse(sparkProperties.get("spark.kubernetes.driver.uploads.jars")) .orNull + kubernetesUploadFiles = Option(kubernetesUploadFiles) + .orElse(sparkProperties.get("spark.kubernetes.driver.uploads.files")) + .orNull // Try to set main class from JAR if no --class argument is given if (mainClass == null && !isPython && !isR && primaryResource != null) { @@ -442,6 +446,9 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S case KUBERNETES_UPLOAD_JARS => kubernetesUploadJars = value + case KUBERNETES_UPLOAD_FILES => + kubernetesUploadFiles = value + case HELP => printUsageAndExit(0) diff --git a/docs/running-on-kubernetes.md b/docs/running-on-kubernetes.md index e256535fbbc9d..5a48bb254a6df 100644 --- a/docs/running-on-kubernetes.md +++ b/docs/running-on-kubernetes.md @@ -217,10 +217,20 @@ from the other deployment modes. See the [configuration page](configuration.html spark.kubernetes.driver.uploads.jars (none) - Comma-separated list of jars to sent to the driver and all executors when submitting the application in cluster + Comma-separated list of jars to send to the driver and all executors when submitting the application in cluster mode. Refer to adding other jars for more information. + + spark.kubernetes.driver.uploads.files + (none) + + Comma-separated list of files to send to the driver and all executors when submitting the application in cluster + mode. The files are added in a flat hierarchy to the current working directory of the driver, having the same + names as the names of the original files. Note that two files with the same name cannot be added, even if they + were in different source directories on the client disk. + + spark.kubernetes.executor.memoryOverhead executorMemory * 0.10, with minimum of 384 diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitOptionParser.java b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitOptionParser.java index f1dac20f52f0d..3369b5d8301be 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitOptionParser.java +++ b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitOptionParser.java @@ -80,6 +80,7 @@ class SparkSubmitOptionParser { protected final String KUBERNETES_MASTER = "--kubernetes-master"; protected final String KUBERNETES_NAMESPACE = "--kubernetes-namespace"; protected final String KUBERNETES_UPLOAD_JARS = "--upload-jars"; + protected final String KUBERNETES_UPLOAD_FILES = "--upload-files"; /** * This is the canonical list of spark-submit options. Each entry in the array contains the @@ -122,7 +123,8 @@ class SparkSubmitOptionParser { { TOTAL_EXECUTOR_CORES }, { KUBERNETES_MASTER }, { KUBERNETES_NAMESPACE }, - { KUBERNETES_UPLOAD_JARS } + { KUBERNETES_UPLOAD_JARS }, + { KUBERNETES_UPLOAD_FILES } }; /** diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/Client.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/Client.scala index 715df54e573c3..c350c4817664d 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/Client.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/Client.scala @@ -61,7 +61,9 @@ private[spark] class Client( private val sslSecretsDirectory = s"$DRIVER_CONTAINER_SECRETS_BASE_DIR/$kubernetesAppId-ssl" private val sslSecretsName = s"$SUBMISSION_SSL_SECRETS_PREFIX-$kubernetesAppId" private val driverDockerImage = sparkConf.get(DRIVER_DOCKER_IMAGE) - private val uploadedJars = sparkConf.get(KUBERNETES_DRIVER_UPLOAD_JARS) + private val uploadedJars = sparkConf.get(KUBERNETES_DRIVER_UPLOAD_JARS).filter(_.nonEmpty) + private val uploadedFiles = sparkConf.get(KUBERNETES_DRIVER_UPLOAD_FILES).filter(_.nonEmpty) + uploadedFiles.foreach(validateNoDuplicateUploadFileNames) private val uiPort = sparkConf.getInt("spark.ui.port", DEFAULT_UI_PORT) private val driverSubmitTimeoutSecs = sparkConf.get(KUBERNETES_DRIVER_SUBMIT_TIMEOUT) @@ -513,18 +515,40 @@ private[spark] class Client( case "container" => ContainerAppResource(appResourceUri.getPath) case other => RemoteAppResource(other) } - - val uploadJarsBase64Contents = compressJars(uploadedJars) + val uploadJarsBase64Contents = compressFiles(uploadedJars) + val uploadFilesBase64Contents = compressFiles(uploadedFiles) KubernetesCreateSubmissionRequest( appResource = resolvedAppResource, mainClass = mainClass, appArgs = appArgs, secret = secretBase64String, sparkProperties = sparkConf.getAll.toMap, - uploadedJarsBase64Contents = uploadJarsBase64Contents) + uploadedJarsBase64Contents = uploadJarsBase64Contents, + uploadedFilesBase64Contents = uploadFilesBase64Contents) + } + + // Because uploaded files should be added to the working directory of the driver, they + // need to not have duplicate file names. They are added to the working directory so the + // user can reliably locate them in their application. This is similar in principle to how + // YARN handles its `spark.files` setting. + private def validateNoDuplicateUploadFileNames(uploadedFilesCommaSeparated: String): Unit = { + val pathsWithDuplicateNames = uploadedFilesCommaSeparated + .split(",") + .groupBy(new File(_).getName) + .filter(_._2.length > 1) + if (pathsWithDuplicateNames.nonEmpty) { + val pathsWithDuplicateNamesSorted = pathsWithDuplicateNames + .values + .flatten + .toList + .sortBy(new File(_).getName) + throw new SparkException("Cannot upload files with duplicate names via" + + s" ${KUBERNETES_DRIVER_UPLOAD_FILES.key}. The following paths have a duplicated" + + s" file name: ${pathsWithDuplicateNamesSorted.mkString(",")}") + } } - private def compressJars(maybeFilePaths: Option[String]): Option[TarGzippedData] = { + private def compressFiles(maybeFilePaths: Option[String]): Option[TarGzippedData] = { maybeFilePaths .map(_.split(",")) .map(CompressionUtils.createTarGzip(_)) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/config.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/config.scala index 9b145370f87d6..3e0c400febca1 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/config.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/config.scala @@ -94,13 +94,27 @@ package object config { private[spark] val KUBERNETES_DRIVER_UPLOAD_JARS = ConfigBuilder("spark.kubernetes.driver.uploads.jars") .doc(""" - | Comma-separated list of jars to sent to the driver and + | Comma-separated list of jars to send to the driver and | all executors when submitting the application in cluster | mode. """.stripMargin) .stringConf .createOptional + private[spark] val KUBERNETES_DRIVER_UPLOAD_FILES = + ConfigBuilder("spark.kubernetes.driver.uploads.files") + .doc(""" + | Comma-separated list of files to send to the driver and + | all executors when submitting the application in cluster + | mode. The files are added in a flat hierarchy to the + | current working directory of the driver, having the same + | names as the names of the original files. Note that two + | files with the same name cannot be added, even if they + | were in different source directories on the client disk. + """.stripMargin) + .stringConf + .createOptional + // Note that while we set a default for this when we start up the // scheduler, the specific default value is dynamically determined // based on the executor memory. diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/KubernetesRestProtocolMessages.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/KubernetesRestProtocolMessages.scala index 8beba23bc8e11..6aeb851a16bf4 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/KubernetesRestProtocolMessages.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/KubernetesRestProtocolMessages.scala @@ -26,7 +26,8 @@ case class KubernetesCreateSubmissionRequest( appArgs: Array[String], sparkProperties: Map[String, String], secret: String, - uploadedJarsBase64Contents: Option[TarGzippedData]) extends SubmitRestProtocolRequest { + uploadedJarsBase64Contents: Option[TarGzippedData], + uploadedFilesBase64Contents: Option[TarGzippedData]) extends SubmitRestProtocolRequest { message = "create" clientSparkVersion = SPARK_VERSION } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/CompressionUtils.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/CompressionUtils.scala index 1c95dacc7eb01..7204cb874aaec 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/CompressionUtils.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/CompressionUtils.scala @@ -68,8 +68,8 @@ private[spark] object CompressionUtils extends Logging { while (usedFileNames.contains(resolvedFileName)) { val oldResolvedFileName = resolvedFileName resolvedFileName = s"$nameWithoutExtension-$deduplicationCounter.$extension" - logWarning(s"File with name $oldResolvedFileName already exists. Trying to add with" + - s" file name $resolvedFileName instead.") + logWarning(s"File with name $oldResolvedFileName already exists. Trying to add" + + s" with file name $resolvedFileName instead.") deduplicationCounter += 1 } usedFileNames += resolvedFileName diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/KubernetesSparkRestServer.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/KubernetesSparkRestServer.scala index 451dc96dd65ed..c5a7e27b15927 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/KubernetesSparkRestServer.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/KubernetesSparkRestServer.scala @@ -18,6 +18,7 @@ package org.apache.spark.deploy.rest.kubernetes import java.io.File import java.net.URI +import java.nio.file.Paths import java.util.concurrent.CountDownLatch import javax.servlet.http.{HttpServletRequest, HttpServletResponse} @@ -27,7 +28,7 @@ import org.apache.commons.codec.binary.Base64 import scala.collection.mutable import scala.collection.mutable.ArrayBuffer -import org.apache.spark.{SecurityManager, SPARK_VERSION => sparkVersion, SparkConf, SparkException, SSLOptions} +import org.apache.spark.{SecurityManager, SPARK_VERSION => sparkVersion, SparkConf, SSLOptions} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.deploy.rest._ import org.apache.spark.util.{ShutdownHookManager, ThreadUtils, Utils} @@ -149,7 +150,8 @@ private[spark] class KubernetesSparkRestServer( appArgs, sparkProperties, secret, - uploadedJars) => + uploadedJars, + uploadedFiles) => val decodedSecret = Base64.decodeBase64(secret) if (!expectedApplicationSecret.sameElements(decodedSecret)) { responseServlet.setStatus(HttpServletResponse.SC_UNAUTHORIZED) @@ -157,29 +159,33 @@ private[spark] class KubernetesSparkRestServer( } else { val tempDir = Utils.createTempDir() val appResourcePath = resolvedAppResource(appResource, tempDir) - val jarsDirectory = new File(tempDir, "jars") - if (!jarsDirectory.mkdir) { - throw new IllegalStateException("Failed to create jars dir at" + - s"${jarsDirectory.getAbsolutePath}") - } - val writtenJars = writeBase64ContentsToFiles(uploadedJars, jarsDirectory) - val driverExtraClasspath = sparkProperties - .get("spark.driver.extraClassPath") - .map(_.split(",")) - .getOrElse(Array.empty[String]) + val writtenJars = writeUploadedJars(uploadedJars, tempDir) + val writtenFiles = writeUploadedFiles(uploadedFiles) + val resolvedSparkProperties = new mutable.HashMap[String, String] + resolvedSparkProperties ++= sparkProperties + + // Resolve driver classpath and jars val originalJars = sparkProperties.get("spark.jars") .map(_.split(",")) .getOrElse(Array.empty[String]) val resolvedJars = writtenJars ++ originalJars ++ Array(appResourcePath) val sparkJars = new File(sparkHome, "jars").listFiles().map(_.getAbsolutePath) + val driverExtraClasspath = sparkProperties + .get("spark.driver.extraClassPath") + .map(_.split(",")) + .getOrElse(Array.empty[String]) val driverClasspath = driverExtraClasspath ++ resolvedJars ++ - sparkJars ++ - Array(appResourcePath) - val resolvedSparkProperties = new mutable.HashMap[String, String] - resolvedSparkProperties ++= sparkProperties + sparkJars resolvedSparkProperties("spark.jars") = resolvedJars.mkString(",") + // Resolve spark.files + val originalFiles = sparkProperties.get("spark.files") + .map(_.split(",")) + .getOrElse(Array.empty[String]) + val resolvedFiles = originalFiles ++ writtenFiles + resolvedSparkProperties("spark.files") = resolvedFiles.mkString(",") + val command = new ArrayBuffer[String] command += javaExecutable command += "-cp" @@ -229,6 +235,21 @@ private[spark] class KubernetesSparkRestServer( } } + private def writeUploadedJars(files: Option[TarGzippedData], rootTempDir: File): + Seq[String] = { + val resolvedDirectory = new File(rootTempDir, "jars") + if (!resolvedDirectory.mkdir()) { + throw new IllegalStateException(s"Failed to create jars dir at " + + resolvedDirectory.getAbsolutePath) + } + writeBase64ContentsToFiles(files, resolvedDirectory) + } + + private def writeUploadedFiles(files: Option[TarGzippedData]): Seq[String] = { + val workingDir = Paths.get("").toFile.getAbsoluteFile + writeBase64ContentsToFiles(files, workingDir) + } + def resolvedAppResource(appResource: AppResource, tempDir: File): String = { val appResourcePath = appResource match { case UploadedAppResource(resourceContentsBase64, resourceName) => diff --git a/resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/kubernetes/integrationtest/jobs/FileExistenceTest.scala b/resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/kubernetes/integrationtest/jobs/FileExistenceTest.scala new file mode 100644 index 0000000000000..8b8d5e05f6479 --- /dev/null +++ b/resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/kubernetes/integrationtest/jobs/FileExistenceTest.scala @@ -0,0 +1,54 @@ +/* + * 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.kubernetes.integrationtest.jobs + +import java.nio.file.Paths + +import com.google.common.base.Charsets +import com.google.common.io.Files + +import org.apache.spark.SparkException +import org.apache.spark.sql.SparkSession + +private[spark] object FileExistenceTest { + + def main(args: Array[String]): Unit = { + if (args.length < 2) { + throw new IllegalArgumentException("Usage: WordCount ") + } + // Can't use SparkContext.textFile since the file is local to the driver + val file = Paths.get(args(0)).toFile + if (!file.exists()) { + throw new SparkException(s"Failed to find file at ${file.getAbsolutePath}") + } else { + // scalastyle:off println + val contents = Files.toString(file, Charsets.UTF_8) + if (args(1) != contents) { + throw new SparkException(s"Contents do not match. Expected: ${args(1)}," + + s" actual, $contents") + } else { + println(s"File found at ${file.getAbsolutePath} with correct contents.") + } + // scalastyle:on println + } + val spark = SparkSession.builder() + .appName("Test") + .getOrCreate() + spark.stop() + } + +} diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KubernetesSuite.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KubernetesSuite.scala index 16de71118dec4..40867c40d4474 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KubernetesSuite.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KubernetesSuite.scala @@ -21,7 +21,9 @@ import java.nio.file.Paths import java.util.UUID import java.util.concurrent.TimeUnit +import com.google.common.base.Charsets import com.google.common.collect.ImmutableList +import com.google.common.io.Files import com.google.common.util.concurrent.SettableFuture import io.fabric8.kubernetes.api.model.Pod import io.fabric8.kubernetes.client.{Config, KubernetesClient, KubernetesClientException, Watcher} @@ -62,10 +64,14 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { .getOrElse(throw new IllegalStateException("Expected to find spark-examples jar; was the" + " pre-integration-test phase run?")) + private val TEST_EXISTENCE_FILE = Paths.get("test-data", "input.txt").toFile + private val TEST_EXISTENCE_FILE_CONTENTS = Files.toString(TEST_EXISTENCE_FILE, Charsets.UTF_8) private val TIMEOUT = PatienceConfiguration.Timeout(Span(2, Minutes)) private val INTERVAL = PatienceConfiguration.Interval(Span(2, Seconds)) - private val MAIN_CLASS = "org.apache.spark.deploy.kubernetes" + + private val SPARK_PI_MAIN_CLASS = "org.apache.spark.deploy.kubernetes" + ".integrationtest.jobs.SparkPiWithInfiniteWait" + private val FILE_EXISTENCE_MAIN_CLASS = "org.apache.spark.deploy.kubernetes" + + ".integrationtest.jobs.FileExistenceTest" private val NAMESPACE = UUID.randomUUID().toString.replaceAll("-", "") private var minikubeKubernetesClient: KubernetesClient = _ private var clientConfig: Config = _ @@ -179,7 +185,7 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { new Client( sparkConf = sparkConf, - mainClass = MAIN_CLASS, + mainClass = SPARK_PI_MAIN_CLASS, mainAppResource = mainAppResource, appArgs = Array.empty[String]).run() val sparkMetricsService = getSparkMetricsService("spark-pi") @@ -196,7 +202,7 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { "--executor-cores", "1", "--num-executors", "1", "--upload-jars", HELPER_JAR, - "--class", MAIN_CLASS, + "--class", SPARK_PI_MAIN_CLASS, "--conf", "spark.ui.enabled=true", "--conf", "spark.testing=false", "--conf", s"spark.kubernetes.submit.caCertFile=${clientConfig.getCaCertFile}", @@ -279,7 +285,7 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { "--executor-cores", "1", "--num-executors", "1", "--upload-jars", HELPER_JAR, - "--class", MAIN_CLASS, + "--class", SPARK_PI_MAIN_CLASS, "--conf", s"spark.kubernetes.submit.caCertFile=${clientConfig.getCaCertFile}", "--conf", s"spark.kubernetes.submit.clientKeyFile=${clientConfig.getClientKeyFile}", "--conf", s"spark.kubernetes.submit.clientCertFile=${clientConfig.getClientCertFile}", @@ -317,7 +323,7 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { "--executor-cores", "1", "--num-executors", "1", "--upload-jars", HELPER_JAR, - "--class", MAIN_CLASS, + "--class", SPARK_PI_MAIN_CLASS, "--conf", s"spark.kubernetes.submit.caCertFile=${clientConfig.getCaCertFile}", "--conf", s"spark.kubernetes.submit.clientKeyFile=${clientConfig.getClientKeyFile}", "--conf", s"spark.kubernetes.submit.clientCertFile=${clientConfig.getClientCertFile}", @@ -334,4 +340,73 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { EXAMPLES_JAR) SparkSubmit.main(args) } + + test("Added files should exist on the driver.") { + val args = Array( + "--master", s"k8s://https://${Minikube.getMinikubeIp}:8443", + "--deploy-mode", "cluster", + "--kubernetes-namespace", NAMESPACE, + "--name", "spark-file-existence-test", + "--executor-memory", "512m", + "--executor-cores", "1", + "--num-executors", "1", + "--upload-jars", HELPER_JAR, + "--upload-files", TEST_EXISTENCE_FILE.getAbsolutePath, + "--class", FILE_EXISTENCE_MAIN_CLASS, + "--conf", "spark.ui.enabled=false", + "--conf", "spark.testing=true", + "--conf", s"spark.kubernetes.submit.caCertFile=${clientConfig.getCaCertFile}", + "--conf", s"spark.kubernetes.submit.clientKeyFile=${clientConfig.getClientKeyFile}", + "--conf", s"spark.kubernetes.submit.clientCertFile=${clientConfig.getClientCertFile}", + "--conf", "spark.kubernetes.executor.docker.image=spark-executor:latest", + "--conf", "spark.kubernetes.driver.docker.image=spark-driver:latest", + EXAMPLES_JAR, + TEST_EXISTENCE_FILE.getName, + TEST_EXISTENCE_FILE_CONTENTS) + val podCompletedFuture = SettableFuture.create[Boolean] + val watch = new Watcher[Pod] { + override def eventReceived(action: Action, pod: Pod): Unit = { + val containerStatuses = pod.getStatus.getContainerStatuses.asScala + val allSuccessful = containerStatuses.nonEmpty && containerStatuses + .forall(status => { + status.getState.getTerminated != null && status.getState.getTerminated.getExitCode == 0 + }) + if (allSuccessful) { + podCompletedFuture.set(true) + } else { + val failedContainers = containerStatuses.filter(container => { + container.getState.getTerminated != null && + container.getState.getTerminated.getExitCode != 0 + }) + if (failedContainers.nonEmpty) { + podCompletedFuture.setException(new SparkException( + "One or more containers in the driver failed with a nonzero exit code.")) + } + } + } + + override def onClose(e: KubernetesClientException): Unit = { + logWarning("Watch closed", e) + } + } + Utils.tryWithResource(minikubeKubernetesClient + .pods + .withLabel("spark-app-name", "spark-file-existence-test") + .watch(watch)) { _ => + SparkSubmit.main(args) + assert(podCompletedFuture.get, "Failed to run driver pod") + val driverPod = minikubeKubernetesClient + .pods + .withLabel("spark-app-name", "spark-file-existence-test") + .list() + .getItems + .get(0) + val podLog = minikubeKubernetesClient + .pods + .withName(driverPod.getMetadata.getName) + .getLog + assert(podLog.contains(s"File found at /opt/spark/${TEST_EXISTENCE_FILE.getName}" + + s" with correct contents."), "Job did not find the file as expected.") + } + } } diff --git a/resource-managers/kubernetes/integration-tests/test-data/input.txt b/resource-managers/kubernetes/integration-tests/test-data/input.txt new file mode 100644 index 0000000000000..dfe437bdebebc --- /dev/null +++ b/resource-managers/kubernetes/integration-tests/test-data/input.txt @@ -0,0 +1 @@ +Contents From de9a82e95687c753450afd535a5365f7c53b5a72 Mon Sep 17 00:00:00 2001 From: Andrew Ash Date: Thu, 2 Feb 2017 15:13:39 -0800 Subject: [PATCH 39/81] Fix NPE around unschedulable pod specs (#79) --- .../main/scala/org/apache/spark/deploy/kubernetes/Client.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/Client.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/Client.scala index c350c4817664d..bef5a605f173b 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/Client.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/Client.scala @@ -570,7 +570,8 @@ private[spark] class Client( .filter(_.getName == SUBMISSION_SERVER_PORT_NAME) .head.getNodePort val nodeUrls = kubernetesClient.nodes.list.getItems.asScala - .filterNot(_.getSpec.getUnschedulable) + .filterNot(node => node.getSpec.getUnschedulable != null && + node.getSpec.getUnschedulable) .flatMap(_.getStatus.getAddresses.asScala.map(address => { s"$urlScheme://${address.getAddress}:$servicePort" })).toArray From fae76a07b9357bed889d5fe66bd80d6cca2c2e87 Mon Sep 17 00:00:00 2001 From: Andrew Ash Date: Thu, 2 Feb 2017 17:34:15 -0800 Subject: [PATCH 40/81] Introduce blocking submit to kubernetes by default (#53) * Introduce blocking submit to kubernetes by default Two new configuration settings: - spark.kubernetes.submit.waitAppCompletion - spark.kubernetes.report.interval * Minor touchups * More succinct logging for pod state * Fix import order * Switch to watch-based logging * Spaces in comma-joined volumes, labels, and containers * Use CountDownLatch instead of SettableFuture * Match parallel ConfigBuilder style * Disable logging in fire-and-forget mode Which is enabled with spark.kubernetes.submit.waitAppCompletion=false (default: true) * Additional log line for when application is launched * Minor wording changes * More logging * Drop log to DEBUG --- .../spark/deploy/kubernetes/Client.scala | 154 +++++++++++------- .../kubernetes/LoggingPodStatusWatcher.scala | 114 +++++++++++++ .../spark/deploy/kubernetes/config.scala | 19 +++ 3 files changed, 225 insertions(+), 62 deletions(-) create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/LoggingPodStatusWatcher.scala diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/Client.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/Client.scala index bef5a605f173b..433c45d51fd6b 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/Client.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/Client.scala @@ -18,7 +18,7 @@ package org.apache.spark.deploy.kubernetes import java.io.{File, FileInputStream} import java.security.{KeyStore, SecureRandom} -import java.util.concurrent.{TimeoutException, TimeUnit} +import java.util.concurrent.{CountDownLatch, TimeoutException, TimeUnit} import java.util.concurrent.atomic.AtomicBoolean import javax.net.ssl.{SSLContext, TrustManagerFactory, X509TrustManager} @@ -26,7 +26,7 @@ import com.google.common.base.Charsets import com.google.common.io.Files import com.google.common.util.concurrent.SettableFuture import io.fabric8.kubernetes.api.model._ -import io.fabric8.kubernetes.client.{ConfigBuilder, DefaultKubernetesClient, KubernetesClient, KubernetesClientException, Watcher} +import io.fabric8.kubernetes.client.{ConfigBuilder => K8SConfigBuilder, DefaultKubernetesClient, KubernetesClient, KubernetesClientException, Watcher} import io.fabric8.kubernetes.client.Watcher.Action import org.apache.commons.codec.binary.Base64 import scala.collection.JavaConverters._ @@ -67,6 +67,8 @@ private[spark] class Client( private val uiPort = sparkConf.getInt("spark.ui.port", DEFAULT_UI_PORT) private val driverSubmitTimeoutSecs = sparkConf.get(KUBERNETES_DRIVER_SUBMIT_TIMEOUT) + private val waitForAppCompletion: Boolean = sparkConf.get(WAIT_FOR_APP_COMPLETION) + private val secretBase64String = { val secretBytes = new Array[Byte](128) SECURE_RANDOM.nextBytes(secretBytes) @@ -81,9 +83,11 @@ private[spark] class Client( ThreadUtils.newDaemonSingleThreadExecutor("kubernetes-client-retryable-futures")) def run(): Unit = { + logInfo(s"Starting application $kubernetesAppId in Kubernetes...") val (driverSubmitSslOptions, isKeyStoreLocalFile) = parseDriverSubmitSslOptions() + val parsedCustomLabels = parseCustomLabels(customLabels) - var k8ConfBuilder = new ConfigBuilder() + var k8ConfBuilder = new K8SConfigBuilder() .withApiVersion("v1") .withMasterUrl(master) .withNamespace(namespace) @@ -116,73 +120,97 @@ private[spark] class Client( SPARK_APP_NAME_LABEL -> appName) ++ parsedCustomLabels).asJava val containerPorts = buildContainerPorts() - val submitCompletedFuture = SettableFuture.create[Boolean] - val submitPending = new AtomicBoolean(false) - val podWatcher = new DriverPodWatcher( - submitCompletedFuture, - submitPending, - kubernetesClient, - driverSubmitSslOptions, - Array(submitServerSecret) ++ sslSecrets, - driverKubernetesSelectors) + + // start outer watch for status logging of driver pod + val driverPodCompletedLatch = new CountDownLatch(1) + // only enable interval logging if in waitForAppCompletion mode + val loggingInterval = if (waitForAppCompletion) sparkConf.get(REPORT_INTERVAL) else 0 + val loggingWatch = new LoggingPodStatusWatcher(driverPodCompletedLatch, kubernetesAppId, + loggingInterval) Utils.tryWithResource(kubernetesClient .pods() .withLabels(driverKubernetesSelectors) - .watch(podWatcher)) { _ => - kubernetesClient.pods().createNew() - .withNewMetadata() - .withName(kubernetesAppId) + .watch(loggingWatch)) { _ => + + // launch driver pod with inner watch to upload jars when it's ready + val submitCompletedFuture = SettableFuture.create[Boolean] + val submitPending = new AtomicBoolean(false) + val podWatcher = new DriverPodWatcher( + submitCompletedFuture, + submitPending, + kubernetesClient, + driverSubmitSslOptions, + Array(submitServerSecret) ++ sslSecrets, + driverKubernetesSelectors) + Utils.tryWithResource(kubernetesClient + .pods() .withLabels(driverKubernetesSelectors) - .endMetadata() - .withNewSpec() - .withRestartPolicy("OnFailure") - .addNewVolume() - .withName(SUBMISSION_APP_SECRET_VOLUME_NAME) - .withNewSecret() - .withSecretName(submitServerSecret.getMetadata.getName) - .endSecret() - .endVolume - .addToVolumes(sslVolumes: _*) - .withServiceAccount(serviceAccount) - .addNewContainer() - .withName(DRIVER_CONTAINER_NAME) - .withImage(driverDockerImage) - .withImagePullPolicy("IfNotPresent") - .addNewVolumeMount() + .watch(podWatcher)) { _ => + kubernetesClient.pods().createNew() + .withNewMetadata() + .withName(kubernetesAppId) + .withLabels(driverKubernetesSelectors) + .endMetadata() + .withNewSpec() + .withRestartPolicy("OnFailure") + .addNewVolume() .withName(SUBMISSION_APP_SECRET_VOLUME_NAME) - .withMountPath(secretDirectory) - .withReadOnly(true) - .endVolumeMount() - .addToVolumeMounts(sslVolumeMounts: _*) - .addNewEnv() - .withName(ENV_SUBMISSION_SECRET_LOCATION) - .withValue(s"$secretDirectory/$SUBMISSION_APP_SECRET_NAME") - .endEnv() - .addNewEnv() - .withName(ENV_SUBMISSION_SERVER_PORT) - .withValue(SUBMISSION_SERVER_PORT.toString) - .endEnv() - .addToEnv(sslEnvs: _*) - .withPorts(containerPorts.asJava) - .endContainer() - .endSpec() - .done() - var submitSucceeded = false - try { - submitCompletedFuture.get(driverSubmitTimeoutSecs, TimeUnit.SECONDS) - submitSucceeded = true - } catch { - case e: TimeoutException => - val finalErrorMessage: String = buildSubmitFailedErrorMessage(kubernetesClient, e) - logError(finalErrorMessage, e) - throw new SparkException(finalErrorMessage, e) - } finally { - if (!submitSucceeded) { - Utils.tryLogNonFatalError { - kubernetesClient.pods.withName(kubernetesAppId).delete() + .withNewSecret() + .withSecretName(submitServerSecret.getMetadata.getName) + .endSecret() + .endVolume + .addToVolumes(sslVolumes: _*) + .withServiceAccount(serviceAccount) + .addNewContainer() + .withName(DRIVER_CONTAINER_NAME) + .withImage(driverDockerImage) + .withImagePullPolicy("IfNotPresent") + .addNewVolumeMount() + .withName(SUBMISSION_APP_SECRET_VOLUME_NAME) + .withMountPath(secretDirectory) + .withReadOnly(true) + .endVolumeMount() + .addToVolumeMounts(sslVolumeMounts: _*) + .addNewEnv() + .withName(ENV_SUBMISSION_SECRET_LOCATION) + .withValue(s"$secretDirectory/$SUBMISSION_APP_SECRET_NAME") + .endEnv() + .addNewEnv() + .withName(ENV_SUBMISSION_SERVER_PORT) + .withValue(SUBMISSION_SERVER_PORT.toString) + .endEnv() + .addToEnv(sslEnvs: _*) + .withPorts(containerPorts.asJava) + .endContainer() + .endSpec() + .done() + var submitSucceeded = false + try { + submitCompletedFuture.get(driverSubmitTimeoutSecs, TimeUnit.SECONDS) + submitSucceeded = true + logInfo(s"Finished launching local resources to application $kubernetesAppId") + } catch { + case e: TimeoutException => + val finalErrorMessage: String = buildSubmitFailedErrorMessage(kubernetesClient, e) + logError(finalErrorMessage, e) + throw new SparkException(finalErrorMessage, e) + } finally { + if (!submitSucceeded) { + Utils.tryLogNonFatalError { + kubernetesClient.pods.withName(kubernetesAppId).delete() + } } } } + + // wait if configured to do so + if (waitForAppCompletion) { + logInfo(s"Waiting for application $kubernetesAppId to finish...") + driverPodCompletedLatch.await() + logInfo(s"Application $kubernetesAppId finished.") + } else { + logInfo(s"Application $kubernetesAppId successfully launched.") + } } } finally { Utils.tryLogNonFatalError { @@ -377,6 +405,8 @@ private[spark] class Client( Future { sparkConf.set("spark.driver.host", pod.getStatus.getPodIP) val submitRequest = buildSubmissionRequest() + logInfo(s"Submitting local resources to driver pod for application " + + s"$kubernetesAppId ...") driverSubmitter.submitApplication(submitRequest) } } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/LoggingPodStatusWatcher.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/LoggingPodStatusWatcher.scala new file mode 100644 index 0000000000000..cbacaf6bda854 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/LoggingPodStatusWatcher.scala @@ -0,0 +1,114 @@ +/* + * 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.kubernetes + +import java.util.concurrent.{CountDownLatch, Executors, TimeUnit} + +import scala.collection.JavaConverters._ + +import io.fabric8.kubernetes.api.model.Pod +import io.fabric8.kubernetes.client.{KubernetesClientException, Watcher} +import io.fabric8.kubernetes.client.Watcher.Action + +import org.apache.spark.internal.Logging + +/** + * A monitor for the running Kubernetes pod of a Spark application. Status logging occurs on + * every state change and also at an interval for liveness. + * + * @param podCompletedFuture a CountDownLatch that is set to true when the watched pod finishes + * @param appId + * @param interval ms between each state request. If set to 0 or a negative number, the periodic + * logging will be disabled. + */ +private[kubernetes] class LoggingPodStatusWatcher(podCompletedFuture: CountDownLatch, + appId: String, + interval: Long) + extends Watcher[Pod] with Logging { + + // start timer for periodic logging + private val scheduler = Executors.newScheduledThreadPool(1) + private val logRunnable: Runnable = new Runnable { + override def run() = logShortStatus() + } + if (interval > 0) { + scheduler.scheduleWithFixedDelay(logRunnable, 0, interval, TimeUnit.MILLISECONDS) + } + + private var pod: Option[Pod] = Option.empty + private var prevPhase: String = null + private def phase: String = pod.map(_.getStatus().getPhase()).getOrElse("unknown") + + override def eventReceived(action: Action, pod: Pod): Unit = { + this.pod = Option(pod) + + logShortStatus() + if (prevPhase != phase) { + logLongStatus() + } + prevPhase = phase + + if (phase == "Succeeded" || phase == "Failed") { + podCompletedFuture.countDown() + } + } + + override def onClose(e: KubernetesClientException): Unit = { + scheduler.shutdown() + logDebug(s"Stopped watching application $appId with last-observed phase $phase") + } + + private def logShortStatus() = { + logInfo(s"Application status for $appId (phase: $phase)") + } + + private def logLongStatus() = { + logInfo("Phase changed, new state: " + pod.map(formatPodState(_)).getOrElse("unknown")) + } + + private def formatPodState(pod: Pod): String = { + + val details = Seq[(String, String)]( + // pod metadata + ("pod name", pod.getMetadata.getName()), + ("namespace", pod.getMetadata.getNamespace()), + ("labels", pod.getMetadata.getLabels().asScala.mkString(", ")), + ("pod uid", pod.getMetadata.getUid), + ("creation time", pod.getMetadata.getCreationTimestamp()), + + // spec details + ("service account name", pod.getSpec.getServiceAccountName()), + ("volumes", pod.getSpec.getVolumes().asScala.map(_.getName).mkString(", ")), + ("node name", pod.getSpec.getNodeName()), + + // status + ("start time", pod.getStatus.getStartTime), + ("container images", + pod.getStatus.getContainerStatuses() + .asScala + .map(_.getImage) + .mkString(", ")), + ("phase", pod.getStatus.getPhase()) + ) + + // Use more loggable format if value is null or empty + details.map { case (k, v) => + val newValue = Option(v).filter(_.nonEmpty).getOrElse("N/A") + s"\n\t $k: $newValue" + }.mkString("") + } +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/config.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/config.scala index 3e0c400febca1..cb4cd42142ca4 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/config.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/config.scala @@ -188,4 +188,23 @@ package object config { .internal() .stringConf .createOptional + + private[spark] val WAIT_FOR_APP_COMPLETION = + ConfigBuilder("spark.kubernetes.submit.waitAppCompletion") + .doc( + """ + | In cluster mode, whether to wait for the application to finish before exiting the + | launcher process. + """.stripMargin) + .booleanConf + .createWithDefault(true) + + private[spark] val REPORT_INTERVAL = + ConfigBuilder("spark.kubernetes.report.interval") + .doc( + """ + | Interval between reports of the current app status in cluster mode. + """.stripMargin) + .timeConf(TimeUnit.MILLISECONDS) + .createWithDefaultString("1s") } From 4bc7c523d5c102ae4fbb55c624cb56efbf4dd3da Mon Sep 17 00:00:00 2001 From: Shuai Lin Date: Fri, 3 Feb 2017 19:40:32 +0000 Subject: [PATCH 41/81] Do not wait for pod finishing in integration tests. (#84) Since the example job are patched to never finish. --- .../deploy/kubernetes/integrationtest/KubernetesSuite.scala | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KubernetesSuite.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KubernetesSuite.scala index 40867c40d4474..c5458eccf830d 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KubernetesSuite.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KubernetesSuite.scala @@ -181,6 +181,7 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { .set("spark.app.name", "spark-pi") .set("spark.ui.enabled", "true") .set("spark.testing", "false") + .set("spark.kubernetes.submit.waitAppCompletion", "false") val mainAppResource = s"file://$EXAMPLES_JAR" new Client( @@ -210,6 +211,7 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { "--conf", s"spark.kubernetes.submit.clientCertFile=${clientConfig.getClientCertFile}", "--conf", "spark.kubernetes.executor.docker.image=spark-executor:latest", "--conf", "spark.kubernetes.driver.docker.image=spark-driver:latest", + "--conf", "spark.kubernetes.submit.waitAppCompletion=false", EXAMPLES_JAR) SparkSubmit.main(args) val sparkMetricsService = getSparkMetricsService("spark-pi") @@ -231,6 +233,7 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { "--conf", s"spark.kubernetes.submit.clientCertFile=${clientConfig.getClientCertFile}", "--conf", "spark.kubernetes.executor.docker.image=spark-executor:latest", "--conf", "spark.kubernetes.driver.docker.image=spark-driver:latest", + "--conf", "spark.kubernetes.submit.waitAppCompletion=false", s"container:///opt/spark/examples/jars/$EXAMPLES_JAR_FILE_NAME") val allContainersSucceeded = SettableFuture.create[Boolean] val watcher = new Watcher[Pod] { @@ -292,6 +295,7 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { "--conf", "spark.kubernetes.executor.docker.image=spark-executor:latest", "--conf", "spark.kubernetes.driver.docker.image=spark-driver:latest", "--conf", "spark.kubernetes.driver.labels=label1=label1value,label2=label2value", + "--conf", "spark.kubernetes.submit.waitAppCompletion=false", EXAMPLES_JAR) SparkSubmit.main(args) val driverPodLabels = minikubeKubernetesClient @@ -337,6 +341,7 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { "--conf", "spark.ssl.kubernetes.submit.trustStore=" + s"file://${trustStoreFile.getAbsolutePath}", "--conf", s"spark.ssl.kubernetes.driverlaunch.trustStorePassword=changeit", + "--conf", "spark.kubernetes.submit.waitAppCompletion=false", EXAMPLES_JAR) SparkSubmit.main(args) } @@ -360,6 +365,7 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { "--conf", s"spark.kubernetes.submit.clientCertFile=${clientConfig.getClientCertFile}", "--conf", "spark.kubernetes.executor.docker.image=spark-executor:latest", "--conf", "spark.kubernetes.driver.docker.image=spark-driver:latest", + "--conf", "spark.kubernetes.submit.waitAppCompletion=false", EXAMPLES_JAR, TEST_EXISTENCE_FILE.getName, TEST_EXISTENCE_FILE_CONTENTS) From 52a7ab2c8b8071ed78f4d5d4bd32ecaa1a7051db Mon Sep 17 00:00:00 2001 From: Shuai Lin Date: Wed, 8 Feb 2017 20:47:41 +0000 Subject: [PATCH 42/81] Check for user jars/files existence before creating the driver pod. (#86) * Check for user jars/files existence before creating the driver pod. Close apache-spark-on-k8s/spark#85 * CR --- .../spark/deploy/kubernetes/Client.scala | 20 ++++++++++++++++++- 1 file changed, 19 insertions(+), 1 deletion(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/Client.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/Client.scala index 433c45d51fd6b..b9b275c190fee 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/Client.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/Client.scala @@ -84,8 +84,10 @@ private[spark] class Client( def run(): Unit = { logInfo(s"Starting application $kubernetesAppId in Kubernetes...") - val (driverSubmitSslOptions, isKeyStoreLocalFile) = parseDriverSubmitSslOptions() + Seq(uploadedFiles, uploadedJars, Some(mainAppResource)).foreach(checkForFilesExistence) + + val (driverSubmitSslOptions, isKeyStoreLocalFile) = parseDriverSubmitSslOptions() val parsedCustomLabels = parseCustomLabels(customLabels) var k8ConfBuilder = new K8SConfigBuilder() .withApiVersion("v1") @@ -661,6 +663,22 @@ private[spark] class Client( }).toMap }).getOrElse(Map.empty[String, String]) } + + private def checkForFilesExistence(maybePaths: Option[String]): Unit = { + maybePaths.foreach { paths => + paths.split(",").foreach { path => + val uri = Utils.resolveURI(path) + uri.getScheme match { + case "file" | null => + val file = new File(uri.getPath) + if (!file.isFile) { + throw new SparkException(s"""file "${uri}" does not exist!""") + } + case _ => + } + } + } + } } private[spark] object Client extends Logging { From 487d1e160e8f953cef0e59f0a8a06e6ad50bcf99 Mon Sep 17 00:00:00 2001 From: mccheah Date: Wed, 8 Feb 2017 17:59:50 -0800 Subject: [PATCH 43/81] Use readiness probe instead of client-side ping. (#75) * Use readiness probe instead of client-side ping. Keep one ping() just as a sanity check, but otherwise set up the readiness probe to report the container as ready only when the ping endpoint can be reached. Also add a liveliness probe for convenience and symmetry. * Extract common HTTP get action * Remove some code * Add delay to liveliness check * Fix merge conflicts. * Fix more merge conflicts * Fix more merge conflicts * Revamp readiness check logic * Add addresses ready condition to endpoints watch * Rearrange the logic some more. * Remove liveness probe, retry against servers * Fix compiler error * Fix another compiler error * Delay between retries. Remove unintended test modification * FIx another compiler error * Extract method * Address comments * Deduplicate node addresses, use lower initial connect timeout * Drop maxRetriesPerServer from 10 to 3 --- .../spark/deploy/kubernetes/Client.scala | 643 +++++++++++------- .../spark/deploy/kubernetes/Retry.scala | 54 -- .../rest/kubernetes/HttpClientUtil.scala | 7 +- .../kubernetes/MultiServerFeignTarget.scala | 34 +- .../integrationtest/minikube/Minikube.scala | 2 +- 5 files changed, 426 insertions(+), 314 deletions(-) delete mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/Retry.scala diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/Client.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/Client.scala index b9b275c190fee..9eed9bfd2cd79 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/Client.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/Client.scala @@ -18,8 +18,8 @@ package org.apache.spark.deploy.kubernetes import java.io.{File, FileInputStream} import java.security.{KeyStore, SecureRandom} -import java.util.concurrent.{CountDownLatch, TimeoutException, TimeUnit} -import java.util.concurrent.atomic.AtomicBoolean +import java.util +import java.util.concurrent.{CountDownLatch, TimeUnit} import javax.net.ssl.{SSLContext, TrustManagerFactory, X509TrustManager} import com.google.common.base.Charsets @@ -31,8 +31,6 @@ import io.fabric8.kubernetes.client.Watcher.Action import org.apache.commons.codec.binary.Base64 import scala.collection.JavaConverters._ import scala.collection.mutable -import scala.concurrent.{ExecutionContext, Future} -import scala.concurrent.duration.DurationInt import org.apache.spark.{SecurityManager, SparkConf, SparkException, SSLOptions} import org.apache.spark.deploy.kubernetes.config._ @@ -40,7 +38,7 @@ import org.apache.spark.deploy.kubernetes.constants._ import org.apache.spark.deploy.rest.{AppResource, ContainerAppResource, KubernetesCreateSubmissionRequest, RemoteAppResource, TarGzippedData, UploadedAppResource} import org.apache.spark.deploy.rest.kubernetes._ import org.apache.spark.internal.Logging -import org.apache.spark.util.{ThreadUtils, Utils} +import org.apache.spark.util.Utils private[spark] class Client( sparkConf: SparkConf, @@ -78,10 +76,6 @@ private[spark] class Client( private val serviceAccount = sparkConf.get(KUBERNETES_SERVICE_ACCOUNT_NAME) private val customLabels = sparkConf.get(KUBERNETES_DRIVER_LABELS) - private implicit val retryableExecutionContext = ExecutionContext - .fromExecutorService( - ThreadUtils.newDaemonSingleThreadExecutor("kubernetes-client-retryable-futures")) - def run(): Unit = { logInfo(s"Starting application $kubernetesAppId in Kubernetes...") @@ -112,119 +106,398 @@ private[spark] class Client( .withData(Map((SUBMISSION_APP_SECRET_NAME, secretBase64String)).asJava) .withType("Opaque") .done() - val (sslEnvs, sslVolumes, sslVolumeMounts, sslSecrets) = configureSsl(kubernetesClient, - driverSubmitSslOptions, - isKeyStoreLocalFile) try { - val driverKubernetesSelectors = (Map( - SPARK_DRIVER_LABEL -> kubernetesAppId, - SPARK_APP_ID_LABEL -> kubernetesAppId, - SPARK_APP_NAME_LABEL -> appName) - ++ parsedCustomLabels).asJava - val containerPorts = buildContainerPorts() - - // start outer watch for status logging of driver pod - val driverPodCompletedLatch = new CountDownLatch(1) - // only enable interval logging if in waitForAppCompletion mode - val loggingInterval = if (waitForAppCompletion) sparkConf.get(REPORT_INTERVAL) else 0 - val loggingWatch = new LoggingPodStatusWatcher(driverPodCompletedLatch, kubernetesAppId, - loggingInterval) - Utils.tryWithResource(kubernetesClient - .pods() - .withLabels(driverKubernetesSelectors) - .watch(loggingWatch)) { _ => - - // launch driver pod with inner watch to upload jars when it's ready - val submitCompletedFuture = SettableFuture.create[Boolean] - val submitPending = new AtomicBoolean(false) - val podWatcher = new DriverPodWatcher( - submitCompletedFuture, - submitPending, - kubernetesClient, - driverSubmitSslOptions, - Array(submitServerSecret) ++ sslSecrets, - driverKubernetesSelectors) + val (sslEnvs, sslVolumes, sslVolumeMounts, sslSecrets) = configureSsl(kubernetesClient, + driverSubmitSslOptions, + isKeyStoreLocalFile) + try { + // start outer watch for status logging of driver pod + val driverPodCompletedLatch = new CountDownLatch(1) + // only enable interval logging if in waitForAppCompletion mode + val loggingInterval = if (waitForAppCompletion) sparkConf.get(REPORT_INTERVAL) else 0 + val loggingWatch = new LoggingPodStatusWatcher(driverPodCompletedLatch, kubernetesAppId, + loggingInterval) Utils.tryWithResource(kubernetesClient .pods() - .withLabels(driverKubernetesSelectors) - .watch(podWatcher)) { _ => - kubernetesClient.pods().createNew() - .withNewMetadata() - .withName(kubernetesAppId) - .withLabels(driverKubernetesSelectors) - .endMetadata() - .withNewSpec() - .withRestartPolicy("OnFailure") - .addNewVolume() - .withName(SUBMISSION_APP_SECRET_VOLUME_NAME) - .withNewSecret() - .withSecretName(submitServerSecret.getMetadata.getName) - .endSecret() - .endVolume - .addToVolumes(sslVolumes: _*) - .withServiceAccount(serviceAccount) - .addNewContainer() - .withName(DRIVER_CONTAINER_NAME) - .withImage(driverDockerImage) - .withImagePullPolicy("IfNotPresent") - .addNewVolumeMount() - .withName(SUBMISSION_APP_SECRET_VOLUME_NAME) - .withMountPath(secretDirectory) - .withReadOnly(true) - .endVolumeMount() - .addToVolumeMounts(sslVolumeMounts: _*) - .addNewEnv() - .withName(ENV_SUBMISSION_SECRET_LOCATION) - .withValue(s"$secretDirectory/$SUBMISSION_APP_SECRET_NAME") - .endEnv() - .addNewEnv() - .withName(ENV_SUBMISSION_SERVER_PORT) - .withValue(SUBMISSION_SERVER_PORT.toString) - .endEnv() - .addToEnv(sslEnvs: _*) - .withPorts(containerPorts.asJava) - .endContainer() - .endSpec() - .done() - var submitSucceeded = false - try { - submitCompletedFuture.get(driverSubmitTimeoutSecs, TimeUnit.SECONDS) - submitSucceeded = true - logInfo(s"Finished launching local resources to application $kubernetesAppId") + .withName(kubernetesAppId) + .watch(loggingWatch)) { _ => + val (driverPod, driverService) = launchDriverKubernetesComponents( + kubernetesClient, + parsedCustomLabels, + submitServerSecret, + driverSubmitSslOptions, + sslSecrets, + sslVolumes, + sslVolumeMounts, + sslEnvs, + isKeyStoreLocalFile) + val ownerReferenceConfiguredDriverService = try { + configureOwnerReferences( + kubernetesClient, + submitServerSecret, + sslSecrets, + driverPod, + driverService) } catch { - case e: TimeoutException => - val finalErrorMessage: String = buildSubmitFailedErrorMessage(kubernetesClient, e) - logError(finalErrorMessage, e) - throw new SparkException(finalErrorMessage, e) - } finally { - if (!submitSucceeded) { - Utils.tryLogNonFatalError { - kubernetesClient.pods.withName(kubernetesAppId).delete() - } + case e: Throwable => + cleanupPodAndService(kubernetesClient, driverPod, driverService) + throw new SparkException("Failed to set owner references to the driver pod.", e) + } + try { + submitApplicationToDriverServer(kubernetesClient, driverSubmitSslOptions, + ownerReferenceConfiguredDriverService) + // wait if configured to do so + if (waitForAppCompletion) { + logInfo(s"Waiting for application $kubernetesAppId to finish...") + driverPodCompletedLatch.await() + logInfo(s"Application $kubernetesAppId finished.") + } else { + logInfo(s"Application $kubernetesAppId successfully launched.") } + } catch { + case e: Throwable => + cleanupPodAndService(kubernetesClient, driverPod, + ownerReferenceConfiguredDriverService) + throw new SparkException("Failed to submit the application to the driver pod.", e) } } - - // wait if configured to do so - if (waitForAppCompletion) { - logInfo(s"Waiting for application $kubernetesAppId to finish...") - driverPodCompletedLatch.await() - logInfo(s"Application $kubernetesAppId finished.") - } else { - logInfo(s"Application $kubernetesAppId successfully launched.") + } finally { + Utils.tryLogNonFatalError { + // Secrets may have been mutated so delete by name to avoid problems with not having + // the latest version. + sslSecrets.foreach { secret => + kubernetesClient.secrets().withName(secret.getMetadata.getName).delete() + } } } } finally { Utils.tryLogNonFatalError { - kubernetesClient.secrets().delete(submitServerSecret) + kubernetesClient.secrets().withName(submitServerSecret.getMetadata.getName).delete() } - Utils.tryLogNonFatalError { - kubernetesClient.secrets().delete(sslSecrets: _*) + } + } + } + + private def cleanupPodAndService( + kubernetesClient: KubernetesClient, + driverPod: Pod, + driverService: Service): Unit = { + Utils.tryLogNonFatalError { + kubernetesClient.services().delete(driverService) + } + Utils.tryLogNonFatalError { + kubernetesClient.pods().delete(driverPod) + } + } + + private def submitApplicationToDriverServer( + kubernetesClient: KubernetesClient, + driverSubmitSslOptions: SSLOptions, + driverService: Service) = { + sparkConf.getOption("spark.app.id").foreach { id => + logWarning(s"Warning: Provided app id in spark.app.id as $id will be" + + s" overridden as $kubernetesAppId") + } + sparkConf.set(KUBERNETES_DRIVER_POD_NAME, kubernetesAppId) + sparkConf.set(KUBERNETES_DRIVER_SERVICE_NAME, driverService.getMetadata.getName) + sparkConf.set("spark.app.id", kubernetesAppId) + sparkConf.setIfMissing("spark.app.name", appName) + sparkConf.setIfMissing("spark.driver.port", DEFAULT_DRIVER_PORT.toString) + sparkConf.setIfMissing("spark.blockmanager.port", + DEFAULT_BLOCKMANAGER_PORT.toString) + val driverSubmitter = buildDriverSubmissionClient(kubernetesClient, driverService, + driverSubmitSslOptions) + // Sanity check to see if the driver submitter is even reachable. + driverSubmitter.ping() + logInfo(s"Submitting local resources to driver pod for application " + + s"$kubernetesAppId ...") + val submitRequest = buildSubmissionRequest() + driverSubmitter.submitApplication(submitRequest) + logInfo("Successfully submitted local resources and driver configuration to" + + " driver pod.") + // After submitting, adjust the service to only expose the Spark UI + val uiServicePort = new ServicePortBuilder() + .withName(UI_PORT_NAME) + .withPort(uiPort) + .withNewTargetPort(uiPort) + .build() + kubernetesClient.services().withName(kubernetesAppId).edit().editSpec() + .withType("ClusterIP") + .withPorts(uiServicePort) + .endSpec() + .done() + logInfo("Finished submitting application to Kubernetes.") + } + + private def launchDriverKubernetesComponents( + kubernetesClient: KubernetesClient, + parsedCustomLabels: Map[String, String], + submitServerSecret: Secret, + driverSubmitSslOptions: SSLOptions, + sslSecrets: Array[Secret], + sslVolumes: Array[Volume], + sslVolumeMounts: Array[VolumeMount], + sslEnvs: Array[EnvVar], + isKeyStoreLocalFile: Boolean): (Pod, Service) = { + val endpointsReadyFuture = SettableFuture.create[Endpoints] + val endpointsReadyWatcher = new DriverEndpointsReadyWatcher(endpointsReadyFuture) + val serviceReadyFuture = SettableFuture.create[Service] + val driverKubernetesSelectors = (Map( + SPARK_DRIVER_LABEL -> kubernetesAppId, + SPARK_APP_ID_LABEL -> kubernetesAppId, + SPARK_APP_NAME_LABEL -> appName) + ++ parsedCustomLabels).asJava + val serviceReadyWatcher = new DriverServiceReadyWatcher(serviceReadyFuture) + val podReadyFuture = SettableFuture.create[Pod] + val podWatcher = new DriverPodReadyWatcher(podReadyFuture) + Utils.tryWithResource(kubernetesClient + .pods() + .withName(kubernetesAppId) + .watch(podWatcher)) { _ => + Utils.tryWithResource(kubernetesClient + .services() + .withName(kubernetesAppId) + .watch(serviceReadyWatcher)) { _ => + Utils.tryWithResource(kubernetesClient + .endpoints() + .withName(kubernetesAppId) + .watch(endpointsReadyWatcher)) { _ => + val driverService = createDriverService( + kubernetesClient, + driverKubernetesSelectors, + submitServerSecret) + val driverPod = try { + createDriverPod( + kubernetesClient, + driverKubernetesSelectors, + submitServerSecret, + driverSubmitSslOptions, + sslVolumes, + sslVolumeMounts, + sslEnvs) + } catch { + case e: Throwable => + Utils.tryLogNonFatalError { + kubernetesClient.services().delete(driverService) + } + throw new SparkException("Failed to create the driver pod.", e) + } + try { + waitForReadyKubernetesComponents(kubernetesClient, endpointsReadyFuture, + serviceReadyFuture, podReadyFuture) + (driverPod, driverService) + } catch { + case e: Throwable => + Utils.tryLogNonFatalError { + kubernetesClient.services().delete(driverService) + } + Utils.tryLogNonFatalError { + kubernetesClient.pods().delete(driverPod) + } + throw new SparkException("Timed out while waiting for a Kubernetes component to be" + + " ready.", e) + } } } } } + /** + * Sets the owner reference for all the kubernetes components to link to the driver pod. + * + * @return The driver service after it has been adjusted to reflect the new owner + * reference. + */ + private def configureOwnerReferences( + kubernetesClient: KubernetesClient, + submitServerSecret: Secret, + sslSecrets: Array[Secret], + driverPod: Pod, + driverService: Service): Service = { + val driverPodOwnerRef = new OwnerReferenceBuilder() + .withName(driverPod.getMetadata.getName) + .withUid(driverPod.getMetadata.getUid) + .withApiVersion(driverPod.getApiVersion) + .withKind(driverPod.getKind) + .withController(true) + .build() + sslSecrets.foreach(secret => { + kubernetesClient.secrets().withName(secret.getMetadata.getName).edit() + .editMetadata() + .addToOwnerReferences(driverPodOwnerRef) + .endMetadata() + .done() + }) + kubernetesClient.secrets().withName(submitServerSecret.getMetadata.getName).edit() + .editMetadata() + .addToOwnerReferences(driverPodOwnerRef) + .endMetadata() + .done() + kubernetesClient.services().withName(driverService.getMetadata.getName).edit() + .editMetadata() + .addToOwnerReferences(driverPodOwnerRef) + .endMetadata() + .done() + } + + private def waitForReadyKubernetesComponents( + kubernetesClient: KubernetesClient, + endpointsReadyFuture: SettableFuture[Endpoints], + serviceReadyFuture: SettableFuture[Service], + podReadyFuture: SettableFuture[Pod]) = { + try { + podReadyFuture.get(driverSubmitTimeoutSecs, TimeUnit.SECONDS) + logInfo("Driver pod successfully created in Kubernetes cluster.") + } catch { + case e: Throwable => + val finalErrorMessage: String = buildSubmitFailedErrorMessage(kubernetesClient, e) + logError(finalErrorMessage, e) + throw new SparkException(finalErrorMessage, e) + } + try { + serviceReadyFuture.get(driverSubmitTimeoutSecs, TimeUnit.SECONDS) + logInfo("Driver service created successfully in Kubernetes.") + } catch { + case e: Throwable => + throw new SparkException(s"The driver service was not ready" + + s" in $driverSubmitTimeoutSecs seconds.", e) + } + try { + endpointsReadyFuture.get(driverSubmitTimeoutSecs, TimeUnit.SECONDS) + logInfo("Driver endpoints ready to receive application submission") + } catch { + case e: Throwable => + throw new SparkException(s"The driver service endpoint was not ready" + + s" in $driverSubmitTimeoutSecs seconds.", e) + } + } + + private def createDriverService( + kubernetesClient: KubernetesClient, + driverKubernetesSelectors: java.util.Map[String, String], + submitServerSecret: Secret): Service = { + val driverSubmissionServicePort = new ServicePortBuilder() + .withName(SUBMISSION_SERVER_PORT_NAME) + .withPort(SUBMISSION_SERVER_PORT) + .withNewTargetPort(SUBMISSION_SERVER_PORT) + .build() + kubernetesClient.services().createNew() + .withNewMetadata() + .withName(kubernetesAppId) + .withLabels(driverKubernetesSelectors) + .endMetadata() + .withNewSpec() + .withType("NodePort") + .withSelector(driverKubernetesSelectors) + .withPorts(driverSubmissionServicePort) + .endSpec() + .done() + } + + private def createDriverPod( + kubernetesClient: KubernetesClient, + driverKubernetesSelectors: util.Map[String, String], + submitServerSecret: Secret, + driverSubmitSslOptions: SSLOptions, + sslVolumes: Array[Volume], + sslVolumeMounts: Array[VolumeMount], + sslEnvs: Array[EnvVar]) = { + val containerPorts = buildContainerPorts() + val probePingHttpGet = new HTTPGetActionBuilder() + .withScheme(if (driverSubmitSslOptions.enabled) "HTTPS" else "HTTP") + .withPath("/v1/submissions/ping") + .withNewPort(SUBMISSION_SERVER_PORT_NAME) + .build() + kubernetesClient.pods().createNew() + .withNewMetadata() + .withName(kubernetesAppId) + .withLabels(driverKubernetesSelectors) + .endMetadata() + .withNewSpec() + .withRestartPolicy("OnFailure") + .addNewVolume() + .withName(SUBMISSION_APP_SECRET_VOLUME_NAME) + .withNewSecret() + .withSecretName(submitServerSecret.getMetadata.getName) + .endSecret() + .endVolume() + .addToVolumes(sslVolumes: _*) + .withServiceAccount(serviceAccount) + .addNewContainer() + .withName(DRIVER_CONTAINER_NAME) + .withImage(driverDockerImage) + .withImagePullPolicy("IfNotPresent") + .addNewVolumeMount() + .withName(SUBMISSION_APP_SECRET_VOLUME_NAME) + .withMountPath(secretDirectory) + .withReadOnly(true) + .endVolumeMount() + .addToVolumeMounts(sslVolumeMounts: _*) + .addNewEnv() + .withName(ENV_SUBMISSION_SECRET_LOCATION) + .withValue(s"$secretDirectory/$SUBMISSION_APP_SECRET_NAME") + .endEnv() + .addNewEnv() + .withName(ENV_SUBMISSION_SERVER_PORT) + .withValue(SUBMISSION_SERVER_PORT.toString) + .endEnv() + .addToEnv(sslEnvs: _*) + .withPorts(containerPorts.asJava) + .withNewReadinessProbe().withHttpGet(probePingHttpGet).endReadinessProbe() + .endContainer() + .endSpec() + .done() + } + + private class DriverPodReadyWatcher(resolvedDriverPod: SettableFuture[Pod]) extends Watcher[Pod] { + override def eventReceived(action: Action, pod: Pod): Unit = { + if ((action == Action.ADDED || action == Action.MODIFIED) + && pod.getStatus.getPhase == "Running" + && !resolvedDriverPod.isDone) { + pod.getStatus + .getContainerStatuses + .asScala + .find(status => + status.getName == DRIVER_CONTAINER_NAME && status.getReady) + .foreach { _ => resolvedDriverPod.set(pod) } + } + } + + override def onClose(cause: KubernetesClientException): Unit = { + logDebug("Driver pod readiness watch closed.", cause) + } + } + + private class DriverEndpointsReadyWatcher(resolvedDriverEndpoints: SettableFuture[Endpoints]) + extends Watcher[Endpoints] { + override def eventReceived(action: Action, endpoints: Endpoints): Unit = { + if ((action == Action.ADDED) || (action == Action.MODIFIED) + && endpoints.getSubsets.asScala.nonEmpty + && endpoints.getSubsets.asScala.exists(_.getAddresses.asScala.nonEmpty) + && !resolvedDriverEndpoints.isDone) { + resolvedDriverEndpoints.set(endpoints) + } + } + + override def onClose(cause: KubernetesClientException): Unit = { + logDebug("Driver endpoints readiness watch closed.", cause) + } + } + + private class DriverServiceReadyWatcher(resolvedDriverService: SettableFuture[Service]) + extends Watcher[Service] { + override def eventReceived(action: Action, service: Service): Unit = { + if ((action == Action.ADDED) || (action == Action.MODIFIED) + && !resolvedDriverService.isDone) { + resolvedDriverService.set(service) + } + } + + override def onClose(cause: KubernetesClientException): Unit = { + logDebug("Driver service readiness watch closed.", cause) + } + } + private def parseDriverSubmitSslOptions(): (SSLOptions, Boolean) = { val maybeKeyStore = sparkConf.get(KUBERNETES_DRIVER_SUBMIT_KEYSTORE) val resolvedSparkConf = sparkConf.clone() @@ -306,18 +579,10 @@ private[spark] class Client( .withName(ENV_SUBMISSION_USE_SSL) .withValue("true") .build() - val sslSecrets = kubernetesClient.secrets().createNew() - .withNewMetadata() - .withName(sslSecretsName) - .endMetadata() - .withData(sslSecretsMap.asJava) - .withType("Opaque") - .done() - secrets += sslSecrets val sslVolume = new VolumeBuilder() .withName(SUBMISSION_SSL_SECRETS_VOLUME_NAME) .withNewSecret() - .withSecretName(sslSecrets.getMetadata.getName) + .withSecretName(sslSecretsName) .endSecret() .build() val sslVolumeMount = new VolumeMountBuilder() @@ -325,147 +590,23 @@ private[spark] class Client( .withReadOnly(true) .withMountPath(sslSecretsDirectory) .build() + val sslSecrets = kubernetesClient.secrets().createNew() + .withNewMetadata() + .withName(sslSecretsName) + .endMetadata() + .withData(sslSecretsMap.asJava) + .withType("Opaque") + .done() + secrets += sslSecrets (sslEnvs.toArray, Array(sslVolume), Array(sslVolumeMount), secrets.toArray) } else { (Array[EnvVar](), Array[Volume](), Array[VolumeMount](), Array[Secret]()) } } - private class DriverPodWatcher( - submitCompletedFuture: SettableFuture[Boolean], - submitPending: AtomicBoolean, - kubernetesClient: KubernetesClient, - driverSubmitSslOptions: SSLOptions, - applicationSecrets: Array[Secret], - driverKubernetesSelectors: java.util.Map[String, String]) extends Watcher[Pod] { - override def eventReceived(action: Action, pod: Pod): Unit = { - if ((action == Action.ADDED || action == Action.MODIFIED) - && pod.getStatus.getPhase == "Running" - && !submitCompletedFuture.isDone) { - if (!submitPending.getAndSet(true)) { - pod.getStatus - .getContainerStatuses - .asScala - .find(status => - status.getName == DRIVER_CONTAINER_NAME && status.getReady) match { - case Some(_) => - val ownerRefs = Seq(new OwnerReferenceBuilder() - .withName(pod.getMetadata.getName) - .withUid(pod.getMetadata.getUid) - .withApiVersion(pod.getApiVersion) - .withKind(pod.getKind) - .withController(true) - .build()) - - applicationSecrets.foreach(secret => { - secret.getMetadata.setOwnerReferences(ownerRefs.asJava) - kubernetesClient.secrets().createOrReplace(secret) - }) - - val driverSubmissionServicePort = new ServicePortBuilder() - .withName(SUBMISSION_SERVER_PORT_NAME) - .withPort(SUBMISSION_SERVER_PORT) - .withNewTargetPort(SUBMISSION_SERVER_PORT) - .build() - val service = kubernetesClient.services().createNew() - .withNewMetadata() - .withName(kubernetesAppId) - .withLabels(driverKubernetesSelectors) - .withOwnerReferences(ownerRefs.asJava) - .endMetadata() - .withNewSpec() - .withType("NodePort") - .withSelector(driverKubernetesSelectors) - .withPorts(driverSubmissionServicePort) - .endSpec() - .done() - try { - sparkConf.getOption("spark.app.id").foreach { id => - logWarning(s"Warning: Provided app id in spark.app.id as $id will be" + - s" overridden as $kubernetesAppId") - } - sparkConf.set(KUBERNETES_DRIVER_POD_NAME, kubernetesAppId) - sparkConf.set(KUBERNETES_DRIVER_SERVICE_NAME, service.getMetadata.getName) - sparkConf.set("spark.app.id", kubernetesAppId) - sparkConf.setIfMissing("spark.app.name", appName) - sparkConf.setIfMissing("spark.driver.port", DEFAULT_DRIVER_PORT.toString) - sparkConf.setIfMissing("spark.blockmanager.port", - DEFAULT_BLOCKMANAGER_PORT.toString) - val driverSubmitter = buildDriverSubmissionClient(kubernetesClient, service, - driverSubmitSslOptions) - val ping = Retry.retry(5, 5.seconds, - Some("Failed to contact the driver server")) { - driverSubmitter.ping() - } - ping onFailure { - case t: Throwable => - logError("Ping failed to the driver server", t) - submitCompletedFuture.setException(t) - kubernetesClient.services().delete(service) - } - val submitComplete = ping.flatMap { _ => - Future { - sparkConf.set("spark.driver.host", pod.getStatus.getPodIP) - val submitRequest = buildSubmissionRequest() - logInfo(s"Submitting local resources to driver pod for application " + - s"$kubernetesAppId ...") - driverSubmitter.submitApplication(submitRequest) - } - } - submitComplete onFailure { - case t: Throwable => - submitCompletedFuture.setException(t) - kubernetesClient.services().delete(service) - } - val adjustServicePort = submitComplete.flatMap { _ => - Future { - // After submitting, adjust the service to only expose the Spark UI - val uiServicePort = new ServicePortBuilder() - .withName(UI_PORT_NAME) - .withPort(uiPort) - .withNewTargetPort(uiPort) - .build() - kubernetesClient.services().withName(kubernetesAppId).edit() - .editSpec() - .withType("ClusterIP") - .withPorts(uiServicePort) - .endSpec() - .done - } - } - adjustServicePort onSuccess { - case _ => - submitCompletedFuture.set(true) - } - adjustServicePort onFailure { - case throwable: Throwable => - submitCompletedFuture.setException(throwable) - kubernetesClient.services().delete(service) - } - } catch { - case e: Throwable => - submitCompletedFuture.setException(e) - Utils.tryLogNonFatalError({ - kubernetesClient.services().delete(service) - }) - throw e - } - case None => - } - } - } - } - - override def onClose(e: KubernetesClientException): Unit = { - if (!submitCompletedFuture.isDone) { - submitCompletedFuture.setException(e) - } - } - } - private def buildSubmitFailedErrorMessage( - kubernetesClient: DefaultKubernetesClient, - e: TimeoutException): String = { + kubernetesClient: KubernetesClient, + e: Throwable): String = { val driverPod = try { kubernetesClient.pods().withName(kubernetesAppId).get() } catch { @@ -606,7 +747,7 @@ private[spark] class Client( node.getSpec.getUnschedulable) .flatMap(_.getStatus.getAddresses.asScala.map(address => { s"$urlScheme://${address.getAddress}:$servicePort" - })).toArray + })).toSet require(nodeUrls.nonEmpty, "No nodes found to contact the driver!") val (trustManager, sslContext): (X509TrustManager, SSLContext) = if (driverSubmitSslOptions.enabled) { @@ -616,8 +757,10 @@ private[spark] class Client( } HttpClientUtil.createClient[KubernetesSparkRestApi]( uris = nodeUrls, + maxRetriesPerServer = 3, sslSocketFactory = sslContext.getSocketFactory, - trustContext = trustManager) + trustContext = trustManager, + connectTimeoutMillis = 5000) } private def buildSslConnectionConfiguration(driverSubmitSslOptions: SSLOptions) = { diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/Retry.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/Retry.scala deleted file mode 100644 index 378583b29c547..0000000000000 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/Retry.scala +++ /dev/null @@ -1,54 +0,0 @@ -/* - * 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.kubernetes - -import scala.concurrent.{ExecutionContext, Future} -import scala.concurrent.duration.Duration - -import org.apache.spark.SparkException -import org.apache.spark.internal.Logging - -private[spark] object Retry extends Logging { - - private def retryableFuture[T] - (attempt: Int, maxAttempts: Int, interval: Duration, retryMessage: Option[String]) - (f: => Future[T]) - (implicit executionContext: ExecutionContext): Future[T] = { - f recoverWith { - case error: Throwable => - if (attempt <= maxAttempts) { - retryMessage.foreach { message => - logWarning(s"$message - attempt $attempt of $maxAttempts", error) - } - Thread.sleep(interval.toMillis) - retryableFuture(attempt + 1, maxAttempts, interval, retryMessage)(f) - } else { - Future.failed(retryMessage.map(message => - new SparkException(s"$message - reached $maxAttempts attempts," + - s" and aborting task.", error) - ).getOrElse(error)) - } - } - } - - def retry[T] - (times: Int, interval: Duration, retryMessage: Option[String] = None) - (f: => T) - (implicit executionContext: ExecutionContext): Future[T] = { - retryableFuture(1, times, interval, retryMessage)(Future[T] { f }) - } -} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/HttpClientUtil.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/HttpClientUtil.scala index 1cabfbad656eb..576f7058f20ee 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/HttpClientUtil.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/HttpClientUtil.scala @@ -32,7 +32,8 @@ import org.apache.spark.status.api.v1.JacksonMessageWriter private[spark] object HttpClientUtil { def createClient[T: ClassTag]( - uris: Array[String], + uris: Set[String], + maxRetriesPerServer: Int = 1, sslSocketFactory: SSLSocketFactory = SSLContext.getDefault.getSocketFactory, trustContext: X509TrustManager = null, readTimeoutMillis: Int = 20000, @@ -45,12 +46,12 @@ private[spark] object HttpClientUtil { .registerModule(new DefaultScalaModule) .setDateFormat(JacksonMessageWriter.makeISODateFormat) objectMapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false) - val target = new MultiServerFeignTarget[T](uris) + val target = new MultiServerFeignTarget[T](uris.toSeq, maxRetriesPerServer) val baseHttpClient = new feign.okhttp.OkHttpClient(httpClientBuilder.build()) val resetTargetHttpClient = new Client { override def execute(request: Request, options: Options): Response = { val response = baseHttpClient.execute(request, options) - if (response.status() >= 200 && response.status() < 300) { + if (response.status() / 100 == 2) { target.reset() } response diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/MultiServerFeignTarget.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/MultiServerFeignTarget.scala index fea7f057cfa1b..51313e00ce2da 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/MultiServerFeignTarget.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/MultiServerFeignTarget.scala @@ -20,20 +20,25 @@ import feign.{Request, RequestTemplate, RetryableException, Retryer, Target} import scala.reflect.ClassTag import scala.util.Random +import org.apache.spark.internal.Logging + private[kubernetes] class MultiServerFeignTarget[T : ClassTag]( - private val servers: Seq[String]) extends Target[T] with Retryer { + private val servers: Seq[String], + private val maxRetriesPerServer: Int = 1, + private val delayBetweenRetriesMillis: Int = 1000) extends Target[T] with Retryer with Logging { require(servers.nonEmpty, "Must provide at least one server URI.") private val threadLocalShuffledServers = new ThreadLocal[Seq[String]] { override def initialValue(): Seq[String] = Random.shuffle(servers) } + private val threadLocalCurrentAttempt = new ThreadLocal[Int] { + override def initialValue(): Int = 0 + } override def `type`(): Class[T] = { implicitly[ClassTag[T]].runtimeClass.asInstanceOf[Class[T]] } - override def url(): String = threadLocalShuffledServers.get.head - /** * Cloning the target is done on every request, for use on the current * thread - thus it's important that clone returns a "fresh" target. @@ -54,14 +59,31 @@ private[kubernetes] class MultiServerFeignTarget[T : ClassTag]( requestTemplate.request() } + override def url(): String = threadLocalShuffledServers.get.head + override def continueOrPropagate(e: RetryableException): Unit = { - threadLocalShuffledServers.set(threadLocalShuffledServers.get.drop(1)) - if (threadLocalShuffledServers.get.isEmpty) { - throw e + threadLocalCurrentAttempt.set(threadLocalCurrentAttempt.get + 1) + val currentAttempt = threadLocalCurrentAttempt.get + if (threadLocalCurrentAttempt.get < maxRetriesPerServer) { + logWarning(s"Attempt $currentAttempt of $maxRetriesPerServer failed for" + + s" server ${url()}. Retrying request...", e) + Thread.sleep(delayBetweenRetriesMillis) + } else { + val previousUrl = url() + threadLocalShuffledServers.set(threadLocalShuffledServers.get.drop(1)) + if (threadLocalShuffledServers.get.isEmpty) { + logError(s"Failed request to all servers $maxRetriesPerServer times.", e) + throw e + } else { + logWarning(s"Failed request to $previousUrl $maxRetriesPerServer times." + + s" Trying to access ${url()} instead.", e) + threadLocalCurrentAttempt.set(0) + } } } def reset(): Unit = { threadLocalShuffledServers.set(Random.shuffle(servers)) + threadLocalCurrentAttempt.set(0) } } diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/minikube/Minikube.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/minikube/Minikube.scala index b42f97952394e..736b92cc2d628 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/minikube/Minikube.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/minikube/Minikube.scala @@ -123,7 +123,7 @@ private[spark] object Minikube extends Logging { .build() val sslContext = SSLUtils.sslContext(kubernetesConf) val trustManager = SSLUtils.trustManagers(kubernetesConf)(0).asInstanceOf[X509TrustManager] - HttpClientUtil.createClient[T](Array(url), sslContext.getSocketFactory, trustManager) + HttpClientUtil.createClient[T](Set(url), 5, sslContext.getSocketFactory, trustManager) } def executeMinikubeSsh(command: String): Unit = { From bdfc4e122cfcf2782ac41028a7fa398bd2ae4e4b Mon Sep 17 00:00:00 2001 From: Andrew Ash Date: Thu, 9 Feb 2017 17:54:39 -0800 Subject: [PATCH 44/81] Note integration tests require Java 8 (#99) --- resource-managers/kubernetes/README.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/resource-managers/kubernetes/README.md b/resource-managers/kubernetes/README.md index 5e4ffaa54cb55..92ec305513f42 100644 --- a/resource-managers/kubernetes/README.md +++ b/resource-managers/kubernetes/README.md @@ -41,6 +41,8 @@ Below is a list of the submodules for this cluster manager and what they do. Note that the integration test framework is currently being heavily revised and is subject to change. +Note that currently the integration tests only run with Java 8. + Running any of the integration tests requires including `kubernetes-integration-tests` profile in the build command. In order to prepare the environment for running the integration tests, the `pre-integration-test` step must be run in Maven on the `resource-managers/kubernetes/integration-tests` module: From fe8b45c3543c8ae2af5f774c47a5f80142533307 Mon Sep 17 00:00:00 2001 From: Anirudh Ramanathan Date: Fri, 10 Feb 2017 14:50:38 -0800 Subject: [PATCH 45/81] Bumping up kubernetes-client version to fix GKE and local proxy (#105) * Bumping up kubernetes-client version to add fixes * Modify wording * Addressed comments --- docs/running-on-kubernetes.md | 30 ++++++++++++++++++- resource-managers/kubernetes/core/pom.xml | 2 +- .../KubernetesClusterSchedulerBackend.scala | 2 +- 3 files changed, 31 insertions(+), 3 deletions(-) diff --git a/docs/running-on-kubernetes.md b/docs/running-on-kubernetes.md index 5a48bb254a6df..19f406039e261 100644 --- a/docs/running-on-kubernetes.md +++ b/docs/running-on-kubernetes.md @@ -51,7 +51,7 @@ connect without SSL on a different port, the master would be set to `k8s://http: Note that applications can currently only be executed in cluster mode, where the driver and its executors are running on the cluster. - + ### Adding Other JARs Spark allows users to provide dependencies that are bundled into the driver's Docker image, or that are on the local @@ -150,6 +150,34 @@ or `container:`. A scheme of `file:` corresponds to the keyStore being located o the driver container as a [secret volume](https://kubernetes.io/docs/user-guide/secrets/). When the URI has the scheme `container:`, the file is assumed to already be on the container's disk at the appropriate path. +### Kubernetes Clusters and the authenticated proxy endpoint + +Spark-submit also supports submission through the +[local kubectl proxy](https://kubernetes.io/docs/user-guide/connecting-to-applications-proxy/). One can use the +authenticating proxy to communicate with the api server directly without passing credentials to spark-submit. + +The local proxy can be started by running: + + kubectl proxy + +If our local proxy were listening on port 8001, we would have our submission looking like the following: + + bin/spark-submit \ + --deploy-mode cluster \ + --class org.apache.spark.examples.SparkPi \ + --master k8s://http://127.0.0.1:8001 \ + --kubernetes-namespace default \ + --conf spark.executor.instances=5 \ + --conf spark.app.name=spark-pi \ + --conf spark.kubernetes.driver.docker.image=registry-host:5000/spark-driver:latest \ + --conf spark.kubernetes.executor.docker.image=registry-host:5000/spark-executor:latest \ + examples/jars/spark_examples_2.11-2.2.0.jar + +Communication between Spark and Kubernetes clusters is performed using the fabric8 kubernetes-client library. +The above mechanism using `kubectl proxy` can be used when we have authentication providers that the fabric8 +kubernetes-client library does not support. Authentication using X509 Client Certs and oauth tokens +is currently supported. + ### Spark Properties Below are some other common properties that are specific to Kubernetes. Most of the other configurations are the same diff --git a/resource-managers/kubernetes/core/pom.xml b/resource-managers/kubernetes/core/pom.xml index 86d7dec2c076f..a7eba625cd56c 100644 --- a/resource-managers/kubernetes/core/pom.xml +++ b/resource-managers/kubernetes/core/pom.xml @@ -29,7 +29,7 @@ Spark Project Kubernetes kubernetes - 1.4.34 + 2.0.3 diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala index 550ddd113fa42..83225098bc651 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala @@ -44,7 +44,7 @@ private[spark] class KubernetesClusterSchedulerBackend( private val EXECUTOR_MODIFICATION_LOCK = new Object private val runningExecutorPods = new scala.collection.mutable.HashMap[String, Pod] - private val kubernetesMaster = Client.resolveK8sMaster(sc.master) + private val kubernetesMaster = "https://kubernetes" private val executorDockerImage = conf.get(EXECUTOR_DOCKER_IMAGE) private val kubernetesNamespace = conf.get(KUBERNETES_NAMESPACE) private val executorPort = conf.getInt("spark.executor.port", DEFAULT_STATIC_PORT) From 7a4075f3a2f063ed26240d66841deeffcf3b5980 Mon Sep 17 00:00:00 2001 From: Andrew Ash Date: Fri, 10 Feb 2017 18:38:29 -0800 Subject: [PATCH 46/81] Truncate k8s hostnames to be no longer than 63 characters (#102) * Truncate k8s hostnames to be no longer than 63 characters * Use only executorId not executorKubernetesId --- .../KubernetesClusterSchedulerBackend.scala | 12 +++++++++--- 1 file changed, 9 insertions(+), 3 deletions(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala index 83225098bc651..d4e7da464be4a 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala @@ -155,9 +155,14 @@ private[spark] class KubernetesClusterSchedulerBackend( } private def allocateNewExecutorPod(): (String, Pod) = { - val executorKubernetesId = UUID.randomUUID().toString.replaceAll("-", "") val executorId = EXECUTOR_ID_COUNTER.incrementAndGet().toString - val name = s"${applicationId()}-exec-$executorKubernetesId" + val name = s"${applicationId()}-exec-$executorId" + + // hostname must be no longer than 63 characters, so take the last 63 characters of the pod + // name as the hostname. This preserves uniqueness since the end of name contains + // executorId and applicationId + val hostname = name.substring(Math.max(0, name.length - 63)) + val selectors = Map(SPARK_EXECUTOR_ID_LABEL -> executorId, SPARK_APP_ID_LABEL -> applicationId()).asJava val executorMemoryQuantity = new QuantityBuilder(false) @@ -190,7 +195,7 @@ private[spark] class KubernetesClusterSchedulerBackend( .build() }) try { - (executorKubernetesId, kubernetesClient.pods().createNew() + (executorId, kubernetesClient.pods().createNew() .withNewMetadata() .withName(name) .withLabels(selectors) @@ -204,6 +209,7 @@ private[spark] class KubernetesClusterSchedulerBackend( .endOwnerReference() .endMetadata() .withNewSpec() + .withHostname(hostname) .addNewContainer() .withName(s"executor") .withImage(executorDockerImage) From 3d80fffea60cb0e6ed14644adfec52d4e6185701 Mon Sep 17 00:00:00 2001 From: Shuai Lin Date: Mon, 13 Feb 2017 15:43:30 +0000 Subject: [PATCH 47/81] Fixed loading the executors page through the kubectl proxy. (#95) Fix apache-spark-on-k8s/spark#87 --- .../apache/spark/ui/static/executorspage.js | 34 +++++++++++++++++++ 1 file changed, 34 insertions(+) diff --git a/core/src/main/resources/org/apache/spark/ui/static/executorspage.js b/core/src/main/resources/org/apache/spark/ui/static/executorspage.js index fe5db6aa26b65..fa0282678d1f4 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/executorspage.js +++ b/core/src/main/resources/org/apache/spark/ui/static/executorspage.js @@ -54,7 +54,28 @@ $(document).ajaxStart(function () { $.blockUI({message: '

    Loading Executors Page...

    '}); }); +function findKubernetesServiceBaseURI() { + var k8sProxyPattern = '/api/v1/proxy/namespaces/'; + var k8sProxyPatternPos = document.baseURI.indexOf(k8sProxyPattern); + if (k8sProxyPatternPos > 0) { + // Spark is running in a kubernetes cluster, and the web ui is served + // through the kubectl proxy. + var remaining = document.baseURI.substr(k8sProxyPatternPos + k8sProxyPattern.length); + var urlSlashesCount = remaining.split('/').length - 3; + var words = document.baseURI.split('/'); + var baseURI = words.slice(0, words.length - urlSlashesCount).join('/'); + return baseURI; + } + + return null; +} + function createTemplateURI(appId) { + var kubernetesBaseURI = findKubernetesServiceBaseURI(); + if (kubernetesBaseURI) { + return kubernetesBaseURI + '/static/executorspage-template.html'; + } + var words = document.baseURI.split('/'); var ind = words.indexOf("proxy"); if (ind > 0) { @@ -70,6 +91,14 @@ function createTemplateURI(appId) { } function getStandAloneppId(cb) { + var kubernetesBaseURI = findKubernetesServiceBaseURI(); + if (kubernetesBaseURI) { + var appIdAndPort = kubernetesBaseURI.split('/').slice(-1)[0]; + var appId = appIdAndPort.split(':')[0]; + cb(appId); + return; + } + var words = document.baseURI.split('/'); var ind = words.indexOf("proxy"); if (ind > 0) { @@ -95,6 +124,11 @@ function getStandAloneppId(cb) { } function createRESTEndPoint(appId) { + var kubernetesBaseURI = findKubernetesServiceBaseURI(); + if (kubernetesBaseURI) { + return kubernetesBaseURI + "/api/v1/applications/" + appId + "/allexecutors"; + } + var words = document.baseURI.split('/'); var ind = words.indexOf("proxy"); if (ind > 0) { From a34a11416d5891bdc4a102c97aed4760d70b3ecd Mon Sep 17 00:00:00 2001 From: Anirudh Ramanathan Date: Mon, 13 Feb 2017 12:49:16 -0800 Subject: [PATCH 48/81] Filter nodes to only try and send files to external IPs (#106) * Filter node addresses * Added comment --- .../org/apache/spark/deploy/kubernetes/Client.scala | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/Client.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/Client.scala index 9eed9bfd2cd79..d3aa515484f78 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/Client.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/Client.scala @@ -745,9 +745,14 @@ private[spark] class Client( val nodeUrls = kubernetesClient.nodes.list.getItems.asScala .filterNot(node => node.getSpec.getUnschedulable != null && node.getSpec.getUnschedulable) - .flatMap(_.getStatus.getAddresses.asScala.map(address => { + .flatMap(_.getStatus.getAddresses.asScala) + // The list contains hostnames, internal and external IP addresses. + // we want only external IP addresses in our list + // (https://kubernetes.io/docs/admin/node/#addresses) + .filter(_.getType == "ExternalIP") + .map(address => { s"$urlScheme://${address.getAddress}:$servicePort" - })).toSet + }).toSet require(nodeUrls.nonEmpty, "No nodes found to contact the driver!") val (trustManager, sslContext): (X509TrustManager, SSLContext) = if (driverSubmitSslOptions.enabled) { From ac4dd917326dad358568549f3d60fa0c91c86d85 Mon Sep 17 00:00:00 2001 From: Andrew Ash Date: Mon, 13 Feb 2017 23:18:01 +0000 Subject: [PATCH 49/81] Parse results of minikube status more rigorously (#97) * Parse results of minikube status more rigorously Prior code assumes the minikubeVM status line is always the first row output from minikube status, and it is not when the version upgrade notifier prints an upgrade suggestion message. * Also filter ip response to expected rows --- .../kubernetes/integrationtest/minikube/Minikube.scala | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/minikube/Minikube.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/minikube/Minikube.scala index 736b92cc2d628..e7eea679adf79 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/minikube/Minikube.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/minikube/Minikube.scala @@ -19,6 +19,7 @@ package org.apache.spark.deploy.kubernetes.integrationtest.minikube import java.io.{BufferedReader, InputStreamReader} import java.nio.file.Paths import java.util.concurrent.TimeUnit +import java.util.regex.Pattern import javax.net.ssl.X509TrustManager import io.fabric8.kubernetes.client.{ConfigBuilder, DefaultKubernetesClient} @@ -58,13 +59,17 @@ private[spark] object Minikube extends Logging { def getMinikubeIp: String = synchronized { assert(MINIKUBE_EXECUTABLE_DEST.exists(), EXPECTED_DOWNLOADED_MINIKUBE_MESSAGE) val outputs = executeMinikube("ip") + .filter(_.matches("^\\d{1,3}\\.\\d{1,3}\\.\\d{1,3}\\.\\d{1,3}$")) assert(outputs.size == 1, "Unexpected amount of output from minikube ip") outputs.head } def getMinikubeStatus: MinikubeStatus.Value = synchronized { assert(MINIKUBE_EXECUTABLE_DEST.exists(), EXPECTED_DOWNLOADED_MINIKUBE_MESSAGE) - val statusString = executeMinikube("status").head.replaceFirst("minikubeVM: ", "") + val statusString = executeMinikube("status") + .filter(_.contains("minikubeVM: ")) + .head + .replaceFirst("minikubeVM: ", "") MinikubeStatus.unapply(statusString) .getOrElse(throw new IllegalStateException(s"Unknown status $statusString")) } From 2112c4a4e76ec594bec2e5e47e7ab5f3d03b7a64 Mon Sep 17 00:00:00 2001 From: Anirudh Ramanathan Date: Tue, 14 Feb 2017 14:32:54 -0800 Subject: [PATCH 50/81] Adding legacyHostIP to the list of IPs we look at (#114) --- .../scala/org/apache/spark/deploy/kubernetes/Client.scala | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/Client.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/Client.scala index d3aa515484f78..279ee505de609 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/Client.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/Client.scala @@ -747,9 +747,11 @@ private[spark] class Client( node.getSpec.getUnschedulable) .flatMap(_.getStatus.getAddresses.asScala) // The list contains hostnames, internal and external IP addresses. - // we want only external IP addresses in our list // (https://kubernetes.io/docs/admin/node/#addresses) - .filter(_.getType == "ExternalIP") + // we want only external IP addresses and legacyHostIP addresses in our list + // legacyHostIPs are deprecated and will be removed in the future. + // (https://github.com/kubernetes/kubernetes/issues/9267) + .filter(address => address.getType == "ExternalIP" || address.getType == "LegacyHostIP") .map(address => { s"$urlScheme://${address.getAddress}:$servicePort" }).toSet From 043cdd9677e4c7a88e415029e4c988a58375429f Mon Sep 17 00:00:00 2001 From: Andrew Ash Date: Wed, 15 Feb 2017 01:29:31 +0000 Subject: [PATCH 51/81] Add -DskipTests to dev docs (#115) * Add -DskipTests to dev docs * Remove extraneous skipTests --- resource-managers/kubernetes/README.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/resource-managers/kubernetes/README.md b/resource-managers/kubernetes/README.md index 92ec305513f42..25b62ba35a193 100644 --- a/resource-managers/kubernetes/README.md +++ b/resource-managers/kubernetes/README.md @@ -17,7 +17,7 @@ important matters to keep in mind when developing this feature. To build Spark with Kubernetes support, use the `kubernetes` profile when invoking Maven. For example, to simply compile the Kubernetes core implementation module along with its dependencies: - build/mvn compile -Pkubernetes -pl resource-managers/kubernetes/core -am + build/mvn compile -Pkubernetes -pl resource-managers/kubernetes/core -am -DskipTests To build a distribution of Spark with Kubernetes support, use the `dev/make-distribution.sh` script, and add the `kubernetes` profile as part of the build arguments. Any other build arguments can be specified as one would expect when @@ -47,7 +47,7 @@ Running any of the integration tests requires including `kubernetes-integration- order to prepare the environment for running the integration tests, the `pre-integration-test` step must be run in Maven on the `resource-managers/kubernetes/integration-tests` module: - build/mvn pre-integration-test -Pkubernetes -Pkubernetes-integration-tests -pl resource-managers/kubernetes/integration-tests -am + build/mvn pre-integration-test -Pkubernetes -Pkubernetes-integration-tests -pl resource-managers/kubernetes/integration-tests -am -DskipTests Afterwards, the integration tests can be executed with Maven or your IDE. Note that when running tests from an IDE, the `pre-integration-test` phase must be run every time the Spark main code changes. When running tests from the From 0e6df1145d9ede8c18612d9d9096e2fdd4274879 Mon Sep 17 00:00:00 2001 From: Varun Date: Wed, 15 Feb 2017 16:38:46 -0800 Subject: [PATCH 52/81] Shutdown the thread scheduler in LoggingPodStatusWatcher on receiving job finish event notifications (#121) --- .../apache/spark/deploy/kubernetes/LoggingPodStatusWatcher.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/LoggingPodStatusWatcher.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/LoggingPodStatusWatcher.scala index cbacaf6bda854..b7a29fedcbd2d 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/LoggingPodStatusWatcher.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/LoggingPodStatusWatcher.scala @@ -64,6 +64,7 @@ private[kubernetes] class LoggingPodStatusWatcher(podCompletedFuture: CountDownL if (phase == "Succeeded" || phase == "Failed") { podCompletedFuture.countDown() + scheduler.shutdown() } } From a800e20950c47fe03a8d0d06c983501debb72f8c Mon Sep 17 00:00:00 2001 From: Kimoon Kim Date: Wed, 15 Feb 2017 17:22:36 -0800 Subject: [PATCH 53/81] Trigger scalatest plugin in the integration-test phase (#93) * Trigger scalatest plugin in the integration-test phase * Clean up unnecessary config section --- .../kubernetes/integration-tests/pom.xml | 27 +++++++++++++++++++ 1 file changed, 27 insertions(+) diff --git a/resource-managers/kubernetes/integration-tests/pom.xml b/resource-managers/kubernetes/integration-tests/pom.xml index f6a322f18cd75..3de10f94c4aca 100644 --- a/resource-managers/kubernetes/integration-tests/pom.xml +++ b/resource-managers/kubernetes/integration-tests/pom.xml @@ -216,6 +216,33 @@ + + + org.scalatest + scalatest-maven-plugin + + + test + + test + + + + (?<!Suite) + + + + integration-test + integration-test + + test + + + +
    From 2773b778b55f08f6279eae645a919ab140de585f Mon Sep 17 00:00:00 2001 From: Anirudh Ramanathan Date: Wed, 15 Feb 2017 17:56:35 -0800 Subject: [PATCH 54/81] Fix issue with DNS resolution (#118) * Fix issue with DNS resolution * Address comments --- .../spark/deploy/kubernetes/KubernetesClientBuilder.scala | 5 +++-- .../scala/org/apache/spark/deploy/kubernetes/constants.scala | 1 + .../kubernetes/KubernetesClusterSchedulerBackend.scala | 3 +-- .../kubernetes/integrationtest/minikube/Minikube.scala | 2 +- 4 files changed, 6 insertions(+), 5 deletions(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/KubernetesClientBuilder.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/KubernetesClientBuilder.scala index 61d3ac17ac34a..89369b30694ee 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/KubernetesClientBuilder.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/KubernetesClientBuilder.scala @@ -22,6 +22,8 @@ import com.google.common.base.Charsets import com.google.common.io.Files import io.fabric8.kubernetes.client.{Config, ConfigBuilder, DefaultKubernetesClient} +import org.apache.spark.deploy.kubernetes.constants._ + private[spark] object KubernetesClientBuilder { private val API_SERVER_TOKEN = new File(Config.KUBERNETES_SERVICE_ACCOUNT_TOKEN_PATH) private val CA_CERT_FILE = new File(Config.KUBERNETES_SERVICE_ACCOUNT_CA_CRT_PATH) @@ -33,11 +35,10 @@ private[spark] object KubernetesClientBuilder { * into the pod's disk space. */ def buildFromWithinPod( - kubernetesMaster: String, kubernetesNamespace: String): DefaultKubernetesClient = { var clientConfigBuilder = new ConfigBuilder() .withApiVersion("v1") - .withMasterUrl(kubernetesMaster) + .withMasterUrl(KUBERNETES_MASTER_INTERNAL_URL) .withNamespace(kubernetesNamespace) if (CA_CERT_FILE.isFile) { diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/constants.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/constants.scala index 027cc3c022b4e..688cd858e79ff 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/constants.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/constants.scala @@ -67,4 +67,5 @@ package object constants { // Miscellaneous private[spark] val DRIVER_CONTAINER_NAME = "spark-kubernetes-driver" private[spark] val KUBERNETES_SUBMIT_SSL_NAMESPACE = "kubernetes.submit" + private[spark] val KUBERNETES_MASTER_INTERNAL_URL = "https://kubernetes.default.svc" } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala index d4e7da464be4a..898b215b92d04 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala @@ -44,7 +44,6 @@ private[spark] class KubernetesClusterSchedulerBackend( private val EXECUTOR_MODIFICATION_LOCK = new Object private val runningExecutorPods = new scala.collection.mutable.HashMap[String, Pod] - private val kubernetesMaster = "https://kubernetes" private val executorDockerImage = conf.get(EXECUTOR_DOCKER_IMAGE) private val kubernetesNamespace = conf.get(KUBERNETES_NAMESPACE) private val executorPort = conf.getInt("spark.executor.port", DEFAULT_STATIC_PORT) @@ -77,7 +76,7 @@ private[spark] class KubernetesClusterSchedulerBackend( ThreadUtils.newDaemonCachedThreadPool("kubernetes-executor-requests")) private val kubernetesClient = KubernetesClientBuilder - .buildFromWithinPod(kubernetesMaster, kubernetesNamespace) + .buildFromWithinPod(kubernetesNamespace) private val driverPod = try { kubernetesClient.pods().inNamespace(kubernetesNamespace). diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/minikube/Minikube.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/minikube/Minikube.scala index e7eea679adf79..07274bf962dde 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/minikube/Minikube.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/minikube/Minikube.scala @@ -76,7 +76,7 @@ private[spark] object Minikube extends Logging { def getDockerEnv: Map[String, String] = synchronized { assert(MINIKUBE_EXECUTABLE_DEST.exists(), EXPECTED_DOWNLOADED_MINIKUBE_MESSAGE) - executeMinikube("docker-env") + executeMinikube("docker-env", "--shell", "bash") .filter(_.startsWith("export")) .map(_.replaceFirst("export ", "").split('=')) .map(arr => (arr(0), arr(1).replaceAllLiterally("\"", ""))) From 6a999cab598edc5fd87974a74f460c131885d012 Mon Sep 17 00:00:00 2001 From: mccheah Date: Thu, 16 Feb 2017 15:28:42 -0800 Subject: [PATCH 55/81] Change the API contract for uploading local files (#107) * Change the API contract for uploading local jars. This mirrors similarly to what YARN and Mesos expects. * Address comments * Fix test --- .../org/apache/spark/deploy/SparkSubmit.scala | 9 +- .../spark/deploy/SparkSubmitArguments.scala | 14 -- docs/running-on-kubernetes.md | 108 ++-------------- .../launcher/SparkSubmitOptionParser.java | 8 +- .../spark/deploy/kubernetes/Client.scala | 121 ++++++------------ .../spark/deploy/kubernetes/config.scala | 24 ---- .../rest/KubernetesRestProtocolMessages.scala | 4 +- .../rest/kubernetes/KubernetesFileUtils.scala | 44 +++++++ .../KubernetesSparkRestServer.scala | 115 +++++++++++------ .../kubernetes/docker-minimal-bundle/pom.xml | 6 - .../src/main/assembly/driver-assembly.xml | 11 -- .../src/main/assembly/executor-assembly.xml | 11 -- .../kubernetes/integration-tests/pom.xml | 50 ++++++++ .../integrationtest/KubernetesSuite.scala | 87 +++---------- 14 files changed, 244 insertions(+), 368 deletions(-) create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/KubernetesFileUtils.scala 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 51eb23560defe..002b29d5564e1 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -472,10 +472,6 @@ object SparkSubmit { OptionAssigner(args.kubernetesNamespace, KUBERNETES, ALL_DEPLOY_MODES, sysProp = "spark.kubernetes.namespace"), - OptionAssigner(args.kubernetesUploadJars, KUBERNETES, CLUSTER, - sysProp = "spark.kubernetes.driver.uploads.jars"), - OptionAssigner(args.kubernetesUploadFiles, KUBERNETES, CLUSTER, - sysProp = "spark.kubernetes.driver.uploads.files"), // Other options OptionAssigner(args.executorCores, STANDALONE | YARN, ALL_DEPLOY_MODES, @@ -484,10 +480,11 @@ object SparkSubmit { sysProp = "spark.executor.memory"), OptionAssigner(args.totalExecutorCores, STANDALONE | MESOS, ALL_DEPLOY_MODES, sysProp = "spark.cores.max"), - OptionAssigner(args.files, LOCAL | STANDALONE | MESOS, ALL_DEPLOY_MODES, + OptionAssigner(args.files, LOCAL | STANDALONE | MESOS | KUBERNETES, ALL_DEPLOY_MODES, sysProp = "spark.files"), OptionAssigner(args.jars, LOCAL, CLIENT, sysProp = "spark.jars"), - OptionAssigner(args.jars, STANDALONE | MESOS, ALL_DEPLOY_MODES, sysProp = "spark.jars"), + OptionAssigner(args.jars, STANDALONE | MESOS | KUBERNETES, ALL_DEPLOY_MODES, + sysProp = "spark.jars"), OptionAssigner(args.driverMemory, STANDALONE | MESOS | YARN, CLUSTER, sysProp = "spark.driver.memory"), OptionAssigner(args.driverCores, STANDALONE | MESOS | YARN, CLUSTER, diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala index f771755244f31..4e297fe3b0e3b 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -73,8 +73,6 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S // Kubernetes only var kubernetesNamespace: String = null - var kubernetesUploadJars: String = null - var kubernetesUploadFiles: String = null // Standalone cluster mode only var supervise: Boolean = false @@ -194,12 +192,6 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S kubernetesNamespace = Option(kubernetesNamespace) .orElse(sparkProperties.get("spark.kubernetes.namespace")) .orNull - kubernetesUploadJars = Option(kubernetesUploadJars) - .orElse(sparkProperties.get("spark.kubernetes.driver.uploads.jars")) - .orNull - kubernetesUploadFiles = Option(kubernetesUploadFiles) - .orElse(sparkProperties.get("spark.kubernetes.driver.uploads.files")) - .orNull // Try to set main class from JAR if no --class argument is given if (mainClass == null && !isPython && !isR && primaryResource != null) { @@ -443,12 +435,6 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S case KUBERNETES_NAMESPACE => kubernetesNamespace = value - case KUBERNETES_UPLOAD_JARS => - kubernetesUploadJars = value - - case KUBERNETES_UPLOAD_FILES => - kubernetesUploadFiles = value - case HELP => printUsageAndExit(0) diff --git a/docs/running-on-kubernetes.md b/docs/running-on-kubernetes.md index 19f406039e261..e5c7e9bb69448 100644 --- a/docs/running-on-kubernetes.md +++ b/docs/running-on-kubernetes.md @@ -51,87 +51,15 @@ connect without SSL on a different port, the master would be set to `k8s://http: Note that applications can currently only be executed in cluster mode, where the driver and its executors are running on the cluster. - -### Adding Other JARs - -Spark allows users to provide dependencies that are bundled into the driver's Docker image, or that are on the local -disk of the submitter's machine. These two types of dependencies are specified via different configuration options to -`spark-submit`: -* Local jars provided by specifying the `--jars` command line argument to `spark-submit`, or by setting `spark.jars` in - the application's configuration, will be treated as jars that are located on the *disk of the driver container*. This - only applies to jar paths that do not specify a scheme or that have the scheme `file://`. Paths with other schemes are - fetched from their appropriate locations. -* Local jars provided by specifying the `--upload-jars` command line argument to `spark-submit`, or by setting - `spark.kubernetes.driver.uploads.jars` in the application's configuration, will be treated as jars that are located on - the *disk of the submitting machine*. These jars are uploaded to the driver docker container before executing the - application. -* A main application resource path that does not have a scheme or that has the scheme `file://` is assumed to be on the - *disk of the submitting machine*. This resource is uploaded to the driver docker container before executing the - application. A remote path can still be specified and the resource will be fetched from the appropriate location. -* A main application resource path that has the scheme `container://` is assumed to be on the *disk of the driver - container*. - -In all of these cases, the jars are placed on the driver's classpath, and are also sent to the executors. Below are some -examples of providing application dependencies. - -To submit an application with both the main resource and two other jars living on the submitting user's machine: - - bin/spark-submit \ - --deploy-mode cluster \ - --class com.example.applications.SampleApplication \ - --master k8s://192.168.99.100 \ - --upload-jars /home/exampleuser/exampleapplication/dep1.jar,/home/exampleuser/exampleapplication/dep2.jar \ - --conf spark.kubernetes.driver.docker.image=registry-host:5000/spark-driver:latest \ - --conf spark.kubernetes.executor.docker.image=registry-host:5000/spark-executor:latest \ - /home/exampleuser/exampleapplication/main.jar - -Note that since passing the jars through the `--upload-jars` command line argument is equivalent to setting the -`spark.kubernetes.driver.uploads.jars` Spark property, the above will behave identically to this command: - - bin/spark-submit \ - --deploy-mode cluster \ - --class com.example.applications.SampleApplication \ - --master k8s://192.168.99.100 \ - --conf spark.kubernetes.driver.uploads.jars=/home/exampleuser/exampleapplication/dep1.jar,/home/exampleuser/exampleapplication/dep2.jar \ - --conf spark.kubernetes.driver.docker.image=registry-host:5000/spark-driver:latest \ - --conf spark.kubernetes.executor.docker.image=registry-host:5000/spark-executor:latest \ - /home/exampleuser/exampleapplication/main.jar - -To specify a main application resource that can be downloaded from an HTTP service, and if a plugin for that application -is located in the jar `/opt/spark-plugins/app-plugin.jar` on the docker image's disk: - - bin/spark-submit \ - --deploy-mode cluster \ - --class com.example.applications.PluggableApplication \ - --master k8s://192.168.99.100 \ - --jars /opt/spark-plugins/app-plugin.jar \ - --conf spark.kubernetes.driver.docker.image=registry-host:5000/spark-driver-custom:latest \ - --conf spark.kubernetes.executor.docker.image=registry-host:5000/spark-executor:latest \ - http://example.com:8080/applications/sparkpluggable/app.jar - -Note that since passing the jars through the `--jars` command line argument is equivalent to setting the `spark.jars` -Spark property, the above will behave identically to this command: - - bin/spark-submit \ - --deploy-mode cluster \ - --class com.example.applications.PluggableApplication \ - --master k8s://192.168.99.100 \ - --conf spark.jars=file:///opt/spark-plugins/app-plugin.jar \ - --conf spark.kubernetes.driver.docker.image=registry-host:5000/spark-driver-custom:latest \ - --conf spark.kubernetes.executor.docker.image=registry-host:5000/spark-executor:latest \ - http://example.com:8080/applications/sparkpluggable/app.jar - -To specify a main application resource that is in the Docker image, and if it has no other dependencies: - - bin/spark-submit \ - --deploy-mode cluster \ - --class com.example.applications.PluggableApplication \ - --master k8s://192.168.99.100:8443 \ - --conf spark.kubernetes.driver.docker.image=registry-host:5000/spark-driver-custom:latest \ - --conf spark.kubernetes.executor.docker.image=registry-host:5000/spark-executor:latest \ - container:///home/applications/examples/example.jar +### Dependency Management and Docker Containers +Spark supports specifying JAR paths that are either on the submitting host's disk, or are located on the disk of the +driver and executors. Refer to the [application submission](submitting-applications.html#advanced-dependency-management) +section for details. Note that files specified with the `local` scheme should be added to the container image of both +the driver and the executors. Files without a scheme or with the scheme `file://` are treated as being on the disk of +the submitting machine, and are uploaded to the driver running in Kubernetes before launching the application. + ### Setting Up SSL For Submitting the Driver When submitting to Kubernetes, a pod is started for the driver, and the pod starts an HTTP server. This HTTP server @@ -146,9 +74,9 @@ pod in starting the application, set `spark.ssl.kubernetes.submit.trustStore`. One note about the keyStore is that it can be specified as either a file on the client machine or a file in the container image's disk. Thus `spark.ssl.kubernetes.submit.keyStore` can be a URI with a scheme of either `file:` -or `container:`. A scheme of `file:` corresponds to the keyStore being located on the client machine; it is mounted onto +or `local:`. A scheme of `file:` corresponds to the keyStore being located on the client machine; it is mounted onto the driver container as a [secret volume](https://kubernetes.io/docs/user-guide/secrets/). When the URI has the scheme -`container:`, the file is assumed to already be on the container's disk at the appropriate path. +`local:`, the file is assumed to already be on the container's disk at the appropriate path. ### Kubernetes Clusters and the authenticated proxy endpoint @@ -241,24 +169,6 @@ from the other deployment modes. See the [configuration page](configuration.html executor pods from the API server. - - spark.kubernetes.driver.uploads.jars - (none) - - Comma-separated list of jars to send to the driver and all executors when submitting the application in cluster - mode. Refer to adding other jars for more information. - - - - spark.kubernetes.driver.uploads.files - (none) - - Comma-separated list of files to send to the driver and all executors when submitting the application in cluster - mode. The files are added in a flat hierarchy to the current working directory of the driver, having the same - names as the names of the original files. Note that two files with the same name cannot be added, even if they - were in different source directories on the client disk. - - spark.kubernetes.executor.memoryOverhead executorMemory * 0.10, with minimum of 384 diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitOptionParser.java b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitOptionParser.java index 3369b5d8301be..a4d43c0795abc 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitOptionParser.java +++ b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitOptionParser.java @@ -77,10 +77,7 @@ class SparkSubmitOptionParser { protected final String QUEUE = "--queue"; // Kubernetes-only options. - protected final String KUBERNETES_MASTER = "--kubernetes-master"; protected final String KUBERNETES_NAMESPACE = "--kubernetes-namespace"; - protected final String KUBERNETES_UPLOAD_JARS = "--upload-jars"; - protected final String KUBERNETES_UPLOAD_FILES = "--upload-files"; /** * This is the canonical list of spark-submit options. Each entry in the array contains the @@ -121,10 +118,7 @@ class SparkSubmitOptionParser { { REPOSITORIES }, { STATUS }, { TOTAL_EXECUTOR_CORES }, - { KUBERNETES_MASTER }, - { KUBERNETES_NAMESPACE }, - { KUBERNETES_UPLOAD_JARS }, - { KUBERNETES_UPLOAD_FILES } + { KUBERNETES_NAMESPACE } }; /** diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/Client.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/Client.scala index 279ee505de609..aa273a024f6f9 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/Client.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/Client.scala @@ -35,7 +35,7 @@ import scala.collection.mutable import org.apache.spark.{SecurityManager, SparkConf, SparkException, SSLOptions} import org.apache.spark.deploy.kubernetes.config._ import org.apache.spark.deploy.kubernetes.constants._ -import org.apache.spark.deploy.rest.{AppResource, ContainerAppResource, KubernetesCreateSubmissionRequest, RemoteAppResource, TarGzippedData, UploadedAppResource} +import org.apache.spark.deploy.rest.{AppResource, ContainerAppResource, KubernetesCreateSubmissionRequest, RemoteAppResource, UploadedAppResource} import org.apache.spark.deploy.rest.kubernetes._ import org.apache.spark.internal.Logging import org.apache.spark.util.Utils @@ -59,11 +59,10 @@ private[spark] class Client( private val sslSecretsDirectory = s"$DRIVER_CONTAINER_SECRETS_BASE_DIR/$kubernetesAppId-ssl" private val sslSecretsName = s"$SUBMISSION_SSL_SECRETS_PREFIX-$kubernetesAppId" private val driverDockerImage = sparkConf.get(DRIVER_DOCKER_IMAGE) - private val uploadedJars = sparkConf.get(KUBERNETES_DRIVER_UPLOAD_JARS).filter(_.nonEmpty) - private val uploadedFiles = sparkConf.get(KUBERNETES_DRIVER_UPLOAD_FILES).filter(_.nonEmpty) - uploadedFiles.foreach(validateNoDuplicateUploadFileNames) private val uiPort = sparkConf.getInt("spark.ui.port", DEFAULT_UI_PORT) private val driverSubmitTimeoutSecs = sparkConf.get(KUBERNETES_DRIVER_SUBMIT_TIMEOUT) + private val sparkFiles = sparkConf.getOption("spark.files") + private val sparkJars = sparkConf.getOption("spark.jars") private val waitForAppCompletion: Boolean = sparkConf.get(WAIT_FOR_APP_COMPLETION) @@ -78,9 +77,18 @@ private[spark] class Client( def run(): Unit = { logInfo(s"Starting application $kubernetesAppId in Kubernetes...") - - Seq(uploadedFiles, uploadedJars, Some(mainAppResource)).foreach(checkForFilesExistence) - + val submitterLocalFiles = KubernetesFileUtils.getOnlySubmitterLocalFiles(sparkFiles) + val submitterLocalJars = KubernetesFileUtils.getOnlySubmitterLocalFiles(sparkJars) + (submitterLocalFiles ++ submitterLocalJars).foreach { file => + if (!new File(Utils.resolveURI(file).getPath).isFile) { + throw new SparkException(s"File $file does not exist or is a directory.") + } + } + if (KubernetesFileUtils.isUriLocalFile(mainAppResource) && + !new File(Utils.resolveURI(mainAppResource).getPath).isFile) { + throw new SparkException(s"Main app resource file $mainAppResource is not a file or" + + s" is a directory.") + } val (driverSubmitSslOptions, isKeyStoreLocalFile) = parseDriverSubmitSslOptions() val parsedCustomLabels = parseCustomLabels(customLabels) var k8ConfBuilder = new K8SConfigBuilder() @@ -145,7 +153,7 @@ private[spark] class Client( } try { submitApplicationToDriverServer(kubernetesClient, driverSubmitSslOptions, - ownerReferenceConfiguredDriverService) + ownerReferenceConfiguredDriverService, submitterLocalFiles, submitterLocalJars) // wait if configured to do so if (waitForAppCompletion) { logInfo(s"Waiting for application $kubernetesAppId to finish...") @@ -193,7 +201,9 @@ private[spark] class Client( private def submitApplicationToDriverServer( kubernetesClient: KubernetesClient, driverSubmitSslOptions: SSLOptions, - driverService: Service) = { + driverService: Service, + submitterLocalFiles: Iterable[String], + submitterLocalJars: Iterable[String]): Unit = { sparkConf.getOption("spark.app.id").foreach { id => logWarning(s"Warning: Provided app id in spark.app.id as $id will be" + s" overridden as $kubernetesAppId") @@ -211,7 +221,7 @@ private[spark] class Client( driverSubmitter.ping() logInfo(s"Submitting local resources to driver pod for application " + s"$kubernetesAppId ...") - val submitRequest = buildSubmissionRequest() + val submitRequest = buildSubmissionRequest(submitterLocalFiles, submitterLocalJars) driverSubmitter.submitApplication(submitRequest) logInfo("Successfully submitted local resources and driver configuration to" + " driver pod.") @@ -502,25 +512,18 @@ private[spark] class Client( val maybeKeyStore = sparkConf.get(KUBERNETES_DRIVER_SUBMIT_KEYSTORE) val resolvedSparkConf = sparkConf.clone() val (isLocalKeyStore, resolvedKeyStore) = maybeKeyStore.map(keyStore => { - val keyStoreURI = Utils.resolveURI(keyStore) - val isProvidedKeyStoreLocal = keyStoreURI.getScheme match { - case "file" | null => true - case "container" => false - case _ => throw new SparkException(s"Invalid KeyStore URI $keyStore; keyStore URI" + - " for submit server must have scheme file:// or container:// (no scheme defaults" + - " to file://)") - } - (isProvidedKeyStoreLocal, Option.apply(keyStoreURI.getPath)) - }).getOrElse((true, Option.empty[String])) + (KubernetesFileUtils.isUriLocalFile(keyStore), + Option.apply(Utils.resolveURI(keyStore).getPath)) + }).getOrElse((false, Option.empty[String])) resolvedKeyStore.foreach { resolvedSparkConf.set(KUBERNETES_DRIVER_SUBMIT_KEYSTORE, _) } sparkConf.get(KUBERNETES_DRIVER_SUBMIT_TRUSTSTORE).foreach { trustStore => - val trustStoreURI = Utils.resolveURI(trustStore) - trustStoreURI.getScheme match { - case "file" | null => - resolvedSparkConf.set(KUBERNETES_DRIVER_SUBMIT_TRUSTSTORE, trustStoreURI.getPath) - case _ => throw new SparkException(s"Invalid trustStore URI $trustStore; trustStore URI" + + if (KubernetesFileUtils.isUriLocalFile(trustStore)) { + resolvedSparkConf.set(KUBERNETES_DRIVER_SUBMIT_TRUSTSTORE, + Utils.resolveURI(trustStore).getPath) + } else { + throw new SparkException(s"Invalid trustStore URI $trustStore; trustStore URI" + " for submit server must have no scheme, or scheme file://") } } @@ -673,23 +676,24 @@ private[spark] class Client( .build()) } - private def buildSubmissionRequest(): KubernetesCreateSubmissionRequest = { - val appResourceUri = Utils.resolveURI(mainAppResource) - val resolvedAppResource: AppResource = appResourceUri.getScheme match { - case "file" | null => - val appFile = new File(appResourceUri.getPath) - if (!appFile.isFile) { - throw new IllegalStateException("Provided local file path does not exist" + - s" or is not a file: ${appFile.getAbsolutePath}") - } + private def buildSubmissionRequest( + submitterLocalFiles: Iterable[String], + submitterLocalJars: Iterable[String]): KubernetesCreateSubmissionRequest = { + val mainResourceUri = Utils.resolveURI(mainAppResource) + val resolvedAppResource: AppResource = Option(mainResourceUri.getScheme) + .getOrElse("file") match { + case "file" => + val appFile = new File(mainResourceUri.getPath) val fileBytes = Files.toByteArray(appFile) val fileBase64 = Base64.encodeBase64String(fileBytes) UploadedAppResource(resourceBase64Contents = fileBase64, name = appFile.getName) - case "container" => ContainerAppResource(appResourceUri.getPath) + case "local" => ContainerAppResource(mainAppResource) case other => RemoteAppResource(other) } - val uploadJarsBase64Contents = compressFiles(uploadedJars) - val uploadFilesBase64Contents = compressFiles(uploadedFiles) + val uploadFilesBase64Contents = CompressionUtils.createTarGzip(submitterLocalFiles.map( + Utils.resolveURI(_).getPath)) + val uploadJarsBase64Contents = CompressionUtils.createTarGzip(submitterLocalJars.map( + Utils.resolveURI(_).getPath)) KubernetesCreateSubmissionRequest( appResource = resolvedAppResource, mainClass = mainClass, @@ -700,33 +704,6 @@ private[spark] class Client( uploadedFilesBase64Contents = uploadFilesBase64Contents) } - // Because uploaded files should be added to the working directory of the driver, they - // need to not have duplicate file names. They are added to the working directory so the - // user can reliably locate them in their application. This is similar in principle to how - // YARN handles its `spark.files` setting. - private def validateNoDuplicateUploadFileNames(uploadedFilesCommaSeparated: String): Unit = { - val pathsWithDuplicateNames = uploadedFilesCommaSeparated - .split(",") - .groupBy(new File(_).getName) - .filter(_._2.length > 1) - if (pathsWithDuplicateNames.nonEmpty) { - val pathsWithDuplicateNamesSorted = pathsWithDuplicateNames - .values - .flatten - .toList - .sortBy(new File(_).getName) - throw new SparkException("Cannot upload files with duplicate names via" + - s" ${KUBERNETES_DRIVER_UPLOAD_FILES.key}. The following paths have a duplicated" + - s" file name: ${pathsWithDuplicateNamesSorted.mkString(",")}") - } - } - - private def compressFiles(maybeFilePaths: Option[String]): Option[TarGzippedData] = { - maybeFilePaths - .map(_.split(",")) - .map(CompressionUtils.createTarGzip(_)) - } - private def buildDriverSubmissionClient( kubernetesClient: KubernetesClient, service: Service, @@ -813,22 +790,6 @@ private[spark] class Client( }).toMap }).getOrElse(Map.empty[String, String]) } - - private def checkForFilesExistence(maybePaths: Option[String]): Unit = { - maybePaths.foreach { paths => - paths.split(",").foreach { path => - val uri = Utils.resolveURI(path) - uri.getScheme match { - case "file" | null => - val file = new File(uri.getPath) - if (!file.isFile) { - throw new SparkException(s"""file "${uri}" does not exist!""") - } - case _ => - } - } - } - } } private[spark] object Client extends Logging { diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/config.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/config.scala index cb4cd42142ca4..ad83b0446538e 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/config.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/config.scala @@ -91,30 +91,6 @@ package object config { .stringConf .createWithDefault("default") - private[spark] val KUBERNETES_DRIVER_UPLOAD_JARS = - ConfigBuilder("spark.kubernetes.driver.uploads.jars") - .doc(""" - | Comma-separated list of jars to send to the driver and - | all executors when submitting the application in cluster - | mode. - """.stripMargin) - .stringConf - .createOptional - - private[spark] val KUBERNETES_DRIVER_UPLOAD_FILES = - ConfigBuilder("spark.kubernetes.driver.uploads.files") - .doc(""" - | Comma-separated list of files to send to the driver and - | all executors when submitting the application in cluster - | mode. The files are added in a flat hierarchy to the - | current working directory of the driver, having the same - | names as the names of the original files. Note that two - | files with the same name cannot be added, even if they - | were in different source directories on the client disk. - """.stripMargin) - .stringConf - .createOptional - // Note that while we set a default for this when we start up the // scheduler, the specific default value is dynamically determined // based on the executor memory. diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/KubernetesRestProtocolMessages.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/KubernetesRestProtocolMessages.scala index 6aeb851a16bf4..0d2d1a1c6f5e3 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/KubernetesRestProtocolMessages.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/KubernetesRestProtocolMessages.scala @@ -26,8 +26,8 @@ case class KubernetesCreateSubmissionRequest( appArgs: Array[String], sparkProperties: Map[String, String], secret: String, - uploadedJarsBase64Contents: Option[TarGzippedData], - uploadedFilesBase64Contents: Option[TarGzippedData]) extends SubmitRestProtocolRequest { + uploadedJarsBase64Contents: TarGzippedData, + uploadedFilesBase64Contents: TarGzippedData) extends SubmitRestProtocolRequest { message = "create" clientSparkVersion = SPARK_VERSION } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/KubernetesFileUtils.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/KubernetesFileUtils.scala new file mode 100644 index 0000000000000..f30be1535f81c --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/KubernetesFileUtils.scala @@ -0,0 +1,44 @@ +/* + * 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.rest.kubernetes + +import org.apache.spark.util.Utils + +private[spark] object KubernetesFileUtils { + + private def filterUriStringsByScheme( + uris: Iterable[String], schemeFilter: (String => Boolean)): Iterable[String] = { + uris.filter(uri => schemeFilter(Option(Utils.resolveURI(uri).getScheme).getOrElse("file"))) + } + + def getNonSubmitterLocalFiles(uris: Iterable[String]): Iterable[String] = { + filterUriStringsByScheme(uris, _ != "file") + } + + def getOnlyContainerLocalFiles(uris: Iterable[String]): Iterable[String] = { + filterUriStringsByScheme(uris, _ == "local") + } + + def getOnlySubmitterLocalFiles(uris: Iterable[String]): Iterable[String] = { + filterUriStringsByScheme(uris, _ == "file") + } + + def isUriLocalFile(uri: String): Boolean = { + Option(Utils.resolveURI(uri).getScheme).getOrElse("file") == "file" + } + +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/KubernetesSparkRestServer.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/KubernetesSparkRestServer.scala index c5a7e27b15927..f0b01b2320982 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/KubernetesSparkRestServer.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/KubernetesSparkRestServer.scala @@ -16,14 +16,14 @@ */ package org.apache.spark.deploy.rest.kubernetes -import java.io.File +import java.io.{File, FileOutputStream, StringReader} import java.net.URI import java.nio.file.Paths import java.util.concurrent.CountDownLatch import javax.servlet.http.{HttpServletRequest, HttpServletResponse} import com.google.common.base.Charsets -import com.google.common.io.Files +import com.google.common.io.{BaseEncoding, ByteStreams, Files} import org.apache.commons.codec.binary.Base64 import scala.collection.mutable import scala.collection.mutable.ArrayBuffer @@ -145,46 +145,73 @@ private[spark] class KubernetesSparkRestServer( } else { requestMessage match { case KubernetesCreateSubmissionRequest( - appResource, - mainClass, - appArgs, - sparkProperties, - secret, - uploadedJars, - uploadedFiles) => + appResource, + mainClass, + appArgs, + sparkProperties, + secret, + uploadedJars, + uploadedFiles) => val decodedSecret = Base64.decodeBase64(secret) if (!expectedApplicationSecret.sameElements(decodedSecret)) { responseServlet.setStatus(HttpServletResponse.SC_UNAUTHORIZED) handleError("Unauthorized to submit application.") } else { val tempDir = Utils.createTempDir() - val appResourcePath = resolvedAppResource(appResource, tempDir) + val resolvedAppResource = resolveAppResource(appResource, tempDir) val writtenJars = writeUploadedJars(uploadedJars, tempDir) val writtenFiles = writeUploadedFiles(uploadedFiles) val resolvedSparkProperties = new mutable.HashMap[String, String] resolvedSparkProperties ++= sparkProperties - - // Resolve driver classpath and jars val originalJars = sparkProperties.get("spark.jars") .map(_.split(",")) - .getOrElse(Array.empty[String]) - val resolvedJars = writtenJars ++ originalJars ++ Array(appResourcePath) - val sparkJars = new File(sparkHome, "jars").listFiles().map(_.getAbsolutePath) + .getOrElse(Array.empty) + + // The driver at this point has handed us the value of spark.jars verbatim as + // specified in spark-submit. At this point, remove all jars that were local + // to the submitting user's disk, and replace them with the paths that were + // written to disk above. + val onlyContainerLocalOrRemoteJars = KubernetesFileUtils + .getNonSubmitterLocalFiles(originalJars) + val resolvedJars = (writtenJars ++ + onlyContainerLocalOrRemoteJars ++ + Array(resolvedAppResource.sparkJarPath)).toSet + if (resolvedJars.nonEmpty) { + resolvedSparkProperties("spark.jars") = resolvedJars.mkString(",") + } else { + resolvedSparkProperties.remove("spark.jars") + } + + // Determining the driver classpath is similar. It's the combination of: + // - Jars written from uploads + // - Jars in (spark.jars + mainAppResource) that has a "local" prefix + // - spark.driver.extraClasspath + // - Spark core jars from the installation + val sparkCoreJars = new File(sparkHome, "jars").listFiles().map(_.getAbsolutePath) val driverExtraClasspath = sparkProperties .get("spark.driver.extraClassPath") .map(_.split(",")) .getOrElse(Array.empty[String]) + val onlyContainerLocalJars = KubernetesFileUtils + .getOnlyContainerLocalFiles(originalJars) val driverClasspath = driverExtraClasspath ++ - resolvedJars ++ - sparkJars - resolvedSparkProperties("spark.jars") = resolvedJars.mkString(",") + Seq(resolvedAppResource.localPath) ++ + writtenJars ++ + onlyContainerLocalJars ++ + sparkCoreJars - // Resolve spark.files + // Resolve spark.files similarly to spark.jars. val originalFiles = sparkProperties.get("spark.files") .map(_.split(",")) .getOrElse(Array.empty[String]) - val resolvedFiles = originalFiles ++ writtenFiles - resolvedSparkProperties("spark.files") = resolvedFiles.mkString(",") + val onlyContainerLocalOrRemoteFiles = KubernetesFileUtils + .getNonSubmitterLocalFiles(originalFiles) + val resolvedFiles = writtenFiles ++ onlyContainerLocalOrRemoteFiles + if (resolvedFiles.nonEmpty) { + resolvedSparkProperties("spark.files") = resolvedFiles.mkString(",") + } else { + resolvedSparkProperties.remove("spark.files") + } val command = new ArrayBuffer[String] command += javaExecutable @@ -235,35 +262,50 @@ private[spark] class KubernetesSparkRestServer( } } - private def writeUploadedJars(files: Option[TarGzippedData], rootTempDir: File): + private def writeUploadedJars(jars: TarGzippedData, rootTempDir: File): Seq[String] = { val resolvedDirectory = new File(rootTempDir, "jars") if (!resolvedDirectory.mkdir()) { throw new IllegalStateException(s"Failed to create jars dir at " + resolvedDirectory.getAbsolutePath) } - writeBase64ContentsToFiles(files, resolvedDirectory) + CompressionUtils.unpackAndWriteCompressedFiles(jars, resolvedDirectory) } - private def writeUploadedFiles(files: Option[TarGzippedData]): Seq[String] = { + private def writeUploadedFiles(files: TarGzippedData): Seq[String] = { val workingDir = Paths.get("").toFile.getAbsoluteFile - writeBase64ContentsToFiles(files, workingDir) + CompressionUtils.unpackAndWriteCompressedFiles(files, workingDir) } - def resolvedAppResource(appResource: AppResource, tempDir: File): String = { - val appResourcePath = appResource match { + + /** + * Retrieve the path on the driver container where the main app resource is, and what value it + * ought to have in the spark.jars property. The two may be different because for non-local + * dependencies, we have to fetch the resource (if it is not "local") but still want to use + * the full URI in spark.jars. + */ + private def resolveAppResource(appResource: AppResource, tempDir: File): + ResolvedAppResource = { + appResource match { case UploadedAppResource(resourceContentsBase64, resourceName) => val resourceFile = new File(tempDir, resourceName) val resourceFilePath = resourceFile.getAbsolutePath if (resourceFile.createNewFile()) { - val resourceContentsBytes = Base64.decodeBase64(resourceContentsBase64) - Files.write(resourceContentsBytes, resourceFile) - resourceFile.getAbsolutePath + Utils.tryWithResource(new StringReader(resourceContentsBase64)) { reader => + Utils.tryWithResource(new FileOutputStream(resourceFile)) { os => + Utils.tryWithResource(BaseEncoding.base64().decodingStream(reader)) { + decodingStream => + ByteStreams.copy(decodingStream, os) + } + } + } + ResolvedAppResource(resourceFile.getAbsolutePath, resourceFile.getAbsolutePath) } else { throw new IllegalStateException(s"Failed to write main app resource file" + s" to $resourceFilePath") } - case ContainerAppResource(resource) => resource + case ContainerAppResource(resource) => + ResolvedAppResource(Utils.resolveURI(resource).getPath, resource) case RemoteAppResource(resource) => Utils.fetchFile(resource, tempDir, conf, securityManager, SparkHadoopUtil.get.newConfiguration(conf), @@ -275,19 +317,12 @@ private[spark] class KubernetesSparkRestServer( throw new IllegalStateException(s"Main app resource is not a file or" + s" does not exist at $downloadedFilePath") } - downloadedFilePath + ResolvedAppResource(downloadedFilePath, resource) } - appResourcePath } } - private def writeBase64ContentsToFiles( - maybeCompressedFiles: Option[TarGzippedData], - rootDir: File): Seq[String] = { - maybeCompressedFiles.map { compressedFiles => - CompressionUtils.unpackAndWriteCompressedFiles(compressedFiles, rootDir) - }.getOrElse(Seq.empty[String]) - } + private case class ResolvedAppResource(localPath: String, sparkJarPath: String) } private[spark] object KubernetesSparkRestServer { diff --git a/resource-managers/kubernetes/docker-minimal-bundle/pom.xml b/resource-managers/kubernetes/docker-minimal-bundle/pom.xml index 0ec2f36075db3..7f4d935e0e243 100644 --- a/resource-managers/kubernetes/docker-minimal-bundle/pom.xml +++ b/resource-managers/kubernetes/docker-minimal-bundle/pom.xml @@ -44,12 +44,6 @@ pom - - org.apache.spark - spark-examples_${scala.binary.version} - ${project.version} - provided -