From d3cf58f9826e5554998bbd6adcee9ca6a2df8244 Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Fri, 16 Jun 2017 13:06:42 -0700 Subject: [PATCH 01/26] Adding PySpark Submit functionality. Launching Python from JVM --- README.md | 1 + .../org/apache/spark/deploy/SparkSubmit.scala | 15 +++- .../spark/deploy/kubernetes/constants.scala | 2 + .../deploy/kubernetes/submit/Client.scala | 50 ++++++++++--- ...riverInitContainerComponentsProvider.scala | 5 ++ .../DriverPodKubernetesFileMounter.scala | 74 +++++++++++++++++++ .../kubernetes/submit/ClientV2Suite.scala | 6 ++ .../src/main/docker/driver-py/Dockerfile | 48 ++++++++++++ .../src/main/docker/executor-py/Dockerfile | 46 ++++++++++++ .../src/main/docker/init-container/Dockerfile | 2 +- .../docker/resource-staging-server/Dockerfile | 3 +- .../main/docker/shuffle-service/Dockerfile | 2 +- .../kubernetes/integration-tests/pom.xml | 20 +++++ .../integration-tests/src/test/python/pi.py | 46 ++++++++++++ .../integrationtest/KubernetesSuite.scala | 37 +++++++++- .../docker/SparkDockerImageBuilder.scala | 4 + 16 files changed, 344 insertions(+), 17 deletions(-) create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/DriverPodKubernetesFileMounter.scala create mode 100644 resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/driver-py/Dockerfile create mode 100644 resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/executor-py/Dockerfile create mode 100755 resource-managers/kubernetes/integration-tests/src/test/python/pi.py diff --git a/README.md b/README.md index cf6b4fa80242b..cb747225a11d4 100644 --- a/README.md +++ b/README.md @@ -24,6 +24,7 @@ We've been asked by an Apache Spark Committer to work outside of the Apache infr This is a collaborative effort by several folks from different companies who are interested in seeing this feature be successful. Companies active in this project include (alphabetically): +- Bloomberg - Google - Haiwen - Hyperpilot 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 59ccf3af24ce7..56122b50adf29 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -335,8 +335,8 @@ object SparkSubmit { (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 (KUBERNETES, CLUSTER) if args.isR => + printErrorAndExit("Kubernetes does not currently support R applications.") case (STANDALONE, CLUSTER) if args.isPython => printErrorAndExit("Cluster deploy mode is currently not supported for python " + "applications on standalone clusters.") @@ -620,8 +620,15 @@ object SparkSubmit { if (isKubernetesCluster) { childMainClass = "org.apache.spark.deploy.kubernetes.submit.Client" - childArgs += args.primaryResource - childArgs += args.mainClass + if (args.isPython) { + childArgs += args.primaryResource + childArgs += "org.apache.spark.deploy.PythonRunner" + childArgs += args.pyFiles + } + else { + childArgs += args.primaryResource + childArgs += args.mainClass + } childArgs ++= args.childArgs } 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 f2f1136e54fe4..92f051b2ac298 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,6 +67,8 @@ package object constants { private[spark] val ENV_DRIVER_ARGS = "SPARK_DRIVER_ARGS" private[spark] val ENV_DRIVER_JAVA_OPTS = "SPARK_DRIVER_JAVA_OPTS" private[spark] val ENV_MOUNTED_FILES_DIR = "SPARK_MOUNTED_FILES_DIR" + private[spark] val ENV_PYSPARK_FILES = "PYSPARK_FILES" + private[spark] val ENV_PYSPARK_PRIMARY = "PYSPARK_PRIMARY" // Bootstrapping dependencies with the init-container private[spark] val INIT_CONTAINER_ANNOTATION = "pod.beta.kubernetes.io/init-containers" diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/Client.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/Client.scala index a9699d8c34b4e..78b6a40efdcc9 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/Client.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/Client.scala @@ -47,11 +47,14 @@ private[spark] class Client( appName: String, kubernetesResourceNamePrefix: String, kubernetesAppId: String, + mainAppResource: String, + isPython: Boolean, mainClass: String, sparkConf: SparkConf, appArgs: Array[String], sparkJars: Seq[String], sparkFiles: Seq[String], + pySparkFiles: List[String], waitForAppCompletion: Boolean, kubernetesClient: KubernetesClient, initContainerComponentsProvider: DriverInitContainerComponentsProvider, @@ -83,7 +86,14 @@ private[spark] class Client( def run(): Unit = { validateNoDuplicateFileNames(sparkJars) validateNoDuplicateFileNames(sparkFiles) - + if (isPython) {validateNoDuplicateFileNames(pySparkFiles)} + val arguments = if (isPython) pySparkFiles match { + case Nil => appArgs + case a::b => a match { + case _ if a==mainAppResource && b==Nil => appArgs + case _ => appArgs.drop(1) + } + } else appArgs val driverCustomLabels = ConfigurationUtils.combinePrefixedKeyValuePairsWithDeprecatedConf( sparkConf, KUBERNETES_DRIVER_LABEL_PREFIX, @@ -135,7 +145,7 @@ private[spark] class Client( .endEnv() .addNewEnv() .withName(ENV_DRIVER_ARGS) - .withValue(appArgs.mkString(" ")) + .withValue(arguments.mkString(" ")) .endEnv() .withNewResources() .addToRequests("cpu", driverCpuQuantity) @@ -204,7 +214,7 @@ private[spark] class Client( val resolvedDriverJavaOpts = resolvedSparkConf.getAll.map { case (confKey, confValue) => s"-D$confKey=$confValue" }.mkString(" ") + driverJavaOptions.map(" " + _).getOrElse("") - val resolvedDriverPod = podWithInitContainerAndMountedCreds.editSpec() + val resolvedDriverPodBuilder = podWithInitContainerAndMountedCreds.editSpec() .editMatchingContainer(new ContainerNameEqualityPredicate(driverContainer.getName)) .addNewEnv() .withName(ENV_MOUNTED_CLASSPATH) @@ -216,7 +226,15 @@ private[spark] class Client( .endEnv() .endContainer() .endSpec() - .build() + val resolvedDriverPod = if (!isPython) { + resolvedDriverPodBuilder.build() + } else { + initContainerComponentsProvider + .provideDriverPodFileMounter() + .addPySparkFiles( + mainAppResource, pySparkFiles, driverContainer.getName, resolvedDriverPodBuilder) + .build() + } Utils.tryWithResource( kubernetesClient .pods() @@ -266,7 +284,7 @@ private[spark] class Client( } } -private[spark] object Client { +private[spark] object Client{ def main(args: Array[String]): Unit = { val sparkConf = new SparkConf(true) val mainAppResource = args(0) @@ -274,22 +292,28 @@ private[spark] object Client { val appArgs = args.drop(2) run(sparkConf, mainAppResource, mainClass, appArgs) } - def run( sparkConf: SparkConf, mainAppResource: String, mainClass: String, appArgs: Array[String]): Unit = { - val sparkJars = sparkConf.getOption("spark.jars") + val isPython = mainAppResource.endsWith(".py") + val sparkJars = if (isPython) Array.empty[String] else { + sparkConf.getOption("spark.jars") .map(_.split(",")) .getOrElse(Array.empty[String]) ++ Option(mainAppResource) .filterNot(_ == SparkLauncher.NO_RESOURCE) - .toSeq + .toSeq } val launchTime = System.currentTimeMillis val sparkFiles = sparkConf.getOption("spark.files") .map(_.split(",")) .getOrElse(Array.empty[String]) + val pySparkFiles: Array[String] = if (isPython) { + appArgs(0) match { + case null => Array(mainAppResource) + case _ => mainAppResource +: appArgs(0).split(",") + }} else {Array.empty[String]} val appName = sparkConf.getOption("spark.app.name").getOrElse("spark") // The resource name prefix is derived from the application name, making it easy to connect the // names of the Kubernetes resources from e.g. Kubectl or the Kubernetes dashboard to the @@ -302,12 +326,17 @@ private[spark] object Client { val namespace = sparkConf.get(KUBERNETES_NAMESPACE) val master = resolveK8sMaster(sparkConf.get("spark.master")) val sslOptionsProvider = new ResourceStagingServerSslOptionsProviderImpl(sparkConf) + // No reason to distribute python files that are locally baked into Docker image + def filterByFile(pFiles: Array[String]) : Array[String] = { + val LocalPattern = "(local://)(.*)" + pFiles.filter(fi => !(fi matches LocalPattern)) + } val initContainerComponentsProvider = new DriverInitContainerComponentsProviderImpl( sparkConf, kubernetesResourceNamePrefix, namespace, sparkJars, - sparkFiles, + sparkFiles ++ filterByFile(pySparkFiles), sslOptionsProvider.getSslOptions) Utils.tryWithResource(SparkKubernetesClientFactory.createKubernetesClient( master, @@ -328,11 +357,14 @@ private[spark] object Client { appName, kubernetesResourceNamePrefix, kubernetesAppId, + mainAppResource, + isPython, mainClass, sparkConf, appArgs, sparkJars, sparkFiles, + pySparkFiles.toList, waitForAppCompletion, kubernetesClient, initContainerComponentsProvider, diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/DriverInitContainerComponentsProvider.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/DriverInitContainerComponentsProvider.scala index cfc61e193dcff..e9da1af35e8ca 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/DriverInitContainerComponentsProvider.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/DriverInitContainerComponentsProvider.scala @@ -41,6 +41,8 @@ private[spark] trait DriverInitContainerComponentsProvider { maybeSubmittedResourceSecrets: Option[SubmittedResourceSecrets]) : Option[SubmittedDependencySecretBuilder] def provideInitContainerBootstrap(): SparkPodInitContainerBootstrap + def provideDriverPodFileMounter(): DriverPodKubernetesFileMounter + } private[spark] class DriverInitContainerComponentsProviderImpl( @@ -202,4 +204,7 @@ private[spark] class DriverInitContainerComponentsProviderImpl( configMapKey, resourceStagingServerSecretPlugin) } + override def provideDriverPodFileMounter(): DriverPodKubernetesFileMounter = { + new DriverPodKubernetesFileMounterImpl(filesDownloadPath) + } } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/DriverPodKubernetesFileMounter.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/DriverPodKubernetesFileMounter.scala new file mode 100644 index 0000000000000..c28d769e5205e --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/DriverPodKubernetesFileMounter.scala @@ -0,0 +1,74 @@ +/* + * 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.submit + +import io.fabric8.kubernetes.api.model.{Container, PodBuilder} + +import org.apache.spark.deploy.kubernetes.constants._ + + /** + * Trait that is responsible for providing full file-paths dynamically after + * the filesDownloadPath has been defined. The file-names are then stored in the + * environmental variables in the driver-pod. + */ +private[spark] trait DriverPodKubernetesFileMounter { + def addPySparkFiles(mainAppResource: String, pythonFiles: List[String], + mainContainerName: String, originalPodSpec: PodBuilder) : PodBuilder +} + +private[spark] class DriverPodKubernetesFileMounterImpl(filesDownloadPath: String) + extends DriverPodKubernetesFileMounter { + val LocalPattern = "(local://)(.*)".r + val FilePattern = "(file:/)(.*)".r + def getName(file: String, separatorChar: Char) : String = { + val index: Int = file.lastIndexOf(separatorChar) + file.substring(index + 1) + } + def fileLoc(file: String) : String = file match { + case "" => "" + case LocalPattern(_, file_name) => file_name + case FilePattern(_, file_name) => filesDownloadPath + "/" + getName(file_name, '/') + case _ => filesDownloadPath + "/" + getName(file, '/') + } + def pythonFileLocations(pFiles: List[String], mainAppResource: String) : String = { + def recFileLoc(file: List[String]): List[String] = file match { + case Nil => List.empty[String] + case a::b => a match { + case _ if a==mainAppResource => recFileLoc(b) + case _ => fileLoc(a) +: recFileLoc(b) + } + } + recFileLoc(pFiles).mkString(",") + } + override def addPySparkFiles(mainAppResource: String, pythonFiles: List[String], + mainContainerName: String, + originalPodSpec: PodBuilder): PodBuilder = { + originalPodSpec + .editSpec() + .editMatchingContainer(new ContainerNameEqualityPredicate(mainContainerName)) + .addNewEnv() + .withName(ENV_PYSPARK_PRIMARY) + .withValue(fileLoc(mainAppResource)) + .endEnv() + .addNewEnv() + .withName(ENV_PYSPARK_FILES) + .withValue(pythonFileLocations(pythonFiles, mainAppResource)) + .endEnv() + .endContainer() + .endSpec() + } +} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/ClientV2Suite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/ClientV2Suite.scala index 3945bef5bcfb8..a28b1f3b4139b 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/ClientV2Suite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/ClientV2Suite.scala @@ -301,11 +301,14 @@ class ClientV2Suite extends SparkFunSuite with BeforeAndAfter { APP_NAME, APP_RESOURCE_PREFIX, APP_ID, + null, + false, MAIN_CLASS, SPARK_CONF, APP_ARGS, SPARK_JARS, SPARK_FILES, + null, true, kubernetesClient, initContainerComponentsProvider, @@ -386,11 +389,14 @@ class ClientV2Suite extends SparkFunSuite with BeforeAndAfter { APP_NAME, APP_RESOURCE_PREFIX, APP_ID, + null, + false, MAIN_CLASS, SPARK_CONF, APP_ARGS, SPARK_JARS, SPARK_FILES, + null, false, kubernetesClient, initContainerComponentsProvider, diff --git a/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/driver-py/Dockerfile b/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/driver-py/Dockerfile new file mode 100644 index 0000000000000..6dcc7511c0dd9 --- /dev/null +++ b/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/driver-py/Dockerfile @@ -0,0 +1,48 @@ +# +# 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. +# + +FROM spark-base + +# 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-py:latest -f dockerfiles/driver-py/Dockerfile . + +ADD examples /opt/spark/examples +ADD python /opt/spark/python + +RUN apk add --no-cache python && \ + python -m ensurepip && \ + rm -r /usr/lib/python*/ensurepip && \ + pip install --upgrade pip setuptools && \ + rm -r /root/.cache +# UNCOMMENT THE FOLLOWING TO START PIP INSTALLING PYTHON PACKAGES +# RUN apk add --update alpine-sdk python-dev +# RUN pip install numpy + +ENV PYTHON_VERSION 2.7.13 +ENV PYSPARK_PYTHON python +ENV PYSPARK_DRIVER_PYTHON python +ENV PYTHONPATH ${SPARK_HOME}/python/:${SPARK_HOME}/python/lib/py4j-0.10.4-src.zip:${PYTHONPATH} + +CMD SPARK_CLASSPATH="${SPARK_HOME}/jars/*" && \ + if ! [ -z ${SPARK_MOUNTED_CLASSPATH+x} ]; then SPARK_CLASSPATH="$SPARK_MOUNTED_CLASSPATH:$SPARK_CLASSPATH"; fi && \ + if ! [ -z ${SPARK_SUBMIT_EXTRA_CLASSPATH+x} ]; then SPARK_CLASSPATH="$SPARK_SUBMIT_EXTRA_CLASSPATH:$SPARK_CLASSPATH"; fi && \ + if ! [ -z ${SPARK_EXTRA_CLASSPATH+x} ]; then SPARK_CLASSPATH="$SPARK_EXTRA_CLASSPATH:$SPARK_CLASSPATH"; fi && \ + if ! [ -z ${SPARK_MOUNTED_FILES_DIR} ]; then cp -R "$SPARK_MOUNTED_FILES_DIR/." .; fi && \ + exec /sbin/tini -- ${JAVA_HOME}/bin/java $SPARK_DRIVER_JAVA_OPTS -cp $SPARK_CLASSPATH \ + -Xms$SPARK_DRIVER_MEMORY -Xmx$SPARK_DRIVER_MEMORY \ + $SPARK_DRIVER_CLASS $PYSPARK_PRIMARY $PYSPARK_FILES $SPARK_DRIVER_ARGS diff --git a/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/executor-py/Dockerfile b/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/executor-py/Dockerfile new file mode 100644 index 0000000000000..7a65a4f879376 --- /dev/null +++ b/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/executor-py/Dockerfile @@ -0,0 +1,46 @@ +# +# 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. +# + +FROM spark-base + +# 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-py:latest -f dockerfiles/executor-py/Dockerfile . + +ADD examples /opt/spark/examples +ADD python /opt/spark/python + +RUN apk add --no-cache python && \ + python -m ensurepip && \ + rm -r /usr/lib/python*/ensurepip && \ + pip install --upgrade pip setuptools && \ + rm -r /root/.cache +# UNCOMMENT THE FOLLOWING TO START PIP INSTALLING PYTHON PACKAGES +# RUN apk add --update alpine-sdk python-dev +# RUN pip install numpy + +ENV PYTHON_VERSION 2.7.13 +ENV PYSPARK_PYTHON python +ENV PYSPARK_DRIVER_PYTHON python +ENV PYTHONPATH ${SPARK_HOME}/python/:${SPARK_HOME}/python/lib/py4j-0.10.4-src.zip:${PYTHONPATH} + +# TODO support spark.executor.extraClassPath +CMD SPARK_CLASSPATH="${SPARK_HOME}/jars/*" && \ + if ! [ -z ${SPARK_MOUNTED_CLASSPATH}+x} ]; then SPARK_CLASSPATH="$SPARK_MOUNTED_CLASSPATH:$SPARK_CLASSPATH"; fi && \ + if ! [ -z ${SPARK_EXECUTOR_EXTRA_CLASSPATH+x} ]; then SPARK_CLASSPATH="$SPARK_EXECUTOR_EXTRA_CLASSPATH:$SPARK_CLASSPATH"; fi && \ + if ! [ -z ${SPARK_MOUNTED_FILES_DIR} ]; then cp -R "$SPARK_MOUNTED_FILES_DIR/." .; fi && \ + exec /sbin/tini -- ${JAVA_HOME}/bin/java -Dspark.executor.port=$SPARK_EXECUTOR_PORT -Xms$SPARK_EXECUTOR_MEMORY -Xmx$SPARK_EXECUTOR_MEMORY -cp $SPARK_CLASSPATH 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 $SPARK_EXECUTOR_POD_IP \ No newline at end of file diff --git a/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/init-container/Dockerfile b/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/init-container/Dockerfile index 6bff06da12840..4bafe25e2608f 100644 --- a/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/init-container/Dockerfile +++ b/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/init-container/Dockerfile @@ -19,6 +19,6 @@ FROM spark-base # 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 . +# docker build -t spark-init:latest -f dockerfiles/init-container/Dockerfile . ENTRYPOINT [ "/sbin/tini", "--", "bin/spark-class", "org.apache.spark.deploy.rest.kubernetes.KubernetesSparkDependencyDownloadInitContainer" ] diff --git a/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/resource-staging-server/Dockerfile b/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/resource-staging-server/Dockerfile index c9a92fa1c5b62..9ca96be0f1a88 100644 --- a/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/resource-staging-server/Dockerfile +++ b/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/resource-staging-server/Dockerfile @@ -17,8 +17,9 @@ FROM spark-base + # 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 . +# docker build -t spark-resource-staging-server:latest -f dockerfiles/resource-staging-server/Dockerfile . ENTRYPOINT [ "/sbin/tini", "--", "bin/spark-class", "org.apache.spark.deploy.rest.kubernetes.ResourceStagingServer" ] diff --git a/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/shuffle-service/Dockerfile b/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/shuffle-service/Dockerfile index 7f4e2aa51b67d..ccb2f1a03d88c 100644 --- a/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/shuffle-service/Dockerfile +++ b/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/shuffle-service/Dockerfile @@ -19,7 +19,7 @@ FROM spark-base # 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-shuffle:latest -f dockerfiles/shuffle/Dockerfile . +# docker build -t spark-shuffle:latest -f dockerfiles/shuffle-service/Dockerfile . COPY examples /opt/spark/examples diff --git a/resource-managers/kubernetes/integration-tests/pom.xml b/resource-managers/kubernetes/integration-tests/pom.xml index bbf4b02cdaaf9..a4f06bcf45808 100644 --- a/resource-managers/kubernetes/integration-tests/pom.xml +++ b/resource-managers/kubernetes/integration-tests/pom.xml @@ -220,6 +220,26 @@ + + copy-integration-python + pre-integration-test + + copy-resources + + + ${project.build.directory}/docker/python + + + ${project.parent.basedir}/python + true + + ${project.parent.basedir}/python/.egg + ${project.parent.basedir}/python/dist + + + + + diff --git a/resource-managers/kubernetes/integration-tests/src/test/python/pi.py b/resource-managers/kubernetes/integration-tests/src/test/python/pi.py new file mode 100755 index 0000000000000..e3f0c4aeef1b7 --- /dev/null +++ b/resource-managers/kubernetes/integration-tests/src/test/python/pi.py @@ -0,0 +1,46 @@ +from __future__ import print_function +# +# 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. +# + +import sys +from random import random +from operator import add + +from pyspark.sql import SparkSession + + +if __name__ == "__main__": + """ + Usage: pi [partitions] + """ + spark = SparkSession\ + .builder\ + .appName("PythonPi")\ + .getOrCreate() + + partitions = int(sys.argv[1]) if len(sys.argv) > 1 else 2 + n = 100000 * partitions + + def f(_): + x = random() * 2 - 1 + y = random() * 2 - 1 + return 1 if x ** 2 + y ** 2 < 1 else 0 + + count = spark.sparkContext.parallelize(range(1, n + 1), partitions).map(f).reduce(add) + print("Pi is roughly %f" % (4.0 * count / n)) + + 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 e377f285eb9a6..cb0fe7332f79f 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 @@ -72,6 +72,31 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { kubernetesTestComponents.deleteNamespace() } + /** + * These tests need to be run in an environment similair to the one provided + * by make_distribution. Further investigation is required + */ +// test("Run PySpark Job on file from SUBMITTER") { +// assume(testBackend.name == MINIKUBE_TEST_BACKEND) +// launchStagingServer(SSLOptions(), None) +// sparkConf.set(DRIVER_DOCKER_IMAGE, +// System.getProperty("spark.docker.test.driverImage", "spark-driver-py:latest")) +// sparkConf.set(EXECUTOR_DOCKER_IMAGE, +// System.getProperty("spark.docker.test.executorImage", "spark-executor-py:latest")) +// runPySparkPiAndVerifyCompletion( +// PYSPARK_PI_SUBMITTER_LOCAL_FILE_LOCATION) +// } +// test("Run PySpark Job on file from CONTAINER") { +// assume(testBackend.name == MINIKUBE_TEST_BACKEND) +// // UPDATE SO THAT IT BUILDS FROM LOCAL DOCKER IMAGE +// sparkConf.set(DRIVER_DOCKER_IMAGE, +// System.getProperty("spark.docker.test.driverImage", "spark-driver-py:latest")) +// sparkConf.set(EXECUTOR_DOCKER_IMAGE, +// System.getProperty("spark.docker.test.executorImage", "spark-executor-py:latest")) +// runPySparkPiAndVerifyCompletion( +// PYSPARK_PI_CONTAINER_LOCAL_FILE_LOCATION) +// } + test("Simple submission test with the resource staging server.") { assume(testBackend.name == MINIKUBE_TEST_BACKEND) @@ -223,6 +248,13 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { appResource, SPARK_PI_MAIN_CLASS, "Pi is roughly 3", Array.empty[String]) } + private def runPySparkPiAndVerifyCompletion( + appResource: String): Unit = { + runSparkApplicationAndVerifyCompletion( + appResource, PYSPARK_PI_MAIN_CLASS, "Pi is roughly 3", + Array(null, "5")) + } + private def runSparkApplicationAndVerifyCompletion( appResource: String, mainClass: String, @@ -305,11 +337,14 @@ private[spark] object KubernetesSuite { s"integration-tests-jars/${EXAMPLES_JAR_FILE.getName}" val CONTAINER_LOCAL_HELPER_JAR_PATH = s"local:///opt/spark/examples/" + s"integration-tests-jars/${HELPER_JAR_FILE.getName}" - val TIMEOUT = PatienceConfiguration.Timeout(Span(2, Minutes)) val INTERVAL = PatienceConfiguration.Interval(Span(2, Seconds)) val SPARK_PI_MAIN_CLASS = "org.apache.spark.deploy.kubernetes" + ".integrationtest.jobs.SparkPiWithInfiniteWait" + val PYSPARK_PI_MAIN_CLASS = "org.apache.spark.deploy.PythonRunner" + val PYSPARK_PI_CONTAINER_LOCAL_FILE_LOCATION = "local:///opt/spark/" + + "examples/src/main/python/pi.py" + val PYSPARK_PI_SUBMITTER_LOCAL_FILE_LOCATION = "src/test/python/pi.py" val FILE_EXISTENCE_MAIN_CLASS = "org.apache.spark.deploy.kubernetes" + ".integrationtest.jobs.FileExistenceTest" val GROUP_BY_MAIN_CLASS = "org.apache.spark.deploy.kubernetes" + diff --git a/resource-managers/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 index 4db19478f44bc..faa29b91ff202 100644 --- a/resource-managers/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 @@ -30,7 +30,9 @@ private[spark] class SparkDockerImageBuilder(private val dockerEnv: Map[String, // Dockerfile paths must be relative to the build path. private val BASE_DOCKER_FILE = "dockerfiles/spark-base/Dockerfile" private val DRIVER_DOCKER_FILE = "dockerfiles/driver/Dockerfile" + private val DRIVERPY_DOCKER_FILE = "dockerfiles/driver-py/Dockerfile" private val EXECUTOR_DOCKER_FILE = "dockerfiles/executor/Dockerfile" + private val EXECUTORPY_DOCKER_FILE = "dockerfiles/executor-py/Dockerfile" private val SHUFFLE_SERVICE_DOCKER_FILE = "dockerfiles/shuffle-service/Dockerfile" private val INIT_CONTAINER_DOCKER_FILE = "dockerfiles/init-container/Dockerfile" private val STAGING_SERVER_DOCKER_FILE = "dockerfiles/resource-staging-server/Dockerfile" @@ -63,7 +65,9 @@ private[spark] class SparkDockerImageBuilder(private val dockerEnv: Map[String, Eventually.eventually(TIMEOUT, INTERVAL) { dockerClient.ping() } buildImage("spark-base", BASE_DOCKER_FILE) buildImage("spark-driver", DRIVER_DOCKER_FILE) + buildImage("spark-driver-py", DRIVERPY_DOCKER_FILE) buildImage("spark-executor", EXECUTOR_DOCKER_FILE) + buildImage("spark-executor-py", EXECUTORPY_DOCKER_FILE) buildImage("spark-shuffle", SHUFFLE_SERVICE_DOCKER_FILE) buildImage("spark-resource-staging-server", STAGING_SERVER_DOCKER_FILE) buildImage("spark-init", INIT_CONTAINER_DOCKER_FILE) From bafc13c55d110c8755fc028dbe989a488e4b6869 Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Fri, 16 Jun 2017 18:20:16 -0700 Subject: [PATCH 02/26] Addressing scala idioms related to PR351 --- .../scala/org/apache/spark/deploy/SparkSubmit.scala | 3 +-- .../apache/spark/deploy/kubernetes/submit/Client.scala | 10 +++++----- .../spark/deploy/kubernetes/submit/ClientV2Suite.scala | 8 ++++---- 3 files changed, 10 insertions(+), 11 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 56122b50adf29..9256a9ddd9960 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -624,8 +624,7 @@ object SparkSubmit { childArgs += args.primaryResource childArgs += "org.apache.spark.deploy.PythonRunner" childArgs += args.pyFiles - } - else { + } else { childArgs += args.primaryResource childArgs += args.mainClass } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/Client.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/Client.scala index 78b6a40efdcc9..63ef05455387d 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/Client.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/Client.scala @@ -284,7 +284,7 @@ private[spark] class Client( } } -private[spark] object Client{ +private[spark] object Client extends Logging{ def main(args: Array[String]): Unit = { val sparkConf = new SparkConf(true) val mainAppResource = args(0) @@ -310,10 +310,10 @@ private[spark] object Client{ .map(_.split(",")) .getOrElse(Array.empty[String]) val pySparkFiles: Array[String] = if (isPython) { - appArgs(0) match { - case null => Array(mainAppResource) - case _ => mainAppResource +: appArgs(0).split(",") - }} else {Array.empty[String]} + Option(appArgs(0)) match { + case None => Array(mainAppResource) + case Some(a) => mainAppResource +: a.split(",") } + } else { Array.empty[String] } val appName = sparkConf.getOption("spark.app.name").getOrElse("spark") // The resource name prefix is derived from the application name, making it easy to connect the // names of the Kubernetes resources from e.g. Kubectl or the Kubernetes dashboard to the diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/ClientV2Suite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/ClientV2Suite.scala index a28b1f3b4139b..644d2d911e269 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/ClientV2Suite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/ClientV2Suite.scala @@ -301,14 +301,14 @@ class ClientV2Suite extends SparkFunSuite with BeforeAndAfter { APP_NAME, APP_RESOURCE_PREFIX, APP_ID, - null, + "", false, MAIN_CLASS, SPARK_CONF, APP_ARGS, SPARK_JARS, SPARK_FILES, - null, + Nil, true, kubernetesClient, initContainerComponentsProvider, @@ -389,14 +389,14 @@ class ClientV2Suite extends SparkFunSuite with BeforeAndAfter { APP_NAME, APP_RESOURCE_PREFIX, APP_ID, - null, + "", false, MAIN_CLASS, SPARK_CONF, APP_ARGS, SPARK_JARS, SPARK_FILES, - null, + Nil, false, kubernetesClient, initContainerComponentsProvider, From 59d9f0af583eafa88c183ddcd6fba9c0918aabc9 Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Fri, 16 Jun 2017 18:25:17 -0700 Subject: [PATCH 03/26] Removing extends Logging which was necessary for LogInfo --- .../org/apache/spark/deploy/kubernetes/submit/Client.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/Client.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/Client.scala index 63ef05455387d..487a971c86f34 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/Client.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/Client.scala @@ -284,7 +284,7 @@ private[spark] class Client( } } -private[spark] object Client extends Logging{ +private[spark] object Client { def main(args: Array[String]): Unit = { val sparkConf = new SparkConf(true) val mainAppResource = args(0) From 4daf6347e9ff84bece9996c264d87102bbee318d Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Mon, 19 Jun 2017 18:53:17 -0700 Subject: [PATCH 04/26] Refactored code to leverage the ContainerLocalizedFileResolver --- .../deploy/kubernetes/submit/Client.scala | 34 +++++++----- .../ContainerLocalizedFilesResolver.scala | 45 +++++++++++---- ...riverInitContainerComponentsProvider.scala | 16 ++++-- .../DriverPodKubernetesFileMounter.scala | 55 ++++++------------- .../kubernetes/submit/ClientV2Suite.scala | 4 +- ...ContainerLocalizedFilesResolverSuite.scala | 22 ++++++++ .../integrationtest/KubernetesSuite.scala | 4 +- 7 files changed, 108 insertions(+), 72 deletions(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/Client.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/Client.scala index 487a971c86f34..7b536e9fca73f 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/Client.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/Client.scala @@ -183,10 +183,12 @@ private[spark] class Client( .bootstrapInitContainerAndVolumes(driverContainer.getName, basePod) val containerLocalizedFilesResolver = initContainerComponentsProvider - .provideContainerLocalizedFilesResolver() + .provideContainerLocalizedFilesResolver(mainAppResource) val resolvedSparkJars = containerLocalizedFilesResolver.resolveSubmittedSparkJars() val resolvedSparkFiles = containerLocalizedFilesResolver.resolveSubmittedSparkFiles() - + val resolvedPySparkFiles = containerLocalizedFilesResolver.resolveSubmittedPySparkFiles() + val resolvedPrimaryPySparkResource = if (!isPython) "" + else { containerLocalizedFilesResolver.resolvePrimaryResourceFile() } val executorInitContainerConfiguration = initContainerComponentsProvider .provideExecutorInitContainerConfiguration() val sparkConfWithExecutorInit = executorInitContainerConfiguration @@ -232,7 +234,10 @@ private[spark] class Client( initContainerComponentsProvider .provideDriverPodFileMounter() .addPySparkFiles( - mainAppResource, pySparkFiles, driverContainer.getName, resolvedDriverPodBuilder) + resolvedPrimaryPySparkResource, + resolvedPySparkFiles.mkString(","), + driverContainer.getName, + resolvedDriverPodBuilder) .build() } Utils.tryWithResource( @@ -298,13 +303,16 @@ private[spark] object Client { mainClass: String, appArgs: Array[String]): Unit = { val isPython = mainAppResource.endsWith(".py") - val sparkJars = if (isPython) Array.empty[String] else { - sparkConf.getOption("spark.jars") - .map(_.split(",")) - .getOrElse(Array.empty[String]) ++ - Option(mainAppResource) + // Since you might need jars for SQL UDFs in PySpark + def sparkJarFilter() : Seq[String] = isPython match { + case true => Seq.empty[String] + case false => Option(mainAppResource) .filterNot(_ == SparkLauncher.NO_RESOURCE) - .toSeq } + .toSeq + } + val sparkJars = sparkConf.getOption("spark.jars") + .map(_.split(",")) + .getOrElse(Array.empty[String]) ++ sparkJarFilter() val launchTime = System.currentTimeMillis val sparkFiles = sparkConf.getOption("spark.files") .map(_.split(",")) @@ -326,17 +334,13 @@ private[spark] object Client { val namespace = sparkConf.get(KUBERNETES_NAMESPACE) val master = resolveK8sMaster(sparkConf.get("spark.master")) val sslOptionsProvider = new ResourceStagingServerSslOptionsProviderImpl(sparkConf) - // No reason to distribute python files that are locally baked into Docker image - def filterByFile(pFiles: Array[String]) : Array[String] = { - val LocalPattern = "(local://)(.*)" - pFiles.filter(fi => !(fi matches LocalPattern)) - } val initContainerComponentsProvider = new DriverInitContainerComponentsProviderImpl( sparkConf, kubernetesResourceNamePrefix, namespace, sparkJars, - sparkFiles ++ filterByFile(pySparkFiles), + sparkFiles, + pySparkFiles, sslOptionsProvider.getSslOptions) Utils.tryWithResource(SparkKubernetesClientFactory.createKubernetesClient( master, diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/ContainerLocalizedFilesResolver.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/ContainerLocalizedFilesResolver.scala index c635484c4c124..999fe6d7e38a0 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/ContainerLocalizedFilesResolver.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/ContainerLocalizedFilesResolver.scala @@ -24,13 +24,19 @@ private[spark] trait ContainerLocalizedFilesResolver { def resolveSubmittedAndRemoteSparkJars(): Seq[String] def resolveSubmittedSparkJars(): Seq[String] def resolveSubmittedSparkFiles(): Seq[String] + def resolveSubmittedPySparkFiles(): Seq[String] + def resolvePrimaryResourceFile(): String } private[spark] class ContainerLocalizedFilesResolverImpl( sparkJars: Seq[String], sparkFiles: Seq[String], + pySparkFiles: Seq[String], + primaryPyFile: String, jarsDownloadPath: String, - filesDownloadPath: String) extends ContainerLocalizedFilesResolver { + filesDownloadPath: String + ) extends ContainerLocalizedFilesResolver { + override def resolveSubmittedAndRemoteSparkJars(): Seq[String] = { sparkJars.map { jar => @@ -53,16 +59,33 @@ private[spark] class ContainerLocalizedFilesResolverImpl( resolveSubmittedFiles(sparkFiles, filesDownloadPath) } - private def resolveSubmittedFiles(files: Seq[String], downloadPath: String): Seq[String] = { - files.map { file => - val fileUri = Utils.resolveURI(file) - Option(fileUri.getScheme).getOrElse("file") match { - case "file" => - val fileName = new File(fileUri.getPath).getName - s"$downloadPath/$fileName" - case _ => - file - } + override def resolveSubmittedPySparkFiles(): Seq[String] = { + def filterMainResource(x: String) = x match { + case `primaryPyFile` => None + case _ => Some(resolveFile(x, filesDownloadPath)) + } + pySparkFiles.flatMap(x => filterMainResource(x)) + } + + override def resolvePrimaryResourceFile(): String = { + Option(primaryPyFile) match { + case None => "" + case Some(p) => resolveFile(p, filesDownloadPath) } } + + private def resolveFile(file: String, downloadPath: String) = { + val fileUri = Utils.resolveURI(file) + Option(fileUri.getScheme).getOrElse("file") match { + case "file" => + val fileName = new File(fileUri.getPath).getName + s"$downloadPath/$fileName" + case _ => + file + } + } + + private def resolveSubmittedFiles(files: Seq[String], downloadPath: String): Seq[String] = { + files.map { file => resolveFile(file, downloadPath) } + } } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/DriverInitContainerComponentsProvider.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/DriverInitContainerComponentsProvider.scala index e9da1af35e8ca..ba5de3acbc897 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/DriverInitContainerComponentsProvider.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/DriverInitContainerComponentsProvider.scala @@ -33,7 +33,8 @@ private[spark] trait DriverInitContainerComponentsProvider { def provideInitContainerConfigMapBuilder( maybeSubmittedResourceIds: Option[SubmittedResourceIds]) : SparkInitContainerConfigMapBuilder - def provideContainerLocalizedFilesResolver(): ContainerLocalizedFilesResolver + def provideContainerLocalizedFilesResolver( + mainAppResource: String) : ContainerLocalizedFilesResolver def provideExecutorInitContainerConfiguration(): ExecutorInitContainerConfiguration def provideInitContainerSubmittedDependencyUploader( driverPodLabels: Map[String, String]): Option[SubmittedDependencyUploader] @@ -51,6 +52,7 @@ private[spark] class DriverInitContainerComponentsProviderImpl( namespace: String, sparkJars: Seq[String], sparkFiles: Seq[String], + pySparkFiles: Seq[String], resourceStagingServerExternalSslOptions: SSLOptions) extends DriverInitContainerComponentsProvider { @@ -106,6 +108,7 @@ private[spark] class DriverInitContainerComponentsProviderImpl( private val initContainerImage = sparkConf.get(INIT_CONTAINER_DOCKER_IMAGE) private val dockerImagePullPolicy = sparkConf.get(DOCKER_IMAGE_PULL_POLICY) private val downloadTimeoutMinutes = sparkConf.get(INIT_CONTAINER_MOUNT_TIMEOUT) + private val pySparkSubmitted = KubernetesFileUtils.getOnlySubmitterLocalFiles(pySparkFiles) override def provideInitContainerConfigMapBuilder( maybeSubmittedResourceIds: Option[SubmittedResourceIds]) @@ -133,7 +136,7 @@ private[spark] class DriverInitContainerComponentsProviderImpl( } new SparkInitContainerConfigMapBuilderImpl( sparkJars, - sparkFiles, + sparkFiles ++ pySparkSubmitted, jarsDownloadPath, filesDownloadPath, configMapName, @@ -141,9 +144,10 @@ private[spark] class DriverInitContainerComponentsProviderImpl( submittedDependencyConfigPlugin) } - override def provideContainerLocalizedFilesResolver(): ContainerLocalizedFilesResolver = { + override def provideContainerLocalizedFilesResolver(mainAppResource: String) + : ContainerLocalizedFilesResolver = { new ContainerLocalizedFilesResolverImpl( - sparkJars, sparkFiles, jarsDownloadPath, filesDownloadPath) + sparkJars, sparkFiles, pySparkFiles, mainAppResource, jarsDownloadPath, filesDownloadPath) } override def provideExecutorInitContainerConfiguration(): ExecutorInitContainerConfiguration = { @@ -162,7 +166,7 @@ private[spark] class DriverInitContainerComponentsProviderImpl( namespace, stagingServerUri, sparkJars, - sparkFiles, + sparkFiles ++ pySparkSubmitted, resourceStagingServerExternalSslOptions, RetrofitClientFactoryImpl) } @@ -205,6 +209,6 @@ private[spark] class DriverInitContainerComponentsProviderImpl( resourceStagingServerSecretPlugin) } override def provideDriverPodFileMounter(): DriverPodKubernetesFileMounter = { - new DriverPodKubernetesFileMounterImpl(filesDownloadPath) + new DriverPodKubernetesFileMounterImpl() } } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/DriverPodKubernetesFileMounter.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/DriverPodKubernetesFileMounter.scala index c28d769e5205e..cc0ef0eedb457 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/DriverPodKubernetesFileMounter.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/DriverPodKubernetesFileMounter.scala @@ -26,49 +26,30 @@ import org.apache.spark.deploy.kubernetes.constants._ * environmental variables in the driver-pod. */ private[spark] trait DriverPodKubernetesFileMounter { - def addPySparkFiles(mainAppResource: String, pythonFiles: List[String], + def addPySparkFiles(primaryFile: String, pySparkFiles: String, mainContainerName: String, originalPodSpec: PodBuilder) : PodBuilder } -private[spark] class DriverPodKubernetesFileMounterImpl(filesDownloadPath: String) +private[spark] class DriverPodKubernetesFileMounterImpl() extends DriverPodKubernetesFileMounter { - val LocalPattern = "(local://)(.*)".r - val FilePattern = "(file:/)(.*)".r - def getName(file: String, separatorChar: Char) : String = { - val index: Int = file.lastIndexOf(separatorChar) - file.substring(index + 1) - } - def fileLoc(file: String) : String = file match { - case "" => "" - case LocalPattern(_, file_name) => file_name - case FilePattern(_, file_name) => filesDownloadPath + "/" + getName(file_name, '/') - case _ => filesDownloadPath + "/" + getName(file, '/') - } - def pythonFileLocations(pFiles: List[String], mainAppResource: String) : String = { - def recFileLoc(file: List[String]): List[String] = file match { - case Nil => List.empty[String] - case a::b => a match { - case _ if a==mainAppResource => recFileLoc(b) - case _ => fileLoc(a) +: recFileLoc(b) - } - } - recFileLoc(pFiles).mkString(",") - } - override def addPySparkFiles(mainAppResource: String, pythonFiles: List[String], - mainContainerName: String, - originalPodSpec: PodBuilder): PodBuilder = { + override def addPySparkFiles( + primaryFile: String, + pySparkFiles: String, + mainContainerName: String, + originalPodSpec: PodBuilder): PodBuilder = { + originalPodSpec .editSpec() - .editMatchingContainer(new ContainerNameEqualityPredicate(mainContainerName)) - .addNewEnv() - .withName(ENV_PYSPARK_PRIMARY) - .withValue(fileLoc(mainAppResource)) - .endEnv() - .addNewEnv() - .withName(ENV_PYSPARK_FILES) - .withValue(pythonFileLocations(pythonFiles, mainAppResource)) - .endEnv() - .endContainer() + .editMatchingContainer(new ContainerNameEqualityPredicate(mainContainerName)) + .addNewEnv() + .withName(ENV_PYSPARK_PRIMARY) + .withValue(primaryFile) + .endEnv() + .addNewEnv() + .withName(ENV_PYSPARK_FILES) + .withValue(pySparkFiles) + .endEnv() + .endContainer() .endSpec() } } diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/ClientV2Suite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/ClientV2Suite.scala index 644d2d911e269..14990b71110da 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/ClientV2Suite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/ClientV2Suite.scala @@ -171,7 +171,7 @@ class ClientV2Suite extends SparkFunSuite with BeforeAndAfter { .endMetadata() } }) - when(initContainerComponentsProvider.provideContainerLocalizedFilesResolver()) + when(initContainerComponentsProvider.provideContainerLocalizedFilesResolver(MAIN_CLASS)) .thenReturn(containerLocalizedFilesResolver) when(initContainerComponentsProvider.provideExecutorInitContainerConfiguration()) .thenReturn(executorInitContainerConfiguration) @@ -465,3 +465,5 @@ class ClientV2Suite extends SparkFunSuite with BeforeAndAfter { pod.getMetadata.getAnnotations.asScala == expectedAnnotations } } + + diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/ContainerLocalizedFilesResolverSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/ContainerLocalizedFilesResolverSuite.scala index ca5cd1fff9b74..a1eb1e91b6ef9 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/ContainerLocalizedFilesResolverSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/ContainerLocalizedFilesResolverSuite.scala @@ -29,11 +29,19 @@ class ContainerLocalizedFilesResolverSuite extends SparkFunSuite { "file:///app/files/file2.txt", "local:///app/files/file3.txt", "http://app/files/file4.txt") + private val PYSPARK_FILES = Seq( + "hdfs://localhost:9000/app/files/file1.py", + "file:///app/files/file2.py", + "local:///app/files/file3.py", + "http://app/files/file4.py") private val JARS_DOWNLOAD_PATH = "/var/data/spark-jars" private val FILES_DOWNLOAD_PATH = "/var/data/spark-files" + private val PYSPARK_PRIMARY_FILE = "file:///app/files/file4.py" private val localizedFilesResolver = new ContainerLocalizedFilesResolverImpl( SPARK_JARS, SPARK_FILES, + PYSPARK_FILES, + PYSPARK_PRIMARY_FILE, JARS_DOWNLOAD_PATH, FILES_DOWNLOAD_PATH) @@ -66,4 +74,18 @@ class ContainerLocalizedFilesResolverSuite extends SparkFunSuite { "http://app/files/file4.txt") assert(resolvedFiles === expectedResolvedFiles) } + test("Submitted PySpark files should resolve to the download path.") { + val resolvedPySparkFiles = localizedFilesResolver.resolveSubmittedPySparkFiles() + val expectedPySparkFiles = Seq( + "hdfs://localhost:9000/app/files/file1.py", + s"$FILES_DOWNLOAD_PATH/file2.py", + "local:///app/files/file3.py") + assert(resolvedPySparkFiles === expectedPySparkFiles) + } + test("Submitted PySpark Primary resource should resolve to the download path.") { + val resolvedPySparkPrimary = + localizedFilesResolver.resolvePrimaryResourceFile() + val expectedPySparkPrimary = s"$FILES_DOWNLOAD_PATH/file4.py" + assert(resolvedPySparkPrimary === expectedPySparkPrimary) + } } 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 cb0fe7332f79f..d18fbd74da4ca 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 @@ -342,8 +342,8 @@ private[spark] object KubernetesSuite { val SPARK_PI_MAIN_CLASS = "org.apache.spark.deploy.kubernetes" + ".integrationtest.jobs.SparkPiWithInfiniteWait" val PYSPARK_PI_MAIN_CLASS = "org.apache.spark.deploy.PythonRunner" - val PYSPARK_PI_CONTAINER_LOCAL_FILE_LOCATION = "local:///opt/spark/" + - "examples/src/main/python/pi.py" + val PYSPARK_PI_CONTAINER_LOCAL_FILE_LOCATION = + "local:///opt/spark/examples/src/main/python/pi.py" val PYSPARK_PI_SUBMITTER_LOCAL_FILE_LOCATION = "src/test/python/pi.py" val FILE_EXISTENCE_MAIN_CLASS = "org.apache.spark.deploy.kubernetes" + ".integrationtest.jobs.FileExistenceTest" From 51105caa715edf787bcdb9af175b2c129955a3a1 Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Mon, 19 Jun 2017 19:45:54 -0700 Subject: [PATCH 05/26] Modified Unit tests so that they would pass --- .../submit/ContainerLocalizedFilesResolverSuite.scala | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/ContainerLocalizedFilesResolverSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/ContainerLocalizedFilesResolverSuite.scala index a1eb1e91b6ef9..7e51abcd7b8e0 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/ContainerLocalizedFilesResolverSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/ContainerLocalizedFilesResolverSuite.scala @@ -33,10 +33,11 @@ class ContainerLocalizedFilesResolverSuite extends SparkFunSuite { "hdfs://localhost:9000/app/files/file1.py", "file:///app/files/file2.py", "local:///app/files/file3.py", - "http://app/files/file4.py") + "http://app/files/file4.py", + "file:///app/files/file5.py") private val JARS_DOWNLOAD_PATH = "/var/data/spark-jars" private val FILES_DOWNLOAD_PATH = "/var/data/spark-files" - private val PYSPARK_PRIMARY_FILE = "file:///app/files/file4.py" + private val PYSPARK_PRIMARY_FILE = "file:///app/files/file5.py" private val localizedFilesResolver = new ContainerLocalizedFilesResolverImpl( SPARK_JARS, SPARK_FILES, @@ -79,13 +80,14 @@ class ContainerLocalizedFilesResolverSuite extends SparkFunSuite { val expectedPySparkFiles = Seq( "hdfs://localhost:9000/app/files/file1.py", s"$FILES_DOWNLOAD_PATH/file2.py", - "local:///app/files/file3.py") + "local:///app/files/file3.py", + "http://app/files/file4.py") assert(resolvedPySparkFiles === expectedPySparkFiles) } test("Submitted PySpark Primary resource should resolve to the download path.") { val resolvedPySparkPrimary = localizedFilesResolver.resolvePrimaryResourceFile() - val expectedPySparkPrimary = s"$FILES_DOWNLOAD_PATH/file4.py" + val expectedPySparkPrimary = s"$FILES_DOWNLOAD_PATH/file5.py" assert(resolvedPySparkPrimary === expectedPySparkPrimary) } } From bd30f409fcab6f2d7b2a5b45ed10e5874b9029aa Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Mon, 19 Jun 2017 21:41:04 -0700 Subject: [PATCH 06/26] Modified Unit Test input to pass Unit Tests --- .../apache/spark/deploy/kubernetes/submit/ClientV2Suite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/ClientV2Suite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/ClientV2Suite.scala index 14990b71110da..4807f162df254 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/ClientV2Suite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/ClientV2Suite.scala @@ -171,7 +171,7 @@ class ClientV2Suite extends SparkFunSuite with BeforeAndAfter { .endMetadata() } }) - when(initContainerComponentsProvider.provideContainerLocalizedFilesResolver(MAIN_CLASS)) + when(initContainerComponentsProvider.provideContainerLocalizedFilesResolver("")) .thenReturn(containerLocalizedFilesResolver) when(initContainerComponentsProvider.provideExecutorInitContainerConfiguration()) .thenReturn(executorInitContainerConfiguration) From 720776ed42ab3aac8d9fe53a5bd5db867ac05868 Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Tue, 20 Jun 2017 19:13:45 -0700 Subject: [PATCH 07/26] Setup working environent for integration tests for PySpark --- .../kubernetes/integration-tests/pom.xml | 84 ++++++++++++++++++- .../integrationtest/KubernetesSuite.scala | 49 +++++------ .../docker/SparkDockerImageBuilder.scala | 15 ++++ 3 files changed, 123 insertions(+), 25 deletions(-) diff --git a/resource-managers/kubernetes/integration-tests/pom.xml b/resource-managers/kubernetes/integration-tests/pom.xml index a4f06bcf45808..cd3ccad0a2b22 100644 --- a/resource-managers/kubernetes/integration-tests/pom.xml +++ b/resource-managers/kubernetes/integration-tests/pom.xml @@ -231,7 +231,6 @@ ${project.parent.basedir}/python - true ${project.parent.basedir}/python/.egg ${project.parent.basedir}/python/dist @@ -240,6 +239,89 @@ + + copy-integration-data + pre-integration-test + + copy-resources + + + ${project.build.directory}/docker/data + + + ${project.parent.basedir}/data + true + + + + + + copy-integration-licenses + pre-integration-test + + copy-resources + + + ${project.build.directory}/docker/licenses + + + ${project.parent.basedir}/licenses + true + + + + + + copy-integration-examples-jar + pre-integration-test + + copy-resources + + + ${project.build.directory}/docker/examples/jars + + + ${project.parent.basedir}/examples/target/scala-2.11/jars + true + + + + + + copy-integration-examples-src + pre-integration-test + + copy-resources + + + ${project.build.directory}/docker/examples/src/main + + + ${project.parent.basedir}/examples/src/main + true + + + + + + + + org.apache.maven.plugins + maven-antrun-plugin + 1.6 + + + create-release-file + pre-integration-test + + run + + + + + + + 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 d18fbd74da4ca..1cb4488efeafd 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 @@ -72,30 +72,31 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { kubernetesTestComponents.deleteNamespace() } - /** - * These tests need to be run in an environment similair to the one provided - * by make_distribution. Further investigation is required - */ -// test("Run PySpark Job on file from SUBMITTER") { -// assume(testBackend.name == MINIKUBE_TEST_BACKEND) -// launchStagingServer(SSLOptions(), None) -// sparkConf.set(DRIVER_DOCKER_IMAGE, -// System.getProperty("spark.docker.test.driverImage", "spark-driver-py:latest")) -// sparkConf.set(EXECUTOR_DOCKER_IMAGE, -// System.getProperty("spark.docker.test.executorImage", "spark-executor-py:latest")) -// runPySparkPiAndVerifyCompletion( -// PYSPARK_PI_SUBMITTER_LOCAL_FILE_LOCATION) -// } -// test("Run PySpark Job on file from CONTAINER") { -// assume(testBackend.name == MINIKUBE_TEST_BACKEND) -// // UPDATE SO THAT IT BUILDS FROM LOCAL DOCKER IMAGE -// sparkConf.set(DRIVER_DOCKER_IMAGE, -// System.getProperty("spark.docker.test.driverImage", "spark-driver-py:latest")) -// sparkConf.set(EXECUTOR_DOCKER_IMAGE, -// System.getProperty("spark.docker.test.executorImage", "spark-executor-py:latest")) -// runPySparkPiAndVerifyCompletion( -// PYSPARK_PI_CONTAINER_LOCAL_FILE_LOCATION) -// } + test("Run PySpark Job on file from SUBMITTER") { + assume(testBackend.name == MINIKUBE_TEST_BACKEND) + + launchStagingServer(SSLOptions(), None) + sparkConf.set(DRIVER_DOCKER_IMAGE, + System.getProperty("spark.docker.test.driverImage", "spark-driver-py:latest")) + sparkConf.set(EXECUTOR_DOCKER_IMAGE, + System.getProperty("spark.docker.test.executorImage", "spark-executor-py:latest")) + + runPySparkPiAndVerifyCompletion( + PYSPARK_PI_SUBMITTER_LOCAL_FILE_LOCATION) + } + + test("Run PySpark Job on file from CONTAINER with spark.jar defined") { + assume(testBackend.name == MINIKUBE_TEST_BACKEND) + + sparkConf.setJars(Seq(CONTAINER_LOCAL_HELPER_JAR_PATH)) + sparkConf.set(DRIVER_DOCKER_IMAGE, + System.getProperty("spark.docker.test.driverImage", "spark-driver-py:latest")) + sparkConf.set(EXECUTOR_DOCKER_IMAGE, + System.getProperty("spark.docker.test.executorImage", "spark-executor-py:latest")) + + runPySparkPiAndVerifyCompletion( + PYSPARK_PI_CONTAINER_LOCAL_FILE_LOCATION) + } test("Simple submission test with the resource staging server.") { assume(testBackend.name == MINIKUBE_TEST_BACKEND) diff --git a/resource-managers/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 index faa29b91ff202..9c8039d2a1159 100644 --- a/resource-managers/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 @@ -23,6 +23,11 @@ import com.spotify.docker.client.{DefaultDockerClient, DockerCertificates, Loggi import org.apache.http.client.utils.URIBuilder import org.scalatest.concurrent.{Eventually, PatienceConfiguration} import org.scalatest.time.{Minutes, Seconds, Span} +import scala.collection.JavaConverters._ + +import org.apache.spark.SparkUserAppException +import org.apache.spark.util.RedirectThread + private[spark] class SparkDockerImageBuilder(private val dockerEnv: Map[String, String]) { @@ -63,6 +68,16 @@ private[spark] class SparkDockerImageBuilder(private val dockerEnv: Map[String, def buildSparkDockerImages(): Unit = { Eventually.eventually(TIMEOUT, INTERVAL) { dockerClient.ping() } + // Building Python distribution environment + val builder = new ProcessBuilder( + Seq("python", "setup.py", "sdist").asJava) + builder.directory(new java.io.File(s"$DOCKER_BUILD_PATH/python")) + val process = builder.start() + new RedirectThread(process.getInputStream, System.out, "redirect output").start() + val exitCode = process.waitFor() + if (exitCode != 0) { + throw new SparkUserAppException(exitCode) + } buildImage("spark-base", BASE_DOCKER_FILE) buildImage("spark-driver", DRIVER_DOCKER_FILE) buildImage("spark-driver-py", DRIVERPY_DOCKER_FILE) From 4b5f4701cb9e2a13fa7b9ef86fc613836da75f68 Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Tue, 20 Jun 2017 21:14:54 -0700 Subject: [PATCH 08/26] Comment out Python thread logic until Jenkins has python in Python --- .../docker/SparkDockerImageBuilder.scala | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/resource-managers/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 index 9c8039d2a1159..e19545885e1a4 100644 --- a/resource-managers/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 @@ -69,15 +69,15 @@ private[spark] class SparkDockerImageBuilder(private val dockerEnv: Map[String, def buildSparkDockerImages(): Unit = { Eventually.eventually(TIMEOUT, INTERVAL) { dockerClient.ping() } // Building Python distribution environment - val builder = new ProcessBuilder( - Seq("python", "setup.py", "sdist").asJava) - builder.directory(new java.io.File(s"$DOCKER_BUILD_PATH/python")) - val process = builder.start() - new RedirectThread(process.getInputStream, System.out, "redirect output").start() - val exitCode = process.waitFor() - if (exitCode != 0) { - throw new SparkUserAppException(exitCode) - } +// val builder = new ProcessBuilder( +// Seq("python", "setup.py", "sdist").asJava) +// builder.directory(new java.io.File(s"$DOCKER_BUILD_PATH/python")) +// val process = builder.start() +// new RedirectThread(process.getInputStream, System.out, "redirect output").start() +// val exitCode = process.waitFor() +// if (exitCode != 0) { +// throw new SparkUserAppException(exitCode) +// } buildImage("spark-base", BASE_DOCKER_FILE) buildImage("spark-driver", DRIVER_DOCKER_FILE) buildImage("spark-driver-py", DRIVERPY_DOCKER_FILE) From 1361a2644dea630f9766b2d16363fd08959b0c3e Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Wed, 21 Jun 2017 09:48:00 -0700 Subject: [PATCH 09/26] Modifying PythonExec to pass on Jenkins --- .../docker/SparkDockerImageBuilder.scala | 24 +++++++++++-------- 1 file changed, 14 insertions(+), 10 deletions(-) diff --git a/resource-managers/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 index e19545885e1a4..c8bed410fc28e 100644 --- a/resource-managers/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 @@ -25,7 +25,6 @@ import org.scalatest.concurrent.{Eventually, PatienceConfiguration} import org.scalatest.time.{Minutes, Seconds, Span} import scala.collection.JavaConverters._ -import org.apache.spark.SparkUserAppException import org.apache.spark.util.RedirectThread @@ -69,15 +68,20 @@ private[spark] class SparkDockerImageBuilder(private val dockerEnv: Map[String, def buildSparkDockerImages(): Unit = { Eventually.eventually(TIMEOUT, INTERVAL) { dockerClient.ping() } // Building Python distribution environment -// val builder = new ProcessBuilder( -// Seq("python", "setup.py", "sdist").asJava) -// builder.directory(new java.io.File(s"$DOCKER_BUILD_PATH/python")) -// val process = builder.start() -// new RedirectThread(process.getInputStream, System.out, "redirect output").start() -// val exitCode = process.waitFor() -// if (exitCode != 0) { -// throw new SparkUserAppException(exitCode) -// } + val pythonExec = sys.env.get("PYSPARK_DRIVER_PYTHON") + .orElse(sys.env.get("PYSPARK_PYTHON")) + .getOrElse("python") + val builder = new ProcessBuilder( + Seq(pythonExec, "setup.py", "sdist").asJava) + builder.directory(new java.io.File(s"$DOCKER_BUILD_PATH/python")) + val process = builder.start() + new RedirectThread(process.getInputStream, System.out, "redirect output").start() + val exitCode = process.waitFor() + if (exitCode != 0) { + // scalastyle:off println + println(s"exitCode: $exitCode") + // scalastyle:on println + } buildImage("spark-base", BASE_DOCKER_FILE) buildImage("spark-driver", DRIVER_DOCKER_FILE) buildImage("spark-driver-py", DRIVERPY_DOCKER_FILE) From 0abc3b1c1d3e655dddb55fb9173e32a4f5b4b020 Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Wed, 21 Jun 2017 10:46:59 -0700 Subject: [PATCH 10/26] Modifying python exec --- .../integrationtest/docker/SparkDockerImageBuilder.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/resource-managers/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 index c8bed410fc28e..2db67c96021b8 100644 --- a/resource-managers/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 @@ -70,10 +70,11 @@ private[spark] class SparkDockerImageBuilder(private val dockerEnv: Map[String, // Building Python distribution environment val pythonExec = sys.env.get("PYSPARK_DRIVER_PYTHON") .orElse(sys.env.get("PYSPARK_PYTHON")) - .getOrElse("python") + .getOrElse("/usr/bin/python") val builder = new ProcessBuilder( Seq(pythonExec, "setup.py", "sdist").asJava) builder.directory(new java.io.File(s"$DOCKER_BUILD_PATH/python")) + builder.redirectErrorStream(true) // Ugly but needed for stdout and stderr to synchronize val process = builder.start() new RedirectThread(process.getInputStream, System.out, "redirect output").start() val exitCode = process.waitFor() From 0869b077c6017980ed6b376d6a2c58d54f4e527a Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Thu, 22 Jun 2017 18:59:50 -0700 Subject: [PATCH 11/26] Added unit tests to ClientV2 and refactored to include pyspark submission resources --- .../deploy/kubernetes/submit/Client.scala | 100 ++++++----- .../submit/PythonSubmissionResources.scala | 60 +++++++ .../kubernetes/submit/ClientV2Suite.scala | 157 ++++++++++++++++-- .../integrationtest/KubernetesSuite.scala | 14 +- .../docker/SparkDockerImageBuilder.scala | 15 +- 5 files changed, 267 insertions(+), 79 deletions(-) create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/PythonSubmissionResources.scala diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/Client.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/Client.scala index 7b536e9fca73f..a33740ec33c01 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/Client.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/Client.scala @@ -48,13 +48,10 @@ private[spark] class Client( kubernetesResourceNamePrefix: String, kubernetesAppId: String, mainAppResource: String, - isPython: Boolean, + pythonResource: Option[PythonSubmissionResources], mainClass: String, sparkConf: SparkConf, appArgs: Array[String], - sparkJars: Seq[String], - sparkFiles: Seq[String], - pySparkFiles: List[String], waitForAppCompletion: Boolean, kubernetesClient: KubernetesClient, initContainerComponentsProvider: DriverInitContainerComponentsProvider, @@ -84,16 +81,10 @@ private[spark] class Client( org.apache.spark.internal.config.DRIVER_JAVA_OPTIONS) def run(): Unit = { - validateNoDuplicateFileNames(sparkJars) - validateNoDuplicateFileNames(sparkFiles) - if (isPython) {validateNoDuplicateFileNames(pySparkFiles)} - val arguments = if (isPython) pySparkFiles match { - case Nil => appArgs - case a::b => a match { - case _ if a==mainAppResource && b==Nil => appArgs - case _ => appArgs.drop(1) - } - } else appArgs + val arguments = pythonResource match { + case Some(p) => p.arguments + case None => appArgs + } val driverCustomLabels = ConfigurationUtils.combinePrefixedKeyValuePairsWithDeprecatedConf( sparkConf, KUBERNETES_DRIVER_LABEL_PREFIX, @@ -187,8 +178,10 @@ private[spark] class Client( val resolvedSparkJars = containerLocalizedFilesResolver.resolveSubmittedSparkJars() val resolvedSparkFiles = containerLocalizedFilesResolver.resolveSubmittedSparkFiles() val resolvedPySparkFiles = containerLocalizedFilesResolver.resolveSubmittedPySparkFiles() - val resolvedPrimaryPySparkResource = if (!isPython) "" - else { containerLocalizedFilesResolver.resolvePrimaryResourceFile() } + val resolvedPrimaryPySparkResource = pythonResource match { + case Some(p) => p.primarySparkResource(containerLocalizedFilesResolver) + case None => "" + } val executorInitContainerConfiguration = initContainerComponentsProvider .provideExecutorInitContainerConfiguration() val sparkConfWithExecutorInit = executorInitContainerConfiguration @@ -228,17 +221,15 @@ private[spark] class Client( .endEnv() .endContainer() .endSpec() - val resolvedDriverPod = if (!isPython) { - resolvedDriverPodBuilder.build() - } else { - initContainerComponentsProvider - .provideDriverPodFileMounter() - .addPySparkFiles( - resolvedPrimaryPySparkResource, - resolvedPySparkFiles.mkString(","), - driverContainer.getName, - resolvedDriverPodBuilder) - .build() + val resolvedDriverPod = pythonResource match { + case Some(p) => p.driverPod( + initContainerComponentsProvider, + resolvedPrimaryPySparkResource, + resolvedPySparkFiles.mkString(","), + driverContainer.getName, + resolvedDriverPodBuilder + ) + case None => resolvedDriverPodBuilder.build() } Utils.tryWithResource( kubernetesClient @@ -276,17 +267,6 @@ private[spark] class Client( } } } - - private def validateNoDuplicateFileNames(allFiles: Seq[String]): Unit = { - val fileNamesToUris = allFiles.map { file => - (new File(Utils.resolveURI(file).getPath).getName, file) - } - fileNamesToUris.groupBy(_._1).foreach { - case (fileName, urisWithFileName) => - require(urisWithFileName.size == 1, "Cannot add multiple files with the same name, but" + - s" file name $fileName is shared by all of these URIs: $urisWithFileName") - } - } } private[spark] object Client { @@ -303,12 +283,19 @@ private[spark] object Client { mainClass: String, appArgs: Array[String]): Unit = { val isPython = mainAppResource.endsWith(".py") + val pythonResource: Option[PythonSubmissionResources] = + if (isPython) { + Option(new PythonSubmissionResources(mainAppResource, appArgs)) + } else { + None + } // Since you might need jars for SQL UDFs in PySpark - def sparkJarFilter() : Seq[String] = isPython match { - case true => Seq.empty[String] - case false => Option(mainAppResource) - .filterNot(_ == SparkLauncher.NO_RESOURCE) - .toSeq + def sparkJarFilter() : Seq[String] = pythonResource match { + case Some(p) => p.sparkJars + case None => + Option(mainAppResource) + .filterNot(_ == SparkLauncher.NO_RESOURCE) + .toSeq } val sparkJars = sparkConf.getOption("spark.jars") .map(_.split(",")) @@ -317,11 +304,13 @@ private[spark] object Client { val sparkFiles = sparkConf.getOption("spark.files") .map(_.split(",")) .getOrElse(Array.empty[String]) - val pySparkFiles: Array[String] = if (isPython) { - Option(appArgs(0)) match { - case None => Array(mainAppResource) - case Some(a) => mainAppResource +: a.split(",") } - } else { Array.empty[String] } + val pySparkFiles: Array[String] = pythonResource match { + case Some(p) => p.pySparkFiles + case None => Array.empty[String] + } + validateNoDuplicateFileNames(sparkJars) + validateNoDuplicateFileNames(sparkFiles) + if (pythonResource.isDefined) {validateNoDuplicateFileNames(pySparkFiles)} val appName = sparkConf.getOption("spark.app.name").getOrElse("spark") // The resource name prefix is derived from the application name, making it easy to connect the // names of the Kubernetes resources from e.g. Kubectl or the Kubernetes dashboard to the @@ -362,13 +351,10 @@ private[spark] object Client { kubernetesResourceNamePrefix, kubernetesAppId, mainAppResource, - isPython, + pythonResource, mainClass, sparkConf, appArgs, - sparkJars, - sparkFiles, - pySparkFiles.toList, waitForAppCompletion, kubernetesClient, initContainerComponentsProvider, @@ -376,4 +362,14 @@ private[spark] object Client { loggingPodStatusWatcher).run() } } + private def validateNoDuplicateFileNames(allFiles: Seq[String]): Unit = { + val fileNamesToUris = allFiles.map { file => + (new File(Utils.resolveURI(file).getPath).getName, file) + } + fileNamesToUris.groupBy(_._1).foreach { + case (fileName, urisWithFileName) => + require(urisWithFileName.size == 1, "Cannot add multiple files with the same name, but" + + s" file name $fileName is shared by all of these URIs: $urisWithFileName") + } + } } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/PythonSubmissionResources.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/PythonSubmissionResources.scala new file mode 100644 index 0000000000000..59285f7d24791 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/PythonSubmissionResources.scala @@ -0,0 +1,60 @@ +/* + * 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.submit + +import io.fabric8.kubernetes.api.model.{Pod, PodBuilder} + +class PythonSubmissionResources( + private val mainAppResource: String, + private val appArgs: Array[String] ) { + + private val pyFiles: Array[String] = Option(appArgs(0)) match { + case None => Array(mainAppResource) + case Some(a) => mainAppResource +: a.split(",") + } + + def sparkJars: Seq[String] = Seq.empty[String] + + def pySparkFiles: Array[String] = pyFiles + + def arguments: Array[String] = + pyFiles.toList match { + case Nil => appArgs + case a :: b => a match { + case _ if a == mainAppResource && b == Nil => appArgs + case _ => appArgs.drop(1) + } + } + def primarySparkResource (containerLocalizedFilesResolver: ContainerLocalizedFilesResolver) + : String = containerLocalizedFilesResolver.resolvePrimaryResourceFile() + + def driverPod( + initContainerComponentsProvider: DriverInitContainerComponentsProvider, + resolvedPrimaryPySparkResource: String, + resolvedPySparkFiles: String, + driverContainerName: String, + driverPodBuilder: PodBuilder) : Pod = { + initContainerComponentsProvider + .provideDriverPodFileMounter() + .addPySparkFiles( + resolvedPrimaryPySparkResource, + resolvedPySparkFiles, + driverContainerName, + driverPodBuilder) + .build() + } +} \ No newline at end of file diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/ClientV2Suite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/ClientV2Suite.scala index 4807f162df254..d0ff87e1fe15b 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/ClientV2Suite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/ClientV2Suite.scala @@ -18,7 +18,7 @@ package org.apache.spark.deploy.kubernetes.submit import java.io.File -import io.fabric8.kubernetes.api.model.{ConfigMap, ConfigMapBuilder, DoneablePod, HasMetadata, Pod, PodBuilder, PodList, Secret, SecretBuilder} +import io.fabric8.kubernetes.api.model._ import io.fabric8.kubernetes.client.{KubernetesClient, Watch} import io.fabric8.kubernetes.client.dsl.{MixedOperation, NamespaceListVisitFromServerGetDeleteRecreateWaitApplicable, PodResource} import org.hamcrest.{BaseMatcher, Description} @@ -27,10 +27,10 @@ import org.mockito.Matchers.{any, anyVararg, argThat, eq => mockitoEq} import org.mockito.Mockito.{times, verify, when} import org.mockito.invocation.InvocationOnMock import org.mockito.stubbing.Answer -import org.scalatest.BeforeAndAfter +import org.scalatest.{BeforeAndAfter, Matchers} + import scala.collection.JavaConverters._ import scala.collection.mutable - import org.apache.spark.{SecurityManager, SparkConf, SparkFunSuite} import org.apache.spark.deploy.kubernetes.{KubernetesExternalShuffleService, KubernetesShuffleBlockHandler, SparkPodInitContainerBootstrap} import org.apache.spark.deploy.kubernetes.config._ @@ -63,6 +63,7 @@ class ClientV2Suite extends SparkFunSuite with BeforeAndAfter { private val INIT_CONTAINER_SECRET_NAME = "init-container-secret" private val INIT_CONTAINER_SECRET_DATA = Map("secret-key" -> "secret-data") private val MAIN_CLASS = "org.apache.spark.examples.SparkPi" + private val PYSPARK_APP_ARGS = Array(null, "500") private val APP_ARGS = Array("3", "20") private val SPARK_JARS = Seq( "hdfs://localhost:9000/app/jars/jar1.jar", "file:///app/jars/jar2.jar") @@ -72,6 +73,20 @@ class ClientV2Suite extends SparkFunSuite with BeforeAndAfter { "/var/data/spark-jars/jar1.jar", "/var/data/spark-jars/jar2.jar") private val SPARK_FILES = Seq( "hdfs://localhost:9000/app/files/file1.txt", "file:///app/files/file2.txt") + private val PYSPARK_FILES = Seq( + "hdfs://localhost:9000/app/files/file1.py", + "file:///app/files/file2.py", + "local:///app/files/file3.py", + "http://app/files/file4.py", + "file:///app/files/file5.py") + private val RESOLVED_PYSPARK_FILES = Seq( + "hdfs://localhost:9000/app/files/file1.py", + "/var/spark-data/spark-files/file2.py", + "local:///app/files/file3.py", + "http://app/files/file4.py") + private val PYSPARK_PRIMARY_FILE = "file:///app/files/file5.py" + private val RESOLVED_PYSPARK_PRIMARY_FILE = "/var/spark-data/spark-file/file5.py" + private val RESOLVED_SPARK_FILES = Seq( "hdfs://localhost:9000/app/files/file1.txt", "file:///var/data/spark-files/file2.txt") private val INIT_CONTAINER_SECRET = new SecretBuilder() @@ -140,14 +155,19 @@ class ClientV2Suite extends SparkFunSuite with BeforeAndAfter { @Mock private var kubernetesClient: KubernetesClient = _ @Mock - private var podOps: MixedOperation[Pod, PodList, DoneablePod, PodResource[Pod, DoneablePod]] = _ + private var podOps: MixedOperation[ + Pod, PodList, DoneablePod, PodResource[Pod, DoneablePod]] = _ private type ResourceListOps = NamespaceListVisitFromServerGetDeleteRecreateWaitApplicable[ HasMetadata, java.lang.Boolean] @Mock + private var pythonSubmissionResources : PythonSubmissionResources = _ + @Mock private var resourceListOps: ResourceListOps = _ @Mock private var credentialsMounterProvider: DriverPodKubernetesCredentialsMounterProvider = _ @Mock + private var fileMounter: DriverPodKubernetesFileMounter = _ + @Mock private var credentialsMounter: DriverPodKubernetesCredentialsMounter = _ @Mock private var loggingPodStatusWatcher: LoggingPodStatusWatcher = _ @@ -171,10 +191,12 @@ class ClientV2Suite extends SparkFunSuite with BeforeAndAfter { .endMetadata() } }) - when(initContainerComponentsProvider.provideContainerLocalizedFilesResolver("")) + when(initContainerComponentsProvider.provideContainerLocalizedFilesResolver(any[String])) .thenReturn(containerLocalizedFilesResolver) when(initContainerComponentsProvider.provideExecutorInitContainerConfiguration()) .thenReturn(executorInitContainerConfiguration) + when(initContainerComponentsProvider.provideDriverPodFileMounter()) + .thenReturn(fileMounter) when(submittedDependenciesSecretBuilder.build()) .thenReturn(INIT_CONTAINER_SECRET) when(initContainerConfigMapBuilder.build()) @@ -184,14 +206,63 @@ class ClientV2Suite extends SparkFunSuite with BeforeAndAfter { override def answer(invocation: InvocationOnMock): Pod = { new PodBuilder(invocation.getArgumentAt(0, classOf[Pod])) .editMetadata() - .withUid(DRIVER_POD_UID) + .withUid(DRIVER_POD_UID) .endMetadata() - .withKind(DRIVER_POD_KIND) + .withKind(DRIVER_POD_KIND) .withApiVersion(DRIVER_POD_API_VERSION) .build() } }) when(podOps.withName(s"$APP_RESOURCE_PREFIX-driver")).thenReturn(namedPodResource) + when(pythonSubmissionResources.sparkJars).thenReturn(Seq.empty[String]) + when(pythonSubmissionResources.primarySparkResource(any())) + .thenReturn(RESOLVED_PYSPARK_PRIMARY_FILE) + when(pythonSubmissionResources.pySparkFiles).thenReturn(PYSPARK_FILES.toArray) + when(pythonSubmissionResources.arguments).thenReturn(Array(PYSPARK_FILES.mkString(","), "500")) + when(pythonSubmissionResources.driverPod( + any[DriverInitContainerComponentsProvider], + mockitoEq(RESOLVED_PYSPARK_PRIMARY_FILE), + mockitoEq(RESOLVED_PYSPARK_FILES.mkString(",")), + any[String], + any[PodBuilder])).thenAnswer( new Answer[Pod] { + override def answer(invocation: InvocationOnMock) : Pod = { + invocation.getArgumentAt(0, classOf[DriverInitContainerComponentsProvider]) + .provideDriverPodFileMounter().addPySparkFiles( + invocation.getArgumentAt(1, classOf[String]), + invocation.getArgumentAt(2, classOf[String]), + invocation.getArgumentAt(3, classOf[String]), + invocation.getArgumentAt(4, classOf[PodBuilder]) + ).build() + } + }) + when(fileMounter.addPySparkFiles( + mockitoEq(RESOLVED_PYSPARK_PRIMARY_FILE), + mockitoEq(RESOLVED_PYSPARK_FILES.mkString(",")), + any[String], + any())).thenAnswer( new Answer[PodBuilder] { + override def answer(invocation: InvocationOnMock) : PodBuilder = { + invocation.getArgumentAt(3, classOf[PodBuilder]) + .editSpec() + .editMatchingContainer(new ContainerNameEqualityPredicate( + invocation.getArgumentAt(2, classOf[String]))) + .addNewEnv() + .withName(ENV_PYSPARK_PRIMARY) + .withValue(invocation.getArgumentAt(0, classOf[String])) + .endEnv() + .addNewEnv() + .withName(ENV_PYSPARK_FILES) + .withValue(invocation.getArgumentAt(1, classOf[String])) + .endEnv() + .endContainer() + .endSpec() + .editMetadata() + .withUid(DRIVER_POD_UID) + .withName(s"$APP_RESOURCE_PREFIX-driver") + .endMetadata() + .withKind(DRIVER_POD_KIND) + .withApiVersion(DRIVER_POD_API_VERSION) + } + }) when(namedPodResource.watch(loggingPodStatusWatcher)).thenReturn(watch) when(containerLocalizedFilesResolver.resolveSubmittedAndRemoteSparkJars()) .thenReturn(RESOLVED_SPARK_REMOTE_AND_LOCAL_JARS) @@ -199,6 +270,10 @@ class ClientV2Suite extends SparkFunSuite with BeforeAndAfter { .thenReturn(RESOLVED_SPARK_JARS) when(containerLocalizedFilesResolver.resolveSubmittedSparkFiles()) .thenReturn(RESOLVED_SPARK_FILES) + when(containerLocalizedFilesResolver.resolvePrimaryResourceFile()) + .thenReturn(RESOLVED_PYSPARK_PRIMARY_FILE) + when(containerLocalizedFilesResolver.resolveSubmittedPySparkFiles()) + .thenReturn(RESOLVED_PYSPARK_FILES) when(executorInitContainerConfiguration.configureSparkConfForExecutorInitContainer(SPARK_CONF)) .thenReturn(SPARK_CONF_WITH_EXECUTOR_INIT_CONF) when(kubernetesClient.resourceList(anyVararg[HasMetadata]())).thenReturn(resourceListOps) @@ -302,13 +377,10 @@ class ClientV2Suite extends SparkFunSuite with BeforeAndAfter { APP_RESOURCE_PREFIX, APP_ID, "", - false, + None, MAIN_CLASS, SPARK_CONF, APP_ARGS, - SPARK_JARS, - SPARK_FILES, - Nil, true, kubernetesClient, initContainerComponentsProvider, @@ -317,6 +389,20 @@ class ClientV2Suite extends SparkFunSuite with BeforeAndAfter { verify(loggingPodStatusWatcher).awaitCompletion() } + test("Mounting environmental variables correctly onto Driver Pod for PySpark Jobs") { + expectationsForNoMountedCredentials() + expectationsForNoDependencyUploader() + expectationsForNoSparkJarsOrFiles() + runAndVerifyDriverPodHasCorrectPySparkProperties() + } + + private def expectationsForNoSparkJarsOrFiles(): Unit = { + when(containerLocalizedFilesResolver.resolveSubmittedSparkFiles()) + .thenReturn(Nil) + when(containerLocalizedFilesResolver.resolveSubmittedSparkJars()) + .thenReturn(Nil) + } + private def expectationsForNoDependencyUploader(): Unit = { when(initContainerComponentsProvider .provideInitContainerSubmittedDependencyUploader(ALL_EXPECTED_LABELS)) @@ -384,19 +470,22 @@ class ClientV2Suite extends SparkFunSuite with BeforeAndAfter { } } + private def runAndVerifyDriverPodHasCorrectPySparkProperties(): Unit = { + runAndVerifyPySparkPodMatchesPredicate { p => + Option(p).exists(pod => containerHasCorrectPySparkEnvs(pod)) + } + } + private def runAndVerifyPodMatchesPredicate(pred: (Pod => Boolean)): Unit = { new Client( APP_NAME, APP_RESOURCE_PREFIX, APP_ID, "", - false, + None, MAIN_CLASS, SPARK_CONF, APP_ARGS, - SPARK_JARS, - SPARK_FILES, - Nil, false, kubernetesClient, initContainerComponentsProvider, @@ -448,6 +537,17 @@ class ClientV2Suite extends SparkFunSuite with BeforeAndAfter { expectedBasicEnvs.toSet.subsetOf(envs.toSet) } + private def containerHasCorrectPySparkEnvs(pod: Pod): Boolean = { + val driverContainer = pod.getSpec.getContainers.asScala.head + val envs = driverContainer.getEnv.asScala.map(env => (env.getName, env.getValue)) + val expectedBasicEnvs = Map( + ENV_PYSPARK_PRIMARY -> RESOLVED_PYSPARK_PRIMARY_FILE, + ENV_PYSPARK_FILES -> RESOLVED_PYSPARK_FILES.mkString(","), + ENV_DRIVER_ARGS -> (RESOLVED_PYSPARK_FILES :+ "500").mkString(",") + ) + expectedBasicEnvs.toSet.subsetOf(envs.toSet) + } + private def containerHasCorrectBasicContainerConfiguration(pod: Pod): Boolean = { val containers = pod.getSpec.getContainers.asScala containers.size == 1 && @@ -464,6 +564,33 @@ class ClientV2Suite extends SparkFunSuite with BeforeAndAfter { BOOTSTRAPPED_POD_ANNOTATION -> TRUE) pod.getMetadata.getAnnotations.asScala == expectedAnnotations } + + private def runAndVerifyPySparkPodMatchesPredicate(pred: (Pod => Boolean)): Unit = { + new Client( + APP_NAME, + APP_RESOURCE_PREFIX, + APP_ID, + PYSPARK_PRIMARY_FILE, + Option(pythonSubmissionResources), + MAIN_CLASS, + SPARK_CONF, + PYSPARK_APP_ARGS, + false, + kubernetesClient, + initContainerComponentsProvider, + credentialsMounterProvider, + loggingPodStatusWatcher).run() + val podMatcher = new BaseMatcher[Pod] { + override def matches(o: scala.Any): Boolean = { + o match { + case p: Pod => pred(p) + case _ => false + } + } + override def describeTo(description: Description): Unit = {} + } + verify(podOps).create(argThat(podMatcher)) + } } 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 1cb4488efeafd..d2082291eba22 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 @@ -76,10 +76,11 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { assume(testBackend.name == MINIKUBE_TEST_BACKEND) launchStagingServer(SSLOptions(), None) - sparkConf.set(DRIVER_DOCKER_IMAGE, - System.getProperty("spark.docker.test.driverImage", "spark-driver-py:latest")) - sparkConf.set(EXECUTOR_DOCKER_IMAGE, - System.getProperty("spark.docker.test.executorImage", "spark-executor-py:latest")) + sparkConf + .set(DRIVER_DOCKER_IMAGE, + System.getProperty("spark.docker.test.driverImage", "spark-driver-py:latest")) + .set(EXECUTOR_DOCKER_IMAGE, + System.getProperty("spark.docker.test.executorImage", "spark-executor-py:latest")) runPySparkPiAndVerifyCompletion( PYSPARK_PI_SUBMITTER_LOCAL_FILE_LOCATION) @@ -89,9 +90,10 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { assume(testBackend.name == MINIKUBE_TEST_BACKEND) sparkConf.setJars(Seq(CONTAINER_LOCAL_HELPER_JAR_PATH)) - sparkConf.set(DRIVER_DOCKER_IMAGE, + sparkConf + .set(DRIVER_DOCKER_IMAGE, System.getProperty("spark.docker.test.driverImage", "spark-driver-py:latest")) - sparkConf.set(EXECUTOR_DOCKER_IMAGE, + .set(EXECUTOR_DOCKER_IMAGE, System.getProperty("spark.docker.test.executorImage", "spark-executor-py:latest")) runPySparkPiAndVerifyCompletion( diff --git a/resource-managers/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 index 2db67c96021b8..e240fcf953f8c 100644 --- a/resource-managers/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 @@ -16,19 +16,24 @@ */ package org.apache.spark.deploy.kubernetes.integrationtest.docker +import java.io.File import java.net.URI import java.nio.file.Paths +import scala.collection.JavaConverters._ + import com.spotify.docker.client.{DefaultDockerClient, DockerCertificates, LoggingBuildHandler} import org.apache.http.client.utils.URIBuilder import org.scalatest.concurrent.{Eventually, PatienceConfiguration} import org.scalatest.time.{Minutes, Seconds, Span} -import scala.collection.JavaConverters._ +import org.apache.spark.internal.Logging import org.apache.spark.util.RedirectThread -private[spark] class SparkDockerImageBuilder(private val dockerEnv: Map[String, String]) { + +private[spark] class SparkDockerImageBuilder + (private val dockerEnv: Map[String, String]) extends Logging{ private val DOCKER_BUILD_PATH = Paths.get("target", "docker") // Dockerfile paths must be relative to the build path. @@ -73,15 +78,13 @@ private[spark] class SparkDockerImageBuilder(private val dockerEnv: Map[String, .getOrElse("/usr/bin/python") val builder = new ProcessBuilder( Seq(pythonExec, "setup.py", "sdist").asJava) - builder.directory(new java.io.File(s"$DOCKER_BUILD_PATH/python")) + builder.directory(new File(DOCKER_BUILD_PATH.toFile, "python")) builder.redirectErrorStream(true) // Ugly but needed for stdout and stderr to synchronize val process = builder.start() new RedirectThread(process.getInputStream, System.out, "redirect output").start() val exitCode = process.waitFor() if (exitCode != 0) { - // scalastyle:off println - println(s"exitCode: $exitCode") - // scalastyle:on println + logInfo(s"exitCode: $exitCode") } buildImage("spark-base", BASE_DOCKER_FILE) buildImage("spark-driver", DRIVER_DOCKER_FILE) From 9bf7b9daa6ed3287a2b6d18b84b214aea9a07aad Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Thu, 22 Jun 2017 19:35:15 -0700 Subject: [PATCH 12/26] Modified unit test check --- .../apache/spark/deploy/kubernetes/submit/ClientV2Suite.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/ClientV2Suite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/ClientV2Suite.scala index d0ff87e1fe15b..3180025b98788 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/ClientV2Suite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/ClientV2Suite.scala @@ -542,8 +542,7 @@ class ClientV2Suite extends SparkFunSuite with BeforeAndAfter { val envs = driverContainer.getEnv.asScala.map(env => (env.getName, env.getValue)) val expectedBasicEnvs = Map( ENV_PYSPARK_PRIMARY -> RESOLVED_PYSPARK_PRIMARY_FILE, - ENV_PYSPARK_FILES -> RESOLVED_PYSPARK_FILES.mkString(","), - ENV_DRIVER_ARGS -> (RESOLVED_PYSPARK_FILES :+ "500").mkString(",") + ENV_PYSPARK_FILES -> RESOLVED_PYSPARK_FILES.mkString(",") ) expectedBasicEnvs.toSet.subsetOf(envs.toSet) } From 4561194a2479ac2f1960a46dba0cb54842e1dc65 Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Fri, 23 Jun 2017 09:36:46 -0700 Subject: [PATCH 13/26] Scalastyle --- .../deploy/kubernetes/submit/PythonSubmissionResources.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/PythonSubmissionResources.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/PythonSubmissionResources.scala index 59285f7d24791..3cc3c2c37b86f 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/PythonSubmissionResources.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/PythonSubmissionResources.scala @@ -57,4 +57,4 @@ class PythonSubmissionResources( driverPodBuilder) .build() } -} \ No newline at end of file +} From eb1079a9ebaabd5542bf5cc3a009607089892404 Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Fri, 23 Jun 2017 12:29:53 -0700 Subject: [PATCH 14/26] PR 348 file conflicts --- .../apache/spark/deploy/kubernetes/submit/Client.scala | 6 ++---- .../submit/DriverInitContainerComponentsProvider.scala | 10 +++++----- .../spark/deploy/kubernetes/submit/ClientV2Suite.scala | 6 +++--- 3 files changed, 10 insertions(+), 12 deletions(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/Client.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/Client.scala index 618107eb0e60c..1e5854f203493 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/Client.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/Client.scala @@ -194,7 +194,7 @@ private[spark] class Client( val initContainerBundler = initContainerComponentsProvider .provideInitContainerBundle(maybeSubmittedResourceIdentifiers.map(_.ids()), - resolvedSparkJars ++ resolvedSparkFiles, mainAppResource) + resolvedSparkJars ++ resolvedSparkFiles) val podWithInitContainer = initContainerBundler.map( _.sparkPodInitContainerBootstrap @@ -305,9 +305,7 @@ private[spark] object Client { val pythonResource: Option[PythonSubmissionResources] = if (isPython) { Option(new PythonSubmissionResources(mainAppResource, appArgs)) - } else { - None - } + } else None // Since you might need jars for SQL UDFs in PySpark def sparkJarFilter() : Seq[String] = pythonResource match { case Some(p) => p.sparkJars diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/DriverInitContainerComponentsProvider.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/DriverInitContainerComponentsProvider.scala index 1fa4e18cb9850..7f5a62e97abc0 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/DriverInitContainerComponentsProvider.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/DriverInitContainerComponentsProvider.scala @@ -42,7 +42,7 @@ private[spark] trait DriverInitContainerComponentsProvider { def provideInitContainerBootstrap(): SparkPodInitContainerBootstrap def provideDriverPodFileMounter(): DriverPodKubernetesFileMounter def provideInitContainerBundle(maybeSubmittedResourceIds: Option[SubmittedResourceIds], - uris: Iterable[String], mainAppResource: String): Option[InitContainerBundle] + uris: Iterable[String]): Option[InitContainerBundle] } private[spark] class DriverInitContainerComponentsProviderImpl( @@ -211,10 +211,10 @@ private[spark] class DriverInitContainerComponentsProviderImpl( } override def provideInitContainerBundle( maybeSubmittedResourceIds: Option[SubmittedResourceIds], - uris: Iterable[String], mainAppResource: String): Option[InitContainerBundle] = { - val containerLocalizedFilesResolver = provideContainerLocalizedFilesResolver(mainAppResource) - // Bypass init-containers if `spark.jars` and `spark.files` is empty or only has `local://` URIs - if (KubernetesFileUtils.getNonContainerLocalFiles(uris).nonEmpty) { + uris: Iterable[String]): Option[InitContainerBundle] = { + // Bypass init-containers if `spark.jars` and `spark.files` and '--py-rilfes' + // is empty or only has `local://` URIs + if ((KubernetesFileUtils.getNonContainerLocalFiles(uris) ++ pySparkSubmitted).nonEmpty) { Some(InitContainerBundle(provideInitContainerConfigMap(maybeSubmittedResourceIds), provideInitContainerBootstrap(), provideExecutorInitContainerConfiguration())) diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/ClientV2Suite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/ClientV2Suite.scala index 9efcbff41f4d2..3f1febf205654 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/ClientV2Suite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/ClientV2Suite.scala @@ -285,7 +285,7 @@ class ClientV2Suite extends SparkFunSuite with BeforeAndAfter { .thenReturn(Some(submittedDependenciesSecretBuilder)) when(initContainerComponentsProvider.provideInitContainerBundle(mockitoEq( Option(SUBMITTED_RESOURCES.ids())), - mockitoEq(RESOLVED_SPARK_JARS ++ RESOLVED_SPARK_FILES), any[String])) + mockitoEq(RESOLVED_SPARK_JARS ++ RESOLVED_SPARK_FILES))) .thenReturn(Option(InitContainerBundle(INIT_CONTAINER_CONFIG_MAP, initContainerBootstrap, executorInitContainerConfiguration))) runAndVerifyDriverPodHasCorrectProperties() @@ -403,7 +403,7 @@ class ClientV2Suite extends SparkFunSuite with BeforeAndAfter { .provideSubmittedDependenciesSecretBuilder(None)) .thenReturn(None) when(initContainerComponentsProvider.provideInitContainerBundle(mockitoEq(None), - mockitoEq(RESOLVED_SPARK_JARS ++ RESOLVED_SPARK_FILES), any[String])) + mockitoEq(RESOLVED_SPARK_JARS ++ RESOLVED_SPARK_FILES))) .thenReturn(Some(InitContainerBundle(INIT_CONTAINER_CONFIG_MAP, initContainerBootstrap, executorInitContainerConfiguration))) } @@ -465,7 +465,7 @@ class ClientV2Suite extends SparkFunSuite with BeforeAndAfter { private def runAndVerifyDriverPodHasCorrectPySparkProperties(): Unit = { when(initContainerComponentsProvider.provideInitContainerBundle( - any[Option[SubmittedResourceIds]], any[Iterable[String]], any[String])) + any[Option[SubmittedResourceIds]], any[Iterable[String]])) .thenReturn(Some(InitContainerBundle(INIT_CONTAINER_CONFIG_MAP, initContainerBootstrap, executorInitContainerConfiguration))) runAndVerifyPySparkPodMatchesPredicate { p => From 4a6b779b7ff2cc5ddad7f0bd415a3b613020a5a1 Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Tue, 27 Jun 2017 18:06:47 -0700 Subject: [PATCH 15/26] Refactored unit tests and styles --- .../deploy/kubernetes/submit/Client.scala | 16 +-- .../ContainerLocalizedFilesResolver.scala | 8 +- .../submit/PythonSubmissionResources.scala | 38 +++-- .../kubernetes/submit/ClientV2Suite.scala | 42 ++---- .../PythonSubmissionResourcesSuite.scala | 133 ++++++++++++++++++ 5 files changed, 177 insertions(+), 60 deletions(-) create mode 100644 resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/PythonSubmissionResourcesSuite.scala diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/Client.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/Client.scala index 1e5854f203493..d583d3a0565c6 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/Client.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/Client.scala @@ -48,7 +48,7 @@ private[spark] class Client( kubernetesResourceNamePrefix: String, kubernetesAppId: String, mainAppResource: String, - pythonResource: Option[PythonSubmissionResources], + pythonResource: Option[PythonSubmissionResourcesImpl], mainClass: String, sparkConf: SparkConf, appArgs: Array[String], @@ -188,7 +188,7 @@ private[spark] class Client( val resolvedSparkFiles = containerLocalizedFilesResolver.resolveSubmittedSparkFiles() val resolvedPySparkFiles = containerLocalizedFilesResolver.resolveSubmittedPySparkFiles() val resolvedPrimaryPySparkResource = pythonResource match { - case Some(p) => p.primarySparkResource(containerLocalizedFilesResolver) + case Some(p) => p.primaryPySparkResource(containerLocalizedFilesResolver) case None => "" } @@ -302,9 +302,9 @@ private[spark] object Client { mainClass: String, appArgs: Array[String]): Unit = { val isPython = mainAppResource.endsWith(".py") - val pythonResource: Option[PythonSubmissionResources] = + val pythonResource: Option[PythonSubmissionResourcesImpl] = if (isPython) { - Option(new PythonSubmissionResources(mainAppResource, appArgs)) + Option(new PythonSubmissionResourcesImpl(mainAppResource, appArgs)) } else None // Since you might need jars for SQL UDFs in PySpark def sparkJarFilter() : Seq[String] = pythonResource match { @@ -321,13 +321,11 @@ private[spark] object Client { val sparkFiles = sparkConf.getOption("spark.files") .map(_.split(",")) .getOrElse(Array.empty[String]) - val pySparkFiles: Array[String] = pythonResource match { - case Some(p) => p.pySparkFiles - case None => Array.empty[String] - } + val pySparkFilesOption = pythonResource map {p => p.pySparkFiles} validateNoDuplicateFileNames(sparkJars) validateNoDuplicateFileNames(sparkFiles) - if (pythonResource.isDefined) {validateNoDuplicateFileNames(pySparkFiles)} + pySparkFilesOption foreach {b => validateNoDuplicateFileNames(b)} + val pySparkFiles = pySparkFilesOption.getOrElse(Array.empty[String]) val appName = sparkConf.getOption("spark.app.name").getOrElse("spark") // The resource name prefix is derived from the application name, making it easy to connect the // names of the Kubernetes resources from e.g. Kubectl or the Kubernetes dashboard to the diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/ContainerLocalizedFilesResolver.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/ContainerLocalizedFilesResolver.scala index 999fe6d7e38a0..7345589c59c42 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/ContainerLocalizedFilesResolver.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/ContainerLocalizedFilesResolver.scala @@ -34,8 +34,7 @@ private[spark] class ContainerLocalizedFilesResolverImpl( pySparkFiles: Seq[String], primaryPyFile: String, jarsDownloadPath: String, - filesDownloadPath: String - ) extends ContainerLocalizedFilesResolver { + filesDownloadPath: String ) extends ContainerLocalizedFilesResolver { override def resolveSubmittedAndRemoteSparkJars(): Seq[String] = { @@ -68,10 +67,7 @@ private[spark] class ContainerLocalizedFilesResolverImpl( } override def resolvePrimaryResourceFile(): String = { - Option(primaryPyFile) match { - case None => "" - case Some(p) => resolveFile(p, filesDownloadPath) - } + Option(primaryPyFile).map(p => resolveFile(p, filesDownloadPath)).getOrElse("") } private def resolveFile(file: String, downloadPath: String) = { diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/PythonSubmissionResources.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/PythonSubmissionResources.scala index 3cc3c2c37b86f..a20d93080c4e1 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/PythonSubmissionResources.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/PythonSubmissionResources.scala @@ -18,20 +18,34 @@ package org.apache.spark.deploy.kubernetes.submit import io.fabric8.kubernetes.api.model.{Pod, PodBuilder} -class PythonSubmissionResources( +private[spark] trait PythonSubmissionResources { + def sparkJars: Seq[String] + def pySparkFiles: Array[String] + def arguments: Array[String] + def primaryPySparkResource(containerLocalizedFilesResolver: ContainerLocalizedFilesResolver) + : String + def driverPod( + initContainerComponentsProvider: DriverInitContainerComponentsProvider, + resolvedPrimaryPySparkResource: String, + resolvedPySparkFiles: String, + driverContainerName: String, + driverPodBuilder: PodBuilder) : Pod +} + +private[spark] class PythonSubmissionResourcesImpl( private val mainAppResource: String, - private val appArgs: Array[String] ) { + private val appArgs: Array[String] ) extends PythonSubmissionResources { - private val pyFiles: Array[String] = Option(appArgs(0)) match { - case None => Array(mainAppResource) - case Some(a) => mainAppResource +: a.split(",") + private val pyFiles: Array[String] = { + (Option(appArgs(0)) map (a => mainAppResource +: a.split(","))) + .getOrElse(Array(mainAppResource)) } - def sparkJars: Seq[String] = Seq.empty[String] + override def sparkJars: Seq[String] = Seq.empty[String] - def pySparkFiles: Array[String] = pyFiles + override def pySparkFiles: Array[String] = pyFiles - def arguments: Array[String] = + override def arguments: Array[String] = { pyFiles.toList match { case Nil => appArgs case a :: b => a match { @@ -39,10 +53,12 @@ class PythonSubmissionResources( case _ => appArgs.drop(1) } } - def primarySparkResource (containerLocalizedFilesResolver: ContainerLocalizedFilesResolver) - : String = containerLocalizedFilesResolver.resolvePrimaryResourceFile() + } + override def primaryPySparkResource ( + containerLocalizedFilesResolver: ContainerLocalizedFilesResolver) : String = + containerLocalizedFilesResolver.resolvePrimaryResourceFile() - def driverPod( + override def driverPod( initContainerComponentsProvider: DriverInitContainerComponentsProvider, resolvedPrimaryPySparkResource: String, resolvedPySparkFiles: String, diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/ClientV2Suite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/ClientV2Suite.scala index 3f1febf205654..fe04fddc746bd 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/ClientV2Suite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/ClientV2Suite.scala @@ -158,8 +158,6 @@ class ClientV2Suite extends SparkFunSuite with BeforeAndAfter { private type ResourceListOps = NamespaceListVisitFromServerGetDeleteRecreateWaitApplicable[ HasMetadata, java.lang.Boolean] @Mock - private var pythonSubmissionResources : PythonSubmissionResources = _ - @Mock private var resourceListOps: ResourceListOps = _ @Mock private var credentialsMounterProvider: DriverPodKubernetesCredentialsMounterProvider = _ @@ -189,8 +187,8 @@ class ClientV2Suite extends SparkFunSuite with BeforeAndAfter { .endMetadata() } }) - when(initContainerComponentsProvider.provideContainerLocalizedFilesResolver(any[String])) - .thenReturn(containerLocalizedFilesResolver) + when(initContainerComponentsProvider.provideContainerLocalizedFilesResolver( + any[String])).thenReturn(containerLocalizedFilesResolver) when(initContainerComponentsProvider.provideDriverPodFileMounter()) .thenReturn(fileMounter) when(submittedDependenciesSecretBuilder.build()) @@ -208,27 +206,6 @@ class ClientV2Suite extends SparkFunSuite with BeforeAndAfter { } }) when(podOps.withName(s"$APP_RESOURCE_PREFIX-driver")).thenReturn(namedPodResource) - when(pythonSubmissionResources.sparkJars).thenReturn(Seq.empty[String]) - when(pythonSubmissionResources.primarySparkResource(any())) - .thenReturn(RESOLVED_PYSPARK_PRIMARY_FILE) - when(pythonSubmissionResources.pySparkFiles).thenReturn(PYSPARK_FILES.toArray) - when(pythonSubmissionResources.arguments).thenReturn(Array(PYSPARK_FILES.mkString(","), "500")) - when(pythonSubmissionResources.driverPod( - any[DriverInitContainerComponentsProvider], - mockitoEq(RESOLVED_PYSPARK_PRIMARY_FILE), - mockitoEq(RESOLVED_PYSPARK_FILES.mkString(",")), - any[String], - any[PodBuilder])).thenAnswer( new Answer[Pod] { - override def answer(invocation: InvocationOnMock) : Pod = { - invocation.getArgumentAt(0, classOf[DriverInitContainerComponentsProvider]) - .provideDriverPodFileMounter().addPySparkFiles( - invocation.getArgumentAt(1, classOf[String]), - invocation.getArgumentAt(2, classOf[String]), - invocation.getArgumentAt(3, classOf[String]), - invocation.getArgumentAt(4, classOf[PodBuilder]) - ).build() - } - }) when(fileMounter.addPySparkFiles( mockitoEq(RESOLVED_PYSPARK_PRIMARY_FILE), mockitoEq(RESOLVED_PYSPARK_FILES.mkString(",")), @@ -240,12 +217,8 @@ class ClientV2Suite extends SparkFunSuite with BeforeAndAfter { .editMatchingContainer(new ContainerNameEqualityPredicate( invocation.getArgumentAt(2, classOf[String]))) .addNewEnv() - .withName(ENV_PYSPARK_PRIMARY) - .withValue(invocation.getArgumentAt(0, classOf[String])) - .endEnv() - .addNewEnv() - .withName(ENV_PYSPARK_FILES) - .withValue(invocation.getArgumentAt(1, classOf[String])) + .withName("pyspark") + .withValue("true") .endEnv() .endContainer() .endSpec() @@ -464,6 +437,8 @@ class ClientV2Suite extends SparkFunSuite with BeforeAndAfter { } private def runAndVerifyDriverPodHasCorrectPySparkProperties(): Unit = { + when(initContainerComponentsProvider.provideContainerLocalizedFilesResolver( + mockitoEq(PYSPARK_PRIMARY_FILE))).thenReturn(containerLocalizedFilesResolver) when(initContainerComponentsProvider.provideInitContainerBundle( any[Option[SubmittedResourceIds]], any[Iterable[String]])) .thenReturn(Some(InitContainerBundle(INIT_CONTAINER_CONFIG_MAP, @@ -538,8 +513,7 @@ class ClientV2Suite extends SparkFunSuite with BeforeAndAfter { val driverContainer = pod.getSpec.getContainers.asScala.head val envs = driverContainer.getEnv.asScala.map(env => (env.getName, env.getValue)) val expectedBasicEnvs = Map( - ENV_PYSPARK_PRIMARY -> RESOLVED_PYSPARK_PRIMARY_FILE, - ENV_PYSPARK_FILES -> RESOLVED_PYSPARK_FILES.mkString(",") + "pyspark" -> "true" ) expectedBasicEnvs.toSet.subsetOf(envs.toSet) } @@ -567,7 +541,7 @@ class ClientV2Suite extends SparkFunSuite with BeforeAndAfter { APP_RESOURCE_PREFIX, APP_ID, PYSPARK_PRIMARY_FILE, - Option(pythonSubmissionResources), + Option(new PythonSubmissionResourcesImpl(PYSPARK_PRIMARY_FILE, PYSPARK_APP_ARGS)), MAIN_CLASS, SPARK_CONF, PYSPARK_APP_ARGS, diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/PythonSubmissionResourcesSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/PythonSubmissionResourcesSuite.scala new file mode 100644 index 0000000000000..bb83bee667ce4 --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/PythonSubmissionResourcesSuite.scala @@ -0,0 +1,133 @@ +/* + * 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.submit + +import org.apache.spark.{SSLOptions, SparkConf, SparkFunSuite} +import org.apache.spark.deploy.kubernetes.config._ +import scala.collection.JavaConverters._ + +import io.fabric8.kubernetes.api.model.{ContainerBuilder, Pod, PodBuilder} + + + +private[spark] class PythonSubmissionResourcesSuite extends SparkFunSuite { + private val PYSPARK_FILES = Seq( + "hdfs://localhost:9000/app/files/file1.py", + "file:///app/files/file2.py", + "local:///app/files/file3.py", + "http://app/files/file4.py", + "file:///app/files/file5.py") + private val RESOLVED_PYSPARK_FILES = Seq( + "hdfs://localhost:9000/app/files/file1.py", + "/var/spark-data/spark-files/file2.py", + "local:///app/file`s/file3.py", + "http://app/files/file4.py") + private val PYSPARK_PRIMARY_FILE = "file:///app/files/file5.py" + private val RESOLVED_PYSPARK_PRIMARY_FILE = "/var/data/spark-files/file5.py" + + private val pyFilesResource = new PythonSubmissionResourcesImpl( + PYSPARK_PRIMARY_FILE, Array(PYSPARK_FILES.mkString(","), "500") + ) + private val pyResource = new PythonSubmissionResourcesImpl( + PYSPARK_PRIMARY_FILE, Array(null, "500") + ) + private val SPARK_FILES = Seq.empty[String] + private val SPARK_JARS = Seq.empty[String] + private val JARS_DOWNLOAD_PATH = "/var/data/spark-jars" + private val FILES_DOWNLOAD_PATH = "/var/data/spark-files" + private val localizedFilesResolver = new ContainerLocalizedFilesResolverImpl( + SPARK_JARS, + SPARK_FILES, + PYSPARK_FILES, + PYSPARK_PRIMARY_FILE, + JARS_DOWNLOAD_PATH, + FILES_DOWNLOAD_PATH) + private val lessLocalizedFilesResolver = new ContainerLocalizedFilesResolverImpl( + SPARK_JARS, + SPARK_FILES, + Seq.empty[String], + PYSPARK_PRIMARY_FILE, + JARS_DOWNLOAD_PATH, + FILES_DOWNLOAD_PATH) + private val NAMESPACE = "example_pyspark" + private val DRIVER_CONTAINER_NAME = "pyspark_container" + private val driverContainer = new ContainerBuilder() + .withName(DRIVER_CONTAINER_NAME) + .build() + private val basePodBuilder = new PodBuilder() + .withNewMetadata() + .withName("base_pod") + .endMetadata() + .withNewSpec() + .addToContainers(driverContainer) + .endSpec() + private val driverInitcontainer = new DriverInitContainerComponentsProviderImpl( + new SparkConf(true).set(KUBERNETES_NAMESPACE, NAMESPACE), + "kubeResourceName", + "namespace", + SPARK_JARS, + SPARK_FILES, + PYSPARK_PRIMARY_FILE +: PYSPARK_FILES, + SSLOptions() + ) + private val lessDriverInitcontainer = new DriverInitContainerComponentsProviderImpl( + new SparkConf(true).set(KUBERNETES_NAMESPACE, NAMESPACE), + "kubeResourceName", + "namespace", + SPARK_JARS, + SPARK_FILES, + Array(PYSPARK_PRIMARY_FILE), + SSLOptions() + ) + + test("Test with --py-files included") { + assert(pyFilesResource.sparkJars === Seq.empty[String]) + assert(pyFilesResource.pySparkFiles === + PYSPARK_PRIMARY_FILE +: PYSPARK_FILES) + assert(pyFilesResource.primaryPySparkResource(localizedFilesResolver) === + RESOLVED_PYSPARK_PRIMARY_FILE) + val driverPod: Pod = pyFilesResource.driverPod( + driverInitcontainer, + RESOLVED_PYSPARK_PRIMARY_FILE, + RESOLVED_PYSPARK_FILES.mkString(","), + DRIVER_CONTAINER_NAME, + basePodBuilder + ) + val driverContainer = driverPod.getSpec.getContainers.asScala.head + val envs = driverContainer.getEnv.asScala.map(env => (env.getName, env.getValue)).toMap + envs.get("PYSPARK_PRIMARY") foreach{ a => assert (a === RESOLVED_PYSPARK_PRIMARY_FILE) } + envs.get("PYSPARK_FILES") foreach{ a => assert (a === RESOLVED_PYSPARK_FILES.mkString(",")) } + } + + test("Test without --py-files") { + assert(pyResource.sparkJars === Seq.empty[String]) + assert(pyResource.pySparkFiles === Array(PYSPARK_PRIMARY_FILE)) + assert(pyResource.primaryPySparkResource(lessLocalizedFilesResolver) === + RESOLVED_PYSPARK_PRIMARY_FILE) + val driverPod: Pod = pyResource.driverPod( + lessDriverInitcontainer, + RESOLVED_PYSPARK_PRIMARY_FILE, + "", + DRIVER_CONTAINER_NAME, + basePodBuilder + ) + val driverContainer = driverPod.getSpec.getContainers.asScala.head + val envs = driverContainer.getEnv.asScala.map(env => (env.getName, env.getValue)).toMap + envs.get("PYSPARK_PRIMARY") foreach{ a => assert (a === RESOLVED_PYSPARK_PRIMARY_FILE) } + envs.get("PYSPARK_FILES") foreach{ a => assert (a === "") } + } +} \ No newline at end of file From 363919a49648a5e3c7c8735b6008a233e38ea612 Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Wed, 28 Jun 2017 11:15:28 -0700 Subject: [PATCH 16/26] further scala stylzing and logic --- .../deploy/kubernetes/submit/Client.scala | 32 +++++++------------ .../submit/PythonSubmissionResources.scala | 7 ++-- .../PythonSubmissionResourcesSuite.scala | 12 +++---- 3 files changed, 21 insertions(+), 30 deletions(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/Client.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/Client.scala index d583d3a0565c6..882672a31ad13 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/Client.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/Client.scala @@ -82,10 +82,7 @@ private[spark] class Client( org.apache.spark.internal.config.DRIVER_JAVA_OPTIONS) def run(): Unit = { - val arguments = pythonResource match { - case Some(p) => p.arguments - case None => appArgs - } + val arguments = (pythonResource map {p => p.arguments}).getOrElse(appArgs) val driverCustomLabels = ConfigurationUtils.combinePrefixedKeyValuePairsWithDeprecatedConf( sparkConf, KUBERNETES_DRIVER_LABEL_PREFIX, @@ -187,11 +184,9 @@ private[spark] class Client( val resolvedSparkJars = containerLocalizedFilesResolver.resolveSubmittedSparkJars() val resolvedSparkFiles = containerLocalizedFilesResolver.resolveSubmittedSparkFiles() val resolvedPySparkFiles = containerLocalizedFilesResolver.resolveSubmittedPySparkFiles() - val resolvedPrimaryPySparkResource = pythonResource match { - case Some(p) => p.primaryPySparkResource(containerLocalizedFilesResolver) - case None => "" - } - + val resolvedPrimaryPySparkResource = (pythonResource map { + p => p.primaryPySparkResource(containerLocalizedFilesResolver) + }).getOrElse("") val initContainerBundler = initContainerComponentsProvider .provideInitContainerBundle(maybeSubmittedResourceIdentifiers.map(_.ids()), resolvedSparkJars ++ resolvedSparkFiles) @@ -239,16 +234,15 @@ private[spark] class Client( .endEnv() .endContainer() .endSpec() - val resolvedDriverPod = pythonResource match { - case Some(p) => p.driverPod( - initContainerComponentsProvider, + val driverPodFileMounter = initContainerComponentsProvider.provideDriverPodFileMounter() + val resolvedDriverPod = (pythonResource map { + p => p.driverPod( + driverPodFileMounter, resolvedPrimaryPySparkResource, resolvedPySparkFiles.mkString(","), driverContainer.getName, resolvedDriverPodBuilder - ) - case None => resolvedDriverPodBuilder.build() - } + )}).getOrElse(resolvedDriverPodBuilder.build()) Utils.tryWithResource( kubernetesClient .pods() @@ -307,13 +301,11 @@ private[spark] object Client { Option(new PythonSubmissionResourcesImpl(mainAppResource, appArgs)) } else None // Since you might need jars for SQL UDFs in PySpark - def sparkJarFilter() : Seq[String] = pythonResource match { - case Some(p) => p.sparkJars - case None => + def sparkJarFilter() : Seq[String] = (pythonResource map { + p => p.sparkJars}).getOrElse( Option(mainAppResource) .filterNot(_ == SparkLauncher.NO_RESOURCE) - .toSeq - } + .toSeq) val sparkJars = sparkConf.getOption("spark.jars") .map(_.split(",")) .getOrElse(Array.empty[String]) ++ sparkJarFilter() diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/PythonSubmissionResources.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/PythonSubmissionResources.scala index a20d93080c4e1..c41542e2ea48b 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/PythonSubmissionResources.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/PythonSubmissionResources.scala @@ -25,7 +25,7 @@ private[spark] trait PythonSubmissionResources { def primaryPySparkResource(containerLocalizedFilesResolver: ContainerLocalizedFilesResolver) : String def driverPod( - initContainerComponentsProvider: DriverInitContainerComponentsProvider, + driverPodFileMounter: DriverPodKubernetesFileMounter, resolvedPrimaryPySparkResource: String, resolvedPySparkFiles: String, driverContainerName: String, @@ -59,13 +59,12 @@ private[spark] class PythonSubmissionResourcesImpl( containerLocalizedFilesResolver.resolvePrimaryResourceFile() override def driverPod( - initContainerComponentsProvider: DriverInitContainerComponentsProvider, + driverPodFileMounter: DriverPodKubernetesFileMounter, resolvedPrimaryPySparkResource: String, resolvedPySparkFiles: String, driverContainerName: String, driverPodBuilder: PodBuilder) : Pod = { - initContainerComponentsProvider - .provideDriverPodFileMounter() + driverPodFileMounter .addPySparkFiles( resolvedPrimaryPySparkResource, resolvedPySparkFiles, diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/PythonSubmissionResourcesSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/PythonSubmissionResourcesSuite.scala index bb83bee667ce4..5c2305a8c61f7 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/PythonSubmissionResourcesSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/PythonSubmissionResourcesSuite.scala @@ -75,7 +75,7 @@ private[spark] class PythonSubmissionResourcesSuite extends SparkFunSuite { .withNewSpec() .addToContainers(driverContainer) .endSpec() - private val driverInitcontainer = new DriverInitContainerComponentsProviderImpl( + private val driverFileMounter = new DriverInitContainerComponentsProviderImpl( new SparkConf(true).set(KUBERNETES_NAMESPACE, NAMESPACE), "kubeResourceName", "namespace", @@ -83,8 +83,8 @@ private[spark] class PythonSubmissionResourcesSuite extends SparkFunSuite { SPARK_FILES, PYSPARK_PRIMARY_FILE +: PYSPARK_FILES, SSLOptions() - ) - private val lessDriverInitcontainer = new DriverInitContainerComponentsProviderImpl( + ).provideDriverPodFileMounter() + private val lessDriverFileMounter = new DriverInitContainerComponentsProviderImpl( new SparkConf(true).set(KUBERNETES_NAMESPACE, NAMESPACE), "kubeResourceName", "namespace", @@ -92,7 +92,7 @@ private[spark] class PythonSubmissionResourcesSuite extends SparkFunSuite { SPARK_FILES, Array(PYSPARK_PRIMARY_FILE), SSLOptions() - ) + ).provideDriverPodFileMounter() test("Test with --py-files included") { assert(pyFilesResource.sparkJars === Seq.empty[String]) @@ -101,7 +101,7 @@ private[spark] class PythonSubmissionResourcesSuite extends SparkFunSuite { assert(pyFilesResource.primaryPySparkResource(localizedFilesResolver) === RESOLVED_PYSPARK_PRIMARY_FILE) val driverPod: Pod = pyFilesResource.driverPod( - driverInitcontainer, + driverFileMounter, RESOLVED_PYSPARK_PRIMARY_FILE, RESOLVED_PYSPARK_FILES.mkString(","), DRIVER_CONTAINER_NAME, @@ -119,7 +119,7 @@ private[spark] class PythonSubmissionResourcesSuite extends SparkFunSuite { assert(pyResource.primaryPySparkResource(lessLocalizedFilesResolver) === RESOLVED_PYSPARK_PRIMARY_FILE) val driverPod: Pod = pyResource.driverPod( - lessDriverInitcontainer, + lessDriverFileMounter, RESOLVED_PYSPARK_PRIMARY_FILE, "", DRIVER_CONTAINER_NAME, From 9c7adb14c8239d3e33677bf63c542b19570d64e2 Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Wed, 28 Jun 2017 13:49:04 -0700 Subject: [PATCH 17/26] Modified unit tests to be more specific towards Class in question --- .../deploy/kubernetes/submit/Client.scala | 2 +- .../submit/PythonSubmissionResources.scala | 4 +- .../kubernetes/submit/ClientV2Suite.scala | 22 ++---- .../PythonSubmissionResourcesSuite.scala | 69 +++++++------------ 4 files changed, 34 insertions(+), 63 deletions(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/Client.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/Client.scala index 882672a31ad13..c3f4ff922b872 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/Client.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/Client.scala @@ -236,7 +236,7 @@ private[spark] class Client( .endSpec() val driverPodFileMounter = initContainerComponentsProvider.provideDriverPodFileMounter() val resolvedDriverPod = (pythonResource map { - p => p.driverPod( + p => p.driverPodWithPySparkEnvs( driverPodFileMounter, resolvedPrimaryPySparkResource, resolvedPySparkFiles.mkString(","), diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/PythonSubmissionResources.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/PythonSubmissionResources.scala index c41542e2ea48b..f7589a3c2f809 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/PythonSubmissionResources.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/PythonSubmissionResources.scala @@ -24,7 +24,7 @@ private[spark] trait PythonSubmissionResources { def arguments: Array[String] def primaryPySparkResource(containerLocalizedFilesResolver: ContainerLocalizedFilesResolver) : String - def driverPod( + def driverPodWithPySparkEnvs( driverPodFileMounter: DriverPodKubernetesFileMounter, resolvedPrimaryPySparkResource: String, resolvedPySparkFiles: String, @@ -58,7 +58,7 @@ private[spark] class PythonSubmissionResourcesImpl( containerLocalizedFilesResolver: ContainerLocalizedFilesResolver) : String = containerLocalizedFilesResolver.resolvePrimaryResourceFile() - override def driverPod( + override def driverPodWithPySparkEnvs( driverPodFileMounter: DriverPodKubernetesFileMounter, resolvedPrimaryPySparkResource: String, resolvedPySparkFiles: String, diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/ClientV2Suite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/ClientV2Suite.scala index fe04fddc746bd..a58a37691f4eb 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/ClientV2Suite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/ClientV2Suite.scala @@ -213,18 +213,10 @@ class ClientV2Suite extends SparkFunSuite with BeforeAndAfter { any())).thenAnswer( new Answer[PodBuilder] { override def answer(invocation: InvocationOnMock) : PodBuilder = { invocation.getArgumentAt(3, classOf[PodBuilder]) - .editSpec() - .editMatchingContainer(new ContainerNameEqualityPredicate( - invocation.getArgumentAt(2, classOf[String]))) - .addNewEnv() - .withName("pyspark") - .withValue("true") - .endEnv() - .endContainer() - .endSpec() .editMetadata() .withUid(DRIVER_POD_UID) .withName(s"$APP_RESOURCE_PREFIX-driver") + .addToLabels("pyspark-test", "true") .endMetadata() .withKind(DRIVER_POD_KIND) .withApiVersion(DRIVER_POD_API_VERSION) @@ -510,12 +502,12 @@ class ClientV2Suite extends SparkFunSuite with BeforeAndAfter { } private def containerHasCorrectPySparkEnvs(pod: Pod): Boolean = { - val driverContainer = pod.getSpec.getContainers.asScala.head - val envs = driverContainer.getEnv.asScala.map(env => (env.getName, env.getValue)) - val expectedBasicEnvs = Map( - "pyspark" -> "true" - ) - expectedBasicEnvs.toSet.subsetOf(envs.toSet) + val driverPodLabels = + pod.getMetadata.getLabels.asScala.map(env => (env._1.toString, env._2.toString)) + val expectedBasicLabels = Map( + "pyspark-test" -> "true", + "spark-role" -> "driver") + expectedBasicLabels.toSet.subsetOf(driverPodLabels.toSet) } private def containerHasCorrectBasicContainerConfiguration(pod: Pod): Boolean = { diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/PythonSubmissionResourcesSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/PythonSubmissionResourcesSuite.scala index 5c2305a8c61f7..5483dc154fe73 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/PythonSubmissionResourcesSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/PythonSubmissionResourcesSuite.scala @@ -18,13 +18,17 @@ package org.apache.spark.deploy.kubernetes.submit import org.apache.spark.{SSLOptions, SparkConf, SparkFunSuite} import org.apache.spark.deploy.kubernetes.config._ -import scala.collection.JavaConverters._ +import scala.collection.JavaConverters._ import io.fabric8.kubernetes.api.model.{ContainerBuilder, Pod, PodBuilder} +import org.mockito.{Mock, MockitoAnnotations} +import org.mockito.Mockito.when +import org.scalatest.BeforeAndAfter + -private[spark] class PythonSubmissionResourcesSuite extends SparkFunSuite { +private[spark] class PythonSubmissionResourcesSuite extends SparkFunSuite with BeforeAndAfter { private val PYSPARK_FILES = Seq( "hdfs://localhost:9000/app/files/file1.py", "file:///app/files/file2.py", @@ -45,25 +49,6 @@ private[spark] class PythonSubmissionResourcesSuite extends SparkFunSuite { private val pyResource = new PythonSubmissionResourcesImpl( PYSPARK_PRIMARY_FILE, Array(null, "500") ) - private val SPARK_FILES = Seq.empty[String] - private val SPARK_JARS = Seq.empty[String] - private val JARS_DOWNLOAD_PATH = "/var/data/spark-jars" - private val FILES_DOWNLOAD_PATH = "/var/data/spark-files" - private val localizedFilesResolver = new ContainerLocalizedFilesResolverImpl( - SPARK_JARS, - SPARK_FILES, - PYSPARK_FILES, - PYSPARK_PRIMARY_FILE, - JARS_DOWNLOAD_PATH, - FILES_DOWNLOAD_PATH) - private val lessLocalizedFilesResolver = new ContainerLocalizedFilesResolverImpl( - SPARK_JARS, - SPARK_FILES, - Seq.empty[String], - PYSPARK_PRIMARY_FILE, - JARS_DOWNLOAD_PATH, - FILES_DOWNLOAD_PATH) - private val NAMESPACE = "example_pyspark" private val DRIVER_CONTAINER_NAME = "pyspark_container" private val driverContainer = new ContainerBuilder() .withName(DRIVER_CONTAINER_NAME) @@ -75,33 +60,27 @@ private[spark] class PythonSubmissionResourcesSuite extends SparkFunSuite { .withNewSpec() .addToContainers(driverContainer) .endSpec() - private val driverFileMounter = new DriverInitContainerComponentsProviderImpl( - new SparkConf(true).set(KUBERNETES_NAMESPACE, NAMESPACE), - "kubeResourceName", - "namespace", - SPARK_JARS, - SPARK_FILES, - PYSPARK_PRIMARY_FILE +: PYSPARK_FILES, - SSLOptions() - ).provideDriverPodFileMounter() - private val lessDriverFileMounter = new DriverInitContainerComponentsProviderImpl( - new SparkConf(true).set(KUBERNETES_NAMESPACE, NAMESPACE), - "kubeResourceName", - "namespace", - SPARK_JARS, - SPARK_FILES, - Array(PYSPARK_PRIMARY_FILE), - SSLOptions() - ).provideDriverPodFileMounter() + @Mock + private var driverInitContainer: DriverInitContainerComponentsProviderImpl = _ + @Mock + private var localizedFileResolver: ContainerLocalizedFilesResolverImpl = _ + before { + MockitoAnnotations.initMocks(this) + when(driverInitContainer.provideDriverPodFileMounter()).thenReturn( + new DriverPodKubernetesFileMounterImpl() + ) + when(localizedFileResolver.resolvePrimaryResourceFile()).thenReturn( + RESOLVED_PYSPARK_PRIMARY_FILE) + } test("Test with --py-files included") { assert(pyFilesResource.sparkJars === Seq.empty[String]) assert(pyFilesResource.pySparkFiles === PYSPARK_PRIMARY_FILE +: PYSPARK_FILES) - assert(pyFilesResource.primaryPySparkResource(localizedFilesResolver) === + assert(pyFilesResource.primaryPySparkResource(localizedFileResolver) === RESOLVED_PYSPARK_PRIMARY_FILE) - val driverPod: Pod = pyFilesResource.driverPod( - driverFileMounter, + val driverPod: Pod = pyFilesResource.driverPodWithPySparkEnvs( + driverInitContainer.provideDriverPodFileMounter(), RESOLVED_PYSPARK_PRIMARY_FILE, RESOLVED_PYSPARK_FILES.mkString(","), DRIVER_CONTAINER_NAME, @@ -116,10 +95,10 @@ private[spark] class PythonSubmissionResourcesSuite extends SparkFunSuite { test("Test without --py-files") { assert(pyResource.sparkJars === Seq.empty[String]) assert(pyResource.pySparkFiles === Array(PYSPARK_PRIMARY_FILE)) - assert(pyResource.primaryPySparkResource(lessLocalizedFilesResolver) === + assert(pyResource.primaryPySparkResource(localizedFileResolver) === RESOLVED_PYSPARK_PRIMARY_FILE) - val driverPod: Pod = pyResource.driverPod( - lessDriverFileMounter, + val driverPod: Pod = pyResource.driverPodWithPySparkEnvs( + driverInitContainer.provideDriverPodFileMounter(), RESOLVED_PYSPARK_PRIMARY_FILE, "", DRIVER_CONTAINER_NAME, From 0388aa4637235876364a9eb3447e42da594227e9 Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Wed, 28 Jun 2017 14:47:49 -0700 Subject: [PATCH 18/26] Removed space delimiting for methods --- .../spark/deploy/kubernetes/submit/Client.scala | 16 ++++++++-------- .../submit/PythonSubmissionResources.scala | 2 +- 2 files changed, 9 insertions(+), 9 deletions(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/Client.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/Client.scala index c3f4ff922b872..e61cece142e3d 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/Client.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/Client.scala @@ -184,9 +184,9 @@ private[spark] class Client( val resolvedSparkJars = containerLocalizedFilesResolver.resolveSubmittedSparkJars() val resolvedSparkFiles = containerLocalizedFilesResolver.resolveSubmittedSparkFiles() val resolvedPySparkFiles = containerLocalizedFilesResolver.resolveSubmittedPySparkFiles() - val resolvedPrimaryPySparkResource = (pythonResource map { + val resolvedPrimaryPySparkResource = pythonResource.map { p => p.primaryPySparkResource(containerLocalizedFilesResolver) - }).getOrElse("") + }.getOrElse("") val initContainerBundler = initContainerComponentsProvider .provideInitContainerBundle(maybeSubmittedResourceIdentifiers.map(_.ids()), resolvedSparkJars ++ resolvedSparkFiles) @@ -235,14 +235,14 @@ private[spark] class Client( .endContainer() .endSpec() val driverPodFileMounter = initContainerComponentsProvider.provideDriverPodFileMounter() - val resolvedDriverPod = (pythonResource map { + val resolvedDriverPod = pythonResource.map { p => p.driverPodWithPySparkEnvs( driverPodFileMounter, resolvedPrimaryPySparkResource, resolvedPySparkFiles.mkString(","), driverContainer.getName, resolvedDriverPodBuilder - )}).getOrElse(resolvedDriverPodBuilder.build()) + )}.getOrElse(resolvedDriverPodBuilder.build()) Utils.tryWithResource( kubernetesClient .pods() @@ -301,8 +301,8 @@ private[spark] object Client { Option(new PythonSubmissionResourcesImpl(mainAppResource, appArgs)) } else None // Since you might need jars for SQL UDFs in PySpark - def sparkJarFilter() : Seq[String] = (pythonResource map { - p => p.sparkJars}).getOrElse( + def sparkJarFilter() : Seq[String] = + pythonResource.map { p => p.sparkJars}.getOrElse( Option(mainAppResource) .filterNot(_ == SparkLauncher.NO_RESOURCE) .toSeq) @@ -313,10 +313,10 @@ private[spark] object Client { val sparkFiles = sparkConf.getOption("spark.files") .map(_.split(",")) .getOrElse(Array.empty[String]) - val pySparkFilesOption = pythonResource map {p => p.pySparkFiles} + val pySparkFilesOption = pythonResource.map {p => p.pySparkFiles} validateNoDuplicateFileNames(sparkJars) validateNoDuplicateFileNames(sparkFiles) - pySparkFilesOption foreach {b => validateNoDuplicateFileNames(b)} + pySparkFilesOption.foreach {b => validateNoDuplicateFileNames(b)} val pySparkFiles = pySparkFilesOption.getOrElse(Array.empty[String]) val appName = sparkConf.getOption("spark.app.name").getOrElse("spark") // The resource name prefix is derived from the application name, making it easy to connect the diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/PythonSubmissionResources.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/PythonSubmissionResources.scala index f7589a3c2f809..e615cf72116e1 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/PythonSubmissionResources.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/PythonSubmissionResources.scala @@ -37,7 +37,7 @@ private[spark] class PythonSubmissionResourcesImpl( private val appArgs: Array[String] ) extends PythonSubmissionResources { private val pyFiles: Array[String] = { - (Option(appArgs(0)) map (a => mainAppResource +: a.split(","))) + Option(appArgs(0)).map(a => mainAppResource +: a.split(",")) .getOrElse(Array(mainAppResource)) } From 5499f6ddf9b42c0526f1dc053317afb38dc71294 Mon Sep 17 00:00:00 2001 From: mcheah Date: Thu, 29 Jun 2017 16:52:02 -0700 Subject: [PATCH 19/26] Submission client redesign to use a step-based builder pattern. This change overhauls the underlying architecture of the submission client, but it is intended to entirely preserve existing behavior of Spark applications. Therefore users will find this to be an invisible change. The philosophy behind this design is to reconsider the breakdown of the submission process. It operates off the abstraction of "submission steps", which are transformation functions that take the previous state of the driver and return the new state of the driver. The driver's state includes its Spark configurations and the Kubernetes resources that will be used to deploy it. Such a refactor moves away from a features-first API design, which considers different containers to serve a set of features. The previous design, for example, had a container files resolver API object that returned different resolutions of the dependencies added by the user. However, it was up to the main Client to know how to intelligently invoke all of those APIs. Therefore the API surface area of the file resolver became untenably large and it was not intuitive of how it was to be used or extended. This design changes the encapsulation layout; every module is now responsible for changing the driver specification directly. An orchestrator builds the correct chain of steps and hands it to the client, which then calls it verbatim. The main client then makes any final modifications that put the different pieces of the driver together, particularly to attach the driver container itself to the pod and to apply the Spark configuration as command-line arguments. --- .../org/apache/spark/deploy/SparkSubmit.scala | 10 +- ...nerResourceStagingServerSecretPlugin.scala | 38 +- .../PodWithDetachedInitContainer.scala | 24 + .../SparkPodInitContainerBootstrap.scala | 51 +- .../spark/deploy/kubernetes/config.scala | 8 + .../deploy/kubernetes/submit/Client.scala | 428 ++++---------- .../ContainerLocalizedFilesResolver.scala | 87 --- ...riverInitContainerComponentsProvider.scala | 223 -------- ...riverPodKubernetesCredentialsMounter.scala | 184 ------ ...KubernetesCredentialsMounterProvider.scala | 49 -- ...iverPodKubernetesCredentialsProvider.scala | 63 -- .../DriverPodKubernetesFileMounter.scala | 55 -- .../kubernetes/submit/InitContainerUtil.scala | 16 +- .../submit/KubernetesFileUtils.scala | 45 ++ ...ubernetesSubmissionStepsOrchestrator.scala | 127 +++++ ...inerBundle.scala => MainAppResource.scala} | 9 +- .../submit/PythonSubmissionResources.scala | 75 --- ...dDependencyInitContainerConfigPlugin.scala | 96 ---- .../SubmittedDependencySecretBuilder.scala | 81 --- .../submitsteps/BaseSubmissionStep.scala | 136 +++++ .../DependencyResolutionStep.scala | 66 +++ .../DriverKubernetesCredentialsStep.scala | 218 +++++++ .../InitContainerBootstrapStep.scala | 64 +++ .../submitsteps/KubernetesDriverSpec.scala | 35 ++ .../KubernetesSubmissionStep.scala | 28 + .../submit/submitsteps/PythonStep.scala | 42 ++ .../BaseInitContainerStep.scala} | 41 +- .../initcontainer/InitContainerSpec.scala | 41 ++ .../initcontainer/InitContainerStep.scala | 25 + .../InitContainerStepsOrchestrator.scala | 131 +++++ .../SubmittedResourcesInitContainerStep.scala | 146 +++++ ...SparkDependencyDownloadInitContainer.scala | 1 - .../kubernetes/KubernetesClusterManager.scala | 9 +- .../KubernetesClusterSchedulerBackend.scala | 149 +++-- .../SparkPodInitContainerBootstrapSuite.scala | 284 +++++---- ...dencyInitContainerVolumesPluginSuite.scala | 60 -- .../kubernetes/submit/ClientV2Suite.scala | 537 +----------------- ...ContainerLocalizedFilesResolverSuite.scala | 93 --- ...PodKubernetesCredentialsMounterSuite.scala | 171 ------ ...cutorInitContainerConfigurationSuite.scala | 56 -- .../PythonSubmissionResourcesSuite.scala | 112 ---- ...rkInitContainerConfigMapBuilderSuite.scala | 101 ---- ...ndencyInitContainerConfigPluginSuite.scala | 89 --- ...ubmittedDependencySecretBuilderSuite.scala | 109 ---- .../integrationtest/KubernetesSuite.scala | 47 +- 45 files changed, 1595 insertions(+), 2865 deletions(-) create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/PodWithDetachedInitContainer.scala delete mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/ContainerLocalizedFilesResolver.scala delete mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/DriverInitContainerComponentsProvider.scala delete mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/DriverPodKubernetesCredentialsMounter.scala delete mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/DriverPodKubernetesCredentialsMounterProvider.scala delete mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/DriverPodKubernetesCredentialsProvider.scala delete mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/DriverPodKubernetesFileMounter.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/KubernetesSubmissionStepsOrchestrator.scala rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/{InitContainerBundle.scala => MainAppResource.scala} (71%) delete mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/PythonSubmissionResources.scala delete mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/SubmittedDependencyInitContainerConfigPlugin.scala delete mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/SubmittedDependencySecretBuilder.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/BaseSubmissionStep.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DependencyResolutionStep.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DriverKubernetesCredentialsStep.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/InitContainerBootstrapStep.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/KubernetesDriverSpec.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/KubernetesSubmissionStep.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/PythonStep.scala rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/{SparkInitContainerConfigMapBuilder.scala => submitsteps/initcontainer/BaseInitContainerStep.scala} (62%) create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/InitContainerSpec.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/InitContainerStep.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/InitContainerStepsOrchestrator.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/SubmittedResourcesInitContainerStep.scala delete mode 100644 resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/SubmittedDependencyInitContainerVolumesPluginSuite.scala delete mode 100644 resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/ContainerLocalizedFilesResolverSuite.scala delete mode 100644 resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/DriverPodKubernetesCredentialsMounterSuite.scala delete mode 100644 resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/ExecutorInitContainerConfigurationSuite.scala delete mode 100644 resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/PythonSubmissionResourcesSuite.scala delete mode 100644 resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/SparkInitContainerConfigMapBuilderSuite.scala delete mode 100644 resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/SubmittedDependencyInitContainerConfigPluginSuite.scala delete mode 100644 resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/SubmittedDependencySecretBuilderSuite.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 9256a9ddd9960..d3c7a016f96bb 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -621,14 +621,22 @@ object SparkSubmit { if (isKubernetesCluster) { childMainClass = "org.apache.spark.deploy.kubernetes.submit.Client" if (args.isPython) { + childArgs += "--py-file" childArgs += args.primaryResource + childArgs += "--main-class" childArgs += "org.apache.spark.deploy.PythonRunner" + childArgs += "--other-py-files" childArgs += args.pyFiles } else { + childArgs += "--primary-java-resource" childArgs += args.primaryResource + childArgs += "--main-class" childArgs += args.mainClass } - childArgs ++= args.childArgs + args.childArgs.foreach { arg => + childArgs += "--arg" + childArgs += arg + } } // Load any properties specified through --conf and the default properties file diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/InitContainerResourceStagingServerSecretPlugin.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/InitContainerResourceStagingServerSecretPlugin.scala index 45b881a8a3737..265b8f197a102 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/InitContainerResourceStagingServerSecretPlugin.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/InitContainerResourceStagingServerSecretPlugin.scala @@ -16,7 +16,7 @@ */ package org.apache.spark.deploy.kubernetes -import io.fabric8.kubernetes.api.model.{ContainerBuilder, PodBuilder, Secret} +import io.fabric8.kubernetes.api.model.{Container, ContainerBuilder, Pod, PodBuilder, Secret} import org.apache.spark.deploy.kubernetes.constants._ @@ -27,13 +27,13 @@ private[spark] trait InitContainerResourceStagingServerSecretPlugin { * from a resource staging server. */ def mountResourceStagingServerSecretIntoInitContainer( - initContainer: ContainerBuilder): ContainerBuilder + initContainer: Container): Container /** * Configure the pod to attach a Secret volume which hosts secret files allowing the * init-container to retrieve dependencies from the resource staging server. */ - def addResourceStagingServerSecretVolumeToPod(basePod: PodBuilder): PodBuilder + def addResourceStagingServerSecretVolumeToPod(basePod: Pod): Pod } private[spark] class InitContainerResourceStagingServerSecretPluginImpl( @@ -42,21 +42,25 @@ private[spark] class InitContainerResourceStagingServerSecretPluginImpl( extends InitContainerResourceStagingServerSecretPlugin { override def mountResourceStagingServerSecretIntoInitContainer( - initContainer: ContainerBuilder): ContainerBuilder = { - initContainer.addNewVolumeMount() - .withName(INIT_CONTAINER_SECRET_VOLUME_NAME) - .withMountPath(initContainerSecretMountPath) - .endVolumeMount() + initContainer: Container): Container = { + new ContainerBuilder(initContainer) + .addNewVolumeMount() + .withName(INIT_CONTAINER_SECRET_VOLUME_NAME) + .withMountPath(initContainerSecretMountPath) + .endVolumeMount() + .build() } - override def addResourceStagingServerSecretVolumeToPod(basePod: PodBuilder): PodBuilder = { - basePod.editSpec() - .addNewVolume() - .withName(INIT_CONTAINER_SECRET_VOLUME_NAME) - .withNewSecret() - .withSecretName(initContainerSecretName) - .endSecret() - .endVolume() - .endSpec() + override def addResourceStagingServerSecretVolumeToPod(basePod: Pod): Pod = { + new PodBuilder(basePod) + .editSpec() + .addNewVolume() + .withName(INIT_CONTAINER_SECRET_VOLUME_NAME) + .withNewSecret() + .withSecretName(initContainerSecretName) + .endSecret() + .endVolume() + .endSpec() + .build() } } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/PodWithDetachedInitContainer.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/PodWithDetachedInitContainer.scala new file mode 100644 index 0000000000000..36b1b07dc6bc4 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/PodWithDetachedInitContainer.scala @@ -0,0 +1,24 @@ +/* + * 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 io.fabric8.kubernetes.api.model.{Container, Pod} + +private[spark] case class PodWithDetachedInitContainer( + pod: Pod, + initContainer: Container, + mainContainer: Container) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/SparkPodInitContainerBootstrap.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/SparkPodInitContainerBootstrap.scala index 87462dbde17a5..c2b04c6c49311 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/SparkPodInitContainerBootstrap.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/SparkPodInitContainerBootstrap.scala @@ -19,8 +19,11 @@ package org.apache.spark.deploy.kubernetes import io.fabric8.kubernetes.api.model.{ContainerBuilder, EmptyDirVolumeSource, PodBuilder, VolumeMount, VolumeMountBuilder} import org.apache.spark.deploy.kubernetes.constants._ -import org.apache.spark.deploy.kubernetes.submit.{ContainerNameEqualityPredicate, InitContainerUtil} +/** + * This is separated out from the init-container steps API because this component can be reused to + * set up the init-container for executors as well. + */ private[spark] trait SparkPodInitContainerBootstrap { /** * Bootstraps an init-container that downloads dependencies to be used by a main container. @@ -28,10 +31,13 @@ private[spark] trait SparkPodInitContainerBootstrap { * by a ConfigMap that was installed by some other component; that is, the implementation * here makes no assumptions about how the init-container is specifically configured. For * example, this class is unaware if the init-container is fetching remote dependencies or if - * it is fetching dependencies from a resource staging server. + * it is fetching dependencies from a resource staging server. Additionally, the container itself + * is not actually attached to the pod, but the init container is returned so it can be attached + * by InitContainerUtil after the caller has decided to make any changes to it. */ def bootstrapInitContainerAndVolumes( - mainContainerName: String, originalPodSpec: PodBuilder): PodBuilder + originalPodWithUnattachedInitContainer: PodWithDetachedInitContainer) + : PodWithDetachedInitContainer } private[spark] class SparkPodInitContainerBootstrapImpl( @@ -41,13 +47,12 @@ private[spark] class SparkPodInitContainerBootstrapImpl( filesDownloadPath: String, downloadTimeoutMinutes: Long, initContainerConfigMapName: String, - initContainerConfigMapKey: String, - resourceStagingServerSecretPlugin: Option[InitContainerResourceStagingServerSecretPlugin]) + initContainerConfigMapKey: String) extends SparkPodInitContainerBootstrap { override def bootstrapInitContainerAndVolumes( - mainContainerName: String, - originalPodSpec: PodBuilder): PodBuilder = { + originalPodWithUnattachedInitContainer: PodWithDetachedInitContainer) + : PodWithDetachedInitContainer = { val sharedVolumeMounts = Seq[VolumeMount]( new VolumeMountBuilder() .withName(INIT_CONTAINER_DOWNLOAD_JARS_VOLUME_NAME) @@ -58,7 +63,7 @@ private[spark] class SparkPodInitContainerBootstrapImpl( .withMountPath(filesDownloadPath) .build()) - val initContainer = new ContainerBuilder() + val initContainer = new ContainerBuilder(originalPodWithUnattachedInitContainer.initContainer) .withName(s"spark-init") .withImage(initContainerImage) .withImagePullPolicy(dockerImagePullPolicy) @@ -68,11 +73,8 @@ private[spark] class SparkPodInitContainerBootstrapImpl( .endVolumeMount() .addToVolumeMounts(sharedVolumeMounts: _*) .addToArgs(INIT_CONTAINER_PROPERTIES_FILE_PATH) - val resolvedInitContainer = resourceStagingServerSecretPlugin.map { plugin => - plugin.mountResourceStagingServerSecretIntoInitContainer(initContainer) - }.getOrElse(initContainer).build() - val podWithBasicVolumes = InitContainerUtil.appendInitContainer( - originalPodSpec, resolvedInitContainer) + .build() + val podWithBasicVolumes = new PodBuilder(originalPodWithUnattachedInitContainer.pod) .editSpec() .addNewVolume() .withName(INIT_CONTAINER_PROPERTIES_FILE_VOLUME) @@ -92,17 +94,20 @@ private[spark] class SparkPodInitContainerBootstrapImpl( .withName(INIT_CONTAINER_DOWNLOAD_FILES_VOLUME_NAME) .withEmptyDir(new EmptyDirVolumeSource()) .endVolume() - .editMatchingContainer(new ContainerNameEqualityPredicate(mainContainerName)) - .addToVolumeMounts(sharedVolumeMounts: _*) - .addNewEnv() - .withName(ENV_MOUNTED_FILES_DIR) - .withValue(filesDownloadPath) - .endEnv() - .endContainer() .endSpec() - resourceStagingServerSecretPlugin.map { plugin => - plugin.addResourceStagingServerSecretVolumeToPod(podWithBasicVolumes) - }.getOrElse(podWithBasicVolumes) + .build() + val mainContainerWithMountedFiles = new ContainerBuilder( + originalPodWithUnattachedInitContainer.mainContainer) + .addToVolumeMounts(sharedVolumeMounts: _*) + .addNewEnv() + .withName(ENV_MOUNTED_FILES_DIR) + .withValue(filesDownloadPath) + .endEnv() + .build() + PodWithDetachedInitContainer( + podWithBasicVolumes, + initContainer, + mainContainerWithMountedFiles) } } 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 e1c1ab9d459fc..1b0e4478a544e 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 @@ -418,6 +418,14 @@ package object config extends Logging { .stringConf .createOptional + private[spark] val INIT_CONTAINER_REMOTE_PYSPARK_FILES = + ConfigBuilder("spark.kubernetes.initcontainer.remotePyFiles") + .doc("Comma-separated list of Python file URIs to download in the init-container. This is" + + " calculated given the list of python files sent to spark-submit.") + .internal() + .stringConf + .createOptional + private[spark] val INIT_CONTAINER_DOCKER_IMAGE = ConfigBuilder("spark.kubernetes.initcontainer.docker.image") .doc("Image for the driver and executor's init-container that downloads dependencies.") diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/Client.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/Client.scala index e61cece142e3d..e5b753d458b34 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/Client.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/Client.scala @@ -16,256 +16,113 @@ */ package org.apache.spark.deploy.kubernetes.submit -import java.io.File import java.util.{Collections, UUID} -import io.fabric8.kubernetes.api.model.{ContainerBuilder, EnvVarBuilder, OwnerReferenceBuilder, PodBuilder, QuantityBuilder} +import io.fabric8.kubernetes.api.model.{ContainerBuilder, HasMetadata, OwnerReferenceBuilder, PodBuilder} import io.fabric8.kubernetes.client.KubernetesClient -import scala.collection.JavaConverters._ +import scala.collection.mutable import org.apache.spark.SparkConf -import org.apache.spark.deploy.kubernetes.{ConfigurationUtils, SparkKubernetesClientFactory} import org.apache.spark.deploy.kubernetes.config._ import org.apache.spark.deploy.kubernetes.constants._ -import org.apache.spark.deploy.rest.kubernetes.ResourceStagingServerSslOptionsProviderImpl +import org.apache.spark.deploy.kubernetes.SparkKubernetesClientFactory +import org.apache.spark.deploy.kubernetes.submit.submitsteps.{KubernetesDriverSpec, KubernetesSubmissionStep} import org.apache.spark.internal.Logging -import org.apache.spark.internal.config.ConfigEntry -import org.apache.spark.launcher.SparkLauncher import org.apache.spark.util.Utils -/** - * Submission client for launching Spark applications on Kubernetes clusters. - * - * This class is responsible for instantiating Kubernetes resources that allow a Spark driver to - * run in a pod on the Kubernetes cluster with the Spark configurations specified by spark-submit. - * The API of this class makes it such that much of the specific behavior can be stubbed for - * testing; most of the detailed logic must be dependency-injected when constructing an instance - * of this client. Therefore the submission process is designed to be as modular as possible, - * where different steps of submission should be factored out into separate classes. - */ +private[spark] case class ClientArguments( + mainAppResource: MainAppResource, + otherPyFiles: Seq[String], + mainClass: String, + driverArgs: Array[String]) + +private[spark] object ClientArguments { + def fromCommandLineArgs(args: Array[String]): ClientArguments = { + var mainAppResource: Option[MainAppResource] = None + var otherPyFiles = Seq.empty[String] + var mainClass: Option[String] = None + val driverArgs = mutable.Buffer.empty[String] + args.sliding(2).toList.collect { + case Array("--py-file", mainPyFile: String) => + mainAppResource = Some(PythonMainAppResource(mainPyFile)) + case Array("--primary-java-resource", primaryJavaResource: String) => + mainAppResource = Some(JavaMainAppResource(primaryJavaResource)) + case Array("--main-class", clazz: String) => + mainClass = Some(clazz) + case Array("--other-py-files", pyFiles: String) => + otherPyFiles = pyFiles.split(",") + case Array("--arg", arg: String) => + driverArgs += arg + case other => + throw new RuntimeException(s"Unknown arguments: $other") + } + require(mainAppResource.isDefined, + "Main app resource must be defined by either --py-file or --main-java-resource.") + require(mainClass.isDefined, "Main class must be specified via --main-class") + ClientArguments( + mainAppResource.get, + otherPyFiles, + mainClass.get, + driverArgs.toArray) + } +} + private[spark] class Client( - appName: String, - kubernetesResourceNamePrefix: String, - kubernetesAppId: String, - mainAppResource: String, - pythonResource: Option[PythonSubmissionResourcesImpl], - mainClass: String, - sparkConf: SparkConf, - appArgs: Array[String], - waitForAppCompletion: Boolean, + submissionSteps: Seq[KubernetesSubmissionStep], + submissionSparkConf: SparkConf, kubernetesClient: KubernetesClient, - initContainerComponentsProvider: DriverInitContainerComponentsProvider, - kubernetesCredentialsMounterProvider: DriverPodKubernetesCredentialsMounterProvider, + waitForAppCompletion: Boolean, + appName: String, loggingPodStatusWatcher: LoggingPodStatusWatcher) extends Logging { - private val kubernetesDriverPodName = sparkConf.get(KUBERNETES_DRIVER_POD_NAME) - .getOrElse(s"$kubernetesResourceNamePrefix-driver") - private val driverDockerImage = sparkConf.get(DRIVER_DOCKER_IMAGE) - private val dockerImagePullPolicy = sparkConf.get(DOCKER_IMAGE_PULL_POLICY) - - // CPU settings - private val driverCpuCores = sparkConf.getOption("spark.driver.cores").getOrElse("1") - private val driverLimitCores = sparkConf.getOption(KUBERNETES_DRIVER_LIMIT_CORES.key) - - // Memory settings - private val driverMemoryMb = sparkConf.get(org.apache.spark.internal.config.DRIVER_MEMORY) - private val memoryOverheadMb = sparkConf - .get(KUBERNETES_DRIVER_MEMORY_OVERHEAD) - .getOrElse(math.max((MEMORY_OVERHEAD_FACTOR * driverMemoryMb).toInt, - MEMORY_OVERHEAD_MIN)) - private val driverContainerMemoryWithOverhead = driverMemoryMb + memoryOverheadMb - private val customLabels = sparkConf.get(KUBERNETES_DRIVER_LABELS) - private val customAnnotations = sparkConf.get(KUBERNETES_DRIVER_ANNOTATIONS) - private val driverExtraClasspath = sparkConf.get( - org.apache.spark.internal.config.DRIVER_CLASS_PATH) - private val driverJavaOptions = sparkConf.get( + private val driverJavaOptions = submissionSparkConf.get( org.apache.spark.internal.config.DRIVER_JAVA_OPTIONS) def run(): Unit = { - val arguments = (pythonResource map {p => p.arguments}).getOrElse(appArgs) - val driverCustomLabels = ConfigurationUtils.combinePrefixedKeyValuePairsWithDeprecatedConf( - sparkConf, - KUBERNETES_DRIVER_LABEL_PREFIX, - KUBERNETES_DRIVER_LABELS, - "label") - require(!driverCustomLabels.contains(SPARK_APP_ID_LABEL), s"Label with key " + - s" $SPARK_APP_ID_LABEL is not allowed as it is reserved for Spark bookkeeping" + - s" operations.") - - val driverCustomAnnotations = ConfigurationUtils.combinePrefixedKeyValuePairsWithDeprecatedConf( - sparkConf, - KUBERNETES_DRIVER_ANNOTATION_PREFIX, - KUBERNETES_DRIVER_ANNOTATIONS, - "annotation") - require(!driverCustomAnnotations.contains(SPARK_APP_NAME_ANNOTATION), - s"Annotation with key $SPARK_APP_NAME_ANNOTATION is not allowed as it is reserved for" + - s" Spark bookkeeping operations.") - val allDriverLabels = driverCustomLabels ++ Map( - SPARK_APP_ID_LABEL -> kubernetesAppId, - SPARK_ROLE_LABEL -> SPARK_POD_DRIVER_ROLE) - - val driverExtraClasspathEnv = driverExtraClasspath.map { classPath => - new EnvVarBuilder() - .withName(ENV_SUBMIT_EXTRA_CLASSPATH) - .withValue(classPath) - .build() + var currentDriverSpec = new KubernetesDriverSpec( + driverPod = new PodBuilder().build(), + driverContainer = new ContainerBuilder().build(), + driverSparkConf = submissionSparkConf.clone(), + otherKubernetesResources = Seq.empty[HasMetadata]) + for (nextStep <- submissionSteps) { + currentDriverSpec = nextStep.prepareSubmission(currentDriverSpec) } - val driverCpuQuantity = new QuantityBuilder(false) - .withAmount(driverCpuCores) - .build() - val driverMemoryQuantity = new QuantityBuilder(false) - .withAmount(s"${driverMemoryMb}M") - .build() - val driverMemoryLimitQuantity = new QuantityBuilder(false) - .withAmount(s"${driverContainerMemoryWithOverhead}M") - .build() - val driverContainer = new ContainerBuilder() - .withName(DRIVER_CONTAINER_NAME) - .withImage(driverDockerImage) - .withImagePullPolicy(dockerImagePullPolicy) - .addToEnv(driverExtraClasspathEnv.toSeq: _*) - .addNewEnv() - .withName(ENV_DRIVER_MEMORY) - .withValue(driverContainerMemoryWithOverhead + "m") - .endEnv() - .addNewEnv() - .withName(ENV_DRIVER_MAIN_CLASS) - .withValue(mainClass) - .endEnv() + val resolvedDriverJavaOpts = currentDriverSpec.driverSparkConf.getAll.map { + case (confKey, confValue) => s"-D$confKey=$confValue" + }.mkString(" ") + driverJavaOptions.map(" " + _).getOrElse("") + val resolvedDriverContainer = new ContainerBuilder(currentDriverSpec.driverContainer) .addNewEnv() - .withName(ENV_DRIVER_ARGS) - .withValue(arguments.mkString(" ")) + .withName(ENV_DRIVER_JAVA_OPTS) + .withValue(resolvedDriverJavaOpts) .endEnv() - .withNewResources() - .addToRequests("cpu", driverCpuQuantity) - .addToRequests("memory", driverMemoryQuantity) - .addToLimits("memory", driverMemoryLimitQuantity) - .endResources() .build() - val basePod = new PodBuilder() - .withNewMetadata() - .withName(kubernetesDriverPodName) - .addToLabels(allDriverLabels.asJava) - .addToAnnotations(driverCustomAnnotations.toMap.asJava) - .addToAnnotations(SPARK_APP_NAME_ANNOTATION, appName) - .endMetadata() - .withNewSpec() - .withRestartPolicy("Never") - .addToContainers(driverContainer) + val resolvedDriverPod = new PodBuilder(currentDriverSpec.driverPod) + .editSpec() + .addToContainers(resolvedDriverContainer) .endSpec() - - driverLimitCores.map { - limitCores => - val driverCpuLimitQuantity = new QuantityBuilder(false) - .withAmount(limitCores) - .build() - basePod - .editSpec() - .editFirstContainer() - .editResources - .addToLimits("cpu", driverCpuLimitQuantity) - .endResources() - .endContainer() - .endSpec() - } - - val maybeSubmittedResourceIdentifiers = initContainerComponentsProvider - .provideInitContainerSubmittedDependencyUploader(allDriverLabels) - .map { uploader => - SubmittedResources(uploader.uploadJars(), uploader.uploadFiles()) - } - val maybeSubmittedDependenciesSecret = initContainerComponentsProvider - .provideSubmittedDependenciesSecretBuilder( - maybeSubmittedResourceIdentifiers.map(_.secrets())) - .map(_.build()) - - val containerLocalizedFilesResolver = initContainerComponentsProvider - .provideContainerLocalizedFilesResolver(mainAppResource) - val resolvedSparkJars = containerLocalizedFilesResolver.resolveSubmittedSparkJars() - val resolvedSparkFiles = containerLocalizedFilesResolver.resolveSubmittedSparkFiles() - val resolvedPySparkFiles = containerLocalizedFilesResolver.resolveSubmittedPySparkFiles() - val resolvedPrimaryPySparkResource = pythonResource.map { - p => p.primaryPySparkResource(containerLocalizedFilesResolver) - }.getOrElse("") - val initContainerBundler = initContainerComponentsProvider - .provideInitContainerBundle(maybeSubmittedResourceIdentifiers.map(_.ids()), - resolvedSparkJars ++ resolvedSparkFiles) - - val podWithInitContainer = initContainerBundler.map( - _.sparkPodInitContainerBootstrap - .bootstrapInitContainerAndVolumes(driverContainer.getName, basePod)) - .getOrElse(basePod) - val sparkConfWithExecutorInit = initContainerBundler.map( - _.executorInitContainerConfiguration - .configureSparkConfForExecutorInitContainer(sparkConf)) - .getOrElse(sparkConf) - val credentialsMounter = kubernetesCredentialsMounterProvider - .getDriverPodKubernetesCredentialsMounter() - val credentialsSecret = credentialsMounter.createCredentialsSecret() - val podWithInitContainerAndMountedCreds = credentialsMounter.mountDriverKubernetesCredentials( - podWithInitContainer, driverContainer.getName, credentialsSecret) - val resolvedSparkConf = credentialsMounter.setDriverPodKubernetesCredentialLocations( - sparkConfWithExecutorInit) - if (resolvedSparkJars.nonEmpty) { - resolvedSparkConf.set("spark.jars", resolvedSparkJars.mkString(",")) - } - if (resolvedSparkFiles.nonEmpty) { - resolvedSparkConf.set("spark.files", resolvedSparkFiles.mkString(",")) - } - resolvedSparkConf.setIfMissing(KUBERNETES_DRIVER_POD_NAME, kubernetesDriverPodName) - resolvedSparkConf.set("spark.app.id", kubernetesAppId) - resolvedSparkConf.set(KUBERNETES_EXECUTOR_POD_NAME_PREFIX, kubernetesResourceNamePrefix) - // We don't need this anymore since we just set the JVM options on the environment - resolvedSparkConf.remove(org.apache.spark.internal.config.DRIVER_JAVA_OPTIONS) - val resolvedLocalClasspath = containerLocalizedFilesResolver - .resolveSubmittedAndRemoteSparkJars() - val resolvedDriverJavaOpts = resolvedSparkConf.getAll.map { - case (confKey, confValue) => s"-D$confKey=$confValue" - }.mkString(" ") + driverJavaOptions.map(" " + _).getOrElse("") - val resolvedDriverPodBuilder = podWithInitContainerAndMountedCreds.editSpec() - .editMatchingContainer(new ContainerNameEqualityPredicate(driverContainer.getName)) - .addNewEnv() - .withName(ENV_MOUNTED_CLASSPATH) - .withValue(resolvedLocalClasspath.mkString(File.pathSeparator)) - .endEnv() - .addNewEnv() - .withName(ENV_DRIVER_JAVA_OPTS) - .withValue(resolvedDriverJavaOpts) - .endEnv() - .endContainer() - .endSpec() - val driverPodFileMounter = initContainerComponentsProvider.provideDriverPodFileMounter() - val resolvedDriverPod = pythonResource.map { - p => p.driverPodWithPySparkEnvs( - driverPodFileMounter, - resolvedPrimaryPySparkResource, - resolvedPySparkFiles.mkString(","), - driverContainer.getName, - resolvedDriverPodBuilder - )}.getOrElse(resolvedDriverPodBuilder.build()) + .build() Utils.tryWithResource( - kubernetesClient - .pods() - .withName(resolvedDriverPod.getMetadata.getName) - .watch(loggingPodStatusWatcher)) { _ => + kubernetesClient + .pods() + .withName(resolvedDriverPod.getMetadata.getName) + .watch(loggingPodStatusWatcher)) { _ => val createdDriverPod = kubernetesClient.pods().create(resolvedDriverPod) try { - val driverOwnedResources = initContainerBundler.map( - _.sparkInitContainerConfigMap).toSeq ++ - maybeSubmittedDependenciesSecret.toSeq ++ - credentialsSecret.toSeq - val driverPodOwnerReference = new OwnerReferenceBuilder() - .withName(createdDriverPod.getMetadata.getName) - .withApiVersion(createdDriverPod.getApiVersion) - .withUid(createdDriverPod.getMetadata.getUid) - .withKind(createdDriverPod.getKind) - .withController(true) - .build() - driverOwnedResources.foreach { resource => - val originalMetadata = resource.getMetadata - originalMetadata.setOwnerReferences(Collections.singletonList(driverPodOwnerReference)) + if (currentDriverSpec.otherKubernetesResources.nonEmpty) { + val driverPodOwnerReference = new OwnerReferenceBuilder() + .withName(createdDriverPod.getMetadata.getName) + .withApiVersion(createdDriverPod.getApiVersion) + .withUid(createdDriverPod.getMetadata.getUid) + .withKind(createdDriverPod.getKind) + .withController(true) + .build() + currentDriverSpec.otherKubernetesResources.foreach { resource => + val originalMetadata = resource.getMetadata + originalMetadata.setOwnerReferences(Collections.singletonList(driverPodOwnerReference)) + } + val otherKubernetesResources = currentDriverSpec.otherKubernetesResources + kubernetesClient.resourceList(otherKubernetesResources: _*).createOrReplace() } - kubernetesClient.resourceList(driverOwnedResources: _*).createOrReplace() } catch { case e: Throwable => kubernetesClient.pods().delete(createdDriverPod) @@ -283,100 +140,47 @@ private[spark] class Client( } private[spark] object Client { - def main(args: Array[String]): Unit = { - val sparkConf = new SparkConf(true) - val mainAppResource = args(0) - val mainClass = args(1) - val appArgs = args.drop(2) - run(sparkConf, mainAppResource, mainClass, appArgs) - } - def run( - sparkConf: SparkConf, - mainAppResource: String, - mainClass: String, - appArgs: Array[String]): Unit = { - val isPython = mainAppResource.endsWith(".py") - val pythonResource: Option[PythonSubmissionResourcesImpl] = - if (isPython) { - Option(new PythonSubmissionResourcesImpl(mainAppResource, appArgs)) - } else None - // Since you might need jars for SQL UDFs in PySpark - def sparkJarFilter() : Seq[String] = - pythonResource.map { p => p.sparkJars}.getOrElse( - Option(mainAppResource) - .filterNot(_ == SparkLauncher.NO_RESOURCE) - .toSeq) - val sparkJars = sparkConf.getOption("spark.jars") - .map(_.split(",")) - .getOrElse(Array.empty[String]) ++ sparkJarFilter() - val launchTime = System.currentTimeMillis - val sparkFiles = sparkConf.getOption("spark.files") - .map(_.split(",")) - .getOrElse(Array.empty[String]) - val pySparkFilesOption = pythonResource.map {p => p.pySparkFiles} - validateNoDuplicateFileNames(sparkJars) - validateNoDuplicateFileNames(sparkFiles) - pySparkFilesOption.foreach {b => validateNoDuplicateFileNames(b)} - val pySparkFiles = pySparkFilesOption.getOrElse(Array.empty[String]) + def run(sparkConf: SparkConf, clientArguments: ClientArguments): Unit = { + val namespace = sparkConf.get(KUBERNETES_NAMESPACE) + val launchTime = System.currentTimeMillis() val appName = sparkConf.getOption("spark.app.name").getOrElse("spark") - // The resource name prefix is derived from the application name, making it easy to connect the - // names of the Kubernetes resources from e.g. Kubectl or the Kubernetes dashboard to the - // application the user submitted. However, we can't use the application name in the label, as - // label values are considerably restrictive, e.g. must be no longer than 63 characters in - // length. So we generate a separate identifier for the app ID itself, and bookkeeping that - // requires finding "all pods for this application" should use the kubernetesAppId. - val kubernetesResourceNamePrefix = s"$appName-$launchTime".toLowerCase.replaceAll("\\.", "-") val kubernetesAppId = s"spark-${UUID.randomUUID().toString.replaceAll("-", "")}" - val namespace = sparkConf.get(KUBERNETES_NAMESPACE) val master = resolveK8sMaster(sparkConf.get("spark.master")) - val sslOptionsProvider = new ResourceStagingServerSslOptionsProviderImpl(sparkConf) - val initContainerComponentsProvider = new DriverInitContainerComponentsProviderImpl( - sparkConf, - kubernetesResourceNamePrefix, - namespace, - sparkJars, - sparkFiles, - pySparkFiles, - sslOptionsProvider.getSslOptions) + val submissionStepsOrchestrator = new KubernetesSubmissionStepsOrchestrator( + namespace, + kubernetesAppId, + launchTime, + clientArguments.mainAppResource, + appName, + clientArguments.mainClass, + clientArguments.driverArgs, + clientArguments.otherPyFiles, + sparkConf) + val waitForAppCompletion = sparkConf.get(WAIT_FOR_APP_COMPLETION) + val loggingInterval = Option(sparkConf.get(REPORT_INTERVAL)) + .filter( _ => waitForAppCompletion) + val loggingPodStatusWatcher = new LoggingPodStatusWatcherImpl( + kubernetesAppId, loggingInterval) Utils.tryWithResource(SparkKubernetesClientFactory.createKubernetesClient( - master, - Some(namespace), - APISERVER_AUTH_SUBMISSION_CONF_PREFIX, - sparkConf, - None, - None)) { kubernetesClient => - val kubernetesCredentialsMounterProvider = - new DriverPodKubernetesCredentialsMounterProviderImpl( - sparkConf, kubernetesResourceNamePrefix) - val waitForAppCompletion = sparkConf.get(WAIT_FOR_APP_COMPLETION) - val loggingInterval = Option(sparkConf.get(REPORT_INTERVAL)) - .filter( _ => waitForAppCompletion) - val loggingPodStatusWatcher = new LoggingPodStatusWatcherImpl( - kubernetesResourceNamePrefix, loggingInterval) + master, + Some(namespace), + APISERVER_AUTH_SUBMISSION_CONF_PREFIX, + sparkConf, + None, + None)) { kubernetesClient => new Client( - appName, - kubernetesResourceNamePrefix, - kubernetesAppId, - mainAppResource, - pythonResource, - mainClass, - sparkConf, - appArgs, - waitForAppCompletion, - kubernetesClient, - initContainerComponentsProvider, - kubernetesCredentialsMounterProvider, - loggingPodStatusWatcher).run() + submissionStepsOrchestrator.getAllSubmissionSteps(), + sparkConf, + kubernetesClient, + waitForAppCompletion, + appName, + loggingPodStatusWatcher).run() } } - private def validateNoDuplicateFileNames(allFiles: Seq[String]): Unit = { - val fileNamesToUris = allFiles.map { file => - (new File(Utils.resolveURI(file).getPath).getName, file) - } - fileNamesToUris.groupBy(_._1).foreach { - case (fileName, urisWithFileName) => - require(urisWithFileName.size == 1, "Cannot add multiple files with the same name, but" + - s" file name $fileName is shared by all of these URIs: $urisWithFileName") - } + + def main(args: Array[String]): Unit = { + val parsedArguments = ClientArguments.fromCommandLineArgs(args) + val sparkConf = new SparkConf() + run(sparkConf, parsedArguments) } } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/ContainerLocalizedFilesResolver.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/ContainerLocalizedFilesResolver.scala deleted file mode 100644 index 7345589c59c42..0000000000000 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/ContainerLocalizedFilesResolver.scala +++ /dev/null @@ -1,87 +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.submit - -import java.io.File - -import org.apache.spark.util.Utils - -private[spark] trait ContainerLocalizedFilesResolver { - def resolveSubmittedAndRemoteSparkJars(): Seq[String] - def resolveSubmittedSparkJars(): Seq[String] - def resolveSubmittedSparkFiles(): Seq[String] - def resolveSubmittedPySparkFiles(): Seq[String] - def resolvePrimaryResourceFile(): String -} - -private[spark] class ContainerLocalizedFilesResolverImpl( - sparkJars: Seq[String], - sparkFiles: Seq[String], - pySparkFiles: Seq[String], - primaryPyFile: String, - jarsDownloadPath: String, - filesDownloadPath: String ) extends ContainerLocalizedFilesResolver { - - - override def resolveSubmittedAndRemoteSparkJars(): Seq[String] = { - sparkJars.map { jar => - val jarUri = Utils.resolveURI(jar) - Option(jarUri.getScheme).getOrElse("file") match { - case "local" => - jarUri.getPath - case _ => - val jarFileName = new File(jarUri.getPath).getName - s"$jarsDownloadPath/$jarFileName" - } - } - } - - override def resolveSubmittedSparkJars(): Seq[String] = { - resolveSubmittedFiles(sparkJars, jarsDownloadPath) - } - - override def resolveSubmittedSparkFiles(): Seq[String] = { - resolveSubmittedFiles(sparkFiles, filesDownloadPath) - } - - override def resolveSubmittedPySparkFiles(): Seq[String] = { - def filterMainResource(x: String) = x match { - case `primaryPyFile` => None - case _ => Some(resolveFile(x, filesDownloadPath)) - } - pySparkFiles.flatMap(x => filterMainResource(x)) - } - - override def resolvePrimaryResourceFile(): String = { - Option(primaryPyFile).map(p => resolveFile(p, filesDownloadPath)).getOrElse("") - } - - private def resolveFile(file: String, downloadPath: String) = { - val fileUri = Utils.resolveURI(file) - Option(fileUri.getScheme).getOrElse("file") match { - case "file" => - val fileName = new File(fileUri.getPath).getName - s"$downloadPath/$fileName" - case _ => - file - } - } - - private def resolveSubmittedFiles(files: Seq[String], downloadPath: String): Seq[String] = { - files.map { file => resolveFile(file, downloadPath) } - } -} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/DriverInitContainerComponentsProvider.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/DriverInitContainerComponentsProvider.scala deleted file mode 100644 index 7f5a62e97abc0..0000000000000 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/DriverInitContainerComponentsProvider.scala +++ /dev/null @@ -1,223 +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.submit - -import io.fabric8.kubernetes.api.model.ConfigMap - -import org.apache.spark.{SparkConf, SSLOptions} -import org.apache.spark.deploy.kubernetes.{InitContainerResourceStagingServerSecretPluginImpl, OptionRequirements, SparkPodInitContainerBootstrap, SparkPodInitContainerBootstrapImpl} -import org.apache.spark.deploy.kubernetes.config._ -import org.apache.spark.deploy.kubernetes.constants._ -import org.apache.spark.deploy.rest.kubernetes.RetrofitClientFactoryImpl -import org.apache.spark.util.Utils - -/** - * Interface that wraps the provision of everything the submission client needs to set up the - * driver's init-container. This is all wrapped in the same place to ensure that related - * components are being constructed with consistent configurations with respect to one another. - */ -private[spark] trait DriverInitContainerComponentsProvider { - - def provideContainerLocalizedFilesResolver( - mainAppResource: String) : ContainerLocalizedFilesResolver - def provideInitContainerSubmittedDependencyUploader( - driverPodLabels: Map[String, String]): Option[SubmittedDependencyUploader] - def provideSubmittedDependenciesSecretBuilder( - maybeSubmittedResourceSecrets: Option[SubmittedResourceSecrets]) - : Option[SubmittedDependencySecretBuilder] - def provideInitContainerBootstrap(): SparkPodInitContainerBootstrap - def provideDriverPodFileMounter(): DriverPodKubernetesFileMounter - def provideInitContainerBundle(maybeSubmittedResourceIds: Option[SubmittedResourceIds], - uris: Iterable[String]): Option[InitContainerBundle] -} - -private[spark] class DriverInitContainerComponentsProviderImpl( - sparkConf: SparkConf, - kubernetesResourceNamePrefix: String, - namespace: String, - sparkJars: Seq[String], - sparkFiles: Seq[String], - pySparkFiles: Seq[String], - resourceStagingServerExternalSslOptions: SSLOptions) - extends DriverInitContainerComponentsProvider { - - private val maybeResourceStagingServerUri = sparkConf.get(RESOURCE_STAGING_SERVER_URI) - private val maybeResourceStagingServerInternalUri = - sparkConf.get(RESOURCE_STAGING_SERVER_INTERNAL_URI) - private val maybeResourceStagingServerInternalTrustStore = - sparkConf.get(RESOURCE_STAGING_SERVER_INTERNAL_TRUSTSTORE_FILE) - .orElse(sparkConf.get(RESOURCE_STAGING_SERVER_TRUSTSTORE_FILE)) - private val maybeResourceStagingServerInternalTrustStorePassword = - sparkConf.get(RESOURCE_STAGING_SERVER_INTERNAL_TRUSTSTORE_PASSWORD) - .orElse(sparkConf.get(RESOURCE_STAGING_SERVER_TRUSTSTORE_PASSWORD)) - private val maybeResourceStagingServerInternalTrustStoreType = - sparkConf.get(RESOURCE_STAGING_SERVER_INTERNAL_TRUSTSTORE_TYPE) - .orElse(sparkConf.get(RESOURCE_STAGING_SERVER_TRUSTSTORE_TYPE)) - private val maybeResourceStagingServerInternalClientCert = - sparkConf.get(RESOURCE_STAGING_SERVER_INTERNAL_CLIENT_CERT_PEM) - .orElse(sparkConf.get(RESOURCE_STAGING_SERVER_CLIENT_CERT_PEM)) - private val resourceStagingServerInternalSslEnabled = - sparkConf.get(RESOURCE_STAGING_SERVER_INTERNAL_SSL_ENABLED) - .orElse(sparkConf.get(RESOURCE_STAGING_SERVER_SSL_ENABLED)) - .getOrElse(false) - - OptionRequirements.requireNandDefined( - maybeResourceStagingServerInternalClientCert, - maybeResourceStagingServerInternalTrustStore, - "Cannot provide both a certificate file and a trustStore file for init-containers to" + - " use for contacting the resource staging server over TLS.") - - require(maybeResourceStagingServerInternalTrustStore.forall { trustStore => - Option(Utils.resolveURI(trustStore).getScheme).getOrElse("file") match { - case "file" | "local" => true - case _ => false - } - }, "TrustStore URI used for contacting the resource staging server from init containers must" + - " have no scheme, or scheme file://, or scheme local://.") - - require(maybeResourceStagingServerInternalClientCert.forall { trustStore => - Option(Utils.resolveURI(trustStore).getScheme).getOrElse("file") match { - case "file" | "local" => true - case _ => false - } - }, "Client cert file URI used for contacting the resource staging server from init containers" + - " must have no scheme, or scheme file://, or scheme local://.") - - private val jarsDownloadPath = sparkConf.get(INIT_CONTAINER_JARS_DOWNLOAD_LOCATION) - private val filesDownloadPath = sparkConf.get(INIT_CONTAINER_FILES_DOWNLOAD_LOCATION) - private val maybeSecretName = maybeResourceStagingServerUri.map { _ => - s"$kubernetesResourceNamePrefix-init-secret" - } - private val configMapName = s"$kubernetesResourceNamePrefix-init-config" - private val configMapKey = s"$kubernetesResourceNamePrefix-init-config-key" - private val initContainerImage = sparkConf.get(INIT_CONTAINER_DOCKER_IMAGE) - private val dockerImagePullPolicy = sparkConf.get(DOCKER_IMAGE_PULL_POLICY) - private val downloadTimeoutMinutes = sparkConf.get(INIT_CONTAINER_MOUNT_TIMEOUT) - private val pySparkSubmitted = KubernetesFileUtils.getOnlySubmitterLocalFiles(pySparkFiles) - - private def provideInitContainerConfigMap( - maybeSubmittedResourceIds: Option[SubmittedResourceIds]): ConfigMap = { - val submittedDependencyConfigPlugin = for { - stagingServerUri <- maybeResourceStagingServerUri - jarsResourceId <- maybeSubmittedResourceIds.map(_.jarsResourceId) - filesResourceId <- maybeSubmittedResourceIds.map(_.filesResourceId) - } yield { - new SubmittedDependencyInitContainerConfigPluginImpl( - // Configure the init-container with the internal URI over the external URI. - maybeResourceStagingServerInternalUri.getOrElse(stagingServerUri), - jarsResourceId, - filesResourceId, - INIT_CONTAINER_SUBMITTED_JARS_SECRET_KEY, - INIT_CONTAINER_SUBMITTED_FILES_SECRET_KEY, - INIT_CONTAINER_STAGING_SERVER_TRUSTSTORE_SECRET_KEY, - INIT_CONTAINER_STAGING_SERVER_CLIENT_CERT_SECRET_KEY, - resourceStagingServerInternalSslEnabled, - maybeResourceStagingServerInternalTrustStore, - maybeResourceStagingServerInternalClientCert, - maybeResourceStagingServerInternalTrustStorePassword, - maybeResourceStagingServerInternalTrustStoreType, - INIT_CONTAINER_SECRET_VOLUME_MOUNT_PATH) - } - new SparkInitContainerConfigMapBuilderImpl( - sparkJars, - sparkFiles ++ pySparkSubmitted, - jarsDownloadPath, - filesDownloadPath, - configMapName, - configMapKey, - submittedDependencyConfigPlugin).build() - } - - override def provideContainerLocalizedFilesResolver(mainAppResource: String) - : ContainerLocalizedFilesResolver = { - new ContainerLocalizedFilesResolverImpl( - sparkJars, sparkFiles, pySparkFiles, mainAppResource, jarsDownloadPath, filesDownloadPath) - } - - private def provideExecutorInitContainerConfiguration(): ExecutorInitContainerConfiguration = { - new ExecutorInitContainerConfigurationImpl( - maybeSecretName, - INIT_CONTAINER_SECRET_VOLUME_MOUNT_PATH, - configMapName, - configMapKey) - } - - override def provideInitContainerSubmittedDependencyUploader( - driverPodLabels: Map[String, String]): Option[SubmittedDependencyUploader] = { - maybeResourceStagingServerUri.map { stagingServerUri => - new SubmittedDependencyUploaderImpl( - driverPodLabels, - namespace, - stagingServerUri, - sparkJars, - sparkFiles ++ pySparkSubmitted, - resourceStagingServerExternalSslOptions, - RetrofitClientFactoryImpl) - } - } - - override def provideSubmittedDependenciesSecretBuilder( - maybeSubmittedResourceSecrets: Option[SubmittedResourceSecrets]) - : Option[SubmittedDependencySecretBuilder] = { - for { - secretName <- maybeSecretName - jarsResourceSecret <- maybeSubmittedResourceSecrets.map(_.jarsResourceSecret) - filesResourceSecret <- maybeSubmittedResourceSecrets.map(_.filesResourceSecret) - } yield { - new SubmittedDependencySecretBuilderImpl( - secretName, - jarsResourceSecret, - filesResourceSecret, - INIT_CONTAINER_SUBMITTED_JARS_SECRET_KEY, - INIT_CONTAINER_SUBMITTED_FILES_SECRET_KEY, - INIT_CONTAINER_STAGING_SERVER_TRUSTSTORE_SECRET_KEY, - INIT_CONTAINER_STAGING_SERVER_CLIENT_CERT_SECRET_KEY, - maybeResourceStagingServerInternalTrustStore, - maybeResourceStagingServerInternalClientCert) - } - } - - override def provideInitContainerBootstrap(): SparkPodInitContainerBootstrap = { - val resourceStagingServerSecretPlugin = maybeSecretName.map { secret => - new InitContainerResourceStagingServerSecretPluginImpl( - secret, INIT_CONTAINER_SECRET_VOLUME_MOUNT_PATH) - } - new SparkPodInitContainerBootstrapImpl( - initContainerImage, - dockerImagePullPolicy, - jarsDownloadPath, - filesDownloadPath, - downloadTimeoutMinutes, - configMapName, - configMapKey, - resourceStagingServerSecretPlugin) - } - override def provideDriverPodFileMounter(): DriverPodKubernetesFileMounter = { - new DriverPodKubernetesFileMounterImpl() - } - override def provideInitContainerBundle( - maybeSubmittedResourceIds: Option[SubmittedResourceIds], - uris: Iterable[String]): Option[InitContainerBundle] = { - // Bypass init-containers if `spark.jars` and `spark.files` and '--py-rilfes' - // is empty or only has `local://` URIs - if ((KubernetesFileUtils.getNonContainerLocalFiles(uris) ++ pySparkSubmitted).nonEmpty) { - Some(InitContainerBundle(provideInitContainerConfigMap(maybeSubmittedResourceIds), - provideInitContainerBootstrap(), - provideExecutorInitContainerConfiguration())) - } else None - } -} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/DriverPodKubernetesCredentialsMounter.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/DriverPodKubernetesCredentialsMounter.scala deleted file mode 100644 index 25e7c3b3ebd89..0000000000000 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/DriverPodKubernetesCredentialsMounter.scala +++ /dev/null @@ -1,184 +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.submit - -import io.fabric8.kubernetes.api.model.{PodBuilder, Secret, SecretBuilder} -import scala.collection.JavaConverters._ -import scala.language.implicitConversions - -import org.apache.spark.SparkConf -import org.apache.spark.deploy.kubernetes.KubernetesCredentials -import org.apache.spark.deploy.kubernetes.config._ -import org.apache.spark.deploy.kubernetes.constants._ - -private[spark] trait DriverPodKubernetesCredentialsMounter { - - /** - * Set fields on the Spark configuration that indicate where the driver pod is - * to find its Kubernetes credentials for requesting executors. - */ - def setDriverPodKubernetesCredentialLocations(sparkConf: SparkConf): SparkConf - - /** - * Create the Kubernetes secret object that correspond to the driver's credentials - * that have to be created and mounted into the driver pod. The single Secret - * object contains all of the data entries for the driver pod's Kubernetes - * credentials. Returns empty if no secrets are to be mounted. - */ - def createCredentialsSecret(): Option[Secret] - - /** - * Mount any Kubernetes credentials from the submitting machine's disk into the driver pod. The - * secret that is passed in here should have been created from createCredentialsSecret so that - * the implementation does not need to hold its state. - */ - def mountDriverKubernetesCredentials( - originalPodSpec: PodBuilder, - driverContainerName: String, - credentialsSecret: Option[Secret]): PodBuilder -} - -private[spark] class DriverPodKubernetesCredentialsMounterImpl( - kubernetesAppId: String, - submitterLocalDriverPodKubernetesCredentials: KubernetesCredentials, - maybeUserSpecifiedMountedClientKeyFile: Option[String], - maybeUserSpecifiedMountedClientCertFile: Option[String], - maybeUserSpecifiedMountedOAuthTokenFile: Option[String], - maybeUserSpecifiedMountedCaCertFile: Option[String]) - extends DriverPodKubernetesCredentialsMounter { - - override def setDriverPodKubernetesCredentialLocations(sparkConf: SparkConf): SparkConf = { - val resolvedMountedClientKeyFile = resolveSecretLocation( - maybeUserSpecifiedMountedClientKeyFile, - submitterLocalDriverPodKubernetesCredentials.clientKeyDataBase64, - DRIVER_CREDENTIALS_CLIENT_KEY_PATH) - val resolvedMountedClientCertFile = resolveSecretLocation( - maybeUserSpecifiedMountedClientCertFile, - submitterLocalDriverPodKubernetesCredentials.clientCertDataBase64, - DRIVER_CREDENTIALS_CLIENT_CERT_PATH) - val resolvedMountedCaCertFile = resolveSecretLocation( - maybeUserSpecifiedMountedCaCertFile, - submitterLocalDriverPodKubernetesCredentials.caCertDataBase64, - DRIVER_CREDENTIALS_CA_CERT_PATH) - val resolvedMountedOAuthTokenFile = resolveSecretLocation( - maybeUserSpecifiedMountedOAuthTokenFile, - submitterLocalDriverPodKubernetesCredentials.oauthTokenBase64, - DRIVER_CREDENTIALS_OAUTH_TOKEN_PATH) - val sparkConfWithCredentialLocations = sparkConf.clone() - .setOption( - s"$APISERVER_AUTH_DRIVER_MOUNTED_CONF_PREFIX.$CA_CERT_FILE_CONF_SUFFIX", - resolvedMountedCaCertFile) - .setOption( - s"$APISERVER_AUTH_DRIVER_MOUNTED_CONF_PREFIX.$CLIENT_KEY_FILE_CONF_SUFFIX", - resolvedMountedClientKeyFile) - .setOption( - s"$APISERVER_AUTH_DRIVER_MOUNTED_CONF_PREFIX.$CLIENT_CERT_FILE_CONF_SUFFIX", - resolvedMountedClientCertFile) - .setOption( - s"$APISERVER_AUTH_DRIVER_MOUNTED_CONF_PREFIX.$OAUTH_TOKEN_FILE_CONF_SUFFIX", - resolvedMountedOAuthTokenFile) - // Redact all OAuth token values - sparkConfWithCredentialLocations - .getAll - .filter(_._1.endsWith(OAUTH_TOKEN_CONF_SUFFIX)).map(_._1) - .foreach { - sparkConfWithCredentialLocations.set(_, "") - } - sparkConfWithCredentialLocations - } - - override def createCredentialsSecret(): Option[Secret] = { - val allSecretData = - resolveSecretData( - maybeUserSpecifiedMountedClientKeyFile, - submitterLocalDriverPodKubernetesCredentials.clientKeyDataBase64, - DRIVER_CREDENTIALS_CLIENT_KEY_SECRET_NAME) ++ - resolveSecretData( - maybeUserSpecifiedMountedClientCertFile, - submitterLocalDriverPodKubernetesCredentials.clientCertDataBase64, - DRIVER_CREDENTIALS_CLIENT_CERT_SECRET_NAME) ++ - resolveSecretData( - maybeUserSpecifiedMountedCaCertFile, - submitterLocalDriverPodKubernetesCredentials.caCertDataBase64, - DRIVER_CREDENTIALS_CA_CERT_SECRET_NAME) ++ - resolveSecretData( - maybeUserSpecifiedMountedOAuthTokenFile, - submitterLocalDriverPodKubernetesCredentials.oauthTokenBase64, - DRIVER_CREDENTIALS_OAUTH_TOKEN_SECRET_NAME) - if (allSecretData.isEmpty) { - None - } else { - Some(new SecretBuilder() - .withNewMetadata().withName(s"$kubernetesAppId-kubernetes-credentials").endMetadata() - .withData(allSecretData.asJava) - .build()) - } - } - - override def mountDriverKubernetesCredentials( - originalPodSpec: PodBuilder, - driverContainerName: String, - credentialsSecret: Option[Secret]): PodBuilder = { - credentialsSecret.map { secret => - originalPodSpec.editSpec() - .addNewVolume() - .withName(DRIVER_CREDENTIALS_SECRET_VOLUME_NAME) - .withNewSecret().withSecretName(secret.getMetadata.getName).endSecret() - .endVolume() - .editMatchingContainer(new ContainerNameEqualityPredicate(driverContainerName)) - .addNewVolumeMount() - .withName(DRIVER_CREDENTIALS_SECRET_VOLUME_NAME) - .withMountPath(DRIVER_CREDENTIALS_SECRETS_BASE_DIR) - .endVolumeMount() - .endContainer() - .endSpec() - }.getOrElse(originalPodSpec) - } - - private def resolveSecretLocation( - mountedUserSpecified: Option[String], - valueMountedFromSubmitter: Option[String], - mountedCanonicalLocation: String): Option[String] = { - mountedUserSpecified.orElse(valueMountedFromSubmitter.map( _ => { - mountedCanonicalLocation - })) - } - - private def resolveSecretData( - mountedUserSpecified: Option[String], - valueMountedFromSubmitter: Option[String], - secretName: String): Map[String, String] = { - mountedUserSpecified.map { _ => Map.empty[String, String]} - .getOrElse { - valueMountedFromSubmitter.map { valueBase64 => - Map(secretName -> valueBase64) - }.getOrElse(Map.empty[String, String]) - } - } - - private implicit def augmentSparkConf(sparkConf: SparkConf): OptionSettableSparkConf = { - new OptionSettableSparkConf(sparkConf) - } -} - -private class OptionSettableSparkConf(sparkConf: SparkConf) { - def setOption(configEntry: String, option: Option[String]): SparkConf = { - option.map( opt => { - sparkConf.set(configEntry, opt) - }).getOrElse(sparkConf) - } -} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/DriverPodKubernetesCredentialsMounterProvider.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/DriverPodKubernetesCredentialsMounterProvider.scala deleted file mode 100644 index 913279198146a..0000000000000 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/DriverPodKubernetesCredentialsMounterProvider.scala +++ /dev/null @@ -1,49 +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.submit - -import org.apache.spark.SparkConf -import org.apache.spark.deploy.kubernetes.config._ - -private[spark] trait DriverPodKubernetesCredentialsMounterProvider { - - def getDriverPodKubernetesCredentialsMounter() - : DriverPodKubernetesCredentialsMounter -} - -private[spark] class DriverPodKubernetesCredentialsMounterProviderImpl( - sparkConf: SparkConf, - kubernetesAppId: String) - extends DriverPodKubernetesCredentialsMounterProvider { - - override def getDriverPodKubernetesCredentialsMounter() - : DriverPodKubernetesCredentialsMounter = { - val submitterLocalDriverPodKubernetesCredentials = - new DriverPodKubernetesCredentialsProvider(sparkConf).get() - new DriverPodKubernetesCredentialsMounterImpl( - kubernetesAppId, - submitterLocalDriverPodKubernetesCredentials, - sparkConf.getOption( - s"$APISERVER_AUTH_DRIVER_MOUNTED_CONF_PREFIX.$CLIENT_KEY_FILE_CONF_SUFFIX"), - sparkConf.getOption( - s"$APISERVER_AUTH_DRIVER_MOUNTED_CONF_PREFIX.$CLIENT_CERT_FILE_CONF_SUFFIX"), - sparkConf.getOption( - s"$APISERVER_AUTH_DRIVER_MOUNTED_CONF_PREFIX.$OAUTH_TOKEN_FILE_CONF_SUFFIX"), - sparkConf.getOption( - s"$APISERVER_AUTH_DRIVER_MOUNTED_CONF_PREFIX.$CA_CERT_FILE_CONF_SUFFIX")) - } -} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/DriverPodKubernetesCredentialsProvider.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/DriverPodKubernetesCredentialsProvider.scala deleted file mode 100644 index 41b0cf8ceaeab..0000000000000 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/DriverPodKubernetesCredentialsProvider.scala +++ /dev/null @@ -1,63 +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.submit - -import java.io.File - -import com.google.common.base.Charsets -import com.google.common.io.{BaseEncoding, Files} - -import org.apache.spark.SparkConf -import org.apache.spark.deploy.kubernetes.KubernetesCredentials -import org.apache.spark.deploy.kubernetes.config._ -import org.apache.spark.internal.config.OptionalConfigEntry - -private[spark] class DriverPodKubernetesCredentialsProvider(sparkConf: SparkConf) { - - def get(): KubernetesCredentials = { - val oauthTokenBase64 = sparkConf - .getOption(s"$APISERVER_AUTH_DRIVER_CONF_PREFIX.$OAUTH_TOKEN_CONF_SUFFIX") - .map { token => - BaseEncoding.base64().encode(token.getBytes(Charsets.UTF_8)) - } - val caCertDataBase64 = safeFileConfToBase64( - s"$APISERVER_AUTH_DRIVER_CONF_PREFIX.$CA_CERT_FILE_CONF_SUFFIX", - s"Driver CA cert file provided at %s does not exist or is not a file.") - val clientKeyDataBase64 = safeFileConfToBase64( - s"$APISERVER_AUTH_DRIVER_CONF_PREFIX.$CLIENT_KEY_FILE_CONF_SUFFIX", - s"Driver client key file provided at %s does not exist or is not a file.") - val clientCertDataBase64 = safeFileConfToBase64( - s"$APISERVER_AUTH_DRIVER_CONF_PREFIX.$CLIENT_CERT_FILE_CONF_SUFFIX", - s"Driver client cert file provided at %s does not exist or is not a file.") - KubernetesCredentials( - oauthTokenBase64 = oauthTokenBase64, - caCertDataBase64 = caCertDataBase64, - clientKeyDataBase64 = clientKeyDataBase64, - clientCertDataBase64 = clientCertDataBase64) - } - - private def safeFileConfToBase64( - conf: String, - fileNotFoundFormatString: String): Option[String] = { - sparkConf.getOption(conf) - .map(new File(_)) - .map { file => - require(file.isFile, String.format(fileNotFoundFormatString, file.getAbsolutePath)) - BaseEncoding.base64().encode(Files.toByteArray(file)) - } - } -} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/DriverPodKubernetesFileMounter.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/DriverPodKubernetesFileMounter.scala deleted file mode 100644 index cc0ef0eedb457..0000000000000 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/DriverPodKubernetesFileMounter.scala +++ /dev/null @@ -1,55 +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.submit - -import io.fabric8.kubernetes.api.model.{Container, PodBuilder} - -import org.apache.spark.deploy.kubernetes.constants._ - - /** - * Trait that is responsible for providing full file-paths dynamically after - * the filesDownloadPath has been defined. The file-names are then stored in the - * environmental variables in the driver-pod. - */ -private[spark] trait DriverPodKubernetesFileMounter { - def addPySparkFiles(primaryFile: String, pySparkFiles: String, - mainContainerName: String, originalPodSpec: PodBuilder) : PodBuilder -} - -private[spark] class DriverPodKubernetesFileMounterImpl() - extends DriverPodKubernetesFileMounter { - override def addPySparkFiles( - primaryFile: String, - pySparkFiles: String, - mainContainerName: String, - originalPodSpec: PodBuilder): PodBuilder = { - - originalPodSpec - .editSpec() - .editMatchingContainer(new ContainerNameEqualityPredicate(mainContainerName)) - .addNewEnv() - .withName(ENV_PYSPARK_PRIMARY) - .withValue(primaryFile) - .endEnv() - .addNewEnv() - .withName(ENV_PYSPARK_FILES) - .withValue(pySparkFiles) - .endEnv() - .endContainer() - .endSpec() - } -} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/InitContainerUtil.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/InitContainerUtil.scala index 9b7faaa78a9aa..1c5e199a52110 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/InitContainerUtil.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/InitContainerUtil.scala @@ -18,7 +18,7 @@ package org.apache.spark.deploy.kubernetes.submit import com.fasterxml.jackson.databind.ObjectMapper import com.fasterxml.jackson.module.scala.DefaultScalaModule -import io.fabric8.kubernetes.api.model.{Container, PodBuilder} +import io.fabric8.kubernetes.api.model.{Container, Pod, PodBuilder} import scala.collection.JavaConverters._ import org.apache.spark.deploy.kubernetes.constants._ @@ -27,10 +27,9 @@ private[spark] object InitContainerUtil { private val OBJECT_MAPPER = new ObjectMapper().registerModule(new DefaultScalaModule) - def appendInitContainer( - originalPodSpec: PodBuilder, initContainer: Container): PodBuilder = { + def appendInitContainer(originalPodSpec: Pod, initContainer: Container): Pod = { val resolvedInitContainers = originalPodSpec - .editMetadata() + .getMetadata .getAnnotations .asScala .get(INIT_CONTAINER_ANNOTATION) @@ -40,10 +39,11 @@ private[spark] object InitContainerUtil { existingInitContainers ++ Seq(initContainer) }.getOrElse(Seq(initContainer)) val resolvedSerializedInitContainers = OBJECT_MAPPER.writeValueAsString(resolvedInitContainers) - originalPodSpec + new PodBuilder(originalPodSpec) .editMetadata() - .removeFromAnnotations(INIT_CONTAINER_ANNOTATION) - .addToAnnotations(INIT_CONTAINER_ANNOTATION, resolvedSerializedInitContainers) - .endMetadata() + .removeFromAnnotations(INIT_CONTAINER_ANNOTATION) + .addToAnnotations(INIT_CONTAINER_ANNOTATION, resolvedSerializedInitContainers) + .endMetadata() + .build() } } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/KubernetesFileUtils.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/KubernetesFileUtils.scala index d688bf29808fb..2006007db1d08 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/KubernetesFileUtils.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/KubernetesFileUtils.scala @@ -16,6 +16,8 @@ */ package org.apache.spark.deploy.kubernetes.submit +import java.io.File + import org.apache.spark.util.Utils private[spark] object KubernetesFileUtils { @@ -48,4 +50,47 @@ private[spark] object KubernetesFileUtils { def getOnlyRemoteFiles(uris: Iterable[String]): Iterable[String] = { filterUriStringsByScheme(uris, scheme => scheme != "file" && scheme != "local") } + + /** + * For the collection of uris, resolves any files as follows: + * - Files with scheme file:// are resolved to the download path + * - Files with scheme local:// resolve to just the path of the URI + * - Otherwise, the URI is returned as-is. + */ + def resolveSubmittedUris(fileUris: Iterable[String], fileDownloadPath: String) + : Iterable[String] = { + fileUris.map { uri => + val fileUri = Utils.resolveURI(uri) + val fileScheme = Option(fileUri.getScheme).getOrElse("file") + fileScheme match { + case "file" => + val fileName = new File(fileUri.getPath).getName + s"$fileDownloadPath/$fileName" + case "local" => + fileUri.getPath + case _ => uri + } + } + } + + /** + * If any file uri has any scheme other than local:// it is mapped as if the file + * was downloaded to the file download path. Otherwise, it is mapped to the path + * part of the URI. + */ + def resolveFilePaths(fileUris: Iterable[String], fileDownloadPath: String): Iterable[String] = { + fileUris.map { uri => + resolveFilePath(uri, fileDownloadPath) + } + } + + def resolveFilePath(uri: String, fileDownloadPath: String): String = { + val fileUri = Utils.resolveURI(uri) + if (Option(fileUri.getScheme).getOrElse("file") == "local") { + fileUri.getPath + } else { + val fileName = new File(fileUri.getPath).getName + s"$fileDownloadPath/$fileName" + } + } } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/KubernetesSubmissionStepsOrchestrator.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/KubernetesSubmissionStepsOrchestrator.scala new file mode 100644 index 0000000000000..f42afafbcb41a --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/KubernetesSubmissionStepsOrchestrator.scala @@ -0,0 +1,127 @@ +/* + * 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.submit + +import org.apache.spark.SparkConf +import org.apache.spark.deploy.kubernetes.ConfigurationUtils +import org.apache.spark.deploy.kubernetes.config._ +import org.apache.spark.deploy.kubernetes.constants._ +import org.apache.spark.deploy.kubernetes.submit.submitsteps.{BaseSubmissionStep, DependencyResolutionStep, DriverKubernetesCredentialsStep, InitContainerBootstrapStep, KubernetesSubmissionStep, PythonStep} +import org.apache.spark.deploy.kubernetes.submit.submitsteps.initcontainer.InitContainerStepsOrchestrator +import org.apache.spark.launcher.SparkLauncher + +/** + * Constructs the complete list of submission steps to run to deploy the Spark application. + */ +private[spark] class KubernetesSubmissionStepsOrchestrator( + namespace: String, + kubernetesAppId: String, + launchTime: Long, + mainAppResource: MainAppResource, + appName: String, + mainClass: String, + appArgs: Array[String], + additionalPythonFiles: Seq[String], + submissionSparkConf: SparkConf) { + + // The resource name prefix is derived from the application name, making it easy to connect the + // names of the Kubernetes resources from e.g. Kubectl or the Kubernetes dashboard to the + // application the user submitted. However, we can't use the application name in the label, as + // label values are considerably restrictive, e.g. must be no longer than 63 characters in + // length. So we generate a separate identifier for the app ID itself, and bookkeeping that + // requires finding "all pods for this application" should use the kubernetesAppId. + private val kubernetesResourceNamePrefix = + s"$appName-$launchTime".toLowerCase.replaceAll("\\.", "-") + private val jarsDownloadPath = submissionSparkConf.get(INIT_CONTAINER_JARS_DOWNLOAD_LOCATION) + private val filesDownloadPath = submissionSparkConf.get(INIT_CONTAINER_FILES_DOWNLOAD_LOCATION) + private val dockerImagePullPolicy = submissionSparkConf.get(DOCKER_IMAGE_PULL_POLICY) + private val initContainerConfigMapName = s"$kubernetesResourceNamePrefix-init-config" + + def getAllSubmissionSteps(): Seq[KubernetesSubmissionStep] = { + val additionalMainAppJar = mainAppResource match { + case JavaMainAppResource(resource) if resource != SparkLauncher.NO_RESOURCE => + Option(resource) + case _ => Option.empty + } + val additionalMainAppPythonFile = mainAppResource match { + case PythonMainAppResource(resource) if resource != SparkLauncher.NO_RESOURCE => + Option(resource) + case _ => Option.empty + } + val sparkJars = submissionSparkConf.getOption("spark.jars") + .map(_.split(",")) + .getOrElse(Array.empty[String]) ++ additionalMainAppJar.toSeq + val sparkFiles = submissionSparkConf.getOption("spark.files") + .map(_.split(",")) + .getOrElse(Array.empty[String]) ++ additionalMainAppPythonFile.toSeq + val driverCustomLabels = ConfigurationUtils.combinePrefixedKeyValuePairsWithDeprecatedConf( + submissionSparkConf, + KUBERNETES_DRIVER_LABEL_PREFIX, + KUBERNETES_DRIVER_LABELS, + "label") + require(!driverCustomLabels.contains(SPARK_APP_ID_LABEL), s"Label with key " + + s" $SPARK_APP_ID_LABEL is not allowed as it is reserved for Spark bookkeeping" + + s" operations.") + val allDriverLabels = driverCustomLabels ++ Map( + SPARK_APP_ID_LABEL -> kubernetesAppId, + SPARK_ROLE_LABEL -> SPARK_POD_DRIVER_ROLE) + val initialSubmissionStep = new BaseSubmissionStep( + kubernetesAppId, + kubernetesResourceNamePrefix, + allDriverLabels, + dockerImagePullPolicy, + appName, + mainClass, + appArgs, + submissionSparkConf) + val kubernetesCredentialsStep = new DriverKubernetesCredentialsStep( + submissionSparkConf, kubernetesAppId) + val pythonStep = mainAppResource match { + case PythonMainAppResource(mainPyResource) => + Option(new PythonStep(mainPyResource, additionalPythonFiles, filesDownloadPath)) + case _ => Option.empty[PythonStep] + } + val initContainerStepsOrchestrator = new InitContainerStepsOrchestrator( + namespace, + kubernetesResourceNamePrefix, + sparkJars, + sparkFiles, + jarsDownloadPath, + filesDownloadPath, + dockerImagePullPolicy, + allDriverLabels, + initContainerConfigMapName, + INIT_CONTAINER_CONFIG_MAP_KEY, + submissionSparkConf) + val initContainerSteps = initContainerStepsOrchestrator.getInitContainerSteps() + val initContainerBootstrapStep = new InitContainerBootstrapStep( + submissionSparkConf, + initContainerSteps, + initContainerConfigMapName, + INIT_CONTAINER_CONFIG_MAP_KEY) + val dependencyResolutionStep = new DependencyResolutionStep( + sparkJars, + sparkFiles, + jarsDownloadPath, + filesDownloadPath) + Seq( + initialSubmissionStep, + kubernetesCredentialsStep, + initContainerBootstrapStep, + dependencyResolutionStep) ++ pythonStep.toSeq + } +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/InitContainerBundle.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/MainAppResource.scala similarity index 71% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/InitContainerBundle.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/MainAppResource.scala index ba44f794d5811..436d531a850ff 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/InitContainerBundle.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/MainAppResource.scala @@ -16,11 +16,8 @@ */ package org.apache.spark.deploy.kubernetes.submit -import io.fabric8.kubernetes.api.model.ConfigMap +private[spark] sealed trait MainAppResource -import org.apache.spark.deploy.kubernetes.{SparkPodInitContainerBootstrap} +private[spark] case class PythonMainAppResource(primaryPyFile: String) extends MainAppResource -case class InitContainerBundle( - sparkInitContainerConfigMap: ConfigMap, - sparkPodInitContainerBootstrap: SparkPodInitContainerBootstrap, - executorInitContainerConfiguration: ExecutorInitContainerConfiguration) +private[spark] case class JavaMainAppResource(primaryResource: String) extends MainAppResource diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/PythonSubmissionResources.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/PythonSubmissionResources.scala deleted file mode 100644 index e615cf72116e1..0000000000000 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/PythonSubmissionResources.scala +++ /dev/null @@ -1,75 +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.submit - -import io.fabric8.kubernetes.api.model.{Pod, PodBuilder} - -private[spark] trait PythonSubmissionResources { - def sparkJars: Seq[String] - def pySparkFiles: Array[String] - def arguments: Array[String] - def primaryPySparkResource(containerLocalizedFilesResolver: ContainerLocalizedFilesResolver) - : String - def driverPodWithPySparkEnvs( - driverPodFileMounter: DriverPodKubernetesFileMounter, - resolvedPrimaryPySparkResource: String, - resolvedPySparkFiles: String, - driverContainerName: String, - driverPodBuilder: PodBuilder) : Pod -} - -private[spark] class PythonSubmissionResourcesImpl( - private val mainAppResource: String, - private val appArgs: Array[String] ) extends PythonSubmissionResources { - - private val pyFiles: Array[String] = { - Option(appArgs(0)).map(a => mainAppResource +: a.split(",")) - .getOrElse(Array(mainAppResource)) - } - - override def sparkJars: Seq[String] = Seq.empty[String] - - override def pySparkFiles: Array[String] = pyFiles - - override def arguments: Array[String] = { - pyFiles.toList match { - case Nil => appArgs - case a :: b => a match { - case _ if a == mainAppResource && b == Nil => appArgs - case _ => appArgs.drop(1) - } - } - } - override def primaryPySparkResource ( - containerLocalizedFilesResolver: ContainerLocalizedFilesResolver) : String = - containerLocalizedFilesResolver.resolvePrimaryResourceFile() - - override def driverPodWithPySparkEnvs( - driverPodFileMounter: DriverPodKubernetesFileMounter, - resolvedPrimaryPySparkResource: String, - resolvedPySparkFiles: String, - driverContainerName: String, - driverPodBuilder: PodBuilder) : Pod = { - driverPodFileMounter - .addPySparkFiles( - resolvedPrimaryPySparkResource, - resolvedPySparkFiles, - driverContainerName, - driverPodBuilder) - .build() - } -} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/SubmittedDependencyInitContainerConfigPlugin.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/SubmittedDependencyInitContainerConfigPlugin.scala deleted file mode 100644 index 06d3648efb89f..0000000000000 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/SubmittedDependencyInitContainerConfigPlugin.scala +++ /dev/null @@ -1,96 +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.submit - -import org.apache.spark.SparkException -import org.apache.spark.deploy.kubernetes.config._ -import org.apache.spark.internal.config.OptionalConfigEntry -import org.apache.spark.util.Utils - -private[spark] trait SubmittedDependencyInitContainerConfigPlugin { - /** - * Obtain configuration to fetch submitted dependencies from a resource staging server. - * This includes the resource identifiers for the jar and file bundles, as well as the - * remote location of the resource staging server, and the location of secret files for - * authenticating to the resource staging server. Note that the secret file paths here need to - * line up with the locations the secrets are mounted by - * SubmittedDependencyInitContainerVolumesPlugin; constants provide the consistency and - * convention for these to line up. - */ - def configurationsToFetchSubmittedDependencies(): Map[String, String] -} - -private[spark] class SubmittedDependencyInitContainerConfigPluginImpl( - internalResourceStagingServerUri: String, - jarsResourceId: String, - filesResourceId: String, - jarsSecretKey: String, - filesSecretKey: String, - trustStoreSecretKey: String, - clientCertSecretKey: String, - resourceStagingServerSslEnabled: Boolean, - maybeInternalTrustStoreUri: Option[String], - maybeInternalClientCertUri: Option[String], - maybeInternalTrustStorePassword: Option[String], - maybeInternalTrustStoreType: Option[String], - secretsVolumeMountPath: String) - extends SubmittedDependencyInitContainerConfigPlugin { - - override def configurationsToFetchSubmittedDependencies(): Map[String, String] = { - Map[String, String]( - RESOURCE_STAGING_SERVER_URI.key -> internalResourceStagingServerUri, - INIT_CONTAINER_DOWNLOAD_JARS_RESOURCE_IDENTIFIER.key -> jarsResourceId, - INIT_CONTAINER_DOWNLOAD_JARS_SECRET_LOCATION.key -> - s"$secretsVolumeMountPath/$jarsSecretKey", - INIT_CONTAINER_DOWNLOAD_FILES_RESOURCE_IDENTIFIER.key -> filesResourceId, - INIT_CONTAINER_DOWNLOAD_FILES_SECRET_LOCATION.key -> - s"$secretsVolumeMountPath/$filesSecretKey", - RESOURCE_STAGING_SERVER_SSL_ENABLED.key -> resourceStagingServerSslEnabled.toString) ++ - resolveSecretPath( - maybeInternalTrustStoreUri, - trustStoreSecretKey, - RESOURCE_STAGING_SERVER_TRUSTSTORE_FILE, - "TrustStore URI") ++ - resolveSecretPath( - maybeInternalClientCertUri, - clientCertSecretKey, - RESOURCE_STAGING_SERVER_CLIENT_CERT_PEM, - "Client certificate URI") ++ - maybeInternalTrustStorePassword.map { password => - (RESOURCE_STAGING_SERVER_TRUSTSTORE_PASSWORD.key, password) - }.toMap ++ - maybeInternalTrustStoreType.map { storeType => - (RESOURCE_STAGING_SERVER_TRUSTSTORE_TYPE.key, storeType) - }.toMap - } - - private def resolveSecretPath( - maybeUri: Option[String], - secretKey: String, - configEntry: OptionalConfigEntry[String], - uriType: String): Map[String, String] = { - maybeUri.map(Utils.resolveURI).map { uri => - val resolvedPath = Option(uri.getScheme).getOrElse("file") match { - case "file" => s"$secretsVolumeMountPath/$secretKey" - case "local" => uri.getPath - case invalid => throw new SparkException(s"$uriType has invalid scheme $invalid must be" + - s" local://, file://, or empty.") - } - (configEntry.key, resolvedPath) - }.toMap - } -} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/SubmittedDependencySecretBuilder.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/SubmittedDependencySecretBuilder.scala deleted file mode 100644 index 7850853df97e6..0000000000000 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/SubmittedDependencySecretBuilder.scala +++ /dev/null @@ -1,81 +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.submit - -import java.io.File - -import com.google.common.base.Charsets -import com.google.common.io.{BaseEncoding, Files} -import io.fabric8.kubernetes.api.model.{Secret, SecretBuilder} -import scala.collection.JavaConverters._ - -import org.apache.spark.util.Utils - -private[spark] trait SubmittedDependencySecretBuilder { - /** - * Construct a Kubernetes secret bundle that init-containers can use to retrieve an - * application's dependencies. - */ - def build(): Secret -} - -private[spark] class SubmittedDependencySecretBuilderImpl( - secretName: String, - jarsResourceSecret: String, - filesResourceSecret: String, - jarsSecretKey: String, - filesSecretKey: String, - trustStoreSecretKey: String, - clientCertSecretKey: String, - internalTrustStoreUri: Option[String], - internalClientCertUri: Option[String]) - extends SubmittedDependencySecretBuilder { - - override def build(): Secret = { - val trustStoreBase64 = convertFileToBase64IfSubmitterLocal( - trustStoreSecretKey, internalTrustStoreUri) - val clientCertBase64 = convertFileToBase64IfSubmitterLocal( - clientCertSecretKey, internalClientCertUri) - val jarsSecretBase64 = BaseEncoding.base64().encode(jarsResourceSecret.getBytes(Charsets.UTF_8)) - val filesSecretBase64 = BaseEncoding.base64().encode( - filesResourceSecret.getBytes(Charsets.UTF_8)) - val secretData = Map( - jarsSecretKey -> jarsSecretBase64, - filesSecretKey -> filesSecretBase64) ++ - trustStoreBase64 ++ - clientCertBase64 - val kubernetesSecret = new SecretBuilder() - .withNewMetadata() - .withName(secretName) - .endMetadata() - .addToData(secretData.asJava) - .build() - kubernetesSecret - } - - private def convertFileToBase64IfSubmitterLocal(secretKey: String, secretUri: Option[String]) - : Map[String, String] = { - secretUri.filter { trustStore => - Option(Utils.resolveURI(trustStore).getScheme).getOrElse("file") == "file" - }.map { uri => - val file = new File(Utils.resolveURI(uri).getPath) - require(file.isFile, "Dependency server trustStore provided at" + - file.getAbsolutePath + " does not exist or is not a file.") - (secretKey, BaseEncoding.base64().encode(Files.toByteArray(file))) - }.toMap - } -} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/BaseSubmissionStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/BaseSubmissionStep.scala new file mode 100644 index 0000000000000..e49262c20c745 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/BaseSubmissionStep.scala @@ -0,0 +1,136 @@ +/* + * 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.submit.submitsteps + +import io.fabric8.kubernetes.api.model.{ContainerBuilder, EnvVarBuilder, PodBuilder, QuantityBuilder} +import scala.collection.JavaConverters._ + +import org.apache.spark.SparkConf +import org.apache.spark.deploy.kubernetes.ConfigurationUtils +import org.apache.spark.deploy.kubernetes.config._ +import org.apache.spark.deploy.kubernetes.constants._ + +/** + * Represents the initial setup required for the driver. + */ +private[spark] class BaseSubmissionStep( + kubernetesAppId: String, + kubernetesResourceNamePrefix: String, + driverLabels: Map[String, String], + dockerImagePullPolicy: String, + appName: String, + mainClass: String, + appArgs: Array[String], + submissionSparkConf: SparkConf) + extends KubernetesSubmissionStep { + + private val kubernetesDriverPodName = submissionSparkConf.get(KUBERNETES_DRIVER_POD_NAME) + .getOrElse(s"$kubernetesResourceNamePrefix-driver") + private val driverExtraClasspath = submissionSparkConf.get( + org.apache.spark.internal.config.DRIVER_CLASS_PATH) + // CPU settings + private val driverCpuCores = submissionSparkConf.getOption("spark.driver.cores").getOrElse("1") + private val driverLimitCores = submissionSparkConf.getOption(KUBERNETES_DRIVER_LIMIT_CORES.key) + + // Memory settings + private val driverMemoryMb = submissionSparkConf.get( + org.apache.spark.internal.config.DRIVER_MEMORY) + private val memoryOverheadMb = submissionSparkConf + .get(KUBERNETES_DRIVER_MEMORY_OVERHEAD) + .getOrElse(math.max((MEMORY_OVERHEAD_FACTOR * driverMemoryMb).toInt, + MEMORY_OVERHEAD_MIN)) + private val driverContainerMemoryWithOverhead = driverMemoryMb + memoryOverheadMb + private val driverDockerImage = submissionSparkConf.get(DRIVER_DOCKER_IMAGE) + + override def prepareSubmission( + driverSpec: KubernetesDriverSpec): KubernetesDriverSpec = { + val driverExtraClasspathEnv = driverExtraClasspath.map { classPath => + new EnvVarBuilder() + .withName(ENV_SUBMIT_EXTRA_CLASSPATH) + .withValue(classPath) + .build() + } + val driverCpuQuantity = new QuantityBuilder(false) + .withAmount(driverCpuCores) + .build() + val driverMemoryQuantity = new QuantityBuilder(false) + .withAmount(s"${driverMemoryMb}M") + .build() + val driverMemoryLimitQuantity = new QuantityBuilder(false) + .withAmount(s"${driverContainerMemoryWithOverhead}M") + .build() + val maybeCpuLimitQuantity = driverLimitCores.map { limitCores => + ("cpu", new QuantityBuilder(false).withAmount(limitCores).build()) + } + val driverContainer = new ContainerBuilder(driverSpec.driverContainer) + .withName(DRIVER_CONTAINER_NAME) + .withImage(driverDockerImage) + .withImagePullPolicy(dockerImagePullPolicy) + .addToEnv(driverExtraClasspathEnv.toSeq: _*) + .addNewEnv() + .withName(ENV_DRIVER_MEMORY) + .withValue(driverContainerMemoryWithOverhead + "m") + .endEnv() + .addNewEnv() + .withName(ENV_DRIVER_MAIN_CLASS) + .withValue(mainClass) + .endEnv() + .addNewEnv() + .withName(ENV_DRIVER_ARGS) + .withValue(appArgs.mkString(" ")) + .endEnv() + .withNewResources() + .addToRequests("cpu", driverCpuQuantity) + .addToRequests("memory", driverMemoryQuantity) + .addToLimits("memory", driverMemoryLimitQuantity) + .addToLimits(maybeCpuLimitQuantity.toMap.asJava) + .endResources() + .build() + val baseDriverPod = new PodBuilder(driverSpec.driverPod) + .withNewMetadata() + .withName(kubernetesDriverPodName) + .addToLabels(driverLabels.asJava) + .addToAnnotations(getAllDriverAnnotations(submissionSparkConf).asJava) + .endMetadata() + .withNewSpec() + .withRestartPolicy("Never") + .endSpec() + .build() + val resolvedSparkConf = driverSpec.driverSparkConf.clone() + .setIfMissing(KUBERNETES_DRIVER_POD_NAME, kubernetesDriverPodName) + .set("spark.app.id", kubernetesAppId) + .set(KUBERNETES_EXECUTOR_POD_NAME_PREFIX, kubernetesResourceNamePrefix) + // We don't need this anymore since we just set the JVM options on the environment + .remove(org.apache.spark.internal.config.DRIVER_JAVA_OPTIONS) + driverSpec.copy( + driverPod = baseDriverPod, + driverSparkConf = resolvedSparkConf, + driverContainer = driverContainer) + } + + private def getAllDriverAnnotations(sparkConf: SparkConf): Map[String, String] = { + val driverCustomAnnotations = ConfigurationUtils.combinePrefixedKeyValuePairsWithDeprecatedConf( + sparkConf, + KUBERNETES_DRIVER_ANNOTATION_PREFIX, + KUBERNETES_DRIVER_ANNOTATIONS, + "annotation") + require(!driverCustomAnnotations.contains(SPARK_APP_NAME_ANNOTATION), + s"Annotation with key $SPARK_APP_NAME_ANNOTATION is not allowed as it is reserved for" + + s" Spark bookkeeping operations.") + driverCustomAnnotations ++ Map(SPARK_APP_NAME_ANNOTATION -> appName) + } +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DependencyResolutionStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DependencyResolutionStep.scala new file mode 100644 index 0000000000000..b248418abb7a8 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DependencyResolutionStep.scala @@ -0,0 +1,66 @@ +/* + * 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.submit.submitsteps + + +import java.io.File + +import io.fabric8.kubernetes.api.model.ContainerBuilder + +import org.apache.spark.deploy.kubernetes.constants._ +import org.apache.spark.deploy.kubernetes.submit.KubernetesFileUtils +import org.apache.spark.util.Utils + +/** + * Step that configures the classpath, spark.jars, and spark.files for the driver given that + * the init-container will download files to the download paths and that the user may provide + * files with local:// schemes. Note that this is separate from the init-container bootstrap + * step because jars with local schemes can still be provided even if the init-container is + * not being used, and those URIs still need to be resolved. + */ +private[spark] class DependencyResolutionStep( + sparkJars: Seq[String], + sparkFiles: Seq[String], + jarsDownloadPath: String, + filesDownloadPath: String) extends KubernetesSubmissionStep { + + override def prepareSubmission(driverSpec: KubernetesDriverSpec): KubernetesDriverSpec = { + val resolvedSparkJars = KubernetesFileUtils.resolveSubmittedUris(sparkJars, jarsDownloadPath) + val resolvedSparkFiles = KubernetesFileUtils.resolveSubmittedUris(sparkFiles, filesDownloadPath) + val sparkConfResolvedSparkDependencies = driverSpec.driverSparkConf.clone() + if (resolvedSparkJars.nonEmpty) { + sparkConfResolvedSparkDependencies.set("spark.jars", resolvedSparkJars.mkString(",")) + } + if (resolvedSparkFiles.nonEmpty) { + sparkConfResolvedSparkDependencies.set("spark.files", resolvedSparkFiles.mkString(",")) + } + val resolvedClasspath = KubernetesFileUtils.resolveFilePaths(sparkJars, jarsDownloadPath) + val driverContainerWithResolvedClasspath = if (resolvedClasspath.nonEmpty) { + new ContainerBuilder(driverSpec.driverContainer) + .addNewEnv() + .withName(ENV_MOUNTED_CLASSPATH) + .withValue(resolvedClasspath.mkString(File.pathSeparator)) + .endEnv() + .build() + } else { + driverSpec.driverContainer + } + driverSpec.copy( + driverContainer = driverContainerWithResolvedClasspath, + driverSparkConf = sparkConfResolvedSparkDependencies) + } +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DriverKubernetesCredentialsStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DriverKubernetesCredentialsStep.scala new file mode 100644 index 0000000000000..c3a6b5e456f79 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DriverKubernetesCredentialsStep.scala @@ -0,0 +1,218 @@ +/* + * 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.submit.submitsteps + +import java.io.File +import java.nio.charset.StandardCharsets + +import com.google.common.io.{BaseEncoding, Files} +import io.fabric8.kubernetes.api.model.{ContainerBuilder, PodBuilder, Secret, SecretBuilder} +import scala.collection.JavaConverters._ + +import org.apache.spark.SparkConf +import org.apache.spark.deploy.kubernetes.config._ +import org.apache.spark.deploy.kubernetes.constants._ + +private[spark] class DriverKubernetesCredentialsStep( + submissionSparkConf: SparkConf, + kubernetesAppId: String) extends KubernetesSubmissionStep { + + private val maybeMountedOAuthTokenFile = submissionSparkConf.getOption( + s"$APISERVER_AUTH_DRIVER_MOUNTED_CONF_PREFIX.$OAUTH_TOKEN_FILE_CONF_SUFFIX") + private val maybeMountedClientKeyFile = submissionSparkConf.getOption( + s"$APISERVER_AUTH_DRIVER_MOUNTED_CONF_PREFIX.$CLIENT_KEY_FILE_CONF_SUFFIX") + private val maybeMountedClientCertFile = submissionSparkConf.getOption( + s"$APISERVER_AUTH_DRIVER_MOUNTED_CONF_PREFIX.$CLIENT_CERT_FILE_CONF_SUFFIX") + private val maybeMountedCaCertFile = submissionSparkConf.getOption( + s"$APISERVER_AUTH_DRIVER_MOUNTED_CONF_PREFIX.$CA_CERT_FILE_CONF_SUFFIX") + /** + * Apply some transformation to the previous state of the driver to add a new feature to it. + */ + override def prepareSubmission(driverSpec: KubernetesDriverSpec): KubernetesDriverSpec = { + val driverSparkConf = driverSpec.driverSparkConf.clone(); + val oauthTokenBase64 = submissionSparkConf + .getOption(s"$APISERVER_AUTH_DRIVER_CONF_PREFIX.$OAUTH_TOKEN_CONF_SUFFIX") + .map { token => + BaseEncoding.base64().encode(token.getBytes(StandardCharsets.UTF_8)) + } + val caCertDataBase64 = safeFileConfToBase64( + s"$APISERVER_AUTH_DRIVER_CONF_PREFIX.$CA_CERT_FILE_CONF_SUFFIX", + s"Driver CA cert file provided at %s does not exist or is not a file.") + val clientKeyDataBase64 = safeFileConfToBase64( + s"$APISERVER_AUTH_DRIVER_CONF_PREFIX.$CLIENT_KEY_FILE_CONF_SUFFIX", + s"Driver client key file provided at %s does not exist or is not a file.") + val clientCertDataBase64 = safeFileConfToBase64( + s"$APISERVER_AUTH_DRIVER_CONF_PREFIX.$CLIENT_CERT_FILE_CONF_SUFFIX", + s"Driver client cert file provided at %s does not exist or is not a file.") + val driverSparkConfWithCredentialsLocations = setDriverPodKubernetesCredentialLocations( + driverSparkConf, + oauthTokenBase64, + caCertDataBase64, + clientKeyDataBase64, + clientCertDataBase64) + val kubernetesCredentialsSecret = createCredentialsSecret( + oauthTokenBase64, + caCertDataBase64, + clientKeyDataBase64, + clientCertDataBase64) + val driverPodWithMountedKubernetesCredentials = kubernetesCredentialsSecret.map { secret => + new PodBuilder(driverSpec.driverPod) + .editSpec() + .addNewVolume() + .withName(DRIVER_CREDENTIALS_SECRET_VOLUME_NAME) + .withNewSecret().withSecretName(secret.getMetadata.getName).endSecret() + .endVolume() + .endSpec() + .build() + }.getOrElse(driverSpec.driverPod) + val driverContainerWithMountedSecretVolume = kubernetesCredentialsSecret.map { secret => + new ContainerBuilder(driverSpec.driverContainer) + .addNewVolumeMount() + .withName(DRIVER_CREDENTIALS_SECRET_VOLUME_NAME) + .withMountPath(DRIVER_CREDENTIALS_SECRETS_BASE_DIR) + .endVolumeMount() + .build() + }.getOrElse(driverSpec.driverContainer) + driverSpec.copy( + driverPod = driverPodWithMountedKubernetesCredentials, + otherKubernetesResources = + driverSpec.otherKubernetesResources ++ kubernetesCredentialsSecret.toSeq, + driverSparkConf = driverSparkConfWithCredentialsLocations, + driverContainer = driverContainerWithMountedSecretVolume) + } + + private def createCredentialsSecret( + driverOAuthTokenBase64: Option[String], + driverCaCertDataBase64: Option[String], + driverClientKeyDataBase64: Option[String], + driverClientCertDataBase64: Option[String]): Option[Secret] = { + val allSecretData = + resolveSecretData( + maybeMountedClientKeyFile, + driverClientKeyDataBase64, + DRIVER_CREDENTIALS_CLIENT_KEY_SECRET_NAME) ++ + resolveSecretData( + maybeMountedClientCertFile, + driverClientCertDataBase64, + DRIVER_CREDENTIALS_CLIENT_CERT_SECRET_NAME) ++ + resolveSecretData( + maybeMountedCaCertFile, + driverCaCertDataBase64, + DRIVER_CREDENTIALS_CA_CERT_SECRET_NAME) ++ + resolveSecretData( + maybeMountedOAuthTokenFile, + driverOAuthTokenBase64, + DRIVER_CREDENTIALS_OAUTH_TOKEN_SECRET_NAME) + if (allSecretData.isEmpty) { + None + } else { + Some(new SecretBuilder() + .withNewMetadata().withName(s"$kubernetesAppId-kubernetes-credentials").endMetadata() + .withData(allSecretData.asJava) + .build()) + } + } + + private def setDriverPodKubernetesCredentialLocations( + driverSparkConf: SparkConf, + driverOauthTokenBase64: Option[String], + driverCaCertDataBase64: Option[String], + driverClientKeyDataBase64: Option[String], + driverClientCertDataBase64: Option[String]): SparkConf = { + val resolvedMountedOAuthTokenFile = resolveSecretLocation( + maybeMountedOAuthTokenFile, + driverOauthTokenBase64, + DRIVER_CREDENTIALS_OAUTH_TOKEN_PATH) + val resolvedMountedClientKeyFile = resolveSecretLocation( + maybeMountedClientKeyFile, + driverClientKeyDataBase64, + DRIVER_CREDENTIALS_CLIENT_KEY_PATH) + val resolvedMountedClientCertFile = resolveSecretLocation( + maybeMountedClientCertFile, + driverClientCertDataBase64, + DRIVER_CREDENTIALS_CLIENT_CERT_PATH) + val resolvedMountedCaCertFile = resolveSecretLocation( + maybeMountedCaCertFile, + driverCaCertDataBase64, + DRIVER_CREDENTIALS_CA_CERT_PATH) + val sparkConfWithCredentialLocations = driverSparkConf + .setOption( + s"$APISERVER_AUTH_DRIVER_MOUNTED_CONF_PREFIX.$CA_CERT_FILE_CONF_SUFFIX", + resolvedMountedCaCertFile) + .setOption( + s"$APISERVER_AUTH_DRIVER_MOUNTED_CONF_PREFIX.$CLIENT_KEY_FILE_CONF_SUFFIX", + resolvedMountedClientKeyFile) + .setOption( + s"$APISERVER_AUTH_DRIVER_MOUNTED_CONF_PREFIX.$CLIENT_CERT_FILE_CONF_SUFFIX", + resolvedMountedClientCertFile) + .setOption( + s"$APISERVER_AUTH_DRIVER_MOUNTED_CONF_PREFIX.$OAUTH_TOKEN_FILE_CONF_SUFFIX", + resolvedMountedOAuthTokenFile) + // Redact all OAuth token values + sparkConfWithCredentialLocations + .getAll + .filter(_._1.endsWith(OAUTH_TOKEN_CONF_SUFFIX)).map(_._1) + .foreach { + sparkConfWithCredentialLocations.set(_, "") + } + sparkConfWithCredentialLocations + } + + private def safeFileConfToBase64( + conf: String, + fileNotFoundFormatString: String): Option[String] = { + submissionSparkConf.getOption(conf) + .map(new File(_)) + .map { file => + require(file.isFile, String.format(fileNotFoundFormatString, file.getAbsolutePath)) + BaseEncoding.base64().encode(Files.toByteArray(file)) + } + } + + private def resolveSecretLocation( + mountedUserSpecified: Option[String], + valueMountedFromSubmitter: Option[String], + mountedCanonicalLocation: String): Option[String] = { + mountedUserSpecified.orElse(valueMountedFromSubmitter.map( _ => { + mountedCanonicalLocation + })) + } + + private def resolveSecretData( + mountedUserSpecified: Option[String], + valueMountedFromSubmitter: Option[String], + secretName: String): Map[String, String] = { + mountedUserSpecified.map { _ => Map.empty[String, String]} + .getOrElse { + valueMountedFromSubmitter.map { valueBase64 => + Map(secretName -> valueBase64) + }.getOrElse(Map.empty[String, String]) + } + } + + private implicit def augmentSparkConf(sparkConf: SparkConf): OptionSettableSparkConf = { + new OptionSettableSparkConf(sparkConf) + } +} + +private class OptionSettableSparkConf(sparkConf: SparkConf) { + def setOption(configEntry: String, option: Option[String]): SparkConf = { + option.map( opt => { + sparkConf.set(configEntry, opt) + }).getOrElse(sparkConf) + } +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/InitContainerBootstrapStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/InitContainerBootstrapStep.scala new file mode 100644 index 0000000000000..7ce7a8472f812 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/InitContainerBootstrapStep.scala @@ -0,0 +1,64 @@ +/* + * 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.submit.submitsteps + +import io.fabric8.kubernetes.api.model.{ContainerBuilder, HasMetadata} + +import org.apache.spark.SparkConf +import org.apache.spark.deploy.kubernetes.config._ +import org.apache.spark.deploy.kubernetes.submit.{InitContainerUtil, PropertiesConfigMapFromScalaMapBuilder} +import org.apache.spark.deploy.kubernetes.submit.submitsteps.initcontainer.{InitContainerSpec, InitContainerStep} + +private[spark] class InitContainerBootstrapStep( + submissionSparkConf: SparkConf, + initContainerSteps: Seq[InitContainerStep], + initContainerConfigMapName: String, + initContainerConfigMapKey: String) + extends KubernetesSubmissionStep { + + override def prepareSubmission(driverSpec: KubernetesDriverSpec): KubernetesDriverSpec = { + val baseInitContainerSpec = InitContainerSpec( + initContainerProperties = Map.empty[String, String], + additionalDriverSparkConf = Map.empty[String, String], + initContainer = new ContainerBuilder().build(), + driverContainer = driverSpec.driverContainer, + podToInitialize = driverSpec.driverPod, + initContainerDependentResources = Seq.empty[HasMetadata]) + var currentInitContainerSpec = baseInitContainerSpec + for (nextStep <- initContainerSteps) { + currentInitContainerSpec = nextStep.prepareInitContainer(currentInitContainerSpec) + } + val configMap = PropertiesConfigMapFromScalaMapBuilder.buildConfigMap( + initContainerConfigMapName, + initContainerConfigMapKey, + currentInitContainerSpec.initContainerProperties) + val resolvedDriverSparkConf = driverSpec.driverSparkConf.clone() + .set(EXECUTOR_INIT_CONTAINER_CONFIG_MAP, initContainerConfigMapName) + .set(EXECUTOR_INIT_CONTAINER_CONFIG_MAP_KEY, initContainerConfigMapKey) + .setAll(currentInitContainerSpec.additionalDriverSparkConf) + val resolvedDriverPod = InitContainerUtil.appendInitContainer( + currentInitContainerSpec.podToInitialize, currentInitContainerSpec.initContainer) + driverSpec.copy( + driverPod = resolvedDriverPod, + driverContainer = currentInitContainerSpec.driverContainer, + driverSparkConf = resolvedDriverSparkConf, + otherKubernetesResources = + driverSpec.otherKubernetesResources ++ + currentInitContainerSpec.initContainerDependentResources ++ + Seq(configMap)) + } +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/KubernetesDriverSpec.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/KubernetesDriverSpec.scala new file mode 100644 index 0000000000000..6c669853e16f0 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/KubernetesDriverSpec.scala @@ -0,0 +1,35 @@ +/* + * 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.submit.submitsteps + +import io.fabric8.kubernetes.api.model.{Container, HasMetadata, Pod} + +import org.apache.spark.SparkConf + +/** + * Represents the components and characteristics of a Spark driver. The driver can be considered + * as being comprised of the driver pod itself, any other Kubernetes resources that the driver + * pod depends on, and the SparkConf that should be supplied to the Spark application. The driver + * container should be operated on via the specific field of this case class as opposed to trying + * to edit the container directly on the pod. The driver container should be attached at the + * end of executing all submission steps. + */ +private[spark] case class KubernetesDriverSpec( + driverPod: Pod, + driverContainer: Container, + otherKubernetesResources: Seq[HasMetadata], + driverSparkConf: SparkConf) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/KubernetesSubmissionStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/KubernetesSubmissionStep.scala new file mode 100644 index 0000000000000..bd810880d4589 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/KubernetesSubmissionStep.scala @@ -0,0 +1,28 @@ +/* + * 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.submit.submitsteps + +/** + * Represents a step in preparing the Kubernetes driver. + */ +private[spark] trait KubernetesSubmissionStep { + + /** + * Apply some transformation to the previous state of the driver to add a new feature to it. + */ + def prepareSubmission(driverSpec: KubernetesDriverSpec): KubernetesDriverSpec +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/PythonStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/PythonStep.scala new file mode 100644 index 0000000000000..484f57087b36e --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/PythonStep.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.submit.submitsteps + +import io.fabric8.kubernetes.api.model.ContainerBuilder + +import org.apache.spark.deploy.kubernetes.constants._ +import org.apache.spark.deploy.kubernetes.submit.KubernetesFileUtils + +private[spark] class PythonStep( + primaryPyFile: String, + otherPyFiles: Seq[String], + filesDownloadPath: String) extends KubernetesSubmissionStep { + + override def prepareSubmission(driverSpec: KubernetesDriverSpec): KubernetesDriverSpec = { + val withPythonPrimaryFileContainer = new ContainerBuilder(driverSpec.driverContainer) + .addNewEnv() + .withName(ENV_PYSPARK_PRIMARY) + .withValue(KubernetesFileUtils.resolveFilePath(primaryPyFile, filesDownloadPath)) + .endEnv() + .addNewEnv() + .withName(ENV_PYSPARK_FILES) + .withValue( + KubernetesFileUtils.resolveFilePaths(otherPyFiles, filesDownloadPath).mkString(",")) + .endEnv() + driverSpec.copy(driverContainer = withPythonPrimaryFileContainer.build()) + } +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/SparkInitContainerConfigMapBuilder.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/BaseInitContainerStep.scala similarity index 62% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/SparkInitContainerConfigMapBuilder.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/BaseInitContainerStep.scala index 4062a3113eddf..7c4b2623ba5d1 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/SparkInitContainerConfigMapBuilder.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/BaseInitContainerStep.scala @@ -14,32 +14,23 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit - -import io.fabric8.kubernetes.api.model.ConfigMap +package org.apache.spark.deploy.kubernetes.submit.submitsteps.initcontainer +import org.apache.spark.deploy.kubernetes.{PodWithDetachedInitContainer, SparkPodInitContainerBootstrap} import org.apache.spark.deploy.kubernetes.config._ +import org.apache.spark.deploy.kubernetes.submit.KubernetesFileUtils -private[spark] trait SparkInitContainerConfigMapBuilder { - /** - * Construct a config map that an init-container should reference for fetching - * remote dependencies. The config map includes the remote jars and files to download, - * as well as details to fetch files from a resource staging server, if applicable. - */ - def build(): ConfigMap -} - -private[spark] class SparkInitContainerConfigMapBuilderImpl( +private[spark] class BaseInitContainerStep( sparkJars: Seq[String], sparkFiles: Seq[String], jarsDownloadPath: String, filesDownloadPath: String, configMapName: String, configMapKey: String, - submittedDependenciesPlugin: Option[SubmittedDependencyInitContainerConfigPlugin]) - extends SparkInitContainerConfigMapBuilder { + podAndInitContainerBootstrap: SparkPodInitContainerBootstrap) + extends InitContainerStep { - override def build(): ConfigMap = { + override def prepareInitContainer(initContainerSpec: InitContainerSpec): InitContainerSpec = { val remoteJarsToDownload = KubernetesFileUtils.getOnlyRemoteFiles(sparkJars) val remoteFilesToDownload = KubernetesFileUtils.getOnlyRemoteFiles(sparkFiles) val remoteJarsConf = if (remoteJarsToDownload.nonEmpty) { @@ -57,12 +48,16 @@ private[spark] class SparkInitContainerConfigMapBuilderImpl( INIT_CONTAINER_FILES_DOWNLOAD_LOCATION.key -> filesDownloadPath) ++ remoteJarsConf ++ remoteFilesConf - val submittedDependenciesConfig = submittedDependenciesPlugin.map { plugin => - plugin.configurationsToFetchSubmittedDependencies() - }.toSeq.flatten.toMap - PropertiesConfigMapFromScalaMapBuilder.buildConfigMap( - configMapName, - configMapKey, - baseInitContainerConfig ++ submittedDependenciesConfig) + val bootstrappedPodAndInitContainer = + podAndInitContainerBootstrap.bootstrapInitContainerAndVolumes( + PodWithDetachedInitContainer( + initContainerSpec.podToInitialize, + initContainerSpec.initContainer, + initContainerSpec.driverContainer)) + initContainerSpec.copy( + initContainer = bootstrappedPodAndInitContainer.initContainer, + driverContainer = bootstrappedPodAndInitContainer.mainContainer, + podToInitialize = bootstrappedPodAndInitContainer.pod, + initContainerProperties = baseInitContainerConfig) } } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/InitContainerSpec.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/InitContainerSpec.scala new file mode 100644 index 0000000000000..5b5ac3c1f17c2 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/InitContainerSpec.scala @@ -0,0 +1,41 @@ +/* + * 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.submit.submitsteps.initcontainer + +import io.fabric8.kubernetes.api.model.{Container, HasMetadata, Pod} + +/** + * Represents a given configuration of the init-container, informing the main + * InitContainerBootstrapStep of how the driver should be configured. This includes: + *

+ * - What properties should be set on the init-container, + * - What Spark properties should be set on the driver's SparkConf given this init-container, + * - The spec of the init container itself, + * - The spec of the main container so that it can be modified to share volumes with the + * init-container + * - The spec of the pod EXCEPT for the addition of the given init-container (e.g. volumes + * the init-container needs or modifications to a main container that shares data with the + * init-container), + * - Any Kubernetes resources that need to be created for the init-container's function. + */ +private[spark] case class InitContainerSpec( + initContainerProperties: Map[String, String], + additionalDriverSparkConf: Map[String, String], + initContainer: Container, + driverContainer: Container, + podToInitialize: Pod, + initContainerDependentResources: Seq[HasMetadata]) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/InitContainerStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/InitContainerStep.scala new file mode 100644 index 0000000000000..c1ae640fb92df --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/InitContainerStep.scala @@ -0,0 +1,25 @@ +/* + * 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.submit.submitsteps.initcontainer + +/** + * Represents a step in preparing the init-container for the driver and executors. + */ +private[spark] trait InitContainerStep { + + def prepareInitContainer(initContainerSpec: InitContainerSpec): InitContainerSpec +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/InitContainerStepsOrchestrator.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/InitContainerStepsOrchestrator.scala new file mode 100644 index 0000000000000..1edff632f8f3c --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/InitContainerStepsOrchestrator.scala @@ -0,0 +1,131 @@ +/* + * 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.submit.submitsteps.initcontainer + +import org.apache.spark.SparkConf +import org.apache.spark.deploy.kubernetes.{InitContainerResourceStagingServerSecretPluginImpl, OptionRequirements, SparkPodInitContainerBootstrapImpl} +import org.apache.spark.deploy.kubernetes.config._ +import org.apache.spark.deploy.kubernetes.constants._ +import org.apache.spark.deploy.kubernetes.submit.SubmittedDependencyUploaderImpl +import org.apache.spark.deploy.rest.kubernetes.{ResourceStagingServerSslOptionsProviderImpl, RetrofitClientFactoryImpl} +import org.apache.spark.util.Utils + +/** + * Returns the complete ordered list of steps required to configure the init-container. + */ +private[spark] class InitContainerStepsOrchestrator( + namespace: String, + kubernetesResourceNamePrefix: String, + sparkJars: Seq[String], + sparkFiles: Seq[String], + jarsDownloadPath: String, + filesDownloadPath: String, + dockerImagePullPolicy: String, + driverLabels: Map[String, String], + initContainerConfigMapName: String, + initContainerConfigMapKey: String, + submissionSparkConf: SparkConf) { + + private val submittedResourcesSecretName = s"$kubernetesResourceNamePrefix-init-secret" + private val resourceStagingServerUri = submissionSparkConf.get(RESOURCE_STAGING_SERVER_URI) + private val resourceStagingServerInternalUri = + submissionSparkConf.get(RESOURCE_STAGING_SERVER_INTERNAL_URI) + private val initContainerImage = submissionSparkConf.get(INIT_CONTAINER_DOCKER_IMAGE) + private val downloadTimeoutMinutes = submissionSparkConf.get(INIT_CONTAINER_MOUNT_TIMEOUT) + private val maybeResourceStagingServerInternalTrustStore = + submissionSparkConf.get(RESOURCE_STAGING_SERVER_INTERNAL_TRUSTSTORE_FILE) + .orElse(submissionSparkConf.get(RESOURCE_STAGING_SERVER_TRUSTSTORE_FILE)) + private val maybeResourceStagingServerInternalTrustStorePassword = + submissionSparkConf.get(RESOURCE_STAGING_SERVER_INTERNAL_TRUSTSTORE_PASSWORD) + .orElse(submissionSparkConf.get(RESOURCE_STAGING_SERVER_TRUSTSTORE_PASSWORD)) + private val maybeResourceStagingServerInternalTrustStoreType = + submissionSparkConf.get(RESOURCE_STAGING_SERVER_INTERNAL_TRUSTSTORE_TYPE) + .orElse(submissionSparkConf.get(RESOURCE_STAGING_SERVER_TRUSTSTORE_TYPE)) + private val maybeResourceStagingServerInternalClientCert = + submissionSparkConf.get(RESOURCE_STAGING_SERVER_INTERNAL_CLIENT_CERT_PEM) + .orElse(submissionSparkConf.get(RESOURCE_STAGING_SERVER_CLIENT_CERT_PEM)) + private val resourceStagingServerInternalSslEnabled = + submissionSparkConf.get(RESOURCE_STAGING_SERVER_INTERNAL_SSL_ENABLED) + .orElse(submissionSparkConf.get(RESOURCE_STAGING_SERVER_SSL_ENABLED)) + .getOrElse(false) + OptionRequirements.requireNandDefined( + maybeResourceStagingServerInternalClientCert, + maybeResourceStagingServerInternalTrustStore, + "Cannot provide both a certificate file and a trustStore file for init-containers to" + + " use for contacting the resource staging server over TLS.") + + require(maybeResourceStagingServerInternalTrustStore.forall { trustStore => + Option(Utils.resolveURI(trustStore).getScheme).getOrElse("file") match { + case "file" | "local" => true + case _ => false + } + }, "TrustStore URI used for contacting the resource staging server from init containers must" + + " have no scheme, or scheme file://, or scheme local://.") + + require(maybeResourceStagingServerInternalClientCert.forall { trustStore => + Option(Utils.resolveURI(trustStore).getScheme).getOrElse("file") match { + case "file" | "local" => true + case _ => false + } + }, "Client cert file URI used for contacting the resource staging server from init containers" + + " must have no scheme, or scheme file://, or scheme local://.") + + def getInitContainerSteps(): Seq[InitContainerStep] = { + val initContainerBootstrap = new SparkPodInitContainerBootstrapImpl( + initContainerImage, + dockerImagePullPolicy, + jarsDownloadPath, + filesDownloadPath, + downloadTimeoutMinutes, + initContainerConfigMapName, + initContainerConfigMapKey) + val baseInitContainerStep = new BaseInitContainerStep( + sparkJars, + sparkFiles, + jarsDownloadPath, + filesDownloadPath, + initContainerConfigMapName, + initContainerConfigMapKey, + initContainerBootstrap) + val submittedResourcesInitContainerStep = resourceStagingServerUri.map { + stagingServerUri => + val mountSecretPlugin = new InitContainerResourceStagingServerSecretPluginImpl( + submittedResourcesSecretName, + INIT_CONTAINER_SECRET_VOLUME_MOUNT_PATH) + val submittedDependencyUploader = new SubmittedDependencyUploaderImpl( + driverLabels, + namespace, + stagingServerUri, + sparkJars, + sparkFiles, + new ResourceStagingServerSslOptionsProviderImpl(submissionSparkConf).getSslOptions, + RetrofitClientFactoryImpl) + new SubmittedResourcesInitContainerStep( + submittedResourcesSecretName, + resourceStagingServerInternalUri.getOrElse(stagingServerUri), + INIT_CONTAINER_SECRET_VOLUME_MOUNT_PATH, + resourceStagingServerInternalSslEnabled, + maybeResourceStagingServerInternalTrustStore, + maybeResourceStagingServerInternalClientCert, + maybeResourceStagingServerInternalTrustStorePassword, + maybeResourceStagingServerInternalTrustStoreType, + submittedDependencyUploader, + mountSecretPlugin) + } + Seq(baseInitContainerStep) ++ submittedResourcesInitContainerStep.toSeq + } +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/SubmittedResourcesInitContainerStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/SubmittedResourcesInitContainerStep.scala new file mode 100644 index 0000000000000..24bd8941c42b0 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/SubmittedResourcesInitContainerStep.scala @@ -0,0 +1,146 @@ +/* + * 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.submit.submitsteps.initcontainer + +import java.io.File + +import com.google.common.base.Charsets +import com.google.common.io.{BaseEncoding, Files} +import io.fabric8.kubernetes.api.model.{Secret, SecretBuilder} +import scala.collection.JavaConverters._ + +import org.apache.spark.SparkException +import org.apache.spark.deploy.kubernetes.InitContainerResourceStagingServerSecretPlugin +import org.apache.spark.deploy.kubernetes.config._ +import org.apache.spark.deploy.kubernetes.constants._ +import org.apache.spark.deploy.kubernetes.submit.SubmittedDependencyUploader +import org.apache.spark.internal.config.OptionalConfigEntry +import org.apache.spark.util.Utils + +private[spark] class SubmittedResourcesInitContainerStep( + submittedResourcesSecretName: String, + internalResourceStagingServerUri: String, + initContainerSecretMountPath: String, + resourceStagingServerSslEnabled: Boolean, + maybeInternalTrustStoreUri: Option[String], + maybeInternalClientCertUri: Option[String], + maybeInternalTrustStorePassword: Option[String], + maybeInternalTrustStoreType: Option[String], + submittedDependencyUploader: SubmittedDependencyUploader, + submittedResourcesSecretPlugin: InitContainerResourceStagingServerSecretPlugin) + extends InitContainerStep { + + override def prepareInitContainer(initContainerSpec: InitContainerSpec): InitContainerSpec = { + val jarsIdAndSecret = submittedDependencyUploader.uploadJars() + val filesIdAndSecret = submittedDependencyUploader.uploadFiles() + + val submittedResourcesInitContainerProperties = Map[String, String]( + RESOURCE_STAGING_SERVER_URI.key -> internalResourceStagingServerUri, + INIT_CONTAINER_DOWNLOAD_JARS_RESOURCE_IDENTIFIER.key -> jarsIdAndSecret.resourceId, + INIT_CONTAINER_DOWNLOAD_JARS_SECRET_LOCATION.key -> + s"$initContainerSecretMountPath/$INIT_CONTAINER_SUBMITTED_JARS_SECRET_KEY", + INIT_CONTAINER_DOWNLOAD_FILES_RESOURCE_IDENTIFIER.key -> filesIdAndSecret.resourceId, + INIT_CONTAINER_DOWNLOAD_FILES_SECRET_LOCATION.key -> + s"$initContainerSecretMountPath/$INIT_CONTAINER_SUBMITTED_FILES_SECRET_KEY", + RESOURCE_STAGING_SERVER_SSL_ENABLED.key -> resourceStagingServerSslEnabled.toString) ++ + resolveSecretPath( + maybeInternalTrustStoreUri, + INIT_CONTAINER_STAGING_SERVER_TRUSTSTORE_SECRET_KEY, + RESOURCE_STAGING_SERVER_TRUSTSTORE_FILE, + "TrustStore URI") ++ + resolveSecretPath( + maybeInternalClientCertUri, + INIT_CONTAINER_STAGING_SERVER_CLIENT_CERT_SECRET_KEY, + RESOURCE_STAGING_SERVER_CLIENT_CERT_PEM, + "Client certificate URI") ++ + maybeInternalTrustStorePassword.map { password => + (RESOURCE_STAGING_SERVER_TRUSTSTORE_PASSWORD.key, password) + }.toMap ++ + maybeInternalTrustStoreType.map { storeType => + (RESOURCE_STAGING_SERVER_TRUSTSTORE_TYPE.key, storeType) + }.toMap + val initContainerSecret = createResourceStagingServerSecret( + jarsIdAndSecret.resourceSecret, filesIdAndSecret.resourceSecret) + val additionalDriverSparkConf = + Map( + EXECUTOR_INIT_CONTAINER_SECRET.key -> initContainerSecret.getMetadata.getName, + EXECUTOR_INIT_CONTAINER_SECRET_MOUNT_DIR.key -> initContainerSecretMountPath) + val initContainerWithSecretVolumeMount = submittedResourcesSecretPlugin + .mountResourceStagingServerSecretIntoInitContainer(initContainerSpec.initContainer) + val podWithSecretVolume = submittedResourcesSecretPlugin + .addResourceStagingServerSecretVolumeToPod(initContainerSpec.podToInitialize) + initContainerSpec.copy( + initContainer = initContainerWithSecretVolumeMount, + podToInitialize = podWithSecretVolume, + initContainerDependentResources = + initContainerSpec.initContainerDependentResources ++ Seq(initContainerSecret), + initContainerProperties = + initContainerSpec.initContainerProperties ++ submittedResourcesInitContainerProperties, + additionalDriverSparkConf = additionalDriverSparkConf) + } + + private def createResourceStagingServerSecret( + jarsResourceSecret: String, filesResourceSecret: String): Secret = { + val trustStoreBase64 = convertFileToBase64IfSubmitterLocal( + INIT_CONTAINER_STAGING_SERVER_TRUSTSTORE_SECRET_KEY, maybeInternalTrustStoreUri) + val clientCertBase64 = convertFileToBase64IfSubmitterLocal( + INIT_CONTAINER_STAGING_SERVER_CLIENT_CERT_SECRET_KEY, maybeInternalClientCertUri) + val jarsSecretBase64 = BaseEncoding.base64().encode(jarsResourceSecret.getBytes(Charsets.UTF_8)) + val filesSecretBase64 = BaseEncoding.base64().encode( + filesResourceSecret.getBytes(Charsets.UTF_8)) + val secretData = Map( + INIT_CONTAINER_SUBMITTED_JARS_SECRET_KEY -> jarsSecretBase64, + INIT_CONTAINER_SUBMITTED_FILES_SECRET_KEY -> filesSecretBase64) ++ + trustStoreBase64 ++ + clientCertBase64 + val kubernetesSecret = new SecretBuilder() + .withNewMetadata() + .withName(submittedResourcesSecretName) + .endMetadata() + .addToData(secretData.asJava) + .build() + kubernetesSecret + } + + private def convertFileToBase64IfSubmitterLocal(secretKey: String, secretUri: Option[String]) + : Map[String, String] = { + secretUri.filter { trustStore => + Option(Utils.resolveURI(trustStore).getScheme).getOrElse("file") == "file" + }.map { uri => + val file = new File(Utils.resolveURI(uri).getPath) + require(file.isFile, "Dependency server trustStore provided at" + + file.getAbsolutePath + " does not exist or is not a file.") + (secretKey, BaseEncoding.base64().encode(Files.toByteArray(file))) + }.toMap + } + + private def resolveSecretPath( + maybeUri: Option[String], + secretKey: String, + configEntry: OptionalConfigEntry[String], + uriType: String): Map[String, String] = { + maybeUri.map(Utils.resolveURI).map { uri => + val resolvedPath = Option(uri.getScheme).getOrElse("file") match { + case "file" => s"$initContainerSecretMountPath/$secretKey" + case "local" => uri.getPath + case invalid => throw new SparkException(s"$uriType has invalid scheme $invalid must be" + + s" local://, file://, or empty.") + } + (configEntry.key, resolvedPath) + }.toMap + } +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/KubernetesSparkDependencyDownloadInitContainer.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/KubernetesSparkDependencyDownloadInitContainer.scala index ac19c2463218b..0e274678ad6f0 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/KubernetesSparkDependencyDownloadInitContainer.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/KubernetesSparkDependencyDownloadInitContainer.scala @@ -76,7 +76,6 @@ private[spark] class KubernetesSparkDependencyDownloadInitContainer( fileFetcher: FileFetcher, resourceStagingServerSslOptions: SSLOptions) extends Logging { - private implicit val downloadExecutor = ExecutionContext.fromExecutorService( ThreadUtils.newDaemonCachedThreadPool("download-executor")) private val maybeResourceStagingServerUri = sparkConf.get(RESOURCE_STAGING_SERVER_URI) diff --git a/resource-managers/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 index 2a0f6e78c2aea..fa0ecca3b4ee6 100644 --- a/resource-managers/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 @@ -70,8 +70,7 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit sparkConf.get(INIT_CONTAINER_FILES_DOWNLOAD_LOCATION), sparkConf.get(INIT_CONTAINER_MOUNT_TIMEOUT), configMap, - configMapKey, - executorInitContainerSecretVolumePlugin) + configMapKey) } if (maybeConfigMap.isEmpty) { logWarning("The executor's init-container config map was not specified. Executors will" + @@ -89,7 +88,11 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit Some(new File(Config.KUBERNETES_SERVICE_ACCOUNT_TOKEN_PATH)), Some(new File(Config.KUBERNETES_SERVICE_ACCOUNT_CA_CRT_PATH))) new KubernetesClusterSchedulerBackend( - sc.taskScheduler.asInstanceOf[TaskSchedulerImpl], sc, bootStrap, kubernetesClient) + sc.taskScheduler.asInstanceOf[TaskSchedulerImpl], + sc, + bootStrap, + executorInitContainerSecretVolumePlugin, + kubernetesClient) } override def initialize(scheduler: TaskScheduler, backend: SchedulerBackend): Unit = { 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 d880cee315c0d..a8af8ee61a2c4 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 @@ -23,7 +23,7 @@ import java.util.concurrent.atomic.{AtomicInteger, AtomicLong, AtomicReference} import com.fasterxml.jackson.databind.ObjectMapper import com.fasterxml.jackson.module.scala.DefaultScalaModule -import io.fabric8.kubernetes.api.model.{ContainerPortBuilder, EnvVarBuilder, EnvVarSourceBuilder, Pod, PodBuilder, QuantityBuilder} +import io.fabric8.kubernetes.api.model.{ContainerBuilder, ContainerPortBuilder, EnvVarBuilder, EnvVarSourceBuilder, Pod, PodBuilder, QuantityBuilder} import io.fabric8.kubernetes.client.{KubernetesClient, KubernetesClientException, Watcher} import io.fabric8.kubernetes.client.Watcher.Action import org.apache.commons.io.FilenameUtils @@ -32,9 +32,10 @@ import scala.collection.mutable import scala.concurrent.{ExecutionContext, Future} import org.apache.spark.{SparkContext, SparkEnv, SparkException} -import org.apache.spark.deploy.kubernetes.{ConfigurationUtils, SparkPodInitContainerBootstrap} +import org.apache.spark.deploy.kubernetes.{ConfigurationUtils, InitContainerResourceStagingServerSecretPlugin, PodWithDetachedInitContainer, SparkPodInitContainerBootstrap} import org.apache.spark.deploy.kubernetes.config._ import org.apache.spark.deploy.kubernetes.constants._ +import org.apache.spark.deploy.kubernetes.submit.InitContainerUtil import org.apache.spark.network.netty.SparkTransportConf import org.apache.spark.network.shuffle.kubernetes.KubernetesExternalShuffleClient import org.apache.spark.rpc.{RpcCallContext, RpcEndpointAddress, RpcEnv} @@ -47,6 +48,7 @@ private[spark] class KubernetesClusterSchedulerBackend( scheduler: TaskSchedulerImpl, val sc: SparkContext, executorInitContainerBootstrap: Option[SparkPodInitContainerBootstrap], + executorMountInitContainerSecretPlugin: Option[InitContainerResourceStagingServerSecretPlugin], kubernetesClient: KubernetesClient) extends CoarseGrainedSchedulerBackend(scheduler, sc.env.rpcEnv) { @@ -319,8 +321,8 @@ private[spark] class KubernetesClusterSchedulerBackend( nodeToLocalTaskCount.toMap[String, Int] } - private def addNodeAffinityAnnotationIfUseful(basePodBuilder: PodBuilder, - nodeToTaskCount: Map[String, Int]): PodBuilder = { + private def addNodeAffinityAnnotationIfUseful( + baseExecutorPod: Pod, nodeToTaskCount: Map[String, Int]): Pod = { def scaleToRange(value: Int, baseMin: Double, baseMax: Double, rangeMin: Double, rangeMax: Double): Int = (((rangeMax - rangeMin) * (value - baseMin) / (baseMax - baseMin)) + rangeMin).toInt @@ -341,11 +343,12 @@ private[spark] class KubernetesClusterSchedulerBackend( ))) // TODO: Use non-annotation syntax when we switch to K8s version 1.6. logDebug(s"Adding nodeAffinity as annotation $nodeAffinityJson") - basePodBuilder.editMetadata() + new PodBuilder(baseExecutorPod).editMetadata() .addToAnnotations(ANNOTATION_EXECUTOR_NODE_AFFINITY, nodeAffinityJson) .endMetadata() + .build() } else { - basePodBuilder + baseExecutorPod } } @@ -416,7 +419,22 @@ private[spark] class KubernetesClusterSchedulerBackend( .build() }) - val basePodBuilder = new PodBuilder() + val executorContainer = new ContainerBuilder() + .withName(s"executor") + .withImage(executorDockerImage) + .withImagePullPolicy(dockerImagePullPolicy) + .withNewResources() + .addToRequests("memory", executorMemoryQuantity) + .addToLimits("memory", executorMemoryLimitQuantity) + .addToRequests("cpu", executorCpuQuantity) + .endResources() + .addAllToEnv(requiredEnv.asJava) + .addToEnv(executorExtraClasspathEnv.toSeq: _*) + .withPorts(requiredPorts.asJava) + .build() + + + val executorPod = new PodBuilder() .withNewMetadata() .withName(name) .withLabels(resolvedExecutorLabels.asJava) @@ -432,69 +450,77 @@ private[spark] class KubernetesClusterSchedulerBackend( .endMetadata() .withNewSpec() .withHostname(hostname) - .addNewContainer() - .withName(s"executor") - .withImage(executorDockerImage) - .withImagePullPolicy(dockerImagePullPolicy) - .withNewResources() - .addToRequests("memory", executorMemoryQuantity) - .addToLimits("memory", executorMemoryLimitQuantity) - .addToRequests("cpu", executorCpuQuantity) - .endResources() - .addAllToEnv(requiredEnv.asJava) - .addToEnv(executorExtraClasspathEnv.toSeq: _*) - .withPorts(requiredPorts.asJava) - .endContainer() .endSpec() + .build() - executorLimitCores.map { + val containerWithExecutorLimitCores = executorLimitCores.map { limitCores => val executorCpuLimitQuantity = new QuantityBuilder(false) .withAmount(limitCores) .build() - basePodBuilder + new ContainerBuilder(executorContainer) + .editResources() + .addToLimits("cpu", executorCpuLimitQuantity) + .endResources() + .build() + }.getOrElse(executorContainer) + + val withMaybeShuffleConfigExecutorContainer = shuffleServiceConfig.map { config => + config.shuffleDirs.foldLeft(containerWithExecutorLimitCores) { (container, dir) => + new ContainerBuilder(container) + .addNewVolumeMount() + .withName(FilenameUtils.getBaseName(dir)) + .withMountPath(dir) + .endVolumeMount() + .build() + } + }.getOrElse(containerWithExecutorLimitCores) + val withMaybeShuffleConfigPod = shuffleServiceConfig.map { config => + config.shuffleDirs.foldLeft(executorPod) { (builder, dir) => + new PodBuilder(builder) .editSpec() - .editFirstContainer() - .editResources - .addToLimits("cpu", executorCpuLimitQuantity) - .endResources() - .endContainer() - .endSpec() - } - - val withMaybeShuffleConfigPodBuilder = shuffleServiceConfig - .map { config => - config.shuffleDirs.foldLeft(basePodBuilder) { (builder, dir) => - builder - .editSpec() - .addNewVolume() - .withName(FilenameUtils.getBaseName(dir)) - .withNewHostPath() - .withPath(dir) + .addNewVolume() + .withName(FilenameUtils.getBaseName(dir)) + .withNewHostPath() + .withPath(dir) .endHostPath() .endVolume() - .editFirstContainer() - .addNewVolumeMount() - .withName(FilenameUtils.getBaseName(dir)) - .withMountPath(dir) - .endVolumeMount() - .endContainer() .endSpec() - } - }.getOrElse(basePodBuilder) - - val executorInitContainerPodBuilder = executorInitContainerBootstrap.map { - bootstrap => - bootstrap.bootstrapInitContainerAndVolumes( - "executor", - withMaybeShuffleConfigPodBuilder) - }.getOrElse(withMaybeShuffleConfigPodBuilder) - - val resolvedExecutorPodBuilder = addNodeAffinityAnnotationIfUseful( - executorInitContainerPodBuilder, nodeToLocalTaskCount) - + .build() + } + }.getOrElse(executorPod) + val (executorPodWithInitContainer, initBootstrappedExecutorContainer) = + executorInitContainerBootstrap.map { bootstrap => + val podWithDetachedInitContainer = bootstrap.bootstrapInitContainerAndVolumes( + PodWithDetachedInitContainer( + withMaybeShuffleConfigPod, + new ContainerBuilder().build(), + withMaybeShuffleConfigExecutorContainer)) + + val resolvedInitContainer = executorMountInitContainerSecretPlugin.map { plugin => + plugin.mountResourceStagingServerSecretIntoInitContainer( + podWithDetachedInitContainer.initContainer) + }.getOrElse(podWithDetachedInitContainer.initContainer) + + val podWithAttachedInitContainer = InitContainerUtil.appendInitContainer( + podWithDetachedInitContainer.pod, resolvedInitContainer) + + val resolvedPodWithMountedSecret = executorMountInitContainerSecretPlugin.map { plugin => + plugin.addResourceStagingServerSecretVolumeToPod(podWithAttachedInitContainer) + }.getOrElse(podWithAttachedInitContainer) + + (resolvedPodWithMountedSecret, podWithDetachedInitContainer.mainContainer) + }.getOrElse((withMaybeShuffleConfigPod, withMaybeShuffleConfigExecutorContainer)) + + val executorPodWithNodeAffinity = addNodeAffinityAnnotationIfUseful( + executorPodWithInitContainer, nodeToLocalTaskCount) + val resolvedExecutorPod = new PodBuilder(executorPodWithNodeAffinity) + .editSpec() + .addToContainers(initBootstrappedExecutorContainer) + .endSpec() + .build() try { - (executorId, kubernetesClient.pods.create(resolvedExecutorPodBuilder.build())) + (executorId, kubernetesClient.pods.create(resolvedExecutorPod)) } catch { case throwable: Throwable => logError("Failed to allocate executor pod.", throwable) @@ -606,10 +632,11 @@ private[spark] class KubernetesClusterSchedulerBackend( } } - case class ShuffleServiceConfig(shuffleNamespace: String, +} +case class ShuffleServiceConfig( + shuffleNamespace: String, shuffleLabels: Map[String, String], shuffleDirs: Seq[String]) -} private object KubernetesClusterSchedulerBackend { private val DEFAULT_STATIC_PORT = 10000 diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/SparkPodInitContainerBootstrapSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/SparkPodInitContainerBootstrapSuite.scala index 90d7b10df211c..ba79736d9c7ba 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/SparkPodInitContainerBootstrapSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/SparkPodInitContainerBootstrapSuite.scala @@ -16,162 +16,140 @@ */ package org.apache.spark.deploy.kubernetes -import com.fasterxml.jackson.databind.ObjectMapper -import io.fabric8.kubernetes.api.model.{Container, ContainerBuilder, Pod, PodBuilder} import org.scalatest.BeforeAndAfter -import scala.collection.JavaConverters._ import org.apache.spark.SparkFunSuite -import org.apache.spark.deploy.kubernetes.constants._ class SparkPodInitContainerBootstrapSuite extends SparkFunSuite with BeforeAndAfter { - private val OBJECT_MAPPER = new ObjectMapper() - private val INIT_CONTAINER_IMAGE = "spark-init:latest" - private val DOCKER_IMAGE_PULL_POLICY = "IfNotPresent" - private val JARS_DOWNLOAD_PATH = "/var/data/spark-jars" - private val FILES_DOWNLOAD_PATH = "/var/data/spark-files" - private val DOWNLOAD_TIMEOUT_MINUTES = 5 - private val INIT_CONTAINER_CONFIG_MAP_NAME = "spark-init-config-map" - private val INIT_CONTAINER_CONFIG_MAP_KEY = "spark-init-config-map-key" - private val ADDED_SUBMITTED_DEPENDENCY_ENV = "ADDED_SUBMITTED_DEPENDENCY" - private val ADDED_SUBMITTED_DEPENDENCY_ANNOTATION = "added-submitted-dependencies" - private val MAIN_CONTAINER_NAME = "spark-main" - private val TRUE = "true" - - private val submittedDependencyPlugin = new InitContainerResourceStagingServerSecretPlugin { - override def addResourceStagingServerSecretVolumeToPod(basePod: PodBuilder) - : PodBuilder = { - basePod.editMetadata() - .addToAnnotations(ADDED_SUBMITTED_DEPENDENCY_ANNOTATION, TRUE) - .endMetadata() - } - - override def mountResourceStagingServerSecretIntoInitContainer(container: ContainerBuilder) - : ContainerBuilder = { - container - .addNewEnv() - .withName(ADDED_SUBMITTED_DEPENDENCY_ENV) - .withValue(TRUE) - .endEnv() - } - } - - test("Running without submitted dependencies adds init-container with volume mounts.") { - val bootstrappedPod = bootstrapPodWithoutSubmittedDependencies() - val podAnnotations = bootstrappedPod.getMetadata.getAnnotations.asScala - assert(podAnnotations.contains(INIT_CONTAINER_ANNOTATION)) - val initContainers = OBJECT_MAPPER.readValue( - podAnnotations(INIT_CONTAINER_ANNOTATION), classOf[Array[Container]]) - assert(initContainers.length === 1) - val initContainer = initContainers.head - val initContainerVolumeMounts = initContainer.getVolumeMounts.asScala.map { - mount => (mount.getName, mount.getMountPath) - }.toMap - val expectedInitContainerVolumeMounts = Map( - INIT_CONTAINER_PROPERTIES_FILE_VOLUME -> INIT_CONTAINER_PROPERTIES_FILE_DIR, - INIT_CONTAINER_DOWNLOAD_JARS_VOLUME_NAME -> JARS_DOWNLOAD_PATH, - INIT_CONTAINER_DOWNLOAD_FILES_VOLUME_NAME -> FILES_DOWNLOAD_PATH) - assert(initContainerVolumeMounts === expectedInitContainerVolumeMounts) - assert(initContainer.getName === "spark-init") - assert(initContainer.getImage === INIT_CONTAINER_IMAGE) - assert(initContainer.getImagePullPolicy === "IfNotPresent") - assert(initContainer.getArgs.asScala === List(INIT_CONTAINER_PROPERTIES_FILE_PATH)) - } - - test("Running without submitted dependencies adds volume mounts to main container.") { - val bootstrappedPod = bootstrapPodWithoutSubmittedDependencies() - val containers = bootstrappedPod.getSpec.getContainers.asScala - val mainContainer = containers.find(_.getName === MAIN_CONTAINER_NAME) - assert(mainContainer.isDefined) - val volumeMounts = mainContainer.map(_.getVolumeMounts.asScala).toSeq.flatten.map { - mount => (mount.getName, mount.getMountPath) - }.toMap - val expectedVolumeMounts = Map( - INIT_CONTAINER_DOWNLOAD_JARS_VOLUME_NAME -> JARS_DOWNLOAD_PATH, - INIT_CONTAINER_DOWNLOAD_FILES_VOLUME_NAME -> FILES_DOWNLOAD_PATH) - assert(volumeMounts === expectedVolumeMounts) - } - - test("Running without submitted dependencies adds volumes to the pod") { - val bootstrappedPod = bootstrapPodWithoutSubmittedDependencies() - val podVolumes = bootstrappedPod.getSpec.getVolumes.asScala - assert(podVolumes.size === 3) - assert(podVolumes.exists { volume => - volume.getName == INIT_CONTAINER_PROPERTIES_FILE_VOLUME && - Option(volume.getConfigMap).map { configMap => - configMap.getItems.asScala.map { - keyToPath => (keyToPath.getKey, keyToPath.getPath) - }.toMap - }.contains(Map(INIT_CONTAINER_CONFIG_MAP_KEY -> INIT_CONTAINER_PROPERTIES_FILE_NAME)) - }) - assert(podVolumes.exists { volume => - volume.getName == INIT_CONTAINER_DOWNLOAD_JARS_VOLUME_NAME && volume.getEmptyDir != null - }) - assert(podVolumes.exists { volume => - volume.getName == INIT_CONTAINER_DOWNLOAD_FILES_VOLUME_NAME && volume.getEmptyDir != null - }) - } - - test("Files download path is set as environment variable") { - val bootstrappedPod = bootstrapPodWithoutSubmittedDependencies() - val containers = bootstrappedPod.getSpec.getContainers.asScala - val maybeMainContainer = containers.find(_.getName === MAIN_CONTAINER_NAME) - assert(maybeMainContainer.exists { mainContainer => - mainContainer.getEnv.asScala.exists(envVar => - envVar.getName == ENV_MOUNTED_FILES_DIR && envVar.getValue == FILES_DOWNLOAD_PATH) - }) - } - - test("Running with submitted dependencies modifies the init container with the plugin.") { - val bootstrappedPod = bootstrapPodWithSubmittedDependencies() - val podAnnotations = bootstrappedPod.getMetadata.getAnnotations.asScala - assert(podAnnotations(ADDED_SUBMITTED_DEPENDENCY_ANNOTATION) === TRUE) - val initContainers = OBJECT_MAPPER.readValue( - podAnnotations(INIT_CONTAINER_ANNOTATION), classOf[Array[Container]]) - assert(initContainers.length === 1) - val initContainer = initContainers.head - assert(initContainer.getEnv.asScala.exists { - env => env.getName === ADDED_SUBMITTED_DEPENDENCY_ENV && env.getValue === TRUE - }) - } - - private def bootstrapPodWithoutSubmittedDependencies(): Pod = { - val bootstrapUnderTest = new SparkPodInitContainerBootstrapImpl( - INIT_CONTAINER_IMAGE, - DOCKER_IMAGE_PULL_POLICY, - JARS_DOWNLOAD_PATH, - FILES_DOWNLOAD_PATH, - DOWNLOAD_TIMEOUT_MINUTES, - INIT_CONTAINER_CONFIG_MAP_NAME, - INIT_CONTAINER_CONFIG_MAP_KEY, - None) - bootstrapUnderTest.bootstrapInitContainerAndVolumes( - MAIN_CONTAINER_NAME, basePod()).build() - } - - private def bootstrapPodWithSubmittedDependencies(): Pod = { - val bootstrapUnderTest = new SparkPodInitContainerBootstrapImpl( - INIT_CONTAINER_IMAGE, - DOCKER_IMAGE_PULL_POLICY, - JARS_DOWNLOAD_PATH, - FILES_DOWNLOAD_PATH, - DOWNLOAD_TIMEOUT_MINUTES, - INIT_CONTAINER_CONFIG_MAP_NAME, - INIT_CONTAINER_CONFIG_MAP_KEY, - Some(submittedDependencyPlugin)) - bootstrapUnderTest.bootstrapInitContainerAndVolumes( - MAIN_CONTAINER_NAME, basePod()).build() - } - - private def basePod(): PodBuilder = { - new PodBuilder() - .withNewMetadata() - .withName("spark-pod") - .endMetadata() - .withNewSpec() - .addNewContainer() - .withName(MAIN_CONTAINER_NAME) - .endContainer() - .endSpec() - } +// private val OBJECT_MAPPER = new ObjectMapper() +// private val INIT_CONTAINER_IMAGE = "spark-init:latest" +// private val DOCKER_IMAGE_PULL_POLICY = "IfNotPresent" +// private val JARS_DOWNLOAD_PATH = "/var/data/spark-jars" +// private val FILES_DOWNLOAD_PATH = "/var/data/spark-files" +// private val DOWNLOAD_TIMEOUT_MINUTES = 5 +// private val INIT_CONTAINER_CONFIG_MAP_NAME = "spark-init-config-map" +// private val INIT_CONTAINER_CONFIG_MAP_KEY = "spark-init-config-map-key" +// private val ADDED_SUBMITTED_DEPENDENCY_ENV = "ADDED_SUBMITTED_DEPENDENCY" +// private val ADDED_SUBMITTED_DEPENDENCY_ANNOTATION = "added-submitted-dependencies" +// private val MAIN_CONTAINER_NAME = "spark-main" +// private val TRUE = "true" +// +// test("Running without submitted dependencies adds init-container with volume mounts.") { +// val bootstrappedPod = bootstrapPodWithoutSubmittedDependencies() +// val podAnnotations = bootstrappedPod.getMetadata.getAnnotations.asScala +// assert(podAnnotations.contains(INIT_CONTAINER_ANNOTATION)) +// val initContainers = OBJECT_MAPPER.readValue( +// podAnnotations(INIT_CONTAINER_ANNOTATION), classOf[Array[Container]]) +// assert(initContainers.length === 1) +// val initContainer = initContainers.head +// val initContainerVolumeMounts = initContainer.getVolumeMounts.asScala.map { +// mount => (mount.getName, mount.getMountPath) +// }.toMap +// val expectedInitContainerVolumeMounts = Map( +// INIT_CONTAINER_PROPERTIES_FILE_VOLUME -> INIT_CONTAINER_PROPERTIES_FILE_DIR, +// INIT_CONTAINER_DOWNLOAD_JARS_VOLUME_NAME -> JARS_DOWNLOAD_PATH, +// INIT_CONTAINER_DOWNLOAD_FILES_VOLUME_NAME -> FILES_DOWNLOAD_PATH) +// assert(initContainerVolumeMounts === expectedInitContainerVolumeMounts) +// assert(initContainer.getName === "spark-init") +// assert(initContainer.getImage === INIT_CONTAINER_IMAGE) +// assert(initContainer.getImagePullPolicy === "IfNotPresent") +// assert(initContainer.getArgs.asScala === List(INIT_CONTAINER_PROPERTIES_FILE_PATH)) +// } +// +// test("Running without submitted dependencies adds volume mounts to main container.") { +// val bootstrappedPod = bootstrapPodWithoutSubmittedDependencies() +// val containers = bootstrappedPod.getSpec.getContainers.asScala +// val mainContainer = containers.find(_.getName === MAIN_CONTAINER_NAME) +// assert(mainContainer.isDefined) +// val volumeMounts = mainContainer.map(_.getVolumeMounts.asScala).toSeq.flatten.map { +// mount => (mount.getName, mount.getMountPath) +// }.toMap +// val expectedVolumeMounts = Map( +// INIT_CONTAINER_DOWNLOAD_JARS_VOLUME_NAME -> JARS_DOWNLOAD_PATH, +// INIT_CONTAINER_DOWNLOAD_FILES_VOLUME_NAME -> FILES_DOWNLOAD_PATH) +// assert(volumeMounts === expectedVolumeMounts) +// } +// +// test("Running without submitted dependencies adds volumes to the pod") { +// val bootstrappedPod = bootstrapPodWithoutSubmittedDependencies() +// val podVolumes = bootstrappedPod.getSpec.getVolumes.asScala +// assert(podVolumes.size === 3) +// assert(podVolumes.exists { volume => +// volume.getName == INIT_CONTAINER_PROPERTIES_FILE_VOLUME && +// Option(volume.getConfigMap).map { configMap => +// configMap.getItems.asScala.map { +// keyToPath => (keyToPath.getKey, keyToPath.getPath) +// }.toMap +// }.contains(Map(INIT_CONTAINER_CONFIG_MAP_KEY -> INIT_CONTAINER_PROPERTIES_FILE_NAME)) +// }) +// assert(podVolumes.exists { volume => +// volume.getName == INIT_CONTAINER_DOWNLOAD_JARS_VOLUME_NAME && volume.getEmptyDir != null +// }) +// assert(podVolumes.exists { volume => +// volume.getName == INIT_CONTAINER_DOWNLOAD_FILES_VOLUME_NAME && volume.getEmptyDir != null +// }) +// } +// +// test("Files download path is set as environment variable") { +// val bootstrappedPod = bootstrapPodWithoutSubmittedDependencies() +// val containers = bootstrappedPod.getSpec.getContainers.asScala +// val maybeMainContainer = containers.find(_.getName === MAIN_CONTAINER_NAME) +// assert(maybeMainContainer.exists { mainContainer => +// mainContainer.getEnv.asScala.exists(envVar => +// envVar.getName == ENV_MOUNTED_FILES_DIR && envVar.getValue == FILES_DOWNLOAD_PATH) +// }) +// } +// +// test("Running with submitted dependencies modifies the init container with the plugin.") { +// val bootstrappedPod = bootstrapPodWithSubmittedDependencies() +// val podAnnotations = bootstrappedPod.getMetadata.getAnnotations.asScala +// assert(podAnnotations(ADDED_SUBMITTED_DEPENDENCY_ANNOTATION) === TRUE) +// val initContainers = OBJECT_MAPPER.readValue( +// podAnnotations(INIT_CONTAINER_ANNOTATION), classOf[Array[Container]]) +// assert(initContainers.length === 1) +// val initContainer = initContainers.head +// assert(initContainer.getEnv.asScala.exists { +// env => env.getName === ADDED_SUBMITTED_DEPENDENCY_ENV && env.getValue === TRUE +// }) +// } +// +// private def bootstrapPodWithoutSubmittedDependencies(): Pod = { +// val bootstrapUnderTest = new SparkPodInitContainerBootstrapImpl( +// INIT_CONTAINER_IMAGE, +// DOCKER_IMAGE_PULL_POLICY, +// JARS_DOWNLOAD_PATH, +// FILES_DOWNLOAD_PATH, +// DOWNLOAD_TIMEOUT_MINUTES, +// INIT_CONTAINER_CONFIG_MAP_NAME, +// INIT_CONTAINER_CONFIG_MAP_KEY, +// None) +// bootstrapUnderTest.bootstrapInitContainerAndVolumes( +// basePod()).build() +// } +// +// private def bootstrapPodWithSubmittedDependencies(): Pod = { +// val bootstrapUnderTest = new SparkPodInitContainerBootstrapImpl( +// INIT_CONTAINER_IMAGE, +// DOCKER_IMAGE_PULL_POLICY, +// JARS_DOWNLOAD_PATH, +// FILES_DOWNLOAD_PATH, +// DOWNLOAD_TIMEOUT_MINUTES, +// INIT_CONTAINER_CONFIG_MAP_NAME, +// INIT_CONTAINER_CONFIG_MAP_KEY, +// Some(submittedDependencyPlugin)) +// bootstrapUnderTest.bootstrapInitContainerAndVolumes( +// MAIN_CONTAINER_NAME, basePod()).build() +// } +// +// private def basePod(): PodBuilder = { +// new PodBuilder() +// .withNewMetadata() +// .withName("spark-pod") +// .endMetadata() +// .withNewSpec() +// .addNewContainer() +// .withName(MAIN_CONTAINER_NAME) +// .endContainer() +// .endSpec() +// } } diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/SubmittedDependencyInitContainerVolumesPluginSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/SubmittedDependencyInitContainerVolumesPluginSuite.scala deleted file mode 100644 index 473d369c8eca3..0000000000000 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/SubmittedDependencyInitContainerVolumesPluginSuite.scala +++ /dev/null @@ -1,60 +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 io.fabric8.kubernetes.api.model.{ContainerBuilder, PodBuilder} -import scala.collection.JavaConverters._ - -import org.apache.spark.SparkFunSuite -import org.apache.spark.deploy.kubernetes.constants._ - -class SubmittedDependencyInitContainerVolumesPluginSuite extends SparkFunSuite { - - private val SECRET_NAME = "secret" - private val SECRET_MOUNT_PATH = "/mnt/secrets" - private val plugin = new InitContainerResourceStagingServerSecretPluginImpl( - SECRET_NAME, SECRET_MOUNT_PATH) - - test("The init container should have the secret volume mount.") { - val baseInitContainer = new ContainerBuilder().withName("container") - val configuredInitContainer = plugin.mountResourceStagingServerSecretIntoInitContainer( - baseInitContainer).build() - val volumeMounts = configuredInitContainer.getVolumeMounts.asScala - assert(volumeMounts.size === 1) - assert(volumeMounts.exists { volumeMount => - volumeMount.getName === INIT_CONTAINER_SECRET_VOLUME_NAME && - volumeMount.getMountPath === SECRET_MOUNT_PATH - }) - } - - test("The pod should have the secret volume.") { - val basePod = new PodBuilder() - .withNewMetadata().withName("pod").endMetadata() - .withNewSpec() - .addNewContainer() - .withName("container") - .endContainer() - .endSpec() - val configuredPod = plugin.addResourceStagingServerSecretVolumeToPod(basePod).build() - val volumes = configuredPod.getSpec.getVolumes.asScala - assert(volumes.size === 1) - assert(volumes.exists { volume => - volume.getName === INIT_CONTAINER_SECRET_VOLUME_NAME && - Option(volume.getSecret).map(_.getSecretName).contains(SECRET_NAME) - }) - } -} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/ClientV2Suite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/ClientV2Suite.scala index a58a37691f4eb..22924df49d4ee 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/ClientV2Suite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/ClientV2Suite.scala @@ -16,543 +16,12 @@ */ package org.apache.spark.deploy.kubernetes.submit -import java.io.File +import org.scalatest.BeforeAndAfter -import io.fabric8.kubernetes.api.model._ -import io.fabric8.kubernetes.client.{KubernetesClient, Watch} -import io.fabric8.kubernetes.client.dsl.{MixedOperation, NamespaceListVisitFromServerGetDeleteRecreateWaitApplicable, PodResource} -import org.hamcrest.{BaseMatcher, Description} -import org.mockito.{AdditionalAnswers, ArgumentCaptor, Mock, MockitoAnnotations} -import org.mockito.Matchers.{any, anyVararg, argThat, eq => mockitoEq} -import org.mockito.Mockito.{times, verify, when} -import org.mockito.invocation.InvocationOnMock -import org.mockito.stubbing.Answer -import org.scalatest.{BeforeAndAfter, Matchers} - -import scala.collection.JavaConverters._ -import scala.collection.mutable -import org.apache.spark.{SecurityManager, SparkConf, SparkFunSuite} -import org.apache.spark.deploy.kubernetes.{KubernetesExternalShuffleService, KubernetesShuffleBlockHandler, SparkPodInitContainerBootstrap} -import org.apache.spark.deploy.kubernetes.config._ -import org.apache.spark.deploy.kubernetes.constants._ -import org.apache.spark.network.netty.SparkTransportConf -import org.apache.spark.network.shuffle.kubernetes.KubernetesExternalShuffleClient +import org.apache.spark.SparkFunSuite class ClientV2Suite extends SparkFunSuite with BeforeAndAfter { - private val JARS_RESOURCE = SubmittedResourceIdAndSecret("jarsId", "jarsSecret") - private val FILES_RESOURCE = SubmittedResourceIdAndSecret("filesId", "filesSecret") - private val SUBMITTED_RESOURCES = SubmittedResources(JARS_RESOURCE, FILES_RESOURCE) - private val BOOTSTRAPPED_POD_ANNOTATION = "bootstrapped" - private val TRUE = "true" - private val APP_NAME = "spark-test" - private val APP_RESOURCE_PREFIX = "spark-prefix" - private val APP_ID = "spark-id" - private val CUSTOM_LABEL_KEY = "customLabel" - private val CUSTOM_LABEL_VALUE = "customLabelValue" - private val DEPRECATED_CUSTOM_LABEL_KEY = "deprecatedCustomLabel" - private val DEPRECATED_CUSTOM_LABEL_VALUE = "deprecatedCustomLabelValue" - private val ALL_EXPECTED_LABELS = Map( - CUSTOM_LABEL_KEY -> CUSTOM_LABEL_VALUE, - DEPRECATED_CUSTOM_LABEL_KEY -> DEPRECATED_CUSTOM_LABEL_VALUE, - SPARK_APP_ID_LABEL -> APP_ID, - SPARK_ROLE_LABEL -> SPARK_POD_DRIVER_ROLE) - private val CUSTOM_ANNOTATION_KEY = "customAnnotation" - private val CUSTOM_ANNOTATION_VALUE = "customAnnotationValue" - private val DEPRECATED_CUSTOM_ANNOTATION_KEY = "deprecatedCustomAnnotation" - private val DEPRECATED_CUSTOM_ANNOTATION_VALUE = "deprecatedCustomAnnotationValue" - private val INIT_CONTAINER_SECRET_NAME = "init-container-secret" - private val INIT_CONTAINER_SECRET_DATA = Map("secret-key" -> "secret-data") - private val MAIN_CLASS = "org.apache.spark.examples.SparkPi" - private val PYSPARK_APP_ARGS = Array(null, "500") - private val APP_ARGS = Array("3", "20") - private val SPARK_JARS = Seq( - "hdfs://localhost:9000/app/jars/jar1.jar", "file:///app/jars/jar2.jar") - private val RESOLVED_SPARK_JARS = Seq( - "hdfs://localhost:9000/app/jars/jar1.jar", "file:///var/data/spark-jars/jar2.jar") - private val RESOLVED_SPARK_REMOTE_AND_LOCAL_JARS = Seq( - "/var/data/spark-jars/jar1.jar", "/var/data/spark-jars/jar2.jar") - private val SPARK_FILES = Seq( - "hdfs://localhost:9000/app/files/file1.txt", "file:///app/files/file2.txt") - private val PYSPARK_FILES = Seq( - "hdfs://localhost:9000/app/files/file1.py", - "file:///app/files/file2.py", - "local:///app/files/file3.py", - "http://app/files/file4.py", - "file:///app/files/file5.py") - private val RESOLVED_PYSPARK_FILES = Seq( - "hdfs://localhost:9000/app/files/file1.py", - "/var/spark-data/spark-files/file2.py", - "local:///app/files/file3.py", - "http://app/files/file4.py") - private val PYSPARK_PRIMARY_FILE = "file:///app/files/file5.py" - private val RESOLVED_PYSPARK_PRIMARY_FILE = "/var/spark-data/spark-file/file5.py" - - private val RESOLVED_SPARK_FILES = Seq( - "hdfs://localhost:9000/app/files/file1.txt", "file:///var/data/spark-files/file2.txt") - private val INIT_CONTAINER_SECRET = new SecretBuilder() - .withNewMetadata() - .withName(INIT_CONTAINER_SECRET_NAME) - .endMetadata() - .withData(INIT_CONTAINER_SECRET_DATA.asJava) - .build() - private val CUSTOM_JAVA_OPTION_KEY = "myappoption" - private val CUSTOM_JAVA_OPTION_VALUE = "myappoptionvalue" - private val DRIVER_JAVA_OPTIONS = s"-D$CUSTOM_JAVA_OPTION_KEY=$CUSTOM_JAVA_OPTION_VALUE" - private val DRIVER_EXTRA_CLASSPATH = "/var/data/spark-app-custom/custom-jar.jar" - private val CONFIG_MAP_NAME = "config-map" - private val CONFIG_MAP_DATA = Map("config-map-key" -> "config-map-data") - private val INIT_CONTAINER_CONFIG_MAP = new ConfigMapBuilder() - .withNewMetadata() - .withName(CONFIG_MAP_NAME) - .endMetadata() - .withData(CONFIG_MAP_DATA.asJava) - .build() - private val CUSTOM_DRIVER_IMAGE = "spark-custom-driver:latest" - private val DRIVER_MEMORY_MB = 512 - private val DRIVER_MEMORY_OVERHEAD_MB = 128 - private val SPARK_CONF = new SparkConf(true) - .set(DRIVER_DOCKER_IMAGE, CUSTOM_DRIVER_IMAGE) - .set(org.apache.spark.internal.config.DRIVER_MEMORY, DRIVER_MEMORY_MB.toLong) - .set(KUBERNETES_DRIVER_MEMORY_OVERHEAD, DRIVER_MEMORY_OVERHEAD_MB.toLong) - .set(KUBERNETES_DRIVER_LABELS, s"$DEPRECATED_CUSTOM_LABEL_KEY=$DEPRECATED_CUSTOM_LABEL_VALUE") - .set(KUBERNETES_DRIVER_ANNOTATIONS, - s"$DEPRECATED_CUSTOM_ANNOTATION_KEY=$DEPRECATED_CUSTOM_ANNOTATION_VALUE") - .set(s"$KUBERNETES_DRIVER_LABEL_PREFIX$CUSTOM_LABEL_KEY", CUSTOM_LABEL_VALUE) - .set(s"$KUBERNETES_DRIVER_ANNOTATION_PREFIX$CUSTOM_ANNOTATION_KEY", CUSTOM_ANNOTATION_VALUE) - .set(org.apache.spark.internal.config.DRIVER_CLASS_PATH, DRIVER_EXTRA_CLASSPATH) - .set(org.apache.spark.internal.config.DRIVER_JAVA_OPTIONS, DRIVER_JAVA_OPTIONS) - private val EXECUTOR_INIT_CONF_KEY = "executor-init-conf" - private val SPARK_CONF_WITH_EXECUTOR_INIT_CONF = SPARK_CONF.clone() - .set(EXECUTOR_INIT_CONF_KEY, TRUE) - private val DRIVER_POD_UID = "driver-pod-uid" - private val DRIVER_POD_KIND = "pod" - private val DRIVER_POD_API_VERSION = "v1" - private val CREDENTIALS_SECRET_NAME = "credentials-secret" - private val CREDENTIALS_SECRET_DATA = Map("credentials-secret-key" -> "credentials-secret-value") - private val CREDENTIALS_SECRET = new SecretBuilder() - .withNewMetadata() - .withName(CREDENTIALS_SECRET_NAME) - .endMetadata() - .withData(CREDENTIALS_SECRET_DATA.asJava) - .build() - private val CREDENTIALS_SET_CONF = "spark.kubernetes.driverCredentials.provided" - private val CREDENTIALS_SET_ANNOTATION = "credentials-set" - - @Mock - private var containerLocalizedFilesResolver: ContainerLocalizedFilesResolver = _ - @Mock - private var executorInitContainerConfiguration: ExecutorInitContainerConfiguration = _ - @Mock - private var submittedDependencyUploader: SubmittedDependencyUploader = _ - @Mock - private var submittedDependenciesSecretBuilder: SubmittedDependencySecretBuilder = _ - @Mock - private var initContainerBootstrap: SparkPodInitContainerBootstrap = _ - @Mock - private var initContainerComponentsProvider: DriverInitContainerComponentsProvider = _ - @Mock - private var kubernetesClient: KubernetesClient = _ - @Mock - private var podOps: MixedOperation[ - Pod, PodList, DoneablePod, PodResource[Pod, DoneablePod]] = _ - private type ResourceListOps = NamespaceListVisitFromServerGetDeleteRecreateWaitApplicable[ - HasMetadata, java.lang.Boolean] - @Mock - private var resourceListOps: ResourceListOps = _ - @Mock - private var credentialsMounterProvider: DriverPodKubernetesCredentialsMounterProvider = _ - @Mock - private var fileMounter: DriverPodKubernetesFileMounter = _ - @Mock - private var credentialsMounter: DriverPodKubernetesCredentialsMounter = _ - @Mock - private var loggingPodStatusWatcher: LoggingPodStatusWatcher = _ - @Mock - private var namedPodResource: PodResource[Pod, DoneablePod] = _ - @Mock - private var watch: Watch = _ - - before { - MockitoAnnotations.initMocks(this) - when(initContainerComponentsProvider.provideInitContainerBootstrap()) - .thenReturn(initContainerBootstrap) - when(submittedDependencyUploader.uploadJars()).thenReturn(JARS_RESOURCE) - when(submittedDependencyUploader.uploadFiles()).thenReturn(FILES_RESOURCE) - when(initContainerBootstrap - .bootstrapInitContainerAndVolumes(mockitoEq(DRIVER_CONTAINER_NAME), any())) - .thenAnswer(new Answer[PodBuilder] { - override def answer(invocationOnMock: InvocationOnMock): PodBuilder = { - invocationOnMock.getArgumentAt(1, classOf[PodBuilder]).editMetadata() - .addToAnnotations(BOOTSTRAPPED_POD_ANNOTATION, TRUE) - .endMetadata() - } - }) - when(initContainerComponentsProvider.provideContainerLocalizedFilesResolver( - any[String])).thenReturn(containerLocalizedFilesResolver) - when(initContainerComponentsProvider.provideDriverPodFileMounter()) - .thenReturn(fileMounter) - when(submittedDependenciesSecretBuilder.build()) - .thenReturn(INIT_CONTAINER_SECRET) - when(kubernetesClient.pods()).thenReturn(podOps) - when(podOps.create(any())).thenAnswer(new Answer[Pod] { - override def answer(invocation: InvocationOnMock): Pod = { - new PodBuilder(invocation.getArgumentAt(0, classOf[Pod])) - .editMetadata() - .withUid(DRIVER_POD_UID) - .endMetadata() - .withKind(DRIVER_POD_KIND) - .withApiVersion(DRIVER_POD_API_VERSION) - .build() - } - }) - when(podOps.withName(s"$APP_RESOURCE_PREFIX-driver")).thenReturn(namedPodResource) - when(fileMounter.addPySparkFiles( - mockitoEq(RESOLVED_PYSPARK_PRIMARY_FILE), - mockitoEq(RESOLVED_PYSPARK_FILES.mkString(",")), - any[String], - any())).thenAnswer( new Answer[PodBuilder] { - override def answer(invocation: InvocationOnMock) : PodBuilder = { - invocation.getArgumentAt(3, classOf[PodBuilder]) - .editMetadata() - .withUid(DRIVER_POD_UID) - .withName(s"$APP_RESOURCE_PREFIX-driver") - .addToLabels("pyspark-test", "true") - .endMetadata() - .withKind(DRIVER_POD_KIND) - .withApiVersion(DRIVER_POD_API_VERSION) - } - }) - when(namedPodResource.watch(loggingPodStatusWatcher)).thenReturn(watch) - when(containerLocalizedFilesResolver.resolveSubmittedAndRemoteSparkJars()) - .thenReturn(RESOLVED_SPARK_REMOTE_AND_LOCAL_JARS) - when(containerLocalizedFilesResolver.resolveSubmittedSparkJars()) - .thenReturn(RESOLVED_SPARK_JARS) - when(containerLocalizedFilesResolver.resolveSubmittedSparkFiles()) - .thenReturn(RESOLVED_SPARK_FILES) - when(containerLocalizedFilesResolver.resolvePrimaryResourceFile()) - .thenReturn(RESOLVED_PYSPARK_PRIMARY_FILE) - when(containerLocalizedFilesResolver.resolveSubmittedPySparkFiles()) - .thenReturn(RESOLVED_PYSPARK_FILES) - when(executorInitContainerConfiguration.configureSparkConfForExecutorInitContainer(SPARK_CONF)) - .thenReturn(SPARK_CONF_WITH_EXECUTOR_INIT_CONF) - when(kubernetesClient.resourceList(anyVararg[HasMetadata]())).thenReturn(resourceListOps) - when(credentialsMounterProvider.getDriverPodKubernetesCredentialsMounter()) - .thenReturn(credentialsMounter) - } - - test("Run with dependency uploader") { - expectationsForNoMountedCredentials() - when(initContainerComponentsProvider - .provideInitContainerSubmittedDependencyUploader(ALL_EXPECTED_LABELS)) - .thenReturn(Some(submittedDependencyUploader)) - when(initContainerComponentsProvider - .provideSubmittedDependenciesSecretBuilder(Some(SUBMITTED_RESOURCES.secrets()))) - .thenReturn(Some(submittedDependenciesSecretBuilder)) - when(initContainerComponentsProvider.provideInitContainerBundle(mockitoEq( - Option(SUBMITTED_RESOURCES.ids())), - mockitoEq(RESOLVED_SPARK_JARS ++ RESOLVED_SPARK_FILES))) - .thenReturn(Option(InitContainerBundle(INIT_CONTAINER_CONFIG_MAP, - initContainerBootstrap, executorInitContainerConfiguration))) - runAndVerifyDriverPodHasCorrectProperties() - val resourceListArgumentCaptor = ArgumentCaptor.forClass(classOf[HasMetadata]) - verify(kubernetesClient).resourceList(resourceListArgumentCaptor.capture()) - val createdResources = resourceListArgumentCaptor.getAllValues.asScala - assert(createdResources.size === 2) - verifyCreatedResourcesHaveOwnerReferences(createdResources) - assert(createdResources.exists { - case secret: Secret => - secret.getMetadata.getName == INIT_CONTAINER_SECRET_NAME && - secret.getData.asScala == INIT_CONTAINER_SECRET_DATA - case _ => false - }) - verifyConfigMapWasCreated(createdResources) - verify(submittedDependencyUploader).uploadJars() - verify(submittedDependencyUploader).uploadFiles() - verify(initContainerComponentsProvider) - .provideSubmittedDependenciesSecretBuilder(Some(SUBMITTED_RESOURCES.secrets())) - } - - test("Run without dependency uploader") { - expectationsForNoMountedCredentials() - expectationsForNoDependencyUploader() - runAndVerifyDriverPodHasCorrectProperties() - val resourceListArgumentCaptor = ArgumentCaptor.forClass(classOf[HasMetadata]) - verify(kubernetesClient).resourceList(resourceListArgumentCaptor.capture()) - val createdResources = resourceListArgumentCaptor.getAllValues.asScala - assert(createdResources.size === 1) - verifyCreatedResourcesHaveOwnerReferences(createdResources) - verifyConfigMapWasCreated(createdResources) - verify(submittedDependencyUploader, times(0)).uploadJars() - verify(submittedDependencyUploader, times(0)).uploadFiles() - verify(initContainerComponentsProvider) - .provideSubmittedDependenciesSecretBuilder(None) - } - - test("Run with mounted credentials") { - expectationsForNoDependencyUploader() - when(credentialsMounter.createCredentialsSecret()).thenReturn(Some(CREDENTIALS_SECRET)) - when(credentialsMounter.mountDriverKubernetesCredentials( - any(), mockitoEq(DRIVER_CONTAINER_NAME), mockitoEq(Some(CREDENTIALS_SECRET)))) - .thenAnswer(new Answer[PodBuilder] { - override def answer(invocation: InvocationOnMock): PodBuilder = { - invocation.getArgumentAt(0, classOf[PodBuilder]).editMetadata() - .addToAnnotations(CREDENTIALS_SET_ANNOTATION, TRUE) - .endMetadata() - } - }) - when(credentialsMounter.setDriverPodKubernetesCredentialLocations(any())) - .thenAnswer(new Answer[SparkConf] { - override def answer(invocation: InvocationOnMock): SparkConf = { - invocation.getArgumentAt(0, classOf[SparkConf]).clone().set(CREDENTIALS_SET_CONF, TRUE) - } - }) - runAndVerifyPodMatchesPredicate { p => - Option(p) - .filter(pod => containerHasCorrectJvmOptions(pod, _(CREDENTIALS_SET_CONF) == TRUE)) - .exists { pod => - pod.getMetadata.getAnnotations.asScala(CREDENTIALS_SET_ANNOTATION) == TRUE - } - } - val resourceListArgumentCaptor = ArgumentCaptor.forClass(classOf[HasMetadata]) - verify(kubernetesClient).resourceList(resourceListArgumentCaptor.capture()) - val createdResources = resourceListArgumentCaptor.getAllValues.asScala - assert(createdResources.size === 2) - verifyCreatedResourcesHaveOwnerReferences(createdResources) - assert(createdResources.exists { - case secret: Secret => - secret.getMetadata.getName == CREDENTIALS_SECRET_NAME && - secret.getData.asScala == CREDENTIALS_SECRET_DATA - case _ => false - }) - } - - test("Waiting for completion should await completion on the status watcher.") { - expectationsForNoMountedCredentials() - expectationsForNoDependencyUploader() - new Client( - APP_NAME, - APP_RESOURCE_PREFIX, - APP_ID, - "", - None, - MAIN_CLASS, - SPARK_CONF, - APP_ARGS, - true, - kubernetesClient, - initContainerComponentsProvider, - credentialsMounterProvider, - loggingPodStatusWatcher).run() - verify(loggingPodStatusWatcher).awaitCompletion() - } - - test("Mounting environmental variables correctly onto Driver Pod for PySpark Jobs") { - expectationsForNoMountedCredentials() - expectationsForNoDependencyUploader() - expectationsForNoSparkJarsOrFiles() - runAndVerifyDriverPodHasCorrectPySparkProperties() - } - - private def expectationsForNoSparkJarsOrFiles(): Unit = { - when(containerLocalizedFilesResolver.resolveSubmittedSparkFiles()) - .thenReturn(Nil) - when(containerLocalizedFilesResolver.resolveSubmittedSparkJars()) - .thenReturn(Nil) - } - - private def expectationsForNoDependencyUploader(): Unit = { - when(initContainerComponentsProvider - .provideInitContainerSubmittedDependencyUploader(ALL_EXPECTED_LABELS)) - .thenReturn(None) - when(initContainerComponentsProvider - .provideSubmittedDependenciesSecretBuilder(None)) - .thenReturn(None) - when(initContainerComponentsProvider.provideInitContainerBundle(mockitoEq(None), - mockitoEq(RESOLVED_SPARK_JARS ++ RESOLVED_SPARK_FILES))) - .thenReturn(Some(InitContainerBundle(INIT_CONTAINER_CONFIG_MAP, - initContainerBootstrap, executorInitContainerConfiguration))) - } - - private def expectationsForNoMountedCredentials(): Unit = { - when(credentialsMounter.setDriverPodKubernetesCredentialLocations(any())) - .thenAnswer(AdditionalAnswers.returnsFirstArg()) - when(credentialsMounter.createCredentialsSecret()).thenReturn(None) - when(credentialsMounter.mountDriverKubernetesCredentials( - any(), mockitoEq(DRIVER_CONTAINER_NAME), mockitoEq(None))) - .thenAnswer(AdditionalAnswers.returnsFirstArg()) - } - - private def verifyCreatedResourcesHaveOwnerReferences( - createdResources: mutable.Buffer[HasMetadata]): Unit = { - assert(createdResources.forall { resource => - val owners = resource.getMetadata.getOwnerReferences.asScala - owners.size === 1 && - owners.head.getController && - owners.head.getKind == DRIVER_POD_KIND && - owners.head.getUid == DRIVER_POD_UID && - owners.head.getName == s"$APP_RESOURCE_PREFIX-driver" && - owners.head.getApiVersion == DRIVER_POD_API_VERSION - }) - } - - private def verifyConfigMapWasCreated(createdResources: mutable.Buffer[HasMetadata]): Unit = { - assert(createdResources.exists { - case configMap: ConfigMap => - configMap.getMetadata.getName == CONFIG_MAP_NAME && - configMap.getData.asScala == CONFIG_MAP_DATA - case _ => false - }) - } - - private def runAndVerifyDriverPodHasCorrectProperties(): Unit = { - val expectedOptions = SPARK_CONF.getAll - .filterNot(_._1 == org.apache.spark.internal.config.DRIVER_JAVA_OPTIONS.key) - .toMap ++ - Map( - "spark.app.id" -> APP_ID, - KUBERNETES_DRIVER_POD_NAME.key -> s"$APP_RESOURCE_PREFIX-driver", - KUBERNETES_EXECUTOR_POD_NAME_PREFIX.key -> APP_RESOURCE_PREFIX, - EXECUTOR_INIT_CONF_KEY -> TRUE, - CUSTOM_JAVA_OPTION_KEY -> CUSTOM_JAVA_OPTION_VALUE, - "spark.jars" -> RESOLVED_SPARK_JARS.mkString(","), - "spark.files" -> RESOLVED_SPARK_FILES.mkString(",")) - runAndVerifyPodMatchesPredicate { p => - Option(p) - .filter(_.getMetadata.getName == s"$APP_RESOURCE_PREFIX-driver") - .filter(podHasCorrectAnnotations) - .filter(_.getMetadata.getLabels.asScala == ALL_EXPECTED_LABELS) - .filter(containerHasCorrectBasicContainerConfiguration) - .filter(containerHasCorrectBasicEnvs) - .filter(containerHasCorrectMountedClasspath) - .exists(pod => containerHasCorrectJvmOptions(pod, _ == expectedOptions)) - } - } - - private def runAndVerifyDriverPodHasCorrectPySparkProperties(): Unit = { - when(initContainerComponentsProvider.provideContainerLocalizedFilesResolver( - mockitoEq(PYSPARK_PRIMARY_FILE))).thenReturn(containerLocalizedFilesResolver) - when(initContainerComponentsProvider.provideInitContainerBundle( - any[Option[SubmittedResourceIds]], any[Iterable[String]])) - .thenReturn(Some(InitContainerBundle(INIT_CONTAINER_CONFIG_MAP, - initContainerBootstrap, executorInitContainerConfiguration))) - runAndVerifyPySparkPodMatchesPredicate { p => - Option(p).exists(pod => containerHasCorrectPySparkEnvs(pod)) - } - } - - private def runAndVerifyPodMatchesPredicate(pred: (Pod => Boolean)): Unit = { - new Client( - APP_NAME, - APP_RESOURCE_PREFIX, - APP_ID, - "", - None, - MAIN_CLASS, - SPARK_CONF, - APP_ARGS, - false, - kubernetesClient, - initContainerComponentsProvider, - credentialsMounterProvider, - loggingPodStatusWatcher).run() - val podMatcher = new BaseMatcher[Pod] { - override def matches(o: scala.Any): Boolean = { - o match { - case p: Pod => pred(p) - case _ => false - } - } - override def describeTo(description: Description): Unit = {} - } - verify(podOps).create(argThat(podMatcher)) - } - - private def containerHasCorrectJvmOptions( - pod: Pod, optionsCorrectnessPredicate: (Map[String, String] => Boolean)): Boolean = { - val driverContainer = pod.getSpec.getContainers.asScala.head - val envs = driverContainer.getEnv.asScala.map(env => (env.getName, env.getValue)) - envs.toMap.get(ENV_DRIVER_JAVA_OPTS).exists { javaOptions => - val splitOptions = javaOptions.split(" ") - splitOptions.forall(_.startsWith("-D")) && - optionsCorrectnessPredicate(splitOptions.map { option => - val withoutPrefix = option.substring(2) - (withoutPrefix.split("=", 2)(0), withoutPrefix.split("=", 2)(1)) - }.toMap) - } - } - - private def containerHasCorrectMountedClasspath(pod: Pod): Boolean = { - val driverContainer = pod.getSpec.getContainers.asScala.head - val envs = driverContainer.getEnv.asScala.map(env => (env.getName, env.getValue)) - envs.toMap.get(ENV_MOUNTED_CLASSPATH).exists { classpath => - val mountedClasspathEntities = classpath.split(File.pathSeparator) - mountedClasspathEntities.toSet == RESOLVED_SPARK_REMOTE_AND_LOCAL_JARS.toSet - } - } - - private def containerHasCorrectBasicEnvs(pod: Pod): Boolean = { - val driverContainer = pod.getSpec.getContainers.asScala.head - val envs = driverContainer.getEnv.asScala.map(env => (env.getName, env.getValue)) - val expectedBasicEnvs = Map( - ENV_SUBMIT_EXTRA_CLASSPATH -> DRIVER_EXTRA_CLASSPATH, - ENV_DRIVER_MEMORY -> s"${DRIVER_MEMORY_MB + DRIVER_MEMORY_OVERHEAD_MB}m", - ENV_DRIVER_MAIN_CLASS -> MAIN_CLASS, - ENV_DRIVER_ARGS -> APP_ARGS.mkString(" ")) - expectedBasicEnvs.toSet.subsetOf(envs.toSet) - } - - private def containerHasCorrectPySparkEnvs(pod: Pod): Boolean = { - val driverPodLabels = - pod.getMetadata.getLabels.asScala.map(env => (env._1.toString, env._2.toString)) - val expectedBasicLabels = Map( - "pyspark-test" -> "true", - "spark-role" -> "driver") - expectedBasicLabels.toSet.subsetOf(driverPodLabels.toSet) - } - - private def containerHasCorrectBasicContainerConfiguration(pod: Pod): Boolean = { - val containers = pod.getSpec.getContainers.asScala - containers.size == 1 && - containers.head.getName == DRIVER_CONTAINER_NAME && - containers.head.getImage == CUSTOM_DRIVER_IMAGE && - containers.head.getImagePullPolicy == "IfNotPresent" - } - - private def podHasCorrectAnnotations(pod: Pod): Boolean = { - val expectedAnnotations = Map( - DEPRECATED_CUSTOM_ANNOTATION_KEY -> DEPRECATED_CUSTOM_ANNOTATION_VALUE, - CUSTOM_ANNOTATION_KEY -> CUSTOM_ANNOTATION_VALUE, - SPARK_APP_NAME_ANNOTATION -> APP_NAME, - BOOTSTRAPPED_POD_ANNOTATION -> TRUE) - pod.getMetadata.getAnnotations.asScala == expectedAnnotations - } - - private def runAndVerifyPySparkPodMatchesPredicate(pred: (Pod => Boolean)): Unit = { - new Client( - APP_NAME, - APP_RESOURCE_PREFIX, - APP_ID, - PYSPARK_PRIMARY_FILE, - Option(new PythonSubmissionResourcesImpl(PYSPARK_PRIMARY_FILE, PYSPARK_APP_ARGS)), - MAIN_CLASS, - SPARK_CONF, - PYSPARK_APP_ARGS, - false, - kubernetesClient, - initContainerComponentsProvider, - credentialsMounterProvider, - loggingPodStatusWatcher).run() - val podMatcher = new BaseMatcher[Pod] { - override def matches(o: scala.Any): Boolean = { - o match { - case p: Pod => pred(p) - case _ => false - } - } - override def describeTo(description: Description): Unit = {} - } - verify(podOps).create(argThat(podMatcher)) - } + // TODO } diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/ContainerLocalizedFilesResolverSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/ContainerLocalizedFilesResolverSuite.scala deleted file mode 100644 index 7e51abcd7b8e0..0000000000000 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/ContainerLocalizedFilesResolverSuite.scala +++ /dev/null @@ -1,93 +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.submit - -import org.apache.spark.SparkFunSuite - -class ContainerLocalizedFilesResolverSuite extends SparkFunSuite { - private val SPARK_JARS = Seq( - "hdfs://localhost:9000/app/jars/jar1.jar", - "file:///app/jars/jar2.jar", - "local:///app/jars/jar3.jar", - "http://app/jars/jar4.jar") - private val SPARK_FILES = Seq( - "hdfs://localhost:9000/app/files/file1.txt", - "file:///app/files/file2.txt", - "local:///app/files/file3.txt", - "http://app/files/file4.txt") - private val PYSPARK_FILES = Seq( - "hdfs://localhost:9000/app/files/file1.py", - "file:///app/files/file2.py", - "local:///app/files/file3.py", - "http://app/files/file4.py", - "file:///app/files/file5.py") - private val JARS_DOWNLOAD_PATH = "/var/data/spark-jars" - private val FILES_DOWNLOAD_PATH = "/var/data/spark-files" - private val PYSPARK_PRIMARY_FILE = "file:///app/files/file5.py" - private val localizedFilesResolver = new ContainerLocalizedFilesResolverImpl( - SPARK_JARS, - SPARK_FILES, - PYSPARK_FILES, - PYSPARK_PRIMARY_FILE, - JARS_DOWNLOAD_PATH, - FILES_DOWNLOAD_PATH) - - test("Submitted and remote Spark jars should resolve non-local uris to download path.") { - val resolvedJars = localizedFilesResolver.resolveSubmittedAndRemoteSparkJars() - val expectedResolvedJars = Seq( - s"$JARS_DOWNLOAD_PATH/jar1.jar", - s"$JARS_DOWNLOAD_PATH/jar2.jar", - "/app/jars/jar3.jar", - s"$JARS_DOWNLOAD_PATH/jar4.jar") - assert(resolvedJars === expectedResolvedJars) - } - - test("Submitted Spark jars should resolve to the download path.") { - val resolvedJars = localizedFilesResolver.resolveSubmittedSparkJars() - val expectedResolvedJars = Seq( - "hdfs://localhost:9000/app/jars/jar1.jar", - s"$JARS_DOWNLOAD_PATH/jar2.jar", - "local:///app/jars/jar3.jar", - "http://app/jars/jar4.jar") - assert(resolvedJars === expectedResolvedJars) - } - - test("Submitted Spark files should resolve to the download path.") { - val resolvedFiles = localizedFilesResolver.resolveSubmittedSparkFiles() - val expectedResolvedFiles = Seq( - "hdfs://localhost:9000/app/files/file1.txt", - s"$FILES_DOWNLOAD_PATH/file2.txt", - "local:///app/files/file3.txt", - "http://app/files/file4.txt") - assert(resolvedFiles === expectedResolvedFiles) - } - test("Submitted PySpark files should resolve to the download path.") { - val resolvedPySparkFiles = localizedFilesResolver.resolveSubmittedPySparkFiles() - val expectedPySparkFiles = Seq( - "hdfs://localhost:9000/app/files/file1.py", - s"$FILES_DOWNLOAD_PATH/file2.py", - "local:///app/files/file3.py", - "http://app/files/file4.py") - assert(resolvedPySparkFiles === expectedPySparkFiles) - } - test("Submitted PySpark Primary resource should resolve to the download path.") { - val resolvedPySparkPrimary = - localizedFilesResolver.resolvePrimaryResourceFile() - val expectedPySparkPrimary = s"$FILES_DOWNLOAD_PATH/file5.py" - assert(resolvedPySparkPrimary === expectedPySparkPrimary) - } -} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/DriverPodKubernetesCredentialsMounterSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/DriverPodKubernetesCredentialsMounterSuite.scala deleted file mode 100644 index 2e0a7ba5098b2..0000000000000 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/DriverPodKubernetesCredentialsMounterSuite.scala +++ /dev/null @@ -1,171 +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.submit - -import io.fabric8.kubernetes.api.model.{PodBuilder, SecretBuilder} -import org.scalatest.prop.TableDrivenPropertyChecks -import scala.collection.JavaConverters._ - -import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.deploy.kubernetes.KubernetesCredentials -import org.apache.spark.deploy.kubernetes.config._ -import org.apache.spark.deploy.kubernetes.constants._ - -class DriverPodKubernetesCredentialsMounterSuite - extends SparkFunSuite with TableDrivenPropertyChecks { - - private val CLIENT_KEY_DATA = "client-key-data" - private val CLIENT_CERT_DATA = "client-cert-data" - private val OAUTH_TOKEN_DATA = "oauth-token" - private val CA_CERT_DATA = "ca-cert-data" - private val SUBMITTER_LOCAL_DRIVER_KUBERNETES_CREDENTIALS = KubernetesCredentials( - caCertDataBase64 = Some(CA_CERT_DATA), - clientKeyDataBase64 = Some(CLIENT_KEY_DATA), - clientCertDataBase64 = Some(CLIENT_CERT_DATA), - oauthTokenBase64 = Some(OAUTH_TOKEN_DATA)) - private val APP_ID = "app-id" - private val USER_SPECIFIED_CLIENT_KEY_FILE = Some("/var/data/client-key.pem") - private val USER_SPECIFIED_CLIENT_CERT_FILE = Some("/var/data/client-cert.pem") - private val USER_SPECIFIED_OAUTH_TOKEN_FILE = Some("/var/data/token.txt") - private val USER_SPECIFIED_CA_CERT_FILE = Some("/var/data/ca.pem") - - // Different configurations of credentials mounters - private val credentialsMounterWithPreMountedFiles = - new DriverPodKubernetesCredentialsMounterImpl( - kubernetesAppId = APP_ID, - submitterLocalDriverPodKubernetesCredentials = SUBMITTER_LOCAL_DRIVER_KUBERNETES_CREDENTIALS, - maybeUserSpecifiedMountedClientKeyFile = USER_SPECIFIED_CLIENT_KEY_FILE, - maybeUserSpecifiedMountedClientCertFile = USER_SPECIFIED_CLIENT_CERT_FILE, - maybeUserSpecifiedMountedOAuthTokenFile = USER_SPECIFIED_OAUTH_TOKEN_FILE, - maybeUserSpecifiedMountedCaCertFile = USER_SPECIFIED_CA_CERT_FILE) - private val credentialsMounterWithoutPreMountedFiles = - new DriverPodKubernetesCredentialsMounterImpl( - kubernetesAppId = APP_ID, - submitterLocalDriverPodKubernetesCredentials = SUBMITTER_LOCAL_DRIVER_KUBERNETES_CREDENTIALS, - maybeUserSpecifiedMountedClientKeyFile = None, - maybeUserSpecifiedMountedClientCertFile = None, - maybeUserSpecifiedMountedOAuthTokenFile = None, - maybeUserSpecifiedMountedCaCertFile = None) - private val credentialsMounterWithoutAnyDriverCredentials = - new DriverPodKubernetesCredentialsMounterImpl( - APP_ID, KubernetesCredentials(None, None, None, None), None, None, None, None) - - // Test matrices - private val TEST_MATRIX_EXPECTED_SPARK_CONFS = Table( - ("Credentials Mounter Implementation", - "Expected client key file", - "Expected client cert file", - "Expected CA Cert file", - "Expected OAuth Token File"), - (credentialsMounterWithoutAnyDriverCredentials, - None, - None, - None, - None), - (credentialsMounterWithoutPreMountedFiles, - Some(DRIVER_CREDENTIALS_CLIENT_KEY_PATH), - Some(DRIVER_CREDENTIALS_CLIENT_CERT_PATH), - Some(DRIVER_CREDENTIALS_CA_CERT_PATH), - Some(DRIVER_CREDENTIALS_OAUTH_TOKEN_PATH)), - (credentialsMounterWithPreMountedFiles, - USER_SPECIFIED_CLIENT_KEY_FILE, - USER_SPECIFIED_CLIENT_CERT_FILE, - USER_SPECIFIED_CA_CERT_FILE, - USER_SPECIFIED_OAUTH_TOKEN_FILE)) - - private val TEST_MATRIX_EXPECTED_CREDENTIALS_SECRET = Table( - ("Credentials Mounter Implementation", "Expected Credentials Secret Data"), - (credentialsMounterWithoutAnyDriverCredentials, None), - (credentialsMounterWithoutPreMountedFiles, - Some(KubernetesSecretNameAndData( - data = Map[String, String]( - DRIVER_CREDENTIALS_CLIENT_KEY_SECRET_NAME -> CLIENT_KEY_DATA, - DRIVER_CREDENTIALS_CLIENT_CERT_SECRET_NAME -> CLIENT_CERT_DATA, - DRIVER_CREDENTIALS_CA_CERT_SECRET_NAME -> CA_CERT_DATA, - DRIVER_CREDENTIALS_OAUTH_TOKEN_SECRET_NAME -> OAUTH_TOKEN_DATA - ), - name = s"$APP_ID-kubernetes-credentials"))), - (credentialsMounterWithPreMountedFiles, None)) - - test("Credentials mounter should set the driver's Kubernetes credentials locations") { - forAll(TEST_MATRIX_EXPECTED_SPARK_CONFS) { - case (credentialsMounter, - expectedClientKeyFile, - expectedClientCertFile, - expectedCaCertFile, - expectedOAuthTokenFile) => - val baseSparkConf = new SparkConf() - val resolvedSparkConf = - credentialsMounter.setDriverPodKubernetesCredentialLocations(baseSparkConf) - assert(resolvedSparkConf.getOption( - s"$APISERVER_AUTH_DRIVER_MOUNTED_CONF_PREFIX.$CLIENT_KEY_FILE_CONF_SUFFIX") === - expectedClientKeyFile) - assert(resolvedSparkConf.getOption( - s"$APISERVER_AUTH_DRIVER_MOUNTED_CONF_PREFIX.$CLIENT_CERT_FILE_CONF_SUFFIX") === - expectedClientCertFile) - assert(resolvedSparkConf.getOption( - s"$APISERVER_AUTH_DRIVER_MOUNTED_CONF_PREFIX.$CA_CERT_FILE_CONF_SUFFIX") === - expectedCaCertFile) - assert(resolvedSparkConf.getOption( - s"$APISERVER_AUTH_DRIVER_MOUNTED_CONF_PREFIX.$OAUTH_TOKEN_FILE_CONF_SUFFIX") === - expectedOAuthTokenFile) - } - } - - test("Credentials mounter should create the correct credentials secret.") { - forAll(TEST_MATRIX_EXPECTED_CREDENTIALS_SECRET) { - case (credentialsMounter, expectedSecretNameAndData) => - val builtSecret = credentialsMounter.createCredentialsSecret() - val secretNameAndData = builtSecret.map { secret => - KubernetesSecretNameAndData(secret.getMetadata.getName, secret.getData.asScala.toMap) - } - assert(secretNameAndData === expectedSecretNameAndData) - } - } - - test("When credentials secret is provided, driver pod should mount the secret volume.") { - val credentialsSecret = new SecretBuilder() - .withNewMetadata().withName("secret").endMetadata() - .addToData("secretKey", "secretValue") - .build() - val originalPodSpec = new PodBuilder() - .withNewMetadata().withName("pod").endMetadata() - .withNewSpec() - .addNewContainer() - .withName("container") - .endContainer() - .endSpec() - val podSpecWithMountedDriverKubernetesCredentials = - credentialsMounterWithoutPreMountedFiles.mountDriverKubernetesCredentials( - originalPodSpec, "container", Some(credentialsSecret)).build() - val volumes = podSpecWithMountedDriverKubernetesCredentials.getSpec.getVolumes.asScala - assert(volumes.exists(_.getName == DRIVER_CREDENTIALS_SECRET_VOLUME_NAME)) - volumes.find(_.getName == DRIVER_CREDENTIALS_SECRET_VOLUME_NAME).foreach { secretVolume => - assert(secretVolume.getSecret != null && secretVolume.getSecret.getSecretName == "secret") - } - } - - test("When credentials secret is absent, driver pod should not be changed.") { - val originalPodSpec = new PodBuilder() - val nonAdjustedPodSpec = - credentialsMounterWithoutAnyDriverCredentials.mountDriverKubernetesCredentials( - originalPodSpec, "driver", None) - assert(nonAdjustedPodSpec === originalPodSpec) - } -} - -private case class KubernetesSecretNameAndData(name: String, data: Map[String, String]) diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/ExecutorInitContainerConfigurationSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/ExecutorInitContainerConfigurationSuite.scala deleted file mode 100644 index ead1d49b8a37c..0000000000000 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/ExecutorInitContainerConfigurationSuite.scala +++ /dev/null @@ -1,56 +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.submit - -import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.deploy.kubernetes.config._ - -class ExecutorInitContainerConfigurationSuite extends SparkFunSuite { - - private val SECRET_NAME = "init-container-secret" - private val SECRET_MOUNT_DIR = "/mnt/secrets/spark" - private val CONFIG_MAP_NAME = "spark-config-map" - private val CONFIG_MAP_KEY = "spark-config-map-key" - - test("Not passing a secret name should not set the secret value.") { - val baseSparkConf = new SparkConf(false) - val configurationUnderTest = new ExecutorInitContainerConfigurationImpl( - None, - SECRET_MOUNT_DIR, - CONFIG_MAP_NAME, - CONFIG_MAP_KEY) - val resolvedSparkConf = configurationUnderTest - .configureSparkConfForExecutorInitContainer(baseSparkConf) - assert(resolvedSparkConf.get(EXECUTOR_INIT_CONTAINER_CONFIG_MAP).contains(CONFIG_MAP_NAME)) - assert(resolvedSparkConf.get(EXECUTOR_INIT_CONTAINER_CONFIG_MAP_KEY).contains(CONFIG_MAP_KEY)) - assert(resolvedSparkConf.get(EXECUTOR_INIT_CONTAINER_SECRET_MOUNT_DIR) - .contains(SECRET_MOUNT_DIR)) - assert(resolvedSparkConf.get(EXECUTOR_INIT_CONTAINER_SECRET).isEmpty) - } - - test("Passing a secret name should set the secret value.") { - val baseSparkConf = new SparkConf(false) - val configurationUnderTest = new ExecutorInitContainerConfigurationImpl( - Some(SECRET_NAME), - SECRET_MOUNT_DIR, - CONFIG_MAP_NAME, - CONFIG_MAP_KEY) - val resolvedSparkConf = configurationUnderTest - .configureSparkConfForExecutorInitContainer(baseSparkConf) - assert(resolvedSparkConf.get(EXECUTOR_INIT_CONTAINER_SECRET).contains(SECRET_NAME)) - } -} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/PythonSubmissionResourcesSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/PythonSubmissionResourcesSuite.scala deleted file mode 100644 index 5483dc154fe73..0000000000000 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/PythonSubmissionResourcesSuite.scala +++ /dev/null @@ -1,112 +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.submit - -import org.apache.spark.{SSLOptions, SparkConf, SparkFunSuite} -import org.apache.spark.deploy.kubernetes.config._ - -import scala.collection.JavaConverters._ -import io.fabric8.kubernetes.api.model.{ContainerBuilder, Pod, PodBuilder} -import org.mockito.{Mock, MockitoAnnotations} -import org.mockito.Mockito.when -import org.scalatest.BeforeAndAfter - - - - -private[spark] class PythonSubmissionResourcesSuite extends SparkFunSuite with BeforeAndAfter { - private val PYSPARK_FILES = Seq( - "hdfs://localhost:9000/app/files/file1.py", - "file:///app/files/file2.py", - "local:///app/files/file3.py", - "http://app/files/file4.py", - "file:///app/files/file5.py") - private val RESOLVED_PYSPARK_FILES = Seq( - "hdfs://localhost:9000/app/files/file1.py", - "/var/spark-data/spark-files/file2.py", - "local:///app/file`s/file3.py", - "http://app/files/file4.py") - private val PYSPARK_PRIMARY_FILE = "file:///app/files/file5.py" - private val RESOLVED_PYSPARK_PRIMARY_FILE = "/var/data/spark-files/file5.py" - - private val pyFilesResource = new PythonSubmissionResourcesImpl( - PYSPARK_PRIMARY_FILE, Array(PYSPARK_FILES.mkString(","), "500") - ) - private val pyResource = new PythonSubmissionResourcesImpl( - PYSPARK_PRIMARY_FILE, Array(null, "500") - ) - private val DRIVER_CONTAINER_NAME = "pyspark_container" - private val driverContainer = new ContainerBuilder() - .withName(DRIVER_CONTAINER_NAME) - .build() - private val basePodBuilder = new PodBuilder() - .withNewMetadata() - .withName("base_pod") - .endMetadata() - .withNewSpec() - .addToContainers(driverContainer) - .endSpec() - - @Mock - private var driverInitContainer: DriverInitContainerComponentsProviderImpl = _ - @Mock - private var localizedFileResolver: ContainerLocalizedFilesResolverImpl = _ - before { - MockitoAnnotations.initMocks(this) - when(driverInitContainer.provideDriverPodFileMounter()).thenReturn( - new DriverPodKubernetesFileMounterImpl() - ) - when(localizedFileResolver.resolvePrimaryResourceFile()).thenReturn( - RESOLVED_PYSPARK_PRIMARY_FILE) - } - test("Test with --py-files included") { - assert(pyFilesResource.sparkJars === Seq.empty[String]) - assert(pyFilesResource.pySparkFiles === - PYSPARK_PRIMARY_FILE +: PYSPARK_FILES) - assert(pyFilesResource.primaryPySparkResource(localizedFileResolver) === - RESOLVED_PYSPARK_PRIMARY_FILE) - val driverPod: Pod = pyFilesResource.driverPodWithPySparkEnvs( - driverInitContainer.provideDriverPodFileMounter(), - RESOLVED_PYSPARK_PRIMARY_FILE, - RESOLVED_PYSPARK_FILES.mkString(","), - DRIVER_CONTAINER_NAME, - basePodBuilder - ) - val driverContainer = driverPod.getSpec.getContainers.asScala.head - val envs = driverContainer.getEnv.asScala.map(env => (env.getName, env.getValue)).toMap - envs.get("PYSPARK_PRIMARY") foreach{ a => assert (a === RESOLVED_PYSPARK_PRIMARY_FILE) } - envs.get("PYSPARK_FILES") foreach{ a => assert (a === RESOLVED_PYSPARK_FILES.mkString(",")) } - } - - test("Test without --py-files") { - assert(pyResource.sparkJars === Seq.empty[String]) - assert(pyResource.pySparkFiles === Array(PYSPARK_PRIMARY_FILE)) - assert(pyResource.primaryPySparkResource(localizedFileResolver) === - RESOLVED_PYSPARK_PRIMARY_FILE) - val driverPod: Pod = pyResource.driverPodWithPySparkEnvs( - driverInitContainer.provideDriverPodFileMounter(), - RESOLVED_PYSPARK_PRIMARY_FILE, - "", - DRIVER_CONTAINER_NAME, - basePodBuilder - ) - val driverContainer = driverPod.getSpec.getContainers.asScala.head - val envs = driverContainer.getEnv.asScala.map(env => (env.getName, env.getValue)).toMap - envs.get("PYSPARK_PRIMARY") foreach{ a => assert (a === RESOLVED_PYSPARK_PRIMARY_FILE) } - envs.get("PYSPARK_FILES") foreach{ a => assert (a === "") } - } -} \ No newline at end of file diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/SparkInitContainerConfigMapBuilderSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/SparkInitContainerConfigMapBuilderSuite.scala deleted file mode 100644 index f1e1ff7013496..0000000000000 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/SparkInitContainerConfigMapBuilderSuite.scala +++ /dev/null @@ -1,101 +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.submit - -import java.io.StringReader -import java.util.Properties - -import com.google.common.collect.Maps -import org.mockito.Mockito.{verify, when} -import org.scalatest.BeforeAndAfter -import org.scalatest.mock.MockitoSugar._ -import scala.collection.JavaConverters._ - -import org.apache.spark.SparkFunSuite -import org.apache.spark.deploy.kubernetes.config._ - -class SparkInitContainerConfigMapBuilderSuite extends SparkFunSuite with BeforeAndAfter { - - private val JARS = Seq( - "hdfs://localhost:9000/app/jars/jar1.jar", - "file:///app/jars/jar2.jar", - "http://localhost:9000/app/jars/jar3.jar", - "local:///app/jars/jar4.jar") - private val FILES = Seq( - "hdfs://localhost:9000/app/files/file1.txt", - "file:///app/files/file2.txt", - "http://localhost:9000/app/files/file3.txt", - "local:///app/files/file4.txt") - private val JARS_DOWNLOAD_PATH = "/var/data/jars" - private val FILES_DOWNLOAD_PATH = "/var/data/files" - private val CONFIG_MAP_NAME = "config-map" - private val CONFIG_MAP_KEY = "config-map-key" - - test("Config map without submitted dependencies sets remote download configurations") { - val configMap = new SparkInitContainerConfigMapBuilderImpl( - JARS, - FILES, - JARS_DOWNLOAD_PATH, - FILES_DOWNLOAD_PATH, - CONFIG_MAP_NAME, - CONFIG_MAP_KEY, - None).build() - assert(configMap.getMetadata.getName === CONFIG_MAP_NAME) - val maybeConfigValue = configMap.getData.asScala.get(CONFIG_MAP_KEY) - assert(maybeConfigValue.isDefined) - maybeConfigValue.foreach { configValue => - val propertiesStringReader = new StringReader(configValue) - val properties = new Properties() - properties.load(propertiesStringReader) - val propertiesMap = Maps.fromProperties(properties).asScala - val remoteJarsString = propertiesMap.get(INIT_CONTAINER_REMOTE_JARS.key) - assert(remoteJarsString.isDefined) - val remoteJars = remoteJarsString.map(_.split(",")).toSet.flatten - assert(remoteJars === - Set("hdfs://localhost:9000/app/jars/jar1.jar", "http://localhost:9000/app/jars/jar3.jar")) - val remoteFilesString = propertiesMap.get(INIT_CONTAINER_REMOTE_FILES.key) - assert(remoteFilesString.isDefined) - val remoteFiles = remoteFilesString.map(_.split(",")).toSet.flatten - assert(remoteFiles === - Set("hdfs://localhost:9000/app/files/file1.txt", - "http://localhost:9000/app/files/file3.txt")) - assert(propertiesMap(INIT_CONTAINER_JARS_DOWNLOAD_LOCATION.key) === JARS_DOWNLOAD_PATH) - assert(propertiesMap(INIT_CONTAINER_FILES_DOWNLOAD_LOCATION.key) === FILES_DOWNLOAD_PATH) - } - } - - test("Config map with submitted dependencies adds configurations from plugin") { - val submittedDependenciesPlugin = mock[SubmittedDependencyInitContainerConfigPlugin] - when(submittedDependenciesPlugin.configurationsToFetchSubmittedDependencies()) - .thenReturn(Map("customConf" -> "customConfValue")) - val configMap = new SparkInitContainerConfigMapBuilderImpl( - JARS, - FILES, - JARS_DOWNLOAD_PATH, - FILES_DOWNLOAD_PATH, - CONFIG_MAP_NAME, - CONFIG_MAP_KEY, - Some(submittedDependenciesPlugin)).build() - val configValue = configMap.getData.asScala(CONFIG_MAP_KEY) - val propertiesStringReader = new StringReader(configValue) - val properties = new Properties() - properties.load(propertiesStringReader) - val propertiesMap = Maps.fromProperties(properties).asScala - assert(propertiesMap("customConf") === "customConfValue") - verify(submittedDependenciesPlugin).configurationsToFetchSubmittedDependencies() - } -} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/SubmittedDependencyInitContainerConfigPluginSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/SubmittedDependencyInitContainerConfigPluginSuite.scala deleted file mode 100644 index 8431b77c9e85f..0000000000000 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/SubmittedDependencyInitContainerConfigPluginSuite.scala +++ /dev/null @@ -1,89 +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.submit - -import org.apache.spark.SparkFunSuite -import org.apache.spark.deploy.kubernetes.config._ - -class SubmittedDependencyInitContainerConfigPluginSuite extends SparkFunSuite { - private val STAGING_SERVER_URI = "http://localhost:9000" - private val STAGING_SERVER_INTERNAL_URI = "http://internalHost:9000" - private val JARS_RESOURCE_ID = "jars-id" - private val FILES_RESOURCE_ID = "files-id" - private val JARS_SECRET_KEY = "jars" - private val FILES_SECRET_KEY = "files" - private val TRUSTSTORE_SECRET_KEY = "trustStore" - private val CLIENT_CERT_SECRET_KEY = "client-cert" - private val SECRETS_VOLUME_MOUNT_PATH = "/var/data" - private val TRUSTSTORE_PASSWORD = "trustStore" - private val TRUSTSTORE_FILE = "/mnt/secrets/trustStore.jks" - private val CLIENT_CERT_URI = "local:///mnt/secrets/client-cert.pem" - private val TRUSTSTORE_TYPE = "jks" - - test("Plugin should provide configuration for fetching uploaded dependencies") { - val configPluginUnderTest = new SubmittedDependencyInitContainerConfigPluginImpl( - STAGING_SERVER_URI, - JARS_RESOURCE_ID, - FILES_RESOURCE_ID, - JARS_SECRET_KEY, - FILES_SECRET_KEY, - TRUSTSTORE_SECRET_KEY, - CLIENT_CERT_SECRET_KEY, - false, - None, - None, - None, - None, - SECRETS_VOLUME_MOUNT_PATH) - val addedConfigurations = configPluginUnderTest.configurationsToFetchSubmittedDependencies() - val expectedConfigurations = Map( - RESOURCE_STAGING_SERVER_URI.key -> STAGING_SERVER_URI, - INIT_CONTAINER_DOWNLOAD_JARS_RESOURCE_IDENTIFIER.key -> JARS_RESOURCE_ID, - INIT_CONTAINER_DOWNLOAD_FILES_RESOURCE_IDENTIFIER.key -> FILES_RESOURCE_ID, - INIT_CONTAINER_DOWNLOAD_JARS_SECRET_LOCATION.key -> - s"$SECRETS_VOLUME_MOUNT_PATH/$JARS_SECRET_KEY", - INIT_CONTAINER_DOWNLOAD_FILES_SECRET_LOCATION.key -> - s"$SECRETS_VOLUME_MOUNT_PATH/$FILES_SECRET_KEY", - RESOURCE_STAGING_SERVER_SSL_ENABLED.key -> "false") - assert(addedConfigurations === expectedConfigurations) - } - - test("Plugin should set up SSL with the appropriate trustStore if it's provided.") { - val configPluginUnderTest = new SubmittedDependencyInitContainerConfigPluginImpl( - STAGING_SERVER_URI, - JARS_RESOURCE_ID, - FILES_RESOURCE_ID, JARS_SECRET_KEY, - FILES_SECRET_KEY, - TRUSTSTORE_SECRET_KEY, - CLIENT_CERT_SECRET_KEY, - true, - Some(TRUSTSTORE_FILE), - Some(CLIENT_CERT_URI), - Some(TRUSTSTORE_PASSWORD), - Some(TRUSTSTORE_TYPE), - SECRETS_VOLUME_MOUNT_PATH) - val addedConfigurations = configPluginUnderTest.configurationsToFetchSubmittedDependencies() - val expectedSslConfigurations = Map( - RESOURCE_STAGING_SERVER_SSL_ENABLED.key -> "true", - RESOURCE_STAGING_SERVER_TRUSTSTORE_FILE.key -> - s"$SECRETS_VOLUME_MOUNT_PATH/$TRUSTSTORE_SECRET_KEY", - RESOURCE_STAGING_SERVER_TRUSTSTORE_PASSWORD.key -> TRUSTSTORE_PASSWORD, - RESOURCE_STAGING_SERVER_TRUSTSTORE_TYPE.key -> TRUSTSTORE_TYPE, - RESOURCE_STAGING_SERVER_CLIENT_CERT_PEM.key -> "/mnt/secrets/client-cert.pem") - assert(expectedSslConfigurations.toSet.subsetOf(addedConfigurations.toSet)) - } -} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/SubmittedDependencySecretBuilderSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/SubmittedDependencySecretBuilderSuite.scala deleted file mode 100644 index 83fd568e7a3aa..0000000000000 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/SubmittedDependencySecretBuilderSuite.scala +++ /dev/null @@ -1,109 +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.submit - -import java.io.File - -import com.google.common.base.Charsets -import com.google.common.io.{BaseEncoding, Files} -import io.fabric8.kubernetes.api.model.Secret -import scala.collection.JavaConverters._ -import scala.collection.Map - -import org.apache.spark.SparkFunSuite -import org.apache.spark.util.Utils - -class SubmittedDependencySecretBuilderSuite extends SparkFunSuite { - - private val SECRET_NAME = "submitted-dependency-secret" - private val JARS_SECRET = "jars-secret" - private val FILES_SECRET = "files-secret" - private val JARS_SECRET_KEY = "jars-secret-key" - private val FILES_SECRET_KEY = "files-secret-key" - private val TRUSTSTORE_SECRET_KEY = "truststore-secret-key" - private val CLIENT_CERT_SECRET_KEY = "client-cert" - private val TRUSTSTORE_STRING_CONTENTS = "trustStore-contents" - private val CLIENT_CERT_STRING_CONTENTS = "client-certificate-contents" - - test("Building the secret without a trustStore") { - val builder = new SubmittedDependencySecretBuilderImpl( - SECRET_NAME, - JARS_SECRET, - FILES_SECRET, - JARS_SECRET_KEY, - FILES_SECRET_KEY, - TRUSTSTORE_SECRET_KEY, - CLIENT_CERT_SECRET_KEY, - None, - None) - val secret = builder.build() - assert(secret.getMetadata.getName === SECRET_NAME) - val secretDecodedData = decodeSecretData(secret) - val expectedSecretData = Map(JARS_SECRET_KEY -> JARS_SECRET, FILES_SECRET_KEY -> FILES_SECRET) - assert(secretDecodedData === expectedSecretData) - } - - private def decodeSecretData(secret: Secret): Map[String, String] = { - val secretData = secret.getData.asScala - secretData.mapValues(encoded => - new String(BaseEncoding.base64().decode(encoded), Charsets.UTF_8)) - } - - test("Building the secret with a trustStore") { - val tempSslDir = Utils.createTempDir(namePrefix = "temp-ssl-tests") - try { - val trustStoreFile = new File(tempSslDir, "trustStore.jks") - Files.write(TRUSTSTORE_STRING_CONTENTS, trustStoreFile, Charsets.UTF_8) - val clientCertFile = new File(tempSslDir, "cert.pem") - Files.write(CLIENT_CERT_STRING_CONTENTS, clientCertFile, Charsets.UTF_8) - val builder = new SubmittedDependencySecretBuilderImpl( - SECRET_NAME, - JARS_SECRET, - FILES_SECRET, - JARS_SECRET_KEY, - FILES_SECRET_KEY, - TRUSTSTORE_SECRET_KEY, - CLIENT_CERT_SECRET_KEY, - Some(trustStoreFile.getAbsolutePath), - Some(clientCertFile.getAbsolutePath)) - val secret = builder.build() - val decodedSecretData = decodeSecretData(secret) - assert(decodedSecretData(TRUSTSTORE_SECRET_KEY) === TRUSTSTORE_STRING_CONTENTS) - assert(decodedSecretData(CLIENT_CERT_SECRET_KEY) === CLIENT_CERT_STRING_CONTENTS) - } finally { - tempSslDir.delete() - } - } - - test("If trustStore and certificate are container-local, don't add secret entries") { - val builder = new SubmittedDependencySecretBuilderImpl( - SECRET_NAME, - JARS_SECRET, - FILES_SECRET, - JARS_SECRET_KEY, - FILES_SECRET_KEY, - TRUSTSTORE_SECRET_KEY, - CLIENT_CERT_SECRET_KEY, - Some("local:///mnt/secrets/trustStore.jks"), - Some("local:///mnt/secrets/cert.pem")) - val secret = builder.build() - val decodedSecretData = decodeSecretData(secret) - assert(!decodedSecretData.contains(TRUSTSTORE_SECRET_KEY)) - assert(!decodedSecretData.contains(CLIENT_CERT_SECRET_KEY)) - } - -} 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 d2082291eba22..650ec4feb6a2b 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 @@ -34,7 +34,7 @@ import org.apache.spark.deploy.kubernetes.config._ import org.apache.spark.deploy.kubernetes.integrationtest.backend.IntegrationTestBackendFactory import org.apache.spark.deploy.kubernetes.integrationtest.backend.minikube.Minikube import org.apache.spark.deploy.kubernetes.integrationtest.constants.MINIKUBE_TEST_BACKEND -import org.apache.spark.deploy.kubernetes.submit.{Client, KeyAndCertPem} +import org.apache.spark.deploy.kubernetes.submit.{Client, ClientArguments, JavaMainAppResource, KeyAndCertPem, MainAppResource, PythonMainAppResource} import org.apache.spark.launcher.SparkLauncher import org.apache.spark.util.Utils @@ -82,8 +82,7 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { .set(EXECUTOR_DOCKER_IMAGE, System.getProperty("spark.docker.test.executorImage", "spark-executor-py:latest")) - runPySparkPiAndVerifyCompletion( - PYSPARK_PI_SUBMITTER_LOCAL_FILE_LOCATION) + runPySparkPiAndVerifyCompletion(PYSPARK_PI_SUBMITTER_LOCAL_FILE_LOCATION, Seq.empty[String]) } test("Run PySpark Job on file from CONTAINER with spark.jar defined") { @@ -96,8 +95,7 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { .set(EXECUTOR_DOCKER_IMAGE, System.getProperty("spark.docker.test.executorImage", "spark-executor-py:latest")) - runPySparkPiAndVerifyCompletion( - PYSPARK_PI_CONTAINER_LOCAL_FILE_LOCATION) + runPySparkPiAndVerifyCompletion(PYSPARK_PI_CONTAINER_LOCAL_FILE_LOCATION, Seq.empty[String]) } test("Simple submission test with the resource staging server.") { @@ -154,10 +152,11 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { sparkConf.set("spark.kubernetes.shuffle.namespace", kubernetesTestComponents.namespace) sparkConf.set("spark.app.name", "group-by-test") runSparkApplicationAndVerifyCompletion( - SUBMITTER_LOCAL_MAIN_APP_RESOURCE, + JavaMainAppResource(SUBMITTER_LOCAL_MAIN_APP_RESOURCE), GROUP_BY_MAIN_CLASS, "The Result is", - Array.empty[String]) + Array.empty[String], + Seq.empty[String]) } test("Use remote resources without the resource staging server.") { @@ -217,10 +216,11 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { launchStagingServer(SSLOptions(), None) sparkConf.set("spark.files", testExistenceFile.getAbsolutePath) runSparkApplicationAndVerifyCompletion( - SUBMITTER_LOCAL_MAIN_APP_RESOURCE, + JavaMainAppResource(SUBMITTER_LOCAL_MAIN_APP_RESOURCE), FILE_EXISTENCE_MAIN_CLASS, s"File found at /opt/spark/${testExistenceFile.getName} with correct contents.", - Array(testExistenceFile.getName, TEST_EXISTENCE_FILE_CONTENTS)) + Array(testExistenceFile.getName, TEST_EXISTENCE_FILE_CONTENTS), + Seq.empty[String]) } test("Use a very long application name.") { @@ -248,26 +248,35 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { private def runSparkPiAndVerifyCompletion(appResource: String): Unit = { runSparkApplicationAndVerifyCompletion( - appResource, SPARK_PI_MAIN_CLASS, "Pi is roughly 3", Array.empty[String]) + JavaMainAppResource(appResource), + SPARK_PI_MAIN_CLASS, + "Pi is roughly 3", + Array.empty[String], + Seq.empty[String]) } private def runPySparkPiAndVerifyCompletion( - appResource: String): Unit = { + appResource: String, otherPyFiles: Seq[String]): Unit = { runSparkApplicationAndVerifyCompletion( - appResource, PYSPARK_PI_MAIN_CLASS, "Pi is roughly 3", - Array(null, "5")) + PythonMainAppResource(appResource), + PYSPARK_PI_MAIN_CLASS, + "Pi is roughly 3", + Array("5"), + otherPyFiles) } private def runSparkApplicationAndVerifyCompletion( - appResource: String, + appResource: MainAppResource, mainClass: String, expectedLogOnCompletion: String, - appArgs: Array[String]): Unit = { - Client.run( - sparkConf = sparkConf, - appArgs = appArgs, + appArgs: Array[String], + otherPyFiles: Seq[String]): Unit = { + val clientArguments = ClientArguments( + mainAppResource = appResource, mainClass = mainClass, - mainAppResource = appResource) + driverArgs = appArgs, + otherPyFiles = otherPyFiles) + Client.run(sparkConf, clientArguments) val driverPod = kubernetesTestComponents.kubernetesClient .pods() .withLabel("spark-app-locator", APP_LOCATOR_LABEL) From e103225d9ff54ca17692279cc6a7999f9b8c3265 Mon Sep 17 00:00:00 2001 From: mcheah Date: Thu, 29 Jun 2017 17:13:53 -0700 Subject: [PATCH 20/26] Don't add the init-container step if all URIs are local. --- ...ubernetesSubmissionStepsOrchestrator.scala | 50 +++++++++++-------- 1 file changed, 29 insertions(+), 21 deletions(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/KubernetesSubmissionStepsOrchestrator.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/KubernetesSubmissionStepsOrchestrator.scala index f42afafbcb41a..b0c98b2575607 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/KubernetesSubmissionStepsOrchestrator.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/KubernetesSubmissionStepsOrchestrator.scala @@ -23,6 +23,7 @@ import org.apache.spark.deploy.kubernetes.constants._ import org.apache.spark.deploy.kubernetes.submit.submitsteps.{BaseSubmissionStep, DependencyResolutionStep, DriverKubernetesCredentialsStep, InitContainerBootstrapStep, KubernetesSubmissionStep, PythonStep} import org.apache.spark.deploy.kubernetes.submit.submitsteps.initcontainer.InitContainerStepsOrchestrator import org.apache.spark.launcher.SparkLauncher +import org.apache.spark.util.Utils /** * Constructs the complete list of submission steps to run to deploy the Spark application. @@ -93,26 +94,32 @@ private[spark] class KubernetesSubmissionStepsOrchestrator( val pythonStep = mainAppResource match { case PythonMainAppResource(mainPyResource) => Option(new PythonStep(mainPyResource, additionalPythonFiles, filesDownloadPath)) - case _ => Option.empty[PythonStep] + case _ => Option.empty[KubernetesSubmissionStep] + } + val initContainerBootstrapStep = if ((sparkJars ++ sparkFiles).exists { uri => + Option(Utils.resolveURI(uri).getScheme).getOrElse("file") != "local" + }) { + val initContainerStepsOrchestrator = new InitContainerStepsOrchestrator( + namespace, + kubernetesResourceNamePrefix, + sparkJars, + sparkFiles, + jarsDownloadPath, + filesDownloadPath, + dockerImagePullPolicy, + allDriverLabels, + initContainerConfigMapName, + INIT_CONTAINER_CONFIG_MAP_KEY, + submissionSparkConf) + val initContainerSteps = initContainerStepsOrchestrator.getInitContainerSteps() + Some(new InitContainerBootstrapStep( + submissionSparkConf, + initContainerSteps, + initContainerConfigMapName, + INIT_CONTAINER_CONFIG_MAP_KEY)) + } else { + Option.empty[KubernetesSubmissionStep] } - val initContainerStepsOrchestrator = new InitContainerStepsOrchestrator( - namespace, - kubernetesResourceNamePrefix, - sparkJars, - sparkFiles, - jarsDownloadPath, - filesDownloadPath, - dockerImagePullPolicy, - allDriverLabels, - initContainerConfigMapName, - INIT_CONTAINER_CONFIG_MAP_KEY, - submissionSparkConf) - val initContainerSteps = initContainerStepsOrchestrator.getInitContainerSteps() - val initContainerBootstrapStep = new InitContainerBootstrapStep( - submissionSparkConf, - initContainerSteps, - initContainerConfigMapName, - INIT_CONTAINER_CONFIG_MAP_KEY) val dependencyResolutionStep = new DependencyResolutionStep( sparkJars, sparkFiles, @@ -121,7 +128,8 @@ private[spark] class KubernetesSubmissionStepsOrchestrator( Seq( initialSubmissionStep, kubernetesCredentialsStep, - initContainerBootstrapStep, - dependencyResolutionStep) ++ pythonStep.toSeq + dependencyResolutionStep) ++ + initContainerBootstrapStep.toSeq ++ + pythonStep.toSeq } } From 4533df2a03e2a8922988b0bd01691ad1f26e5d03 Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Fri, 30 Jun 2017 11:59:44 -0700 Subject: [PATCH 21/26] Python arguments patch + tests + docs --- .../deploy/kubernetes/submit/Client.scala | 22 +++++++++--- ...ubernetesSubmissionStepsOrchestrator.scala | 10 +++--- .../submitsteps/BaseSubmissionStep.scala | 4 --- .../NonPythonArgumentResolver.scala | 35 +++++++++++++++++++ .../submit/submitsteps/PythonStep.scala | 16 ++++++++- .../integrationtest/KubernetesSuite.scala | 33 ++++++++++------- 6 files changed, 93 insertions(+), 27 deletions(-) create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/NonPythonArgumentResolver.scala diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/Client.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/Client.scala index e5b753d458b34..1776210200ed6 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/Client.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/Client.scala @@ -47,8 +47,8 @@ private[spark] object ClientArguments { mainAppResource = Some(PythonMainAppResource(mainPyFile)) case Array("--primary-java-resource", primaryJavaResource: String) => mainAppResource = Some(JavaMainAppResource(primaryJavaResource)) - case Array("--main-class", clazz: String) => - mainClass = Some(clazz) + case Array("--main-class", m_class: String) => + mainClass = Some(m_class) case Array("--other-py-files", pyFiles: String) => otherPyFiles = pyFiles.split(",") case Array("--arg", arg: String) => @@ -77,7 +77,13 @@ private[spark] class Client( private val driverJavaOptions = submissionSparkConf.get( org.apache.spark.internal.config.DRIVER_JAVA_OPTIONS) - + /** + * Run command that initalizes a DriverSpec that will be updated + * after each KubernetesSubmissionStep in the sequence that is passed in. + * The final driver-spec will be used to build the Driver Container, + * Driver Pod, and Kubernetes Resources + * + */ def run(): Unit = { var currentDriverSpec = new KubernetesDriverSpec( driverPod = new PodBuilder().build(), @@ -146,6 +152,8 @@ private[spark] object Client { val appName = sparkConf.getOption("spark.app.name").getOrElse("spark") val kubernetesAppId = s"spark-${UUID.randomUUID().toString.replaceAll("-", "")}" val master = resolveK8sMaster(sparkConf.get("spark.master")) + // This orchestrator determines which steps are necessary to take to resolve varying + // client arguments that are passed in. Use cases include: Scala/Java and Python submission val submissionStepsOrchestrator = new KubernetesSubmissionStepsOrchestrator( namespace, kubernetesAppId, @@ -177,7 +185,13 @@ private[spark] object Client { loggingPodStatusWatcher).run() } } - + /** + * Entry point from SparkSubmit in spark-core + * + * + * @param args Array of strings that have interchanging values that will be + * parsed by ClientArguments with the identifiers that preceed the values + */ def main(args: Array[String]): Unit = { val parsedArguments = ClientArguments.fromCommandLineArgs(args) val sparkConf = new SparkConf() diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/KubernetesSubmissionStepsOrchestrator.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/KubernetesSubmissionStepsOrchestrator.scala index b0c98b2575607..9628632fae3d7 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/KubernetesSubmissionStepsOrchestrator.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/KubernetesSubmissionStepsOrchestrator.scala @@ -20,7 +20,7 @@ import org.apache.spark.SparkConf import org.apache.spark.deploy.kubernetes.ConfigurationUtils import org.apache.spark.deploy.kubernetes.config._ import org.apache.spark.deploy.kubernetes.constants._ -import org.apache.spark.deploy.kubernetes.submit.submitsteps.{BaseSubmissionStep, DependencyResolutionStep, DriverKubernetesCredentialsStep, InitContainerBootstrapStep, KubernetesSubmissionStep, PythonStep} +import org.apache.spark.deploy.kubernetes.submit.submitsteps._ import org.apache.spark.deploy.kubernetes.submit.submitsteps.initcontainer.InitContainerStepsOrchestrator import org.apache.spark.launcher.SparkLauncher import org.apache.spark.util.Utils @@ -91,10 +91,10 @@ private[spark] class KubernetesSubmissionStepsOrchestrator( submissionSparkConf) val kubernetesCredentialsStep = new DriverKubernetesCredentialsStep( submissionSparkConf, kubernetesAppId) - val pythonStep = mainAppResource match { + val pythonResolverStep = mainAppResource match { case PythonMainAppResource(mainPyResource) => - Option(new PythonStep(mainPyResource, additionalPythonFiles, filesDownloadPath)) - case _ => Option.empty[KubernetesSubmissionStep] + Option(new PythonStep(mainPyResource, additionalPythonFiles, appArgs, filesDownloadPath)) + case _ => Option(new NonPythonArgumentResolver(appArgs)) } val initContainerBootstrapStep = if ((sparkJars ++ sparkFiles).exists { uri => Option(Utils.resolveURI(uri).getScheme).getOrElse("file") != "local" @@ -130,6 +130,6 @@ private[spark] class KubernetesSubmissionStepsOrchestrator( kubernetesCredentialsStep, dependencyResolutionStep) ++ initContainerBootstrapStep.toSeq ++ - pythonStep.toSeq + pythonResolverStep.toSeq } } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/BaseSubmissionStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/BaseSubmissionStep.scala index e49262c20c745..74c6de927568c 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/BaseSubmissionStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/BaseSubmissionStep.scala @@ -89,10 +89,6 @@ private[spark] class BaseSubmissionStep( .withName(ENV_DRIVER_MAIN_CLASS) .withValue(mainClass) .endEnv() - .addNewEnv() - .withName(ENV_DRIVER_ARGS) - .withValue(appArgs.mkString(" ")) - .endEnv() .withNewResources() .addToRequests("cpu", driverCpuQuantity) .addToRequests("memory", driverMemoryQuantity) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/NonPythonArgumentResolver.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/NonPythonArgumentResolver.scala new file mode 100644 index 0000000000000..74f45fc58b8bd --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/NonPythonArgumentResolver.scala @@ -0,0 +1,35 @@ +/* + * 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.submit.submitsteps + +import io.fabric8.kubernetes.api.model.ContainerBuilder + +import org.apache.spark.deploy.kubernetes.constants._ +import org.apache.spark.deploy.kubernetes.submit.KubernetesFileUtils + +private[spark] class NonPythonArgumentResolver( + appArgs: Array[String]) extends KubernetesSubmissionStep { + + override def prepareSubmission(driverSpec: KubernetesDriverSpec): KubernetesDriverSpec = { + val withNonPythonArgumentsResolvedContainer = new ContainerBuilder(driverSpec.driverContainer) + .addNewEnv() + .withName(ENV_DRIVER_ARGS) + .withValue(appArgs.mkString(" ")) + .endEnv() + driverSpec.copy(driverContainer = withNonPythonArgumentsResolvedContainer.build()) + } +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/PythonStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/PythonStep.scala index 484f57087b36e..dd0981c3802ad 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/PythonStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/PythonStep.scala @@ -16,18 +16,32 @@ */ package org.apache.spark.deploy.kubernetes.submit.submitsteps -import io.fabric8.kubernetes.api.model.ContainerBuilder +import org.apache.spark.internal.Logging +import io.fabric8.kubernetes.api.model.ContainerBuilder import org.apache.spark.deploy.kubernetes.constants._ import org.apache.spark.deploy.kubernetes.submit.KubernetesFileUtils + private[spark] class PythonStep( primaryPyFile: String, otherPyFiles: Seq[String], + appArgs: Array[String], filesDownloadPath: String) extends KubernetesSubmissionStep { override def prepareSubmission(driverSpec: KubernetesDriverSpec): KubernetesDriverSpec = { + val arguments : Array[String] = otherPyFiles.toList match { + case Nil => null +: appArgs + case a :: b => a match { + case _ if a == "" && b == Nil => null +: appArgs + case _ => appArgs + } + } val withPythonPrimaryFileContainer = new ContainerBuilder(driverSpec.driverContainer) + .addNewEnv() + .withName(ENV_DRIVER_ARGS) + .withValue(arguments.mkString(" ")) + .endEnv() .addNewEnv() .withName(ENV_PYSPARK_PRIMARY) .withValue(KubernetesFileUtils.resolveFilePath(primaryPyFile, filesDownloadPath)) 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 650ec4feb6a2b..d1e6637ee2ea4 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 @@ -72,7 +72,7 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { kubernetesTestComponents.deleteNamespace() } - test("Run PySpark Job on file from SUBMITTER") { + test("Run PySpark Job on file from SUBMITTER with --py-files") { assume(testBackend.name == MINIKUBE_TEST_BACKEND) launchStagingServer(SSLOptions(), None) @@ -82,7 +82,10 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { .set(EXECUTOR_DOCKER_IMAGE, System.getProperty("spark.docker.test.executorImage", "spark-executor-py:latest")) - runPySparkPiAndVerifyCompletion(PYSPARK_PI_SUBMITTER_LOCAL_FILE_LOCATION, Seq.empty[String]) + runPySparkPiAndVerifyCompletion( + PYSPARK_PI_SUBMITTER_LOCAL_FILE_LOCATION, + Seq(PYSPARK_SORT_CONTAINER_LOCAL_FILE_LOCATION) + ) } test("Run PySpark Job on file from CONTAINER with spark.jar defined") { @@ -154,7 +157,7 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { runSparkApplicationAndVerifyCompletion( JavaMainAppResource(SUBMITTER_LOCAL_MAIN_APP_RESOURCE), GROUP_BY_MAIN_CLASS, - "The Result is", + Array("The Result is"), Array.empty[String], Seq.empty[String]) } @@ -218,7 +221,7 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { runSparkApplicationAndVerifyCompletion( JavaMainAppResource(SUBMITTER_LOCAL_MAIN_APP_RESOURCE), FILE_EXISTENCE_MAIN_CLASS, - s"File found at /opt/spark/${testExistenceFile.getName} with correct contents.", + Array(s"File found at /opt/spark/${testExistenceFile.getName} with correct contents."), Array(testExistenceFile.getName, TEST_EXISTENCE_FILE_CONTENTS), Seq.empty[String]) } @@ -250,7 +253,7 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { runSparkApplicationAndVerifyCompletion( JavaMainAppResource(appResource), SPARK_PI_MAIN_CLASS, - "Pi is roughly 3", + Array("Pi is roughly 3"), Array.empty[String], Seq.empty[String]) } @@ -260,15 +263,15 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { runSparkApplicationAndVerifyCompletion( PythonMainAppResource(appResource), PYSPARK_PI_MAIN_CLASS, - "Pi is roughly 3", - Array("5"), + Array("(10/10)", "Pi is roughly 3"), + Array("10"), otherPyFiles) } private def runSparkApplicationAndVerifyCompletion( appResource: MainAppResource, mainClass: String, - expectedLogOnCompletion: String, + expectedLogOnCompletion: Array[String], appArgs: Array[String], otherPyFiles: Seq[String]): Unit = { val clientArguments = ClientArguments( @@ -284,11 +287,13 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { .getItems .get(0) Eventually.eventually(TIMEOUT, INTERVAL) { - assert(kubernetesTestComponents.kubernetesClient - .pods() - .withName(driverPod.getMetadata.getName) - .getLog - .contains(expectedLogOnCompletion), "The application did not complete.") + expectedLogOnCompletion.foreach { e => + assert(kubernetesTestComponents.kubernetesClient + .pods() + .withName(driverPod.getMetadata.getName) + .getLog + .contains(e), "The application did not complete.") + } } } @@ -357,6 +362,8 @@ private[spark] object KubernetesSuite { val PYSPARK_PI_CONTAINER_LOCAL_FILE_LOCATION = "local:///opt/spark/examples/src/main/python/pi.py" val PYSPARK_PI_SUBMITTER_LOCAL_FILE_LOCATION = "src/test/python/pi.py" + val PYSPARK_SORT_CONTAINER_LOCAL_FILE_LOCATION = + "local:///opt/spark/examples/src/main/python/sort.py" val FILE_EXISTENCE_MAIN_CLASS = "org.apache.spark.deploy.kubernetes" + ".integrationtest.jobs.FileExistenceTest" val GROUP_BY_MAIN_CLASS = "org.apache.spark.deploy.kubernetes" + From cc289f11d5bf05dce8870382566047efa735eff3 Mon Sep 17 00:00:00 2001 From: mcheah Date: Fri, 30 Jun 2017 12:14:58 -0700 Subject: [PATCH 22/26] Revert "Python arguments patch + tests + docs" This reverts commit 4533df2a03e2a8922988b0bd01691ad1f26e5d03. --- .../deploy/kubernetes/submit/Client.scala | 22 +++--------- ...ubernetesSubmissionStepsOrchestrator.scala | 10 +++--- .../submitsteps/BaseSubmissionStep.scala | 4 +++ .../NonPythonArgumentResolver.scala | 35 ------------------- .../submit/submitsteps/PythonStep.scala | 16 +-------- .../integrationtest/KubernetesSuite.scala | 33 +++++++---------- 6 files changed, 27 insertions(+), 93 deletions(-) delete mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/NonPythonArgumentResolver.scala diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/Client.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/Client.scala index 1776210200ed6..e5b753d458b34 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/Client.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/Client.scala @@ -47,8 +47,8 @@ private[spark] object ClientArguments { mainAppResource = Some(PythonMainAppResource(mainPyFile)) case Array("--primary-java-resource", primaryJavaResource: String) => mainAppResource = Some(JavaMainAppResource(primaryJavaResource)) - case Array("--main-class", m_class: String) => - mainClass = Some(m_class) + case Array("--main-class", clazz: String) => + mainClass = Some(clazz) case Array("--other-py-files", pyFiles: String) => otherPyFiles = pyFiles.split(",") case Array("--arg", arg: String) => @@ -77,13 +77,7 @@ private[spark] class Client( private val driverJavaOptions = submissionSparkConf.get( org.apache.spark.internal.config.DRIVER_JAVA_OPTIONS) - /** - * Run command that initalizes a DriverSpec that will be updated - * after each KubernetesSubmissionStep in the sequence that is passed in. - * The final driver-spec will be used to build the Driver Container, - * Driver Pod, and Kubernetes Resources - * - */ + def run(): Unit = { var currentDriverSpec = new KubernetesDriverSpec( driverPod = new PodBuilder().build(), @@ -152,8 +146,6 @@ private[spark] object Client { val appName = sparkConf.getOption("spark.app.name").getOrElse("spark") val kubernetesAppId = s"spark-${UUID.randomUUID().toString.replaceAll("-", "")}" val master = resolveK8sMaster(sparkConf.get("spark.master")) - // This orchestrator determines which steps are necessary to take to resolve varying - // client arguments that are passed in. Use cases include: Scala/Java and Python submission val submissionStepsOrchestrator = new KubernetesSubmissionStepsOrchestrator( namespace, kubernetesAppId, @@ -185,13 +177,7 @@ private[spark] object Client { loggingPodStatusWatcher).run() } } - /** - * Entry point from SparkSubmit in spark-core - * - * - * @param args Array of strings that have interchanging values that will be - * parsed by ClientArguments with the identifiers that preceed the values - */ + def main(args: Array[String]): Unit = { val parsedArguments = ClientArguments.fromCommandLineArgs(args) val sparkConf = new SparkConf() diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/KubernetesSubmissionStepsOrchestrator.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/KubernetesSubmissionStepsOrchestrator.scala index 9628632fae3d7..b0c98b2575607 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/KubernetesSubmissionStepsOrchestrator.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/KubernetesSubmissionStepsOrchestrator.scala @@ -20,7 +20,7 @@ import org.apache.spark.SparkConf import org.apache.spark.deploy.kubernetes.ConfigurationUtils import org.apache.spark.deploy.kubernetes.config._ import org.apache.spark.deploy.kubernetes.constants._ -import org.apache.spark.deploy.kubernetes.submit.submitsteps._ +import org.apache.spark.deploy.kubernetes.submit.submitsteps.{BaseSubmissionStep, DependencyResolutionStep, DriverKubernetesCredentialsStep, InitContainerBootstrapStep, KubernetesSubmissionStep, PythonStep} import org.apache.spark.deploy.kubernetes.submit.submitsteps.initcontainer.InitContainerStepsOrchestrator import org.apache.spark.launcher.SparkLauncher import org.apache.spark.util.Utils @@ -91,10 +91,10 @@ private[spark] class KubernetesSubmissionStepsOrchestrator( submissionSparkConf) val kubernetesCredentialsStep = new DriverKubernetesCredentialsStep( submissionSparkConf, kubernetesAppId) - val pythonResolverStep = mainAppResource match { + val pythonStep = mainAppResource match { case PythonMainAppResource(mainPyResource) => - Option(new PythonStep(mainPyResource, additionalPythonFiles, appArgs, filesDownloadPath)) - case _ => Option(new NonPythonArgumentResolver(appArgs)) + Option(new PythonStep(mainPyResource, additionalPythonFiles, filesDownloadPath)) + case _ => Option.empty[KubernetesSubmissionStep] } val initContainerBootstrapStep = if ((sparkJars ++ sparkFiles).exists { uri => Option(Utils.resolveURI(uri).getScheme).getOrElse("file") != "local" @@ -130,6 +130,6 @@ private[spark] class KubernetesSubmissionStepsOrchestrator( kubernetesCredentialsStep, dependencyResolutionStep) ++ initContainerBootstrapStep.toSeq ++ - pythonResolverStep.toSeq + pythonStep.toSeq } } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/BaseSubmissionStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/BaseSubmissionStep.scala index 74c6de927568c..e49262c20c745 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/BaseSubmissionStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/BaseSubmissionStep.scala @@ -89,6 +89,10 @@ private[spark] class BaseSubmissionStep( .withName(ENV_DRIVER_MAIN_CLASS) .withValue(mainClass) .endEnv() + .addNewEnv() + .withName(ENV_DRIVER_ARGS) + .withValue(appArgs.mkString(" ")) + .endEnv() .withNewResources() .addToRequests("cpu", driverCpuQuantity) .addToRequests("memory", driverMemoryQuantity) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/NonPythonArgumentResolver.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/NonPythonArgumentResolver.scala deleted file mode 100644 index 74f45fc58b8bd..0000000000000 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/NonPythonArgumentResolver.scala +++ /dev/null @@ -1,35 +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.submit.submitsteps - -import io.fabric8.kubernetes.api.model.ContainerBuilder - -import org.apache.spark.deploy.kubernetes.constants._ -import org.apache.spark.deploy.kubernetes.submit.KubernetesFileUtils - -private[spark] class NonPythonArgumentResolver( - appArgs: Array[String]) extends KubernetesSubmissionStep { - - override def prepareSubmission(driverSpec: KubernetesDriverSpec): KubernetesDriverSpec = { - val withNonPythonArgumentsResolvedContainer = new ContainerBuilder(driverSpec.driverContainer) - .addNewEnv() - .withName(ENV_DRIVER_ARGS) - .withValue(appArgs.mkString(" ")) - .endEnv() - driverSpec.copy(driverContainer = withNonPythonArgumentsResolvedContainer.build()) - } -} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/PythonStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/PythonStep.scala index dd0981c3802ad..484f57087b36e 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/PythonStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/PythonStep.scala @@ -16,32 +16,18 @@ */ package org.apache.spark.deploy.kubernetes.submit.submitsteps -import org.apache.spark.internal.Logging - import io.fabric8.kubernetes.api.model.ContainerBuilder + import org.apache.spark.deploy.kubernetes.constants._ import org.apache.spark.deploy.kubernetes.submit.KubernetesFileUtils - private[spark] class PythonStep( primaryPyFile: String, otherPyFiles: Seq[String], - appArgs: Array[String], filesDownloadPath: String) extends KubernetesSubmissionStep { override def prepareSubmission(driverSpec: KubernetesDriverSpec): KubernetesDriverSpec = { - val arguments : Array[String] = otherPyFiles.toList match { - case Nil => null +: appArgs - case a :: b => a match { - case _ if a == "" && b == Nil => null +: appArgs - case _ => appArgs - } - } val withPythonPrimaryFileContainer = new ContainerBuilder(driverSpec.driverContainer) - .addNewEnv() - .withName(ENV_DRIVER_ARGS) - .withValue(arguments.mkString(" ")) - .endEnv() .addNewEnv() .withName(ENV_PYSPARK_PRIMARY) .withValue(KubernetesFileUtils.resolveFilePath(primaryPyFile, filesDownloadPath)) 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 d1e6637ee2ea4..650ec4feb6a2b 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 @@ -72,7 +72,7 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { kubernetesTestComponents.deleteNamespace() } - test("Run PySpark Job on file from SUBMITTER with --py-files") { + test("Run PySpark Job on file from SUBMITTER") { assume(testBackend.name == MINIKUBE_TEST_BACKEND) launchStagingServer(SSLOptions(), None) @@ -82,10 +82,7 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { .set(EXECUTOR_DOCKER_IMAGE, System.getProperty("spark.docker.test.executorImage", "spark-executor-py:latest")) - runPySparkPiAndVerifyCompletion( - PYSPARK_PI_SUBMITTER_LOCAL_FILE_LOCATION, - Seq(PYSPARK_SORT_CONTAINER_LOCAL_FILE_LOCATION) - ) + runPySparkPiAndVerifyCompletion(PYSPARK_PI_SUBMITTER_LOCAL_FILE_LOCATION, Seq.empty[String]) } test("Run PySpark Job on file from CONTAINER with spark.jar defined") { @@ -157,7 +154,7 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { runSparkApplicationAndVerifyCompletion( JavaMainAppResource(SUBMITTER_LOCAL_MAIN_APP_RESOURCE), GROUP_BY_MAIN_CLASS, - Array("The Result is"), + "The Result is", Array.empty[String], Seq.empty[String]) } @@ -221,7 +218,7 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { runSparkApplicationAndVerifyCompletion( JavaMainAppResource(SUBMITTER_LOCAL_MAIN_APP_RESOURCE), FILE_EXISTENCE_MAIN_CLASS, - Array(s"File found at /opt/spark/${testExistenceFile.getName} with correct contents."), + s"File found at /opt/spark/${testExistenceFile.getName} with correct contents.", Array(testExistenceFile.getName, TEST_EXISTENCE_FILE_CONTENTS), Seq.empty[String]) } @@ -253,7 +250,7 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { runSparkApplicationAndVerifyCompletion( JavaMainAppResource(appResource), SPARK_PI_MAIN_CLASS, - Array("Pi is roughly 3"), + "Pi is roughly 3", Array.empty[String], Seq.empty[String]) } @@ -263,15 +260,15 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { runSparkApplicationAndVerifyCompletion( PythonMainAppResource(appResource), PYSPARK_PI_MAIN_CLASS, - Array("(10/10)", "Pi is roughly 3"), - Array("10"), + "Pi is roughly 3", + Array("5"), otherPyFiles) } private def runSparkApplicationAndVerifyCompletion( appResource: MainAppResource, mainClass: String, - expectedLogOnCompletion: Array[String], + expectedLogOnCompletion: String, appArgs: Array[String], otherPyFiles: Seq[String]): Unit = { val clientArguments = ClientArguments( @@ -287,13 +284,11 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { .getItems .get(0) Eventually.eventually(TIMEOUT, INTERVAL) { - expectedLogOnCompletion.foreach { e => - assert(kubernetesTestComponents.kubernetesClient - .pods() - .withName(driverPod.getMetadata.getName) - .getLog - .contains(e), "The application did not complete.") - } + assert(kubernetesTestComponents.kubernetesClient + .pods() + .withName(driverPod.getMetadata.getName) + .getLog + .contains(expectedLogOnCompletion), "The application did not complete.") } } @@ -362,8 +357,6 @@ private[spark] object KubernetesSuite { val PYSPARK_PI_CONTAINER_LOCAL_FILE_LOCATION = "local:///opt/spark/examples/src/main/python/pi.py" val PYSPARK_PI_SUBMITTER_LOCAL_FILE_LOCATION = "src/test/python/pi.py" - val PYSPARK_SORT_CONTAINER_LOCAL_FILE_LOCATION = - "local:///opt/spark/examples/src/main/python/sort.py" val FILE_EXISTENCE_MAIN_CLASS = "org.apache.spark.deploy.kubernetes" + ".integrationtest.jobs.FileExistenceTest" val GROUP_BY_MAIN_CLASS = "org.apache.spark.deploy.kubernetes" + From c267286ed958d0a6efbad2b0ab3929ab66dc25b5 Mon Sep 17 00:00:00 2001 From: mcheah Date: Fri, 30 Jun 2017 12:15:03 -0700 Subject: [PATCH 23/26] Revert "Don't add the init-container step if all URIs are local." This reverts commit e103225d9ff54ca17692279cc6a7999f9b8c3265. --- ...ubernetesSubmissionStepsOrchestrator.scala | 50 ++++++++----------- 1 file changed, 21 insertions(+), 29 deletions(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/KubernetesSubmissionStepsOrchestrator.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/KubernetesSubmissionStepsOrchestrator.scala index b0c98b2575607..f42afafbcb41a 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/KubernetesSubmissionStepsOrchestrator.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/KubernetesSubmissionStepsOrchestrator.scala @@ -23,7 +23,6 @@ import org.apache.spark.deploy.kubernetes.constants._ import org.apache.spark.deploy.kubernetes.submit.submitsteps.{BaseSubmissionStep, DependencyResolutionStep, DriverKubernetesCredentialsStep, InitContainerBootstrapStep, KubernetesSubmissionStep, PythonStep} import org.apache.spark.deploy.kubernetes.submit.submitsteps.initcontainer.InitContainerStepsOrchestrator import org.apache.spark.launcher.SparkLauncher -import org.apache.spark.util.Utils /** * Constructs the complete list of submission steps to run to deploy the Spark application. @@ -94,32 +93,26 @@ private[spark] class KubernetesSubmissionStepsOrchestrator( val pythonStep = mainAppResource match { case PythonMainAppResource(mainPyResource) => Option(new PythonStep(mainPyResource, additionalPythonFiles, filesDownloadPath)) - case _ => Option.empty[KubernetesSubmissionStep] - } - val initContainerBootstrapStep = if ((sparkJars ++ sparkFiles).exists { uri => - Option(Utils.resolveURI(uri).getScheme).getOrElse("file") != "local" - }) { - val initContainerStepsOrchestrator = new InitContainerStepsOrchestrator( - namespace, - kubernetesResourceNamePrefix, - sparkJars, - sparkFiles, - jarsDownloadPath, - filesDownloadPath, - dockerImagePullPolicy, - allDriverLabels, - initContainerConfigMapName, - INIT_CONTAINER_CONFIG_MAP_KEY, - submissionSparkConf) - val initContainerSteps = initContainerStepsOrchestrator.getInitContainerSteps() - Some(new InitContainerBootstrapStep( - submissionSparkConf, - initContainerSteps, - initContainerConfigMapName, - INIT_CONTAINER_CONFIG_MAP_KEY)) - } else { - Option.empty[KubernetesSubmissionStep] + case _ => Option.empty[PythonStep] } + val initContainerStepsOrchestrator = new InitContainerStepsOrchestrator( + namespace, + kubernetesResourceNamePrefix, + sparkJars, + sparkFiles, + jarsDownloadPath, + filesDownloadPath, + dockerImagePullPolicy, + allDriverLabels, + initContainerConfigMapName, + INIT_CONTAINER_CONFIG_MAP_KEY, + submissionSparkConf) + val initContainerSteps = initContainerStepsOrchestrator.getInitContainerSteps() + val initContainerBootstrapStep = new InitContainerBootstrapStep( + submissionSparkConf, + initContainerSteps, + initContainerConfigMapName, + INIT_CONTAINER_CONFIG_MAP_KEY) val dependencyResolutionStep = new DependencyResolutionStep( sparkJars, sparkFiles, @@ -128,8 +121,7 @@ private[spark] class KubernetesSubmissionStepsOrchestrator( Seq( initialSubmissionStep, kubernetesCredentialsStep, - dependencyResolutionStep) ++ - initContainerBootstrapStep.toSeq ++ - pythonStep.toSeq + initContainerBootstrapStep, + dependencyResolutionStep) ++ pythonStep.toSeq } } From 8045c94c1cf7327434ea008e48dbe07f59fb9861 Mon Sep 17 00:00:00 2001 From: mcheah Date: Fri, 30 Jun 2017 12:15:07 -0700 Subject: [PATCH 24/26] Revert "Submission client redesign to use a step-based builder pattern." This reverts commit 5499f6ddf9b42c0526f1dc053317afb38dc71294. --- .../org/apache/spark/deploy/SparkSubmit.scala | 10 +- ...nerResourceStagingServerSecretPlugin.scala | 38 +- .../PodWithDetachedInitContainer.scala | 24 - .../SparkPodInitContainerBootstrap.scala | 51 +- .../spark/deploy/kubernetes/config.scala | 8 - .../deploy/kubernetes/submit/Client.scala | 428 ++++++++++---- .../ContainerLocalizedFilesResolver.scala | 87 +++ ...riverInitContainerComponentsProvider.scala | 223 ++++++++ ...riverPodKubernetesCredentialsMounter.scala | 184 ++++++ ...KubernetesCredentialsMounterProvider.scala | 49 ++ ...iverPodKubernetesCredentialsProvider.scala | 63 ++ .../DriverPodKubernetesFileMounter.scala | 55 ++ ...source.scala => InitContainerBundle.scala} | 9 +- .../kubernetes/submit/InitContainerUtil.scala | 16 +- .../submit/KubernetesFileUtils.scala | 45 -- ...ubernetesSubmissionStepsOrchestrator.scala | 127 ----- .../submit/PythonSubmissionResources.scala | 75 +++ ... SparkInitContainerConfigMapBuilder.scala} | 41 +- ...dDependencyInitContainerConfigPlugin.scala | 96 ++++ .../SubmittedDependencySecretBuilder.scala | 81 +++ .../submitsteps/BaseSubmissionStep.scala | 136 ----- .../DependencyResolutionStep.scala | 66 --- .../DriverKubernetesCredentialsStep.scala | 218 ------- .../InitContainerBootstrapStep.scala | 64 --- .../submitsteps/KubernetesDriverSpec.scala | 35 -- .../KubernetesSubmissionStep.scala | 28 - .../submit/submitsteps/PythonStep.scala | 42 -- .../initcontainer/InitContainerSpec.scala | 41 -- .../initcontainer/InitContainerStep.scala | 25 - .../InitContainerStepsOrchestrator.scala | 131 ----- .../SubmittedResourcesInitContainerStep.scala | 146 ----- ...SparkDependencyDownloadInitContainer.scala | 1 + .../kubernetes/KubernetesClusterManager.scala | 9 +- .../KubernetesClusterSchedulerBackend.scala | 149 ++--- .../SparkPodInitContainerBootstrapSuite.scala | 284 ++++----- ...dencyInitContainerVolumesPluginSuite.scala | 60 ++ .../kubernetes/submit/ClientV2Suite.scala | 537 +++++++++++++++++- ...ContainerLocalizedFilesResolverSuite.scala | 93 +++ ...PodKubernetesCredentialsMounterSuite.scala | 171 ++++++ ...cutorInitContainerConfigurationSuite.scala | 56 ++ .../PythonSubmissionResourcesSuite.scala | 112 ++++ ...rkInitContainerConfigMapBuilderSuite.scala | 101 ++++ ...ndencyInitContainerConfigPluginSuite.scala | 89 +++ ...ubmittedDependencySecretBuilderSuite.scala | 109 ++++ .../integrationtest/KubernetesSuite.scala | 47 +- 45 files changed, 2865 insertions(+), 1595 deletions(-) delete mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/PodWithDetachedInitContainer.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/ContainerLocalizedFilesResolver.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/DriverInitContainerComponentsProvider.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/DriverPodKubernetesCredentialsMounter.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/DriverPodKubernetesCredentialsMounterProvider.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/DriverPodKubernetesCredentialsProvider.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/DriverPodKubernetesFileMounter.scala rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/{MainAppResource.scala => InitContainerBundle.scala} (71%) delete mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/KubernetesSubmissionStepsOrchestrator.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/PythonSubmissionResources.scala rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/{submitsteps/initcontainer/BaseInitContainerStep.scala => SparkInitContainerConfigMapBuilder.scala} (62%) create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/SubmittedDependencyInitContainerConfigPlugin.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/SubmittedDependencySecretBuilder.scala delete mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/BaseSubmissionStep.scala delete mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DependencyResolutionStep.scala delete mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DriverKubernetesCredentialsStep.scala delete mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/InitContainerBootstrapStep.scala delete mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/KubernetesDriverSpec.scala delete mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/KubernetesSubmissionStep.scala delete mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/PythonStep.scala delete mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/InitContainerSpec.scala delete mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/InitContainerStep.scala delete mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/InitContainerStepsOrchestrator.scala delete mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/SubmittedResourcesInitContainerStep.scala create mode 100644 resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/SubmittedDependencyInitContainerVolumesPluginSuite.scala create mode 100644 resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/ContainerLocalizedFilesResolverSuite.scala create mode 100644 resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/DriverPodKubernetesCredentialsMounterSuite.scala create mode 100644 resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/ExecutorInitContainerConfigurationSuite.scala create mode 100644 resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/PythonSubmissionResourcesSuite.scala create mode 100644 resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/SparkInitContainerConfigMapBuilderSuite.scala create mode 100644 resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/SubmittedDependencyInitContainerConfigPluginSuite.scala create mode 100644 resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/SubmittedDependencySecretBuilderSuite.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 d3c7a016f96bb..9256a9ddd9960 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -621,22 +621,14 @@ object SparkSubmit { if (isKubernetesCluster) { childMainClass = "org.apache.spark.deploy.kubernetes.submit.Client" if (args.isPython) { - childArgs += "--py-file" childArgs += args.primaryResource - childArgs += "--main-class" childArgs += "org.apache.spark.deploy.PythonRunner" - childArgs += "--other-py-files" childArgs += args.pyFiles } else { - childArgs += "--primary-java-resource" childArgs += args.primaryResource - childArgs += "--main-class" childArgs += args.mainClass } - args.childArgs.foreach { arg => - childArgs += "--arg" - childArgs += arg - } + childArgs ++= args.childArgs } // Load any properties specified through --conf and the default properties file diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/InitContainerResourceStagingServerSecretPlugin.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/InitContainerResourceStagingServerSecretPlugin.scala index 265b8f197a102..45b881a8a3737 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/InitContainerResourceStagingServerSecretPlugin.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/InitContainerResourceStagingServerSecretPlugin.scala @@ -16,7 +16,7 @@ */ package org.apache.spark.deploy.kubernetes -import io.fabric8.kubernetes.api.model.{Container, ContainerBuilder, Pod, PodBuilder, Secret} +import io.fabric8.kubernetes.api.model.{ContainerBuilder, PodBuilder, Secret} import org.apache.spark.deploy.kubernetes.constants._ @@ -27,13 +27,13 @@ private[spark] trait InitContainerResourceStagingServerSecretPlugin { * from a resource staging server. */ def mountResourceStagingServerSecretIntoInitContainer( - initContainer: Container): Container + initContainer: ContainerBuilder): ContainerBuilder /** * Configure the pod to attach a Secret volume which hosts secret files allowing the * init-container to retrieve dependencies from the resource staging server. */ - def addResourceStagingServerSecretVolumeToPod(basePod: Pod): Pod + def addResourceStagingServerSecretVolumeToPod(basePod: PodBuilder): PodBuilder } private[spark] class InitContainerResourceStagingServerSecretPluginImpl( @@ -42,25 +42,21 @@ private[spark] class InitContainerResourceStagingServerSecretPluginImpl( extends InitContainerResourceStagingServerSecretPlugin { override def mountResourceStagingServerSecretIntoInitContainer( - initContainer: Container): Container = { - new ContainerBuilder(initContainer) - .addNewVolumeMount() - .withName(INIT_CONTAINER_SECRET_VOLUME_NAME) - .withMountPath(initContainerSecretMountPath) - .endVolumeMount() - .build() + initContainer: ContainerBuilder): ContainerBuilder = { + initContainer.addNewVolumeMount() + .withName(INIT_CONTAINER_SECRET_VOLUME_NAME) + .withMountPath(initContainerSecretMountPath) + .endVolumeMount() } - override def addResourceStagingServerSecretVolumeToPod(basePod: Pod): Pod = { - new PodBuilder(basePod) - .editSpec() - .addNewVolume() - .withName(INIT_CONTAINER_SECRET_VOLUME_NAME) - .withNewSecret() - .withSecretName(initContainerSecretName) - .endSecret() - .endVolume() - .endSpec() - .build() + override def addResourceStagingServerSecretVolumeToPod(basePod: PodBuilder): PodBuilder = { + basePod.editSpec() + .addNewVolume() + .withName(INIT_CONTAINER_SECRET_VOLUME_NAME) + .withNewSecret() + .withSecretName(initContainerSecretName) + .endSecret() + .endVolume() + .endSpec() } } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/PodWithDetachedInitContainer.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/PodWithDetachedInitContainer.scala deleted file mode 100644 index 36b1b07dc6bc4..0000000000000 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/PodWithDetachedInitContainer.scala +++ /dev/null @@ -1,24 +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 io.fabric8.kubernetes.api.model.{Container, Pod} - -private[spark] case class PodWithDetachedInitContainer( - pod: Pod, - initContainer: Container, - mainContainer: Container) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/SparkPodInitContainerBootstrap.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/SparkPodInitContainerBootstrap.scala index c2b04c6c49311..87462dbde17a5 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/SparkPodInitContainerBootstrap.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/SparkPodInitContainerBootstrap.scala @@ -19,11 +19,8 @@ package org.apache.spark.deploy.kubernetes import io.fabric8.kubernetes.api.model.{ContainerBuilder, EmptyDirVolumeSource, PodBuilder, VolumeMount, VolumeMountBuilder} import org.apache.spark.deploy.kubernetes.constants._ +import org.apache.spark.deploy.kubernetes.submit.{ContainerNameEqualityPredicate, InitContainerUtil} -/** - * This is separated out from the init-container steps API because this component can be reused to - * set up the init-container for executors as well. - */ private[spark] trait SparkPodInitContainerBootstrap { /** * Bootstraps an init-container that downloads dependencies to be used by a main container. @@ -31,13 +28,10 @@ private[spark] trait SparkPodInitContainerBootstrap { * by a ConfigMap that was installed by some other component; that is, the implementation * here makes no assumptions about how the init-container is specifically configured. For * example, this class is unaware if the init-container is fetching remote dependencies or if - * it is fetching dependencies from a resource staging server. Additionally, the container itself - * is not actually attached to the pod, but the init container is returned so it can be attached - * by InitContainerUtil after the caller has decided to make any changes to it. + * it is fetching dependencies from a resource staging server. */ def bootstrapInitContainerAndVolumes( - originalPodWithUnattachedInitContainer: PodWithDetachedInitContainer) - : PodWithDetachedInitContainer + mainContainerName: String, originalPodSpec: PodBuilder): PodBuilder } private[spark] class SparkPodInitContainerBootstrapImpl( @@ -47,12 +41,13 @@ private[spark] class SparkPodInitContainerBootstrapImpl( filesDownloadPath: String, downloadTimeoutMinutes: Long, initContainerConfigMapName: String, - initContainerConfigMapKey: String) + initContainerConfigMapKey: String, + resourceStagingServerSecretPlugin: Option[InitContainerResourceStagingServerSecretPlugin]) extends SparkPodInitContainerBootstrap { override def bootstrapInitContainerAndVolumes( - originalPodWithUnattachedInitContainer: PodWithDetachedInitContainer) - : PodWithDetachedInitContainer = { + mainContainerName: String, + originalPodSpec: PodBuilder): PodBuilder = { val sharedVolumeMounts = Seq[VolumeMount]( new VolumeMountBuilder() .withName(INIT_CONTAINER_DOWNLOAD_JARS_VOLUME_NAME) @@ -63,7 +58,7 @@ private[spark] class SparkPodInitContainerBootstrapImpl( .withMountPath(filesDownloadPath) .build()) - val initContainer = new ContainerBuilder(originalPodWithUnattachedInitContainer.initContainer) + val initContainer = new ContainerBuilder() .withName(s"spark-init") .withImage(initContainerImage) .withImagePullPolicy(dockerImagePullPolicy) @@ -73,8 +68,11 @@ private[spark] class SparkPodInitContainerBootstrapImpl( .endVolumeMount() .addToVolumeMounts(sharedVolumeMounts: _*) .addToArgs(INIT_CONTAINER_PROPERTIES_FILE_PATH) - .build() - val podWithBasicVolumes = new PodBuilder(originalPodWithUnattachedInitContainer.pod) + val resolvedInitContainer = resourceStagingServerSecretPlugin.map { plugin => + plugin.mountResourceStagingServerSecretIntoInitContainer(initContainer) + }.getOrElse(initContainer).build() + val podWithBasicVolumes = InitContainerUtil.appendInitContainer( + originalPodSpec, resolvedInitContainer) .editSpec() .addNewVolume() .withName(INIT_CONTAINER_PROPERTIES_FILE_VOLUME) @@ -94,20 +92,17 @@ private[spark] class SparkPodInitContainerBootstrapImpl( .withName(INIT_CONTAINER_DOWNLOAD_FILES_VOLUME_NAME) .withEmptyDir(new EmptyDirVolumeSource()) .endVolume() + .editMatchingContainer(new ContainerNameEqualityPredicate(mainContainerName)) + .addToVolumeMounts(sharedVolumeMounts: _*) + .addNewEnv() + .withName(ENV_MOUNTED_FILES_DIR) + .withValue(filesDownloadPath) + .endEnv() + .endContainer() .endSpec() - .build() - val mainContainerWithMountedFiles = new ContainerBuilder( - originalPodWithUnattachedInitContainer.mainContainer) - .addToVolumeMounts(sharedVolumeMounts: _*) - .addNewEnv() - .withName(ENV_MOUNTED_FILES_DIR) - .withValue(filesDownloadPath) - .endEnv() - .build() - PodWithDetachedInitContainer( - podWithBasicVolumes, - initContainer, - mainContainerWithMountedFiles) + resourceStagingServerSecretPlugin.map { plugin => + plugin.addResourceStagingServerSecretVolumeToPod(podWithBasicVolumes) + }.getOrElse(podWithBasicVolumes) } } 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 1b0e4478a544e..e1c1ab9d459fc 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 @@ -418,14 +418,6 @@ package object config extends Logging { .stringConf .createOptional - private[spark] val INIT_CONTAINER_REMOTE_PYSPARK_FILES = - ConfigBuilder("spark.kubernetes.initcontainer.remotePyFiles") - .doc("Comma-separated list of Python file URIs to download in the init-container. This is" + - " calculated given the list of python files sent to spark-submit.") - .internal() - .stringConf - .createOptional - private[spark] val INIT_CONTAINER_DOCKER_IMAGE = ConfigBuilder("spark.kubernetes.initcontainer.docker.image") .doc("Image for the driver and executor's init-container that downloads dependencies.") diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/Client.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/Client.scala index e5b753d458b34..e61cece142e3d 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/Client.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/Client.scala @@ -16,113 +16,256 @@ */ package org.apache.spark.deploy.kubernetes.submit +import java.io.File import java.util.{Collections, UUID} -import io.fabric8.kubernetes.api.model.{ContainerBuilder, HasMetadata, OwnerReferenceBuilder, PodBuilder} +import io.fabric8.kubernetes.api.model.{ContainerBuilder, EnvVarBuilder, OwnerReferenceBuilder, PodBuilder, QuantityBuilder} import io.fabric8.kubernetes.client.KubernetesClient -import scala.collection.mutable +import scala.collection.JavaConverters._ import org.apache.spark.SparkConf +import org.apache.spark.deploy.kubernetes.{ConfigurationUtils, SparkKubernetesClientFactory} import org.apache.spark.deploy.kubernetes.config._ import org.apache.spark.deploy.kubernetes.constants._ -import org.apache.spark.deploy.kubernetes.SparkKubernetesClientFactory -import org.apache.spark.deploy.kubernetes.submit.submitsteps.{KubernetesDriverSpec, KubernetesSubmissionStep} +import org.apache.spark.deploy.rest.kubernetes.ResourceStagingServerSslOptionsProviderImpl import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.ConfigEntry +import org.apache.spark.launcher.SparkLauncher import org.apache.spark.util.Utils -private[spark] case class ClientArguments( - mainAppResource: MainAppResource, - otherPyFiles: Seq[String], - mainClass: String, - driverArgs: Array[String]) - -private[spark] object ClientArguments { - def fromCommandLineArgs(args: Array[String]): ClientArguments = { - var mainAppResource: Option[MainAppResource] = None - var otherPyFiles = Seq.empty[String] - var mainClass: Option[String] = None - val driverArgs = mutable.Buffer.empty[String] - args.sliding(2).toList.collect { - case Array("--py-file", mainPyFile: String) => - mainAppResource = Some(PythonMainAppResource(mainPyFile)) - case Array("--primary-java-resource", primaryJavaResource: String) => - mainAppResource = Some(JavaMainAppResource(primaryJavaResource)) - case Array("--main-class", clazz: String) => - mainClass = Some(clazz) - case Array("--other-py-files", pyFiles: String) => - otherPyFiles = pyFiles.split(",") - case Array("--arg", arg: String) => - driverArgs += arg - case other => - throw new RuntimeException(s"Unknown arguments: $other") - } - require(mainAppResource.isDefined, - "Main app resource must be defined by either --py-file or --main-java-resource.") - require(mainClass.isDefined, "Main class must be specified via --main-class") - ClientArguments( - mainAppResource.get, - otherPyFiles, - mainClass.get, - driverArgs.toArray) - } -} - +/** + * Submission client for launching Spark applications on Kubernetes clusters. + * + * This class is responsible for instantiating Kubernetes resources that allow a Spark driver to + * run in a pod on the Kubernetes cluster with the Spark configurations specified by spark-submit. + * The API of this class makes it such that much of the specific behavior can be stubbed for + * testing; most of the detailed logic must be dependency-injected when constructing an instance + * of this client. Therefore the submission process is designed to be as modular as possible, + * where different steps of submission should be factored out into separate classes. + */ private[spark] class Client( - submissionSteps: Seq[KubernetesSubmissionStep], - submissionSparkConf: SparkConf, - kubernetesClient: KubernetesClient, - waitForAppCompletion: Boolean, appName: String, + kubernetesResourceNamePrefix: String, + kubernetesAppId: String, + mainAppResource: String, + pythonResource: Option[PythonSubmissionResourcesImpl], + mainClass: String, + sparkConf: SparkConf, + appArgs: Array[String], + waitForAppCompletion: Boolean, + kubernetesClient: KubernetesClient, + initContainerComponentsProvider: DriverInitContainerComponentsProvider, + kubernetesCredentialsMounterProvider: DriverPodKubernetesCredentialsMounterProvider, loggingPodStatusWatcher: LoggingPodStatusWatcher) extends Logging { + private val kubernetesDriverPodName = sparkConf.get(KUBERNETES_DRIVER_POD_NAME) + .getOrElse(s"$kubernetesResourceNamePrefix-driver") + private val driverDockerImage = sparkConf.get(DRIVER_DOCKER_IMAGE) + private val dockerImagePullPolicy = sparkConf.get(DOCKER_IMAGE_PULL_POLICY) + + // CPU settings + private val driverCpuCores = sparkConf.getOption("spark.driver.cores").getOrElse("1") + private val driverLimitCores = sparkConf.getOption(KUBERNETES_DRIVER_LIMIT_CORES.key) + + // Memory settings + private val driverMemoryMb = sparkConf.get(org.apache.spark.internal.config.DRIVER_MEMORY) + private val memoryOverheadMb = sparkConf + .get(KUBERNETES_DRIVER_MEMORY_OVERHEAD) + .getOrElse(math.max((MEMORY_OVERHEAD_FACTOR * driverMemoryMb).toInt, + MEMORY_OVERHEAD_MIN)) + private val driverContainerMemoryWithOverhead = driverMemoryMb + memoryOverheadMb + private val customLabels = sparkConf.get(KUBERNETES_DRIVER_LABELS) + private val customAnnotations = sparkConf.get(KUBERNETES_DRIVER_ANNOTATIONS) - private val driverJavaOptions = submissionSparkConf.get( + private val driverExtraClasspath = sparkConf.get( + org.apache.spark.internal.config.DRIVER_CLASS_PATH) + private val driverJavaOptions = sparkConf.get( org.apache.spark.internal.config.DRIVER_JAVA_OPTIONS) def run(): Unit = { - var currentDriverSpec = new KubernetesDriverSpec( - driverPod = new PodBuilder().build(), - driverContainer = new ContainerBuilder().build(), - driverSparkConf = submissionSparkConf.clone(), - otherKubernetesResources = Seq.empty[HasMetadata]) - for (nextStep <- submissionSteps) { - currentDriverSpec = nextStep.prepareSubmission(currentDriverSpec) + val arguments = (pythonResource map {p => p.arguments}).getOrElse(appArgs) + val driverCustomLabels = ConfigurationUtils.combinePrefixedKeyValuePairsWithDeprecatedConf( + sparkConf, + KUBERNETES_DRIVER_LABEL_PREFIX, + KUBERNETES_DRIVER_LABELS, + "label") + require(!driverCustomLabels.contains(SPARK_APP_ID_LABEL), s"Label with key " + + s" $SPARK_APP_ID_LABEL is not allowed as it is reserved for Spark bookkeeping" + + s" operations.") + + val driverCustomAnnotations = ConfigurationUtils.combinePrefixedKeyValuePairsWithDeprecatedConf( + sparkConf, + KUBERNETES_DRIVER_ANNOTATION_PREFIX, + KUBERNETES_DRIVER_ANNOTATIONS, + "annotation") + require(!driverCustomAnnotations.contains(SPARK_APP_NAME_ANNOTATION), + s"Annotation with key $SPARK_APP_NAME_ANNOTATION is not allowed as it is reserved for" + + s" Spark bookkeeping operations.") + val allDriverLabels = driverCustomLabels ++ Map( + SPARK_APP_ID_LABEL -> kubernetesAppId, + SPARK_ROLE_LABEL -> SPARK_POD_DRIVER_ROLE) + + val driverExtraClasspathEnv = driverExtraClasspath.map { classPath => + new EnvVarBuilder() + .withName(ENV_SUBMIT_EXTRA_CLASSPATH) + .withValue(classPath) + .build() } - val resolvedDriverJavaOpts = currentDriverSpec.driverSparkConf.getAll.map { - case (confKey, confValue) => s"-D$confKey=$confValue" - }.mkString(" ") + driverJavaOptions.map(" " + _).getOrElse("") - val resolvedDriverContainer = new ContainerBuilder(currentDriverSpec.driverContainer) + val driverCpuQuantity = new QuantityBuilder(false) + .withAmount(driverCpuCores) + .build() + val driverMemoryQuantity = new QuantityBuilder(false) + .withAmount(s"${driverMemoryMb}M") + .build() + val driverMemoryLimitQuantity = new QuantityBuilder(false) + .withAmount(s"${driverContainerMemoryWithOverhead}M") + .build() + val driverContainer = new ContainerBuilder() + .withName(DRIVER_CONTAINER_NAME) + .withImage(driverDockerImage) + .withImagePullPolicy(dockerImagePullPolicy) + .addToEnv(driverExtraClasspathEnv.toSeq: _*) .addNewEnv() - .withName(ENV_DRIVER_JAVA_OPTS) - .withValue(resolvedDriverJavaOpts) + .withName(ENV_DRIVER_MEMORY) + .withValue(driverContainerMemoryWithOverhead + "m") .endEnv() + .addNewEnv() + .withName(ENV_DRIVER_MAIN_CLASS) + .withValue(mainClass) + .endEnv() + .addNewEnv() + .withName(ENV_DRIVER_ARGS) + .withValue(arguments.mkString(" ")) + .endEnv() + .withNewResources() + .addToRequests("cpu", driverCpuQuantity) + .addToRequests("memory", driverMemoryQuantity) + .addToLimits("memory", driverMemoryLimitQuantity) + .endResources() .build() - val resolvedDriverPod = new PodBuilder(currentDriverSpec.driverPod) - .editSpec() - .addToContainers(resolvedDriverContainer) + val basePod = new PodBuilder() + .withNewMetadata() + .withName(kubernetesDriverPodName) + .addToLabels(allDriverLabels.asJava) + .addToAnnotations(driverCustomAnnotations.toMap.asJava) + .addToAnnotations(SPARK_APP_NAME_ANNOTATION, appName) + .endMetadata() + .withNewSpec() + .withRestartPolicy("Never") + .addToContainers(driverContainer) .endSpec() - .build() + + driverLimitCores.map { + limitCores => + val driverCpuLimitQuantity = new QuantityBuilder(false) + .withAmount(limitCores) + .build() + basePod + .editSpec() + .editFirstContainer() + .editResources + .addToLimits("cpu", driverCpuLimitQuantity) + .endResources() + .endContainer() + .endSpec() + } + + val maybeSubmittedResourceIdentifiers = initContainerComponentsProvider + .provideInitContainerSubmittedDependencyUploader(allDriverLabels) + .map { uploader => + SubmittedResources(uploader.uploadJars(), uploader.uploadFiles()) + } + val maybeSubmittedDependenciesSecret = initContainerComponentsProvider + .provideSubmittedDependenciesSecretBuilder( + maybeSubmittedResourceIdentifiers.map(_.secrets())) + .map(_.build()) + + val containerLocalizedFilesResolver = initContainerComponentsProvider + .provideContainerLocalizedFilesResolver(mainAppResource) + val resolvedSparkJars = containerLocalizedFilesResolver.resolveSubmittedSparkJars() + val resolvedSparkFiles = containerLocalizedFilesResolver.resolveSubmittedSparkFiles() + val resolvedPySparkFiles = containerLocalizedFilesResolver.resolveSubmittedPySparkFiles() + val resolvedPrimaryPySparkResource = pythonResource.map { + p => p.primaryPySparkResource(containerLocalizedFilesResolver) + }.getOrElse("") + val initContainerBundler = initContainerComponentsProvider + .provideInitContainerBundle(maybeSubmittedResourceIdentifiers.map(_.ids()), + resolvedSparkJars ++ resolvedSparkFiles) + + val podWithInitContainer = initContainerBundler.map( + _.sparkPodInitContainerBootstrap + .bootstrapInitContainerAndVolumes(driverContainer.getName, basePod)) + .getOrElse(basePod) + val sparkConfWithExecutorInit = initContainerBundler.map( + _.executorInitContainerConfiguration + .configureSparkConfForExecutorInitContainer(sparkConf)) + .getOrElse(sparkConf) + val credentialsMounter = kubernetesCredentialsMounterProvider + .getDriverPodKubernetesCredentialsMounter() + val credentialsSecret = credentialsMounter.createCredentialsSecret() + val podWithInitContainerAndMountedCreds = credentialsMounter.mountDriverKubernetesCredentials( + podWithInitContainer, driverContainer.getName, credentialsSecret) + val resolvedSparkConf = credentialsMounter.setDriverPodKubernetesCredentialLocations( + sparkConfWithExecutorInit) + if (resolvedSparkJars.nonEmpty) { + resolvedSparkConf.set("spark.jars", resolvedSparkJars.mkString(",")) + } + if (resolvedSparkFiles.nonEmpty) { + resolvedSparkConf.set("spark.files", resolvedSparkFiles.mkString(",")) + } + resolvedSparkConf.setIfMissing(KUBERNETES_DRIVER_POD_NAME, kubernetesDriverPodName) + resolvedSparkConf.set("spark.app.id", kubernetesAppId) + resolvedSparkConf.set(KUBERNETES_EXECUTOR_POD_NAME_PREFIX, kubernetesResourceNamePrefix) + // We don't need this anymore since we just set the JVM options on the environment + resolvedSparkConf.remove(org.apache.spark.internal.config.DRIVER_JAVA_OPTIONS) + val resolvedLocalClasspath = containerLocalizedFilesResolver + .resolveSubmittedAndRemoteSparkJars() + val resolvedDriverJavaOpts = resolvedSparkConf.getAll.map { + case (confKey, confValue) => s"-D$confKey=$confValue" + }.mkString(" ") + driverJavaOptions.map(" " + _).getOrElse("") + val resolvedDriverPodBuilder = podWithInitContainerAndMountedCreds.editSpec() + .editMatchingContainer(new ContainerNameEqualityPredicate(driverContainer.getName)) + .addNewEnv() + .withName(ENV_MOUNTED_CLASSPATH) + .withValue(resolvedLocalClasspath.mkString(File.pathSeparator)) + .endEnv() + .addNewEnv() + .withName(ENV_DRIVER_JAVA_OPTS) + .withValue(resolvedDriverJavaOpts) + .endEnv() + .endContainer() + .endSpec() + val driverPodFileMounter = initContainerComponentsProvider.provideDriverPodFileMounter() + val resolvedDriverPod = pythonResource.map { + p => p.driverPodWithPySparkEnvs( + driverPodFileMounter, + resolvedPrimaryPySparkResource, + resolvedPySparkFiles.mkString(","), + driverContainer.getName, + resolvedDriverPodBuilder + )}.getOrElse(resolvedDriverPodBuilder.build()) Utils.tryWithResource( - kubernetesClient - .pods() - .withName(resolvedDriverPod.getMetadata.getName) - .watch(loggingPodStatusWatcher)) { _ => + kubernetesClient + .pods() + .withName(resolvedDriverPod.getMetadata.getName) + .watch(loggingPodStatusWatcher)) { _ => val createdDriverPod = kubernetesClient.pods().create(resolvedDriverPod) try { - if (currentDriverSpec.otherKubernetesResources.nonEmpty) { - val driverPodOwnerReference = new OwnerReferenceBuilder() - .withName(createdDriverPod.getMetadata.getName) - .withApiVersion(createdDriverPod.getApiVersion) - .withUid(createdDriverPod.getMetadata.getUid) - .withKind(createdDriverPod.getKind) - .withController(true) - .build() - currentDriverSpec.otherKubernetesResources.foreach { resource => - val originalMetadata = resource.getMetadata - originalMetadata.setOwnerReferences(Collections.singletonList(driverPodOwnerReference)) - } - val otherKubernetesResources = currentDriverSpec.otherKubernetesResources - kubernetesClient.resourceList(otherKubernetesResources: _*).createOrReplace() + val driverOwnedResources = initContainerBundler.map( + _.sparkInitContainerConfigMap).toSeq ++ + maybeSubmittedDependenciesSecret.toSeq ++ + credentialsSecret.toSeq + val driverPodOwnerReference = new OwnerReferenceBuilder() + .withName(createdDriverPod.getMetadata.getName) + .withApiVersion(createdDriverPod.getApiVersion) + .withUid(createdDriverPod.getMetadata.getUid) + .withKind(createdDriverPod.getKind) + .withController(true) + .build() + driverOwnedResources.foreach { resource => + val originalMetadata = resource.getMetadata + originalMetadata.setOwnerReferences(Collections.singletonList(driverPodOwnerReference)) } + kubernetesClient.resourceList(driverOwnedResources: _*).createOrReplace() } catch { case e: Throwable => kubernetesClient.pods().delete(createdDriverPod) @@ -140,47 +283,100 @@ private[spark] class Client( } private[spark] object Client { - def run(sparkConf: SparkConf, clientArguments: ClientArguments): Unit = { - val namespace = sparkConf.get(KUBERNETES_NAMESPACE) - val launchTime = System.currentTimeMillis() + def main(args: Array[String]): Unit = { + val sparkConf = new SparkConf(true) + val mainAppResource = args(0) + val mainClass = args(1) + val appArgs = args.drop(2) + run(sparkConf, mainAppResource, mainClass, appArgs) + } + def run( + sparkConf: SparkConf, + mainAppResource: String, + mainClass: String, + appArgs: Array[String]): Unit = { + val isPython = mainAppResource.endsWith(".py") + val pythonResource: Option[PythonSubmissionResourcesImpl] = + if (isPython) { + Option(new PythonSubmissionResourcesImpl(mainAppResource, appArgs)) + } else None + // Since you might need jars for SQL UDFs in PySpark + def sparkJarFilter() : Seq[String] = + pythonResource.map { p => p.sparkJars}.getOrElse( + Option(mainAppResource) + .filterNot(_ == SparkLauncher.NO_RESOURCE) + .toSeq) + val sparkJars = sparkConf.getOption("spark.jars") + .map(_.split(",")) + .getOrElse(Array.empty[String]) ++ sparkJarFilter() + val launchTime = System.currentTimeMillis + val sparkFiles = sparkConf.getOption("spark.files") + .map(_.split(",")) + .getOrElse(Array.empty[String]) + val pySparkFilesOption = pythonResource.map {p => p.pySparkFiles} + validateNoDuplicateFileNames(sparkJars) + validateNoDuplicateFileNames(sparkFiles) + pySparkFilesOption.foreach {b => validateNoDuplicateFileNames(b)} + val pySparkFiles = pySparkFilesOption.getOrElse(Array.empty[String]) val appName = sparkConf.getOption("spark.app.name").getOrElse("spark") + // The resource name prefix is derived from the application name, making it easy to connect the + // names of the Kubernetes resources from e.g. Kubectl or the Kubernetes dashboard to the + // application the user submitted. However, we can't use the application name in the label, as + // label values are considerably restrictive, e.g. must be no longer than 63 characters in + // length. So we generate a separate identifier for the app ID itself, and bookkeeping that + // requires finding "all pods for this application" should use the kubernetesAppId. + val kubernetesResourceNamePrefix = s"$appName-$launchTime".toLowerCase.replaceAll("\\.", "-") val kubernetesAppId = s"spark-${UUID.randomUUID().toString.replaceAll("-", "")}" + val namespace = sparkConf.get(KUBERNETES_NAMESPACE) val master = resolveK8sMaster(sparkConf.get("spark.master")) - val submissionStepsOrchestrator = new KubernetesSubmissionStepsOrchestrator( - namespace, - kubernetesAppId, - launchTime, - clientArguments.mainAppResource, - appName, - clientArguments.mainClass, - clientArguments.driverArgs, - clientArguments.otherPyFiles, - sparkConf) - val waitForAppCompletion = sparkConf.get(WAIT_FOR_APP_COMPLETION) - val loggingInterval = Option(sparkConf.get(REPORT_INTERVAL)) - .filter( _ => waitForAppCompletion) - val loggingPodStatusWatcher = new LoggingPodStatusWatcherImpl( - kubernetesAppId, loggingInterval) + val sslOptionsProvider = new ResourceStagingServerSslOptionsProviderImpl(sparkConf) + val initContainerComponentsProvider = new DriverInitContainerComponentsProviderImpl( + sparkConf, + kubernetesResourceNamePrefix, + namespace, + sparkJars, + sparkFiles, + pySparkFiles, + sslOptionsProvider.getSslOptions) Utils.tryWithResource(SparkKubernetesClientFactory.createKubernetesClient( - master, - Some(namespace), - APISERVER_AUTH_SUBMISSION_CONF_PREFIX, - sparkConf, - None, - None)) { kubernetesClient => - new Client( - submissionStepsOrchestrator.getAllSubmissionSteps(), + master, + Some(namespace), + APISERVER_AUTH_SUBMISSION_CONF_PREFIX, sparkConf, - kubernetesClient, - waitForAppCompletion, - appName, - loggingPodStatusWatcher).run() + None, + None)) { kubernetesClient => + val kubernetesCredentialsMounterProvider = + new DriverPodKubernetesCredentialsMounterProviderImpl( + sparkConf, kubernetesResourceNamePrefix) + val waitForAppCompletion = sparkConf.get(WAIT_FOR_APP_COMPLETION) + val loggingInterval = Option(sparkConf.get(REPORT_INTERVAL)) + .filter( _ => waitForAppCompletion) + val loggingPodStatusWatcher = new LoggingPodStatusWatcherImpl( + kubernetesResourceNamePrefix, loggingInterval) + new Client( + appName, + kubernetesResourceNamePrefix, + kubernetesAppId, + mainAppResource, + pythonResource, + mainClass, + sparkConf, + appArgs, + waitForAppCompletion, + kubernetesClient, + initContainerComponentsProvider, + kubernetesCredentialsMounterProvider, + loggingPodStatusWatcher).run() } } - - def main(args: Array[String]): Unit = { - val parsedArguments = ClientArguments.fromCommandLineArgs(args) - val sparkConf = new SparkConf() - run(sparkConf, parsedArguments) + private def validateNoDuplicateFileNames(allFiles: Seq[String]): Unit = { + val fileNamesToUris = allFiles.map { file => + (new File(Utils.resolveURI(file).getPath).getName, file) + } + fileNamesToUris.groupBy(_._1).foreach { + case (fileName, urisWithFileName) => + require(urisWithFileName.size == 1, "Cannot add multiple files with the same name, but" + + s" file name $fileName is shared by all of these URIs: $urisWithFileName") + } } } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/ContainerLocalizedFilesResolver.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/ContainerLocalizedFilesResolver.scala new file mode 100644 index 0000000000000..7345589c59c42 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/ContainerLocalizedFilesResolver.scala @@ -0,0 +1,87 @@ +/* + * 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.submit + +import java.io.File + +import org.apache.spark.util.Utils + +private[spark] trait ContainerLocalizedFilesResolver { + def resolveSubmittedAndRemoteSparkJars(): Seq[String] + def resolveSubmittedSparkJars(): Seq[String] + def resolveSubmittedSparkFiles(): Seq[String] + def resolveSubmittedPySparkFiles(): Seq[String] + def resolvePrimaryResourceFile(): String +} + +private[spark] class ContainerLocalizedFilesResolverImpl( + sparkJars: Seq[String], + sparkFiles: Seq[String], + pySparkFiles: Seq[String], + primaryPyFile: String, + jarsDownloadPath: String, + filesDownloadPath: String ) extends ContainerLocalizedFilesResolver { + + + override def resolveSubmittedAndRemoteSparkJars(): Seq[String] = { + sparkJars.map { jar => + val jarUri = Utils.resolveURI(jar) + Option(jarUri.getScheme).getOrElse("file") match { + case "local" => + jarUri.getPath + case _ => + val jarFileName = new File(jarUri.getPath).getName + s"$jarsDownloadPath/$jarFileName" + } + } + } + + override def resolveSubmittedSparkJars(): Seq[String] = { + resolveSubmittedFiles(sparkJars, jarsDownloadPath) + } + + override def resolveSubmittedSparkFiles(): Seq[String] = { + resolveSubmittedFiles(sparkFiles, filesDownloadPath) + } + + override def resolveSubmittedPySparkFiles(): Seq[String] = { + def filterMainResource(x: String) = x match { + case `primaryPyFile` => None + case _ => Some(resolveFile(x, filesDownloadPath)) + } + pySparkFiles.flatMap(x => filterMainResource(x)) + } + + override def resolvePrimaryResourceFile(): String = { + Option(primaryPyFile).map(p => resolveFile(p, filesDownloadPath)).getOrElse("") + } + + private def resolveFile(file: String, downloadPath: String) = { + val fileUri = Utils.resolveURI(file) + Option(fileUri.getScheme).getOrElse("file") match { + case "file" => + val fileName = new File(fileUri.getPath).getName + s"$downloadPath/$fileName" + case _ => + file + } + } + + private def resolveSubmittedFiles(files: Seq[String], downloadPath: String): Seq[String] = { + files.map { file => resolveFile(file, downloadPath) } + } +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/DriverInitContainerComponentsProvider.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/DriverInitContainerComponentsProvider.scala new file mode 100644 index 0000000000000..7f5a62e97abc0 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/DriverInitContainerComponentsProvider.scala @@ -0,0 +1,223 @@ +/* + * 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.submit + +import io.fabric8.kubernetes.api.model.ConfigMap + +import org.apache.spark.{SparkConf, SSLOptions} +import org.apache.spark.deploy.kubernetes.{InitContainerResourceStagingServerSecretPluginImpl, OptionRequirements, SparkPodInitContainerBootstrap, SparkPodInitContainerBootstrapImpl} +import org.apache.spark.deploy.kubernetes.config._ +import org.apache.spark.deploy.kubernetes.constants._ +import org.apache.spark.deploy.rest.kubernetes.RetrofitClientFactoryImpl +import org.apache.spark.util.Utils + +/** + * Interface that wraps the provision of everything the submission client needs to set up the + * driver's init-container. This is all wrapped in the same place to ensure that related + * components are being constructed with consistent configurations with respect to one another. + */ +private[spark] trait DriverInitContainerComponentsProvider { + + def provideContainerLocalizedFilesResolver( + mainAppResource: String) : ContainerLocalizedFilesResolver + def provideInitContainerSubmittedDependencyUploader( + driverPodLabels: Map[String, String]): Option[SubmittedDependencyUploader] + def provideSubmittedDependenciesSecretBuilder( + maybeSubmittedResourceSecrets: Option[SubmittedResourceSecrets]) + : Option[SubmittedDependencySecretBuilder] + def provideInitContainerBootstrap(): SparkPodInitContainerBootstrap + def provideDriverPodFileMounter(): DriverPodKubernetesFileMounter + def provideInitContainerBundle(maybeSubmittedResourceIds: Option[SubmittedResourceIds], + uris: Iterable[String]): Option[InitContainerBundle] +} + +private[spark] class DriverInitContainerComponentsProviderImpl( + sparkConf: SparkConf, + kubernetesResourceNamePrefix: String, + namespace: String, + sparkJars: Seq[String], + sparkFiles: Seq[String], + pySparkFiles: Seq[String], + resourceStagingServerExternalSslOptions: SSLOptions) + extends DriverInitContainerComponentsProvider { + + private val maybeResourceStagingServerUri = sparkConf.get(RESOURCE_STAGING_SERVER_URI) + private val maybeResourceStagingServerInternalUri = + sparkConf.get(RESOURCE_STAGING_SERVER_INTERNAL_URI) + private val maybeResourceStagingServerInternalTrustStore = + sparkConf.get(RESOURCE_STAGING_SERVER_INTERNAL_TRUSTSTORE_FILE) + .orElse(sparkConf.get(RESOURCE_STAGING_SERVER_TRUSTSTORE_FILE)) + private val maybeResourceStagingServerInternalTrustStorePassword = + sparkConf.get(RESOURCE_STAGING_SERVER_INTERNAL_TRUSTSTORE_PASSWORD) + .orElse(sparkConf.get(RESOURCE_STAGING_SERVER_TRUSTSTORE_PASSWORD)) + private val maybeResourceStagingServerInternalTrustStoreType = + sparkConf.get(RESOURCE_STAGING_SERVER_INTERNAL_TRUSTSTORE_TYPE) + .orElse(sparkConf.get(RESOURCE_STAGING_SERVER_TRUSTSTORE_TYPE)) + private val maybeResourceStagingServerInternalClientCert = + sparkConf.get(RESOURCE_STAGING_SERVER_INTERNAL_CLIENT_CERT_PEM) + .orElse(sparkConf.get(RESOURCE_STAGING_SERVER_CLIENT_CERT_PEM)) + private val resourceStagingServerInternalSslEnabled = + sparkConf.get(RESOURCE_STAGING_SERVER_INTERNAL_SSL_ENABLED) + .orElse(sparkConf.get(RESOURCE_STAGING_SERVER_SSL_ENABLED)) + .getOrElse(false) + + OptionRequirements.requireNandDefined( + maybeResourceStagingServerInternalClientCert, + maybeResourceStagingServerInternalTrustStore, + "Cannot provide both a certificate file and a trustStore file for init-containers to" + + " use for contacting the resource staging server over TLS.") + + require(maybeResourceStagingServerInternalTrustStore.forall { trustStore => + Option(Utils.resolveURI(trustStore).getScheme).getOrElse("file") match { + case "file" | "local" => true + case _ => false + } + }, "TrustStore URI used for contacting the resource staging server from init containers must" + + " have no scheme, or scheme file://, or scheme local://.") + + require(maybeResourceStagingServerInternalClientCert.forall { trustStore => + Option(Utils.resolveURI(trustStore).getScheme).getOrElse("file") match { + case "file" | "local" => true + case _ => false + } + }, "Client cert file URI used for contacting the resource staging server from init containers" + + " must have no scheme, or scheme file://, or scheme local://.") + + private val jarsDownloadPath = sparkConf.get(INIT_CONTAINER_JARS_DOWNLOAD_LOCATION) + private val filesDownloadPath = sparkConf.get(INIT_CONTAINER_FILES_DOWNLOAD_LOCATION) + private val maybeSecretName = maybeResourceStagingServerUri.map { _ => + s"$kubernetesResourceNamePrefix-init-secret" + } + private val configMapName = s"$kubernetesResourceNamePrefix-init-config" + private val configMapKey = s"$kubernetesResourceNamePrefix-init-config-key" + private val initContainerImage = sparkConf.get(INIT_CONTAINER_DOCKER_IMAGE) + private val dockerImagePullPolicy = sparkConf.get(DOCKER_IMAGE_PULL_POLICY) + private val downloadTimeoutMinutes = sparkConf.get(INIT_CONTAINER_MOUNT_TIMEOUT) + private val pySparkSubmitted = KubernetesFileUtils.getOnlySubmitterLocalFiles(pySparkFiles) + + private def provideInitContainerConfigMap( + maybeSubmittedResourceIds: Option[SubmittedResourceIds]): ConfigMap = { + val submittedDependencyConfigPlugin = for { + stagingServerUri <- maybeResourceStagingServerUri + jarsResourceId <- maybeSubmittedResourceIds.map(_.jarsResourceId) + filesResourceId <- maybeSubmittedResourceIds.map(_.filesResourceId) + } yield { + new SubmittedDependencyInitContainerConfigPluginImpl( + // Configure the init-container with the internal URI over the external URI. + maybeResourceStagingServerInternalUri.getOrElse(stagingServerUri), + jarsResourceId, + filesResourceId, + INIT_CONTAINER_SUBMITTED_JARS_SECRET_KEY, + INIT_CONTAINER_SUBMITTED_FILES_SECRET_KEY, + INIT_CONTAINER_STAGING_SERVER_TRUSTSTORE_SECRET_KEY, + INIT_CONTAINER_STAGING_SERVER_CLIENT_CERT_SECRET_KEY, + resourceStagingServerInternalSslEnabled, + maybeResourceStagingServerInternalTrustStore, + maybeResourceStagingServerInternalClientCert, + maybeResourceStagingServerInternalTrustStorePassword, + maybeResourceStagingServerInternalTrustStoreType, + INIT_CONTAINER_SECRET_VOLUME_MOUNT_PATH) + } + new SparkInitContainerConfigMapBuilderImpl( + sparkJars, + sparkFiles ++ pySparkSubmitted, + jarsDownloadPath, + filesDownloadPath, + configMapName, + configMapKey, + submittedDependencyConfigPlugin).build() + } + + override def provideContainerLocalizedFilesResolver(mainAppResource: String) + : ContainerLocalizedFilesResolver = { + new ContainerLocalizedFilesResolverImpl( + sparkJars, sparkFiles, pySparkFiles, mainAppResource, jarsDownloadPath, filesDownloadPath) + } + + private def provideExecutorInitContainerConfiguration(): ExecutorInitContainerConfiguration = { + new ExecutorInitContainerConfigurationImpl( + maybeSecretName, + INIT_CONTAINER_SECRET_VOLUME_MOUNT_PATH, + configMapName, + configMapKey) + } + + override def provideInitContainerSubmittedDependencyUploader( + driverPodLabels: Map[String, String]): Option[SubmittedDependencyUploader] = { + maybeResourceStagingServerUri.map { stagingServerUri => + new SubmittedDependencyUploaderImpl( + driverPodLabels, + namespace, + stagingServerUri, + sparkJars, + sparkFiles ++ pySparkSubmitted, + resourceStagingServerExternalSslOptions, + RetrofitClientFactoryImpl) + } + } + + override def provideSubmittedDependenciesSecretBuilder( + maybeSubmittedResourceSecrets: Option[SubmittedResourceSecrets]) + : Option[SubmittedDependencySecretBuilder] = { + for { + secretName <- maybeSecretName + jarsResourceSecret <- maybeSubmittedResourceSecrets.map(_.jarsResourceSecret) + filesResourceSecret <- maybeSubmittedResourceSecrets.map(_.filesResourceSecret) + } yield { + new SubmittedDependencySecretBuilderImpl( + secretName, + jarsResourceSecret, + filesResourceSecret, + INIT_CONTAINER_SUBMITTED_JARS_SECRET_KEY, + INIT_CONTAINER_SUBMITTED_FILES_SECRET_KEY, + INIT_CONTAINER_STAGING_SERVER_TRUSTSTORE_SECRET_KEY, + INIT_CONTAINER_STAGING_SERVER_CLIENT_CERT_SECRET_KEY, + maybeResourceStagingServerInternalTrustStore, + maybeResourceStagingServerInternalClientCert) + } + } + + override def provideInitContainerBootstrap(): SparkPodInitContainerBootstrap = { + val resourceStagingServerSecretPlugin = maybeSecretName.map { secret => + new InitContainerResourceStagingServerSecretPluginImpl( + secret, INIT_CONTAINER_SECRET_VOLUME_MOUNT_PATH) + } + new SparkPodInitContainerBootstrapImpl( + initContainerImage, + dockerImagePullPolicy, + jarsDownloadPath, + filesDownloadPath, + downloadTimeoutMinutes, + configMapName, + configMapKey, + resourceStagingServerSecretPlugin) + } + override def provideDriverPodFileMounter(): DriverPodKubernetesFileMounter = { + new DriverPodKubernetesFileMounterImpl() + } + override def provideInitContainerBundle( + maybeSubmittedResourceIds: Option[SubmittedResourceIds], + uris: Iterable[String]): Option[InitContainerBundle] = { + // Bypass init-containers if `spark.jars` and `spark.files` and '--py-rilfes' + // is empty or only has `local://` URIs + if ((KubernetesFileUtils.getNonContainerLocalFiles(uris) ++ pySparkSubmitted).nonEmpty) { + Some(InitContainerBundle(provideInitContainerConfigMap(maybeSubmittedResourceIds), + provideInitContainerBootstrap(), + provideExecutorInitContainerConfiguration())) + } else None + } +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/DriverPodKubernetesCredentialsMounter.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/DriverPodKubernetesCredentialsMounter.scala new file mode 100644 index 0000000000000..25e7c3b3ebd89 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/DriverPodKubernetesCredentialsMounter.scala @@ -0,0 +1,184 @@ +/* + * 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.submit + +import io.fabric8.kubernetes.api.model.{PodBuilder, Secret, SecretBuilder} +import scala.collection.JavaConverters._ +import scala.language.implicitConversions + +import org.apache.spark.SparkConf +import org.apache.spark.deploy.kubernetes.KubernetesCredentials +import org.apache.spark.deploy.kubernetes.config._ +import org.apache.spark.deploy.kubernetes.constants._ + +private[spark] trait DriverPodKubernetesCredentialsMounter { + + /** + * Set fields on the Spark configuration that indicate where the driver pod is + * to find its Kubernetes credentials for requesting executors. + */ + def setDriverPodKubernetesCredentialLocations(sparkConf: SparkConf): SparkConf + + /** + * Create the Kubernetes secret object that correspond to the driver's credentials + * that have to be created and mounted into the driver pod. The single Secret + * object contains all of the data entries for the driver pod's Kubernetes + * credentials. Returns empty if no secrets are to be mounted. + */ + def createCredentialsSecret(): Option[Secret] + + /** + * Mount any Kubernetes credentials from the submitting machine's disk into the driver pod. The + * secret that is passed in here should have been created from createCredentialsSecret so that + * the implementation does not need to hold its state. + */ + def mountDriverKubernetesCredentials( + originalPodSpec: PodBuilder, + driverContainerName: String, + credentialsSecret: Option[Secret]): PodBuilder +} + +private[spark] class DriverPodKubernetesCredentialsMounterImpl( + kubernetesAppId: String, + submitterLocalDriverPodKubernetesCredentials: KubernetesCredentials, + maybeUserSpecifiedMountedClientKeyFile: Option[String], + maybeUserSpecifiedMountedClientCertFile: Option[String], + maybeUserSpecifiedMountedOAuthTokenFile: Option[String], + maybeUserSpecifiedMountedCaCertFile: Option[String]) + extends DriverPodKubernetesCredentialsMounter { + + override def setDriverPodKubernetesCredentialLocations(sparkConf: SparkConf): SparkConf = { + val resolvedMountedClientKeyFile = resolveSecretLocation( + maybeUserSpecifiedMountedClientKeyFile, + submitterLocalDriverPodKubernetesCredentials.clientKeyDataBase64, + DRIVER_CREDENTIALS_CLIENT_KEY_PATH) + val resolvedMountedClientCertFile = resolveSecretLocation( + maybeUserSpecifiedMountedClientCertFile, + submitterLocalDriverPodKubernetesCredentials.clientCertDataBase64, + DRIVER_CREDENTIALS_CLIENT_CERT_PATH) + val resolvedMountedCaCertFile = resolveSecretLocation( + maybeUserSpecifiedMountedCaCertFile, + submitterLocalDriverPodKubernetesCredentials.caCertDataBase64, + DRIVER_CREDENTIALS_CA_CERT_PATH) + val resolvedMountedOAuthTokenFile = resolveSecretLocation( + maybeUserSpecifiedMountedOAuthTokenFile, + submitterLocalDriverPodKubernetesCredentials.oauthTokenBase64, + DRIVER_CREDENTIALS_OAUTH_TOKEN_PATH) + val sparkConfWithCredentialLocations = sparkConf.clone() + .setOption( + s"$APISERVER_AUTH_DRIVER_MOUNTED_CONF_PREFIX.$CA_CERT_FILE_CONF_SUFFIX", + resolvedMountedCaCertFile) + .setOption( + s"$APISERVER_AUTH_DRIVER_MOUNTED_CONF_PREFIX.$CLIENT_KEY_FILE_CONF_SUFFIX", + resolvedMountedClientKeyFile) + .setOption( + s"$APISERVER_AUTH_DRIVER_MOUNTED_CONF_PREFIX.$CLIENT_CERT_FILE_CONF_SUFFIX", + resolvedMountedClientCertFile) + .setOption( + s"$APISERVER_AUTH_DRIVER_MOUNTED_CONF_PREFIX.$OAUTH_TOKEN_FILE_CONF_SUFFIX", + resolvedMountedOAuthTokenFile) + // Redact all OAuth token values + sparkConfWithCredentialLocations + .getAll + .filter(_._1.endsWith(OAUTH_TOKEN_CONF_SUFFIX)).map(_._1) + .foreach { + sparkConfWithCredentialLocations.set(_, "") + } + sparkConfWithCredentialLocations + } + + override def createCredentialsSecret(): Option[Secret] = { + val allSecretData = + resolveSecretData( + maybeUserSpecifiedMountedClientKeyFile, + submitterLocalDriverPodKubernetesCredentials.clientKeyDataBase64, + DRIVER_CREDENTIALS_CLIENT_KEY_SECRET_NAME) ++ + resolveSecretData( + maybeUserSpecifiedMountedClientCertFile, + submitterLocalDriverPodKubernetesCredentials.clientCertDataBase64, + DRIVER_CREDENTIALS_CLIENT_CERT_SECRET_NAME) ++ + resolveSecretData( + maybeUserSpecifiedMountedCaCertFile, + submitterLocalDriverPodKubernetesCredentials.caCertDataBase64, + DRIVER_CREDENTIALS_CA_CERT_SECRET_NAME) ++ + resolveSecretData( + maybeUserSpecifiedMountedOAuthTokenFile, + submitterLocalDriverPodKubernetesCredentials.oauthTokenBase64, + DRIVER_CREDENTIALS_OAUTH_TOKEN_SECRET_NAME) + if (allSecretData.isEmpty) { + None + } else { + Some(new SecretBuilder() + .withNewMetadata().withName(s"$kubernetesAppId-kubernetes-credentials").endMetadata() + .withData(allSecretData.asJava) + .build()) + } + } + + override def mountDriverKubernetesCredentials( + originalPodSpec: PodBuilder, + driverContainerName: String, + credentialsSecret: Option[Secret]): PodBuilder = { + credentialsSecret.map { secret => + originalPodSpec.editSpec() + .addNewVolume() + .withName(DRIVER_CREDENTIALS_SECRET_VOLUME_NAME) + .withNewSecret().withSecretName(secret.getMetadata.getName).endSecret() + .endVolume() + .editMatchingContainer(new ContainerNameEqualityPredicate(driverContainerName)) + .addNewVolumeMount() + .withName(DRIVER_CREDENTIALS_SECRET_VOLUME_NAME) + .withMountPath(DRIVER_CREDENTIALS_SECRETS_BASE_DIR) + .endVolumeMount() + .endContainer() + .endSpec() + }.getOrElse(originalPodSpec) + } + + private def resolveSecretLocation( + mountedUserSpecified: Option[String], + valueMountedFromSubmitter: Option[String], + mountedCanonicalLocation: String): Option[String] = { + mountedUserSpecified.orElse(valueMountedFromSubmitter.map( _ => { + mountedCanonicalLocation + })) + } + + private def resolveSecretData( + mountedUserSpecified: Option[String], + valueMountedFromSubmitter: Option[String], + secretName: String): Map[String, String] = { + mountedUserSpecified.map { _ => Map.empty[String, String]} + .getOrElse { + valueMountedFromSubmitter.map { valueBase64 => + Map(secretName -> valueBase64) + }.getOrElse(Map.empty[String, String]) + } + } + + private implicit def augmentSparkConf(sparkConf: SparkConf): OptionSettableSparkConf = { + new OptionSettableSparkConf(sparkConf) + } +} + +private class OptionSettableSparkConf(sparkConf: SparkConf) { + def setOption(configEntry: String, option: Option[String]): SparkConf = { + option.map( opt => { + sparkConf.set(configEntry, opt) + }).getOrElse(sparkConf) + } +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/DriverPodKubernetesCredentialsMounterProvider.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/DriverPodKubernetesCredentialsMounterProvider.scala new file mode 100644 index 0000000000000..913279198146a --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/DriverPodKubernetesCredentialsMounterProvider.scala @@ -0,0 +1,49 @@ +/* + * 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.submit + +import org.apache.spark.SparkConf +import org.apache.spark.deploy.kubernetes.config._ + +private[spark] trait DriverPodKubernetesCredentialsMounterProvider { + + def getDriverPodKubernetesCredentialsMounter() + : DriverPodKubernetesCredentialsMounter +} + +private[spark] class DriverPodKubernetesCredentialsMounterProviderImpl( + sparkConf: SparkConf, + kubernetesAppId: String) + extends DriverPodKubernetesCredentialsMounterProvider { + + override def getDriverPodKubernetesCredentialsMounter() + : DriverPodKubernetesCredentialsMounter = { + val submitterLocalDriverPodKubernetesCredentials = + new DriverPodKubernetesCredentialsProvider(sparkConf).get() + new DriverPodKubernetesCredentialsMounterImpl( + kubernetesAppId, + submitterLocalDriverPodKubernetesCredentials, + sparkConf.getOption( + s"$APISERVER_AUTH_DRIVER_MOUNTED_CONF_PREFIX.$CLIENT_KEY_FILE_CONF_SUFFIX"), + sparkConf.getOption( + s"$APISERVER_AUTH_DRIVER_MOUNTED_CONF_PREFIX.$CLIENT_CERT_FILE_CONF_SUFFIX"), + sparkConf.getOption( + s"$APISERVER_AUTH_DRIVER_MOUNTED_CONF_PREFIX.$OAUTH_TOKEN_FILE_CONF_SUFFIX"), + sparkConf.getOption( + s"$APISERVER_AUTH_DRIVER_MOUNTED_CONF_PREFIX.$CA_CERT_FILE_CONF_SUFFIX")) + } +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/DriverPodKubernetesCredentialsProvider.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/DriverPodKubernetesCredentialsProvider.scala new file mode 100644 index 0000000000000..41b0cf8ceaeab --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/DriverPodKubernetesCredentialsProvider.scala @@ -0,0 +1,63 @@ +/* + * 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.submit + +import java.io.File + +import com.google.common.base.Charsets +import com.google.common.io.{BaseEncoding, Files} + +import org.apache.spark.SparkConf +import org.apache.spark.deploy.kubernetes.KubernetesCredentials +import org.apache.spark.deploy.kubernetes.config._ +import org.apache.spark.internal.config.OptionalConfigEntry + +private[spark] class DriverPodKubernetesCredentialsProvider(sparkConf: SparkConf) { + + def get(): KubernetesCredentials = { + val oauthTokenBase64 = sparkConf + .getOption(s"$APISERVER_AUTH_DRIVER_CONF_PREFIX.$OAUTH_TOKEN_CONF_SUFFIX") + .map { token => + BaseEncoding.base64().encode(token.getBytes(Charsets.UTF_8)) + } + val caCertDataBase64 = safeFileConfToBase64( + s"$APISERVER_AUTH_DRIVER_CONF_PREFIX.$CA_CERT_FILE_CONF_SUFFIX", + s"Driver CA cert file provided at %s does not exist or is not a file.") + val clientKeyDataBase64 = safeFileConfToBase64( + s"$APISERVER_AUTH_DRIVER_CONF_PREFIX.$CLIENT_KEY_FILE_CONF_SUFFIX", + s"Driver client key file provided at %s does not exist or is not a file.") + val clientCertDataBase64 = safeFileConfToBase64( + s"$APISERVER_AUTH_DRIVER_CONF_PREFIX.$CLIENT_CERT_FILE_CONF_SUFFIX", + s"Driver client cert file provided at %s does not exist or is not a file.") + KubernetesCredentials( + oauthTokenBase64 = oauthTokenBase64, + caCertDataBase64 = caCertDataBase64, + clientKeyDataBase64 = clientKeyDataBase64, + clientCertDataBase64 = clientCertDataBase64) + } + + private def safeFileConfToBase64( + conf: String, + fileNotFoundFormatString: String): Option[String] = { + sparkConf.getOption(conf) + .map(new File(_)) + .map { file => + require(file.isFile, String.format(fileNotFoundFormatString, file.getAbsolutePath)) + BaseEncoding.base64().encode(Files.toByteArray(file)) + } + } +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/DriverPodKubernetesFileMounter.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/DriverPodKubernetesFileMounter.scala new file mode 100644 index 0000000000000..cc0ef0eedb457 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/DriverPodKubernetesFileMounter.scala @@ -0,0 +1,55 @@ +/* + * 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.submit + +import io.fabric8.kubernetes.api.model.{Container, PodBuilder} + +import org.apache.spark.deploy.kubernetes.constants._ + + /** + * Trait that is responsible for providing full file-paths dynamically after + * the filesDownloadPath has been defined. The file-names are then stored in the + * environmental variables in the driver-pod. + */ +private[spark] trait DriverPodKubernetesFileMounter { + def addPySparkFiles(primaryFile: String, pySparkFiles: String, + mainContainerName: String, originalPodSpec: PodBuilder) : PodBuilder +} + +private[spark] class DriverPodKubernetesFileMounterImpl() + extends DriverPodKubernetesFileMounter { + override def addPySparkFiles( + primaryFile: String, + pySparkFiles: String, + mainContainerName: String, + originalPodSpec: PodBuilder): PodBuilder = { + + originalPodSpec + .editSpec() + .editMatchingContainer(new ContainerNameEqualityPredicate(mainContainerName)) + .addNewEnv() + .withName(ENV_PYSPARK_PRIMARY) + .withValue(primaryFile) + .endEnv() + .addNewEnv() + .withName(ENV_PYSPARK_FILES) + .withValue(pySparkFiles) + .endEnv() + .endContainer() + .endSpec() + } +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/MainAppResource.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/InitContainerBundle.scala similarity index 71% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/MainAppResource.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/InitContainerBundle.scala index 436d531a850ff..ba44f794d5811 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/MainAppResource.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/InitContainerBundle.scala @@ -16,8 +16,11 @@ */ package org.apache.spark.deploy.kubernetes.submit -private[spark] sealed trait MainAppResource +import io.fabric8.kubernetes.api.model.ConfigMap -private[spark] case class PythonMainAppResource(primaryPyFile: String) extends MainAppResource +import org.apache.spark.deploy.kubernetes.{SparkPodInitContainerBootstrap} -private[spark] case class JavaMainAppResource(primaryResource: String) extends MainAppResource +case class InitContainerBundle( + sparkInitContainerConfigMap: ConfigMap, + sparkPodInitContainerBootstrap: SparkPodInitContainerBootstrap, + executorInitContainerConfiguration: ExecutorInitContainerConfiguration) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/InitContainerUtil.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/InitContainerUtil.scala index 1c5e199a52110..9b7faaa78a9aa 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/InitContainerUtil.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/InitContainerUtil.scala @@ -18,7 +18,7 @@ package org.apache.spark.deploy.kubernetes.submit import com.fasterxml.jackson.databind.ObjectMapper import com.fasterxml.jackson.module.scala.DefaultScalaModule -import io.fabric8.kubernetes.api.model.{Container, Pod, PodBuilder} +import io.fabric8.kubernetes.api.model.{Container, PodBuilder} import scala.collection.JavaConverters._ import org.apache.spark.deploy.kubernetes.constants._ @@ -27,9 +27,10 @@ private[spark] object InitContainerUtil { private val OBJECT_MAPPER = new ObjectMapper().registerModule(new DefaultScalaModule) - def appendInitContainer(originalPodSpec: Pod, initContainer: Container): Pod = { + def appendInitContainer( + originalPodSpec: PodBuilder, initContainer: Container): PodBuilder = { val resolvedInitContainers = originalPodSpec - .getMetadata + .editMetadata() .getAnnotations .asScala .get(INIT_CONTAINER_ANNOTATION) @@ -39,11 +40,10 @@ private[spark] object InitContainerUtil { existingInitContainers ++ Seq(initContainer) }.getOrElse(Seq(initContainer)) val resolvedSerializedInitContainers = OBJECT_MAPPER.writeValueAsString(resolvedInitContainers) - new PodBuilder(originalPodSpec) + originalPodSpec .editMetadata() - .removeFromAnnotations(INIT_CONTAINER_ANNOTATION) - .addToAnnotations(INIT_CONTAINER_ANNOTATION, resolvedSerializedInitContainers) - .endMetadata() - .build() + .removeFromAnnotations(INIT_CONTAINER_ANNOTATION) + .addToAnnotations(INIT_CONTAINER_ANNOTATION, resolvedSerializedInitContainers) + .endMetadata() } } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/KubernetesFileUtils.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/KubernetesFileUtils.scala index 2006007db1d08..d688bf29808fb 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/KubernetesFileUtils.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/KubernetesFileUtils.scala @@ -16,8 +16,6 @@ */ package org.apache.spark.deploy.kubernetes.submit -import java.io.File - import org.apache.spark.util.Utils private[spark] object KubernetesFileUtils { @@ -50,47 +48,4 @@ private[spark] object KubernetesFileUtils { def getOnlyRemoteFiles(uris: Iterable[String]): Iterable[String] = { filterUriStringsByScheme(uris, scheme => scheme != "file" && scheme != "local") } - - /** - * For the collection of uris, resolves any files as follows: - * - Files with scheme file:// are resolved to the download path - * - Files with scheme local:// resolve to just the path of the URI - * - Otherwise, the URI is returned as-is. - */ - def resolveSubmittedUris(fileUris: Iterable[String], fileDownloadPath: String) - : Iterable[String] = { - fileUris.map { uri => - val fileUri = Utils.resolveURI(uri) - val fileScheme = Option(fileUri.getScheme).getOrElse("file") - fileScheme match { - case "file" => - val fileName = new File(fileUri.getPath).getName - s"$fileDownloadPath/$fileName" - case "local" => - fileUri.getPath - case _ => uri - } - } - } - - /** - * If any file uri has any scheme other than local:// it is mapped as if the file - * was downloaded to the file download path. Otherwise, it is mapped to the path - * part of the URI. - */ - def resolveFilePaths(fileUris: Iterable[String], fileDownloadPath: String): Iterable[String] = { - fileUris.map { uri => - resolveFilePath(uri, fileDownloadPath) - } - } - - def resolveFilePath(uri: String, fileDownloadPath: String): String = { - val fileUri = Utils.resolveURI(uri) - if (Option(fileUri.getScheme).getOrElse("file") == "local") { - fileUri.getPath - } else { - val fileName = new File(fileUri.getPath).getName - s"$fileDownloadPath/$fileName" - } - } } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/KubernetesSubmissionStepsOrchestrator.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/KubernetesSubmissionStepsOrchestrator.scala deleted file mode 100644 index f42afafbcb41a..0000000000000 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/KubernetesSubmissionStepsOrchestrator.scala +++ /dev/null @@ -1,127 +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.submit - -import org.apache.spark.SparkConf -import org.apache.spark.deploy.kubernetes.ConfigurationUtils -import org.apache.spark.deploy.kubernetes.config._ -import org.apache.spark.deploy.kubernetes.constants._ -import org.apache.spark.deploy.kubernetes.submit.submitsteps.{BaseSubmissionStep, DependencyResolutionStep, DriverKubernetesCredentialsStep, InitContainerBootstrapStep, KubernetesSubmissionStep, PythonStep} -import org.apache.spark.deploy.kubernetes.submit.submitsteps.initcontainer.InitContainerStepsOrchestrator -import org.apache.spark.launcher.SparkLauncher - -/** - * Constructs the complete list of submission steps to run to deploy the Spark application. - */ -private[spark] class KubernetesSubmissionStepsOrchestrator( - namespace: String, - kubernetesAppId: String, - launchTime: Long, - mainAppResource: MainAppResource, - appName: String, - mainClass: String, - appArgs: Array[String], - additionalPythonFiles: Seq[String], - submissionSparkConf: SparkConf) { - - // The resource name prefix is derived from the application name, making it easy to connect the - // names of the Kubernetes resources from e.g. Kubectl or the Kubernetes dashboard to the - // application the user submitted. However, we can't use the application name in the label, as - // label values are considerably restrictive, e.g. must be no longer than 63 characters in - // length. So we generate a separate identifier for the app ID itself, and bookkeeping that - // requires finding "all pods for this application" should use the kubernetesAppId. - private val kubernetesResourceNamePrefix = - s"$appName-$launchTime".toLowerCase.replaceAll("\\.", "-") - private val jarsDownloadPath = submissionSparkConf.get(INIT_CONTAINER_JARS_DOWNLOAD_LOCATION) - private val filesDownloadPath = submissionSparkConf.get(INIT_CONTAINER_FILES_DOWNLOAD_LOCATION) - private val dockerImagePullPolicy = submissionSparkConf.get(DOCKER_IMAGE_PULL_POLICY) - private val initContainerConfigMapName = s"$kubernetesResourceNamePrefix-init-config" - - def getAllSubmissionSteps(): Seq[KubernetesSubmissionStep] = { - val additionalMainAppJar = mainAppResource match { - case JavaMainAppResource(resource) if resource != SparkLauncher.NO_RESOURCE => - Option(resource) - case _ => Option.empty - } - val additionalMainAppPythonFile = mainAppResource match { - case PythonMainAppResource(resource) if resource != SparkLauncher.NO_RESOURCE => - Option(resource) - case _ => Option.empty - } - val sparkJars = submissionSparkConf.getOption("spark.jars") - .map(_.split(",")) - .getOrElse(Array.empty[String]) ++ additionalMainAppJar.toSeq - val sparkFiles = submissionSparkConf.getOption("spark.files") - .map(_.split(",")) - .getOrElse(Array.empty[String]) ++ additionalMainAppPythonFile.toSeq - val driverCustomLabels = ConfigurationUtils.combinePrefixedKeyValuePairsWithDeprecatedConf( - submissionSparkConf, - KUBERNETES_DRIVER_LABEL_PREFIX, - KUBERNETES_DRIVER_LABELS, - "label") - require(!driverCustomLabels.contains(SPARK_APP_ID_LABEL), s"Label with key " + - s" $SPARK_APP_ID_LABEL is not allowed as it is reserved for Spark bookkeeping" + - s" operations.") - val allDriverLabels = driverCustomLabels ++ Map( - SPARK_APP_ID_LABEL -> kubernetesAppId, - SPARK_ROLE_LABEL -> SPARK_POD_DRIVER_ROLE) - val initialSubmissionStep = new BaseSubmissionStep( - kubernetesAppId, - kubernetesResourceNamePrefix, - allDriverLabels, - dockerImagePullPolicy, - appName, - mainClass, - appArgs, - submissionSparkConf) - val kubernetesCredentialsStep = new DriverKubernetesCredentialsStep( - submissionSparkConf, kubernetesAppId) - val pythonStep = mainAppResource match { - case PythonMainAppResource(mainPyResource) => - Option(new PythonStep(mainPyResource, additionalPythonFiles, filesDownloadPath)) - case _ => Option.empty[PythonStep] - } - val initContainerStepsOrchestrator = new InitContainerStepsOrchestrator( - namespace, - kubernetesResourceNamePrefix, - sparkJars, - sparkFiles, - jarsDownloadPath, - filesDownloadPath, - dockerImagePullPolicy, - allDriverLabels, - initContainerConfigMapName, - INIT_CONTAINER_CONFIG_MAP_KEY, - submissionSparkConf) - val initContainerSteps = initContainerStepsOrchestrator.getInitContainerSteps() - val initContainerBootstrapStep = new InitContainerBootstrapStep( - submissionSparkConf, - initContainerSteps, - initContainerConfigMapName, - INIT_CONTAINER_CONFIG_MAP_KEY) - val dependencyResolutionStep = new DependencyResolutionStep( - sparkJars, - sparkFiles, - jarsDownloadPath, - filesDownloadPath) - Seq( - initialSubmissionStep, - kubernetesCredentialsStep, - initContainerBootstrapStep, - dependencyResolutionStep) ++ pythonStep.toSeq - } -} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/PythonSubmissionResources.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/PythonSubmissionResources.scala new file mode 100644 index 0000000000000..e615cf72116e1 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/PythonSubmissionResources.scala @@ -0,0 +1,75 @@ +/* + * 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.submit + +import io.fabric8.kubernetes.api.model.{Pod, PodBuilder} + +private[spark] trait PythonSubmissionResources { + def sparkJars: Seq[String] + def pySparkFiles: Array[String] + def arguments: Array[String] + def primaryPySparkResource(containerLocalizedFilesResolver: ContainerLocalizedFilesResolver) + : String + def driverPodWithPySparkEnvs( + driverPodFileMounter: DriverPodKubernetesFileMounter, + resolvedPrimaryPySparkResource: String, + resolvedPySparkFiles: String, + driverContainerName: String, + driverPodBuilder: PodBuilder) : Pod +} + +private[spark] class PythonSubmissionResourcesImpl( + private val mainAppResource: String, + private val appArgs: Array[String] ) extends PythonSubmissionResources { + + private val pyFiles: Array[String] = { + Option(appArgs(0)).map(a => mainAppResource +: a.split(",")) + .getOrElse(Array(mainAppResource)) + } + + override def sparkJars: Seq[String] = Seq.empty[String] + + override def pySparkFiles: Array[String] = pyFiles + + override def arguments: Array[String] = { + pyFiles.toList match { + case Nil => appArgs + case a :: b => a match { + case _ if a == mainAppResource && b == Nil => appArgs + case _ => appArgs.drop(1) + } + } + } + override def primaryPySparkResource ( + containerLocalizedFilesResolver: ContainerLocalizedFilesResolver) : String = + containerLocalizedFilesResolver.resolvePrimaryResourceFile() + + override def driverPodWithPySparkEnvs( + driverPodFileMounter: DriverPodKubernetesFileMounter, + resolvedPrimaryPySparkResource: String, + resolvedPySparkFiles: String, + driverContainerName: String, + driverPodBuilder: PodBuilder) : Pod = { + driverPodFileMounter + .addPySparkFiles( + resolvedPrimaryPySparkResource, + resolvedPySparkFiles, + driverContainerName, + driverPodBuilder) + .build() + } +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/BaseInitContainerStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/SparkInitContainerConfigMapBuilder.scala similarity index 62% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/BaseInitContainerStep.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/SparkInitContainerConfigMapBuilder.scala index 7c4b2623ba5d1..4062a3113eddf 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/BaseInitContainerStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/SparkInitContainerConfigMapBuilder.scala @@ -14,23 +14,32 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit.submitsteps.initcontainer +package org.apache.spark.deploy.kubernetes.submit + +import io.fabric8.kubernetes.api.model.ConfigMap -import org.apache.spark.deploy.kubernetes.{PodWithDetachedInitContainer, SparkPodInitContainerBootstrap} import org.apache.spark.deploy.kubernetes.config._ -import org.apache.spark.deploy.kubernetes.submit.KubernetesFileUtils -private[spark] class BaseInitContainerStep( +private[spark] trait SparkInitContainerConfigMapBuilder { + /** + * Construct a config map that an init-container should reference for fetching + * remote dependencies. The config map includes the remote jars and files to download, + * as well as details to fetch files from a resource staging server, if applicable. + */ + def build(): ConfigMap +} + +private[spark] class SparkInitContainerConfigMapBuilderImpl( sparkJars: Seq[String], sparkFiles: Seq[String], jarsDownloadPath: String, filesDownloadPath: String, configMapName: String, configMapKey: String, - podAndInitContainerBootstrap: SparkPodInitContainerBootstrap) - extends InitContainerStep { + submittedDependenciesPlugin: Option[SubmittedDependencyInitContainerConfigPlugin]) + extends SparkInitContainerConfigMapBuilder { - override def prepareInitContainer(initContainerSpec: InitContainerSpec): InitContainerSpec = { + override def build(): ConfigMap = { val remoteJarsToDownload = KubernetesFileUtils.getOnlyRemoteFiles(sparkJars) val remoteFilesToDownload = KubernetesFileUtils.getOnlyRemoteFiles(sparkFiles) val remoteJarsConf = if (remoteJarsToDownload.nonEmpty) { @@ -48,16 +57,12 @@ private[spark] class BaseInitContainerStep( INIT_CONTAINER_FILES_DOWNLOAD_LOCATION.key -> filesDownloadPath) ++ remoteJarsConf ++ remoteFilesConf - val bootstrappedPodAndInitContainer = - podAndInitContainerBootstrap.bootstrapInitContainerAndVolumes( - PodWithDetachedInitContainer( - initContainerSpec.podToInitialize, - initContainerSpec.initContainer, - initContainerSpec.driverContainer)) - initContainerSpec.copy( - initContainer = bootstrappedPodAndInitContainer.initContainer, - driverContainer = bootstrappedPodAndInitContainer.mainContainer, - podToInitialize = bootstrappedPodAndInitContainer.pod, - initContainerProperties = baseInitContainerConfig) + val submittedDependenciesConfig = submittedDependenciesPlugin.map { plugin => + plugin.configurationsToFetchSubmittedDependencies() + }.toSeq.flatten.toMap + PropertiesConfigMapFromScalaMapBuilder.buildConfigMap( + configMapName, + configMapKey, + baseInitContainerConfig ++ submittedDependenciesConfig) } } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/SubmittedDependencyInitContainerConfigPlugin.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/SubmittedDependencyInitContainerConfigPlugin.scala new file mode 100644 index 0000000000000..06d3648efb89f --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/SubmittedDependencyInitContainerConfigPlugin.scala @@ -0,0 +1,96 @@ +/* + * 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.submit + +import org.apache.spark.SparkException +import org.apache.spark.deploy.kubernetes.config._ +import org.apache.spark.internal.config.OptionalConfigEntry +import org.apache.spark.util.Utils + +private[spark] trait SubmittedDependencyInitContainerConfigPlugin { + /** + * Obtain configuration to fetch submitted dependencies from a resource staging server. + * This includes the resource identifiers for the jar and file bundles, as well as the + * remote location of the resource staging server, and the location of secret files for + * authenticating to the resource staging server. Note that the secret file paths here need to + * line up with the locations the secrets are mounted by + * SubmittedDependencyInitContainerVolumesPlugin; constants provide the consistency and + * convention for these to line up. + */ + def configurationsToFetchSubmittedDependencies(): Map[String, String] +} + +private[spark] class SubmittedDependencyInitContainerConfigPluginImpl( + internalResourceStagingServerUri: String, + jarsResourceId: String, + filesResourceId: String, + jarsSecretKey: String, + filesSecretKey: String, + trustStoreSecretKey: String, + clientCertSecretKey: String, + resourceStagingServerSslEnabled: Boolean, + maybeInternalTrustStoreUri: Option[String], + maybeInternalClientCertUri: Option[String], + maybeInternalTrustStorePassword: Option[String], + maybeInternalTrustStoreType: Option[String], + secretsVolumeMountPath: String) + extends SubmittedDependencyInitContainerConfigPlugin { + + override def configurationsToFetchSubmittedDependencies(): Map[String, String] = { + Map[String, String]( + RESOURCE_STAGING_SERVER_URI.key -> internalResourceStagingServerUri, + INIT_CONTAINER_DOWNLOAD_JARS_RESOURCE_IDENTIFIER.key -> jarsResourceId, + INIT_CONTAINER_DOWNLOAD_JARS_SECRET_LOCATION.key -> + s"$secretsVolumeMountPath/$jarsSecretKey", + INIT_CONTAINER_DOWNLOAD_FILES_RESOURCE_IDENTIFIER.key -> filesResourceId, + INIT_CONTAINER_DOWNLOAD_FILES_SECRET_LOCATION.key -> + s"$secretsVolumeMountPath/$filesSecretKey", + RESOURCE_STAGING_SERVER_SSL_ENABLED.key -> resourceStagingServerSslEnabled.toString) ++ + resolveSecretPath( + maybeInternalTrustStoreUri, + trustStoreSecretKey, + RESOURCE_STAGING_SERVER_TRUSTSTORE_FILE, + "TrustStore URI") ++ + resolveSecretPath( + maybeInternalClientCertUri, + clientCertSecretKey, + RESOURCE_STAGING_SERVER_CLIENT_CERT_PEM, + "Client certificate URI") ++ + maybeInternalTrustStorePassword.map { password => + (RESOURCE_STAGING_SERVER_TRUSTSTORE_PASSWORD.key, password) + }.toMap ++ + maybeInternalTrustStoreType.map { storeType => + (RESOURCE_STAGING_SERVER_TRUSTSTORE_TYPE.key, storeType) + }.toMap + } + + private def resolveSecretPath( + maybeUri: Option[String], + secretKey: String, + configEntry: OptionalConfigEntry[String], + uriType: String): Map[String, String] = { + maybeUri.map(Utils.resolveURI).map { uri => + val resolvedPath = Option(uri.getScheme).getOrElse("file") match { + case "file" => s"$secretsVolumeMountPath/$secretKey" + case "local" => uri.getPath + case invalid => throw new SparkException(s"$uriType has invalid scheme $invalid must be" + + s" local://, file://, or empty.") + } + (configEntry.key, resolvedPath) + }.toMap + } +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/SubmittedDependencySecretBuilder.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/SubmittedDependencySecretBuilder.scala new file mode 100644 index 0000000000000..7850853df97e6 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/SubmittedDependencySecretBuilder.scala @@ -0,0 +1,81 @@ +/* + * 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.submit + +import java.io.File + +import com.google.common.base.Charsets +import com.google.common.io.{BaseEncoding, Files} +import io.fabric8.kubernetes.api.model.{Secret, SecretBuilder} +import scala.collection.JavaConverters._ + +import org.apache.spark.util.Utils + +private[spark] trait SubmittedDependencySecretBuilder { + /** + * Construct a Kubernetes secret bundle that init-containers can use to retrieve an + * application's dependencies. + */ + def build(): Secret +} + +private[spark] class SubmittedDependencySecretBuilderImpl( + secretName: String, + jarsResourceSecret: String, + filesResourceSecret: String, + jarsSecretKey: String, + filesSecretKey: String, + trustStoreSecretKey: String, + clientCertSecretKey: String, + internalTrustStoreUri: Option[String], + internalClientCertUri: Option[String]) + extends SubmittedDependencySecretBuilder { + + override def build(): Secret = { + val trustStoreBase64 = convertFileToBase64IfSubmitterLocal( + trustStoreSecretKey, internalTrustStoreUri) + val clientCertBase64 = convertFileToBase64IfSubmitterLocal( + clientCertSecretKey, internalClientCertUri) + val jarsSecretBase64 = BaseEncoding.base64().encode(jarsResourceSecret.getBytes(Charsets.UTF_8)) + val filesSecretBase64 = BaseEncoding.base64().encode( + filesResourceSecret.getBytes(Charsets.UTF_8)) + val secretData = Map( + jarsSecretKey -> jarsSecretBase64, + filesSecretKey -> filesSecretBase64) ++ + trustStoreBase64 ++ + clientCertBase64 + val kubernetesSecret = new SecretBuilder() + .withNewMetadata() + .withName(secretName) + .endMetadata() + .addToData(secretData.asJava) + .build() + kubernetesSecret + } + + private def convertFileToBase64IfSubmitterLocal(secretKey: String, secretUri: Option[String]) + : Map[String, String] = { + secretUri.filter { trustStore => + Option(Utils.resolveURI(trustStore).getScheme).getOrElse("file") == "file" + }.map { uri => + val file = new File(Utils.resolveURI(uri).getPath) + require(file.isFile, "Dependency server trustStore provided at" + + file.getAbsolutePath + " does not exist or is not a file.") + (secretKey, BaseEncoding.base64().encode(Files.toByteArray(file))) + }.toMap + } +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/BaseSubmissionStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/BaseSubmissionStep.scala deleted file mode 100644 index e49262c20c745..0000000000000 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/BaseSubmissionStep.scala +++ /dev/null @@ -1,136 +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.submit.submitsteps - -import io.fabric8.kubernetes.api.model.{ContainerBuilder, EnvVarBuilder, PodBuilder, QuantityBuilder} -import scala.collection.JavaConverters._ - -import org.apache.spark.SparkConf -import org.apache.spark.deploy.kubernetes.ConfigurationUtils -import org.apache.spark.deploy.kubernetes.config._ -import org.apache.spark.deploy.kubernetes.constants._ - -/** - * Represents the initial setup required for the driver. - */ -private[spark] class BaseSubmissionStep( - kubernetesAppId: String, - kubernetesResourceNamePrefix: String, - driverLabels: Map[String, String], - dockerImagePullPolicy: String, - appName: String, - mainClass: String, - appArgs: Array[String], - submissionSparkConf: SparkConf) - extends KubernetesSubmissionStep { - - private val kubernetesDriverPodName = submissionSparkConf.get(KUBERNETES_DRIVER_POD_NAME) - .getOrElse(s"$kubernetesResourceNamePrefix-driver") - private val driverExtraClasspath = submissionSparkConf.get( - org.apache.spark.internal.config.DRIVER_CLASS_PATH) - // CPU settings - private val driverCpuCores = submissionSparkConf.getOption("spark.driver.cores").getOrElse("1") - private val driverLimitCores = submissionSparkConf.getOption(KUBERNETES_DRIVER_LIMIT_CORES.key) - - // Memory settings - private val driverMemoryMb = submissionSparkConf.get( - org.apache.spark.internal.config.DRIVER_MEMORY) - private val memoryOverheadMb = submissionSparkConf - .get(KUBERNETES_DRIVER_MEMORY_OVERHEAD) - .getOrElse(math.max((MEMORY_OVERHEAD_FACTOR * driverMemoryMb).toInt, - MEMORY_OVERHEAD_MIN)) - private val driverContainerMemoryWithOverhead = driverMemoryMb + memoryOverheadMb - private val driverDockerImage = submissionSparkConf.get(DRIVER_DOCKER_IMAGE) - - override def prepareSubmission( - driverSpec: KubernetesDriverSpec): KubernetesDriverSpec = { - val driverExtraClasspathEnv = driverExtraClasspath.map { classPath => - new EnvVarBuilder() - .withName(ENV_SUBMIT_EXTRA_CLASSPATH) - .withValue(classPath) - .build() - } - val driverCpuQuantity = new QuantityBuilder(false) - .withAmount(driverCpuCores) - .build() - val driverMemoryQuantity = new QuantityBuilder(false) - .withAmount(s"${driverMemoryMb}M") - .build() - val driverMemoryLimitQuantity = new QuantityBuilder(false) - .withAmount(s"${driverContainerMemoryWithOverhead}M") - .build() - val maybeCpuLimitQuantity = driverLimitCores.map { limitCores => - ("cpu", new QuantityBuilder(false).withAmount(limitCores).build()) - } - val driverContainer = new ContainerBuilder(driverSpec.driverContainer) - .withName(DRIVER_CONTAINER_NAME) - .withImage(driverDockerImage) - .withImagePullPolicy(dockerImagePullPolicy) - .addToEnv(driverExtraClasspathEnv.toSeq: _*) - .addNewEnv() - .withName(ENV_DRIVER_MEMORY) - .withValue(driverContainerMemoryWithOverhead + "m") - .endEnv() - .addNewEnv() - .withName(ENV_DRIVER_MAIN_CLASS) - .withValue(mainClass) - .endEnv() - .addNewEnv() - .withName(ENV_DRIVER_ARGS) - .withValue(appArgs.mkString(" ")) - .endEnv() - .withNewResources() - .addToRequests("cpu", driverCpuQuantity) - .addToRequests("memory", driverMemoryQuantity) - .addToLimits("memory", driverMemoryLimitQuantity) - .addToLimits(maybeCpuLimitQuantity.toMap.asJava) - .endResources() - .build() - val baseDriverPod = new PodBuilder(driverSpec.driverPod) - .withNewMetadata() - .withName(kubernetesDriverPodName) - .addToLabels(driverLabels.asJava) - .addToAnnotations(getAllDriverAnnotations(submissionSparkConf).asJava) - .endMetadata() - .withNewSpec() - .withRestartPolicy("Never") - .endSpec() - .build() - val resolvedSparkConf = driverSpec.driverSparkConf.clone() - .setIfMissing(KUBERNETES_DRIVER_POD_NAME, kubernetesDriverPodName) - .set("spark.app.id", kubernetesAppId) - .set(KUBERNETES_EXECUTOR_POD_NAME_PREFIX, kubernetesResourceNamePrefix) - // We don't need this anymore since we just set the JVM options on the environment - .remove(org.apache.spark.internal.config.DRIVER_JAVA_OPTIONS) - driverSpec.copy( - driverPod = baseDriverPod, - driverSparkConf = resolvedSparkConf, - driverContainer = driverContainer) - } - - private def getAllDriverAnnotations(sparkConf: SparkConf): Map[String, String] = { - val driverCustomAnnotations = ConfigurationUtils.combinePrefixedKeyValuePairsWithDeprecatedConf( - sparkConf, - KUBERNETES_DRIVER_ANNOTATION_PREFIX, - KUBERNETES_DRIVER_ANNOTATIONS, - "annotation") - require(!driverCustomAnnotations.contains(SPARK_APP_NAME_ANNOTATION), - s"Annotation with key $SPARK_APP_NAME_ANNOTATION is not allowed as it is reserved for" + - s" Spark bookkeeping operations.") - driverCustomAnnotations ++ Map(SPARK_APP_NAME_ANNOTATION -> appName) - } -} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DependencyResolutionStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DependencyResolutionStep.scala deleted file mode 100644 index b248418abb7a8..0000000000000 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DependencyResolutionStep.scala +++ /dev/null @@ -1,66 +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.submit.submitsteps - - -import java.io.File - -import io.fabric8.kubernetes.api.model.ContainerBuilder - -import org.apache.spark.deploy.kubernetes.constants._ -import org.apache.spark.deploy.kubernetes.submit.KubernetesFileUtils -import org.apache.spark.util.Utils - -/** - * Step that configures the classpath, spark.jars, and spark.files for the driver given that - * the init-container will download files to the download paths and that the user may provide - * files with local:// schemes. Note that this is separate from the init-container bootstrap - * step because jars with local schemes can still be provided even if the init-container is - * not being used, and those URIs still need to be resolved. - */ -private[spark] class DependencyResolutionStep( - sparkJars: Seq[String], - sparkFiles: Seq[String], - jarsDownloadPath: String, - filesDownloadPath: String) extends KubernetesSubmissionStep { - - override def prepareSubmission(driverSpec: KubernetesDriverSpec): KubernetesDriverSpec = { - val resolvedSparkJars = KubernetesFileUtils.resolveSubmittedUris(sparkJars, jarsDownloadPath) - val resolvedSparkFiles = KubernetesFileUtils.resolveSubmittedUris(sparkFiles, filesDownloadPath) - val sparkConfResolvedSparkDependencies = driverSpec.driverSparkConf.clone() - if (resolvedSparkJars.nonEmpty) { - sparkConfResolvedSparkDependencies.set("spark.jars", resolvedSparkJars.mkString(",")) - } - if (resolvedSparkFiles.nonEmpty) { - sparkConfResolvedSparkDependencies.set("spark.files", resolvedSparkFiles.mkString(",")) - } - val resolvedClasspath = KubernetesFileUtils.resolveFilePaths(sparkJars, jarsDownloadPath) - val driverContainerWithResolvedClasspath = if (resolvedClasspath.nonEmpty) { - new ContainerBuilder(driverSpec.driverContainer) - .addNewEnv() - .withName(ENV_MOUNTED_CLASSPATH) - .withValue(resolvedClasspath.mkString(File.pathSeparator)) - .endEnv() - .build() - } else { - driverSpec.driverContainer - } - driverSpec.copy( - driverContainer = driverContainerWithResolvedClasspath, - driverSparkConf = sparkConfResolvedSparkDependencies) - } -} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DriverKubernetesCredentialsStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DriverKubernetesCredentialsStep.scala deleted file mode 100644 index c3a6b5e456f79..0000000000000 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DriverKubernetesCredentialsStep.scala +++ /dev/null @@ -1,218 +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.submit.submitsteps - -import java.io.File -import java.nio.charset.StandardCharsets - -import com.google.common.io.{BaseEncoding, Files} -import io.fabric8.kubernetes.api.model.{ContainerBuilder, PodBuilder, Secret, SecretBuilder} -import scala.collection.JavaConverters._ - -import org.apache.spark.SparkConf -import org.apache.spark.deploy.kubernetes.config._ -import org.apache.spark.deploy.kubernetes.constants._ - -private[spark] class DriverKubernetesCredentialsStep( - submissionSparkConf: SparkConf, - kubernetesAppId: String) extends KubernetesSubmissionStep { - - private val maybeMountedOAuthTokenFile = submissionSparkConf.getOption( - s"$APISERVER_AUTH_DRIVER_MOUNTED_CONF_PREFIX.$OAUTH_TOKEN_FILE_CONF_SUFFIX") - private val maybeMountedClientKeyFile = submissionSparkConf.getOption( - s"$APISERVER_AUTH_DRIVER_MOUNTED_CONF_PREFIX.$CLIENT_KEY_FILE_CONF_SUFFIX") - private val maybeMountedClientCertFile = submissionSparkConf.getOption( - s"$APISERVER_AUTH_DRIVER_MOUNTED_CONF_PREFIX.$CLIENT_CERT_FILE_CONF_SUFFIX") - private val maybeMountedCaCertFile = submissionSparkConf.getOption( - s"$APISERVER_AUTH_DRIVER_MOUNTED_CONF_PREFIX.$CA_CERT_FILE_CONF_SUFFIX") - /** - * Apply some transformation to the previous state of the driver to add a new feature to it. - */ - override def prepareSubmission(driverSpec: KubernetesDriverSpec): KubernetesDriverSpec = { - val driverSparkConf = driverSpec.driverSparkConf.clone(); - val oauthTokenBase64 = submissionSparkConf - .getOption(s"$APISERVER_AUTH_DRIVER_CONF_PREFIX.$OAUTH_TOKEN_CONF_SUFFIX") - .map { token => - BaseEncoding.base64().encode(token.getBytes(StandardCharsets.UTF_8)) - } - val caCertDataBase64 = safeFileConfToBase64( - s"$APISERVER_AUTH_DRIVER_CONF_PREFIX.$CA_CERT_FILE_CONF_SUFFIX", - s"Driver CA cert file provided at %s does not exist or is not a file.") - val clientKeyDataBase64 = safeFileConfToBase64( - s"$APISERVER_AUTH_DRIVER_CONF_PREFIX.$CLIENT_KEY_FILE_CONF_SUFFIX", - s"Driver client key file provided at %s does not exist or is not a file.") - val clientCertDataBase64 = safeFileConfToBase64( - s"$APISERVER_AUTH_DRIVER_CONF_PREFIX.$CLIENT_CERT_FILE_CONF_SUFFIX", - s"Driver client cert file provided at %s does not exist or is not a file.") - val driverSparkConfWithCredentialsLocations = setDriverPodKubernetesCredentialLocations( - driverSparkConf, - oauthTokenBase64, - caCertDataBase64, - clientKeyDataBase64, - clientCertDataBase64) - val kubernetesCredentialsSecret = createCredentialsSecret( - oauthTokenBase64, - caCertDataBase64, - clientKeyDataBase64, - clientCertDataBase64) - val driverPodWithMountedKubernetesCredentials = kubernetesCredentialsSecret.map { secret => - new PodBuilder(driverSpec.driverPod) - .editSpec() - .addNewVolume() - .withName(DRIVER_CREDENTIALS_SECRET_VOLUME_NAME) - .withNewSecret().withSecretName(secret.getMetadata.getName).endSecret() - .endVolume() - .endSpec() - .build() - }.getOrElse(driverSpec.driverPod) - val driverContainerWithMountedSecretVolume = kubernetesCredentialsSecret.map { secret => - new ContainerBuilder(driverSpec.driverContainer) - .addNewVolumeMount() - .withName(DRIVER_CREDENTIALS_SECRET_VOLUME_NAME) - .withMountPath(DRIVER_CREDENTIALS_SECRETS_BASE_DIR) - .endVolumeMount() - .build() - }.getOrElse(driverSpec.driverContainer) - driverSpec.copy( - driverPod = driverPodWithMountedKubernetesCredentials, - otherKubernetesResources = - driverSpec.otherKubernetesResources ++ kubernetesCredentialsSecret.toSeq, - driverSparkConf = driverSparkConfWithCredentialsLocations, - driverContainer = driverContainerWithMountedSecretVolume) - } - - private def createCredentialsSecret( - driverOAuthTokenBase64: Option[String], - driverCaCertDataBase64: Option[String], - driverClientKeyDataBase64: Option[String], - driverClientCertDataBase64: Option[String]): Option[Secret] = { - val allSecretData = - resolveSecretData( - maybeMountedClientKeyFile, - driverClientKeyDataBase64, - DRIVER_CREDENTIALS_CLIENT_KEY_SECRET_NAME) ++ - resolveSecretData( - maybeMountedClientCertFile, - driverClientCertDataBase64, - DRIVER_CREDENTIALS_CLIENT_CERT_SECRET_NAME) ++ - resolveSecretData( - maybeMountedCaCertFile, - driverCaCertDataBase64, - DRIVER_CREDENTIALS_CA_CERT_SECRET_NAME) ++ - resolveSecretData( - maybeMountedOAuthTokenFile, - driverOAuthTokenBase64, - DRIVER_CREDENTIALS_OAUTH_TOKEN_SECRET_NAME) - if (allSecretData.isEmpty) { - None - } else { - Some(new SecretBuilder() - .withNewMetadata().withName(s"$kubernetesAppId-kubernetes-credentials").endMetadata() - .withData(allSecretData.asJava) - .build()) - } - } - - private def setDriverPodKubernetesCredentialLocations( - driverSparkConf: SparkConf, - driverOauthTokenBase64: Option[String], - driverCaCertDataBase64: Option[String], - driverClientKeyDataBase64: Option[String], - driverClientCertDataBase64: Option[String]): SparkConf = { - val resolvedMountedOAuthTokenFile = resolveSecretLocation( - maybeMountedOAuthTokenFile, - driverOauthTokenBase64, - DRIVER_CREDENTIALS_OAUTH_TOKEN_PATH) - val resolvedMountedClientKeyFile = resolveSecretLocation( - maybeMountedClientKeyFile, - driverClientKeyDataBase64, - DRIVER_CREDENTIALS_CLIENT_KEY_PATH) - val resolvedMountedClientCertFile = resolveSecretLocation( - maybeMountedClientCertFile, - driverClientCertDataBase64, - DRIVER_CREDENTIALS_CLIENT_CERT_PATH) - val resolvedMountedCaCertFile = resolveSecretLocation( - maybeMountedCaCertFile, - driverCaCertDataBase64, - DRIVER_CREDENTIALS_CA_CERT_PATH) - val sparkConfWithCredentialLocations = driverSparkConf - .setOption( - s"$APISERVER_AUTH_DRIVER_MOUNTED_CONF_PREFIX.$CA_CERT_FILE_CONF_SUFFIX", - resolvedMountedCaCertFile) - .setOption( - s"$APISERVER_AUTH_DRIVER_MOUNTED_CONF_PREFIX.$CLIENT_KEY_FILE_CONF_SUFFIX", - resolvedMountedClientKeyFile) - .setOption( - s"$APISERVER_AUTH_DRIVER_MOUNTED_CONF_PREFIX.$CLIENT_CERT_FILE_CONF_SUFFIX", - resolvedMountedClientCertFile) - .setOption( - s"$APISERVER_AUTH_DRIVER_MOUNTED_CONF_PREFIX.$OAUTH_TOKEN_FILE_CONF_SUFFIX", - resolvedMountedOAuthTokenFile) - // Redact all OAuth token values - sparkConfWithCredentialLocations - .getAll - .filter(_._1.endsWith(OAUTH_TOKEN_CONF_SUFFIX)).map(_._1) - .foreach { - sparkConfWithCredentialLocations.set(_, "") - } - sparkConfWithCredentialLocations - } - - private def safeFileConfToBase64( - conf: String, - fileNotFoundFormatString: String): Option[String] = { - submissionSparkConf.getOption(conf) - .map(new File(_)) - .map { file => - require(file.isFile, String.format(fileNotFoundFormatString, file.getAbsolutePath)) - BaseEncoding.base64().encode(Files.toByteArray(file)) - } - } - - private def resolveSecretLocation( - mountedUserSpecified: Option[String], - valueMountedFromSubmitter: Option[String], - mountedCanonicalLocation: String): Option[String] = { - mountedUserSpecified.orElse(valueMountedFromSubmitter.map( _ => { - mountedCanonicalLocation - })) - } - - private def resolveSecretData( - mountedUserSpecified: Option[String], - valueMountedFromSubmitter: Option[String], - secretName: String): Map[String, String] = { - mountedUserSpecified.map { _ => Map.empty[String, String]} - .getOrElse { - valueMountedFromSubmitter.map { valueBase64 => - Map(secretName -> valueBase64) - }.getOrElse(Map.empty[String, String]) - } - } - - private implicit def augmentSparkConf(sparkConf: SparkConf): OptionSettableSparkConf = { - new OptionSettableSparkConf(sparkConf) - } -} - -private class OptionSettableSparkConf(sparkConf: SparkConf) { - def setOption(configEntry: String, option: Option[String]): SparkConf = { - option.map( opt => { - sparkConf.set(configEntry, opt) - }).getOrElse(sparkConf) - } -} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/InitContainerBootstrapStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/InitContainerBootstrapStep.scala deleted file mode 100644 index 7ce7a8472f812..0000000000000 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/InitContainerBootstrapStep.scala +++ /dev/null @@ -1,64 +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.submit.submitsteps - -import io.fabric8.kubernetes.api.model.{ContainerBuilder, HasMetadata} - -import org.apache.spark.SparkConf -import org.apache.spark.deploy.kubernetes.config._ -import org.apache.spark.deploy.kubernetes.submit.{InitContainerUtil, PropertiesConfigMapFromScalaMapBuilder} -import org.apache.spark.deploy.kubernetes.submit.submitsteps.initcontainer.{InitContainerSpec, InitContainerStep} - -private[spark] class InitContainerBootstrapStep( - submissionSparkConf: SparkConf, - initContainerSteps: Seq[InitContainerStep], - initContainerConfigMapName: String, - initContainerConfigMapKey: String) - extends KubernetesSubmissionStep { - - override def prepareSubmission(driverSpec: KubernetesDriverSpec): KubernetesDriverSpec = { - val baseInitContainerSpec = InitContainerSpec( - initContainerProperties = Map.empty[String, String], - additionalDriverSparkConf = Map.empty[String, String], - initContainer = new ContainerBuilder().build(), - driverContainer = driverSpec.driverContainer, - podToInitialize = driverSpec.driverPod, - initContainerDependentResources = Seq.empty[HasMetadata]) - var currentInitContainerSpec = baseInitContainerSpec - for (nextStep <- initContainerSteps) { - currentInitContainerSpec = nextStep.prepareInitContainer(currentInitContainerSpec) - } - val configMap = PropertiesConfigMapFromScalaMapBuilder.buildConfigMap( - initContainerConfigMapName, - initContainerConfigMapKey, - currentInitContainerSpec.initContainerProperties) - val resolvedDriverSparkConf = driverSpec.driverSparkConf.clone() - .set(EXECUTOR_INIT_CONTAINER_CONFIG_MAP, initContainerConfigMapName) - .set(EXECUTOR_INIT_CONTAINER_CONFIG_MAP_KEY, initContainerConfigMapKey) - .setAll(currentInitContainerSpec.additionalDriverSparkConf) - val resolvedDriverPod = InitContainerUtil.appendInitContainer( - currentInitContainerSpec.podToInitialize, currentInitContainerSpec.initContainer) - driverSpec.copy( - driverPod = resolvedDriverPod, - driverContainer = currentInitContainerSpec.driverContainer, - driverSparkConf = resolvedDriverSparkConf, - otherKubernetesResources = - driverSpec.otherKubernetesResources ++ - currentInitContainerSpec.initContainerDependentResources ++ - Seq(configMap)) - } -} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/KubernetesDriverSpec.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/KubernetesDriverSpec.scala deleted file mode 100644 index 6c669853e16f0..0000000000000 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/KubernetesDriverSpec.scala +++ /dev/null @@ -1,35 +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.submit.submitsteps - -import io.fabric8.kubernetes.api.model.{Container, HasMetadata, Pod} - -import org.apache.spark.SparkConf - -/** - * Represents the components and characteristics of a Spark driver. The driver can be considered - * as being comprised of the driver pod itself, any other Kubernetes resources that the driver - * pod depends on, and the SparkConf that should be supplied to the Spark application. The driver - * container should be operated on via the specific field of this case class as opposed to trying - * to edit the container directly on the pod. The driver container should be attached at the - * end of executing all submission steps. - */ -private[spark] case class KubernetesDriverSpec( - driverPod: Pod, - driverContainer: Container, - otherKubernetesResources: Seq[HasMetadata], - driverSparkConf: SparkConf) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/KubernetesSubmissionStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/KubernetesSubmissionStep.scala deleted file mode 100644 index bd810880d4589..0000000000000 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/KubernetesSubmissionStep.scala +++ /dev/null @@ -1,28 +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.submit.submitsteps - -/** - * Represents a step in preparing the Kubernetes driver. - */ -private[spark] trait KubernetesSubmissionStep { - - /** - * Apply some transformation to the previous state of the driver to add a new feature to it. - */ - def prepareSubmission(driverSpec: KubernetesDriverSpec): KubernetesDriverSpec -} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/PythonStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/PythonStep.scala deleted file mode 100644 index 484f57087b36e..0000000000000 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/PythonStep.scala +++ /dev/null @@ -1,42 +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.submit.submitsteps - -import io.fabric8.kubernetes.api.model.ContainerBuilder - -import org.apache.spark.deploy.kubernetes.constants._ -import org.apache.spark.deploy.kubernetes.submit.KubernetesFileUtils - -private[spark] class PythonStep( - primaryPyFile: String, - otherPyFiles: Seq[String], - filesDownloadPath: String) extends KubernetesSubmissionStep { - - override def prepareSubmission(driverSpec: KubernetesDriverSpec): KubernetesDriverSpec = { - val withPythonPrimaryFileContainer = new ContainerBuilder(driverSpec.driverContainer) - .addNewEnv() - .withName(ENV_PYSPARK_PRIMARY) - .withValue(KubernetesFileUtils.resolveFilePath(primaryPyFile, filesDownloadPath)) - .endEnv() - .addNewEnv() - .withName(ENV_PYSPARK_FILES) - .withValue( - KubernetesFileUtils.resolveFilePaths(otherPyFiles, filesDownloadPath).mkString(",")) - .endEnv() - driverSpec.copy(driverContainer = withPythonPrimaryFileContainer.build()) - } -} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/InitContainerSpec.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/InitContainerSpec.scala deleted file mode 100644 index 5b5ac3c1f17c2..0000000000000 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/InitContainerSpec.scala +++ /dev/null @@ -1,41 +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.submit.submitsteps.initcontainer - -import io.fabric8.kubernetes.api.model.{Container, HasMetadata, Pod} - -/** - * Represents a given configuration of the init-container, informing the main - * InitContainerBootstrapStep of how the driver should be configured. This includes: - *

- * - What properties should be set on the init-container, - * - What Spark properties should be set on the driver's SparkConf given this init-container, - * - The spec of the init container itself, - * - The spec of the main container so that it can be modified to share volumes with the - * init-container - * - The spec of the pod EXCEPT for the addition of the given init-container (e.g. volumes - * the init-container needs or modifications to a main container that shares data with the - * init-container), - * - Any Kubernetes resources that need to be created for the init-container's function. - */ -private[spark] case class InitContainerSpec( - initContainerProperties: Map[String, String], - additionalDriverSparkConf: Map[String, String], - initContainer: Container, - driverContainer: Container, - podToInitialize: Pod, - initContainerDependentResources: Seq[HasMetadata]) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/InitContainerStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/InitContainerStep.scala deleted file mode 100644 index c1ae640fb92df..0000000000000 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/InitContainerStep.scala +++ /dev/null @@ -1,25 +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.submit.submitsteps.initcontainer - -/** - * Represents a step in preparing the init-container for the driver and executors. - */ -private[spark] trait InitContainerStep { - - def prepareInitContainer(initContainerSpec: InitContainerSpec): InitContainerSpec -} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/InitContainerStepsOrchestrator.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/InitContainerStepsOrchestrator.scala deleted file mode 100644 index 1edff632f8f3c..0000000000000 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/InitContainerStepsOrchestrator.scala +++ /dev/null @@ -1,131 +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.submit.submitsteps.initcontainer - -import org.apache.spark.SparkConf -import org.apache.spark.deploy.kubernetes.{InitContainerResourceStagingServerSecretPluginImpl, OptionRequirements, SparkPodInitContainerBootstrapImpl} -import org.apache.spark.deploy.kubernetes.config._ -import org.apache.spark.deploy.kubernetes.constants._ -import org.apache.spark.deploy.kubernetes.submit.SubmittedDependencyUploaderImpl -import org.apache.spark.deploy.rest.kubernetes.{ResourceStagingServerSslOptionsProviderImpl, RetrofitClientFactoryImpl} -import org.apache.spark.util.Utils - -/** - * Returns the complete ordered list of steps required to configure the init-container. - */ -private[spark] class InitContainerStepsOrchestrator( - namespace: String, - kubernetesResourceNamePrefix: String, - sparkJars: Seq[String], - sparkFiles: Seq[String], - jarsDownloadPath: String, - filesDownloadPath: String, - dockerImagePullPolicy: String, - driverLabels: Map[String, String], - initContainerConfigMapName: String, - initContainerConfigMapKey: String, - submissionSparkConf: SparkConf) { - - private val submittedResourcesSecretName = s"$kubernetesResourceNamePrefix-init-secret" - private val resourceStagingServerUri = submissionSparkConf.get(RESOURCE_STAGING_SERVER_URI) - private val resourceStagingServerInternalUri = - submissionSparkConf.get(RESOURCE_STAGING_SERVER_INTERNAL_URI) - private val initContainerImage = submissionSparkConf.get(INIT_CONTAINER_DOCKER_IMAGE) - private val downloadTimeoutMinutes = submissionSparkConf.get(INIT_CONTAINER_MOUNT_TIMEOUT) - private val maybeResourceStagingServerInternalTrustStore = - submissionSparkConf.get(RESOURCE_STAGING_SERVER_INTERNAL_TRUSTSTORE_FILE) - .orElse(submissionSparkConf.get(RESOURCE_STAGING_SERVER_TRUSTSTORE_FILE)) - private val maybeResourceStagingServerInternalTrustStorePassword = - submissionSparkConf.get(RESOURCE_STAGING_SERVER_INTERNAL_TRUSTSTORE_PASSWORD) - .orElse(submissionSparkConf.get(RESOURCE_STAGING_SERVER_TRUSTSTORE_PASSWORD)) - private val maybeResourceStagingServerInternalTrustStoreType = - submissionSparkConf.get(RESOURCE_STAGING_SERVER_INTERNAL_TRUSTSTORE_TYPE) - .orElse(submissionSparkConf.get(RESOURCE_STAGING_SERVER_TRUSTSTORE_TYPE)) - private val maybeResourceStagingServerInternalClientCert = - submissionSparkConf.get(RESOURCE_STAGING_SERVER_INTERNAL_CLIENT_CERT_PEM) - .orElse(submissionSparkConf.get(RESOURCE_STAGING_SERVER_CLIENT_CERT_PEM)) - private val resourceStagingServerInternalSslEnabled = - submissionSparkConf.get(RESOURCE_STAGING_SERVER_INTERNAL_SSL_ENABLED) - .orElse(submissionSparkConf.get(RESOURCE_STAGING_SERVER_SSL_ENABLED)) - .getOrElse(false) - OptionRequirements.requireNandDefined( - maybeResourceStagingServerInternalClientCert, - maybeResourceStagingServerInternalTrustStore, - "Cannot provide both a certificate file and a trustStore file for init-containers to" + - " use for contacting the resource staging server over TLS.") - - require(maybeResourceStagingServerInternalTrustStore.forall { trustStore => - Option(Utils.resolveURI(trustStore).getScheme).getOrElse("file") match { - case "file" | "local" => true - case _ => false - } - }, "TrustStore URI used for contacting the resource staging server from init containers must" + - " have no scheme, or scheme file://, or scheme local://.") - - require(maybeResourceStagingServerInternalClientCert.forall { trustStore => - Option(Utils.resolveURI(trustStore).getScheme).getOrElse("file") match { - case "file" | "local" => true - case _ => false - } - }, "Client cert file URI used for contacting the resource staging server from init containers" + - " must have no scheme, or scheme file://, or scheme local://.") - - def getInitContainerSteps(): Seq[InitContainerStep] = { - val initContainerBootstrap = new SparkPodInitContainerBootstrapImpl( - initContainerImage, - dockerImagePullPolicy, - jarsDownloadPath, - filesDownloadPath, - downloadTimeoutMinutes, - initContainerConfigMapName, - initContainerConfigMapKey) - val baseInitContainerStep = new BaseInitContainerStep( - sparkJars, - sparkFiles, - jarsDownloadPath, - filesDownloadPath, - initContainerConfigMapName, - initContainerConfigMapKey, - initContainerBootstrap) - val submittedResourcesInitContainerStep = resourceStagingServerUri.map { - stagingServerUri => - val mountSecretPlugin = new InitContainerResourceStagingServerSecretPluginImpl( - submittedResourcesSecretName, - INIT_CONTAINER_SECRET_VOLUME_MOUNT_PATH) - val submittedDependencyUploader = new SubmittedDependencyUploaderImpl( - driverLabels, - namespace, - stagingServerUri, - sparkJars, - sparkFiles, - new ResourceStagingServerSslOptionsProviderImpl(submissionSparkConf).getSslOptions, - RetrofitClientFactoryImpl) - new SubmittedResourcesInitContainerStep( - submittedResourcesSecretName, - resourceStagingServerInternalUri.getOrElse(stagingServerUri), - INIT_CONTAINER_SECRET_VOLUME_MOUNT_PATH, - resourceStagingServerInternalSslEnabled, - maybeResourceStagingServerInternalTrustStore, - maybeResourceStagingServerInternalClientCert, - maybeResourceStagingServerInternalTrustStorePassword, - maybeResourceStagingServerInternalTrustStoreType, - submittedDependencyUploader, - mountSecretPlugin) - } - Seq(baseInitContainerStep) ++ submittedResourcesInitContainerStep.toSeq - } -} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/SubmittedResourcesInitContainerStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/SubmittedResourcesInitContainerStep.scala deleted file mode 100644 index 24bd8941c42b0..0000000000000 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/SubmittedResourcesInitContainerStep.scala +++ /dev/null @@ -1,146 +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.submit.submitsteps.initcontainer - -import java.io.File - -import com.google.common.base.Charsets -import com.google.common.io.{BaseEncoding, Files} -import io.fabric8.kubernetes.api.model.{Secret, SecretBuilder} -import scala.collection.JavaConverters._ - -import org.apache.spark.SparkException -import org.apache.spark.deploy.kubernetes.InitContainerResourceStagingServerSecretPlugin -import org.apache.spark.deploy.kubernetes.config._ -import org.apache.spark.deploy.kubernetes.constants._ -import org.apache.spark.deploy.kubernetes.submit.SubmittedDependencyUploader -import org.apache.spark.internal.config.OptionalConfigEntry -import org.apache.spark.util.Utils - -private[spark] class SubmittedResourcesInitContainerStep( - submittedResourcesSecretName: String, - internalResourceStagingServerUri: String, - initContainerSecretMountPath: String, - resourceStagingServerSslEnabled: Boolean, - maybeInternalTrustStoreUri: Option[String], - maybeInternalClientCertUri: Option[String], - maybeInternalTrustStorePassword: Option[String], - maybeInternalTrustStoreType: Option[String], - submittedDependencyUploader: SubmittedDependencyUploader, - submittedResourcesSecretPlugin: InitContainerResourceStagingServerSecretPlugin) - extends InitContainerStep { - - override def prepareInitContainer(initContainerSpec: InitContainerSpec): InitContainerSpec = { - val jarsIdAndSecret = submittedDependencyUploader.uploadJars() - val filesIdAndSecret = submittedDependencyUploader.uploadFiles() - - val submittedResourcesInitContainerProperties = Map[String, String]( - RESOURCE_STAGING_SERVER_URI.key -> internalResourceStagingServerUri, - INIT_CONTAINER_DOWNLOAD_JARS_RESOURCE_IDENTIFIER.key -> jarsIdAndSecret.resourceId, - INIT_CONTAINER_DOWNLOAD_JARS_SECRET_LOCATION.key -> - s"$initContainerSecretMountPath/$INIT_CONTAINER_SUBMITTED_JARS_SECRET_KEY", - INIT_CONTAINER_DOWNLOAD_FILES_RESOURCE_IDENTIFIER.key -> filesIdAndSecret.resourceId, - INIT_CONTAINER_DOWNLOAD_FILES_SECRET_LOCATION.key -> - s"$initContainerSecretMountPath/$INIT_CONTAINER_SUBMITTED_FILES_SECRET_KEY", - RESOURCE_STAGING_SERVER_SSL_ENABLED.key -> resourceStagingServerSslEnabled.toString) ++ - resolveSecretPath( - maybeInternalTrustStoreUri, - INIT_CONTAINER_STAGING_SERVER_TRUSTSTORE_SECRET_KEY, - RESOURCE_STAGING_SERVER_TRUSTSTORE_FILE, - "TrustStore URI") ++ - resolveSecretPath( - maybeInternalClientCertUri, - INIT_CONTAINER_STAGING_SERVER_CLIENT_CERT_SECRET_KEY, - RESOURCE_STAGING_SERVER_CLIENT_CERT_PEM, - "Client certificate URI") ++ - maybeInternalTrustStorePassword.map { password => - (RESOURCE_STAGING_SERVER_TRUSTSTORE_PASSWORD.key, password) - }.toMap ++ - maybeInternalTrustStoreType.map { storeType => - (RESOURCE_STAGING_SERVER_TRUSTSTORE_TYPE.key, storeType) - }.toMap - val initContainerSecret = createResourceStagingServerSecret( - jarsIdAndSecret.resourceSecret, filesIdAndSecret.resourceSecret) - val additionalDriverSparkConf = - Map( - EXECUTOR_INIT_CONTAINER_SECRET.key -> initContainerSecret.getMetadata.getName, - EXECUTOR_INIT_CONTAINER_SECRET_MOUNT_DIR.key -> initContainerSecretMountPath) - val initContainerWithSecretVolumeMount = submittedResourcesSecretPlugin - .mountResourceStagingServerSecretIntoInitContainer(initContainerSpec.initContainer) - val podWithSecretVolume = submittedResourcesSecretPlugin - .addResourceStagingServerSecretVolumeToPod(initContainerSpec.podToInitialize) - initContainerSpec.copy( - initContainer = initContainerWithSecretVolumeMount, - podToInitialize = podWithSecretVolume, - initContainerDependentResources = - initContainerSpec.initContainerDependentResources ++ Seq(initContainerSecret), - initContainerProperties = - initContainerSpec.initContainerProperties ++ submittedResourcesInitContainerProperties, - additionalDriverSparkConf = additionalDriverSparkConf) - } - - private def createResourceStagingServerSecret( - jarsResourceSecret: String, filesResourceSecret: String): Secret = { - val trustStoreBase64 = convertFileToBase64IfSubmitterLocal( - INIT_CONTAINER_STAGING_SERVER_TRUSTSTORE_SECRET_KEY, maybeInternalTrustStoreUri) - val clientCertBase64 = convertFileToBase64IfSubmitterLocal( - INIT_CONTAINER_STAGING_SERVER_CLIENT_CERT_SECRET_KEY, maybeInternalClientCertUri) - val jarsSecretBase64 = BaseEncoding.base64().encode(jarsResourceSecret.getBytes(Charsets.UTF_8)) - val filesSecretBase64 = BaseEncoding.base64().encode( - filesResourceSecret.getBytes(Charsets.UTF_8)) - val secretData = Map( - INIT_CONTAINER_SUBMITTED_JARS_SECRET_KEY -> jarsSecretBase64, - INIT_CONTAINER_SUBMITTED_FILES_SECRET_KEY -> filesSecretBase64) ++ - trustStoreBase64 ++ - clientCertBase64 - val kubernetesSecret = new SecretBuilder() - .withNewMetadata() - .withName(submittedResourcesSecretName) - .endMetadata() - .addToData(secretData.asJava) - .build() - kubernetesSecret - } - - private def convertFileToBase64IfSubmitterLocal(secretKey: String, secretUri: Option[String]) - : Map[String, String] = { - secretUri.filter { trustStore => - Option(Utils.resolveURI(trustStore).getScheme).getOrElse("file") == "file" - }.map { uri => - val file = new File(Utils.resolveURI(uri).getPath) - require(file.isFile, "Dependency server trustStore provided at" + - file.getAbsolutePath + " does not exist or is not a file.") - (secretKey, BaseEncoding.base64().encode(Files.toByteArray(file))) - }.toMap - } - - private def resolveSecretPath( - maybeUri: Option[String], - secretKey: String, - configEntry: OptionalConfigEntry[String], - uriType: String): Map[String, String] = { - maybeUri.map(Utils.resolveURI).map { uri => - val resolvedPath = Option(uri.getScheme).getOrElse("file") match { - case "file" => s"$initContainerSecretMountPath/$secretKey" - case "local" => uri.getPath - case invalid => throw new SparkException(s"$uriType has invalid scheme $invalid must be" + - s" local://, file://, or empty.") - } - (configEntry.key, resolvedPath) - }.toMap - } -} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/KubernetesSparkDependencyDownloadInitContainer.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/KubernetesSparkDependencyDownloadInitContainer.scala index 0e274678ad6f0..ac19c2463218b 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/KubernetesSparkDependencyDownloadInitContainer.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/KubernetesSparkDependencyDownloadInitContainer.scala @@ -76,6 +76,7 @@ private[spark] class KubernetesSparkDependencyDownloadInitContainer( fileFetcher: FileFetcher, resourceStagingServerSslOptions: SSLOptions) extends Logging { + private implicit val downloadExecutor = ExecutionContext.fromExecutorService( ThreadUtils.newDaemonCachedThreadPool("download-executor")) private val maybeResourceStagingServerUri = sparkConf.get(RESOURCE_STAGING_SERVER_URI) diff --git a/resource-managers/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 index fa0ecca3b4ee6..2a0f6e78c2aea 100644 --- a/resource-managers/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 @@ -70,7 +70,8 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit sparkConf.get(INIT_CONTAINER_FILES_DOWNLOAD_LOCATION), sparkConf.get(INIT_CONTAINER_MOUNT_TIMEOUT), configMap, - configMapKey) + configMapKey, + executorInitContainerSecretVolumePlugin) } if (maybeConfigMap.isEmpty) { logWarning("The executor's init-container config map was not specified. Executors will" + @@ -88,11 +89,7 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit Some(new File(Config.KUBERNETES_SERVICE_ACCOUNT_TOKEN_PATH)), Some(new File(Config.KUBERNETES_SERVICE_ACCOUNT_CA_CRT_PATH))) new KubernetesClusterSchedulerBackend( - sc.taskScheduler.asInstanceOf[TaskSchedulerImpl], - sc, - bootStrap, - executorInitContainerSecretVolumePlugin, - kubernetesClient) + sc.taskScheduler.asInstanceOf[TaskSchedulerImpl], sc, bootStrap, kubernetesClient) } override def initialize(scheduler: TaskScheduler, backend: SchedulerBackend): Unit = { 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 a8af8ee61a2c4..d880cee315c0d 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 @@ -23,7 +23,7 @@ import java.util.concurrent.atomic.{AtomicInteger, AtomicLong, AtomicReference} import com.fasterxml.jackson.databind.ObjectMapper import com.fasterxml.jackson.module.scala.DefaultScalaModule -import io.fabric8.kubernetes.api.model.{ContainerBuilder, ContainerPortBuilder, EnvVarBuilder, EnvVarSourceBuilder, Pod, PodBuilder, QuantityBuilder} +import io.fabric8.kubernetes.api.model.{ContainerPortBuilder, EnvVarBuilder, EnvVarSourceBuilder, Pod, PodBuilder, QuantityBuilder} import io.fabric8.kubernetes.client.{KubernetesClient, KubernetesClientException, Watcher} import io.fabric8.kubernetes.client.Watcher.Action import org.apache.commons.io.FilenameUtils @@ -32,10 +32,9 @@ import scala.collection.mutable import scala.concurrent.{ExecutionContext, Future} import org.apache.spark.{SparkContext, SparkEnv, SparkException} -import org.apache.spark.deploy.kubernetes.{ConfigurationUtils, InitContainerResourceStagingServerSecretPlugin, PodWithDetachedInitContainer, SparkPodInitContainerBootstrap} +import org.apache.spark.deploy.kubernetes.{ConfigurationUtils, SparkPodInitContainerBootstrap} import org.apache.spark.deploy.kubernetes.config._ import org.apache.spark.deploy.kubernetes.constants._ -import org.apache.spark.deploy.kubernetes.submit.InitContainerUtil import org.apache.spark.network.netty.SparkTransportConf import org.apache.spark.network.shuffle.kubernetes.KubernetesExternalShuffleClient import org.apache.spark.rpc.{RpcCallContext, RpcEndpointAddress, RpcEnv} @@ -48,7 +47,6 @@ private[spark] class KubernetesClusterSchedulerBackend( scheduler: TaskSchedulerImpl, val sc: SparkContext, executorInitContainerBootstrap: Option[SparkPodInitContainerBootstrap], - executorMountInitContainerSecretPlugin: Option[InitContainerResourceStagingServerSecretPlugin], kubernetesClient: KubernetesClient) extends CoarseGrainedSchedulerBackend(scheduler, sc.env.rpcEnv) { @@ -321,8 +319,8 @@ private[spark] class KubernetesClusterSchedulerBackend( nodeToLocalTaskCount.toMap[String, Int] } - private def addNodeAffinityAnnotationIfUseful( - baseExecutorPod: Pod, nodeToTaskCount: Map[String, Int]): Pod = { + private def addNodeAffinityAnnotationIfUseful(basePodBuilder: PodBuilder, + nodeToTaskCount: Map[String, Int]): PodBuilder = { def scaleToRange(value: Int, baseMin: Double, baseMax: Double, rangeMin: Double, rangeMax: Double): Int = (((rangeMax - rangeMin) * (value - baseMin) / (baseMax - baseMin)) + rangeMin).toInt @@ -343,12 +341,11 @@ private[spark] class KubernetesClusterSchedulerBackend( ))) // TODO: Use non-annotation syntax when we switch to K8s version 1.6. logDebug(s"Adding nodeAffinity as annotation $nodeAffinityJson") - new PodBuilder(baseExecutorPod).editMetadata() + basePodBuilder.editMetadata() .addToAnnotations(ANNOTATION_EXECUTOR_NODE_AFFINITY, nodeAffinityJson) .endMetadata() - .build() } else { - baseExecutorPod + basePodBuilder } } @@ -419,22 +416,7 @@ private[spark] class KubernetesClusterSchedulerBackend( .build() }) - val executorContainer = new ContainerBuilder() - .withName(s"executor") - .withImage(executorDockerImage) - .withImagePullPolicy(dockerImagePullPolicy) - .withNewResources() - .addToRequests("memory", executorMemoryQuantity) - .addToLimits("memory", executorMemoryLimitQuantity) - .addToRequests("cpu", executorCpuQuantity) - .endResources() - .addAllToEnv(requiredEnv.asJava) - .addToEnv(executorExtraClasspathEnv.toSeq: _*) - .withPorts(requiredPorts.asJava) - .build() - - - val executorPod = new PodBuilder() + val basePodBuilder = new PodBuilder() .withNewMetadata() .withName(name) .withLabels(resolvedExecutorLabels.asJava) @@ -450,77 +432,69 @@ private[spark] class KubernetesClusterSchedulerBackend( .endMetadata() .withNewSpec() .withHostname(hostname) + .addNewContainer() + .withName(s"executor") + .withImage(executorDockerImage) + .withImagePullPolicy(dockerImagePullPolicy) + .withNewResources() + .addToRequests("memory", executorMemoryQuantity) + .addToLimits("memory", executorMemoryLimitQuantity) + .addToRequests("cpu", executorCpuQuantity) + .endResources() + .addAllToEnv(requiredEnv.asJava) + .addToEnv(executorExtraClasspathEnv.toSeq: _*) + .withPorts(requiredPorts.asJava) + .endContainer() .endSpec() - .build() - val containerWithExecutorLimitCores = executorLimitCores.map { + executorLimitCores.map { limitCores => val executorCpuLimitQuantity = new QuantityBuilder(false) .withAmount(limitCores) .build() - new ContainerBuilder(executorContainer) - .editResources() - .addToLimits("cpu", executorCpuLimitQuantity) - .endResources() - .build() - }.getOrElse(executorContainer) - - val withMaybeShuffleConfigExecutorContainer = shuffleServiceConfig.map { config => - config.shuffleDirs.foldLeft(containerWithExecutorLimitCores) { (container, dir) => - new ContainerBuilder(container) - .addNewVolumeMount() - .withName(FilenameUtils.getBaseName(dir)) - .withMountPath(dir) - .endVolumeMount() - .build() - } - }.getOrElse(containerWithExecutorLimitCores) - val withMaybeShuffleConfigPod = shuffleServiceConfig.map { config => - config.shuffleDirs.foldLeft(executorPod) { (builder, dir) => - new PodBuilder(builder) + basePodBuilder .editSpec() - .addNewVolume() - .withName(FilenameUtils.getBaseName(dir)) - .withNewHostPath() - .withPath(dir) + .editFirstContainer() + .editResources + .addToLimits("cpu", executorCpuLimitQuantity) + .endResources() + .endContainer() + .endSpec() + } + + val withMaybeShuffleConfigPodBuilder = shuffleServiceConfig + .map { config => + config.shuffleDirs.foldLeft(basePodBuilder) { (builder, dir) => + builder + .editSpec() + .addNewVolume() + .withName(FilenameUtils.getBaseName(dir)) + .withNewHostPath() + .withPath(dir) .endHostPath() .endVolume() + .editFirstContainer() + .addNewVolumeMount() + .withName(FilenameUtils.getBaseName(dir)) + .withMountPath(dir) + .endVolumeMount() + .endContainer() .endSpec() - .build() - } - }.getOrElse(executorPod) - val (executorPodWithInitContainer, initBootstrappedExecutorContainer) = - executorInitContainerBootstrap.map { bootstrap => - val podWithDetachedInitContainer = bootstrap.bootstrapInitContainerAndVolumes( - PodWithDetachedInitContainer( - withMaybeShuffleConfigPod, - new ContainerBuilder().build(), - withMaybeShuffleConfigExecutorContainer)) - - val resolvedInitContainer = executorMountInitContainerSecretPlugin.map { plugin => - plugin.mountResourceStagingServerSecretIntoInitContainer( - podWithDetachedInitContainer.initContainer) - }.getOrElse(podWithDetachedInitContainer.initContainer) - - val podWithAttachedInitContainer = InitContainerUtil.appendInitContainer( - podWithDetachedInitContainer.pod, resolvedInitContainer) - - val resolvedPodWithMountedSecret = executorMountInitContainerSecretPlugin.map { plugin => - plugin.addResourceStagingServerSecretVolumeToPod(podWithAttachedInitContainer) - }.getOrElse(podWithAttachedInitContainer) - - (resolvedPodWithMountedSecret, podWithDetachedInitContainer.mainContainer) - }.getOrElse((withMaybeShuffleConfigPod, withMaybeShuffleConfigExecutorContainer)) - - val executorPodWithNodeAffinity = addNodeAffinityAnnotationIfUseful( - executorPodWithInitContainer, nodeToLocalTaskCount) - val resolvedExecutorPod = new PodBuilder(executorPodWithNodeAffinity) - .editSpec() - .addToContainers(initBootstrappedExecutorContainer) - .endSpec() - .build() + } + }.getOrElse(basePodBuilder) + + val executorInitContainerPodBuilder = executorInitContainerBootstrap.map { + bootstrap => + bootstrap.bootstrapInitContainerAndVolumes( + "executor", + withMaybeShuffleConfigPodBuilder) + }.getOrElse(withMaybeShuffleConfigPodBuilder) + + val resolvedExecutorPodBuilder = addNodeAffinityAnnotationIfUseful( + executorInitContainerPodBuilder, nodeToLocalTaskCount) + try { - (executorId, kubernetesClient.pods.create(resolvedExecutorPod)) + (executorId, kubernetesClient.pods.create(resolvedExecutorPodBuilder.build())) } catch { case throwable: Throwable => logError("Failed to allocate executor pod.", throwable) @@ -632,11 +606,10 @@ private[spark] class KubernetesClusterSchedulerBackend( } } -} -case class ShuffleServiceConfig( - shuffleNamespace: String, + case class ShuffleServiceConfig(shuffleNamespace: String, shuffleLabels: Map[String, String], shuffleDirs: Seq[String]) +} private object KubernetesClusterSchedulerBackend { private val DEFAULT_STATIC_PORT = 10000 diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/SparkPodInitContainerBootstrapSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/SparkPodInitContainerBootstrapSuite.scala index ba79736d9c7ba..90d7b10df211c 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/SparkPodInitContainerBootstrapSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/SparkPodInitContainerBootstrapSuite.scala @@ -16,140 +16,162 @@ */ package org.apache.spark.deploy.kubernetes +import com.fasterxml.jackson.databind.ObjectMapper +import io.fabric8.kubernetes.api.model.{Container, ContainerBuilder, Pod, PodBuilder} import org.scalatest.BeforeAndAfter +import scala.collection.JavaConverters._ import org.apache.spark.SparkFunSuite +import org.apache.spark.deploy.kubernetes.constants._ class SparkPodInitContainerBootstrapSuite extends SparkFunSuite with BeforeAndAfter { -// private val OBJECT_MAPPER = new ObjectMapper() -// private val INIT_CONTAINER_IMAGE = "spark-init:latest" -// private val DOCKER_IMAGE_PULL_POLICY = "IfNotPresent" -// private val JARS_DOWNLOAD_PATH = "/var/data/spark-jars" -// private val FILES_DOWNLOAD_PATH = "/var/data/spark-files" -// private val DOWNLOAD_TIMEOUT_MINUTES = 5 -// private val INIT_CONTAINER_CONFIG_MAP_NAME = "spark-init-config-map" -// private val INIT_CONTAINER_CONFIG_MAP_KEY = "spark-init-config-map-key" -// private val ADDED_SUBMITTED_DEPENDENCY_ENV = "ADDED_SUBMITTED_DEPENDENCY" -// private val ADDED_SUBMITTED_DEPENDENCY_ANNOTATION = "added-submitted-dependencies" -// private val MAIN_CONTAINER_NAME = "spark-main" -// private val TRUE = "true" -// -// test("Running without submitted dependencies adds init-container with volume mounts.") { -// val bootstrappedPod = bootstrapPodWithoutSubmittedDependencies() -// val podAnnotations = bootstrappedPod.getMetadata.getAnnotations.asScala -// assert(podAnnotations.contains(INIT_CONTAINER_ANNOTATION)) -// val initContainers = OBJECT_MAPPER.readValue( -// podAnnotations(INIT_CONTAINER_ANNOTATION), classOf[Array[Container]]) -// assert(initContainers.length === 1) -// val initContainer = initContainers.head -// val initContainerVolumeMounts = initContainer.getVolumeMounts.asScala.map { -// mount => (mount.getName, mount.getMountPath) -// }.toMap -// val expectedInitContainerVolumeMounts = Map( -// INIT_CONTAINER_PROPERTIES_FILE_VOLUME -> INIT_CONTAINER_PROPERTIES_FILE_DIR, -// INIT_CONTAINER_DOWNLOAD_JARS_VOLUME_NAME -> JARS_DOWNLOAD_PATH, -// INIT_CONTAINER_DOWNLOAD_FILES_VOLUME_NAME -> FILES_DOWNLOAD_PATH) -// assert(initContainerVolumeMounts === expectedInitContainerVolumeMounts) -// assert(initContainer.getName === "spark-init") -// assert(initContainer.getImage === INIT_CONTAINER_IMAGE) -// assert(initContainer.getImagePullPolicy === "IfNotPresent") -// assert(initContainer.getArgs.asScala === List(INIT_CONTAINER_PROPERTIES_FILE_PATH)) -// } -// -// test("Running without submitted dependencies adds volume mounts to main container.") { -// val bootstrappedPod = bootstrapPodWithoutSubmittedDependencies() -// val containers = bootstrappedPod.getSpec.getContainers.asScala -// val mainContainer = containers.find(_.getName === MAIN_CONTAINER_NAME) -// assert(mainContainer.isDefined) -// val volumeMounts = mainContainer.map(_.getVolumeMounts.asScala).toSeq.flatten.map { -// mount => (mount.getName, mount.getMountPath) -// }.toMap -// val expectedVolumeMounts = Map( -// INIT_CONTAINER_DOWNLOAD_JARS_VOLUME_NAME -> JARS_DOWNLOAD_PATH, -// INIT_CONTAINER_DOWNLOAD_FILES_VOLUME_NAME -> FILES_DOWNLOAD_PATH) -// assert(volumeMounts === expectedVolumeMounts) -// } -// -// test("Running without submitted dependencies adds volumes to the pod") { -// val bootstrappedPod = bootstrapPodWithoutSubmittedDependencies() -// val podVolumes = bootstrappedPod.getSpec.getVolumes.asScala -// assert(podVolumes.size === 3) -// assert(podVolumes.exists { volume => -// volume.getName == INIT_CONTAINER_PROPERTIES_FILE_VOLUME && -// Option(volume.getConfigMap).map { configMap => -// configMap.getItems.asScala.map { -// keyToPath => (keyToPath.getKey, keyToPath.getPath) -// }.toMap -// }.contains(Map(INIT_CONTAINER_CONFIG_MAP_KEY -> INIT_CONTAINER_PROPERTIES_FILE_NAME)) -// }) -// assert(podVolumes.exists { volume => -// volume.getName == INIT_CONTAINER_DOWNLOAD_JARS_VOLUME_NAME && volume.getEmptyDir != null -// }) -// assert(podVolumes.exists { volume => -// volume.getName == INIT_CONTAINER_DOWNLOAD_FILES_VOLUME_NAME && volume.getEmptyDir != null -// }) -// } -// -// test("Files download path is set as environment variable") { -// val bootstrappedPod = bootstrapPodWithoutSubmittedDependencies() -// val containers = bootstrappedPod.getSpec.getContainers.asScala -// val maybeMainContainer = containers.find(_.getName === MAIN_CONTAINER_NAME) -// assert(maybeMainContainer.exists { mainContainer => -// mainContainer.getEnv.asScala.exists(envVar => -// envVar.getName == ENV_MOUNTED_FILES_DIR && envVar.getValue == FILES_DOWNLOAD_PATH) -// }) -// } -// -// test("Running with submitted dependencies modifies the init container with the plugin.") { -// val bootstrappedPod = bootstrapPodWithSubmittedDependencies() -// val podAnnotations = bootstrappedPod.getMetadata.getAnnotations.asScala -// assert(podAnnotations(ADDED_SUBMITTED_DEPENDENCY_ANNOTATION) === TRUE) -// val initContainers = OBJECT_MAPPER.readValue( -// podAnnotations(INIT_CONTAINER_ANNOTATION), classOf[Array[Container]]) -// assert(initContainers.length === 1) -// val initContainer = initContainers.head -// assert(initContainer.getEnv.asScala.exists { -// env => env.getName === ADDED_SUBMITTED_DEPENDENCY_ENV && env.getValue === TRUE -// }) -// } -// -// private def bootstrapPodWithoutSubmittedDependencies(): Pod = { -// val bootstrapUnderTest = new SparkPodInitContainerBootstrapImpl( -// INIT_CONTAINER_IMAGE, -// DOCKER_IMAGE_PULL_POLICY, -// JARS_DOWNLOAD_PATH, -// FILES_DOWNLOAD_PATH, -// DOWNLOAD_TIMEOUT_MINUTES, -// INIT_CONTAINER_CONFIG_MAP_NAME, -// INIT_CONTAINER_CONFIG_MAP_KEY, -// None) -// bootstrapUnderTest.bootstrapInitContainerAndVolumes( -// basePod()).build() -// } -// -// private def bootstrapPodWithSubmittedDependencies(): Pod = { -// val bootstrapUnderTest = new SparkPodInitContainerBootstrapImpl( -// INIT_CONTAINER_IMAGE, -// DOCKER_IMAGE_PULL_POLICY, -// JARS_DOWNLOAD_PATH, -// FILES_DOWNLOAD_PATH, -// DOWNLOAD_TIMEOUT_MINUTES, -// INIT_CONTAINER_CONFIG_MAP_NAME, -// INIT_CONTAINER_CONFIG_MAP_KEY, -// Some(submittedDependencyPlugin)) -// bootstrapUnderTest.bootstrapInitContainerAndVolumes( -// MAIN_CONTAINER_NAME, basePod()).build() -// } -// -// private def basePod(): PodBuilder = { -// new PodBuilder() -// .withNewMetadata() -// .withName("spark-pod") -// .endMetadata() -// .withNewSpec() -// .addNewContainer() -// .withName(MAIN_CONTAINER_NAME) -// .endContainer() -// .endSpec() -// } + private val OBJECT_MAPPER = new ObjectMapper() + private val INIT_CONTAINER_IMAGE = "spark-init:latest" + private val DOCKER_IMAGE_PULL_POLICY = "IfNotPresent" + private val JARS_DOWNLOAD_PATH = "/var/data/spark-jars" + private val FILES_DOWNLOAD_PATH = "/var/data/spark-files" + private val DOWNLOAD_TIMEOUT_MINUTES = 5 + private val INIT_CONTAINER_CONFIG_MAP_NAME = "spark-init-config-map" + private val INIT_CONTAINER_CONFIG_MAP_KEY = "spark-init-config-map-key" + private val ADDED_SUBMITTED_DEPENDENCY_ENV = "ADDED_SUBMITTED_DEPENDENCY" + private val ADDED_SUBMITTED_DEPENDENCY_ANNOTATION = "added-submitted-dependencies" + private val MAIN_CONTAINER_NAME = "spark-main" + private val TRUE = "true" + + private val submittedDependencyPlugin = new InitContainerResourceStagingServerSecretPlugin { + override def addResourceStagingServerSecretVolumeToPod(basePod: PodBuilder) + : PodBuilder = { + basePod.editMetadata() + .addToAnnotations(ADDED_SUBMITTED_DEPENDENCY_ANNOTATION, TRUE) + .endMetadata() + } + + override def mountResourceStagingServerSecretIntoInitContainer(container: ContainerBuilder) + : ContainerBuilder = { + container + .addNewEnv() + .withName(ADDED_SUBMITTED_DEPENDENCY_ENV) + .withValue(TRUE) + .endEnv() + } + } + + test("Running without submitted dependencies adds init-container with volume mounts.") { + val bootstrappedPod = bootstrapPodWithoutSubmittedDependencies() + val podAnnotations = bootstrappedPod.getMetadata.getAnnotations.asScala + assert(podAnnotations.contains(INIT_CONTAINER_ANNOTATION)) + val initContainers = OBJECT_MAPPER.readValue( + podAnnotations(INIT_CONTAINER_ANNOTATION), classOf[Array[Container]]) + assert(initContainers.length === 1) + val initContainer = initContainers.head + val initContainerVolumeMounts = initContainer.getVolumeMounts.asScala.map { + mount => (mount.getName, mount.getMountPath) + }.toMap + val expectedInitContainerVolumeMounts = Map( + INIT_CONTAINER_PROPERTIES_FILE_VOLUME -> INIT_CONTAINER_PROPERTIES_FILE_DIR, + INIT_CONTAINER_DOWNLOAD_JARS_VOLUME_NAME -> JARS_DOWNLOAD_PATH, + INIT_CONTAINER_DOWNLOAD_FILES_VOLUME_NAME -> FILES_DOWNLOAD_PATH) + assert(initContainerVolumeMounts === expectedInitContainerVolumeMounts) + assert(initContainer.getName === "spark-init") + assert(initContainer.getImage === INIT_CONTAINER_IMAGE) + assert(initContainer.getImagePullPolicy === "IfNotPresent") + assert(initContainer.getArgs.asScala === List(INIT_CONTAINER_PROPERTIES_FILE_PATH)) + } + + test("Running without submitted dependencies adds volume mounts to main container.") { + val bootstrappedPod = bootstrapPodWithoutSubmittedDependencies() + val containers = bootstrappedPod.getSpec.getContainers.asScala + val mainContainer = containers.find(_.getName === MAIN_CONTAINER_NAME) + assert(mainContainer.isDefined) + val volumeMounts = mainContainer.map(_.getVolumeMounts.asScala).toSeq.flatten.map { + mount => (mount.getName, mount.getMountPath) + }.toMap + val expectedVolumeMounts = Map( + INIT_CONTAINER_DOWNLOAD_JARS_VOLUME_NAME -> JARS_DOWNLOAD_PATH, + INIT_CONTAINER_DOWNLOAD_FILES_VOLUME_NAME -> FILES_DOWNLOAD_PATH) + assert(volumeMounts === expectedVolumeMounts) + } + + test("Running without submitted dependencies adds volumes to the pod") { + val bootstrappedPod = bootstrapPodWithoutSubmittedDependencies() + val podVolumes = bootstrappedPod.getSpec.getVolumes.asScala + assert(podVolumes.size === 3) + assert(podVolumes.exists { volume => + volume.getName == INIT_CONTAINER_PROPERTIES_FILE_VOLUME && + Option(volume.getConfigMap).map { configMap => + configMap.getItems.asScala.map { + keyToPath => (keyToPath.getKey, keyToPath.getPath) + }.toMap + }.contains(Map(INIT_CONTAINER_CONFIG_MAP_KEY -> INIT_CONTAINER_PROPERTIES_FILE_NAME)) + }) + assert(podVolumes.exists { volume => + volume.getName == INIT_CONTAINER_DOWNLOAD_JARS_VOLUME_NAME && volume.getEmptyDir != null + }) + assert(podVolumes.exists { volume => + volume.getName == INIT_CONTAINER_DOWNLOAD_FILES_VOLUME_NAME && volume.getEmptyDir != null + }) + } + + test("Files download path is set as environment variable") { + val bootstrappedPod = bootstrapPodWithoutSubmittedDependencies() + val containers = bootstrappedPod.getSpec.getContainers.asScala + val maybeMainContainer = containers.find(_.getName === MAIN_CONTAINER_NAME) + assert(maybeMainContainer.exists { mainContainer => + mainContainer.getEnv.asScala.exists(envVar => + envVar.getName == ENV_MOUNTED_FILES_DIR && envVar.getValue == FILES_DOWNLOAD_PATH) + }) + } + + test("Running with submitted dependencies modifies the init container with the plugin.") { + val bootstrappedPod = bootstrapPodWithSubmittedDependencies() + val podAnnotations = bootstrappedPod.getMetadata.getAnnotations.asScala + assert(podAnnotations(ADDED_SUBMITTED_DEPENDENCY_ANNOTATION) === TRUE) + val initContainers = OBJECT_MAPPER.readValue( + podAnnotations(INIT_CONTAINER_ANNOTATION), classOf[Array[Container]]) + assert(initContainers.length === 1) + val initContainer = initContainers.head + assert(initContainer.getEnv.asScala.exists { + env => env.getName === ADDED_SUBMITTED_DEPENDENCY_ENV && env.getValue === TRUE + }) + } + + private def bootstrapPodWithoutSubmittedDependencies(): Pod = { + val bootstrapUnderTest = new SparkPodInitContainerBootstrapImpl( + INIT_CONTAINER_IMAGE, + DOCKER_IMAGE_PULL_POLICY, + JARS_DOWNLOAD_PATH, + FILES_DOWNLOAD_PATH, + DOWNLOAD_TIMEOUT_MINUTES, + INIT_CONTAINER_CONFIG_MAP_NAME, + INIT_CONTAINER_CONFIG_MAP_KEY, + None) + bootstrapUnderTest.bootstrapInitContainerAndVolumes( + MAIN_CONTAINER_NAME, basePod()).build() + } + + private def bootstrapPodWithSubmittedDependencies(): Pod = { + val bootstrapUnderTest = new SparkPodInitContainerBootstrapImpl( + INIT_CONTAINER_IMAGE, + DOCKER_IMAGE_PULL_POLICY, + JARS_DOWNLOAD_PATH, + FILES_DOWNLOAD_PATH, + DOWNLOAD_TIMEOUT_MINUTES, + INIT_CONTAINER_CONFIG_MAP_NAME, + INIT_CONTAINER_CONFIG_MAP_KEY, + Some(submittedDependencyPlugin)) + bootstrapUnderTest.bootstrapInitContainerAndVolumes( + MAIN_CONTAINER_NAME, basePod()).build() + } + + private def basePod(): PodBuilder = { + new PodBuilder() + .withNewMetadata() + .withName("spark-pod") + .endMetadata() + .withNewSpec() + .addNewContainer() + .withName(MAIN_CONTAINER_NAME) + .endContainer() + .endSpec() + } } diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/SubmittedDependencyInitContainerVolumesPluginSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/SubmittedDependencyInitContainerVolumesPluginSuite.scala new file mode 100644 index 0000000000000..473d369c8eca3 --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/SubmittedDependencyInitContainerVolumesPluginSuite.scala @@ -0,0 +1,60 @@ +/* + * 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 io.fabric8.kubernetes.api.model.{ContainerBuilder, PodBuilder} +import scala.collection.JavaConverters._ + +import org.apache.spark.SparkFunSuite +import org.apache.spark.deploy.kubernetes.constants._ + +class SubmittedDependencyInitContainerVolumesPluginSuite extends SparkFunSuite { + + private val SECRET_NAME = "secret" + private val SECRET_MOUNT_PATH = "/mnt/secrets" + private val plugin = new InitContainerResourceStagingServerSecretPluginImpl( + SECRET_NAME, SECRET_MOUNT_PATH) + + test("The init container should have the secret volume mount.") { + val baseInitContainer = new ContainerBuilder().withName("container") + val configuredInitContainer = plugin.mountResourceStagingServerSecretIntoInitContainer( + baseInitContainer).build() + val volumeMounts = configuredInitContainer.getVolumeMounts.asScala + assert(volumeMounts.size === 1) + assert(volumeMounts.exists { volumeMount => + volumeMount.getName === INIT_CONTAINER_SECRET_VOLUME_NAME && + volumeMount.getMountPath === SECRET_MOUNT_PATH + }) + } + + test("The pod should have the secret volume.") { + val basePod = new PodBuilder() + .withNewMetadata().withName("pod").endMetadata() + .withNewSpec() + .addNewContainer() + .withName("container") + .endContainer() + .endSpec() + val configuredPod = plugin.addResourceStagingServerSecretVolumeToPod(basePod).build() + val volumes = configuredPod.getSpec.getVolumes.asScala + assert(volumes.size === 1) + assert(volumes.exists { volume => + volume.getName === INIT_CONTAINER_SECRET_VOLUME_NAME && + Option(volume.getSecret).map(_.getSecretName).contains(SECRET_NAME) + }) + } +} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/ClientV2Suite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/ClientV2Suite.scala index 22924df49d4ee..a58a37691f4eb 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/ClientV2Suite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/ClientV2Suite.scala @@ -16,12 +16,543 @@ */ package org.apache.spark.deploy.kubernetes.submit -import org.scalatest.BeforeAndAfter +import java.io.File -import org.apache.spark.SparkFunSuite +import io.fabric8.kubernetes.api.model._ +import io.fabric8.kubernetes.client.{KubernetesClient, Watch} +import io.fabric8.kubernetes.client.dsl.{MixedOperation, NamespaceListVisitFromServerGetDeleteRecreateWaitApplicable, PodResource} +import org.hamcrest.{BaseMatcher, Description} +import org.mockito.{AdditionalAnswers, ArgumentCaptor, Mock, MockitoAnnotations} +import org.mockito.Matchers.{any, anyVararg, argThat, eq => mockitoEq} +import org.mockito.Mockito.{times, verify, when} +import org.mockito.invocation.InvocationOnMock +import org.mockito.stubbing.Answer +import org.scalatest.{BeforeAndAfter, Matchers} + +import scala.collection.JavaConverters._ +import scala.collection.mutable +import org.apache.spark.{SecurityManager, SparkConf, SparkFunSuite} +import org.apache.spark.deploy.kubernetes.{KubernetesExternalShuffleService, KubernetesShuffleBlockHandler, SparkPodInitContainerBootstrap} +import org.apache.spark.deploy.kubernetes.config._ +import org.apache.spark.deploy.kubernetes.constants._ +import org.apache.spark.network.netty.SparkTransportConf +import org.apache.spark.network.shuffle.kubernetes.KubernetesExternalShuffleClient class ClientV2Suite extends SparkFunSuite with BeforeAndAfter { - // TODO + private val JARS_RESOURCE = SubmittedResourceIdAndSecret("jarsId", "jarsSecret") + private val FILES_RESOURCE = SubmittedResourceIdAndSecret("filesId", "filesSecret") + private val SUBMITTED_RESOURCES = SubmittedResources(JARS_RESOURCE, FILES_RESOURCE) + private val BOOTSTRAPPED_POD_ANNOTATION = "bootstrapped" + private val TRUE = "true" + private val APP_NAME = "spark-test" + private val APP_RESOURCE_PREFIX = "spark-prefix" + private val APP_ID = "spark-id" + private val CUSTOM_LABEL_KEY = "customLabel" + private val CUSTOM_LABEL_VALUE = "customLabelValue" + private val DEPRECATED_CUSTOM_LABEL_KEY = "deprecatedCustomLabel" + private val DEPRECATED_CUSTOM_LABEL_VALUE = "deprecatedCustomLabelValue" + private val ALL_EXPECTED_LABELS = Map( + CUSTOM_LABEL_KEY -> CUSTOM_LABEL_VALUE, + DEPRECATED_CUSTOM_LABEL_KEY -> DEPRECATED_CUSTOM_LABEL_VALUE, + SPARK_APP_ID_LABEL -> APP_ID, + SPARK_ROLE_LABEL -> SPARK_POD_DRIVER_ROLE) + private val CUSTOM_ANNOTATION_KEY = "customAnnotation" + private val CUSTOM_ANNOTATION_VALUE = "customAnnotationValue" + private val DEPRECATED_CUSTOM_ANNOTATION_KEY = "deprecatedCustomAnnotation" + private val DEPRECATED_CUSTOM_ANNOTATION_VALUE = "deprecatedCustomAnnotationValue" + private val INIT_CONTAINER_SECRET_NAME = "init-container-secret" + private val INIT_CONTAINER_SECRET_DATA = Map("secret-key" -> "secret-data") + private val MAIN_CLASS = "org.apache.spark.examples.SparkPi" + private val PYSPARK_APP_ARGS = Array(null, "500") + private val APP_ARGS = Array("3", "20") + private val SPARK_JARS = Seq( + "hdfs://localhost:9000/app/jars/jar1.jar", "file:///app/jars/jar2.jar") + private val RESOLVED_SPARK_JARS = Seq( + "hdfs://localhost:9000/app/jars/jar1.jar", "file:///var/data/spark-jars/jar2.jar") + private val RESOLVED_SPARK_REMOTE_AND_LOCAL_JARS = Seq( + "/var/data/spark-jars/jar1.jar", "/var/data/spark-jars/jar2.jar") + private val SPARK_FILES = Seq( + "hdfs://localhost:9000/app/files/file1.txt", "file:///app/files/file2.txt") + private val PYSPARK_FILES = Seq( + "hdfs://localhost:9000/app/files/file1.py", + "file:///app/files/file2.py", + "local:///app/files/file3.py", + "http://app/files/file4.py", + "file:///app/files/file5.py") + private val RESOLVED_PYSPARK_FILES = Seq( + "hdfs://localhost:9000/app/files/file1.py", + "/var/spark-data/spark-files/file2.py", + "local:///app/files/file3.py", + "http://app/files/file4.py") + private val PYSPARK_PRIMARY_FILE = "file:///app/files/file5.py" + private val RESOLVED_PYSPARK_PRIMARY_FILE = "/var/spark-data/spark-file/file5.py" + + private val RESOLVED_SPARK_FILES = Seq( + "hdfs://localhost:9000/app/files/file1.txt", "file:///var/data/spark-files/file2.txt") + private val INIT_CONTAINER_SECRET = new SecretBuilder() + .withNewMetadata() + .withName(INIT_CONTAINER_SECRET_NAME) + .endMetadata() + .withData(INIT_CONTAINER_SECRET_DATA.asJava) + .build() + private val CUSTOM_JAVA_OPTION_KEY = "myappoption" + private val CUSTOM_JAVA_OPTION_VALUE = "myappoptionvalue" + private val DRIVER_JAVA_OPTIONS = s"-D$CUSTOM_JAVA_OPTION_KEY=$CUSTOM_JAVA_OPTION_VALUE" + private val DRIVER_EXTRA_CLASSPATH = "/var/data/spark-app-custom/custom-jar.jar" + private val CONFIG_MAP_NAME = "config-map" + private val CONFIG_MAP_DATA = Map("config-map-key" -> "config-map-data") + private val INIT_CONTAINER_CONFIG_MAP = new ConfigMapBuilder() + .withNewMetadata() + .withName(CONFIG_MAP_NAME) + .endMetadata() + .withData(CONFIG_MAP_DATA.asJava) + .build() + private val CUSTOM_DRIVER_IMAGE = "spark-custom-driver:latest" + private val DRIVER_MEMORY_MB = 512 + private val DRIVER_MEMORY_OVERHEAD_MB = 128 + private val SPARK_CONF = new SparkConf(true) + .set(DRIVER_DOCKER_IMAGE, CUSTOM_DRIVER_IMAGE) + .set(org.apache.spark.internal.config.DRIVER_MEMORY, DRIVER_MEMORY_MB.toLong) + .set(KUBERNETES_DRIVER_MEMORY_OVERHEAD, DRIVER_MEMORY_OVERHEAD_MB.toLong) + .set(KUBERNETES_DRIVER_LABELS, s"$DEPRECATED_CUSTOM_LABEL_KEY=$DEPRECATED_CUSTOM_LABEL_VALUE") + .set(KUBERNETES_DRIVER_ANNOTATIONS, + s"$DEPRECATED_CUSTOM_ANNOTATION_KEY=$DEPRECATED_CUSTOM_ANNOTATION_VALUE") + .set(s"$KUBERNETES_DRIVER_LABEL_PREFIX$CUSTOM_LABEL_KEY", CUSTOM_LABEL_VALUE) + .set(s"$KUBERNETES_DRIVER_ANNOTATION_PREFIX$CUSTOM_ANNOTATION_KEY", CUSTOM_ANNOTATION_VALUE) + .set(org.apache.spark.internal.config.DRIVER_CLASS_PATH, DRIVER_EXTRA_CLASSPATH) + .set(org.apache.spark.internal.config.DRIVER_JAVA_OPTIONS, DRIVER_JAVA_OPTIONS) + private val EXECUTOR_INIT_CONF_KEY = "executor-init-conf" + private val SPARK_CONF_WITH_EXECUTOR_INIT_CONF = SPARK_CONF.clone() + .set(EXECUTOR_INIT_CONF_KEY, TRUE) + private val DRIVER_POD_UID = "driver-pod-uid" + private val DRIVER_POD_KIND = "pod" + private val DRIVER_POD_API_VERSION = "v1" + private val CREDENTIALS_SECRET_NAME = "credentials-secret" + private val CREDENTIALS_SECRET_DATA = Map("credentials-secret-key" -> "credentials-secret-value") + private val CREDENTIALS_SECRET = new SecretBuilder() + .withNewMetadata() + .withName(CREDENTIALS_SECRET_NAME) + .endMetadata() + .withData(CREDENTIALS_SECRET_DATA.asJava) + .build() + private val CREDENTIALS_SET_CONF = "spark.kubernetes.driverCredentials.provided" + private val CREDENTIALS_SET_ANNOTATION = "credentials-set" + + @Mock + private var containerLocalizedFilesResolver: ContainerLocalizedFilesResolver = _ + @Mock + private var executorInitContainerConfiguration: ExecutorInitContainerConfiguration = _ + @Mock + private var submittedDependencyUploader: SubmittedDependencyUploader = _ + @Mock + private var submittedDependenciesSecretBuilder: SubmittedDependencySecretBuilder = _ + @Mock + private var initContainerBootstrap: SparkPodInitContainerBootstrap = _ + @Mock + private var initContainerComponentsProvider: DriverInitContainerComponentsProvider = _ + @Mock + private var kubernetesClient: KubernetesClient = _ + @Mock + private var podOps: MixedOperation[ + Pod, PodList, DoneablePod, PodResource[Pod, DoneablePod]] = _ + private type ResourceListOps = NamespaceListVisitFromServerGetDeleteRecreateWaitApplicable[ + HasMetadata, java.lang.Boolean] + @Mock + private var resourceListOps: ResourceListOps = _ + @Mock + private var credentialsMounterProvider: DriverPodKubernetesCredentialsMounterProvider = _ + @Mock + private var fileMounter: DriverPodKubernetesFileMounter = _ + @Mock + private var credentialsMounter: DriverPodKubernetesCredentialsMounter = _ + @Mock + private var loggingPodStatusWatcher: LoggingPodStatusWatcher = _ + @Mock + private var namedPodResource: PodResource[Pod, DoneablePod] = _ + @Mock + private var watch: Watch = _ + + before { + MockitoAnnotations.initMocks(this) + when(initContainerComponentsProvider.provideInitContainerBootstrap()) + .thenReturn(initContainerBootstrap) + when(submittedDependencyUploader.uploadJars()).thenReturn(JARS_RESOURCE) + when(submittedDependencyUploader.uploadFiles()).thenReturn(FILES_RESOURCE) + when(initContainerBootstrap + .bootstrapInitContainerAndVolumes(mockitoEq(DRIVER_CONTAINER_NAME), any())) + .thenAnswer(new Answer[PodBuilder] { + override def answer(invocationOnMock: InvocationOnMock): PodBuilder = { + invocationOnMock.getArgumentAt(1, classOf[PodBuilder]).editMetadata() + .addToAnnotations(BOOTSTRAPPED_POD_ANNOTATION, TRUE) + .endMetadata() + } + }) + when(initContainerComponentsProvider.provideContainerLocalizedFilesResolver( + any[String])).thenReturn(containerLocalizedFilesResolver) + when(initContainerComponentsProvider.provideDriverPodFileMounter()) + .thenReturn(fileMounter) + when(submittedDependenciesSecretBuilder.build()) + .thenReturn(INIT_CONTAINER_SECRET) + when(kubernetesClient.pods()).thenReturn(podOps) + when(podOps.create(any())).thenAnswer(new Answer[Pod] { + override def answer(invocation: InvocationOnMock): Pod = { + new PodBuilder(invocation.getArgumentAt(0, classOf[Pod])) + .editMetadata() + .withUid(DRIVER_POD_UID) + .endMetadata() + .withKind(DRIVER_POD_KIND) + .withApiVersion(DRIVER_POD_API_VERSION) + .build() + } + }) + when(podOps.withName(s"$APP_RESOURCE_PREFIX-driver")).thenReturn(namedPodResource) + when(fileMounter.addPySparkFiles( + mockitoEq(RESOLVED_PYSPARK_PRIMARY_FILE), + mockitoEq(RESOLVED_PYSPARK_FILES.mkString(",")), + any[String], + any())).thenAnswer( new Answer[PodBuilder] { + override def answer(invocation: InvocationOnMock) : PodBuilder = { + invocation.getArgumentAt(3, classOf[PodBuilder]) + .editMetadata() + .withUid(DRIVER_POD_UID) + .withName(s"$APP_RESOURCE_PREFIX-driver") + .addToLabels("pyspark-test", "true") + .endMetadata() + .withKind(DRIVER_POD_KIND) + .withApiVersion(DRIVER_POD_API_VERSION) + } + }) + when(namedPodResource.watch(loggingPodStatusWatcher)).thenReturn(watch) + when(containerLocalizedFilesResolver.resolveSubmittedAndRemoteSparkJars()) + .thenReturn(RESOLVED_SPARK_REMOTE_AND_LOCAL_JARS) + when(containerLocalizedFilesResolver.resolveSubmittedSparkJars()) + .thenReturn(RESOLVED_SPARK_JARS) + when(containerLocalizedFilesResolver.resolveSubmittedSparkFiles()) + .thenReturn(RESOLVED_SPARK_FILES) + when(containerLocalizedFilesResolver.resolvePrimaryResourceFile()) + .thenReturn(RESOLVED_PYSPARK_PRIMARY_FILE) + when(containerLocalizedFilesResolver.resolveSubmittedPySparkFiles()) + .thenReturn(RESOLVED_PYSPARK_FILES) + when(executorInitContainerConfiguration.configureSparkConfForExecutorInitContainer(SPARK_CONF)) + .thenReturn(SPARK_CONF_WITH_EXECUTOR_INIT_CONF) + when(kubernetesClient.resourceList(anyVararg[HasMetadata]())).thenReturn(resourceListOps) + when(credentialsMounterProvider.getDriverPodKubernetesCredentialsMounter()) + .thenReturn(credentialsMounter) + } + + test("Run with dependency uploader") { + expectationsForNoMountedCredentials() + when(initContainerComponentsProvider + .provideInitContainerSubmittedDependencyUploader(ALL_EXPECTED_LABELS)) + .thenReturn(Some(submittedDependencyUploader)) + when(initContainerComponentsProvider + .provideSubmittedDependenciesSecretBuilder(Some(SUBMITTED_RESOURCES.secrets()))) + .thenReturn(Some(submittedDependenciesSecretBuilder)) + when(initContainerComponentsProvider.provideInitContainerBundle(mockitoEq( + Option(SUBMITTED_RESOURCES.ids())), + mockitoEq(RESOLVED_SPARK_JARS ++ RESOLVED_SPARK_FILES))) + .thenReturn(Option(InitContainerBundle(INIT_CONTAINER_CONFIG_MAP, + initContainerBootstrap, executorInitContainerConfiguration))) + runAndVerifyDriverPodHasCorrectProperties() + val resourceListArgumentCaptor = ArgumentCaptor.forClass(classOf[HasMetadata]) + verify(kubernetesClient).resourceList(resourceListArgumentCaptor.capture()) + val createdResources = resourceListArgumentCaptor.getAllValues.asScala + assert(createdResources.size === 2) + verifyCreatedResourcesHaveOwnerReferences(createdResources) + assert(createdResources.exists { + case secret: Secret => + secret.getMetadata.getName == INIT_CONTAINER_SECRET_NAME && + secret.getData.asScala == INIT_CONTAINER_SECRET_DATA + case _ => false + }) + verifyConfigMapWasCreated(createdResources) + verify(submittedDependencyUploader).uploadJars() + verify(submittedDependencyUploader).uploadFiles() + verify(initContainerComponentsProvider) + .provideSubmittedDependenciesSecretBuilder(Some(SUBMITTED_RESOURCES.secrets())) + } + + test("Run without dependency uploader") { + expectationsForNoMountedCredentials() + expectationsForNoDependencyUploader() + runAndVerifyDriverPodHasCorrectProperties() + val resourceListArgumentCaptor = ArgumentCaptor.forClass(classOf[HasMetadata]) + verify(kubernetesClient).resourceList(resourceListArgumentCaptor.capture()) + val createdResources = resourceListArgumentCaptor.getAllValues.asScala + assert(createdResources.size === 1) + verifyCreatedResourcesHaveOwnerReferences(createdResources) + verifyConfigMapWasCreated(createdResources) + verify(submittedDependencyUploader, times(0)).uploadJars() + verify(submittedDependencyUploader, times(0)).uploadFiles() + verify(initContainerComponentsProvider) + .provideSubmittedDependenciesSecretBuilder(None) + } + + test("Run with mounted credentials") { + expectationsForNoDependencyUploader() + when(credentialsMounter.createCredentialsSecret()).thenReturn(Some(CREDENTIALS_SECRET)) + when(credentialsMounter.mountDriverKubernetesCredentials( + any(), mockitoEq(DRIVER_CONTAINER_NAME), mockitoEq(Some(CREDENTIALS_SECRET)))) + .thenAnswer(new Answer[PodBuilder] { + override def answer(invocation: InvocationOnMock): PodBuilder = { + invocation.getArgumentAt(0, classOf[PodBuilder]).editMetadata() + .addToAnnotations(CREDENTIALS_SET_ANNOTATION, TRUE) + .endMetadata() + } + }) + when(credentialsMounter.setDriverPodKubernetesCredentialLocations(any())) + .thenAnswer(new Answer[SparkConf] { + override def answer(invocation: InvocationOnMock): SparkConf = { + invocation.getArgumentAt(0, classOf[SparkConf]).clone().set(CREDENTIALS_SET_CONF, TRUE) + } + }) + runAndVerifyPodMatchesPredicate { p => + Option(p) + .filter(pod => containerHasCorrectJvmOptions(pod, _(CREDENTIALS_SET_CONF) == TRUE)) + .exists { pod => + pod.getMetadata.getAnnotations.asScala(CREDENTIALS_SET_ANNOTATION) == TRUE + } + } + val resourceListArgumentCaptor = ArgumentCaptor.forClass(classOf[HasMetadata]) + verify(kubernetesClient).resourceList(resourceListArgumentCaptor.capture()) + val createdResources = resourceListArgumentCaptor.getAllValues.asScala + assert(createdResources.size === 2) + verifyCreatedResourcesHaveOwnerReferences(createdResources) + assert(createdResources.exists { + case secret: Secret => + secret.getMetadata.getName == CREDENTIALS_SECRET_NAME && + secret.getData.asScala == CREDENTIALS_SECRET_DATA + case _ => false + }) + } + + test("Waiting for completion should await completion on the status watcher.") { + expectationsForNoMountedCredentials() + expectationsForNoDependencyUploader() + new Client( + APP_NAME, + APP_RESOURCE_PREFIX, + APP_ID, + "", + None, + MAIN_CLASS, + SPARK_CONF, + APP_ARGS, + true, + kubernetesClient, + initContainerComponentsProvider, + credentialsMounterProvider, + loggingPodStatusWatcher).run() + verify(loggingPodStatusWatcher).awaitCompletion() + } + + test("Mounting environmental variables correctly onto Driver Pod for PySpark Jobs") { + expectationsForNoMountedCredentials() + expectationsForNoDependencyUploader() + expectationsForNoSparkJarsOrFiles() + runAndVerifyDriverPodHasCorrectPySparkProperties() + } + + private def expectationsForNoSparkJarsOrFiles(): Unit = { + when(containerLocalizedFilesResolver.resolveSubmittedSparkFiles()) + .thenReturn(Nil) + when(containerLocalizedFilesResolver.resolveSubmittedSparkJars()) + .thenReturn(Nil) + } + + private def expectationsForNoDependencyUploader(): Unit = { + when(initContainerComponentsProvider + .provideInitContainerSubmittedDependencyUploader(ALL_EXPECTED_LABELS)) + .thenReturn(None) + when(initContainerComponentsProvider + .provideSubmittedDependenciesSecretBuilder(None)) + .thenReturn(None) + when(initContainerComponentsProvider.provideInitContainerBundle(mockitoEq(None), + mockitoEq(RESOLVED_SPARK_JARS ++ RESOLVED_SPARK_FILES))) + .thenReturn(Some(InitContainerBundle(INIT_CONTAINER_CONFIG_MAP, + initContainerBootstrap, executorInitContainerConfiguration))) + } + + private def expectationsForNoMountedCredentials(): Unit = { + when(credentialsMounter.setDriverPodKubernetesCredentialLocations(any())) + .thenAnswer(AdditionalAnswers.returnsFirstArg()) + when(credentialsMounter.createCredentialsSecret()).thenReturn(None) + when(credentialsMounter.mountDriverKubernetesCredentials( + any(), mockitoEq(DRIVER_CONTAINER_NAME), mockitoEq(None))) + .thenAnswer(AdditionalAnswers.returnsFirstArg()) + } + + private def verifyCreatedResourcesHaveOwnerReferences( + createdResources: mutable.Buffer[HasMetadata]): Unit = { + assert(createdResources.forall { resource => + val owners = resource.getMetadata.getOwnerReferences.asScala + owners.size === 1 && + owners.head.getController && + owners.head.getKind == DRIVER_POD_KIND && + owners.head.getUid == DRIVER_POD_UID && + owners.head.getName == s"$APP_RESOURCE_PREFIX-driver" && + owners.head.getApiVersion == DRIVER_POD_API_VERSION + }) + } + + private def verifyConfigMapWasCreated(createdResources: mutable.Buffer[HasMetadata]): Unit = { + assert(createdResources.exists { + case configMap: ConfigMap => + configMap.getMetadata.getName == CONFIG_MAP_NAME && + configMap.getData.asScala == CONFIG_MAP_DATA + case _ => false + }) + } + + private def runAndVerifyDriverPodHasCorrectProperties(): Unit = { + val expectedOptions = SPARK_CONF.getAll + .filterNot(_._1 == org.apache.spark.internal.config.DRIVER_JAVA_OPTIONS.key) + .toMap ++ + Map( + "spark.app.id" -> APP_ID, + KUBERNETES_DRIVER_POD_NAME.key -> s"$APP_RESOURCE_PREFIX-driver", + KUBERNETES_EXECUTOR_POD_NAME_PREFIX.key -> APP_RESOURCE_PREFIX, + EXECUTOR_INIT_CONF_KEY -> TRUE, + CUSTOM_JAVA_OPTION_KEY -> CUSTOM_JAVA_OPTION_VALUE, + "spark.jars" -> RESOLVED_SPARK_JARS.mkString(","), + "spark.files" -> RESOLVED_SPARK_FILES.mkString(",")) + runAndVerifyPodMatchesPredicate { p => + Option(p) + .filter(_.getMetadata.getName == s"$APP_RESOURCE_PREFIX-driver") + .filter(podHasCorrectAnnotations) + .filter(_.getMetadata.getLabels.asScala == ALL_EXPECTED_LABELS) + .filter(containerHasCorrectBasicContainerConfiguration) + .filter(containerHasCorrectBasicEnvs) + .filter(containerHasCorrectMountedClasspath) + .exists(pod => containerHasCorrectJvmOptions(pod, _ == expectedOptions)) + } + } + + private def runAndVerifyDriverPodHasCorrectPySparkProperties(): Unit = { + when(initContainerComponentsProvider.provideContainerLocalizedFilesResolver( + mockitoEq(PYSPARK_PRIMARY_FILE))).thenReturn(containerLocalizedFilesResolver) + when(initContainerComponentsProvider.provideInitContainerBundle( + any[Option[SubmittedResourceIds]], any[Iterable[String]])) + .thenReturn(Some(InitContainerBundle(INIT_CONTAINER_CONFIG_MAP, + initContainerBootstrap, executorInitContainerConfiguration))) + runAndVerifyPySparkPodMatchesPredicate { p => + Option(p).exists(pod => containerHasCorrectPySparkEnvs(pod)) + } + } + + private def runAndVerifyPodMatchesPredicate(pred: (Pod => Boolean)): Unit = { + new Client( + APP_NAME, + APP_RESOURCE_PREFIX, + APP_ID, + "", + None, + MAIN_CLASS, + SPARK_CONF, + APP_ARGS, + false, + kubernetesClient, + initContainerComponentsProvider, + credentialsMounterProvider, + loggingPodStatusWatcher).run() + val podMatcher = new BaseMatcher[Pod] { + override def matches(o: scala.Any): Boolean = { + o match { + case p: Pod => pred(p) + case _ => false + } + } + override def describeTo(description: Description): Unit = {} + } + verify(podOps).create(argThat(podMatcher)) + } + + private def containerHasCorrectJvmOptions( + pod: Pod, optionsCorrectnessPredicate: (Map[String, String] => Boolean)): Boolean = { + val driverContainer = pod.getSpec.getContainers.asScala.head + val envs = driverContainer.getEnv.asScala.map(env => (env.getName, env.getValue)) + envs.toMap.get(ENV_DRIVER_JAVA_OPTS).exists { javaOptions => + val splitOptions = javaOptions.split(" ") + splitOptions.forall(_.startsWith("-D")) && + optionsCorrectnessPredicate(splitOptions.map { option => + val withoutPrefix = option.substring(2) + (withoutPrefix.split("=", 2)(0), withoutPrefix.split("=", 2)(1)) + }.toMap) + } + } + + private def containerHasCorrectMountedClasspath(pod: Pod): Boolean = { + val driverContainer = pod.getSpec.getContainers.asScala.head + val envs = driverContainer.getEnv.asScala.map(env => (env.getName, env.getValue)) + envs.toMap.get(ENV_MOUNTED_CLASSPATH).exists { classpath => + val mountedClasspathEntities = classpath.split(File.pathSeparator) + mountedClasspathEntities.toSet == RESOLVED_SPARK_REMOTE_AND_LOCAL_JARS.toSet + } + } + + private def containerHasCorrectBasicEnvs(pod: Pod): Boolean = { + val driverContainer = pod.getSpec.getContainers.asScala.head + val envs = driverContainer.getEnv.asScala.map(env => (env.getName, env.getValue)) + val expectedBasicEnvs = Map( + ENV_SUBMIT_EXTRA_CLASSPATH -> DRIVER_EXTRA_CLASSPATH, + ENV_DRIVER_MEMORY -> s"${DRIVER_MEMORY_MB + DRIVER_MEMORY_OVERHEAD_MB}m", + ENV_DRIVER_MAIN_CLASS -> MAIN_CLASS, + ENV_DRIVER_ARGS -> APP_ARGS.mkString(" ")) + expectedBasicEnvs.toSet.subsetOf(envs.toSet) + } + + private def containerHasCorrectPySparkEnvs(pod: Pod): Boolean = { + val driverPodLabels = + pod.getMetadata.getLabels.asScala.map(env => (env._1.toString, env._2.toString)) + val expectedBasicLabels = Map( + "pyspark-test" -> "true", + "spark-role" -> "driver") + expectedBasicLabels.toSet.subsetOf(driverPodLabels.toSet) + } + + private def containerHasCorrectBasicContainerConfiguration(pod: Pod): Boolean = { + val containers = pod.getSpec.getContainers.asScala + containers.size == 1 && + containers.head.getName == DRIVER_CONTAINER_NAME && + containers.head.getImage == CUSTOM_DRIVER_IMAGE && + containers.head.getImagePullPolicy == "IfNotPresent" + } + + private def podHasCorrectAnnotations(pod: Pod): Boolean = { + val expectedAnnotations = Map( + DEPRECATED_CUSTOM_ANNOTATION_KEY -> DEPRECATED_CUSTOM_ANNOTATION_VALUE, + CUSTOM_ANNOTATION_KEY -> CUSTOM_ANNOTATION_VALUE, + SPARK_APP_NAME_ANNOTATION -> APP_NAME, + BOOTSTRAPPED_POD_ANNOTATION -> TRUE) + pod.getMetadata.getAnnotations.asScala == expectedAnnotations + } + + private def runAndVerifyPySparkPodMatchesPredicate(pred: (Pod => Boolean)): Unit = { + new Client( + APP_NAME, + APP_RESOURCE_PREFIX, + APP_ID, + PYSPARK_PRIMARY_FILE, + Option(new PythonSubmissionResourcesImpl(PYSPARK_PRIMARY_FILE, PYSPARK_APP_ARGS)), + MAIN_CLASS, + SPARK_CONF, + PYSPARK_APP_ARGS, + false, + kubernetesClient, + initContainerComponentsProvider, + credentialsMounterProvider, + loggingPodStatusWatcher).run() + val podMatcher = new BaseMatcher[Pod] { + override def matches(o: scala.Any): Boolean = { + o match { + case p: Pod => pred(p) + case _ => false + } + } + override def describeTo(description: Description): Unit = {} + } + verify(podOps).create(argThat(podMatcher)) + } } diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/ContainerLocalizedFilesResolverSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/ContainerLocalizedFilesResolverSuite.scala new file mode 100644 index 0000000000000..7e51abcd7b8e0 --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/ContainerLocalizedFilesResolverSuite.scala @@ -0,0 +1,93 @@ +/* + * 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.submit + +import org.apache.spark.SparkFunSuite + +class ContainerLocalizedFilesResolverSuite extends SparkFunSuite { + private val SPARK_JARS = Seq( + "hdfs://localhost:9000/app/jars/jar1.jar", + "file:///app/jars/jar2.jar", + "local:///app/jars/jar3.jar", + "http://app/jars/jar4.jar") + private val SPARK_FILES = Seq( + "hdfs://localhost:9000/app/files/file1.txt", + "file:///app/files/file2.txt", + "local:///app/files/file3.txt", + "http://app/files/file4.txt") + private val PYSPARK_FILES = Seq( + "hdfs://localhost:9000/app/files/file1.py", + "file:///app/files/file2.py", + "local:///app/files/file3.py", + "http://app/files/file4.py", + "file:///app/files/file5.py") + private val JARS_DOWNLOAD_PATH = "/var/data/spark-jars" + private val FILES_DOWNLOAD_PATH = "/var/data/spark-files" + private val PYSPARK_PRIMARY_FILE = "file:///app/files/file5.py" + private val localizedFilesResolver = new ContainerLocalizedFilesResolverImpl( + SPARK_JARS, + SPARK_FILES, + PYSPARK_FILES, + PYSPARK_PRIMARY_FILE, + JARS_DOWNLOAD_PATH, + FILES_DOWNLOAD_PATH) + + test("Submitted and remote Spark jars should resolve non-local uris to download path.") { + val resolvedJars = localizedFilesResolver.resolveSubmittedAndRemoteSparkJars() + val expectedResolvedJars = Seq( + s"$JARS_DOWNLOAD_PATH/jar1.jar", + s"$JARS_DOWNLOAD_PATH/jar2.jar", + "/app/jars/jar3.jar", + s"$JARS_DOWNLOAD_PATH/jar4.jar") + assert(resolvedJars === expectedResolvedJars) + } + + test("Submitted Spark jars should resolve to the download path.") { + val resolvedJars = localizedFilesResolver.resolveSubmittedSparkJars() + val expectedResolvedJars = Seq( + "hdfs://localhost:9000/app/jars/jar1.jar", + s"$JARS_DOWNLOAD_PATH/jar2.jar", + "local:///app/jars/jar3.jar", + "http://app/jars/jar4.jar") + assert(resolvedJars === expectedResolvedJars) + } + + test("Submitted Spark files should resolve to the download path.") { + val resolvedFiles = localizedFilesResolver.resolveSubmittedSparkFiles() + val expectedResolvedFiles = Seq( + "hdfs://localhost:9000/app/files/file1.txt", + s"$FILES_DOWNLOAD_PATH/file2.txt", + "local:///app/files/file3.txt", + "http://app/files/file4.txt") + assert(resolvedFiles === expectedResolvedFiles) + } + test("Submitted PySpark files should resolve to the download path.") { + val resolvedPySparkFiles = localizedFilesResolver.resolveSubmittedPySparkFiles() + val expectedPySparkFiles = Seq( + "hdfs://localhost:9000/app/files/file1.py", + s"$FILES_DOWNLOAD_PATH/file2.py", + "local:///app/files/file3.py", + "http://app/files/file4.py") + assert(resolvedPySparkFiles === expectedPySparkFiles) + } + test("Submitted PySpark Primary resource should resolve to the download path.") { + val resolvedPySparkPrimary = + localizedFilesResolver.resolvePrimaryResourceFile() + val expectedPySparkPrimary = s"$FILES_DOWNLOAD_PATH/file5.py" + assert(resolvedPySparkPrimary === expectedPySparkPrimary) + } +} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/DriverPodKubernetesCredentialsMounterSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/DriverPodKubernetesCredentialsMounterSuite.scala new file mode 100644 index 0000000000000..2e0a7ba5098b2 --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/DriverPodKubernetesCredentialsMounterSuite.scala @@ -0,0 +1,171 @@ +/* + * 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.submit + +import io.fabric8.kubernetes.api.model.{PodBuilder, SecretBuilder} +import org.scalatest.prop.TableDrivenPropertyChecks +import scala.collection.JavaConverters._ + +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.deploy.kubernetes.KubernetesCredentials +import org.apache.spark.deploy.kubernetes.config._ +import org.apache.spark.deploy.kubernetes.constants._ + +class DriverPodKubernetesCredentialsMounterSuite + extends SparkFunSuite with TableDrivenPropertyChecks { + + private val CLIENT_KEY_DATA = "client-key-data" + private val CLIENT_CERT_DATA = "client-cert-data" + private val OAUTH_TOKEN_DATA = "oauth-token" + private val CA_CERT_DATA = "ca-cert-data" + private val SUBMITTER_LOCAL_DRIVER_KUBERNETES_CREDENTIALS = KubernetesCredentials( + caCertDataBase64 = Some(CA_CERT_DATA), + clientKeyDataBase64 = Some(CLIENT_KEY_DATA), + clientCertDataBase64 = Some(CLIENT_CERT_DATA), + oauthTokenBase64 = Some(OAUTH_TOKEN_DATA)) + private val APP_ID = "app-id" + private val USER_SPECIFIED_CLIENT_KEY_FILE = Some("/var/data/client-key.pem") + private val USER_SPECIFIED_CLIENT_CERT_FILE = Some("/var/data/client-cert.pem") + private val USER_SPECIFIED_OAUTH_TOKEN_FILE = Some("/var/data/token.txt") + private val USER_SPECIFIED_CA_CERT_FILE = Some("/var/data/ca.pem") + + // Different configurations of credentials mounters + private val credentialsMounterWithPreMountedFiles = + new DriverPodKubernetesCredentialsMounterImpl( + kubernetesAppId = APP_ID, + submitterLocalDriverPodKubernetesCredentials = SUBMITTER_LOCAL_DRIVER_KUBERNETES_CREDENTIALS, + maybeUserSpecifiedMountedClientKeyFile = USER_SPECIFIED_CLIENT_KEY_FILE, + maybeUserSpecifiedMountedClientCertFile = USER_SPECIFIED_CLIENT_CERT_FILE, + maybeUserSpecifiedMountedOAuthTokenFile = USER_SPECIFIED_OAUTH_TOKEN_FILE, + maybeUserSpecifiedMountedCaCertFile = USER_SPECIFIED_CA_CERT_FILE) + private val credentialsMounterWithoutPreMountedFiles = + new DriverPodKubernetesCredentialsMounterImpl( + kubernetesAppId = APP_ID, + submitterLocalDriverPodKubernetesCredentials = SUBMITTER_LOCAL_DRIVER_KUBERNETES_CREDENTIALS, + maybeUserSpecifiedMountedClientKeyFile = None, + maybeUserSpecifiedMountedClientCertFile = None, + maybeUserSpecifiedMountedOAuthTokenFile = None, + maybeUserSpecifiedMountedCaCertFile = None) + private val credentialsMounterWithoutAnyDriverCredentials = + new DriverPodKubernetesCredentialsMounterImpl( + APP_ID, KubernetesCredentials(None, None, None, None), None, None, None, None) + + // Test matrices + private val TEST_MATRIX_EXPECTED_SPARK_CONFS = Table( + ("Credentials Mounter Implementation", + "Expected client key file", + "Expected client cert file", + "Expected CA Cert file", + "Expected OAuth Token File"), + (credentialsMounterWithoutAnyDriverCredentials, + None, + None, + None, + None), + (credentialsMounterWithoutPreMountedFiles, + Some(DRIVER_CREDENTIALS_CLIENT_KEY_PATH), + Some(DRIVER_CREDENTIALS_CLIENT_CERT_PATH), + Some(DRIVER_CREDENTIALS_CA_CERT_PATH), + Some(DRIVER_CREDENTIALS_OAUTH_TOKEN_PATH)), + (credentialsMounterWithPreMountedFiles, + USER_SPECIFIED_CLIENT_KEY_FILE, + USER_SPECIFIED_CLIENT_CERT_FILE, + USER_SPECIFIED_CA_CERT_FILE, + USER_SPECIFIED_OAUTH_TOKEN_FILE)) + + private val TEST_MATRIX_EXPECTED_CREDENTIALS_SECRET = Table( + ("Credentials Mounter Implementation", "Expected Credentials Secret Data"), + (credentialsMounterWithoutAnyDriverCredentials, None), + (credentialsMounterWithoutPreMountedFiles, + Some(KubernetesSecretNameAndData( + data = Map[String, String]( + DRIVER_CREDENTIALS_CLIENT_KEY_SECRET_NAME -> CLIENT_KEY_DATA, + DRIVER_CREDENTIALS_CLIENT_CERT_SECRET_NAME -> CLIENT_CERT_DATA, + DRIVER_CREDENTIALS_CA_CERT_SECRET_NAME -> CA_CERT_DATA, + DRIVER_CREDENTIALS_OAUTH_TOKEN_SECRET_NAME -> OAUTH_TOKEN_DATA + ), + name = s"$APP_ID-kubernetes-credentials"))), + (credentialsMounterWithPreMountedFiles, None)) + + test("Credentials mounter should set the driver's Kubernetes credentials locations") { + forAll(TEST_MATRIX_EXPECTED_SPARK_CONFS) { + case (credentialsMounter, + expectedClientKeyFile, + expectedClientCertFile, + expectedCaCertFile, + expectedOAuthTokenFile) => + val baseSparkConf = new SparkConf() + val resolvedSparkConf = + credentialsMounter.setDriverPodKubernetesCredentialLocations(baseSparkConf) + assert(resolvedSparkConf.getOption( + s"$APISERVER_AUTH_DRIVER_MOUNTED_CONF_PREFIX.$CLIENT_KEY_FILE_CONF_SUFFIX") === + expectedClientKeyFile) + assert(resolvedSparkConf.getOption( + s"$APISERVER_AUTH_DRIVER_MOUNTED_CONF_PREFIX.$CLIENT_CERT_FILE_CONF_SUFFIX") === + expectedClientCertFile) + assert(resolvedSparkConf.getOption( + s"$APISERVER_AUTH_DRIVER_MOUNTED_CONF_PREFIX.$CA_CERT_FILE_CONF_SUFFIX") === + expectedCaCertFile) + assert(resolvedSparkConf.getOption( + s"$APISERVER_AUTH_DRIVER_MOUNTED_CONF_PREFIX.$OAUTH_TOKEN_FILE_CONF_SUFFIX") === + expectedOAuthTokenFile) + } + } + + test("Credentials mounter should create the correct credentials secret.") { + forAll(TEST_MATRIX_EXPECTED_CREDENTIALS_SECRET) { + case (credentialsMounter, expectedSecretNameAndData) => + val builtSecret = credentialsMounter.createCredentialsSecret() + val secretNameAndData = builtSecret.map { secret => + KubernetesSecretNameAndData(secret.getMetadata.getName, secret.getData.asScala.toMap) + } + assert(secretNameAndData === expectedSecretNameAndData) + } + } + + test("When credentials secret is provided, driver pod should mount the secret volume.") { + val credentialsSecret = new SecretBuilder() + .withNewMetadata().withName("secret").endMetadata() + .addToData("secretKey", "secretValue") + .build() + val originalPodSpec = new PodBuilder() + .withNewMetadata().withName("pod").endMetadata() + .withNewSpec() + .addNewContainer() + .withName("container") + .endContainer() + .endSpec() + val podSpecWithMountedDriverKubernetesCredentials = + credentialsMounterWithoutPreMountedFiles.mountDriverKubernetesCredentials( + originalPodSpec, "container", Some(credentialsSecret)).build() + val volumes = podSpecWithMountedDriverKubernetesCredentials.getSpec.getVolumes.asScala + assert(volumes.exists(_.getName == DRIVER_CREDENTIALS_SECRET_VOLUME_NAME)) + volumes.find(_.getName == DRIVER_CREDENTIALS_SECRET_VOLUME_NAME).foreach { secretVolume => + assert(secretVolume.getSecret != null && secretVolume.getSecret.getSecretName == "secret") + } + } + + test("When credentials secret is absent, driver pod should not be changed.") { + val originalPodSpec = new PodBuilder() + val nonAdjustedPodSpec = + credentialsMounterWithoutAnyDriverCredentials.mountDriverKubernetesCredentials( + originalPodSpec, "driver", None) + assert(nonAdjustedPodSpec === originalPodSpec) + } +} + +private case class KubernetesSecretNameAndData(name: String, data: Map[String, String]) diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/ExecutorInitContainerConfigurationSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/ExecutorInitContainerConfigurationSuite.scala new file mode 100644 index 0000000000000..ead1d49b8a37c --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/ExecutorInitContainerConfigurationSuite.scala @@ -0,0 +1,56 @@ +/* + * 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.submit + +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.deploy.kubernetes.config._ + +class ExecutorInitContainerConfigurationSuite extends SparkFunSuite { + + private val SECRET_NAME = "init-container-secret" + private val SECRET_MOUNT_DIR = "/mnt/secrets/spark" + private val CONFIG_MAP_NAME = "spark-config-map" + private val CONFIG_MAP_KEY = "spark-config-map-key" + + test("Not passing a secret name should not set the secret value.") { + val baseSparkConf = new SparkConf(false) + val configurationUnderTest = new ExecutorInitContainerConfigurationImpl( + None, + SECRET_MOUNT_DIR, + CONFIG_MAP_NAME, + CONFIG_MAP_KEY) + val resolvedSparkConf = configurationUnderTest + .configureSparkConfForExecutorInitContainer(baseSparkConf) + assert(resolvedSparkConf.get(EXECUTOR_INIT_CONTAINER_CONFIG_MAP).contains(CONFIG_MAP_NAME)) + assert(resolvedSparkConf.get(EXECUTOR_INIT_CONTAINER_CONFIG_MAP_KEY).contains(CONFIG_MAP_KEY)) + assert(resolvedSparkConf.get(EXECUTOR_INIT_CONTAINER_SECRET_MOUNT_DIR) + .contains(SECRET_MOUNT_DIR)) + assert(resolvedSparkConf.get(EXECUTOR_INIT_CONTAINER_SECRET).isEmpty) + } + + test("Passing a secret name should set the secret value.") { + val baseSparkConf = new SparkConf(false) + val configurationUnderTest = new ExecutorInitContainerConfigurationImpl( + Some(SECRET_NAME), + SECRET_MOUNT_DIR, + CONFIG_MAP_NAME, + CONFIG_MAP_KEY) + val resolvedSparkConf = configurationUnderTest + .configureSparkConfForExecutorInitContainer(baseSparkConf) + assert(resolvedSparkConf.get(EXECUTOR_INIT_CONTAINER_SECRET).contains(SECRET_NAME)) + } +} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/PythonSubmissionResourcesSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/PythonSubmissionResourcesSuite.scala new file mode 100644 index 0000000000000..5483dc154fe73 --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/PythonSubmissionResourcesSuite.scala @@ -0,0 +1,112 @@ +/* + * 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.submit + +import org.apache.spark.{SSLOptions, SparkConf, SparkFunSuite} +import org.apache.spark.deploy.kubernetes.config._ + +import scala.collection.JavaConverters._ +import io.fabric8.kubernetes.api.model.{ContainerBuilder, Pod, PodBuilder} +import org.mockito.{Mock, MockitoAnnotations} +import org.mockito.Mockito.when +import org.scalatest.BeforeAndAfter + + + + +private[spark] class PythonSubmissionResourcesSuite extends SparkFunSuite with BeforeAndAfter { + private val PYSPARK_FILES = Seq( + "hdfs://localhost:9000/app/files/file1.py", + "file:///app/files/file2.py", + "local:///app/files/file3.py", + "http://app/files/file4.py", + "file:///app/files/file5.py") + private val RESOLVED_PYSPARK_FILES = Seq( + "hdfs://localhost:9000/app/files/file1.py", + "/var/spark-data/spark-files/file2.py", + "local:///app/file`s/file3.py", + "http://app/files/file4.py") + private val PYSPARK_PRIMARY_FILE = "file:///app/files/file5.py" + private val RESOLVED_PYSPARK_PRIMARY_FILE = "/var/data/spark-files/file5.py" + + private val pyFilesResource = new PythonSubmissionResourcesImpl( + PYSPARK_PRIMARY_FILE, Array(PYSPARK_FILES.mkString(","), "500") + ) + private val pyResource = new PythonSubmissionResourcesImpl( + PYSPARK_PRIMARY_FILE, Array(null, "500") + ) + private val DRIVER_CONTAINER_NAME = "pyspark_container" + private val driverContainer = new ContainerBuilder() + .withName(DRIVER_CONTAINER_NAME) + .build() + private val basePodBuilder = new PodBuilder() + .withNewMetadata() + .withName("base_pod") + .endMetadata() + .withNewSpec() + .addToContainers(driverContainer) + .endSpec() + + @Mock + private var driverInitContainer: DriverInitContainerComponentsProviderImpl = _ + @Mock + private var localizedFileResolver: ContainerLocalizedFilesResolverImpl = _ + before { + MockitoAnnotations.initMocks(this) + when(driverInitContainer.provideDriverPodFileMounter()).thenReturn( + new DriverPodKubernetesFileMounterImpl() + ) + when(localizedFileResolver.resolvePrimaryResourceFile()).thenReturn( + RESOLVED_PYSPARK_PRIMARY_FILE) + } + test("Test with --py-files included") { + assert(pyFilesResource.sparkJars === Seq.empty[String]) + assert(pyFilesResource.pySparkFiles === + PYSPARK_PRIMARY_FILE +: PYSPARK_FILES) + assert(pyFilesResource.primaryPySparkResource(localizedFileResolver) === + RESOLVED_PYSPARK_PRIMARY_FILE) + val driverPod: Pod = pyFilesResource.driverPodWithPySparkEnvs( + driverInitContainer.provideDriverPodFileMounter(), + RESOLVED_PYSPARK_PRIMARY_FILE, + RESOLVED_PYSPARK_FILES.mkString(","), + DRIVER_CONTAINER_NAME, + basePodBuilder + ) + val driverContainer = driverPod.getSpec.getContainers.asScala.head + val envs = driverContainer.getEnv.asScala.map(env => (env.getName, env.getValue)).toMap + envs.get("PYSPARK_PRIMARY") foreach{ a => assert (a === RESOLVED_PYSPARK_PRIMARY_FILE) } + envs.get("PYSPARK_FILES") foreach{ a => assert (a === RESOLVED_PYSPARK_FILES.mkString(",")) } + } + + test("Test without --py-files") { + assert(pyResource.sparkJars === Seq.empty[String]) + assert(pyResource.pySparkFiles === Array(PYSPARK_PRIMARY_FILE)) + assert(pyResource.primaryPySparkResource(localizedFileResolver) === + RESOLVED_PYSPARK_PRIMARY_FILE) + val driverPod: Pod = pyResource.driverPodWithPySparkEnvs( + driverInitContainer.provideDriverPodFileMounter(), + RESOLVED_PYSPARK_PRIMARY_FILE, + "", + DRIVER_CONTAINER_NAME, + basePodBuilder + ) + val driverContainer = driverPod.getSpec.getContainers.asScala.head + val envs = driverContainer.getEnv.asScala.map(env => (env.getName, env.getValue)).toMap + envs.get("PYSPARK_PRIMARY") foreach{ a => assert (a === RESOLVED_PYSPARK_PRIMARY_FILE) } + envs.get("PYSPARK_FILES") foreach{ a => assert (a === "") } + } +} \ No newline at end of file diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/SparkInitContainerConfigMapBuilderSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/SparkInitContainerConfigMapBuilderSuite.scala new file mode 100644 index 0000000000000..f1e1ff7013496 --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/SparkInitContainerConfigMapBuilderSuite.scala @@ -0,0 +1,101 @@ +/* + * 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.submit + +import java.io.StringReader +import java.util.Properties + +import com.google.common.collect.Maps +import org.mockito.Mockito.{verify, when} +import org.scalatest.BeforeAndAfter +import org.scalatest.mock.MockitoSugar._ +import scala.collection.JavaConverters._ + +import org.apache.spark.SparkFunSuite +import org.apache.spark.deploy.kubernetes.config._ + +class SparkInitContainerConfigMapBuilderSuite extends SparkFunSuite with BeforeAndAfter { + + private val JARS = Seq( + "hdfs://localhost:9000/app/jars/jar1.jar", + "file:///app/jars/jar2.jar", + "http://localhost:9000/app/jars/jar3.jar", + "local:///app/jars/jar4.jar") + private val FILES = Seq( + "hdfs://localhost:9000/app/files/file1.txt", + "file:///app/files/file2.txt", + "http://localhost:9000/app/files/file3.txt", + "local:///app/files/file4.txt") + private val JARS_DOWNLOAD_PATH = "/var/data/jars" + private val FILES_DOWNLOAD_PATH = "/var/data/files" + private val CONFIG_MAP_NAME = "config-map" + private val CONFIG_MAP_KEY = "config-map-key" + + test("Config map without submitted dependencies sets remote download configurations") { + val configMap = new SparkInitContainerConfigMapBuilderImpl( + JARS, + FILES, + JARS_DOWNLOAD_PATH, + FILES_DOWNLOAD_PATH, + CONFIG_MAP_NAME, + CONFIG_MAP_KEY, + None).build() + assert(configMap.getMetadata.getName === CONFIG_MAP_NAME) + val maybeConfigValue = configMap.getData.asScala.get(CONFIG_MAP_KEY) + assert(maybeConfigValue.isDefined) + maybeConfigValue.foreach { configValue => + val propertiesStringReader = new StringReader(configValue) + val properties = new Properties() + properties.load(propertiesStringReader) + val propertiesMap = Maps.fromProperties(properties).asScala + val remoteJarsString = propertiesMap.get(INIT_CONTAINER_REMOTE_JARS.key) + assert(remoteJarsString.isDefined) + val remoteJars = remoteJarsString.map(_.split(",")).toSet.flatten + assert(remoteJars === + Set("hdfs://localhost:9000/app/jars/jar1.jar", "http://localhost:9000/app/jars/jar3.jar")) + val remoteFilesString = propertiesMap.get(INIT_CONTAINER_REMOTE_FILES.key) + assert(remoteFilesString.isDefined) + val remoteFiles = remoteFilesString.map(_.split(",")).toSet.flatten + assert(remoteFiles === + Set("hdfs://localhost:9000/app/files/file1.txt", + "http://localhost:9000/app/files/file3.txt")) + assert(propertiesMap(INIT_CONTAINER_JARS_DOWNLOAD_LOCATION.key) === JARS_DOWNLOAD_PATH) + assert(propertiesMap(INIT_CONTAINER_FILES_DOWNLOAD_LOCATION.key) === FILES_DOWNLOAD_PATH) + } + } + + test("Config map with submitted dependencies adds configurations from plugin") { + val submittedDependenciesPlugin = mock[SubmittedDependencyInitContainerConfigPlugin] + when(submittedDependenciesPlugin.configurationsToFetchSubmittedDependencies()) + .thenReturn(Map("customConf" -> "customConfValue")) + val configMap = new SparkInitContainerConfigMapBuilderImpl( + JARS, + FILES, + JARS_DOWNLOAD_PATH, + FILES_DOWNLOAD_PATH, + CONFIG_MAP_NAME, + CONFIG_MAP_KEY, + Some(submittedDependenciesPlugin)).build() + val configValue = configMap.getData.asScala(CONFIG_MAP_KEY) + val propertiesStringReader = new StringReader(configValue) + val properties = new Properties() + properties.load(propertiesStringReader) + val propertiesMap = Maps.fromProperties(properties).asScala + assert(propertiesMap("customConf") === "customConfValue") + verify(submittedDependenciesPlugin).configurationsToFetchSubmittedDependencies() + } +} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/SubmittedDependencyInitContainerConfigPluginSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/SubmittedDependencyInitContainerConfigPluginSuite.scala new file mode 100644 index 0000000000000..8431b77c9e85f --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/SubmittedDependencyInitContainerConfigPluginSuite.scala @@ -0,0 +1,89 @@ +/* + * 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.submit + +import org.apache.spark.SparkFunSuite +import org.apache.spark.deploy.kubernetes.config._ + +class SubmittedDependencyInitContainerConfigPluginSuite extends SparkFunSuite { + private val STAGING_SERVER_URI = "http://localhost:9000" + private val STAGING_SERVER_INTERNAL_URI = "http://internalHost:9000" + private val JARS_RESOURCE_ID = "jars-id" + private val FILES_RESOURCE_ID = "files-id" + private val JARS_SECRET_KEY = "jars" + private val FILES_SECRET_KEY = "files" + private val TRUSTSTORE_SECRET_KEY = "trustStore" + private val CLIENT_CERT_SECRET_KEY = "client-cert" + private val SECRETS_VOLUME_MOUNT_PATH = "/var/data" + private val TRUSTSTORE_PASSWORD = "trustStore" + private val TRUSTSTORE_FILE = "/mnt/secrets/trustStore.jks" + private val CLIENT_CERT_URI = "local:///mnt/secrets/client-cert.pem" + private val TRUSTSTORE_TYPE = "jks" + + test("Plugin should provide configuration for fetching uploaded dependencies") { + val configPluginUnderTest = new SubmittedDependencyInitContainerConfigPluginImpl( + STAGING_SERVER_URI, + JARS_RESOURCE_ID, + FILES_RESOURCE_ID, + JARS_SECRET_KEY, + FILES_SECRET_KEY, + TRUSTSTORE_SECRET_KEY, + CLIENT_CERT_SECRET_KEY, + false, + None, + None, + None, + None, + SECRETS_VOLUME_MOUNT_PATH) + val addedConfigurations = configPluginUnderTest.configurationsToFetchSubmittedDependencies() + val expectedConfigurations = Map( + RESOURCE_STAGING_SERVER_URI.key -> STAGING_SERVER_URI, + INIT_CONTAINER_DOWNLOAD_JARS_RESOURCE_IDENTIFIER.key -> JARS_RESOURCE_ID, + INIT_CONTAINER_DOWNLOAD_FILES_RESOURCE_IDENTIFIER.key -> FILES_RESOURCE_ID, + INIT_CONTAINER_DOWNLOAD_JARS_SECRET_LOCATION.key -> + s"$SECRETS_VOLUME_MOUNT_PATH/$JARS_SECRET_KEY", + INIT_CONTAINER_DOWNLOAD_FILES_SECRET_LOCATION.key -> + s"$SECRETS_VOLUME_MOUNT_PATH/$FILES_SECRET_KEY", + RESOURCE_STAGING_SERVER_SSL_ENABLED.key -> "false") + assert(addedConfigurations === expectedConfigurations) + } + + test("Plugin should set up SSL with the appropriate trustStore if it's provided.") { + val configPluginUnderTest = new SubmittedDependencyInitContainerConfigPluginImpl( + STAGING_SERVER_URI, + JARS_RESOURCE_ID, + FILES_RESOURCE_ID, JARS_SECRET_KEY, + FILES_SECRET_KEY, + TRUSTSTORE_SECRET_KEY, + CLIENT_CERT_SECRET_KEY, + true, + Some(TRUSTSTORE_FILE), + Some(CLIENT_CERT_URI), + Some(TRUSTSTORE_PASSWORD), + Some(TRUSTSTORE_TYPE), + SECRETS_VOLUME_MOUNT_PATH) + val addedConfigurations = configPluginUnderTest.configurationsToFetchSubmittedDependencies() + val expectedSslConfigurations = Map( + RESOURCE_STAGING_SERVER_SSL_ENABLED.key -> "true", + RESOURCE_STAGING_SERVER_TRUSTSTORE_FILE.key -> + s"$SECRETS_VOLUME_MOUNT_PATH/$TRUSTSTORE_SECRET_KEY", + RESOURCE_STAGING_SERVER_TRUSTSTORE_PASSWORD.key -> TRUSTSTORE_PASSWORD, + RESOURCE_STAGING_SERVER_TRUSTSTORE_TYPE.key -> TRUSTSTORE_TYPE, + RESOURCE_STAGING_SERVER_CLIENT_CERT_PEM.key -> "/mnt/secrets/client-cert.pem") + assert(expectedSslConfigurations.toSet.subsetOf(addedConfigurations.toSet)) + } +} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/SubmittedDependencySecretBuilderSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/SubmittedDependencySecretBuilderSuite.scala new file mode 100644 index 0000000000000..83fd568e7a3aa --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/SubmittedDependencySecretBuilderSuite.scala @@ -0,0 +1,109 @@ +/* + * 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.submit + +import java.io.File + +import com.google.common.base.Charsets +import com.google.common.io.{BaseEncoding, Files} +import io.fabric8.kubernetes.api.model.Secret +import scala.collection.JavaConverters._ +import scala.collection.Map + +import org.apache.spark.SparkFunSuite +import org.apache.spark.util.Utils + +class SubmittedDependencySecretBuilderSuite extends SparkFunSuite { + + private val SECRET_NAME = "submitted-dependency-secret" + private val JARS_SECRET = "jars-secret" + private val FILES_SECRET = "files-secret" + private val JARS_SECRET_KEY = "jars-secret-key" + private val FILES_SECRET_KEY = "files-secret-key" + private val TRUSTSTORE_SECRET_KEY = "truststore-secret-key" + private val CLIENT_CERT_SECRET_KEY = "client-cert" + private val TRUSTSTORE_STRING_CONTENTS = "trustStore-contents" + private val CLIENT_CERT_STRING_CONTENTS = "client-certificate-contents" + + test("Building the secret without a trustStore") { + val builder = new SubmittedDependencySecretBuilderImpl( + SECRET_NAME, + JARS_SECRET, + FILES_SECRET, + JARS_SECRET_KEY, + FILES_SECRET_KEY, + TRUSTSTORE_SECRET_KEY, + CLIENT_CERT_SECRET_KEY, + None, + None) + val secret = builder.build() + assert(secret.getMetadata.getName === SECRET_NAME) + val secretDecodedData = decodeSecretData(secret) + val expectedSecretData = Map(JARS_SECRET_KEY -> JARS_SECRET, FILES_SECRET_KEY -> FILES_SECRET) + assert(secretDecodedData === expectedSecretData) + } + + private def decodeSecretData(secret: Secret): Map[String, String] = { + val secretData = secret.getData.asScala + secretData.mapValues(encoded => + new String(BaseEncoding.base64().decode(encoded), Charsets.UTF_8)) + } + + test("Building the secret with a trustStore") { + val tempSslDir = Utils.createTempDir(namePrefix = "temp-ssl-tests") + try { + val trustStoreFile = new File(tempSslDir, "trustStore.jks") + Files.write(TRUSTSTORE_STRING_CONTENTS, trustStoreFile, Charsets.UTF_8) + val clientCertFile = new File(tempSslDir, "cert.pem") + Files.write(CLIENT_CERT_STRING_CONTENTS, clientCertFile, Charsets.UTF_8) + val builder = new SubmittedDependencySecretBuilderImpl( + SECRET_NAME, + JARS_SECRET, + FILES_SECRET, + JARS_SECRET_KEY, + FILES_SECRET_KEY, + TRUSTSTORE_SECRET_KEY, + CLIENT_CERT_SECRET_KEY, + Some(trustStoreFile.getAbsolutePath), + Some(clientCertFile.getAbsolutePath)) + val secret = builder.build() + val decodedSecretData = decodeSecretData(secret) + assert(decodedSecretData(TRUSTSTORE_SECRET_KEY) === TRUSTSTORE_STRING_CONTENTS) + assert(decodedSecretData(CLIENT_CERT_SECRET_KEY) === CLIENT_CERT_STRING_CONTENTS) + } finally { + tempSslDir.delete() + } + } + + test("If trustStore and certificate are container-local, don't add secret entries") { + val builder = new SubmittedDependencySecretBuilderImpl( + SECRET_NAME, + JARS_SECRET, + FILES_SECRET, + JARS_SECRET_KEY, + FILES_SECRET_KEY, + TRUSTSTORE_SECRET_KEY, + CLIENT_CERT_SECRET_KEY, + Some("local:///mnt/secrets/trustStore.jks"), + Some("local:///mnt/secrets/cert.pem")) + val secret = builder.build() + val decodedSecretData = decodeSecretData(secret) + assert(!decodedSecretData.contains(TRUSTSTORE_SECRET_KEY)) + assert(!decodedSecretData.contains(CLIENT_CERT_SECRET_KEY)) + } + +} 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 650ec4feb6a2b..d2082291eba22 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 @@ -34,7 +34,7 @@ import org.apache.spark.deploy.kubernetes.config._ import org.apache.spark.deploy.kubernetes.integrationtest.backend.IntegrationTestBackendFactory import org.apache.spark.deploy.kubernetes.integrationtest.backend.minikube.Minikube import org.apache.spark.deploy.kubernetes.integrationtest.constants.MINIKUBE_TEST_BACKEND -import org.apache.spark.deploy.kubernetes.submit.{Client, ClientArguments, JavaMainAppResource, KeyAndCertPem, MainAppResource, PythonMainAppResource} +import org.apache.spark.deploy.kubernetes.submit.{Client, KeyAndCertPem} import org.apache.spark.launcher.SparkLauncher import org.apache.spark.util.Utils @@ -82,7 +82,8 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { .set(EXECUTOR_DOCKER_IMAGE, System.getProperty("spark.docker.test.executorImage", "spark-executor-py:latest")) - runPySparkPiAndVerifyCompletion(PYSPARK_PI_SUBMITTER_LOCAL_FILE_LOCATION, Seq.empty[String]) + runPySparkPiAndVerifyCompletion( + PYSPARK_PI_SUBMITTER_LOCAL_FILE_LOCATION) } test("Run PySpark Job on file from CONTAINER with spark.jar defined") { @@ -95,7 +96,8 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { .set(EXECUTOR_DOCKER_IMAGE, System.getProperty("spark.docker.test.executorImage", "spark-executor-py:latest")) - runPySparkPiAndVerifyCompletion(PYSPARK_PI_CONTAINER_LOCAL_FILE_LOCATION, Seq.empty[String]) + runPySparkPiAndVerifyCompletion( + PYSPARK_PI_CONTAINER_LOCAL_FILE_LOCATION) } test("Simple submission test with the resource staging server.") { @@ -152,11 +154,10 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { sparkConf.set("spark.kubernetes.shuffle.namespace", kubernetesTestComponents.namespace) sparkConf.set("spark.app.name", "group-by-test") runSparkApplicationAndVerifyCompletion( - JavaMainAppResource(SUBMITTER_LOCAL_MAIN_APP_RESOURCE), + SUBMITTER_LOCAL_MAIN_APP_RESOURCE, GROUP_BY_MAIN_CLASS, "The Result is", - Array.empty[String], - Seq.empty[String]) + Array.empty[String]) } test("Use remote resources without the resource staging server.") { @@ -216,11 +217,10 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { launchStagingServer(SSLOptions(), None) sparkConf.set("spark.files", testExistenceFile.getAbsolutePath) runSparkApplicationAndVerifyCompletion( - JavaMainAppResource(SUBMITTER_LOCAL_MAIN_APP_RESOURCE), + SUBMITTER_LOCAL_MAIN_APP_RESOURCE, FILE_EXISTENCE_MAIN_CLASS, s"File found at /opt/spark/${testExistenceFile.getName} with correct contents.", - Array(testExistenceFile.getName, TEST_EXISTENCE_FILE_CONTENTS), - Seq.empty[String]) + Array(testExistenceFile.getName, TEST_EXISTENCE_FILE_CONTENTS)) } test("Use a very long application name.") { @@ -248,35 +248,26 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { private def runSparkPiAndVerifyCompletion(appResource: String): Unit = { runSparkApplicationAndVerifyCompletion( - JavaMainAppResource(appResource), - SPARK_PI_MAIN_CLASS, - "Pi is roughly 3", - Array.empty[String], - Seq.empty[String]) + appResource, SPARK_PI_MAIN_CLASS, "Pi is roughly 3", Array.empty[String]) } private def runPySparkPiAndVerifyCompletion( - appResource: String, otherPyFiles: Seq[String]): Unit = { + appResource: String): Unit = { runSparkApplicationAndVerifyCompletion( - PythonMainAppResource(appResource), - PYSPARK_PI_MAIN_CLASS, - "Pi is roughly 3", - Array("5"), - otherPyFiles) + appResource, PYSPARK_PI_MAIN_CLASS, "Pi is roughly 3", + Array(null, "5")) } private def runSparkApplicationAndVerifyCompletion( - appResource: MainAppResource, + appResource: String, mainClass: String, expectedLogOnCompletion: String, - appArgs: Array[String], - otherPyFiles: Seq[String]): Unit = { - val clientArguments = ClientArguments( - mainAppResource = appResource, + appArgs: Array[String]): Unit = { + Client.run( + sparkConf = sparkConf, + appArgs = appArgs, mainClass = mainClass, - driverArgs = appArgs, - otherPyFiles = otherPyFiles) - Client.run(sparkConf, clientArguments) + mainAppResource = appResource) val driverPod = kubernetesTestComponents.kubernetesClient .pods() .withLabel("spark-app-locator", APP_LOCATOR_LABEL) From 41b6b8cae4dc1b580d2bc2fe61140c7b8f5ce21a Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Fri, 30 Jun 2017 16:13:25 -0700 Subject: [PATCH 25/26] style changes --- .../org/apache/spark/deploy/kubernetes/submit/Client.scala | 4 ++-- .../kubernetes/submit/ContainerLocalizedFilesResolver.scala | 2 +- .../deploy/kubernetes/submit/PythonSubmissionResources.scala | 4 ++-- .../kubernetes/submit/PythonSubmissionResourcesSuite.scala | 3 --- 4 files changed, 5 insertions(+), 8 deletions(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/Client.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/Client.scala index e61cece142e3d..781ecbd6c5416 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/Client.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/Client.scala @@ -301,8 +301,8 @@ private[spark] object Client { Option(new PythonSubmissionResourcesImpl(mainAppResource, appArgs)) } else None // Since you might need jars for SQL UDFs in PySpark - def sparkJarFilter() : Seq[String] = - pythonResource.map { p => p.sparkJars}.getOrElse( + def sparkJarFilter(): Seq[String] = + pythonResource.map {p => p.sparkJars}.getOrElse( Option(mainAppResource) .filterNot(_ == SparkLauncher.NO_RESOURCE) .toSeq) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/ContainerLocalizedFilesResolver.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/ContainerLocalizedFilesResolver.scala index 7345589c59c42..c31aa5f306bea 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/ContainerLocalizedFilesResolver.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/ContainerLocalizedFilesResolver.scala @@ -34,7 +34,7 @@ private[spark] class ContainerLocalizedFilesResolverImpl( pySparkFiles: Seq[String], primaryPyFile: String, jarsDownloadPath: String, - filesDownloadPath: String ) extends ContainerLocalizedFilesResolver { + filesDownloadPath: String) extends ContainerLocalizedFilesResolver { override def resolveSubmittedAndRemoteSparkJars(): Seq[String] = { diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/PythonSubmissionResources.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/PythonSubmissionResources.scala index e615cf72116e1..c61e930a2b97f 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/PythonSubmissionResources.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/PythonSubmissionResources.scala @@ -29,7 +29,7 @@ private[spark] trait PythonSubmissionResources { resolvedPrimaryPySparkResource: String, resolvedPySparkFiles: String, driverContainerName: String, - driverPodBuilder: PodBuilder) : Pod + driverPodBuilder: PodBuilder): Pod } private[spark] class PythonSubmissionResourcesImpl( @@ -54,7 +54,7 @@ private[spark] class PythonSubmissionResourcesImpl( } } } - override def primaryPySparkResource ( + override def primaryPySparkResource( containerLocalizedFilesResolver: ContainerLocalizedFilesResolver) : String = containerLocalizedFilesResolver.resolvePrimaryResourceFile() diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/PythonSubmissionResourcesSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/PythonSubmissionResourcesSuite.scala index 5483dc154fe73..9b60b7ef2b786 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/PythonSubmissionResourcesSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/PythonSubmissionResourcesSuite.scala @@ -25,9 +25,6 @@ import org.mockito.{Mock, MockitoAnnotations} import org.mockito.Mockito.when import org.scalatest.BeforeAndAfter - - - private[spark] class PythonSubmissionResourcesSuite extends SparkFunSuite with BeforeAndAfter { private val PYSPARK_FILES = Seq( "hdfs://localhost:9000/app/files/file1.py", From 923f956afb2420ca607965ce7e9803d135e5b37d Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Fri, 30 Jun 2017 16:15:38 -0700 Subject: [PATCH 26/26] space for styling --- .../submit/DriverInitContainerComponentsProvider.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/DriverInitContainerComponentsProvider.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/DriverInitContainerComponentsProvider.scala index 7f5a62e97abc0..6e185d2c069f6 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/DriverInitContainerComponentsProvider.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/DriverInitContainerComponentsProvider.scala @@ -33,7 +33,7 @@ import org.apache.spark.util.Utils private[spark] trait DriverInitContainerComponentsProvider { def provideContainerLocalizedFilesResolver( - mainAppResource: String) : ContainerLocalizedFilesResolver + mainAppResource: String): ContainerLocalizedFilesResolver def provideInitContainerSubmittedDependencyUploader( driverPodLabels: Map[String, String]): Option[SubmittedDependencyUploader] def provideSubmittedDependenciesSecretBuilder(