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..df50af13f71a3 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,17 @@ object SparkSubmit { if (isKubernetesCluster) { childMainClass = "org.apache.spark.deploy.kubernetes.submit.Client" if (args.isPython) { - childArgs += args.primaryResource - childArgs += "org.apache.spark.deploy.PythonRunner" - childArgs += args.pyFiles + childArgs ++= Array("--primary-py-file", args.primaryResource) + childArgs ++= Array("--main-class", "org.apache.spark.deploy.PythonRunner") + childArgs ++= Array("--other-py-files", args.pyFiles) } else { - childArgs += args.primaryResource - childArgs += args.mainClass + childArgs ++= Array("--primary-java-resource", args.primaryResource) + childArgs ++= Array("--main-class", args.mainClass) + } + args.childArgs.foreach { arg => + childArgs += "--arg" + childArgs += arg } - childArgs ++= args.childArgs } // Load any properties specified through --conf and the default properties file diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/InitContainerResourceStagingServerSecretPlugin.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/InitContainerResourceStagingServerSecretPlugin.scala index 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..2df7ac7a204c4 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,11 @@ 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 = { + podWithDetachedInitContainer: PodWithDetachedInitContainer): PodWithDetachedInitContainer = { val sharedVolumeMounts = Seq[VolumeMount]( new VolumeMountBuilder() .withName(INIT_CONTAINER_DOWNLOAD_JARS_VOLUME_NAME) @@ -58,7 +62,7 @@ private[spark] class SparkPodInitContainerBootstrapImpl( .withMountPath(filesDownloadPath) .build()) - val initContainer = new ContainerBuilder() + val initContainer = new ContainerBuilder(podWithDetachedInitContainer.initContainer) .withName(s"spark-init") .withImage(initContainerImage) .withImagePullPolicy(dockerImagePullPolicy) @@ -68,11 +72,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(podWithDetachedInitContainer.pod) .editSpec() .addNewVolume() .withName(INIT_CONTAINER_PROPERTIES_FILE_VOLUME) @@ -92,17 +93,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( + podWithDetachedInitContainer.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/submit/Client.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/Client.scala index 781ecbd6c5416..98cd7afcd204d 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,233 +16,99 @@ */ 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, 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.submit.submitsteps.{DriverConfigurationStep, KubernetesDriverSpec} +import org.apache.spark.deploy.kubernetes.SparkKubernetesClientFactory 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("--primary-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 --primary-py-file or --primary-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[DriverConfigurationStep], + 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) + /** + * Run command that initalizes a DriverSpec that will be updated after each + * DriverConfigurationStep in the sequence that is passed in. The final KubernetesDriverSpec + * will be used to build the Driver Container, Driver Pod, and Kubernetes Resources + */ 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 = KubernetesDriverSpec.initialSpec(submissionSparkConf) + // submissionSteps contain steps necessary to take, to resolve varying + // client arguments that are passed in, created by orchestrator + for (nextStep <- submissionSteps) { + currentDriverSpec = nextStep.configureDriver(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: _*) + val resolvedDriverJavaOpts = currentDriverSpec + .driverSparkConf + // 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) + .getAll + .map { + case (confKey, confValue) => s"-D$confKey=$confValue" + }.mkString(" ") + driverJavaOptions.map(" " + _).getOrElse("") + val resolvedDriverContainer = new ContainerBuilder(currentDriverSpec.driverContainer) .addNewEnv() - .withName(ENV_DRIVER_MEMORY) - .withValue(driverContainerMemoryWithOverhead + "m") + .withName(ENV_DRIVER_JAVA_OPTS) + .withValue(resolvedDriverJavaOpts) .endEnv() - .addNewEnv() - .withName(ENV_DRIVER_MAIN_CLASS) - .withValue(mainClass) - .endEnv() - .addNewEnv() - .withName(ENV_DRIVER_ARGS) - .withValue(arguments.mkString(" ")) - .endEnv() - .withNewResources() - .addToRequests("cpu", driverCpuQuantity) - .addToRequests("memory", driverMemoryQuantity) - .addToLimits("memory", driverMemoryLimitQuantity) - .endResources() .build() - val 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() @@ -250,22 +116,21 @@ private[spark] class Client( .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,61 +148,26 @@ 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]) - 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("-", "")}" + def run(sparkConf: SparkConf, clientArguments: ClientArguments): Unit = { val namespace = sparkConf.get(KUBERNETES_NAMESPACE) + val kubernetesAppId = s"spark-${UUID.randomUUID().toString.replaceAll("-", "")}" + val launchTime = System.currentTimeMillis() + val waitForAppCompletion = sparkConf.get(WAIT_FOR_APP_COMPLETION) + val appName = sparkConf.getOption("spark.app.name").getOrElse("spark") val master = resolveK8sMaster(sparkConf.get("spark.master")) - val sslOptionsProvider = new ResourceStagingServerSslOptionsProviderImpl(sparkConf) - val initContainerComponentsProvider = new DriverInitContainerComponentsProviderImpl( - sparkConf, - kubernetesResourceNamePrefix, + val loggingInterval = Option(sparkConf.get(REPORT_INTERVAL)).filter( _ => waitForAppCompletion) + val loggingPodStatusWatcher = new LoggingPodStatusWatcherImpl( + kubernetesAppId, loggingInterval) + val configurationStepsOrchestrator = new DriverConfigurationStepsOrchestrator( namespace, - sparkJars, - sparkFiles, - pySparkFiles, - sslOptionsProvider.getSslOptions) + kubernetesAppId, + launchTime, + clientArguments.mainAppResource, + appName, + clientArguments.mainClass, + clientArguments.driverArgs, + clientArguments.otherPyFiles, + sparkConf) Utils.tryWithResource(SparkKubernetesClientFactory.createKubernetesClient( master, Some(namespace), @@ -345,38 +175,25 @@ private[spark] object Client { 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) new Client( - appName, - kubernetesResourceNamePrefix, - kubernetesAppId, - mainAppResource, - pythonResource, - mainClass, + configurationStepsOrchestrator.getAllConfigurationSteps(), sparkConf, - appArgs, - waitForAppCompletion, kubernetesClient, - initContainerComponentsProvider, - kubernetesCredentialsMounterProvider, + 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") - } + + /** + * Entry point from SparkSubmit in spark-core + * + * @param args Array of strings that have interchanging values that will be + * parsed by ClientArguments with the identifiers that precede the values + */ + 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 c31aa5f306bea..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/DriverConfigurationStepsOrchestrator.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/DriverConfigurationStepsOrchestrator.scala new file mode 100644 index 0000000000000..82abe55ac6989 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/DriverConfigurationStepsOrchestrator.scala @@ -0,0 +1,138 @@ +/* + * 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.{BaseDriverConfigurationStep, DependencyResolutionStep, DriverConfigurationStep, DriverKubernetesCredentialsStep, InitContainerBootstrapStep, PythonStep} +import org.apache.spark.deploy.kubernetes.submit.submitsteps.initcontainer.InitContainerConfigurationStepsOrchestrator +import org.apache.spark.launcher.SparkLauncher +import org.apache.spark.util.Utils + +/** + * Constructs the complete list of driver configuration steps to run to deploy the Spark driver. + */ +private[spark] class DriverConfigurationStepsOrchestrator( + 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 getAllConfigurationSteps(): Seq[DriverConfigurationStep] = { + 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 ++ + additionalPythonFiles + 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 BaseDriverConfigurationStep( + kubernetesAppId, + kubernetesResourceNamePrefix, + allDriverLabels, + dockerImagePullPolicy, + appName, + mainClass, + appArgs, + submissionSparkConf) + val kubernetesCredentialsStep = new DriverKubernetesCredentialsStep( + submissionSparkConf, kubernetesResourceNamePrefix) + val pythonStep = mainAppResource match { + case PythonMainAppResource(mainPyResource) => + Option(new PythonStep(mainPyResource, additionalPythonFiles, filesDownloadPath)) + case _ => Option.empty[DriverConfigurationStep] + } + val initContainerBootstrapStep = if ((sparkJars ++ sparkFiles).exists { uri => + Option(Utils.resolveURI(uri).getScheme).getOrElse("file") != "local" + }) { + val initContainerConfigurationStepsOrchestrator = + new InitContainerConfigurationStepsOrchestrator( + namespace, + kubernetesResourceNamePrefix, + sparkJars, + sparkFiles, + jarsDownloadPath, + filesDownloadPath, + dockerImagePullPolicy, + allDriverLabels, + initContainerConfigMapName, + INIT_CONTAINER_CONFIG_MAP_KEY, + submissionSparkConf) + val initContainerConfigurationSteps = + initContainerConfigurationStepsOrchestrator.getAllConfigurationSteps() + Some(new InitContainerBootstrapStep(initContainerConfigurationSteps, + initContainerConfigMapName, + INIT_CONTAINER_CONFIG_MAP_KEY)) + } else { + Option.empty[DriverConfigurationStep] + } + val dependencyResolutionStep = new DependencyResolutionStep( + sparkJars, + sparkFiles, + jarsDownloadPath, + filesDownloadPath) + Seq( + initialSubmissionStep, + kubernetesCredentialsStep, + dependencyResolutionStep) ++ + initContainerBootstrapStep.toSeq ++ + pythonStep.toSeq + } +} 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 6e185d2c069f6..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/ExecutorInitContainerConfiguration.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/ExecutorInitContainerConfiguration.scala deleted file mode 100644 index 2292365995d1f..0000000000000 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/ExecutorInitContainerConfiguration.scala +++ /dev/null @@ -1,47 +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 ExecutorInitContainerConfiguration { - /** - * Provide the driver with configuration that allows it to configure executors to - * fetch resources in the same way the driver does. - */ - def configureSparkConfForExecutorInitContainer(originalSparkConf: SparkConf): SparkConf -} - -private[spark] class ExecutorInitContainerConfigurationImpl( - initContainerSecretName: Option[String], - initContainerSecretMountDir: String, - initContainerConfigMapName: String, - initContainerConfigMapKey: String) - extends ExecutorInitContainerConfiguration { - def configureSparkConfForExecutorInitContainer(originalSparkConf: SparkConf): SparkConf = { - val configuredSparkConf = originalSparkConf.clone() - .set(EXECUTOR_INIT_CONTAINER_CONFIG_MAP, - initContainerConfigMapName) - .set(EXECUTOR_INIT_CONTAINER_CONFIG_MAP_KEY, - initContainerConfigMapKey) - .set(EXECUTOR_INIT_CONTAINER_SECRET_MOUNT_DIR, initContainerSecretMountDir) - initContainerSecretName.map { secret => - configuredSparkConf.set(EXECUTOR_INIT_CONTAINER_SECRET, secret) - }.getOrElse(configuredSparkConf) - } -} 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..837ec0e8c867e 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,19 +18,18 @@ 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._ private[spark] object InitContainerUtil { - private val OBJECT_MAPPER = new ObjectMapper().registerModule(new DefaultScalaModule) + private val OBJECT_MAPPER = new ObjectMapper().registerModule(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..ec591923f1472 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 given 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/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 c61e930a2b97f..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/BaseDriverConfigurationStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/BaseDriverConfigurationStep.scala new file mode 100644 index 0000000000000..022b5fccdc5e1 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/BaseDriverConfigurationStep.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 + +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 BaseDriverConfigurationStep( + kubernetesAppId: String, + kubernetesResourceNamePrefix: String, + driverLabels: Map[String, String], + dockerImagePullPolicy: String, + appName: String, + mainClass: String, + appArgs: Array[String], + submissionSparkConf: SparkConf) extends DriverConfigurationStep { + + 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.get(KUBERNETES_DRIVER_LIMIT_CORES) + + // 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 configureDriver( + driverSpec: KubernetesDriverSpec): KubernetesDriverSpec = { + val driverExtraClasspathEnv = driverExtraClasspath.map { classPath => + new EnvVarBuilder() + .withName(ENV_SUBMIT_EXTRA_CLASSPATH) + .withValue(classPath) + .build() + } + val driverCustomAnnotations = ConfigurationUtils + .combinePrefixedKeyValuePairsWithDeprecatedConf( + submissionSparkConf, + 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 allDriverAnnotations = driverCustomAnnotations ++ Map(SPARK_APP_NAME_ANNOTATION -> appName) + 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) + .editOrNewMetadata() + .withName(kubernetesDriverPodName) + .addToLabels(driverLabels.asJava) + .addToAnnotations(allDriverAnnotations.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) + driverSpec.copy( + driverPod = baseDriverPod, + driverSparkConf = resolvedSparkConf, + driverContainer = driverContainer) + } +} 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..dddc62410d6c9 --- /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 DriverConfigurationStep { + + override def configureDriver(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/DriverConfigurationStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DriverConfigurationStep.scala new file mode 100644 index 0000000000000..8070e32371f94 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DriverConfigurationStep.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 DriverConfigurationStep { + + /** + * Apply some transformation to the previous state of the driver to add a new feature to it. + */ + def configureDriver(driverSpec: KubernetesDriverSpec): KubernetesDriverSpec +} 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..0c58006130659 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DriverKubernetesCredentialsStep.scala @@ -0,0 +1,222 @@ +/* + * 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._ + +/** + * Mounts Kubernetes credentials into the driver pod. The driver will use such mounted credentials + * to request executors. + */ +private[spark] class DriverKubernetesCredentialsStep( + submissionSparkConf: SparkConf, + kubernetesResourceNamePrefix: String) extends DriverConfigurationStep { + + 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") + + override def configureDriver(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) + .editOrNewSpec() + .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"$kubernetesResourceNamePrefix-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..29cad18c484c0 --- /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.deploy.kubernetes.config._ +import org.apache.spark.deploy.kubernetes.submit.{InitContainerUtil, PropertiesConfigMapFromScalaMapBuilder} +import org.apache.spark.deploy.kubernetes.submit.submitsteps.initcontainer.{InitContainerConfigurationStep, InitContainerSpec} + +/** + * Configures the init-container that bootstraps dependencies into the driver pod. + */ +private[spark] class InitContainerBootstrapStep( + initContainerConfigurationSteps: Seq[InitContainerConfigurationStep], + initContainerConfigMapName: String, + initContainerConfigMapKey: String) + extends DriverConfigurationStep { + + override def configureDriver(driverSpec: KubernetesDriverSpec): KubernetesDriverSpec = { + var currentInitContainerSpec = 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]) + for (nextStep <- initContainerConfigurationSteps) { + currentInitContainerSpec = nextStep.configureInitContainer(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..3ec4b6c4df10f --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/KubernetesDriverSpec.scala @@ -0,0 +1,47 @@ +/* + * 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, ContainerBuilder, HasMetadata, Pod, PodBuilder} + +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) + +private[spark] object KubernetesDriverSpec { + def initialSpec(initialSparkConf: SparkConf): KubernetesDriverSpec = { + KubernetesDriverSpec( + // Set new metadata and a new spec so that submission steps can use + // PodBuilder#editMetadata() and/or PodBuilder#editSpec() safely. + new PodBuilder().withNewMetadata().endMetadata().withNewSpec().endSpec().build(), + new ContainerBuilder().build(), + Seq.empty[HasMetadata], + initialSparkConf.clone()) + } +} 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..024d643ddf9fd --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/PythonStep.scala @@ -0,0 +1,46 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +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 DriverConfigurationStep { + + override def configureDriver(driverSpec: KubernetesDriverSpec): KubernetesDriverSpec = { + val resolvedOtherPyFilesString = if (otherPyFiles.isEmpty) { + "null" + } else { + KubernetesFileUtils.resolveFilePaths(otherPyFiles, filesDownloadPath).mkString(",") + } + val withPythonPrimaryFileContainer = new ContainerBuilder(driverSpec.driverContainer) + .addNewEnv() + .withName(ENV_PYSPARK_PRIMARY) + .withValue(KubernetesFileUtils.resolveFilePath(primaryPyFile, filesDownloadPath)) + .endEnv() + .addNewEnv() + .withName(ENV_PYSPARK_FILES) + .withValue(resolvedOtherPyFilesString) + .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/BaseInitContainerConfigurationStep.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/BaseInitContainerConfigurationStep.scala index 4062a3113eddf..60bf27beacaaf 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/BaseInitContainerConfigurationStep.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 BaseInitContainerConfigurationStep( sparkJars: Seq[String], sparkFiles: Seq[String], jarsDownloadPath: String, filesDownloadPath: String, configMapName: String, configMapKey: String, - submittedDependenciesPlugin: Option[SubmittedDependencyInitContainerConfigPlugin]) - extends SparkInitContainerConfigMapBuilder { + podAndInitContainerBootstrap: SparkPodInitContainerBootstrap) + extends InitContainerConfigurationStep { - override def build(): ConfigMap = { + override def configureInitContainer(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/InitContainerConfigurationStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/InitContainerConfigurationStep.scala new file mode 100644 index 0000000000000..7b7622c3d4f8b --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/InitContainerConfigurationStep.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 InitContainerConfigurationStep { + + def configureInitContainer(initContainerSpec: InitContainerSpec): InitContainerSpec +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/InitContainerConfigurationStepsOrchestrator.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/InitContainerConfigurationStepsOrchestrator.scala new file mode 100644 index 0000000000000..e4ea5235af18f --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/InitContainerConfigurationStepsOrchestrator.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 InitContainerConfigurationStepsOrchestrator( + 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 getAllConfigurationSteps(): Seq[InitContainerConfigurationStep] = { + val initContainerBootstrap = new SparkPodInitContainerBootstrapImpl( + initContainerImage, + dockerImagePullPolicy, + jarsDownloadPath, + filesDownloadPath, + downloadTimeoutMinutes, + initContainerConfigMapName, + initContainerConfigMapKey) + val baseInitContainerStep = new BaseInitContainerConfigurationStep( + 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 SubmittedResourcesInitContainerConfigurationStep( + 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/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/SubmittedResourcesInitContainerConfigurationStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/SubmittedResourcesInitContainerConfigurationStep.scala new file mode 100644 index 0000000000000..7aa27a1de6811 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/SubmittedResourcesInitContainerConfigurationStep.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 SubmittedResourcesInitContainerConfigurationStep( + 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 InitContainerConfigurationStep { + + override def configureInitContainer(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..e5f980ad1f366 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,21 @@ 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 +449,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 +631,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/InitContainerResourceStagingServerSecretPluginSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/InitContainerResourceStagingServerSecretPluginSuite.scala new file mode 100644 index 0000000000000..f5b2db36aff8f --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/InitContainerResourceStagingServerSecretPluginSuite.scala @@ -0,0 +1,59 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.kubernetes + +import org.scalatest.BeforeAndAfter +import io.fabric8.kubernetes.api.model._ +import org.apache.spark.deploy.kubernetes.constants._ + +import scala.collection.JavaConverters._ +import org.apache.spark.SparkFunSuite + +class InitContainerResourceStagingServerSecretPluginSuite extends SparkFunSuite with BeforeAndAfter{ + private val INIT_CONTAINER_SECRET_NAME = "init-secret" + private val INIT_CONTAINER_SECRET_MOUNT = "/tmp/secret" + + private val initContainerSecretPlugin = new InitContainerResourceStagingServerSecretPluginImpl( + INIT_CONTAINER_SECRET_NAME, + INIT_CONTAINER_SECRET_MOUNT) + + test("Volume Mount into InitContainer") { + val returnedCont = initContainerSecretPlugin.mountResourceStagingServerSecretIntoInitContainer( + new ContainerBuilder().withName("init-container").build()) + assert(returnedCont.getName === "init-container") + assert(returnedCont.getVolumeMounts.asScala.map( + vm => (vm.getName, vm.getMountPath)) === + List((INIT_CONTAINER_SECRET_VOLUME_NAME, INIT_CONTAINER_SECRET_MOUNT))) + } + + test("Add Volume with Secret to Pod") { + val returnedPod = initContainerSecretPlugin.addResourceStagingServerSecretVolumeToPod( + basePod().build) + assert(returnedPod.getMetadata.getName === "spark-pod") + val volume = returnedPod.getSpec.getVolumes.asScala.head + assert(volume.getName === INIT_CONTAINER_SECRET_VOLUME_NAME) + assert(volume.getSecret.getSecretName === INIT_CONTAINER_SECRET_NAME) + } + private def basePod(): PodBuilder = { + new PodBuilder() + .withNewMetadata() + .withName("spark-pod") + .endMetadata() + .withNewSpec() + .endSpec() + } +} 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..0557b5677b919 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,16 +16,14 @@ */ 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 io.fabric8.kubernetes.api.model._ +import org.apache.spark.deploy.kubernetes.constants._ +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" @@ -33,134 +31,66 @@ class SparkPodInitContainerBootstrapSuite extends SparkFunSuite with BeforeAndAf 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() - } + private val sparkPodInit = 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) + private val expectedSharedVolumeMap = Map( + JARS_DOWNLOAD_PATH -> INIT_CONTAINER_DOWNLOAD_JARS_VOLUME_NAME, + FILES_DOWNLOAD_PATH -> INIT_CONTAINER_DOWNLOAD_FILES_VOLUME_NAME) - 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) + test("InitContainer: Volume mounts, args, and builder specs") { + val returnedPodWithCont = sparkPodInit.bootstrapInitContainerAndVolumes( + PodWithDetachedInitContainer( + pod = basePod().build(), + initContainer = new Container(), + mainContainer = new ContainerBuilder().withName(MAIN_CONTAINER_NAME).build())) + val initContainer: Container = returnedPodWithCont.initContainer + val volumes = initContainer.getVolumeMounts.asScala + assert(volumes.map(vm => (vm.getMountPath, vm.getName)).toMap === expectedSharedVolumeMap + ++ Map("/etc/spark-init" -> "spark-init-properties")) 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)) + assert(initContainer.getImagePullPolicy === DOCKER_IMAGE_PULL_POLICY) + assert(initContainer.getArgs.asScala.head === 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("Main: Volume mounts and env") { + val returnedPodWithCont = sparkPodInit.bootstrapInitContainerAndVolumes( + PodWithDetachedInitContainer( + pod = basePod().build(), + initContainer = new Container(), + mainContainer = new ContainerBuilder().withName(MAIN_CONTAINER_NAME).build())) + val mainContainer: Container = returnedPodWithCont.mainContainer + assert(mainContainer.getName === MAIN_CONTAINER_NAME) + val volumeMounts = mainContainer.getVolumeMounts.asScala + assert(volumeMounts.map(vm => (vm.getMountPath, vm.getName)).toMap === expectedSharedVolumeMap) + assert(mainContainer.getEnv.asScala.map(e => (e.getName, e.getValue)).toMap === + Map(ENV_MOUNTED_FILES_DIR -> FILES_DOWNLOAD_PATH)) } - - 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() + test("Pod: Volume Mounts") { + val returnedPodWithCont = sparkPodInit.bootstrapInitContainerAndVolumes( + PodWithDetachedInitContainer( + pod = basePod().build(), + initContainer = new Container(), + mainContainer = new ContainerBuilder().withName(MAIN_CONTAINER_NAME).build())) + val returnedPod = returnedPodWithCont.pod + assert(returnedPod.getMetadata.getName === "spark-pod") + val volumes = returnedPod.getSpec.getVolumes.asScala.toList + assert(volumes.head.getName === INIT_CONTAINER_PROPERTIES_FILE_VOLUME) + assert(volumes.head.getConfigMap.getName === INIT_CONTAINER_CONFIG_MAP_NAME) + assert(volumes.head.getConfigMap.getItems.asScala.map( + i => (i.getKey, i.getPath)) === + List((INIT_CONTAINER_CONFIG_MAP_KEY, INIT_CONTAINER_PROPERTIES_FILE_NAME))) + assert(volumes(1).getName === INIT_CONTAINER_DOWNLOAD_JARS_VOLUME_NAME) + assert(volumes(1).getEmptyDir === new EmptyDirVolumeSource()) + assert(volumes(2).getName === INIT_CONTAINER_DOWNLOAD_FILES_VOLUME_NAME) + assert(volumes(2).getEmptyDir === new EmptyDirVolumeSource()) } private def basePod(): PodBuilder = { @@ -169,9 +99,6 @@ class SparkPodInitContainerBootstrapSuite extends SparkFunSuite with BeforeAndAf .withName("spark-pod") .endMetadata() .withNewSpec() - .addNewContainer() - .withName(MAIN_CONTAINER_NAME) - .endContainer() - .endSpec() + .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/ClientSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/ClientSuite.scala new file mode 100644 index 0000000000000..965ee75c248b8 --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/ClientSuite.scala @@ -0,0 +1,226 @@ +/* + * 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 com.google.common.collect.Iterables +import io.fabric8.kubernetes.api.model.{ContainerBuilder, DoneablePod, HasMetadata, Pod, PodBuilder, PodList, Secret, SecretBuilder} +import io.fabric8.kubernetes.client.{KubernetesClient, Watch} +import io.fabric8.kubernetes.client.dsl.{MixedOperation, NamespaceListVisitFromServerGetDeleteRecreateWaitApplicable, NamespaceVisitFromServerGetWatchDeleteRecreateWaitApplicable, PodResource, Resource} +import org.mockito.{ArgumentCaptor, Mock, MockitoAnnotations} +import org.mockito.Mockito.{doReturn, verify, when} +import org.mockito.invocation.InvocationOnMock +import org.mockito.stubbing.Answer +import org.scalatest.BeforeAndAfter +import org.scalatest.mock.MockitoSugar._ +import scala.collection.JavaConverters._ + +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.deploy.kubernetes.constants._ +import org.apache.spark.deploy.kubernetes.submit.submitsteps.{DriverConfigurationStep, KubernetesDriverSpec} + +class ClientSuite extends SparkFunSuite with BeforeAndAfter { + + private val DRIVER_POD_UID = "pod-id" + private val DRIVER_POD_API_VERSION = "v1" + private val DRIVER_POD_KIND = "pod" + + private type ResourceList = NamespaceListVisitFromServerGetDeleteRecreateWaitApplicable[ + HasMetadata, Boolean] + private type Pods = MixedOperation[Pod, PodList, DoneablePod, PodResource[Pod, DoneablePod]] + + @Mock + private var kubernetesClient: KubernetesClient = _ + + @Mock + private var podOperations: Pods = _ + + @Mock + private var namedPods: PodResource[Pod, DoneablePod] = _ + + @Mock + private var loggingPodStatusWatcher: LoggingPodStatusWatcher = _ + + @Mock + private var resourceList: ResourceList = _ + + private val submissionSteps = Seq(FirstTestConfigurationStep, SecondTestConfigurationStep) + private var createdPodArgumentCaptor: ArgumentCaptor[Pod] = _ + private var createdResourcesArgumentCaptor: ArgumentCaptor[HasMetadata] = _ + + before { + MockitoAnnotations.initMocks(this) + when(kubernetesClient.pods()).thenReturn(podOperations) + when(podOperations.withName(FirstTestConfigurationStep.podName)).thenReturn(namedPods) + + createdPodArgumentCaptor = ArgumentCaptor.forClass(classOf[Pod]) + createdResourcesArgumentCaptor = ArgumentCaptor.forClass(classOf[HasMetadata]) + when(podOperations.create(createdPodArgumentCaptor.capture())).thenAnswer(new Answer[Pod] { + override def answer(invocation: InvocationOnMock): Pod = { + new PodBuilder(invocation.getArgumentAt(0, classOf[Pod])) + .editMetadata() + .withUid(DRIVER_POD_UID) + .endMetadata() + .withApiVersion(DRIVER_POD_API_VERSION) + .withKind(DRIVER_POD_KIND) + .build() + } + }) + when(podOperations.withName(FirstTestConfigurationStep.podName)).thenReturn(namedPods) + when(namedPods.watch(loggingPodStatusWatcher)).thenReturn(mock[Watch]) + doReturn(resourceList) + .when(kubernetesClient) + .resourceList(createdResourcesArgumentCaptor.capture()) + } + + test("The client should configure the pod with the submission steps.") { + val submissionClient = new Client( + submissionSteps, + new SparkConf(false), + kubernetesClient, + false, + "spark", + loggingPodStatusWatcher) + submissionClient.run() + val createdPod = createdPodArgumentCaptor.getValue + assert(createdPod.getMetadata.getName === FirstTestConfigurationStep.podName) + assert(createdPod.getMetadata.getLabels.asScala === + Map(FirstTestConfigurationStep.labelKey -> FirstTestConfigurationStep.labelValue)) + assert(createdPod.getMetadata.getAnnotations.asScala === + Map(SecondTestConfigurationStep.annotationKey -> + SecondTestConfigurationStep.annotationValue)) + assert(createdPod.getSpec.getContainers.size() === 1) + assert(createdPod.getSpec.getContainers.get(0).getName === + SecondTestConfigurationStep.containerName) + } + + test("The client should create the secondary Kubernetes resources.") { + val submissionClient = new Client( + submissionSteps, + new SparkConf(false), + kubernetesClient, + false, + "spark", + loggingPodStatusWatcher) + submissionClient.run() + val createdPod = createdPodArgumentCaptor.getValue + val otherCreatedResources = createdResourcesArgumentCaptor.getAllValues + assert(otherCreatedResources.size === 1) + val createdResource = Iterables.getOnlyElement(otherCreatedResources).asInstanceOf[Secret] + assert(createdResource.getMetadata.getName === FirstTestConfigurationStep.secretName) + assert(createdResource.getData.asScala === + Map(FirstTestConfigurationStep.secretKey -> FirstTestConfigurationStep.secretData)) + val ownerReference = Iterables.getOnlyElement(createdResource.getMetadata.getOwnerReferences) + assert(ownerReference.getName === createdPod.getMetadata.getName) + assert(ownerReference.getKind === DRIVER_POD_KIND) + assert(ownerReference.getUid === DRIVER_POD_UID) + assert(ownerReference.getApiVersion === DRIVER_POD_API_VERSION) + } + + test("The client should attach the driver container with the appropriate JVM options.") { + val sparkConf = new SparkConf(false) + .set("spark.logConf", "true") + .set( + org.apache.spark.internal.config.DRIVER_JAVA_OPTIONS, + "-XX:+|-HeapDumpOnOutOfMemoryError") + val submissionClient = new Client( + submissionSteps, + sparkConf, + kubernetesClient, + false, + "spark", + loggingPodStatusWatcher) + submissionClient.run() + val createdPod = createdPodArgumentCaptor.getValue + val driverContainer = Iterables.getOnlyElement(createdPod.getSpec.getContainers) + assert(driverContainer.getName === SecondTestConfigurationStep.containerName) + val driverJvmOptsEnv = Iterables.getOnlyElement(driverContainer.getEnv) + assert(driverJvmOptsEnv.getName === ENV_DRIVER_JAVA_OPTS) + val driverJvmOpts = driverJvmOptsEnv.getValue.split(" ").toSet + assert(driverJvmOpts.contains("-Dspark.logConf=true")) + assert(driverJvmOpts.contains( + s"-D${SecondTestConfigurationStep.sparkConfKey}=" + + SecondTestConfigurationStep.sparkConfValue)) + assert(driverJvmOpts.contains( + "-XX:+|-HeapDumpOnOutOfMemoryError")) + } + + test("Waiting for app completion should stall on the watcher") { + val submissionClient = new Client( + submissionSteps, + new SparkConf(false), + kubernetesClient, + true, + "spark", + loggingPodStatusWatcher) + submissionClient.run() + verify(loggingPodStatusWatcher).awaitCompletion() + } + +} + +private object FirstTestConfigurationStep extends DriverConfigurationStep { + + val podName = "test-pod" + val secretName = "test-secret" + val labelKey = "first-submit" + val labelValue = "true" + val secretKey = "secretKey" + val secretData = "secretData" + + override def configureDriver(driverSpec: KubernetesDriverSpec): KubernetesDriverSpec = { + val modifiedPod = new PodBuilder(driverSpec.driverPod) + .editMetadata() + .withName(podName) + .addToLabels(labelKey, labelValue) + .endMetadata() + .build() + val additionalResource = new SecretBuilder() + .withNewMetadata() + .withName(secretName) + .endMetadata() + .addToData(secretKey, secretData) + .build() + driverSpec.copy( + driverPod = modifiedPod, + otherKubernetesResources = driverSpec.otherKubernetesResources ++ Seq(additionalResource)) + } +} + +private object SecondTestConfigurationStep extends DriverConfigurationStep { + + val annotationKey = "second-submit" + val annotationValue = "submitted" + val sparkConfKey = "spark.custom-conf" + val sparkConfValue = "custom-conf-value" + val containerName = "driverContainer" + + override def configureDriver(driverSpec: KubernetesDriverSpec): KubernetesDriverSpec = { + val modifiedPod = new PodBuilder(driverSpec.driverPod) + .editMetadata() + .addToAnnotations(annotationKey, annotationValue) + .endMetadata() + .build() + val resolvedSparkConf = driverSpec.driverSparkConf.clone().set(sparkConfKey, sparkConfValue) + val modifiedContainer = new ContainerBuilder(driverSpec.driverContainer) + .withName(containerName) + .build() + driverSpec.copy( + driverPod = modifiedPod, + driverSparkConf = resolvedSparkConf, + driverContainer = modifiedContainer) + } +} 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 deleted file mode 100644 index a58a37691f4eb..0000000000000 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/ClientV2Suite.scala +++ /dev/null @@ -1,558 +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 io.fabric8.kubernetes.api.model._ -import io.fabric8.kubernetes.client.{KubernetesClient, Watch} -import io.fabric8.kubernetes.client.dsl.{MixedOperation, NamespaceListVisitFromServerGetDeleteRecreateWaitApplicable, PodResource} -import org.hamcrest.{BaseMatcher, Description} -import org.mockito.{AdditionalAnswers, ArgumentCaptor, Mock, MockitoAnnotations} -import org.mockito.Matchers.{any, anyVararg, argThat, eq => mockitoEq} -import org.mockito.Mockito.{times, verify, when} -import org.mockito.invocation.InvocationOnMock -import org.mockito.stubbing.Answer -import org.scalatest.{BeforeAndAfter, Matchers} - -import scala.collection.JavaConverters._ -import scala.collection.mutable -import org.apache.spark.{SecurityManager, SparkConf, SparkFunSuite} -import org.apache.spark.deploy.kubernetes.{KubernetesExternalShuffleService, KubernetesShuffleBlockHandler, SparkPodInitContainerBootstrap} -import org.apache.spark.deploy.kubernetes.config._ -import org.apache.spark.deploy.kubernetes.constants._ -import org.apache.spark.network.netty.SparkTransportConf -import org.apache.spark.network.shuffle.kubernetes.KubernetesExternalShuffleClient - -class ClientV2Suite extends SparkFunSuite with BeforeAndAfter { - private val JARS_RESOURCE = SubmittedResourceIdAndSecret("jarsId", "jarsSecret") - private val FILES_RESOURCE = SubmittedResourceIdAndSecret("filesId", "filesSecret") - private val SUBMITTED_RESOURCES = SubmittedResources(JARS_RESOURCE, FILES_RESOURCE) - private val BOOTSTRAPPED_POD_ANNOTATION = "bootstrapped" - private val TRUE = "true" - private val APP_NAME = "spark-test" - private val APP_RESOURCE_PREFIX = "spark-prefix" - private val APP_ID = "spark-id" - private val CUSTOM_LABEL_KEY = "customLabel" - private val CUSTOM_LABEL_VALUE = "customLabelValue" - private val DEPRECATED_CUSTOM_LABEL_KEY = "deprecatedCustomLabel" - private val DEPRECATED_CUSTOM_LABEL_VALUE = "deprecatedCustomLabelValue" - private val ALL_EXPECTED_LABELS = Map( - CUSTOM_LABEL_KEY -> CUSTOM_LABEL_VALUE, - DEPRECATED_CUSTOM_LABEL_KEY -> DEPRECATED_CUSTOM_LABEL_VALUE, - SPARK_APP_ID_LABEL -> APP_ID, - SPARK_ROLE_LABEL -> SPARK_POD_DRIVER_ROLE) - private val CUSTOM_ANNOTATION_KEY = "customAnnotation" - private val CUSTOM_ANNOTATION_VALUE = "customAnnotationValue" - private val DEPRECATED_CUSTOM_ANNOTATION_KEY = "deprecatedCustomAnnotation" - private val DEPRECATED_CUSTOM_ANNOTATION_VALUE = "deprecatedCustomAnnotationValue" - private val INIT_CONTAINER_SECRET_NAME = "init-container-secret" - private val INIT_CONTAINER_SECRET_DATA = Map("secret-key" -> "secret-data") - private val MAIN_CLASS = "org.apache.spark.examples.SparkPi" - private val PYSPARK_APP_ARGS = Array(null, "500") - private val APP_ARGS = Array("3", "20") - private val SPARK_JARS = Seq( - "hdfs://localhost:9000/app/jars/jar1.jar", "file:///app/jars/jar2.jar") - private val RESOLVED_SPARK_JARS = Seq( - "hdfs://localhost:9000/app/jars/jar1.jar", "file:///var/data/spark-jars/jar2.jar") - private val RESOLVED_SPARK_REMOTE_AND_LOCAL_JARS = Seq( - "/var/data/spark-jars/jar1.jar", "/var/data/spark-jars/jar2.jar") - private val SPARK_FILES = Seq( - "hdfs://localhost:9000/app/files/file1.txt", "file:///app/files/file2.txt") - private val PYSPARK_FILES = Seq( - "hdfs://localhost:9000/app/files/file1.py", - "file:///app/files/file2.py", - "local:///app/files/file3.py", - "http://app/files/file4.py", - "file:///app/files/file5.py") - private val RESOLVED_PYSPARK_FILES = Seq( - "hdfs://localhost:9000/app/files/file1.py", - "/var/spark-data/spark-files/file2.py", - "local:///app/files/file3.py", - "http://app/files/file4.py") - private val PYSPARK_PRIMARY_FILE = "file:///app/files/file5.py" - private val RESOLVED_PYSPARK_PRIMARY_FILE = "/var/spark-data/spark-file/file5.py" - - private val RESOLVED_SPARK_FILES = Seq( - "hdfs://localhost:9000/app/files/file1.txt", "file:///var/data/spark-files/file2.txt") - private val INIT_CONTAINER_SECRET = new SecretBuilder() - .withNewMetadata() - .withName(INIT_CONTAINER_SECRET_NAME) - .endMetadata() - .withData(INIT_CONTAINER_SECRET_DATA.asJava) - .build() - private val CUSTOM_JAVA_OPTION_KEY = "myappoption" - private val CUSTOM_JAVA_OPTION_VALUE = "myappoptionvalue" - private val DRIVER_JAVA_OPTIONS = s"-D$CUSTOM_JAVA_OPTION_KEY=$CUSTOM_JAVA_OPTION_VALUE" - private val DRIVER_EXTRA_CLASSPATH = "/var/data/spark-app-custom/custom-jar.jar" - private val CONFIG_MAP_NAME = "config-map" - private val CONFIG_MAP_DATA = Map("config-map-key" -> "config-map-data") - private val INIT_CONTAINER_CONFIG_MAP = new ConfigMapBuilder() - .withNewMetadata() - .withName(CONFIG_MAP_NAME) - .endMetadata() - .withData(CONFIG_MAP_DATA.asJava) - .build() - private val CUSTOM_DRIVER_IMAGE = "spark-custom-driver:latest" - private val DRIVER_MEMORY_MB = 512 - private val DRIVER_MEMORY_OVERHEAD_MB = 128 - private val SPARK_CONF = new SparkConf(true) - .set(DRIVER_DOCKER_IMAGE, CUSTOM_DRIVER_IMAGE) - .set(org.apache.spark.internal.config.DRIVER_MEMORY, DRIVER_MEMORY_MB.toLong) - .set(KUBERNETES_DRIVER_MEMORY_OVERHEAD, DRIVER_MEMORY_OVERHEAD_MB.toLong) - .set(KUBERNETES_DRIVER_LABELS, s"$DEPRECATED_CUSTOM_LABEL_KEY=$DEPRECATED_CUSTOM_LABEL_VALUE") - .set(KUBERNETES_DRIVER_ANNOTATIONS, - s"$DEPRECATED_CUSTOM_ANNOTATION_KEY=$DEPRECATED_CUSTOM_ANNOTATION_VALUE") - .set(s"$KUBERNETES_DRIVER_LABEL_PREFIX$CUSTOM_LABEL_KEY", CUSTOM_LABEL_VALUE) - .set(s"$KUBERNETES_DRIVER_ANNOTATION_PREFIX$CUSTOM_ANNOTATION_KEY", CUSTOM_ANNOTATION_VALUE) - .set(org.apache.spark.internal.config.DRIVER_CLASS_PATH, DRIVER_EXTRA_CLASSPATH) - .set(org.apache.spark.internal.config.DRIVER_JAVA_OPTIONS, DRIVER_JAVA_OPTIONS) - private val EXECUTOR_INIT_CONF_KEY = "executor-init-conf" - private val SPARK_CONF_WITH_EXECUTOR_INIT_CONF = SPARK_CONF.clone() - .set(EXECUTOR_INIT_CONF_KEY, TRUE) - private val DRIVER_POD_UID = "driver-pod-uid" - private val DRIVER_POD_KIND = "pod" - private val DRIVER_POD_API_VERSION = "v1" - private val CREDENTIALS_SECRET_NAME = "credentials-secret" - private val CREDENTIALS_SECRET_DATA = Map("credentials-secret-key" -> "credentials-secret-value") - private val CREDENTIALS_SECRET = new SecretBuilder() - .withNewMetadata() - .withName(CREDENTIALS_SECRET_NAME) - .endMetadata() - .withData(CREDENTIALS_SECRET_DATA.asJava) - .build() - private val CREDENTIALS_SET_CONF = "spark.kubernetes.driverCredentials.provided" - private val CREDENTIALS_SET_ANNOTATION = "credentials-set" - - @Mock - private var containerLocalizedFilesResolver: ContainerLocalizedFilesResolver = _ - @Mock - private var executorInitContainerConfiguration: ExecutorInitContainerConfiguration = _ - @Mock - private var submittedDependencyUploader: SubmittedDependencyUploader = _ - @Mock - private var submittedDependenciesSecretBuilder: SubmittedDependencySecretBuilder = _ - @Mock - private var initContainerBootstrap: SparkPodInitContainerBootstrap = _ - @Mock - private var initContainerComponentsProvider: DriverInitContainerComponentsProvider = _ - @Mock - private var kubernetesClient: KubernetesClient = _ - @Mock - private var podOps: MixedOperation[ - Pod, PodList, DoneablePod, PodResource[Pod, DoneablePod]] = _ - private type ResourceListOps = NamespaceListVisitFromServerGetDeleteRecreateWaitApplicable[ - HasMetadata, java.lang.Boolean] - @Mock - private var resourceListOps: ResourceListOps = _ - @Mock - private var credentialsMounterProvider: DriverPodKubernetesCredentialsMounterProvider = _ - @Mock - private var fileMounter: DriverPodKubernetesFileMounter = _ - @Mock - private var credentialsMounter: DriverPodKubernetesCredentialsMounter = _ - @Mock - private var loggingPodStatusWatcher: LoggingPodStatusWatcher = _ - @Mock - private var namedPodResource: PodResource[Pod, DoneablePod] = _ - @Mock - private var watch: Watch = _ - - before { - MockitoAnnotations.initMocks(this) - when(initContainerComponentsProvider.provideInitContainerBootstrap()) - .thenReturn(initContainerBootstrap) - when(submittedDependencyUploader.uploadJars()).thenReturn(JARS_RESOURCE) - when(submittedDependencyUploader.uploadFiles()).thenReturn(FILES_RESOURCE) - when(initContainerBootstrap - .bootstrapInitContainerAndVolumes(mockitoEq(DRIVER_CONTAINER_NAME), any())) - .thenAnswer(new Answer[PodBuilder] { - override def answer(invocationOnMock: InvocationOnMock): PodBuilder = { - invocationOnMock.getArgumentAt(1, classOf[PodBuilder]).editMetadata() - .addToAnnotations(BOOTSTRAPPED_POD_ANNOTATION, TRUE) - .endMetadata() - } - }) - when(initContainerComponentsProvider.provideContainerLocalizedFilesResolver( - any[String])).thenReturn(containerLocalizedFilesResolver) - when(initContainerComponentsProvider.provideDriverPodFileMounter()) - .thenReturn(fileMounter) - when(submittedDependenciesSecretBuilder.build()) - .thenReturn(INIT_CONTAINER_SECRET) - when(kubernetesClient.pods()).thenReturn(podOps) - when(podOps.create(any())).thenAnswer(new Answer[Pod] { - override def answer(invocation: InvocationOnMock): Pod = { - new PodBuilder(invocation.getArgumentAt(0, classOf[Pod])) - .editMetadata() - .withUid(DRIVER_POD_UID) - .endMetadata() - .withKind(DRIVER_POD_KIND) - .withApiVersion(DRIVER_POD_API_VERSION) - .build() - } - }) - when(podOps.withName(s"$APP_RESOURCE_PREFIX-driver")).thenReturn(namedPodResource) - when(fileMounter.addPySparkFiles( - mockitoEq(RESOLVED_PYSPARK_PRIMARY_FILE), - mockitoEq(RESOLVED_PYSPARK_FILES.mkString(",")), - any[String], - any())).thenAnswer( new Answer[PodBuilder] { - override def answer(invocation: InvocationOnMock) : PodBuilder = { - invocation.getArgumentAt(3, classOf[PodBuilder]) - .editMetadata() - .withUid(DRIVER_POD_UID) - .withName(s"$APP_RESOURCE_PREFIX-driver") - .addToLabels("pyspark-test", "true") - .endMetadata() - .withKind(DRIVER_POD_KIND) - .withApiVersion(DRIVER_POD_API_VERSION) - } - }) - when(namedPodResource.watch(loggingPodStatusWatcher)).thenReturn(watch) - when(containerLocalizedFilesResolver.resolveSubmittedAndRemoteSparkJars()) - .thenReturn(RESOLVED_SPARK_REMOTE_AND_LOCAL_JARS) - when(containerLocalizedFilesResolver.resolveSubmittedSparkJars()) - .thenReturn(RESOLVED_SPARK_JARS) - when(containerLocalizedFilesResolver.resolveSubmittedSparkFiles()) - .thenReturn(RESOLVED_SPARK_FILES) - when(containerLocalizedFilesResolver.resolvePrimaryResourceFile()) - .thenReturn(RESOLVED_PYSPARK_PRIMARY_FILE) - when(containerLocalizedFilesResolver.resolveSubmittedPySparkFiles()) - .thenReturn(RESOLVED_PYSPARK_FILES) - when(executorInitContainerConfiguration.configureSparkConfForExecutorInitContainer(SPARK_CONF)) - .thenReturn(SPARK_CONF_WITH_EXECUTOR_INIT_CONF) - when(kubernetesClient.resourceList(anyVararg[HasMetadata]())).thenReturn(resourceListOps) - when(credentialsMounterProvider.getDriverPodKubernetesCredentialsMounter()) - .thenReturn(credentialsMounter) - } - - test("Run with dependency uploader") { - expectationsForNoMountedCredentials() - when(initContainerComponentsProvider - .provideInitContainerSubmittedDependencyUploader(ALL_EXPECTED_LABELS)) - .thenReturn(Some(submittedDependencyUploader)) - when(initContainerComponentsProvider - .provideSubmittedDependenciesSecretBuilder(Some(SUBMITTED_RESOURCES.secrets()))) - .thenReturn(Some(submittedDependenciesSecretBuilder)) - when(initContainerComponentsProvider.provideInitContainerBundle(mockitoEq( - Option(SUBMITTED_RESOURCES.ids())), - mockitoEq(RESOLVED_SPARK_JARS ++ RESOLVED_SPARK_FILES))) - .thenReturn(Option(InitContainerBundle(INIT_CONTAINER_CONFIG_MAP, - initContainerBootstrap, executorInitContainerConfiguration))) - runAndVerifyDriverPodHasCorrectProperties() - val resourceListArgumentCaptor = ArgumentCaptor.forClass(classOf[HasMetadata]) - verify(kubernetesClient).resourceList(resourceListArgumentCaptor.capture()) - val createdResources = resourceListArgumentCaptor.getAllValues.asScala - assert(createdResources.size === 2) - verifyCreatedResourcesHaveOwnerReferences(createdResources) - assert(createdResources.exists { - case secret: Secret => - secret.getMetadata.getName == INIT_CONTAINER_SECRET_NAME && - secret.getData.asScala == INIT_CONTAINER_SECRET_DATA - case _ => false - }) - verifyConfigMapWasCreated(createdResources) - verify(submittedDependencyUploader).uploadJars() - verify(submittedDependencyUploader).uploadFiles() - verify(initContainerComponentsProvider) - .provideSubmittedDependenciesSecretBuilder(Some(SUBMITTED_RESOURCES.secrets())) - } - - test("Run without dependency uploader") { - expectationsForNoMountedCredentials() - expectationsForNoDependencyUploader() - runAndVerifyDriverPodHasCorrectProperties() - val resourceListArgumentCaptor = ArgumentCaptor.forClass(classOf[HasMetadata]) - verify(kubernetesClient).resourceList(resourceListArgumentCaptor.capture()) - val createdResources = resourceListArgumentCaptor.getAllValues.asScala - assert(createdResources.size === 1) - verifyCreatedResourcesHaveOwnerReferences(createdResources) - verifyConfigMapWasCreated(createdResources) - verify(submittedDependencyUploader, times(0)).uploadJars() - verify(submittedDependencyUploader, times(0)).uploadFiles() - verify(initContainerComponentsProvider) - .provideSubmittedDependenciesSecretBuilder(None) - } - - test("Run with mounted credentials") { - expectationsForNoDependencyUploader() - when(credentialsMounter.createCredentialsSecret()).thenReturn(Some(CREDENTIALS_SECRET)) - when(credentialsMounter.mountDriverKubernetesCredentials( - any(), mockitoEq(DRIVER_CONTAINER_NAME), mockitoEq(Some(CREDENTIALS_SECRET)))) - .thenAnswer(new Answer[PodBuilder] { - override def answer(invocation: InvocationOnMock): PodBuilder = { - invocation.getArgumentAt(0, classOf[PodBuilder]).editMetadata() - .addToAnnotations(CREDENTIALS_SET_ANNOTATION, TRUE) - .endMetadata() - } - }) - when(credentialsMounter.setDriverPodKubernetesCredentialLocations(any())) - .thenAnswer(new Answer[SparkConf] { - override def answer(invocation: InvocationOnMock): SparkConf = { - invocation.getArgumentAt(0, classOf[SparkConf]).clone().set(CREDENTIALS_SET_CONF, TRUE) - } - }) - runAndVerifyPodMatchesPredicate { p => - Option(p) - .filter(pod => containerHasCorrectJvmOptions(pod, _(CREDENTIALS_SET_CONF) == TRUE)) - .exists { pod => - pod.getMetadata.getAnnotations.asScala(CREDENTIALS_SET_ANNOTATION) == TRUE - } - } - val resourceListArgumentCaptor = ArgumentCaptor.forClass(classOf[HasMetadata]) - verify(kubernetesClient).resourceList(resourceListArgumentCaptor.capture()) - val createdResources = resourceListArgumentCaptor.getAllValues.asScala - assert(createdResources.size === 2) - verifyCreatedResourcesHaveOwnerReferences(createdResources) - assert(createdResources.exists { - case secret: Secret => - secret.getMetadata.getName == CREDENTIALS_SECRET_NAME && - secret.getData.asScala == CREDENTIALS_SECRET_DATA - case _ => false - }) - } - - test("Waiting for completion should await completion on the status watcher.") { - expectationsForNoMountedCredentials() - expectationsForNoDependencyUploader() - new Client( - APP_NAME, - APP_RESOURCE_PREFIX, - APP_ID, - "", - None, - MAIN_CLASS, - SPARK_CONF, - APP_ARGS, - true, - kubernetesClient, - initContainerComponentsProvider, - credentialsMounterProvider, - loggingPodStatusWatcher).run() - verify(loggingPodStatusWatcher).awaitCompletion() - } - - test("Mounting environmental variables correctly onto Driver Pod for PySpark Jobs") { - expectationsForNoMountedCredentials() - expectationsForNoDependencyUploader() - expectationsForNoSparkJarsOrFiles() - runAndVerifyDriverPodHasCorrectPySparkProperties() - } - - private def expectationsForNoSparkJarsOrFiles(): Unit = { - when(containerLocalizedFilesResolver.resolveSubmittedSparkFiles()) - .thenReturn(Nil) - when(containerLocalizedFilesResolver.resolveSubmittedSparkJars()) - .thenReturn(Nil) - } - - private def expectationsForNoDependencyUploader(): Unit = { - when(initContainerComponentsProvider - .provideInitContainerSubmittedDependencyUploader(ALL_EXPECTED_LABELS)) - .thenReturn(None) - when(initContainerComponentsProvider - .provideSubmittedDependenciesSecretBuilder(None)) - .thenReturn(None) - when(initContainerComponentsProvider.provideInitContainerBundle(mockitoEq(None), - mockitoEq(RESOLVED_SPARK_JARS ++ RESOLVED_SPARK_FILES))) - .thenReturn(Some(InitContainerBundle(INIT_CONTAINER_CONFIG_MAP, - initContainerBootstrap, executorInitContainerConfiguration))) - } - - private def expectationsForNoMountedCredentials(): Unit = { - when(credentialsMounter.setDriverPodKubernetesCredentialLocations(any())) - .thenAnswer(AdditionalAnswers.returnsFirstArg()) - when(credentialsMounter.createCredentialsSecret()).thenReturn(None) - when(credentialsMounter.mountDriverKubernetesCredentials( - any(), mockitoEq(DRIVER_CONTAINER_NAME), mockitoEq(None))) - .thenAnswer(AdditionalAnswers.returnsFirstArg()) - } - - private def verifyCreatedResourcesHaveOwnerReferences( - createdResources: mutable.Buffer[HasMetadata]): Unit = { - assert(createdResources.forall { resource => - val owners = resource.getMetadata.getOwnerReferences.asScala - owners.size === 1 && - owners.head.getController && - owners.head.getKind == DRIVER_POD_KIND && - owners.head.getUid == DRIVER_POD_UID && - owners.head.getName == s"$APP_RESOURCE_PREFIX-driver" && - owners.head.getApiVersion == DRIVER_POD_API_VERSION - }) - } - - private def verifyConfigMapWasCreated(createdResources: mutable.Buffer[HasMetadata]): Unit = { - assert(createdResources.exists { - case configMap: ConfigMap => - configMap.getMetadata.getName == CONFIG_MAP_NAME && - configMap.getData.asScala == CONFIG_MAP_DATA - case _ => false - }) - } - - private def runAndVerifyDriverPodHasCorrectProperties(): Unit = { - val expectedOptions = SPARK_CONF.getAll - .filterNot(_._1 == org.apache.spark.internal.config.DRIVER_JAVA_OPTIONS.key) - .toMap ++ - Map( - "spark.app.id" -> APP_ID, - KUBERNETES_DRIVER_POD_NAME.key -> s"$APP_RESOURCE_PREFIX-driver", - KUBERNETES_EXECUTOR_POD_NAME_PREFIX.key -> APP_RESOURCE_PREFIX, - EXECUTOR_INIT_CONF_KEY -> TRUE, - CUSTOM_JAVA_OPTION_KEY -> CUSTOM_JAVA_OPTION_VALUE, - "spark.jars" -> RESOLVED_SPARK_JARS.mkString(","), - "spark.files" -> RESOLVED_SPARK_FILES.mkString(",")) - runAndVerifyPodMatchesPredicate { p => - Option(p) - .filter(_.getMetadata.getName == s"$APP_RESOURCE_PREFIX-driver") - .filter(podHasCorrectAnnotations) - .filter(_.getMetadata.getLabels.asScala == ALL_EXPECTED_LABELS) - .filter(containerHasCorrectBasicContainerConfiguration) - .filter(containerHasCorrectBasicEnvs) - .filter(containerHasCorrectMountedClasspath) - .exists(pod => containerHasCorrectJvmOptions(pod, _ == expectedOptions)) - } - } - - private def runAndVerifyDriverPodHasCorrectPySparkProperties(): Unit = { - when(initContainerComponentsProvider.provideContainerLocalizedFilesResolver( - mockitoEq(PYSPARK_PRIMARY_FILE))).thenReturn(containerLocalizedFilesResolver) - when(initContainerComponentsProvider.provideInitContainerBundle( - any[Option[SubmittedResourceIds]], any[Iterable[String]])) - .thenReturn(Some(InitContainerBundle(INIT_CONTAINER_CONFIG_MAP, - initContainerBootstrap, executorInitContainerConfiguration))) - runAndVerifyPySparkPodMatchesPredicate { p => - Option(p).exists(pod => containerHasCorrectPySparkEnvs(pod)) - } - } - - private def runAndVerifyPodMatchesPredicate(pred: (Pod => Boolean)): Unit = { - new Client( - APP_NAME, - APP_RESOURCE_PREFIX, - APP_ID, - "", - None, - MAIN_CLASS, - SPARK_CONF, - APP_ARGS, - false, - kubernetesClient, - initContainerComponentsProvider, - credentialsMounterProvider, - loggingPodStatusWatcher).run() - val podMatcher = new BaseMatcher[Pod] { - override def matches(o: scala.Any): Boolean = { - o match { - case p: Pod => pred(p) - case _ => false - } - } - override def describeTo(description: Description): Unit = {} - } - verify(podOps).create(argThat(podMatcher)) - } - - private def containerHasCorrectJvmOptions( - pod: Pod, optionsCorrectnessPredicate: (Map[String, String] => Boolean)): Boolean = { - val driverContainer = pod.getSpec.getContainers.asScala.head - val envs = driverContainer.getEnv.asScala.map(env => (env.getName, env.getValue)) - envs.toMap.get(ENV_DRIVER_JAVA_OPTS).exists { javaOptions => - val splitOptions = javaOptions.split(" ") - splitOptions.forall(_.startsWith("-D")) && - optionsCorrectnessPredicate(splitOptions.map { option => - val withoutPrefix = option.substring(2) - (withoutPrefix.split("=", 2)(0), withoutPrefix.split("=", 2)(1)) - }.toMap) - } - } - - private def containerHasCorrectMountedClasspath(pod: Pod): Boolean = { - val driverContainer = pod.getSpec.getContainers.asScala.head - val envs = driverContainer.getEnv.asScala.map(env => (env.getName, env.getValue)) - envs.toMap.get(ENV_MOUNTED_CLASSPATH).exists { classpath => - val mountedClasspathEntities = classpath.split(File.pathSeparator) - mountedClasspathEntities.toSet == RESOLVED_SPARK_REMOTE_AND_LOCAL_JARS.toSet - } - } - - private def containerHasCorrectBasicEnvs(pod: Pod): Boolean = { - val driverContainer = pod.getSpec.getContainers.asScala.head - val envs = driverContainer.getEnv.asScala.map(env => (env.getName, env.getValue)) - val expectedBasicEnvs = Map( - ENV_SUBMIT_EXTRA_CLASSPATH -> DRIVER_EXTRA_CLASSPATH, - ENV_DRIVER_MEMORY -> s"${DRIVER_MEMORY_MB + DRIVER_MEMORY_OVERHEAD_MB}m", - ENV_DRIVER_MAIN_CLASS -> MAIN_CLASS, - ENV_DRIVER_ARGS -> APP_ARGS.mkString(" ")) - expectedBasicEnvs.toSet.subsetOf(envs.toSet) - } - - private def containerHasCorrectPySparkEnvs(pod: Pod): Boolean = { - val driverPodLabels = - pod.getMetadata.getLabels.asScala.map(env => (env._1.toString, env._2.toString)) - val expectedBasicLabels = Map( - "pyspark-test" -> "true", - "spark-role" -> "driver") - expectedBasicLabels.toSet.subsetOf(driverPodLabels.toSet) - } - - private def containerHasCorrectBasicContainerConfiguration(pod: Pod): Boolean = { - val containers = pod.getSpec.getContainers.asScala - containers.size == 1 && - containers.head.getName == DRIVER_CONTAINER_NAME && - containers.head.getImage == CUSTOM_DRIVER_IMAGE && - containers.head.getImagePullPolicy == "IfNotPresent" - } - - private def podHasCorrectAnnotations(pod: Pod): Boolean = { - val expectedAnnotations = Map( - DEPRECATED_CUSTOM_ANNOTATION_KEY -> DEPRECATED_CUSTOM_ANNOTATION_VALUE, - CUSTOM_ANNOTATION_KEY -> CUSTOM_ANNOTATION_VALUE, - SPARK_APP_NAME_ANNOTATION -> APP_NAME, - BOOTSTRAPPED_POD_ANNOTATION -> TRUE) - pod.getMetadata.getAnnotations.asScala == expectedAnnotations - } - - private def runAndVerifyPySparkPodMatchesPredicate(pred: (Pod => Boolean)): Unit = { - new Client( - APP_NAME, - APP_RESOURCE_PREFIX, - APP_ID, - PYSPARK_PRIMARY_FILE, - Option(new PythonSubmissionResourcesImpl(PYSPARK_PRIMARY_FILE, PYSPARK_APP_ARGS)), - MAIN_CLASS, - SPARK_CONF, - PYSPARK_APP_ARGS, - false, - kubernetesClient, - initContainerComponentsProvider, - credentialsMounterProvider, - loggingPodStatusWatcher).run() - val podMatcher = new BaseMatcher[Pod] { - override def matches(o: scala.Any): Boolean = { - o match { - case p: Pod => pred(p) - case _ => false - } - } - override def describeTo(description: Description): Unit = {} - } - verify(podOps).create(argThat(podMatcher)) - } -} - - diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/ContainerLocalizedFilesResolverSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/ContainerLocalizedFilesResolverSuite.scala 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/DriverConfigurationStepsOrchestratorSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/DriverConfigurationStepsOrchestratorSuite.scala new file mode 100644 index 0000000000000..e4f221ad99cc5 --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/DriverConfigurationStepsOrchestratorSuite.scala @@ -0,0 +1,95 @@ +/* + * 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.submit.submitsteps.{BaseDriverConfigurationStep, DependencyResolutionStep, DriverKubernetesCredentialsStep, InitContainerBootstrapStep, PythonStep} + +private[spark] class DriverConfigurationStepsOrchestratorSuite extends SparkFunSuite { + + private val NAMESPACE = "default" + private val APP_ID = "spark-app-id" + private val LAUNCH_TIME = 975256L + private val APP_NAME = "spark" + private val MAIN_CLASS = "org.apache.spark.examples.SparkPi" + private val APP_ARGS = Array("arg1", "arg2") + private val ADDITIONAL_PYTHON_FILES = Seq("local:///var/apps/python/py1.py") + + test("Base submission steps without an init-container or python files.") { + val sparkConf = new SparkConf(false) + .set("spark.jars", "local:///var/apps/jars/jar1.jar") + val mainAppResource = JavaMainAppResource("local:///var/apps/jars/main.jar") + val orchestrator = new DriverConfigurationStepsOrchestrator( + NAMESPACE, + APP_ID, + LAUNCH_TIME, + mainAppResource, + APP_NAME, + MAIN_CLASS, + APP_ARGS, + ADDITIONAL_PYTHON_FILES, + sparkConf) + val steps = orchestrator.getAllConfigurationSteps() + assert(steps.size === 3) + assert(steps(0).isInstanceOf[BaseDriverConfigurationStep]) + assert(steps(1).isInstanceOf[DriverKubernetesCredentialsStep]) + assert(steps(2).isInstanceOf[DependencyResolutionStep]) + } + + test("Submission steps with an init-container.") { + val sparkConf = new SparkConf(false) + .set("spark.jars", "hdfs://localhost:9000/var/apps/jars/jar1.jar") + val mainAppResource = JavaMainAppResource("local:///var/apps/jars/main.jar") + val orchestrator = new DriverConfigurationStepsOrchestrator( + NAMESPACE, + APP_ID, + LAUNCH_TIME, + mainAppResource, + APP_NAME, + MAIN_CLASS, + APP_ARGS, + ADDITIONAL_PYTHON_FILES, + sparkConf) + val steps = orchestrator.getAllConfigurationSteps() + assert(steps.size === 4) + assert(steps(0).isInstanceOf[BaseDriverConfigurationStep]) + assert(steps(1).isInstanceOf[DriverKubernetesCredentialsStep]) + assert(steps(2).isInstanceOf[DependencyResolutionStep]) + assert(steps(3).isInstanceOf[InitContainerBootstrapStep]) + } + + test("Submission steps with python files.") { + val sparkConf = new SparkConf(false) + val mainAppResource = PythonMainAppResource("local:///var/apps/python/main.py") + val orchestrator = new DriverConfigurationStepsOrchestrator( + NAMESPACE, + APP_ID, + LAUNCH_TIME, + mainAppResource, + APP_NAME, + MAIN_CLASS, + APP_ARGS, + ADDITIONAL_PYTHON_FILES, + sparkConf) + val steps = orchestrator.getAllConfigurationSteps() + assert(steps.size === 4) + assert(steps(0).isInstanceOf[BaseDriverConfigurationStep]) + assert(steps(1).isInstanceOf[DriverKubernetesCredentialsStep]) + assert(steps(2).isInstanceOf[DependencyResolutionStep]) + assert(steps(3).isInstanceOf[PythonStep]) + } +} 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 9b60b7ef2b786..0000000000000 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/PythonSubmissionResourcesSuite.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 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/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/BaseDriverConfigurationStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/BaseDriverConfigurationStepSuite.scala new file mode 100644 index 0000000000000..c7d80a16a1532 --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/BaseDriverConfigurationStepSuite.scala @@ -0,0 +1,106 @@ +/* + * 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, PodBuilder} +import scala.collection.JavaConverters._ + +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.deploy.kubernetes.config._ +import org.apache.spark.deploy.kubernetes.constants._ + +private[spark] class BaseDriverConfigurationStepSuite extends SparkFunSuite { + + private val APP_ID = "spark-app-id" + private val RESOURCE_NAME_PREFIX = "spark" + private val DRIVER_LABELS = Map("labelkey" -> "labelvalue") + private val DOCKER_IMAGE_PULL_POLICY = "IfNotPresent" + private val APP_NAME = "spark-test" + private val MAIN_CLASS = "org.apache.spark.examples.SparkPi" + private val APP_ARGS = Array("arg1", "arg2") + private val CUSTOM_ANNOTATION_KEY = "customAnnotation" + private val CUSTOM_ANNOTATION_VALUE = "customAnnotationValue" + private val DEPRECATED_CUSTOM_ANNOTATION_KEY = "customAnnotationDeprecated" + private val DEPRECATED_CUSTOM_ANNOTATION_VALUE = "customAnnotationDeprecatedValue" + + test("Set all possible configurations from the user.") { + val sparkConf = new SparkConf() + .set(KUBERNETES_DRIVER_POD_NAME, "spark-driver-pod") + .set(org.apache.spark.internal.config.DRIVER_CLASS_PATH, "/opt/spark/spark-exmaples.jar") + .set("spark.driver.cores", "2") + .set(KUBERNETES_DRIVER_LIMIT_CORES, "4") + .set(org.apache.spark.internal.config.DRIVER_MEMORY, 256L) + .set(KUBERNETES_DRIVER_MEMORY_OVERHEAD, 200L) + .set(DRIVER_DOCKER_IMAGE, "spark-driver:latest") + .set(s"spark.kubernetes.driver.annotation.$CUSTOM_ANNOTATION_KEY", CUSTOM_ANNOTATION_VALUE) + .set("spark.kubernetes.driver.annotations", + s"$DEPRECATED_CUSTOM_ANNOTATION_KEY=$DEPRECATED_CUSTOM_ANNOTATION_VALUE") + val submissionStep = new BaseDriverConfigurationStep( + APP_ID, + RESOURCE_NAME_PREFIX, + DRIVER_LABELS, + DOCKER_IMAGE_PULL_POLICY, + APP_NAME, + MAIN_CLASS, + APP_ARGS, + sparkConf) + val basePod = new PodBuilder().withNewMetadata().endMetadata().withNewSpec().endSpec().build() + val baseDriverSpec = KubernetesDriverSpec( + driverPod = basePod, + driverContainer = new ContainerBuilder().build(), + driverSparkConf = new SparkConf(false), + otherKubernetesResources = Seq.empty[HasMetadata]) + + val preparedDriverSpec = submissionStep.configureDriver(baseDriverSpec) + assert(preparedDriverSpec.driverContainer.getName === DRIVER_CONTAINER_NAME) + assert(preparedDriverSpec.driverContainer.getImage === "spark-driver:latest") + assert(preparedDriverSpec.driverContainer.getImagePullPolicy === DOCKER_IMAGE_PULL_POLICY) + val envs = preparedDriverSpec.driverContainer + .getEnv + .asScala + .map(env => (env.getName, env.getValue)) + .toMap + assert(envs.size === 4) + assert(envs(ENV_SUBMIT_EXTRA_CLASSPATH) === "/opt/spark/spark-exmaples.jar") + assert(envs(ENV_DRIVER_MEMORY) === "456m") + assert(envs(ENV_DRIVER_MAIN_CLASS) === MAIN_CLASS) + assert(envs(ENV_DRIVER_ARGS) === "arg1 arg2") + val resourceRequirements = preparedDriverSpec.driverContainer.getResources + val requests = resourceRequirements.getRequests.asScala + assert(requests("cpu").getAmount === "2") + assert(requests("memory").getAmount === "256M") + val limits = resourceRequirements.getLimits.asScala + assert(limits("memory").getAmount === "456M") + assert(limits("cpu").getAmount === "4") + val driverPodMetadata = preparedDriverSpec.driverPod.getMetadata + assert(driverPodMetadata.getName === "spark-driver-pod") + assert(driverPodMetadata.getLabels.asScala === DRIVER_LABELS) + val expectedAnnotations = Map( + CUSTOM_ANNOTATION_KEY -> CUSTOM_ANNOTATION_VALUE, + DEPRECATED_CUSTOM_ANNOTATION_KEY -> DEPRECATED_CUSTOM_ANNOTATION_VALUE, + SPARK_APP_NAME_ANNOTATION -> APP_NAME) + assert(driverPodMetadata.getAnnotations.asScala === expectedAnnotations) + assert(preparedDriverSpec.driverPod.getSpec.getRestartPolicy === "Never") + val resolvedSparkConf = preparedDriverSpec.driverSparkConf.getAll.toMap + val expectedSparkConf = Map( + KUBERNETES_DRIVER_POD_NAME.key -> "spark-driver-pod", + "spark.app.id" -> APP_ID, + KUBERNETES_EXECUTOR_POD_NAME_PREFIX.key -> RESOURCE_NAME_PREFIX) + assert(resolvedSparkConf === expectedSparkConf) + + } +} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DependencyResolutionStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DependencyResolutionStepSuite.scala new file mode 100644 index 0000000000000..3f7ec61074b0c --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DependencyResolutionStepSuite.scala @@ -0,0 +1,79 @@ +/* + * 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, HasMetadata, PodBuilder} +import scala.collection.JavaConverters._ + +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.deploy.kubernetes.constants._ + +private[spark] class DependencyResolutionStepSuite extends SparkFunSuite { + + private val SPARK_JARS = Seq( + "hdfs://localhost:9000/apps/jars/jar1.jar", + "file:///home/user/apps/jars/jar2.jar", + "local:///var/apps/jars/jar3.jar") + + private val SPARK_FILES = Seq( + "file:///home/user/apps/files/file1.txt", + "hdfs://localhost:9000/apps/files/file2.txt", + "local:///var/apps/files/file3.txt") + + private val JARS_DOWNLOAD_PATH = "/mnt/spark-data/jars" + private val FILES_DOWNLOAD_PATH = "/mnt/spark-data/files" + + test("Added dependencies should be resolved in Spark configuration and environment") { + val dependencyResolutionStep = new DependencyResolutionStep( + SPARK_JARS, + SPARK_FILES, + JARS_DOWNLOAD_PATH, + FILES_DOWNLOAD_PATH) + val driverPod = new PodBuilder().build() + val baseDriverSpec = KubernetesDriverSpec( + driverPod = driverPod, + driverContainer = new ContainerBuilder().build(), + driverSparkConf = new SparkConf(false), + otherKubernetesResources = Seq.empty[HasMetadata]) + val preparedDriverSpec = dependencyResolutionStep.configureDriver(baseDriverSpec) + assert(preparedDriverSpec.driverPod === driverPod) + assert(preparedDriverSpec.otherKubernetesResources.isEmpty) + val resolvedSparkJars = preparedDriverSpec.driverSparkConf.get("spark.jars").split(",").toSet + val expectedResolvedSparkJars = Set( + "hdfs://localhost:9000/apps/jars/jar1.jar", + s"$JARS_DOWNLOAD_PATH/jar2.jar", + "/var/apps/jars/jar3.jar") + assert(resolvedSparkJars === expectedResolvedSparkJars) + val resolvedSparkFiles = preparedDriverSpec.driverSparkConf.get("spark.files").split(",").toSet + val expectedResolvedSparkFiles = Set( + s"$FILES_DOWNLOAD_PATH/file1.txt", + s"hdfs://localhost:9000/apps/files/file2.txt", + s"/var/apps/files/file3.txt") + assert(resolvedSparkFiles === expectedResolvedSparkFiles) + val driverEnv = preparedDriverSpec.driverContainer.getEnv.asScala + assert(driverEnv.size === 1) + assert(driverEnv.head.getName === ENV_MOUNTED_CLASSPATH) + val resolvedDriverClasspath = driverEnv.head.getValue.split(File.pathSeparator).toSet + val expectedResolvedDriverClasspath = Set( + s"$JARS_DOWNLOAD_PATH/jar1.jar", + s"$JARS_DOWNLOAD_PATH/jar2.jar", + "/var/apps/jars/jar3.jar") + assert(resolvedDriverClasspath === expectedResolvedDriverClasspath) + } +} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DriverKubernetesCredentialsStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DriverKubernetesCredentialsStepSuite.scala new file mode 100644 index 0000000000000..3d5664713a2b8 --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DriverKubernetesCredentialsStepSuite.scala @@ -0,0 +1,152 @@ +/* + * 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 com.google.common.base.Charsets +import com.google.common.io.{BaseEncoding, Files} +import io.fabric8.kubernetes.api.model.{ContainerBuilder, HasMetadata, PodBuilder, Secret} +import org.scalatest.BeforeAndAfter +import scala.collection.JavaConverters._ + +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.deploy.kubernetes.config._ +import org.apache.spark.deploy.kubernetes.constants._ +import org.apache.spark.util.Utils + +private[spark] class DriverKubernetesCredentialsStepSuite + extends SparkFunSuite with BeforeAndAfter { + + private val KUBERNETES_RESOURCE_NAME_PREFIX = "spark" + private var credentialsTempDirectory: File = _ + private val BASE_DRIVER_SPEC = new KubernetesDriverSpec( + driverPod = new PodBuilder().build(), + driverContainer = new ContainerBuilder().build(), + driverSparkConf = new SparkConf(false), + otherKubernetesResources = Seq.empty[HasMetadata]) + + before { + credentialsTempDirectory = Utils.createTempDir() + } + + after { + credentialsTempDirectory.delete() + } + + test("Don't set any credentials") { + val kubernetesCredentialsStep = new DriverKubernetesCredentialsStep( + new SparkConf(false), KUBERNETES_RESOURCE_NAME_PREFIX) + val preparedDriverSpec = kubernetesCredentialsStep.configureDriver(BASE_DRIVER_SPEC) + assert(preparedDriverSpec.driverPod === BASE_DRIVER_SPEC.driverPod) + assert(preparedDriverSpec.driverContainer === BASE_DRIVER_SPEC.driverContainer) + assert(preparedDriverSpec.otherKubernetesResources.isEmpty) + assert(preparedDriverSpec.driverSparkConf.getAll.isEmpty) + } + + test("Only set credentials that are manually mounted.") { + val submissionSparkConf = new SparkConf(false) + .set( + s"$APISERVER_AUTH_DRIVER_MOUNTED_CONF_PREFIX.$OAUTH_TOKEN_FILE_CONF_SUFFIX", + "/mnt/secrets/my-token.txt") + .set( + s"$APISERVER_AUTH_DRIVER_MOUNTED_CONF_PREFIX.$CLIENT_KEY_FILE_CONF_SUFFIX", + "/mnt/secrets/my-key.pem") + .set( + s"$APISERVER_AUTH_DRIVER_MOUNTED_CONF_PREFIX.$CLIENT_CERT_FILE_CONF_SUFFIX", + "/mnt/secrets/my-cert.pem") + .set( + s"$APISERVER_AUTH_DRIVER_MOUNTED_CONF_PREFIX.$CA_CERT_FILE_CONF_SUFFIX", + "/mnt/secrets/my-ca.pem") + + val kubernetesCredentialsStep = new DriverKubernetesCredentialsStep( + submissionSparkConf, KUBERNETES_RESOURCE_NAME_PREFIX) + val preparedDriverSpec = kubernetesCredentialsStep.configureDriver(BASE_DRIVER_SPEC) + assert(preparedDriverSpec.driverPod === BASE_DRIVER_SPEC.driverPod) + assert(preparedDriverSpec.driverContainer === BASE_DRIVER_SPEC.driverContainer) + assert(preparedDriverSpec.otherKubernetesResources.isEmpty) + assert(preparedDriverSpec.driverSparkConf.getAll.toMap === submissionSparkConf.getAll.toMap) + } + + test("Mount credentials from the submission client as a secret.") { + val caCertFile = writeCredentials("ca.pem", "ca-cert") + val clientKeyFile = writeCredentials("key.pem", "key") + val clientCertFile = writeCredentials("cert.pem", "cert") + val submissionSparkConf = new SparkConf(false) + .set( + s"$APISERVER_AUTH_DRIVER_CONF_PREFIX.$OAUTH_TOKEN_CONF_SUFFIX", + "token") + .set( + s"$APISERVER_AUTH_DRIVER_CONF_PREFIX.$CLIENT_KEY_FILE_CONF_SUFFIX", + clientKeyFile.getAbsolutePath) + .set( + s"$APISERVER_AUTH_DRIVER_CONF_PREFIX.$CLIENT_CERT_FILE_CONF_SUFFIX", + clientCertFile.getAbsolutePath) + .set( + s"$APISERVER_AUTH_DRIVER_CONF_PREFIX.$CA_CERT_FILE_CONF_SUFFIX", + caCertFile.getAbsolutePath) + val kubernetesCredentialsStep = new DriverKubernetesCredentialsStep( + submissionSparkConf, KUBERNETES_RESOURCE_NAME_PREFIX) + val preparedDriverSpec = kubernetesCredentialsStep.configureDriver( + BASE_DRIVER_SPEC.copy(driverSparkConf = submissionSparkConf)) + val expectedSparkConf = Map( + s"$APISERVER_AUTH_DRIVER_CONF_PREFIX.$OAUTH_TOKEN_CONF_SUFFIX" -> "", + s"$APISERVER_AUTH_DRIVER_MOUNTED_CONF_PREFIX.$OAUTH_TOKEN_FILE_CONF_SUFFIX" -> + DRIVER_CREDENTIALS_OAUTH_TOKEN_PATH, + s"$APISERVER_AUTH_DRIVER_MOUNTED_CONF_PREFIX.$CLIENT_KEY_FILE_CONF_SUFFIX" -> + DRIVER_CREDENTIALS_CLIENT_KEY_PATH, + s"$APISERVER_AUTH_DRIVER_MOUNTED_CONF_PREFIX.$CLIENT_CERT_FILE_CONF_SUFFIX" -> + DRIVER_CREDENTIALS_CLIENT_CERT_PATH, + s"$APISERVER_AUTH_DRIVER_MOUNTED_CONF_PREFIX.$CA_CERT_FILE_CONF_SUFFIX" -> + DRIVER_CREDENTIALS_CA_CERT_PATH, + s"$APISERVER_AUTH_DRIVER_CONF_PREFIX.$CLIENT_KEY_FILE_CONF_SUFFIX" -> + clientKeyFile.getAbsolutePath, + s"$APISERVER_AUTH_DRIVER_CONF_PREFIX.$CLIENT_CERT_FILE_CONF_SUFFIX" -> + clientCertFile.getAbsolutePath, + s"$APISERVER_AUTH_DRIVER_CONF_PREFIX.$CA_CERT_FILE_CONF_SUFFIX" -> + caCertFile.getAbsolutePath) + assert(preparedDriverSpec.driverSparkConf.getAll.toMap === expectedSparkConf) + assert(preparedDriverSpec.otherKubernetesResources.size === 1) + val credentialsSecret = preparedDriverSpec.otherKubernetesResources.head.asInstanceOf[Secret] + assert(credentialsSecret.getMetadata.getName === + s"$KUBERNETES_RESOURCE_NAME_PREFIX-kubernetes-credentials") + val decodedSecretData = credentialsSecret.getData.asScala.map { data => + (data._1, new String(BaseEncoding.base64().decode(data._2), Charsets.UTF_8)) + } + val expectedSecretData = Map( + DRIVER_CREDENTIALS_CA_CERT_SECRET_NAME -> "ca-cert", + DRIVER_CREDENTIALS_OAUTH_TOKEN_SECRET_NAME -> "token", + DRIVER_CREDENTIALS_CLIENT_KEY_SECRET_NAME -> "key", + DRIVER_CREDENTIALS_CLIENT_CERT_SECRET_NAME -> "cert") + assert(decodedSecretData === expectedSecretData) + val driverPodVolumes = preparedDriverSpec.driverPod.getSpec.getVolumes.asScala + assert(driverPodVolumes.size === 1) + assert(driverPodVolumes.head.getName === DRIVER_CREDENTIALS_SECRET_VOLUME_NAME) + assert(driverPodVolumes.head.getSecret != null) + assert(driverPodVolumes.head.getSecret.getSecretName === credentialsSecret.getMetadata.getName) + val driverContainerVolumeMount = preparedDriverSpec.driverContainer.getVolumeMounts.asScala + assert(driverContainerVolumeMount.size === 1) + assert(driverContainerVolumeMount.head.getName === DRIVER_CREDENTIALS_SECRET_VOLUME_NAME) + assert(driverContainerVolumeMount.head.getMountPath === DRIVER_CREDENTIALS_SECRETS_BASE_DIR) + } + + private def writeCredentials(credentialsFileName: String, credentialsContents: String): File = { + val credentialsFile = new File(credentialsTempDirectory, credentialsFileName) + Files.write(credentialsContents, credentialsFile, Charsets.UTF_8) + credentialsFile + } +} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/PythonStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/PythonStepSuite.scala new file mode 100644 index 0000000000000..ce0dcee6acc46 --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/PythonStepSuite.scala @@ -0,0 +1,76 @@ +/* + * 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._ +import org.scalatest.BeforeAndAfter +import scala.collection.JavaConverters._ + +import org.apache.spark.{SparkConf, SparkFunSuite} + +class PythonStepSuite extends SparkFunSuite with BeforeAndAfter { + private val FILE_DOWNLOAD_PATH = "/var/data/spark-files" + private val PYSPARK_FILES = Seq( + "hdfs://localhost:9000/app/files/file1.py", + "file:///app/files/file2.py", + "local:///app/files/file3.py", + "http://app/files/file4.py") + private val RESOLVED_PYSPARK_FILES = Seq( + FILE_DOWNLOAD_PATH + "/file1.py", + FILE_DOWNLOAD_PATH + "/file2.py", + "/app/files/file3.py", + FILE_DOWNLOAD_PATH + "/file4.py").mkString(",") + private val PYSPARK_PRIMARY_FILE = "file:///app/files/file5.py" + private val RESOLVED_PYSPARK_PRIMARY_FILE = FILE_DOWNLOAD_PATH + "/file5.py" + + test("testing PySpark with --py-files both local and remote files") { + val pyStep = new PythonStep( + PYSPARK_PRIMARY_FILE, + PYSPARK_FILES, + FILE_DOWNLOAD_PATH) + val returnedDriverContainer = pyStep.configureDriver( + KubernetesDriverSpec( + new Pod(), + new Container(), + Seq.empty[HasMetadata], + new SparkConf)) + assert(returnedDriverContainer.driverContainer.getEnv + .asScala.map(env => (env.getName, env.getValue)).toMap === + Map( + "PYSPARK_PRIMARY" -> RESOLVED_PYSPARK_PRIMARY_FILE, + "PYSPARK_FILES" -> RESOLVED_PYSPARK_FILES)) + } + + test("testing PySpark with empty --py-files ") { + val pyStep = new PythonStep( + PYSPARK_PRIMARY_FILE, + Seq.empty[String], + FILE_DOWNLOAD_PATH) + val returnedDriverContainer = pyStep.configureDriver( + KubernetesDriverSpec( + new Pod(), + new Container(), + Seq.empty[HasMetadata], + new SparkConf)) + assert(returnedDriverContainer.driverContainer.getEnv + .asScala.map(env => (env.getName, env.getValue)).toMap === + Map( + "PYSPARK_PRIMARY" -> RESOLVED_PYSPARK_PRIMARY_FILE, + "PYSPARK_FILES" -> "null")) + } + +} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initContainerBootstrapStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initContainerBootstrapStepSuite.scala new file mode 100644 index 0000000000000..b11b487111496 --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initContainerBootstrapStepSuite.scala @@ -0,0 +1,159 @@ +/* + * 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.StringReader +import java.util.Properties + +import com.fasterxml.jackson.databind.ObjectMapper +import com.fasterxml.jackson.module.scala.DefaultScalaModule +import com.google.common.collect.Maps +import io.fabric8.kubernetes.api.model.{ConfigMap, Container, ContainerBuilder, HasMetadata, PodBuilder, SecretBuilder} +import scala.collection.JavaConverters._ + +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.deploy.kubernetes.config._ +import org.apache.spark.deploy.kubernetes.constants._ +import org.apache.spark.deploy.kubernetes.submit.submitsteps.initcontainer.{InitContainerConfigurationStep, InitContainerSpec} +import org.apache.spark.util.Utils + +private[spark] class initContainerBootstrapStepSuite extends SparkFunSuite { + + private val OBJECT_MAPPER = new ObjectMapper().registerModule(DefaultScalaModule) + private val CONFIG_MAP_NAME = "spark-init-config-map" + private val CONFIG_MAP_KEY = "spark-init-config-map-key" + + test("The init container bootstrap step should use all of the init container steps") { + val baseDriverSpec = KubernetesDriverSpec( + driverPod = new PodBuilder().build(), + driverContainer = new ContainerBuilder().build(), + driverSparkConf = new SparkConf(false), + otherKubernetesResources = Seq.empty[HasMetadata]) + val initContainerSteps = Seq( + FirstTestInitContainerConfigurationStep$, + SecondTestInitContainerConfigurationStep$) + val bootstrapStep = new InitContainerBootstrapStep( + initContainerSteps, + CONFIG_MAP_NAME, + CONFIG_MAP_KEY) + val preparedDriverSpec = bootstrapStep.configureDriver(baseDriverSpec) + assert(preparedDriverSpec.driverPod.getMetadata.getLabels.asScala === + FirstTestInitContainerConfigurationStep$.additionalLabels) + val additionalDriverEnv = preparedDriverSpec.driverContainer.getEnv.asScala + assert(additionalDriverEnv.size === 1) + assert(additionalDriverEnv.head.getName === + FirstTestInitContainerConfigurationStep$.additionalMainContainerEnvKey) + assert(additionalDriverEnv.head.getValue === + FirstTestInitContainerConfigurationStep$.additionalMainContainerEnvValue) + val driverAnnotations = preparedDriverSpec.driverPod.getMetadata.getAnnotations.asScala + assert(driverAnnotations.size === 1) + val initContainers = OBJECT_MAPPER.readValue( + driverAnnotations(INIT_CONTAINER_ANNOTATION), classOf[Array[Container]]) + assert(initContainers.length === 1) + val initContainerEnv = initContainers.head.getEnv.asScala + assert(initContainerEnv.size === 1) + assert(initContainerEnv.head.getName === + SecondTestInitContainerConfigurationStep$.additionalInitContainerEnvKey) + assert(initContainerEnv.head.getValue === + SecondTestInitContainerConfigurationStep$.additionalInitContainerEnvValue) + val expectedSparkConf = Map( + EXECUTOR_INIT_CONTAINER_CONFIG_MAP.key -> CONFIG_MAP_NAME, + EXECUTOR_INIT_CONTAINER_CONFIG_MAP_KEY.key -> CONFIG_MAP_KEY, + SecondTestInitContainerConfigurationStep$.additionalDriverSparkConfKey -> + SecondTestInitContainerConfigurationStep$.additionalDriverSparkConfValue) + assert(preparedDriverSpec.driverSparkConf.getAll.toMap === expectedSparkConf) + assert(preparedDriverSpec.otherKubernetesResources.size === 2) + assert(preparedDriverSpec.otherKubernetesResources.contains( + FirstTestInitContainerConfigurationStep$.additionalKubernetesResource)) + assert(preparedDriverSpec.otherKubernetesResources.exists { + case configMap: ConfigMap => + val hasMatchingName = configMap.getMetadata.getName == CONFIG_MAP_NAME + val configMapData = configMap.getData.asScala + val hasCorrectNumberOfEntries = configMapData.size == 1 + val initContainerPropertiesRaw = configMapData(CONFIG_MAP_KEY) + val initContainerProperties = new Properties() + Utils.tryWithResource(new StringReader(initContainerPropertiesRaw)) { + initContainerProperties.load(_) + } + val initContainerPropertiesMap = Maps.fromProperties(initContainerProperties).asScala + val expectedInitContainerProperties = Map( + SecondTestInitContainerConfigurationStep$.additionalInitContainerPropertyKey -> + SecondTestInitContainerConfigurationStep$.additionalInitContainerPropertyValue) + val hasMatchingProperties = initContainerPropertiesMap == expectedInitContainerProperties + hasMatchingName && hasCorrectNumberOfEntries && hasMatchingProperties + case _ => false + }) + } +} + +private object FirstTestInitContainerConfigurationStep$ extends InitContainerConfigurationStep { + + val additionalLabels = Map("additionalLabelkey" -> "additionalLabelValue") + val additionalMainContainerEnvKey = "TEST_ENV_MAIN_KEY" + val additionalMainContainerEnvValue = "TEST_ENV_MAIN_VALUE" + val additionalKubernetesResource = new SecretBuilder() + .withNewMetadata() + .withName("test-secret") + .endMetadata() + .addToData("secret-key", "secret-value") + .build() + + override def configureInitContainer(initContainerSpec: InitContainerSpec): InitContainerSpec = { + val driverPod = new PodBuilder(initContainerSpec.podToInitialize) + .editOrNewMetadata() + .addToLabels(additionalLabels.asJava) + .endMetadata() + .build() + val mainContainer = new ContainerBuilder(initContainerSpec.driverContainer) + .addNewEnv() + .withName(additionalMainContainerEnvKey) + .withValue(additionalMainContainerEnvValue) + .endEnv() + .build() + initContainerSpec.copy( + podToInitialize = driverPod, + driverContainer = mainContainer, + initContainerDependentResources = initContainerSpec.initContainerDependentResources ++ + Seq(additionalKubernetesResource)) + } +} + +private object SecondTestInitContainerConfigurationStep$ extends InitContainerConfigurationStep { + val additionalInitContainerEnvKey = "TEST_ENV_INIT_KEY" + val additionalInitContainerEnvValue = "TEST_ENV_INIT_VALUE" + val additionalInitContainerPropertyKey = "spark.initcontainer.testkey" + val additionalInitContainerPropertyValue = "testvalue" + val additionalDriverSparkConfKey = "spark.driver.testkey" + val additionalDriverSparkConfValue = "spark.driver.testvalue" + + override def configureInitContainer(initContainerSpec: InitContainerSpec): InitContainerSpec = { + val initContainer = new ContainerBuilder(initContainerSpec.initContainer) + .addNewEnv() + .withName(additionalInitContainerEnvKey) + .withValue(additionalInitContainerEnvValue) + .endEnv() + .build() + val initContainerProperties = initContainerSpec.initContainerProperties ++ + Map(additionalInitContainerPropertyKey -> additionalInitContainerPropertyValue) + val driverSparkConf = initContainerSpec.additionalDriverSparkConf ++ + Map(additionalDriverSparkConfKey -> additionalDriverSparkConfValue) + initContainerSpec.copy( + initContainer = initContainer, + initContainerProperties = initContainerProperties, + additionalDriverSparkConf = driverSparkConf) + } +} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/BaseInitContainerConfigurationStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/BaseInitContainerConfigurationStepSuite.scala new file mode 100644 index 0000000000000..fe1af4bc5be2a --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/BaseInitContainerConfigurationStepSuite.scala @@ -0,0 +1,98 @@ +/* + * 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._ +import org.mockito.{Mock, MockitoAnnotations} +import org.mockito.Matchers.any +import org.mockito.Mockito.when +import org.mockito.invocation.InvocationOnMock +import org.mockito.stubbing.Answer +import org.scalatest.BeforeAndAfter +import scala.collection.JavaConverters._ + +import org.apache.spark.SparkFunSuite +import org.apache.spark.deploy.kubernetes.{PodWithDetachedInitContainer, SparkPodInitContainerBootstrap} +import org.apache.spark.deploy.kubernetes.config._ + +class BaseInitContainerConfigurationStepSuite extends SparkFunSuite with BeforeAndAfter{ + private val SPARK_JARS = Seq( + "hdfs://localhost:9000/app/jars/jar1.jar", "file:///app/jars/jar2.jar") + private val SPARK_FILES = Seq( + "hdfs://localhost:9000/app/files/file1.txt", "file:///app/files/file2.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" + private val POD_LABEL = Map("bootstrap" -> "true") + private val INIT_CONTAINER_NAME = "init-container" + private val DRIVER_CONTAINER_NAME = "driver-container" + + @Mock + private var podAndInitContainerBootstrap : SparkPodInitContainerBootstrap = _ + + before { + MockitoAnnotations.initMocks(this) + when(podAndInitContainerBootstrap.bootstrapInitContainerAndVolumes( + any[PodWithDetachedInitContainer])).thenAnswer(new Answer[PodWithDetachedInitContainer] { + override def answer(invocation: InvocationOnMock) : PodWithDetachedInitContainer = { + val pod = invocation.getArgumentAt(0, classOf[PodWithDetachedInitContainer]) + pod.copy( + pod = + new PodBuilder(pod.pod) + .withNewMetadata() + .addToLabels("bootstrap", "true") + .endMetadata() + .withNewSpec().endSpec() + .build(), + initContainer = + new ContainerBuilder() + .withName(INIT_CONTAINER_NAME).build(), + mainContainer = + new ContainerBuilder() + .withName(DRIVER_CONTAINER_NAME).build() + )}}) + } + + test("Test of additionalDriverSparkConf with mix of remote files and jars") { + val baseInitStep = new BaseInitContainerConfigurationStep( + SPARK_JARS, + SPARK_FILES, + JARS_DOWNLOAD_PATH, + FILES_DOWNLOAD_PATH, + CONFIG_MAP_NAME, + CONFIG_MAP_KEY, + podAndInitContainerBootstrap) + val expectedDriverSparkConf = Map( + INIT_CONTAINER_JARS_DOWNLOAD_LOCATION.key -> JARS_DOWNLOAD_PATH, + INIT_CONTAINER_FILES_DOWNLOAD_LOCATION.key -> FILES_DOWNLOAD_PATH, + INIT_CONTAINER_REMOTE_JARS.key -> "hdfs://localhost:9000/app/jars/jar1.jar", + INIT_CONTAINER_REMOTE_FILES.key -> "hdfs://localhost:9000/app/files/file1.txt") + val initContainerSpec = InitContainerSpec( + Map.empty[String, String], + Map.empty[String, String], + new Container(), + new Container(), + new Pod, + Seq.empty[HasMetadata]) + val returnContainerSpec = baseInitStep.configureInitContainer(initContainerSpec) + assert(expectedDriverSparkConf === returnContainerSpec.initContainerProperties) + assert(returnContainerSpec.initContainer.getName == INIT_CONTAINER_NAME) + assert(returnContainerSpec.driverContainer.getName == DRIVER_CONTAINER_NAME) + assert(returnContainerSpec.podToInitialize.getMetadata.getLabels.asScala === POD_LABEL) + } +} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/InitContainerConfigurationStepsOrchestratorSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/InitContainerConfigurationStepsOrchestratorSuite.scala new file mode 100644 index 0000000000000..1cc8007803457 --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/InitContainerConfigurationStepsOrchestratorSuite.scala @@ -0,0 +1,94 @@ +/* + * 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, SparkFunSuite} +import org.apache.spark.deploy.kubernetes.config._ +import org.apache.spark.deploy.kubernetes.constants._ + +class InitContainerConfigurationStepsOrchestratorSuite extends SparkFunSuite { + private val NAMESPACE = "namespace" + private val APP_RESOURCE_PREFIX = "spark-prefix" + private val SPARK_JARS = Seq( + "hdfs://localhost:9000/app/jars/jar1.jar", "file:///app/jars/jar2.jar") + private val SPARK_FILES = Seq( + "hdfs://localhost:9000/app/files/file1.txt", "file:///app/files/file2.txt") + private val JARS_DOWNLOAD_PATH = "/var/data/jars" + private val FILES_DOWNLOAD_PATH = "/var/data/files" + private val DOCKER_IMAGE_PULL_POLICY: String = "IfNotPresent" + 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 DRIVER_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 INIT_CONTAINER_CONFIG_MAP_NAME = "spark-init-config-map" + private val INIT_CONTAINER_CONFIG_MAP_KEY = "spark-init-config-map-key" + private val STAGING_SERVER_URI = "http://localhost:8000" + + test ("including step to contact resource staging server") { + val sparkConf = new SparkConf(true) + .set(KUBERNETES_DRIVER_LABELS, s"$DEPRECATED_CUSTOM_LABEL_KEY=$DEPRECATED_CUSTOM_LABEL_VALUE") + .set(s"$KUBERNETES_DRIVER_LABEL_PREFIX$CUSTOM_LABEL_KEY", CUSTOM_LABEL_VALUE) + .set(RESOURCE_STAGING_SERVER_URI, STAGING_SERVER_URI) + + val orchestrator = new InitContainerConfigurationStepsOrchestrator( + NAMESPACE, + APP_RESOURCE_PREFIX, + SPARK_JARS, + SPARK_FILES, + JARS_DOWNLOAD_PATH, + FILES_DOWNLOAD_PATH, + DOCKER_IMAGE_PULL_POLICY, + DRIVER_LABELS, + INIT_CONTAINER_CONFIG_MAP_NAME, + INIT_CONTAINER_CONFIG_MAP_KEY, + sparkConf) + val initSteps : Seq[InitContainerConfigurationStep] = + orchestrator.getAllConfigurationSteps() + assert(initSteps.length == 2) + assert(initSteps.head.isInstanceOf[BaseInitContainerConfigurationStep]) + assert(initSteps(1).isInstanceOf[SubmittedResourcesInitContainerConfigurationStep]) + } + + test ("not including steps because no contact to resource staging server") { + val sparkConf = new SparkConf(true) + .set(KUBERNETES_DRIVER_LABELS, s"$DEPRECATED_CUSTOM_LABEL_KEY=$DEPRECATED_CUSTOM_LABEL_VALUE") + .set(s"$KUBERNETES_DRIVER_LABEL_PREFIX$CUSTOM_LABEL_KEY", CUSTOM_LABEL_VALUE) + + val orchestrator = new InitContainerConfigurationStepsOrchestrator( + NAMESPACE, + APP_RESOURCE_PREFIX, + SPARK_JARS, + SPARK_FILES, + JARS_DOWNLOAD_PATH, + FILES_DOWNLOAD_PATH, + DOCKER_IMAGE_PULL_POLICY, + DRIVER_LABELS, + INIT_CONTAINER_CONFIG_MAP_NAME, + INIT_CONTAINER_CONFIG_MAP_KEY, + sparkConf) + val initSteps : Seq[InitContainerConfigurationStep] = + orchestrator.getAllConfigurationSteps() + assert(initSteps.length === 1) + assert(initSteps.head.isInstanceOf[BaseInitContainerConfigurationStep]) + } +} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/SubmittedResourcesInitContainerStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/SubmittedResourcesInitContainerStepSuite.scala new file mode 100644 index 0000000000000..2edaba93fe07f --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/SubmittedResourcesInitContainerStepSuite.scala @@ -0,0 +1,252 @@ +/* + * 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 java.util.UUID + +import com.google.common.base.Charsets +import com.google.common.io.{BaseEncoding, Files} +import io.fabric8.kubernetes.api.model._ +import org.mockito.{Mock, MockitoAnnotations} +import org.mockito.Matchers.any +import org.mockito.Mockito.when +import org.mockito.invocation.InvocationOnMock +import org.mockito.stubbing.Answer +import org.scalatest.BeforeAndAfter +import scala.collection.JavaConverters._ + +import org.apache.spark.SparkFunSuite +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, SubmittedResourceIdAndSecret} +import org.apache.spark.util.Utils + +class SubmittedResourcesInitContainerStepSuite extends SparkFunSuite with BeforeAndAfter { + private val RESOURCE_SECRET_NAME = "secret" + private val JARS_RESOURCE_ID = "jarsID" + private val JARS_SECRET = "jarsSecret" + private val FILES_RESOURCE_ID = "filesID" + private val FILES_SECRET = "filesSecret" + private val STAGING_SERVER_URI = "http://localhost:8000" + private val SECRET_MOUNT_PATH = "/tmp" + private val RSS_SECRET = Map( + INIT_CONTAINER_SUBMITTED_JARS_SECRET_KEY -> + BaseEncoding.base64().encode(JARS_SECRET.getBytes(Charsets.UTF_8)), + INIT_CONTAINER_SUBMITTED_FILES_SECRET_KEY -> + BaseEncoding.base64().encode(FILES_SECRET.getBytes(Charsets.UTF_8)) + ).asJava + private var RSS_WITH_SSL_SECRET: java.util.Map[String, String] = _ + private var TRUSTSTORE_FILENAME: String = "" + private var TRUSTSTORE_FILE: File = _ + private var TRUSTSTORE_URI: Option[String] = None + private val TRUSTSTORE_PASS = "trustStorePassword" + private val TRUSTSTORE_TYPE = "jks" + private var CERT_FILENAME: String = "" + private var CERT_FILE: File = _ + private var CERT_URI: Option[String] = None + + @Mock + private var submittedDependencyUploader: SubmittedDependencyUploader = _ + @Mock + private var submittedResourcesSecretPlugin: InitContainerResourceStagingServerSecretPlugin = _ + + before { + MockitoAnnotations.initMocks(this) + TRUSTSTORE_FILENAME = createTempFile(".jks") + TRUSTSTORE_FILE = new File(TRUSTSTORE_FILENAME) + TRUSTSTORE_URI = Some(TRUSTSTORE_FILENAME) + CERT_FILENAME = createTempFile("pem") + CERT_FILE = new File(CERT_FILENAME) + CERT_URI = Some(CERT_FILENAME) + RSS_WITH_SSL_SECRET = + (RSS_SECRET.asScala ++ Map( + INIT_CONTAINER_STAGING_SERVER_TRUSTSTORE_SECRET_KEY -> + BaseEncoding.base64().encode(Files.toByteArray(TRUSTSTORE_FILE)), + INIT_CONTAINER_STAGING_SERVER_CLIENT_CERT_SECRET_KEY -> + BaseEncoding.base64().encode(Files.toByteArray(CERT_FILE)) + )).asJava + when(submittedDependencyUploader.uploadJars()).thenReturn( + SubmittedResourceIdAndSecret(JARS_RESOURCE_ID, JARS_SECRET) + ) + when(submittedDependencyUploader.uploadFiles()).thenReturn( + SubmittedResourceIdAndSecret(FILES_RESOURCE_ID, FILES_SECRET) + ) + when(submittedResourcesSecretPlugin.addResourceStagingServerSecretVolumeToPod( + any[Pod])).thenAnswer(new Answer[Pod] { + override def answer(invocation: InvocationOnMock) : Pod = { + val pod = invocation.getArgumentAt(0, classOf[Pod]) + new PodBuilder(pod) + .withNewMetadata() + .addToLabels("mountedSecret", "true") + .endMetadata() + .withNewSpec().endSpec() + .build() + }}) + when(submittedResourcesSecretPlugin.mountResourceStagingServerSecretIntoInitContainer( + any[Container])).thenAnswer(new Answer[Container] { + override def answer(invocation: InvocationOnMock) : Container = { + val con = invocation.getArgumentAt(0, classOf[Container]) + new ContainerBuilder(con).withName("mountedSecret").build() + }}) + } + after { + TRUSTSTORE_FILE.delete() + CERT_FILE.delete() + } + test ("testing vanilla prepareInitContainer on resources and properties") { + val submittedResourceStep = new SubmittedResourcesInitContainerConfigurationStep( + RESOURCE_SECRET_NAME, + STAGING_SERVER_URI, + SECRET_MOUNT_PATH, + false, + None, + None, + None, + None, + submittedDependencyUploader, + submittedResourcesSecretPlugin + ) + val returnedInitContainer = + submittedResourceStep.configureInitContainer(InitContainerSpec( + Map.empty[String, String], + Map.empty[String, String], + new Container(), + new Container(), + new Pod(), + Seq.empty[HasMetadata])) + assert(returnedInitContainer.initContainer.getName === "mountedSecret") + assert(returnedInitContainer.podToInitialize.getMetadata.getLabels.asScala + === Map("mountedSecret" -> "true")) + assert(returnedInitContainer.initContainerDependentResources.length == 1) + val secret = returnedInitContainer.initContainerDependentResources.head.asInstanceOf[Secret] + assert(secret.getData === RSS_SECRET) + assert(secret.getMetadata.getName == RESOURCE_SECRET_NAME) + val expectedinitContainerProperties = Map( + RESOURCE_STAGING_SERVER_URI.key -> STAGING_SERVER_URI, + INIT_CONTAINER_DOWNLOAD_JARS_RESOURCE_IDENTIFIER.key -> JARS_RESOURCE_ID, + INIT_CONTAINER_DOWNLOAD_JARS_SECRET_LOCATION.key -> + s"$SECRET_MOUNT_PATH/$INIT_CONTAINER_SUBMITTED_JARS_SECRET_KEY", + INIT_CONTAINER_DOWNLOAD_FILES_RESOURCE_IDENTIFIER.key -> FILES_RESOURCE_ID, + INIT_CONTAINER_DOWNLOAD_FILES_SECRET_LOCATION.key -> + s"$SECRET_MOUNT_PATH/$INIT_CONTAINER_SUBMITTED_FILES_SECRET_KEY", + RESOURCE_STAGING_SERVER_SSL_ENABLED.key -> false.toString) + assert(returnedInitContainer.initContainerProperties === expectedinitContainerProperties) + assert(returnedInitContainer.additionalDriverSparkConf === + Map( + EXECUTOR_INIT_CONTAINER_SECRET.key -> RESOURCE_SECRET_NAME, + EXECUTOR_INIT_CONTAINER_SECRET_MOUNT_DIR.key -> SECRET_MOUNT_PATH)) + } + + test ("testing prepareInitContainer w/ CERT and TrustStore Files w/o SSL") { + val submittedResourceStep = new SubmittedResourcesInitContainerConfigurationStep( + RESOURCE_SECRET_NAME, + STAGING_SERVER_URI, + SECRET_MOUNT_PATH, + false, + TRUSTSTORE_URI, + CERT_URI, + Some(TRUSTSTORE_PASS), + Some(TRUSTSTORE_TYPE), + submittedDependencyUploader, + submittedResourcesSecretPlugin + ) + val returnedInitContainer = + submittedResourceStep.configureInitContainer(InitContainerSpec( + Map.empty[String, String], + Map.empty[String, String], + new Container(), + new Container(), + new Pod(), + Seq.empty[HasMetadata])) + val expectedinitContainerProperties = Map( + RESOURCE_STAGING_SERVER_URI.key -> STAGING_SERVER_URI, + INIT_CONTAINER_DOWNLOAD_JARS_RESOURCE_IDENTIFIER.key -> JARS_RESOURCE_ID, + INIT_CONTAINER_DOWNLOAD_JARS_SECRET_LOCATION.key -> + s"$SECRET_MOUNT_PATH/$INIT_CONTAINER_SUBMITTED_JARS_SECRET_KEY", + INIT_CONTAINER_DOWNLOAD_FILES_RESOURCE_IDENTIFIER.key -> FILES_RESOURCE_ID, + INIT_CONTAINER_DOWNLOAD_FILES_SECRET_LOCATION.key -> + s"$SECRET_MOUNT_PATH/$INIT_CONTAINER_SUBMITTED_FILES_SECRET_KEY", + RESOURCE_STAGING_SERVER_SSL_ENABLED.key -> false.toString, + RESOURCE_STAGING_SERVER_TRUSTSTORE_PASSWORD.key -> TRUSTSTORE_PASS, + RESOURCE_STAGING_SERVER_TRUSTSTORE_TYPE.key -> TRUSTSTORE_TYPE, + RESOURCE_STAGING_SERVER_TRUSTSTORE_FILE.key -> + s"$SECRET_MOUNT_PATH/$INIT_CONTAINER_STAGING_SERVER_TRUSTSTORE_SECRET_KEY", + RESOURCE_STAGING_SERVER_CLIENT_CERT_PEM.key -> + s"$SECRET_MOUNT_PATH/$INIT_CONTAINER_STAGING_SERVER_CLIENT_CERT_SECRET_KEY" + ) + assert(returnedInitContainer.initContainerProperties === expectedinitContainerProperties) + assert(returnedInitContainer.initContainerDependentResources.length == 1) + val secret = returnedInitContainer.initContainerDependentResources.head.asInstanceOf[Secret] + assert(secret.getData === RSS_WITH_SSL_SECRET) + assert(secret.getMetadata.getName == RESOURCE_SECRET_NAME) + + } + + test ("testing prepareInitContainer w/ local CERT and TrustStore Files w/o SSL") { + val LOCAL_TRUST_FILE = "local:///tmp/trust.jsk" + val LOCAL_CERT_FILE = "local:///tmp/cert.pem" + val submittedResourceStep = new SubmittedResourcesInitContainerConfigurationStep( + RESOURCE_SECRET_NAME, + STAGING_SERVER_URI, + SECRET_MOUNT_PATH, + false, + Some(LOCAL_TRUST_FILE), + Some(LOCAL_CERT_FILE), + Some(TRUSTSTORE_PASS), + Some(TRUSTSTORE_TYPE), + submittedDependencyUploader, + submittedResourcesSecretPlugin + ) + val returnedInitContainer = + submittedResourceStep.configureInitContainer(InitContainerSpec( + Map.empty[String, String], + Map.empty[String, String], + new Container(), + new Container(), + new Pod(), + Seq.empty[HasMetadata])) + val expectedinitContainerProperties = Map( + RESOURCE_STAGING_SERVER_URI.key -> STAGING_SERVER_URI, + INIT_CONTAINER_DOWNLOAD_JARS_RESOURCE_IDENTIFIER.key -> JARS_RESOURCE_ID, + INIT_CONTAINER_DOWNLOAD_JARS_SECRET_LOCATION.key -> + s"$SECRET_MOUNT_PATH/$INIT_CONTAINER_SUBMITTED_JARS_SECRET_KEY", + INIT_CONTAINER_DOWNLOAD_FILES_RESOURCE_IDENTIFIER.key -> FILES_RESOURCE_ID, + INIT_CONTAINER_DOWNLOAD_FILES_SECRET_LOCATION.key -> + s"$SECRET_MOUNT_PATH/$INIT_CONTAINER_SUBMITTED_FILES_SECRET_KEY", + RESOURCE_STAGING_SERVER_SSL_ENABLED.key -> false.toString, + RESOURCE_STAGING_SERVER_TRUSTSTORE_PASSWORD.key -> TRUSTSTORE_PASS, + RESOURCE_STAGING_SERVER_TRUSTSTORE_TYPE.key -> TRUSTSTORE_TYPE, + RESOURCE_STAGING_SERVER_TRUSTSTORE_FILE.key -> + "/tmp/trust.jsk", + RESOURCE_STAGING_SERVER_CLIENT_CERT_PEM.key -> + "/tmp/cert.pem" + ) + assert(returnedInitContainer.initContainerProperties === expectedinitContainerProperties) + assert(returnedInitContainer.initContainerDependentResources.length == 1) + val secret = returnedInitContainer.initContainerDependentResources.head.asInstanceOf[Secret] + assert(secret.getData === RSS_SECRET) + assert(secret.getMetadata.getName == RESOURCE_SECRET_NAME) + } + private def createTempFile(extension: String): String = { + val dir = Utils.createTempDir() + val file = new File(dir, s"${UUID.randomUUID().toString}.$extension") + Files.write(UUID.randomUUID().toString, file, Charsets.UTF_8) + file.getAbsolutePath + } +} 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..c6cd6a74c88d1 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 @@ -72,7 +72,7 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { kubernetesTestComponents.deleteNamespace() } - test("Run PySpark Job on file from SUBMITTER") { + test("Run PySpark Job on file from SUBMITTER with --py-files") { assume(testBackend.name == MINIKUBE_TEST_BACKEND) launchStagingServer(SSLOptions(), None) @@ -83,7 +83,9 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { System.getProperty("spark.docker.test.executorImage", "spark-executor-py:latest")) runPySparkPiAndVerifyCompletion( - PYSPARK_PI_SUBMITTER_LOCAL_FILE_LOCATION) + PYSPARK_PI_SUBMITTER_LOCAL_FILE_LOCATION, + Seq(PYSPARK_SORT_CONTAINER_LOCAL_FILE_LOCATION) + ) } test("Run PySpark Job on file from CONTAINER with spark.jar defined") { @@ -96,8 +98,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 +155,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]) + Seq("The Result is"), + Array.empty[String], + Seq.empty[String]) } test("Use remote resources without the resource staging server.") { @@ -217,10 +219,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)) + Seq(s"File found at /opt/spark/${testExistenceFile.getName} with correct contents."), + Array(testExistenceFile.getName, TEST_EXISTENCE_FILE_CONTENTS), + Seq.empty[String]) } test("Use a very long application name.") { @@ -248,26 +251,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, + Seq("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, + Seq("Submitting 5 missing tasks from ResultStage", "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, + expectedLogOnCompletion: Seq[String], + 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) @@ -275,11 +287,13 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { .getItems .get(0) Eventually.eventually(TIMEOUT, INTERVAL) { - assert(kubernetesTestComponents.kubernetesClient - .pods() - .withName(driverPod.getMetadata.getName) - .getLog - .contains(expectedLogOnCompletion), "The application did not complete.") + expectedLogOnCompletion.foreach { e => + assert(kubernetesTestComponents.kubernetesClient + .pods() + .withName(driverPod.getMetadata.getName) + .getLog + .contains(e), "The application did not complete.") + } } } @@ -347,6 +361,8 @@ private[spark] object KubernetesSuite { val PYSPARK_PI_MAIN_CLASS = "org.apache.spark.deploy.PythonRunner" val PYSPARK_PI_CONTAINER_LOCAL_FILE_LOCATION = "local:///opt/spark/examples/src/main/python/pi.py" + val PYSPARK_SORT_CONTAINER_LOCAL_FILE_LOCATION = + "local:///opt/spark/examples/src/main/python/sort.py" val PYSPARK_PI_SUBMITTER_LOCAL_FILE_LOCATION = "src/test/python/pi.py" val FILE_EXISTENCE_MAIN_CLASS = "org.apache.spark.deploy.kubernetes" + ".integrationtest.jobs.FileExistenceTest"