From 5499f6ddf9b42c0526f1dc053317afb38dc71294 Mon Sep 17 00:00:00 2001 From: mcheah Date: Thu, 29 Jun 2017 16:52:02 -0700 Subject: [PATCH] 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)