From d3cbbdd8ce45592caa706d0b4d29c873a176e40c Mon Sep 17 00:00:00 2001 From: Yinan Li Date: Tue, 12 Dec 2017 10:24:27 -0800 Subject: [PATCH 01/14] [SPARK-22757][Kubernetes] Enable use of remote dependencies in Kubernetes mode --- .../org/apache/spark/deploy/k8s/Config.scala | 61 ++++++- .../apache/spark/deploy/k8s/Constants.scala | 11 ++ .../deploy/k8s/InitContainerBootstrap.scala | 128 ++++++++++++++ .../deploy/k8s/MountSecretsBootstrap.scala | 67 ++++++++ .../k8s/PodWithDetachedInitContainer.scala | 31 ++++ ...DriverConfigurationStepsOrchestrator.scala | 54 +++++- .../deploy/k8s/submit/InitContainerUtil.scala | 37 ++++ .../k8s/submit/KubernetesFileUtils.scala | 12 ++ .../DriverInitContainerBootstrapStep.scala | 91 ++++++++++ .../submit/steps/DriverMountSecretsStep.scala | 38 +++++ .../BaseInitContainerConfigurationStep.scala | 63 +++++++ .../InitContainerConfigurationStep.scala | 25 +++ ...tainerConfigurationStepsOrchestrator.scala | 83 +++++++++ .../InitContainerMountSecretsStep.scala | 40 +++++ .../initcontainer/InitContainerSpec.scala | 37 ++++ .../spark/deploy/rest/k8s/FileFetcher.scala | 27 +++ ...SparkDependencyDownloadInitContainer.scala | 129 ++++++++++++++ .../rest/k8s/SparkConfPropertiesParser.scala | 50 ++++++ .../cluster/k8s/ExecutorPodFactory.scala | 37 +++- .../k8s/KubernetesClusterManager.scala | 63 ++++++- ...rConfigurationStepsOrchestratorSuite.scala | 54 +++++- .../deploy/k8s/submit/SecretVolumeUtils.scala | 36 ++++ ...riverInitContainerBootstrapStepSuite.scala | 159 ++++++++++++++++++ .../steps/DriverMountSecretsStepSuite.scala | 47 ++++++ ...eInitContainerConfigurationStepSuite.scala | 97 +++++++++++ ...rConfigurationStepsOrchestratorSuite.scala | 98 +++++++++++ .../InitContainerMountSecretsStepSuite.scala | 57 +++++++ ...DependencyDownloadInitContainerSuite.scala | 94 +++++++++++ .../cluster/k8s/ExecutorPodFactorySuite.scala | 82 ++++++++- .../src/main/dockerfiles/driver/Dockerfile | 1 + .../src/main/dockerfiles/executor/Dockerfile | 1 + .../dockerfiles/init-container/Dockerfile | 24 +++ 32 files changed, 1813 insertions(+), 21 deletions(-) create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/InitContainerBootstrap.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/MountSecretsBootstrap.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/PodWithDetachedInitContainer.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/InitContainerUtil.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DriverInitContainerBootstrapStep.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DriverMountSecretsStep.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/BaseInitContainerConfigurationStep.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerConfigurationStep.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerConfigurationStepsOrchestrator.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerMountSecretsStep.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerSpec.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/FileFetcher.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/KubernetesSparkDependencyDownloadInitContainer.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/SparkConfPropertiesParser.scala create mode 100644 resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/SecretVolumeUtils.scala create mode 100644 resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/DriverInitContainerBootstrapStepSuite.scala create mode 100644 resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/DriverMountSecretsStepSuite.scala create mode 100644 resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/BaseInitContainerConfigurationStepSuite.scala create mode 100644 resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerConfigurationStepsOrchestratorSuite.scala create mode 100644 resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerMountSecretsStepSuite.scala create mode 100644 resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/k8s/KubernetesSparkDependencyDownloadInitContainerSuite.scala create mode 100644 resource-managers/kubernetes/docker/src/main/dockerfiles/init-container/Dockerfile diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala index 45f527959cbe..52a7b2eb672e 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala @@ -20,7 +20,6 @@ import java.util.concurrent.TimeUnit import org.apache.spark.internal.Logging import org.apache.spark.internal.config.ConfigBuilder -import org.apache.spark.network.util.ByteUnit private[spark] object Config extends Logging { @@ -132,20 +131,66 @@ private[spark] object Config extends Logging { val JARS_DOWNLOAD_LOCATION = ConfigBuilder("spark.kubernetes.mountDependencies.jarsDownloadDir") - .doc("Location to download jars to in the driver and executors. When using" + - " spark-submit, this directory must be empty and will be mounted as an empty directory" + - " volume on the driver and executor pod.") + .doc("Location to download jars to in the driver and executors. When using " + + "spark-submit, this directory must be empty and will be mounted as an empty directory " + + "volume on the driver and executor pod.") .stringConf .createWithDefault("/var/spark-data/spark-jars") val FILES_DOWNLOAD_LOCATION = ConfigBuilder("spark.kubernetes.mountDependencies.filesDownloadDir") - .doc("Location to download files to in the driver and executors. When using" + - " spark-submit, this directory must be empty and will be mounted as an empty directory" + - " volume on the driver and executor pods.") + .doc("Location to download files to in the driver and executors. When using " + + "spark-submit, this directory must be empty and will be mounted as an empty directory " + + "volume on the driver and executor pods.") .stringConf .createWithDefault("/var/spark-data/spark-files") + val INIT_CONTAINER_DOCKER_IMAGE = + ConfigBuilder("spark.kubernetes.initContainer.docker.image") + .doc("Image for the driver and executor's init-container that downloads dependencies.") + .stringConf + .createOptional + + val INIT_CONTAINER_MOUNT_TIMEOUT = + ConfigBuilder("spark.kubernetes.mountDependencies.mountTimeout") + .doc("Timeout before aborting the attempt to download and unpack local dependencies from " + + "remote locations and the resource staging server when initializing the driver and " + + "executor pods.") + .timeConf(TimeUnit.MINUTES) + .createWithDefault(5) + + val INIT_CONTAINER_REMOTE_JARS = + ConfigBuilder("spark.kubernetes.initContainer.remoteJars") + .doc("Comma-separated list of jar URIs to download in the init-container. This is " + + "calculated from spark.jars.") + .internal() + .stringConf + .createOptional + + val INIT_CONTAINER_REMOTE_FILES = + ConfigBuilder("spark.kubernetes.initContainer.remoteFiles") + .doc("Comma-separated list of file URIs to download in the init-container. This is " + + "calculated from spark.files.") + .internal() + .stringConf + .createOptional + + val INIT_CONTAINER_CONFIG_MAP_NAME = + ConfigBuilder("spark.kubernetes.initContainer.configMapName") + .doc("Name of the config map to use in the init-container that retrieves submitted files " + + "for the executor.") + .internal() + .stringConf + .createOptional + + val INIT_CONTAINER_CONFIG_MAP_KEY_CONF = + ConfigBuilder("spark.kubernetes.initContainer.configMapKey") + .doc("Key for the entry in the init container config map for submitted files that " + + "corresponds to the properties for this init-container.") + .internal() + .stringConf + .createOptional + val KUBERNETES_AUTH_SUBMISSION_CONF_PREFIX = "spark.kubernetes.authenticate.submission" @@ -153,9 +198,11 @@ private[spark] object Config extends Logging { val KUBERNETES_DRIVER_LABEL_PREFIX = "spark.kubernetes.driver.label." val KUBERNETES_DRIVER_ANNOTATION_PREFIX = "spark.kubernetes.driver.annotation." + val KUBERNETES_DRIVER_SECRETS_PREFIX = "spark.kubernetes.driver.secrets." val KUBERNETES_EXECUTOR_LABEL_PREFIX = "spark.kubernetes.executor.label." val KUBERNETES_EXECUTOR_ANNOTATION_PREFIX = "spark.kubernetes.executor.annotation." + val KUBERNETES_EXECUTOR_SECRETS_PREFIX = "spark.kubernetes.executor.secrets." val KUBERNETES_DRIVER_ENV_KEY = "spark.kubernetes.driverEnv." } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Constants.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Constants.scala index 0b91145405d3..111cb2a3b75e 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Constants.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Constants.scala @@ -69,6 +69,17 @@ private[spark] object Constants { val ENV_DRIVER_JAVA_OPTS = "SPARK_DRIVER_JAVA_OPTS" val ENV_DRIVER_BIND_ADDRESS = "SPARK_DRIVER_BIND_ADDRESS" val ENV_DRIVER_MEMORY = "SPARK_DRIVER_MEMORY" + val ENV_MOUNTED_FILES_DIR = "SPARK_MOUNTED_FILES_DIR" + + // Bootstrapping dependencies with the init-container + val INIT_CONTAINER_DOWNLOAD_JARS_VOLUME_NAME = "download-jars-volume" + val INIT_CONTAINER_DOWNLOAD_FILES_VOLUME_NAME = "download-files-volume" + val INIT_CONTAINER_PROPERTIES_FILE_VOLUME = "spark-init-properties" + val INIT_CONTAINER_PROPERTIES_FILE_DIR = "/etc/spark-init" + val INIT_CONTAINER_PROPERTIES_FILE_NAME = "spark-init.properties" + val INIT_CONTAINER_PROPERTIES_FILE_PATH = + s"$INIT_CONTAINER_PROPERTIES_FILE_DIR/$INIT_CONTAINER_PROPERTIES_FILE_NAME" + val INIT_CONTAINER_SECRET_VOLUME_NAME = "spark-init-secret" // Miscellaneous val KUBERNETES_MASTER_INTERNAL_URL = "https://kubernetes.default.svc" diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/InitContainerBootstrap.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/InitContainerBootstrap.scala new file mode 100644 index 000000000000..bae699b1e118 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/InitContainerBootstrap.scala @@ -0,0 +1,128 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.k8s + +import scala.collection.JavaConverters._ + +import io.fabric8.kubernetes.api.model.{ContainerBuilder, EmptyDirVolumeSource, EnvVarBuilder, PodBuilder, VolumeMount, VolumeMountBuilder} + +import org.apache.spark.{SparkConf, SparkException} +import org.apache.spark.deploy.k8s.Config._ +import org.apache.spark.deploy.k8s.Constants._ + +/** + * 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 InitContainerBootstrap { + /** + * Bootstraps an init-container that downloads dependencies to be used by a main container. + */ + def bootstrapInitContainer( + originalPodWithInitContainer: PodWithDetachedInitContainer) + : PodWithDetachedInitContainer +} + +private[spark] class InitContainerBootstrapImpl( + initContainerImage: String, + dockerImagePullPolicy: String, + jarsDownloadPath: String, + filesDownloadPath: String, + downloadTimeoutMinutes: Long, + initContainerConfigMapName: String, + initContainerConfigMapKey: String, + sparkRole: String, + sparkConf: SparkConf) + extends InitContainerBootstrap { + + override def bootstrapInitContainer( + podWithDetachedInitContainer: PodWithDetachedInitContainer): PodWithDetachedInitContainer = { + val sharedVolumeMounts = Seq[VolumeMount]( + new VolumeMountBuilder() + .withName(INIT_CONTAINER_DOWNLOAD_JARS_VOLUME_NAME) + .withMountPath(jarsDownloadPath) + .build(), + new VolumeMountBuilder() + .withName(INIT_CONTAINER_DOWNLOAD_FILES_VOLUME_NAME) + .withMountPath(filesDownloadPath) + .build()) + + val initContainerCustomEnvVarKeyPrefix = sparkRole match { + case SPARK_POD_DRIVER_ROLE => KUBERNETES_DRIVER_ENV_KEY + case SPARK_POD_EXECUTOR_ROLE => "spark.executorEnv." + case _ => throw new SparkException(s"$sparkRole is not a valid Spark pod role") + } + val initContainerCustomEnvVars = sparkConf.getAllWithPrefix(initContainerCustomEnvVarKeyPrefix) + .toSeq + .map(env => + new EnvVarBuilder() + .withName(env._1) + .withValue(env._2) + .build()) + + val initContainer = new ContainerBuilder(podWithDetachedInitContainer.initContainer) + .withName(s"spark-init") + .withImage(initContainerImage) + .withImagePullPolicy(dockerImagePullPolicy) + .addAllToEnv(initContainerCustomEnvVars.asJava) + .addNewVolumeMount() + .withName(INIT_CONTAINER_PROPERTIES_FILE_VOLUME) + .withMountPath(INIT_CONTAINER_PROPERTIES_FILE_DIR) + .endVolumeMount() + .addToVolumeMounts(sharedVolumeMounts: _*) + .addToArgs(INIT_CONTAINER_PROPERTIES_FILE_PATH) + .build() + + val podWithBasicVolumes = new PodBuilder(podWithDetachedInitContainer.pod) + .editSpec() + .addNewVolume() + .withName(INIT_CONTAINER_PROPERTIES_FILE_VOLUME) + .withNewConfigMap() + .withName(initContainerConfigMapName) + .addNewItem() + .withKey(initContainerConfigMapKey) + .withPath(INIT_CONTAINER_PROPERTIES_FILE_NAME) + .endItem() + .endConfigMap() + .endVolume() + .addNewVolume() + .withName(INIT_CONTAINER_DOWNLOAD_JARS_VOLUME_NAME) + .withEmptyDir(new EmptyDirVolumeSource()) + .endVolume() + .addNewVolume() + .withName(INIT_CONTAINER_DOWNLOAD_FILES_VOLUME_NAME) + .withEmptyDir(new EmptyDirVolumeSource()) + .endVolume() + .endSpec() + .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/k8s/MountSecretsBootstrap.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/MountSecretsBootstrap.scala new file mode 100644 index 000000000000..9a0326aab797 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/MountSecretsBootstrap.scala @@ -0,0 +1,67 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.k8s + +import io.fabric8.kubernetes.api.model.{Container, ContainerBuilder, Pod, PodBuilder} + +/** + * Bootstraps a driver or executor container or an init-container with needed secrets mounted. + */ +private[spark] trait MountSecretsBootstrap { + + /** + * Mounts Kubernetes secrets as secret volumes into the given container in the given pod. + * + * @param pod the pod into which the secret volumes are being added. + * @param container the container into which the secret volumes are being mounted. + * @return the updated pod and container with the secrets mounted. + */ + def mountSecrets(pod: Pod, container: Container): (Pod, Container) +} + +private[spark] class MountSecretsBootstrapImpl( + secretNamesToMountPaths: Map[String, String]) extends MountSecretsBootstrap { + + override def mountSecrets(pod: Pod, container: Container): (Pod, Container) = { + var podBuilder = new PodBuilder(pod) + secretNamesToMountPaths.keys.foreach(name => + podBuilder = podBuilder + .editOrNewSpec() + .addNewVolume() + .withName(secretVolumeName(name)) + .withNewSecret() + .withSecretName(name) + .endSecret() + .endVolume() + .endSpec()) + + var containerBuilder = new ContainerBuilder(container) + secretNamesToMountPaths.foreach(namePath => + containerBuilder = containerBuilder + .addNewVolumeMount() + .withName(secretVolumeName(namePath._1)) + .withMountPath(namePath._2) + .endVolumeMount() + ) + + (podBuilder.build(), containerBuilder.build()) + } + + private def secretVolumeName(secretName: String): String = { + secretName + "-volume" + } +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/PodWithDetachedInitContainer.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/PodWithDetachedInitContainer.scala new file mode 100644 index 000000000000..0b79f8b12e80 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/PodWithDetachedInitContainer.scala @@ -0,0 +1,31 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.k8s + +import io.fabric8.kubernetes.api.model.{Container, Pod} + +/** + * Represents a pod with a detached init-container (not yet added to the pod). + * + * @param pod the pod + * @param initContainer the init-container in the pod + * @param mainContainer the main container in the 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/k8s/submit/DriverConfigurationStepsOrchestrator.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestrator.scala index 1411e6f40b46..9a0c3689b76c 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestrator.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestrator.scala @@ -21,12 +21,14 @@ import java.util.UUID import com.google.common.primitives.Longs import org.apache.spark.SparkConf +import org.apache.spark.deploy.k8s.{ConfigurationUtils, MountSecretsBootstrapImpl} import org.apache.spark.deploy.k8s.Config._ -import org.apache.spark.deploy.k8s.ConfigurationUtils import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.deploy.k8s.submit.steps._ +import org.apache.spark.deploy.k8s.submit.steps.initcontainer.InitContainerConfigurationStepsOrchestrator import org.apache.spark.launcher.SparkLauncher import org.apache.spark.util.SystemClock +import org.apache.spark.util.Utils /** * Constructs the complete list of driver configuration steps to run to deploy the Spark driver. @@ -50,6 +52,7 @@ private[spark] class DriverConfigurationStepsOrchestrator( } private val imagePullPolicy = submissionSparkConf.get(CONTAINER_IMAGE_PULL_POLICY) + private val initContainerConfigMapName = s"$kubernetesResourceNamePrefix-init-config" private val jarsDownloadPath = submissionSparkConf.get(JARS_DOWNLOAD_LOCATION) private val filesDownloadPath = submissionSparkConf.get(FILES_DOWNLOAD_LOCATION) @@ -64,6 +67,10 @@ private[spark] class DriverConfigurationStepsOrchestrator( s"$SPARK_ROLE_LABEL is not allowed as it is reserved for Spark bookkeeping " + "operations.") + val driverSecretNamesToMountPaths = ConfigurationUtils.parsePrefixedKeyValuePairs( + submissionSparkConf, + KUBERNETES_DRIVER_SECRETS_PREFIX) + val allDriverLabels = driverCustomLabels ++ Map( SPARK_APP_ID_LABEL -> kubernetesAppId, SPARK_ROLE_LABEL -> SPARK_POD_DRIVER_ROLE) @@ -116,10 +123,53 @@ private[spark] class DriverConfigurationStepsOrchestrator( None } + val mayBeInitContainerBootstrapStep = + if (areAnyFilesNonContainerLocal(sparkJars ++ sparkFiles)) { + val initContainerConfigurationStepsOrchestrator = + new InitContainerConfigurationStepsOrchestrator( + namespace, + kubernetesResourceNamePrefix, + sparkJars, + sparkFiles, + jarsDownloadPath, + filesDownloadPath, + imagePullPolicy, + allDriverLabels, + initContainerConfigMapName, + INIT_CONTAINER_PROPERTIES_FILE_NAME, + submissionSparkConf) + val initContainerConfigurationSteps = + initContainerConfigurationStepsOrchestrator.getAllConfigurationSteps() + val initContainerBootstrapStep = + new DriverInitContainerBootstrapStep( + initContainerConfigurationSteps, + initContainerConfigMapName, + INIT_CONTAINER_PROPERTIES_FILE_NAME) + + Some(initContainerBootstrapStep) + } else { + None + } + + val mayBeMountSecretsStep = if (driverSecretNamesToMountPaths.nonEmpty) { + val mountSecretsBootstrap = new MountSecretsBootstrapImpl(driverSecretNamesToMountPaths) + Some(new DriverMountSecretsStep(mountSecretsBootstrap)) + } else { + None + } + Seq( initialSubmissionStep, driverAddressStep, kubernetesCredentialsStep) ++ - maybeDependencyResolutionStep.toSeq + maybeDependencyResolutionStep.toSeq ++ + mayBeInitContainerBootstrapStep.toSeq ++ + mayBeMountSecretsStep.toSeq + } + + private def areAnyFilesNonContainerLocal(files: Seq[String]): Boolean = { + files.exists { uri => + Option(Utils.resolveURI(uri).getScheme).getOrElse("file") != "local" + } } } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/InitContainerUtil.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/InitContainerUtil.scala new file mode 100644 index 000000000000..3b156138ecba --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/InitContainerUtil.scala @@ -0,0 +1,37 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.k8s.submit + +import io.fabric8.kubernetes.api.model.{Container, Pod, PodBuilder} + +private[spark] object InitContainerUtil { + + /** + * Append (add to the list of InitContainers) a given init-container to a pod. + * + * @param originalPodSpec original specification of the pod + * @param initContainer the init-container to add to the pod + * @return the pod with the init-container added to the list of InitContainers + */ + def appendInitContainer(originalPodSpec: Pod, initContainer: Container): Pod = { + new PodBuilder(originalPodSpec) + .editOrNewSpec() + .addToInitContainers(initContainer) + .endSpec() + .build() + } +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesFileUtils.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesFileUtils.scala index a38cf55fc3d5..3bbd1d48e298 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesFileUtils.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesFileUtils.scala @@ -47,6 +47,18 @@ private[spark] object KubernetesFileUtils { } } + /** + * Get from a given collection of file URIs the ones that represent remote files. + */ + def getOnlyRemoteFiles(uris: Iterable[String]): Iterable[String] = { + filterUriStringsByScheme(uris, scheme => scheme != "file" && scheme != "local") + } + + private def filterUriStringsByScheme( + uris: Iterable[String], schemeFilter: (String => Boolean)): Iterable[String] = { + uris.filter(uri => schemeFilter(Option(Utils.resolveURI(uri).getScheme).getOrElse("file"))) + } + private def resolveFileUri( uri: String, fileDownloadPath: String, diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DriverInitContainerBootstrapStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DriverInitContainerBootstrapStep.scala new file mode 100644 index 000000000000..4b8fd0234ec7 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DriverInitContainerBootstrapStep.scala @@ -0,0 +1,91 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.k8s.submit.steps + +import java.io.StringWriter +import java.util.Properties + +import io.fabric8.kubernetes.api.model.{ConfigMap, ConfigMapBuilder, ContainerBuilder, HasMetadata} + +import org.apache.spark.deploy.k8s.Config._ +import org.apache.spark.deploy.k8s.submit.{InitContainerUtil, KubernetesDriverSpec} +import org.apache.spark.deploy.k8s.submit.steps.initcontainer.{InitContainerConfigurationStep, InitContainerSpec} + +/** + * Configures the init-container that bootstraps dependencies into the driver pod, including + * building a ConfigMap that will be mounted into the init-container. The ConfigMap carries + * configuration properties for the init-container. + */ +private[spark] class DriverInitContainerBootstrapStep( + initContainerConfigurationSteps: Seq[InitContainerConfigurationStep], + initContainerConfigMapName: String, + initContainerConfigMapKey: String) + extends DriverConfigurationStep { + + override def configureDriver(driverSpec: KubernetesDriverSpec): KubernetesDriverSpec = { + var currentInitContainerSpec = InitContainerSpec( + initContainerProperties = Map.empty[String, String], + driverSparkConf = Map.empty[String, String], + initContainer = new ContainerBuilder().build(), + driverContainer = driverSpec.driverContainer, + driverPod = driverSpec.driverPod, + initContainerDependentResources = Seq.empty[HasMetadata]) + for (nextStep <- initContainerConfigurationSteps) { + currentInitContainerSpec = nextStep.configureInitContainer(currentInitContainerSpec) + } + + val configMap = buildConfigMap( + initContainerConfigMapName, + initContainerConfigMapKey, + currentInitContainerSpec.initContainerProperties) + val resolvedDriverSparkConf = driverSpec.driverSparkConf + .clone() + .set(INIT_CONTAINER_CONFIG_MAP_NAME, initContainerConfigMapName) + .set(INIT_CONTAINER_CONFIG_MAP_KEY_CONF, initContainerConfigMapKey) + .setAll(currentInitContainerSpec.driverSparkConf) + val resolvedDriverPod = InitContainerUtil.appendInitContainer( + currentInitContainerSpec.driverPod, currentInitContainerSpec.initContainer) + + driverSpec.copy( + driverPod = resolvedDriverPod, + driverContainer = currentInitContainerSpec.driverContainer, + driverSparkConf = resolvedDriverSparkConf, + otherKubernetesResources = + driverSpec.otherKubernetesResources ++ + currentInitContainerSpec.initContainerDependentResources ++ + Seq(configMap)) + } + + private def buildConfigMap( + configMapName: String, + configMapKey: String, + config: Map[String, String]): ConfigMap = { + val properties = new Properties() + config.foreach { entry => + properties.setProperty(entry._1, entry._2) } + val propertiesWriter = new StringWriter() + properties.store(propertiesWriter, + s"Java properties built from Kubernetes config map with name: $configMapName " + + s"and config map key: $configMapKey") + new ConfigMapBuilder() + .withNewMetadata() + .withName(configMapName) + .endMetadata() + .addToData(configMapKey, propertiesWriter.toString) + .build() + } +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DriverMountSecretsStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DriverMountSecretsStep.scala new file mode 100644 index 000000000000..9eaaf32d5f58 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DriverMountSecretsStep.scala @@ -0,0 +1,38 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.k8s.submit.steps + +import org.apache.spark.deploy.k8s.MountSecretsBootstrap +import org.apache.spark.deploy.k8s.submit.KubernetesDriverSpec + +/** + * A driver configuration step for mounting user-specified secrets onto user-specified paths. + * + * @param mountSecretsBootstrap a utility actually handling mounting of the secrets. + */ +private[spark] class DriverMountSecretsStep( + mountSecretsBootstrap: MountSecretsBootstrap) extends DriverConfigurationStep { + + override def configureDriver(driverSpec: KubernetesDriverSpec): KubernetesDriverSpec = { + val (driverPodWithSecretsMounted, driverContainerWithSecretsMounted) = + mountSecretsBootstrap.mountSecrets(driverSpec.driverPod, driverSpec.driverContainer) + driverSpec.copy( + driverPod = driverPodWithSecretsMounted, + driverContainer = driverContainerWithSecretsMounted + ) + } +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/BaseInitContainerConfigurationStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/BaseInitContainerConfigurationStep.scala new file mode 100644 index 000000000000..f4e740b0caa1 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/BaseInitContainerConfigurationStep.scala @@ -0,0 +1,63 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.k8s.submit.steps.initcontainer + +import org.apache.spark.deploy.k8s.{InitContainerBootstrap, PodWithDetachedInitContainer} +import org.apache.spark.deploy.k8s.Config._ +import org.apache.spark.deploy.k8s.submit.KubernetesFileUtils + +private[spark] class BaseInitContainerConfigurationStep( + sparkJars: Seq[String], + sparkFiles: Seq[String], + jarsDownloadPath: String, + filesDownloadPath: String, + initContainerBootstrap: InitContainerBootstrap) + extends InitContainerConfigurationStep { + + override def configureInitContainer(initContainerSpec: InitContainerSpec): InitContainerSpec = { + val remoteJarsToDownload = KubernetesFileUtils.getOnlyRemoteFiles(sparkJars) + val remoteFilesToDownload = KubernetesFileUtils.getOnlyRemoteFiles(sparkFiles) + val remoteJarsConf = if (remoteJarsToDownload.nonEmpty) { + Map(INIT_CONTAINER_REMOTE_JARS.key -> remoteJarsToDownload.mkString(",")) + } else { + Map.empty[String, String] + } + val remoteFilesConf = if (remoteFilesToDownload.nonEmpty) { + Map(INIT_CONTAINER_REMOTE_FILES.key -> remoteFilesToDownload.mkString(",")) + } else { + Map.empty[String, String] + } + + val baseInitContainerConfig = Map[String, String]( + JARS_DOWNLOAD_LOCATION.key -> jarsDownloadPath, + FILES_DOWNLOAD_LOCATION.key -> filesDownloadPath) ++ + remoteJarsConf ++ + remoteFilesConf + val bootstrappedPodAndInitContainer = + initContainerBootstrap.bootstrapInitContainer( + PodWithDetachedInitContainer( + initContainerSpec.driverPod, + initContainerSpec.initContainer, + initContainerSpec.driverContainer)) + + initContainerSpec.copy( + initContainer = bootstrappedPodAndInitContainer.initContainer, + driverContainer = bootstrappedPodAndInitContainer.mainContainer, + driverPod = bootstrappedPodAndInitContainer.pod, + initContainerProperties = baseInitContainerConfig) + } +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerConfigurationStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerConfigurationStep.scala new file mode 100644 index 000000000000..0cca50539ec8 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/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.k8s.submit.steps.initcontainer + +/** + * Represents a step in configuring 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/k8s/submit/steps/initcontainer/InitContainerConfigurationStepsOrchestrator.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerConfigurationStepsOrchestrator.scala new file mode 100644 index 000000000000..76e72ec94737 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerConfigurationStepsOrchestrator.scala @@ -0,0 +1,83 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.k8s.submit.steps.initcontainer + +import org.apache.spark.{SparkConf, SparkException} +import org.apache.spark.deploy.k8s.{ConfigurationUtils, InitContainerBootstrapImpl, MountSecretsBootstrapImpl} +import org.apache.spark.deploy.k8s.Config._ +import org.apache.spark.deploy.k8s.Constants._ + +/** + * Returns the complete ordered list of steps required to configure the init-container. This is + * only used when there are remote application dependencies to localize. + */ +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 initContainerImage = submissionSparkConf + .get(INIT_CONTAINER_DOCKER_IMAGE) + .getOrElse(throw new SparkException( + "Must specify the init-container Docker image when there are remote dependencies")) + private val downloadTimeoutMinutes = submissionSparkConf.get(INIT_CONTAINER_MOUNT_TIMEOUT) + + def getAllConfigurationSteps(): Seq[InitContainerConfigurationStep] = { + val initContainerBootstrap = new InitContainerBootstrapImpl( + initContainerImage, + dockerImagePullPolicy, + jarsDownloadPath, + filesDownloadPath, + downloadTimeoutMinutes, + initContainerConfigMapName, + initContainerConfigMapKey, + SPARK_POD_DRIVER_ROLE, + submissionSparkConf) + val baseInitContainerStep = new BaseInitContainerConfigurationStep( + sparkJars, + sparkFiles, + jarsDownloadPath, + filesDownloadPath, + initContainerBootstrap) + + val driverSecretNamesToMountPaths = ConfigurationUtils.parsePrefixedKeyValuePairs( + submissionSparkConf, + KUBERNETES_DRIVER_SECRETS_PREFIX) + // Mount user-specified driver secrets also into the driver's init-container. The + // init-container may need credentials in the secrets to be able to download remote + // dependencies. The driver's main container and its init-container share the secrets + // because the init-container is sort of an implementation details and this sharing + // avoids introducing a dedicated configuration property just for the init-container. + val maybeMountSecretsStep = if (driverSecretNamesToMountPaths.nonEmpty) { + val mountSecretsBootstrap = new MountSecretsBootstrapImpl(driverSecretNamesToMountPaths) + Some(new InitContainerMountSecretsStep(mountSecretsBootstrap)) + } else { + None + } + + Seq(baseInitContainerStep) ++ + maybeMountSecretsStep.toSeq + } +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerMountSecretsStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerMountSecretsStep.scala new file mode 100644 index 000000000000..6eea1b5c6c93 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerMountSecretsStep.scala @@ -0,0 +1,40 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.k8s.submit.steps.initcontainer + +import org.apache.spark.deploy.k8s.MountSecretsBootstrap + +/** + * An init-container configuration step for mounting user-specified secrets onto user-specified + * paths. + * + * @param mountSecretsBootstrap a utility actually handling mounting of the secrets + */ +private[spark] class InitContainerMountSecretsStep( + mountSecretsBootstrap: MountSecretsBootstrap) extends InitContainerConfigurationStep { + + override def configureInitContainer(initContainerSpec: InitContainerSpec) : InitContainerSpec = { + val (podWithSecretsMounted, initContainerWithSecretsMounted) = + mountSecretsBootstrap.mountSecrets( + initContainerSpec.driverPod, + initContainerSpec.initContainer) + initContainerSpec.copy( + driverPod = podWithSecretsMounted, + initContainer = initContainerWithSecretsMounted + ) + } +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerSpec.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerSpec.scala new file mode 100644 index 000000000000..73fcf19c3936 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerSpec.scala @@ -0,0 +1,37 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.k8s.submit.steps.initcontainer + +import io.fabric8.kubernetes.api.model.{Container, HasMetadata, Pod} + +/** + * Represents a specification of the init-container for the driver pod. + * + * @param initContainerProperties properties that should be set on the init-container + * @param driverSparkConf Spark configuration properties that will be carried back to the driver + * @param initContainer the init-container object + * @param driverContainer the driver container object + * @param driverPod the driver pod object + * @param initContainerDependentResources resources the init-container depends on to work + */ +private[spark] case class InitContainerSpec( + initContainerProperties: Map[String, String], + driverSparkConf: Map[String, String], + initContainer: Container, + driverContainer: Container, + driverPod: Pod, + initContainerDependentResources: Seq[HasMetadata]) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/FileFetcher.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/FileFetcher.scala new file mode 100644 index 000000000000..76e5507f4653 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/FileFetcher.scala @@ -0,0 +1,27 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.rest.k8s + +import java.io.File + +/** + * Utility for fetching remote file dependencies. + */ +private[spark] trait FileFetcher { + + def fetchFile(uri: String, targetDir: File): Unit +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/KubernetesSparkDependencyDownloadInitContainer.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/KubernetesSparkDependencyDownloadInitContainer.scala new file mode 100644 index 000000000000..616e8446e8b0 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/KubernetesSparkDependencyDownloadInitContainer.scala @@ -0,0 +1,129 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.rest.k8s + +import java.io.File +import java.util.concurrent.TimeUnit + +import scala.concurrent.{ExecutionContext, Future} +import scala.concurrent.duration.Duration + +import org.apache.spark.{SecurityManager => SparkSecurityManager, SparkConf} +import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.deploy.k8s.Config._ +import org.apache.spark.internal.Logging +import org.apache.spark.util.{ThreadUtils, Utils} + +/** + * Process that fetches files from a resource staging server and/or arbitrary remote locations. + * + * The init-container can handle fetching files from any of those sources, but not all of the + * sources need to be specified. This allows for composing multiple instances of this container + * with different configurations for different download sources, or using the same container to + * download everything at once. + */ +private[spark] class KubernetesSparkDependencyDownloadInitContainer( + sparkConf: SparkConf, + fileFetcher: FileFetcher) extends Logging { + + private implicit val downloadExecutor = ExecutionContext.fromExecutorService( + ThreadUtils.newDaemonCachedThreadPool("download-executor")) + + private val jarsDownloadDir = new File( + sparkConf.get(JARS_DOWNLOAD_LOCATION)) + private val filesDownloadDir = new File( + sparkConf.get(FILES_DOWNLOAD_LOCATION)) + + private val remoteJars = sparkConf.get(INIT_CONTAINER_REMOTE_JARS) + private val remoteFiles = sparkConf.get(INIT_CONTAINER_REMOTE_FILES) + + private val downloadTimeoutMinutes = sparkConf.get(INIT_CONTAINER_MOUNT_TIMEOUT) + + def run(): Unit = { + val remoteJarsDownload = Future[Unit] { + logInfo(s"Downloading remote jars: $remoteJars") + downloadFiles( + remoteJars, + jarsDownloadDir, + s"Remote jars download directory specified at $jarsDownloadDir does not exist " + + s"or is not a directory.") + } + val remoteFilesDownload = Future[Unit] { + logInfo(s"Downloading remote files: $remoteFiles") + downloadFiles( + remoteFiles, + filesDownloadDir, + s"Remote files download directory specified at $filesDownloadDir does not exist " + + s"or is not a directory.") + } + waitForFutures( + remoteJarsDownload, + remoteFilesDownload) + } + + private def downloadFiles( + filesCommaSeparated: Option[String], + downloadDir: File, + errMessageOnDestinationNotADirectory: String): Unit = { + if (filesCommaSeparated.isDefined) { + require(downloadDir.isDirectory, errMessageOnDestinationNotADirectory) + } + filesCommaSeparated.map(_.split(",")).toSeq.flatten.foreach { file => + fileFetcher.fetchFile(file, downloadDir) + } + } + + private def waitForFutures(futures: Future[_]*) { + futures.foreach { + ThreadUtils.awaitResult(_, Duration.create(downloadTimeoutMinutes, TimeUnit.MINUTES)) + } + } +} + +private class FileFetcherImpl(sparkConf: SparkConf, securityManager: SparkSecurityManager) + extends FileFetcher { + + def fetchFile(uri: String, targetDir: File): Unit = { + Utils.fetchFile( + url = uri, + targetDir = targetDir, + conf = sparkConf, + securityMgr = securityManager, + hadoopConf = SparkHadoopUtil.get.newConfiguration(sparkConf), + timestamp = System.currentTimeMillis(), + useCache = false) + } +} + +object KubernetesSparkDependencyDownloadInitContainer extends Logging { + + def main(args: Array[String]): Unit = { + logInfo("Starting init-container to download Spark application dependencies.") + val sparkConf = if (args.nonEmpty) { + SparkConfPropertiesParser.getSparkConfFromPropertiesFile(new File(args(0))) + } else { + new SparkConf(true) + } + + val securityManager = new SparkSecurityManager(sparkConf) + val fileFetcher = new FileFetcherImpl(sparkConf, securityManager) + new KubernetesSparkDependencyDownloadInitContainer( + sparkConf, + fileFetcher).run() + logInfo("Finished downloading application dependencies.") + } +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/SparkConfPropertiesParser.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/SparkConfPropertiesParser.scala new file mode 100644 index 000000000000..176166c1fee0 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/SparkConfPropertiesParser.scala @@ -0,0 +1,50 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.rest.k8s + +import java.io.{File, FileInputStream} +import java.util.Properties + +import scala.collection.JavaConverters._ + +import com.google.common.collect.Maps + +import org.apache.spark.SparkConf +import org.apache.spark.internal.config.{ConfigReader, SparkConfigProvider} +import org.apache.spark.util.Utils + +private[spark] object SparkConfPropertiesParser { + + def getSparkConfFromPropertiesFile(propertiesFile: File): SparkConf = { + val sparkConf = new SparkConf(true) + + if (!propertiesFile.isFile) { + throw new IllegalArgumentException(s"Server properties file given at" + + s" ${propertiesFile.getAbsoluteFile} does not exist or is not a file.") + } + + val properties = new Properties + Utils.tryWithResource(new FileInputStream(propertiesFile))(properties.load) + val propertiesMap = Maps.fromProperties(properties) + val configReader = new ConfigReader(new SparkConfigProvider(propertiesMap)) + propertiesMap.asScala.keys.foreach { key => + configReader.get(key).foreach(sparkConf.set(key, _)) + } + + sparkConf + } +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala index 70226157dd68..846b5e3c7954 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala @@ -21,9 +21,10 @@ import scala.collection.JavaConverters._ import io.fabric8.kubernetes.api.model._ import org.apache.spark.{SparkConf, SparkException} +import org.apache.spark.deploy.k8s.{ConfigurationUtils, InitContainerBootstrap, MountSecretsBootstrap, PodWithDetachedInitContainer} import org.apache.spark.deploy.k8s.Config._ -import org.apache.spark.deploy.k8s.ConfigurationUtils import org.apache.spark.deploy.k8s.Constants._ +import org.apache.spark.deploy.k8s.submit.InitContainerUtil import org.apache.spark.internal.config.{EXECUTOR_CLASS_PATH, EXECUTOR_JAVA_OPTIONS, EXECUTOR_MEMORY, EXECUTOR_MEMORY_OVERHEAD} import org.apache.spark.util.Utils @@ -44,7 +45,11 @@ private[spark] trait ExecutorPodFactory { nodeToLocalTaskCount: Map[String, Int]): Pod } -private[spark] class ExecutorPodFactoryImpl(sparkConf: SparkConf) +private[spark] class ExecutorPodFactoryImpl( + sparkConf: SparkConf, + mountSecretsBootstrap: Option[MountSecretsBootstrap], + executorInitContainerBootstrap: Option[InitContainerBootstrap], + executorInitContainerMountSecretsBootstrap: Option[MountSecretsBootstrap]) extends ExecutorPodFactory { private val executorExtraClasspath = sparkConf.get(EXECUTOR_CLASS_PATH) @@ -209,9 +214,33 @@ private[spark] class ExecutorPodFactoryImpl(sparkConf: SparkConf) .build() }.getOrElse(executorContainer) - new PodBuilder(executorPod) + val (withMaybeSecretsMountedPod, withMaybeSecretsMountedContainer) = + mountSecretsBootstrap.map {bootstrap => + bootstrap.mountSecrets(executorPod, containerWithExecutorLimitCores) + }.getOrElse((executorPod, containerWithExecutorLimitCores)) + + val (executorPodWithInitContainer, initBootstrappedExecutorContainer) = + executorInitContainerBootstrap.map { bootstrap => + val podWithInitContainer = bootstrap.bootstrapInitContainer( + PodWithDetachedInitContainer( + withMaybeSecretsMountedPod, + new ContainerBuilder().build(), + withMaybeSecretsMountedContainer)) + + val (mayBePodWithSecretsMountedToInitContainer, mayBeInitContainerWithSecretsMounted) = + executorInitContainerMountSecretsBootstrap.map { bootstrap => + bootstrap.mountSecrets(podWithInitContainer.pod, podWithInitContainer.initContainer) + }.getOrElse((podWithInitContainer.pod, podWithInitContainer.initContainer)) + + val podWithAttachedInitContainer = InitContainerUtil.appendInitContainer( + mayBePodWithSecretsMountedToInitContainer, mayBeInitContainerWithSecretsMounted) + + (podWithAttachedInitContainer, podWithInitContainer.mainContainer) + }.getOrElse((withMaybeSecretsMountedPod, withMaybeSecretsMountedContainer)) + + new PodBuilder(executorPodWithInitContainer) .editSpec() - .addToContainers(containerWithExecutorLimitCores) + .addToContainers(initBootstrappedExecutorContainer) .endSpec() .build() } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala index b8bb152d1791..0bdd3a7e382b 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala @@ -21,9 +21,9 @@ import java.io.File import io.fabric8.kubernetes.client.Config import org.apache.spark.{SparkContext, SparkException} +import org.apache.spark.deploy.k8s.{ConfigurationUtils, InitContainerBootstrapImpl, MountSecretsBootstrapImpl, SparkKubernetesClientFactory} import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ -import org.apache.spark.deploy.k8s.SparkKubernetesClientFactory import org.apache.spark.internal.Logging import org.apache.spark.scheduler.{ExternalClusterManager, SchedulerBackend, TaskScheduler, TaskSchedulerImpl} import org.apache.spark.util.ThreadUtils @@ -45,6 +45,60 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit masterURL: String, scheduler: TaskScheduler): SchedulerBackend = { val sparkConf = sc.getConf + val maybeInitContainerConfigMap = sparkConf.get(INIT_CONTAINER_CONFIG_MAP_NAME) + val maybeInitContainerConfigMapKey = sparkConf.get(INIT_CONTAINER_CONFIG_MAP_KEY_CONF) + + if (maybeInitContainerConfigMap.isEmpty) { + logWarning("The executor's init-container config map was not specified. Executors will " + + "therefore not attempt to fetch remote or submitted dependencies.") + } + + if (maybeInitContainerConfigMapKey.isEmpty) { + logWarning("The executor's init-container config map key was not specified. Executors will " + + "therefore not attempt to fetch remote or submitted dependencies.") + } + + // Only set up the bootstrap if they've provided both the config map key and the config map + // name. The config map might not be provided if init-containers aren't being used to + // bootstrap dependencies. + val maybeInitContainerBootstrap = for { + configMap <- maybeInitContainerConfigMap + configMapKey <- maybeInitContainerConfigMapKey + } yield { + val initContainerImage = sparkConf + .get(INIT_CONTAINER_DOCKER_IMAGE) + .getOrElse(throw new SparkException( + "Must specify the init-container Docker image when there are remote dependencies")) + new InitContainerBootstrapImpl( + initContainerImage, + sparkConf.get(CONTAINER_IMAGE_PULL_POLICY), + sparkConf.get(JARS_DOWNLOAD_LOCATION), + sparkConf.get(FILES_DOWNLOAD_LOCATION), + sparkConf.get(INIT_CONTAINER_MOUNT_TIMEOUT), + configMap, + configMapKey, + SPARK_POD_EXECUTOR_ROLE, + sparkConf) + } + + val executorSecretNamesToMountPaths = ConfigurationUtils.parsePrefixedKeyValuePairs( + sparkConf, KUBERNETES_EXECUTOR_SECRETS_PREFIX) + val mayBeMountSecretBootstrap = if (executorSecretNamesToMountPaths.nonEmpty) { + Some(new MountSecretsBootstrapImpl(executorSecretNamesToMountPaths)) + } else { + None + } + // Mount user-specified executor secrets also into the executor's init-container. The + // init-container may need credentials in the secrets to be able to download remote + // dependencies. The executor's main container and its init-container share the secrets + // because the init-container is sort of an implementation details and this sharing + // avoids introducing a dedicated configuration property just for the init-container. + val mayBeInitContainerMountSecretsBootstrap = if (maybeInitContainerBootstrap.nonEmpty && + executorSecretNamesToMountPaths.nonEmpty) { + Some(new MountSecretsBootstrapImpl(executorSecretNamesToMountPaths)) + } else { + None + } val kubernetesClient = SparkKubernetesClientFactory.createKubernetesClient( KUBERNETES_MASTER_INTERNAL_URL, @@ -54,7 +108,12 @@ 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))) - val executorPodFactory = new ExecutorPodFactoryImpl(sparkConf) + val executorPodFactory = new ExecutorPodFactoryImpl( + sparkConf, + mayBeMountSecretBootstrap, + maybeInitContainerBootstrap, + mayBeInitContainerMountSecretsBootstrap) + val allocatorExecutor = ThreadUtils .newDaemonSingleThreadScheduledExecutor("kubernetes-pod-allocator") val requestExecutorsService = ThreadUtils.newDaemonCachedThreadPool( diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestratorSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestratorSuite.scala index 98f9f27da5cd..68efa3e29a6b 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestratorSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestratorSuite.scala @@ -17,18 +17,22 @@ package org.apache.spark.deploy.k8s.submit import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.deploy.k8s.Config.DRIVER_CONTAINER_IMAGE +import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.submit.steps._ class DriverConfigurationStepsOrchestratorSuite extends SparkFunSuite { private val NAMESPACE = "default" private val DRIVER_IMAGE = "driver-image" + private val INIT_CONTAINER_IMAGE = "init-container-image" 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 SECRET_FOO = "foo" + private val SECRET_BAR = "bar" + private val SECRET_MOUNT_PATH = "/etc/secrets/driver" test("Base submission steps with a main app resource.") { val sparkConf = new SparkConf(false) @@ -72,6 +76,54 @@ class DriverConfigurationStepsOrchestratorSuite extends SparkFunSuite { ) } + test("Submission steps with an init-container.") { + val sparkConf = new SparkConf(false) + .set(DRIVER_CONTAINER_IMAGE, DRIVER_IMAGE) + .set(INIT_CONTAINER_DOCKER_IMAGE, INIT_CONTAINER_IMAGE) + .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, + Some(mainAppResource), + APP_NAME, + MAIN_CLASS, + APP_ARGS, + sparkConf) + validateStepTypes( + orchestrator, + classOf[BaseDriverConfigurationStep], + classOf[DriverServiceBootstrapStep], + classOf[DriverKubernetesCredentialsStep], + classOf[DependencyResolutionStep], + classOf[DriverInitContainerBootstrapStep]) + } + + test("Submission steps with driver secrets to mount") { + val sparkConf = new SparkConf(false) + .set(DRIVER_CONTAINER_IMAGE, DRIVER_IMAGE) + .set(s"$KUBERNETES_DRIVER_SECRETS_PREFIX$SECRET_FOO", SECRET_MOUNT_PATH) + .set(s"$KUBERNETES_DRIVER_SECRETS_PREFIX$SECRET_BAR", SECRET_MOUNT_PATH) + val mainAppResource = JavaMainAppResource("local:///var/apps/jars/main.jar") + val orchestrator = new DriverConfigurationStepsOrchestrator( + NAMESPACE, + APP_ID, + LAUNCH_TIME, + Some(mainAppResource), + APP_NAME, + MAIN_CLASS, + APP_ARGS, + sparkConf) + validateStepTypes( + orchestrator, + classOf[BaseDriverConfigurationStep], + classOf[DriverServiceBootstrapStep], + classOf[DriverKubernetesCredentialsStep], + classOf[DependencyResolutionStep], + classOf[DriverMountSecretsStep]) + } + private def validateStepTypes( orchestrator: DriverConfigurationStepsOrchestrator, types: Class[_ <: DriverConfigurationStep]*): Unit = { diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/SecretVolumeUtils.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/SecretVolumeUtils.scala new file mode 100644 index 000000000000..8388c16ded26 --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/SecretVolumeUtils.scala @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.k8s.submit + +import scala.collection.JavaConverters._ + +import io.fabric8.kubernetes.api.model.{Container, Pod} + +private[spark] object SecretVolumeUtils { + + def podHasVolume(driverPod: Pod, volumeName: String): Boolean = { + driverPod.getSpec.getVolumes.asScala.exists(volume => volume.getName == volumeName) + } + + def containerHasVolume( + driverContainer: Container, + volumeName: String, + mountPath: String): Boolean = { + driverContainer.getVolumeMounts.asScala.exists(volumeMount => + volumeMount.getName == volumeName && volumeMount.getMountPath == mountPath) + } +} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/DriverInitContainerBootstrapStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/DriverInitContainerBootstrapStepSuite.scala new file mode 100644 index 000000000000..36461754772a --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/DriverInitContainerBootstrapStepSuite.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.k8s.submit.steps + +import java.io.StringReader +import java.util.Properties + +import scala.collection.JavaConverters._ + +import com.google.common.collect.Maps +import io.fabric8.kubernetes.api.model.{ConfigMap, ContainerBuilder, HasMetadata, PodBuilder, SecretBuilder} + +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.deploy.k8s.Config._ +import org.apache.spark.deploy.k8s.submit.KubernetesDriverSpec +import org.apache.spark.deploy.k8s.submit.steps.initcontainer.{InitContainerConfigurationStep, InitContainerSpec} +import org.apache.spark.util.Utils + +class DriverInitContainerBootstrapStepSuite extends SparkFunSuite { + + 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 DriverInitContainerBootstrapStep( + 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) + + 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 + }) + + val initContainers = preparedDriverSpec.driverPod.getSpec.getInitContainers + assert(initContainers.size() === 1) + val initContainerEnv = initContainers.get(0).getEnv.asScala + assert(initContainerEnv.size === 1) + assert(initContainerEnv.head.getName === + SecondTestInitContainerConfigurationStep$.additionalInitContainerEnvKey) + assert(initContainerEnv.head.getValue === + SecondTestInitContainerConfigurationStep$.additionalInitContainerEnvValue) + + val expectedSparkConf = Map( + INIT_CONTAINER_CONFIG_MAP_NAME.key -> CONFIG_MAP_NAME, + INIT_CONTAINER_CONFIG_MAP_KEY_CONF.key -> CONFIG_MAP_KEY, + SecondTestInitContainerConfigurationStep$.additionalDriverSparkConfKey -> + SecondTestInitContainerConfigurationStep$.additionalDriverSparkConfValue) + assert(preparedDriverSpec.driverSparkConf.getAll.toMap === expectedSparkConf) + } +} + +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.driverPod) + .editOrNewMetadata() + .addToLabels(additionalLabels.asJava) + .endMetadata() + .build() + val mainContainer = new ContainerBuilder(initContainerSpec.driverContainer) + .addNewEnv() + .withName(additionalMainContainerEnvKey) + .withValue(additionalMainContainerEnvValue) + .endEnv() + .build() + initContainerSpec.copy( + driverPod = 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.driverSparkConf ++ + Map(additionalDriverSparkConfKey -> additionalDriverSparkConfValue) + initContainerSpec.copy( + initContainer = initContainer, + initContainerProperties = initContainerProperties, + driverSparkConf = driverSparkConf) + } +} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/DriverMountSecretsStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/DriverMountSecretsStepSuite.scala new file mode 100644 index 000000000000..9a40bb859143 --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/DriverMountSecretsStepSuite.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.k8s.submit.steps + +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.deploy.k8s.MountSecretsBootstrapImpl +import org.apache.spark.deploy.k8s.submit.{KubernetesDriverSpec, SecretVolumeUtils} + +class DriverMountSecretsStepSuite extends SparkFunSuite { + + private val SECRET_FOO = "foo" + private val SECRET_BAR = "bar" + private val SECRET_MOUNT_PATH = "/etc/secrets/driver" + + test("Mounts all given secrets") { + val baseDriverSpec = KubernetesDriverSpec.initialSpec(new SparkConf(false)) + val secretNamesToMountPaths = Map( + SECRET_FOO -> SECRET_MOUNT_PATH, + SECRET_BAR -> SECRET_MOUNT_PATH) + + val mountSecretsBootstrap = new MountSecretsBootstrapImpl(secretNamesToMountPaths) + val mountSecretsStep = new DriverMountSecretsStep(mountSecretsBootstrap) + val configuredDriverSpec = mountSecretsStep.configureDriver(baseDriverSpec) + val driverPodWithSecretsMounted = configuredDriverSpec.driverPod + val driverContainerWithSecretsMounted = configuredDriverSpec.driverContainer + + Seq(s"$SECRET_FOO-volume", s"$SECRET_BAR-volume").foreach(volumeName => + assert(SecretVolumeUtils.podHasVolume(driverPodWithSecretsMounted, volumeName))) + Seq(s"$SECRET_FOO-volume", s"$SECRET_BAR-volume").foreach(volumeName => + assert(SecretVolumeUtils.containerHasVolume( + driverContainerWithSecretsMounted, volumeName, SECRET_MOUNT_PATH))) + } +} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/BaseInitContainerConfigurationStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/BaseInitContainerConfigurationStepSuite.scala new file mode 100644 index 000000000000..55a6a0624fd2 --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/BaseInitContainerConfigurationStepSuite.scala @@ -0,0 +1,97 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.k8s.submit.steps.initcontainer + +import scala.collection.JavaConverters._ + +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 org.apache.spark.SparkFunSuite +import org.apache.spark.deploy.k8s.{InitContainerBootstrap, PodWithDetachedInitContainer} +import org.apache.spark.deploy.k8s.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 : InitContainerBootstrap = _ + + before { + MockitoAnnotations.initMocks(this) + when(podAndInitContainerBootstrap.bootstrapInitContainer( + 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, + podAndInitContainerBootstrap) + val expectedDriverSparkConf = Map( + JARS_DOWNLOAD_LOCATION.key -> JARS_DOWNLOAD_PATH, + 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.driverPod.getMetadata.getLabels.asScala === POD_LABEL) + } +} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerConfigurationStepsOrchestratorSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerConfigurationStepsOrchestratorSuite.scala new file mode 100644 index 000000000000..5e177efdb640 --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerConfigurationStepsOrchestratorSuite.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.k8s.submit.steps.initcontainer + +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.deploy.k8s.Config._ +import org.apache.spark.deploy.k8s.Constants._ + +class InitContainerConfigurationStepsOrchestratorSuite extends SparkFunSuite { + + private val NAMESPACE = "namespace" + private val DOCKER_IMAGE = "init-container" + 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 SECRET_FOO = "foo" + private val SECRET_BAR = "bar" + private val SECRET_MOUNT_PATH = "/etc/secrets/init-container" + + test ("including basic configuration step") { + val sparkConf = new SparkConf(true) + .set(INIT_CONTAINER_DOCKER_IMAGE, DOCKER_IMAGE) + .set(s"$KUBERNETES_DRIVER_LABEL_PREFIX$CUSTOM_LABEL_KEY", CUSTOM_LABEL_VALUE) + + val orchestrator = new InitContainerConfigurationStepsOrchestrator( + NAMESPACE, + APP_RESOURCE_PREFIX, + SPARK_JARS.take(1), + 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]) + } + + test("including step to mount user-specified secrets") { + val sparkConf = new SparkConf(false) + .set(INIT_CONTAINER_DOCKER_IMAGE, DOCKER_IMAGE) + .set(s"$KUBERNETES_DRIVER_SECRETS_PREFIX$SECRET_FOO", SECRET_MOUNT_PATH) + .set(s"$KUBERNETES_DRIVER_SECRETS_PREFIX$SECRET_BAR", SECRET_MOUNT_PATH) + + val orchestrator = new InitContainerConfigurationStepsOrchestrator( + NAMESPACE, + APP_RESOURCE_PREFIX, + SPARK_JARS.take(1), + 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[InitContainerMountSecretsStep]) + } +} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerMountSecretsStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerMountSecretsStepSuite.scala new file mode 100644 index 000000000000..ee6c9c4745a4 --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerMountSecretsStepSuite.scala @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.k8s.submit.steps.initcontainer + +import io.fabric8.kubernetes.api.model.{ContainerBuilder, PodBuilder} + +import org.apache.spark.SparkFunSuite +import org.apache.spark.deploy.k8s.MountSecretsBootstrapImpl +import org.apache.spark.deploy.k8s.submit.SecretVolumeUtils + +class InitContainerMountSecretsStepSuite extends SparkFunSuite { + + private val SECRET_FOO = "foo" + private val SECRET_BAR = "bar" + private val SECRET_MOUNT_PATH = "/etc/secrets/init-container" + + test("Mounts all given secrets") { + val baseInitContainerSpec = InitContainerSpec( + Map.empty, + Map.empty, + new ContainerBuilder().build(), + new ContainerBuilder().build(), + new PodBuilder().withNewMetadata().endMetadata().withNewSpec().endSpec().build(), + Seq.empty) + val secretNamesToMountPaths = Map( + SECRET_FOO -> SECRET_MOUNT_PATH, + SECRET_BAR -> SECRET_MOUNT_PATH) + + val mountSecretsBootstrap = new MountSecretsBootstrapImpl(secretNamesToMountPaths) + val initContainerMountSecretsStep = new InitContainerMountSecretsStep(mountSecretsBootstrap) + val configuredInitContainerSpec = initContainerMountSecretsStep.configureInitContainer( + baseInitContainerSpec) + + val podWithSecretsMounted = configuredInitContainerSpec.driverPod + val initContainerWithSecretsMounted = configuredInitContainerSpec.initContainer + + Seq(s"$SECRET_FOO-volume", s"$SECRET_BAR-volume").foreach(volumeName => + assert(SecretVolumeUtils.podHasVolume(podWithSecretsMounted, volumeName))) + Seq(s"$SECRET_FOO-volume", s"$SECRET_BAR-volume").foreach(volumeName => + assert(SecretVolumeUtils.containerHasVolume( + initContainerWithSecretsMounted, volumeName, SECRET_MOUNT_PATH))) + } +} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/k8s/KubernetesSparkDependencyDownloadInitContainerSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/k8s/KubernetesSparkDependencyDownloadInitContainerSuite.scala new file mode 100644 index 000000000000..3f1997febe5e --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/k8s/KubernetesSparkDependencyDownloadInitContainerSuite.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.rest.k8s + +import java.io.File +import java.util.UUID + +import com.google.common.base.Charsets +import com.google.common.io.Files +import org.mockito.Mockito +import org.scalatest.BeforeAndAfter +import org.scalatest.mockito.MockitoSugar._ + +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.deploy.k8s.Config._ +import org.apache.spark.util.Utils + +class KubernetesSparkDependencyDownloadInitContainerSuite + extends SparkFunSuite with BeforeAndAfter { + + import KubernetesSparkDependencyDownloadInitContainerSuite.createTempFile + + private val DOWNLOAD_JARS_SECRET_LOCATION = createTempFile("txt") + private val DOWNLOAD_FILES_SECRET_LOCATION = createTempFile("txt") + + private var downloadJarsDir: File = _ + private var downloadFilesDir: File = _ + private var downloadJarsSecretValue: String = _ + private var downloadFilesSecretValue: String = _ + private var fileFetcher: FileFetcher = _ + + override def beforeAll(): Unit = { + downloadJarsSecretValue = Files.toString( + new File(DOWNLOAD_JARS_SECRET_LOCATION), Charsets.UTF_8) + downloadFilesSecretValue = Files.toString( + new File(DOWNLOAD_FILES_SECRET_LOCATION), Charsets.UTF_8) + } + + before { + downloadJarsDir = Utils.createTempDir() + downloadFilesDir = Utils.createTempDir() + fileFetcher = mock[FileFetcher] + } + + after { + downloadJarsDir.delete() + downloadFilesDir.delete() + } + + test("Downloads from remote server should invoke the file fetcher") { + val sparkConf = getSparkConfForRemoteFileDownloads + val initContainerUnderTest = new KubernetesSparkDependencyDownloadInitContainer( + sparkConf, + fileFetcher) + initContainerUnderTest.run() + Mockito.verify(fileFetcher).fetchFile("http://localhost:9000/jar1.jar", downloadJarsDir) + Mockito.verify(fileFetcher).fetchFile("hdfs://localhost:9000/jar2.jar", downloadJarsDir) + Mockito.verify(fileFetcher).fetchFile("http://localhost:9000/file.txt", downloadFilesDir) + + } + + private def getSparkConfForRemoteFileDownloads: SparkConf = { + new SparkConf(true) + .set(INIT_CONTAINER_REMOTE_JARS, + "http://localhost:9000/jar1.jar,hdfs://localhost:9000/jar2.jar") + .set(INIT_CONTAINER_REMOTE_FILES, + "http://localhost:9000/file.txt") + .set(JARS_DOWNLOAD_LOCATION, downloadJarsDir.getAbsolutePath) + .set(FILES_DOWNLOAD_LOCATION, downloadFilesDir.getAbsolutePath) + } +} + +private object KubernetesSparkDependencyDownloadInitContainerSuite { + 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/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactorySuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactorySuite.scala index 3a55d7cb37b1..96915ac7a930 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactorySuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactorySuite.scala @@ -18,15 +18,19 @@ package org.apache.spark.scheduler.cluster.k8s import scala.collection.JavaConverters._ -import io.fabric8.kubernetes.api.model.{Pod, _} -import org.mockito.MockitoAnnotations +import io.fabric8.kubernetes.api.model._ +import org.mockito.{AdditionalAnswers, MockitoAnnotations} +import org.mockito.Matchers.any +import org.mockito.Mockito._ import org.scalatest.{BeforeAndAfter, BeforeAndAfterEach} import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.deploy.k8s.{InitContainerBootstrap, MountSecretsBootstrapImpl, PodWithDetachedInitContainer} import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ class ExecutorPodFactorySuite extends SparkFunSuite with BeforeAndAfter with BeforeAndAfterEach { + private val driverPodName: String = "driver-pod" private val driverPodUid: String = "driver-uid" private val executorPrefix: String = "base" @@ -54,7 +58,7 @@ class ExecutorPodFactorySuite extends SparkFunSuite with BeforeAndAfter with Bef } test("basic executor pod has reasonable defaults") { - val factory = new ExecutorPodFactoryImpl(baseConf) + val factory = new ExecutorPodFactoryImpl(baseConf, None, None, None) val executor = factory.createExecutorPod( "1", "dummy", "dummy", Seq[(String, String)](), driverPod, Map[String, Int]()) @@ -85,7 +89,7 @@ class ExecutorPodFactorySuite extends SparkFunSuite with BeforeAndAfter with Bef conf.set(KUBERNETES_EXECUTOR_POD_NAME_PREFIX, "loremipsumdolorsitametvimatelitrefficiendisuscipianturvixlegeresple") - val factory = new ExecutorPodFactoryImpl(conf) + val factory = new ExecutorPodFactoryImpl(conf, None, None, None) val executor = factory.createExecutorPod( "1", "dummy", "dummy", Seq[(String, String)](), driverPod, Map[String, Int]()) @@ -97,7 +101,7 @@ class ExecutorPodFactorySuite extends SparkFunSuite with BeforeAndAfter with Bef conf.set(org.apache.spark.internal.config.EXECUTOR_JAVA_OPTIONS, "foo=bar") conf.set(org.apache.spark.internal.config.EXECUTOR_CLASS_PATH, "bar=baz") - val factory = new ExecutorPodFactoryImpl(conf) + val factory = new ExecutorPodFactoryImpl(conf, None, None, None) val executor = factory.createExecutorPod( "1", "dummy", "dummy", Seq[(String, String)]("qux" -> "quux"), driverPod, Map[String, Int]()) @@ -108,6 +112,74 @@ class ExecutorPodFactorySuite extends SparkFunSuite with BeforeAndAfter with Bef checkOwnerReferences(executor, driverPodUid) } + test("executor secrets get mounted") { + val conf = baseConf.clone() + + val secretsBootstrap = new MountSecretsBootstrapImpl(Map("secret1" -> "/var/secret1")) + val factory = new ExecutorPodFactoryImpl( + conf, + Some(secretsBootstrap), + None, + None) + val executor = factory.createExecutorPod( + "1", "dummy", "dummy", Seq[(String, String)](), driverPod, Map[String, Int]()) + + assert(executor.getSpec.getContainers.size() === 1) + assert(executor.getSpec.getContainers.get(0).getVolumeMounts.size() === 1) + assert(executor.getSpec.getContainers.get(0).getVolumeMounts.get(0).getName + === "secret1-volume") + assert(executor.getSpec.getContainers.get(0).getVolumeMounts.get(0) + .getMountPath === "/var/secret1") + + // check volume mounted. + assert(executor.getSpec.getVolumes.size() === 1) + assert(executor.getSpec.getVolumes.get(0).getSecret.getSecretName === "secret1") + + checkOwnerReferences(executor, driverPodUid) + } + + test("init-container bootstrap step adds an init container") { + val conf = baseConf.clone() + val initContainerBootstrap = mock(classOf[InitContainerBootstrap]) + when(initContainerBootstrap.bootstrapInitContainer( + any(classOf[PodWithDetachedInitContainer]))).thenAnswer(AdditionalAnswers.returnsFirstArg()) + + val factory = new ExecutorPodFactoryImpl( + conf, + None, + Some(initContainerBootstrap), + None) + val executor = factory.createExecutorPod( + "1", "dummy", "dummy", Seq[(String, String)](), driverPod, Map[String, Int]()) + + assert(executor.getSpec.getInitContainers.size() === 1) + checkOwnerReferences(executor, driverPodUid) + } + + test("init-container with secrets mount bootstrap") { + val conf = baseConf.clone() + val initContainerBootstrap = mock(classOf[InitContainerBootstrap]) + when(initContainerBootstrap.bootstrapInitContainer( + any(classOf[PodWithDetachedInitContainer]))).thenAnswer(AdditionalAnswers.returnsFirstArg()) + val secretsBootstrap = new MountSecretsBootstrapImpl(Map("secret1" -> "/var/secret1")) + + val factory = new ExecutorPodFactoryImpl( + conf, + None, + Some(initContainerBootstrap), + Some(secretsBootstrap)) + val executor = factory.createExecutorPod( + "1", "dummy", "dummy", Seq[(String, String)](), driverPod, Map[String, Int]()) + + assert(executor.getSpec.getInitContainers.size() === 1) + assert(executor.getSpec.getInitContainers.get(0).getVolumeMounts.get(0).getName + === "secret1-volume") + assert(executor.getSpec.getInitContainers.get(0).getVolumeMounts.get(0) + .getMountPath === "/var/secret1") + + checkOwnerReferences(executor, driverPodUid) + } + // There is always exactly one controller reference, and it points to the driver pod. private def checkOwnerReferences(executor: Pod, driverPodUid: String): Unit = { assert(executor.getMetadata.getOwnerReferences.size() === 1) diff --git a/resource-managers/kubernetes/docker/src/main/dockerfiles/driver/Dockerfile b/resource-managers/kubernetes/docker/src/main/dockerfiles/driver/Dockerfile index 9b682f8673c6..db32ffed2a37 100644 --- a/resource-managers/kubernetes/docker/src/main/dockerfiles/driver/Dockerfile +++ b/resource-managers/kubernetes/docker/src/main/dockerfiles/driver/Dockerfile @@ -31,4 +31,5 @@ CMD SPARK_CLASSPATH="${SPARK_HOME}/jars/*" && \ readarray -t SPARK_DRIVER_JAVA_OPTS < /tmp/java_opts.txt && \ if ! [ -z ${SPARK_MOUNTED_CLASSPATH+x} ]; then SPARK_CLASSPATH="$SPARK_MOUNTED_CLASSPATH:$SPARK_CLASSPATH"; fi && \ if ! [ -z ${SPARK_SUBMIT_EXTRA_CLASSPATH+x} ]; then SPARK_CLASSPATH="$SPARK_SUBMIT_EXTRA_CLASSPATH:$SPARK_CLASSPATH"; fi && \ + if ! [ -z ${SPARK_MOUNTED_FILES_DIR+x} ]; then cp -R "$SPARK_MOUNTED_FILES_DIR/." .; fi && \ ${JAVA_HOME}/bin/java "${SPARK_DRIVER_JAVA_OPTS[@]}" -cp "$SPARK_CLASSPATH" -Xms$SPARK_DRIVER_MEMORY -Xmx$SPARK_DRIVER_MEMORY -Dspark.driver.bindAddress=$SPARK_DRIVER_BIND_ADDRESS $SPARK_DRIVER_CLASS $SPARK_DRIVER_ARGS diff --git a/resource-managers/kubernetes/docker/src/main/dockerfiles/executor/Dockerfile b/resource-managers/kubernetes/docker/src/main/dockerfiles/executor/Dockerfile index 168cd4cb6c57..c25f4b82978a 100644 --- a/resource-managers/kubernetes/docker/src/main/dockerfiles/executor/Dockerfile +++ b/resource-managers/kubernetes/docker/src/main/dockerfiles/executor/Dockerfile @@ -31,4 +31,5 @@ CMD SPARK_CLASSPATH="${SPARK_HOME}/jars/*" && \ readarray -t SPARK_EXECUTOR_JAVA_OPTS < /tmp/java_opts.txt && \ if ! [ -z ${SPARK_MOUNTED_CLASSPATH}+x} ]; then SPARK_CLASSPATH="$SPARK_MOUNTED_CLASSPATH:$SPARK_CLASSPATH"; fi && \ if ! [ -z ${SPARK_EXECUTOR_EXTRA_CLASSPATH+x} ]; then SPARK_CLASSPATH="$SPARK_EXECUTOR_EXTRA_CLASSPATH:$SPARK_CLASSPATH"; fi && \ + if ! [ -z ${SPARK_MOUNTED_FILES_DIR+x} ]; then cp -R "$SPARK_MOUNTED_FILES_DIR/." .; fi && \ ${JAVA_HOME}/bin/java "${SPARK_EXECUTOR_JAVA_OPTS[@]}" -Xms$SPARK_EXECUTOR_MEMORY -Xmx$SPARK_EXECUTOR_MEMORY -cp "$SPARK_CLASSPATH" org.apache.spark.executor.CoarseGrainedExecutorBackend --driver-url $SPARK_DRIVER_URL --executor-id $SPARK_EXECUTOR_ID --cores $SPARK_EXECUTOR_CORES --app-id $SPARK_APPLICATION_ID --hostname $SPARK_EXECUTOR_POD_IP diff --git a/resource-managers/kubernetes/docker/src/main/dockerfiles/init-container/Dockerfile b/resource-managers/kubernetes/docker/src/main/dockerfiles/init-container/Dockerfile new file mode 100644 index 000000000000..a52f5e130559 --- /dev/null +++ b/resource-managers/kubernetes/docker/src/main/dockerfiles/init-container/Dockerfile @@ -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. +# + +FROM spark-base + +# If this docker file is being used in the context of building your images from a Spark distribution, the docker build +# command should be invoked from the top level directory of the Spark distribution. E.g.: +# docker build -t spark-init:latest -f dockerfiles/init-container/Dockerfile . + +ENTRYPOINT [ "/opt/entrypoint.sh", "/opt/spark/bin/spark-class", "org.apache.spark.deploy.rest.k8s.KubernetesSparkDependencyDownloadInitContainer" ] From 5d2cbc8c142a25febab8f475ff0eca5db2919403 Mon Sep 17 00:00:00 2001 From: Yinan Li Date: Fri, 15 Dec 2017 08:46:05 -0800 Subject: [PATCH 02/14] Addressed first round of comments --- .../org/apache/spark/deploy/k8s/InitContainerBootstrap.scala | 3 +-- .../k8s/KubernetesSparkDependencyDownloadInitContainer.scala | 4 ++-- .../spark/scheduler/cluster/k8s/ExecutorPodFactory.scala | 2 +- 3 files changed, 4 insertions(+), 5 deletions(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/InitContainerBootstrap.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/InitContainerBootstrap.scala index bae699b1e118..1b67389cfd56 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/InitContainerBootstrap.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/InitContainerBootstrap.scala @@ -75,7 +75,7 @@ private[spark] class InitContainerBootstrapImpl( .build()) val initContainer = new ContainerBuilder(podWithDetachedInitContainer.initContainer) - .withName(s"spark-init") + .withName("spark-init") .withImage(initContainerImage) .withImagePullPolicy(dockerImagePullPolicy) .addAllToEnv(initContainerCustomEnvVars.asJava) @@ -124,5 +124,4 @@ private[spark] class InitContainerBootstrapImpl( initContainer, mainContainerWithMountedFiles) } - } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/KubernetesSparkDependencyDownloadInitContainer.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/KubernetesSparkDependencyDownloadInitContainer.scala index 616e8446e8b0..3c7e4609ecbc 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/KubernetesSparkDependencyDownloadInitContainer.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/KubernetesSparkDependencyDownloadInitContainer.scala @@ -60,7 +60,7 @@ private[spark] class KubernetesSparkDependencyDownloadInitContainer( remoteJars, jarsDownloadDir, s"Remote jars download directory specified at $jarsDownloadDir does not exist " + - s"or is not a directory.") + "or is not a directory.") } val remoteFilesDownload = Future[Unit] { logInfo(s"Downloading remote files: $remoteFiles") @@ -68,7 +68,7 @@ private[spark] class KubernetesSparkDependencyDownloadInitContainer( remoteFiles, filesDownloadDir, s"Remote files download directory specified at $filesDownloadDir does not exist " + - s"or is not a directory.") + "or is not a directory.") } waitForFutures( remoteJarsDownload, diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala index 846b5e3c7954..32a02ddd4107 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala @@ -215,7 +215,7 @@ private[spark] class ExecutorPodFactoryImpl( }.getOrElse(executorContainer) val (withMaybeSecretsMountedPod, withMaybeSecretsMountedContainer) = - mountSecretsBootstrap.map {bootstrap => + mountSecretsBootstrap.map { bootstrap => bootstrap.mountSecrets(executorPod, containerWithExecutorLimitCores) }.getOrElse((executorPod, containerWithExecutorLimitCores)) From 4ee76afa3500a4315d3a62fb911f219c4f3a7cd7 Mon Sep 17 00:00:00 2001 From: Yinan Li Date: Fri, 15 Dec 2017 16:59:48 -0800 Subject: [PATCH 03/14] Addressed the second round of comments --- .../org/apache/spark/deploy/k8s/Config.scala | 6 ++--- .../deploy/k8s/InitContainerBootstrap.scala | 5 ++-- .../deploy/k8s/MountSecretsBootstrap.scala | 9 ++++--- ...r.scala => DriverConfigOrchestrator.scala} | 27 ++++++++++++++++--- .../submit/KubernetesClientApplication.scala | 2 +- .../k8s/submit/KubernetesFileUtils.scala | 10 +++---- .../DriverInitContainerBootstrapStep.scala | 3 ++- .../BaseInitContainerConfigurationStep.scala | 6 ++--- ... => InitContainerConfigOrchestrator.scala} | 6 ++--- .../spark/deploy/rest/k8s/FileFetcher.scala | 27 ------------------- ...iner.scala => SparkPodInitContainer.scala} | 26 +++++++----------- .../k8s/KubernetesClusterManager.scala | 4 +-- ...la => DriverConfigOrchestratorSuite.scala} | 22 ++++++++++----- ...nitContainerConfigOrchestratorSuite.scala} | 10 +++---- ...scala => SparkPodInitContainerSuite.scala} | 10 +++---- .../dockerfiles/init-container/Dockerfile | 2 +- 16 files changed, 85 insertions(+), 90 deletions(-) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/{DriverConfigurationStepsOrchestrator.scala => DriverConfigOrchestrator.scala} (85%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/{InitContainerConfigurationStepsOrchestrator.scala => InitContainerConfigOrchestrator.scala} (94%) delete mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/FileFetcher.scala rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/{KubernetesSparkDependencyDownloadInitContainer.scala => SparkPodInitContainer.scala} (87%) rename resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/{DriverConfigurationStepsOrchestratorSuite.scala => DriverConfigOrchestratorSuite.scala} (78%) rename resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/{InitContainerConfigurationStepsOrchestratorSuite.scala => InitContainerConfigOrchestratorSuite.scala} (92%) rename resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/k8s/{KubernetesSparkDependencyDownloadInitContainerSuite.scala => SparkPodInitContainerSuite.scala} (90%) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala index 52a7b2eb672e..c34028a2280d 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala @@ -145,9 +145,9 @@ private[spark] object Config extends Logging { .stringConf .createWithDefault("/var/spark-data/spark-files") - val INIT_CONTAINER_DOCKER_IMAGE = - ConfigBuilder("spark.kubernetes.initContainer.docker.image") - .doc("Image for the driver and executor's init-container that downloads dependencies.") + val INIT_CONTAINER_IMAGE = + ConfigBuilder("spark.kubernetes.initContainer.image") + .doc("Image for the driver and executor's init-container for downloading dependencies.") .stringConf .createOptional diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/InitContainerBootstrap.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/InitContainerBootstrap.scala index 1b67389cfd56..60adb05821ac 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/InitContainerBootstrap.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/InitContainerBootstrap.scala @@ -68,11 +68,12 @@ private[spark] class InitContainerBootstrapImpl( } val initContainerCustomEnvVars = sparkConf.getAllWithPrefix(initContainerCustomEnvVarKeyPrefix) .toSeq - .map(env => + .map { env => new EnvVarBuilder() .withName(env._1) .withValue(env._2) - .build()) + .build() + } val initContainer = new ContainerBuilder(podWithDetachedInitContainer.initContainer) .withName("spark-init") diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/MountSecretsBootstrap.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/MountSecretsBootstrap.scala index 9a0326aab797..a1f159ac1031 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/MountSecretsBootstrap.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/MountSecretsBootstrap.scala @@ -38,7 +38,7 @@ private[spark] class MountSecretsBootstrapImpl( override def mountSecrets(pod: Pod, container: Container): (Pod, Container) = { var podBuilder = new PodBuilder(pod) - secretNamesToMountPaths.keys.foreach(name => + secretNamesToMountPaths.keys.foreach { name => podBuilder = podBuilder .editOrNewSpec() .addNewVolume() @@ -47,16 +47,17 @@ private[spark] class MountSecretsBootstrapImpl( .withSecretName(name) .endSecret() .endVolume() - .endSpec()) + .endSpec() + } var containerBuilder = new ContainerBuilder(container) - secretNamesToMountPaths.foreach(namePath => + secretNamesToMountPaths.foreach { namePath => containerBuilder = containerBuilder .addNewVolumeMount() .withName(secretVolumeName(namePath._1)) .withMountPath(namePath._2) .endVolumeMount() - ) + } (podBuilder.build(), containerBuilder.build()) } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestrator.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/DriverConfigOrchestrator.scala similarity index 85% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestrator.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/DriverConfigOrchestrator.scala index 9a0c3689b76c..5d150996bffe 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestrator.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/DriverConfigOrchestrator.scala @@ -25,7 +25,7 @@ import org.apache.spark.deploy.k8s.{ConfigurationUtils, MountSecretsBootstrapImp import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.deploy.k8s.submit.steps._ -import org.apache.spark.deploy.k8s.submit.steps.initcontainer.InitContainerConfigurationStepsOrchestrator +import org.apache.spark.deploy.k8s.submit.steps.initcontainer.InitContainerConfigOrchestrator import org.apache.spark.launcher.SparkLauncher import org.apache.spark.util.SystemClock import org.apache.spark.util.Utils @@ -33,7 +33,7 @@ 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( +private[spark] class DriverConfigOrchestrator( namespace: String, kubernetesAppId: String, launchTime: Long, @@ -125,6 +125,7 @@ private[spark] class DriverConfigurationStepsOrchestrator( val mayBeInitContainerBootstrapStep = if (areAnyFilesNonContainerLocal(sparkJars ++ sparkFiles)) { +<<<<<<< HEAD:resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestrator.scala val initContainerConfigurationStepsOrchestrator = new InitContainerConfigurationStepsOrchestrator( namespace, @@ -147,6 +148,26 @@ private[spark] class DriverConfigurationStepsOrchestrator( INIT_CONTAINER_PROPERTIES_FILE_NAME) Some(initContainerBootstrapStep) +======= + val orchestrator = new InitContainerConfigOrchestrator( + namespace, + kubernetesResourceNamePrefix, + sparkJars, + sparkFiles, + jarsDownloadPath, + filesDownloadPath, + dockerImagePullPolicy, + allDriverLabels, + initContainerConfigMapName, + INIT_CONTAINER_PROPERTIES_FILE_NAME, + submissionSparkConf) + val bootstrapStep = new DriverInitContainerBootstrapStep( + orchestrator.getAllConfigurationSteps(), + initContainerConfigMapName, + INIT_CONTAINER_PROPERTIES_FILE_NAME) + + Some(bootstrapStep) +>>>>>>> Addressed the second round of comments:resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/DriverConfigOrchestrator.scala } else { None } @@ -169,7 +190,7 @@ private[spark] class DriverConfigurationStepsOrchestrator( private def areAnyFilesNonContainerLocal(files: Seq[String]): Boolean = { files.exists { uri => - Option(Utils.resolveURI(uri).getScheme).getOrElse("file") != "local" + Utils.resolveURI(uri).getScheme != "local" } } } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesClientApplication.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesClientApplication.scala index 240a1144577b..0bbdaa0933f3 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesClientApplication.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesClientApplication.scala @@ -210,7 +210,7 @@ private[spark] class KubernetesClientApplication extends SparkApplication { val loggingPodStatusWatcher = new LoggingPodStatusWatcherImpl( kubernetesAppId, loggingInterval) - val configurationStepsOrchestrator = new DriverConfigurationStepsOrchestrator( + val configurationStepsOrchestrator = new DriverConfigOrchestrator( namespace, kubernetesAppId, launchTime, diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesFileUtils.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesFileUtils.scala index 3bbd1d48e298..8a467b9384dd 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesFileUtils.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesFileUtils.scala @@ -51,12 +51,10 @@ private[spark] object KubernetesFileUtils { * Get from a given collection of file URIs the ones that represent remote files. */ def getOnlyRemoteFiles(uris: Iterable[String]): Iterable[String] = { - filterUriStringsByScheme(uris, scheme => scheme != "file" && scheme != "local") - } - - private def filterUriStringsByScheme( - uris: Iterable[String], schemeFilter: (String => Boolean)): Iterable[String] = { - uris.filter(uri => schemeFilter(Option(Utils.resolveURI(uri).getScheme).getOrElse("file"))) + uris.filter { uri => + val scheme = Utils.resolveURI(uri).getScheme + scheme != "file" && scheme != "local" + } } private def resolveFileUri( diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DriverInitContainerBootstrapStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DriverInitContainerBootstrapStep.scala index 4b8fd0234ec7..98a132723326 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DriverInitContainerBootstrapStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DriverInitContainerBootstrapStep.scala @@ -76,7 +76,8 @@ private[spark] class DriverInitContainerBootstrapStep( config: Map[String, String]): ConfigMap = { val properties = new Properties() config.foreach { entry => - properties.setProperty(entry._1, entry._2) } + properties.setProperty(entry._1, entry._2) + } val propertiesWriter = new StringWriter() properties.store(propertiesWriter, s"Java properties built from Kubernetes config map with name: $configMapName " + diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/BaseInitContainerConfigurationStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/BaseInitContainerConfigurationStep.scala index f4e740b0caa1..5c2108777c06 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/BaseInitContainerConfigurationStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/BaseInitContainerConfigurationStep.scala @@ -34,15 +34,15 @@ private[spark] class BaseInitContainerConfigurationStep( val remoteJarsConf = if (remoteJarsToDownload.nonEmpty) { Map(INIT_CONTAINER_REMOTE_JARS.key -> remoteJarsToDownload.mkString(",")) } else { - Map.empty[String, String] + Map() } val remoteFilesConf = if (remoteFilesToDownload.nonEmpty) { Map(INIT_CONTAINER_REMOTE_FILES.key -> remoteFilesToDownload.mkString(",")) } else { - Map.empty[String, String] + Map() } - val baseInitContainerConfig = Map[String, String]( + val baseInitContainerConfig = Map( JARS_DOWNLOAD_LOCATION.key -> jarsDownloadPath, FILES_DOWNLOAD_LOCATION.key -> filesDownloadPath) ++ remoteJarsConf ++ diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerConfigurationStepsOrchestrator.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerConfigOrchestrator.scala similarity index 94% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerConfigurationStepsOrchestrator.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerConfigOrchestrator.scala index 76e72ec94737..f90b46b38277 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerConfigurationStepsOrchestrator.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerConfigOrchestrator.scala @@ -25,7 +25,7 @@ import org.apache.spark.deploy.k8s.Constants._ * Returns the complete ordered list of steps required to configure the init-container. This is * only used when there are remote application dependencies to localize. */ -private[spark] class InitContainerConfigurationStepsOrchestrator( +private[spark] class InitContainerConfigOrchestrator( namespace: String, kubernetesResourceNamePrefix: String, sparkJars: Seq[String], @@ -39,9 +39,9 @@ private[spark] class InitContainerConfigurationStepsOrchestrator( submissionSparkConf: SparkConf) { private val initContainerImage = submissionSparkConf - .get(INIT_CONTAINER_DOCKER_IMAGE) + .get(INIT_CONTAINER_IMAGE) .getOrElse(throw new SparkException( - "Must specify the init-container Docker image when there are remote dependencies")) + "Must specify the init-container image when there are remote dependencies")) private val downloadTimeoutMinutes = submissionSparkConf.get(INIT_CONTAINER_MOUNT_TIMEOUT) def getAllConfigurationSteps(): Seq[InitContainerConfigurationStep] = { diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/FileFetcher.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/FileFetcher.scala deleted file mode 100644 index 76e5507f4653..000000000000 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/FileFetcher.scala +++ /dev/null @@ -1,27 +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.rest.k8s - -import java.io.File - -/** - * Utility for fetching remote file dependencies. - */ -private[spark] trait FileFetcher { - - def fetchFile(uri: String, targetDir: File): Unit -} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/KubernetesSparkDependencyDownloadInitContainer.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/SparkPodInitContainer.scala similarity index 87% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/KubernetesSparkDependencyDownloadInitContainer.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/SparkPodInitContainer.scala index 3c7e4609ecbc..1142b78d409c 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/KubernetesSparkDependencyDownloadInitContainer.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/SparkPodInitContainer.scala @@ -36,7 +36,7 @@ import org.apache.spark.util.{ThreadUtils, Utils} * with different configurations for different download sources, or using the same container to * download everything at once. */ -private[spark] class KubernetesSparkDependencyDownloadInitContainer( +private[spark] class SparkPodInitContainer( sparkConf: SparkConf, fileFetcher: FileFetcher) extends Logging { @@ -70,9 +70,10 @@ private[spark] class KubernetesSparkDependencyDownloadInitContainer( s"Remote files download directory specified at $filesDownloadDir does not exist " + "or is not a directory.") } - waitForFutures( - remoteJarsDownload, - remoteFilesDownload) + + Seq(remoteJarsDownload, remoteFilesDownload).foreach { + ThreadUtils.awaitResult(_, Duration.create(downloadTimeoutMinutes, TimeUnit.MINUTES)) + } } private def downloadFiles( @@ -86,16 +87,9 @@ private[spark] class KubernetesSparkDependencyDownloadInitContainer( fileFetcher.fetchFile(file, downloadDir) } } - - private def waitForFutures(futures: Future[_]*) { - futures.foreach { - ThreadUtils.awaitResult(_, Duration.create(downloadTimeoutMinutes, TimeUnit.MINUTES)) - } - } } -private class FileFetcherImpl(sparkConf: SparkConf, securityManager: SparkSecurityManager) - extends FileFetcher { +private class FileFetcher(sparkConf: SparkConf, securityManager: SparkSecurityManager) { def fetchFile(uri: String, targetDir: File): Unit = { Utils.fetchFile( @@ -109,7 +103,7 @@ private class FileFetcherImpl(sparkConf: SparkConf, securityManager: SparkSecuri } } -object KubernetesSparkDependencyDownloadInitContainer extends Logging { +object SparkPodInitContainer extends Logging { def main(args: Array[String]): Unit = { logInfo("Starting init-container to download Spark application dependencies.") @@ -120,10 +114,8 @@ object KubernetesSparkDependencyDownloadInitContainer extends Logging { } val securityManager = new SparkSecurityManager(sparkConf) - val fileFetcher = new FileFetcherImpl(sparkConf, securityManager) - new KubernetesSparkDependencyDownloadInitContainer( - sparkConf, - fileFetcher).run() + val fileFetcher = new FileFetcher(sparkConf, securityManager) + new SparkPodInitContainer(sparkConf, fileFetcher).run() logInfo("Finished downloading application dependencies.") } } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala index 0bdd3a7e382b..db4b38ecb95b 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala @@ -66,9 +66,9 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit configMapKey <- maybeInitContainerConfigMapKey } yield { val initContainerImage = sparkConf - .get(INIT_CONTAINER_DOCKER_IMAGE) + .get(INIT_CONTAINER_IMAGE) .getOrElse(throw new SparkException( - "Must specify the init-container Docker image when there are remote dependencies")) + "Must specify the init-container image when there are remote dependencies")) new InitContainerBootstrapImpl( initContainerImage, sparkConf.get(CONTAINER_IMAGE_PULL_POLICY), diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestratorSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/DriverConfigOrchestratorSuite.scala similarity index 78% rename from resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestratorSuite.scala rename to resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/DriverConfigOrchestratorSuite.scala index 68efa3e29a6b..6642335d12c9 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestratorSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/DriverConfigOrchestratorSuite.scala @@ -20,7 +20,7 @@ import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.submit.steps._ -class DriverConfigurationStepsOrchestratorSuite extends SparkFunSuite { +class DriverConfigOrchestratorSuite extends SparkFunSuite { private val NAMESPACE = "default" private val DRIVER_IMAGE = "driver-image" @@ -38,7 +38,7 @@ class DriverConfigurationStepsOrchestratorSuite extends SparkFunSuite { val sparkConf = new SparkConf(false) .set(DRIVER_CONTAINER_IMAGE, DRIVER_IMAGE) val mainAppResource = JavaMainAppResource("local:///var/apps/jars/main.jar") - val orchestrator = new DriverConfigurationStepsOrchestrator( + val orchestrator = new DriverConfigOrchestrator( NAMESPACE, APP_ID, LAUNCH_TIME, @@ -58,8 +58,13 @@ class DriverConfigurationStepsOrchestratorSuite extends SparkFunSuite { test("Base submission steps without a main app resource.") { val sparkConf = new SparkConf(false) +<<<<<<< HEAD:resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestratorSuite.scala .set(DRIVER_CONTAINER_IMAGE, DRIVER_IMAGE) val orchestrator = new DriverConfigurationStepsOrchestrator( +======= + .set(DRIVER_DOCKER_IMAGE, DRIVER_IMAGE) + val orchestrator = new DriverConfigOrchestrator( +>>>>>>> Addressed the second round of comments:resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/DriverConfigOrchestratorSuite.scala NAMESPACE, APP_ID, LAUNCH_TIME, @@ -78,11 +83,16 @@ class DriverConfigurationStepsOrchestratorSuite extends SparkFunSuite { test("Submission steps with an init-container.") { val sparkConf = new SparkConf(false) +<<<<<<< HEAD:resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestratorSuite.scala .set(DRIVER_CONTAINER_IMAGE, DRIVER_IMAGE) .set(INIT_CONTAINER_DOCKER_IMAGE, INIT_CONTAINER_IMAGE) +======= + .set(DRIVER_DOCKER_IMAGE, DRIVER_IMAGE) + .set(INIT_CONTAINER_IMAGE, INIT_CONTAINER_IMAGE) +>>>>>>> Addressed the second round of comments:resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/DriverConfigOrchestratorSuite.scala .set("spark.jars", "hdfs://localhost:9000/var/apps/jars/jar1.jar") val mainAppResource = JavaMainAppResource("local:///var/apps/jars/main.jar") - val orchestrator = new DriverConfigurationStepsOrchestrator( + val orchestrator = new DriverConfigOrchestrator( NAMESPACE, APP_ID, LAUNCH_TIME, @@ -106,7 +116,7 @@ class DriverConfigurationStepsOrchestratorSuite extends SparkFunSuite { .set(s"$KUBERNETES_DRIVER_SECRETS_PREFIX$SECRET_FOO", SECRET_MOUNT_PATH) .set(s"$KUBERNETES_DRIVER_SECRETS_PREFIX$SECRET_BAR", SECRET_MOUNT_PATH) val mainAppResource = JavaMainAppResource("local:///var/apps/jars/main.jar") - val orchestrator = new DriverConfigurationStepsOrchestrator( + val orchestrator = new DriverConfigOrchestrator( NAMESPACE, APP_ID, LAUNCH_TIME, @@ -125,8 +135,8 @@ class DriverConfigurationStepsOrchestratorSuite extends SparkFunSuite { } private def validateStepTypes( - orchestrator: DriverConfigurationStepsOrchestrator, - types: Class[_ <: DriverConfigurationStep]*): Unit = { + orchestrator: DriverConfigOrchestrator, + types: Class[_ <: DriverConfigurationStep]*): Unit = { val steps = orchestrator.getAllConfigurationSteps() assert(steps.size === types.size) assert(steps.map(_.getClass) === types) diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerConfigurationStepsOrchestratorSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerConfigOrchestratorSuite.scala similarity index 92% rename from resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerConfigurationStepsOrchestratorSuite.scala rename to resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerConfigOrchestratorSuite.scala index 5e177efdb640..68302d256cbe 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerConfigurationStepsOrchestratorSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerConfigOrchestratorSuite.scala @@ -20,7 +20,7 @@ import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ -class InitContainerConfigurationStepsOrchestratorSuite extends SparkFunSuite { +class InitContainerConfigOrchestratorSuite extends SparkFunSuite { private val NAMESPACE = "namespace" private val DOCKER_IMAGE = "init-container" @@ -50,10 +50,10 @@ class InitContainerConfigurationStepsOrchestratorSuite extends SparkFunSuite { test ("including basic configuration step") { val sparkConf = new SparkConf(true) - .set(INIT_CONTAINER_DOCKER_IMAGE, DOCKER_IMAGE) + .set(INIT_CONTAINER_IMAGE, DOCKER_IMAGE) .set(s"$KUBERNETES_DRIVER_LABEL_PREFIX$CUSTOM_LABEL_KEY", CUSTOM_LABEL_VALUE) - val orchestrator = new InitContainerConfigurationStepsOrchestrator( + val orchestrator = new InitContainerConfigOrchestrator( NAMESPACE, APP_RESOURCE_PREFIX, SPARK_JARS.take(1), @@ -73,11 +73,11 @@ class InitContainerConfigurationStepsOrchestratorSuite extends SparkFunSuite { test("including step to mount user-specified secrets") { val sparkConf = new SparkConf(false) - .set(INIT_CONTAINER_DOCKER_IMAGE, DOCKER_IMAGE) + .set(INIT_CONTAINER_IMAGE, DOCKER_IMAGE) .set(s"$KUBERNETES_DRIVER_SECRETS_PREFIX$SECRET_FOO", SECRET_MOUNT_PATH) .set(s"$KUBERNETES_DRIVER_SECRETS_PREFIX$SECRET_BAR", SECRET_MOUNT_PATH) - val orchestrator = new InitContainerConfigurationStepsOrchestrator( + val orchestrator = new InitContainerConfigOrchestrator( NAMESPACE, APP_RESOURCE_PREFIX, SPARK_JARS.take(1), diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/k8s/KubernetesSparkDependencyDownloadInitContainerSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/k8s/SparkPodInitContainerSuite.scala similarity index 90% rename from resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/k8s/KubernetesSparkDependencyDownloadInitContainerSuite.scala rename to resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/k8s/SparkPodInitContainerSuite.scala index 3f1997febe5e..15956cf71e89 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/k8s/KubernetesSparkDependencyDownloadInitContainerSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/k8s/SparkPodInitContainerSuite.scala @@ -29,10 +29,10 @@ import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.util.Utils -class KubernetesSparkDependencyDownloadInitContainerSuite +class SparkPodInitContainerSuite extends SparkFunSuite with BeforeAndAfter { - import KubernetesSparkDependencyDownloadInitContainerSuite.createTempFile + import SparkPodInitContainerSuite.createTempFile private val DOWNLOAD_JARS_SECRET_LOCATION = createTempFile("txt") private val DOWNLOAD_FILES_SECRET_LOCATION = createTempFile("txt") @@ -63,9 +63,7 @@ class KubernetesSparkDependencyDownloadInitContainerSuite test("Downloads from remote server should invoke the file fetcher") { val sparkConf = getSparkConfForRemoteFileDownloads - val initContainerUnderTest = new KubernetesSparkDependencyDownloadInitContainer( - sparkConf, - fileFetcher) + val initContainerUnderTest = new SparkPodInitContainer(sparkConf, fileFetcher) initContainerUnderTest.run() Mockito.verify(fileFetcher).fetchFile("http://localhost:9000/jar1.jar", downloadJarsDir) Mockito.verify(fileFetcher).fetchFile("hdfs://localhost:9000/jar2.jar", downloadJarsDir) @@ -84,7 +82,7 @@ class KubernetesSparkDependencyDownloadInitContainerSuite } } -private object KubernetesSparkDependencyDownloadInitContainerSuite { +private object SparkPodInitContainerSuite { def createTempFile(extension: String): String = { val dir = Utils.createTempDir() val file = new File(dir, s"${UUID.randomUUID().toString}.$extension") diff --git a/resource-managers/kubernetes/docker/src/main/dockerfiles/init-container/Dockerfile b/resource-managers/kubernetes/docker/src/main/dockerfiles/init-container/Dockerfile index a52f5e130559..2f13c26090b3 100644 --- a/resource-managers/kubernetes/docker/src/main/dockerfiles/init-container/Dockerfile +++ b/resource-managers/kubernetes/docker/src/main/dockerfiles/init-container/Dockerfile @@ -21,4 +21,4 @@ FROM spark-base # command should be invoked from the top level directory of the Spark distribution. E.g.: # docker build -t spark-init:latest -f dockerfiles/init-container/Dockerfile . -ENTRYPOINT [ "/opt/entrypoint.sh", "/opt/spark/bin/spark-class", "org.apache.spark.deploy.rest.k8s.KubernetesSparkDependencyDownloadInitContainer" ] +ENTRYPOINT [ "/opt/entrypoint.sh", "/opt/spark/bin/spark-class", "org.apache.spark.deploy.rest.k8s.SparkPodInitContainer" ] From 9c8051a0156d58d13b91eb740aab83447ceddf66 Mon Sep 17 00:00:00 2001 From: Yinan Li Date: Sat, 16 Dec 2017 13:34:45 -0800 Subject: [PATCH 04/14] Create one task per jar/file to download in the init-container --- .../k8s/submit/DriverConfigOrchestrator.scala | 27 +----------- .../rest/k8s/SparkPodInitContainer.scala | 41 +++++++++---------- .../DriverConfigOrchestratorSuite.scala | 14 +------ .../rest/k8s/SparkPodInitContainerSuite.scala | 1 - 4 files changed, 22 insertions(+), 61 deletions(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/DriverConfigOrchestrator.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/DriverConfigOrchestrator.scala index 5d150996bffe..65470a89b233 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/DriverConfigOrchestrator.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/DriverConfigOrchestrator.scala @@ -125,30 +125,6 @@ private[spark] class DriverConfigOrchestrator( val mayBeInitContainerBootstrapStep = if (areAnyFilesNonContainerLocal(sparkJars ++ sparkFiles)) { -<<<<<<< HEAD:resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestrator.scala - val initContainerConfigurationStepsOrchestrator = - new InitContainerConfigurationStepsOrchestrator( - namespace, - kubernetesResourceNamePrefix, - sparkJars, - sparkFiles, - jarsDownloadPath, - filesDownloadPath, - imagePullPolicy, - allDriverLabels, - initContainerConfigMapName, - INIT_CONTAINER_PROPERTIES_FILE_NAME, - submissionSparkConf) - val initContainerConfigurationSteps = - initContainerConfigurationStepsOrchestrator.getAllConfigurationSteps() - val initContainerBootstrapStep = - new DriverInitContainerBootstrapStep( - initContainerConfigurationSteps, - initContainerConfigMapName, - INIT_CONTAINER_PROPERTIES_FILE_NAME) - - Some(initContainerBootstrapStep) -======= val orchestrator = new InitContainerConfigOrchestrator( namespace, kubernetesResourceNamePrefix, @@ -156,7 +132,7 @@ private[spark] class DriverConfigOrchestrator( sparkFiles, jarsDownloadPath, filesDownloadPath, - dockerImagePullPolicy, + imagePullPolicy, allDriverLabels, initContainerConfigMapName, INIT_CONTAINER_PROPERTIES_FILE_NAME, @@ -167,7 +143,6 @@ private[spark] class DriverConfigOrchestrator( INIT_CONTAINER_PROPERTIES_FILE_NAME) Some(bootstrapStep) ->>>>>>> Addressed the second round of comments:resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/DriverConfigOrchestrator.scala } else { None } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/SparkPodInitContainer.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/SparkPodInitContainer.scala index 1142b78d409c..49759ff0ff47 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/SparkPodInitContainer.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/SparkPodInitContainer.scala @@ -20,7 +20,6 @@ import java.io.File import java.util.concurrent.TimeUnit import scala.concurrent.{ExecutionContext, Future} -import scala.concurrent.duration.Duration import org.apache.spark.{SecurityManager => SparkSecurityManager, SparkConf} import org.apache.spark.deploy.SparkHadoopUtil @@ -54,26 +53,22 @@ private[spark] class SparkPodInitContainer( private val downloadTimeoutMinutes = sparkConf.get(INIT_CONTAINER_MOUNT_TIMEOUT) def run(): Unit = { - val remoteJarsDownload = Future[Unit] { - logInfo(s"Downloading remote jars: $remoteJars") - downloadFiles( - remoteJars, - jarsDownloadDir, - s"Remote jars download directory specified at $jarsDownloadDir does not exist " + - "or is not a directory.") - } - val remoteFilesDownload = Future[Unit] { - logInfo(s"Downloading remote files: $remoteFiles") - downloadFiles( - remoteFiles, - filesDownloadDir, - s"Remote files download directory specified at $filesDownloadDir does not exist " + - "or is not a directory.") - } - - Seq(remoteJarsDownload, remoteFilesDownload).foreach { - ThreadUtils.awaitResult(_, Duration.create(downloadTimeoutMinutes, TimeUnit.MINUTES)) - } + logInfo(s"Downloading remote jars: $remoteJars") + downloadFiles( + remoteJars, + jarsDownloadDir, + s"Remote jars download directory specified at $jarsDownloadDir does not exist " + + "or is not a directory.") + + logInfo(s"Downloading remote files: $remoteFiles") + downloadFiles( + remoteFiles, + filesDownloadDir, + s"Remote files download directory specified at $filesDownloadDir does not exist " + + "or is not a directory.") + + downloadExecutor.shutdown() + downloadExecutor.awaitTermination(downloadTimeoutMinutes, TimeUnit.MINUTES) } private def downloadFiles( @@ -84,7 +79,9 @@ private[spark] class SparkPodInitContainer( require(downloadDir.isDirectory, errMessageOnDestinationNotADirectory) } filesCommaSeparated.map(_.split(",")).toSeq.flatten.foreach { file => - fileFetcher.fetchFile(file, downloadDir) + Future[Unit] { + fileFetcher.fetchFile(file, downloadDir) + } } } } diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/DriverConfigOrchestratorSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/DriverConfigOrchestratorSuite.scala index 6642335d12c9..513b2de068d2 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/DriverConfigOrchestratorSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/DriverConfigOrchestratorSuite.scala @@ -24,7 +24,7 @@ class DriverConfigOrchestratorSuite extends SparkFunSuite { private val NAMESPACE = "default" private val DRIVER_IMAGE = "driver-image" - private val INIT_CONTAINER_IMAGE = "init-container-image" + private val IC_IMAGE = "init-container-image" private val APP_ID = "spark-app-id" private val LAUNCH_TIME = 975256L private val APP_NAME = "spark" @@ -58,13 +58,8 @@ class DriverConfigOrchestratorSuite extends SparkFunSuite { test("Base submission steps without a main app resource.") { val sparkConf = new SparkConf(false) -<<<<<<< HEAD:resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestratorSuite.scala .set(DRIVER_CONTAINER_IMAGE, DRIVER_IMAGE) - val orchestrator = new DriverConfigurationStepsOrchestrator( -======= - .set(DRIVER_DOCKER_IMAGE, DRIVER_IMAGE) val orchestrator = new DriverConfigOrchestrator( ->>>>>>> Addressed the second round of comments:resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/DriverConfigOrchestratorSuite.scala NAMESPACE, APP_ID, LAUNCH_TIME, @@ -83,13 +78,8 @@ class DriverConfigOrchestratorSuite extends SparkFunSuite { test("Submission steps with an init-container.") { val sparkConf = new SparkConf(false) -<<<<<<< HEAD:resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestratorSuite.scala .set(DRIVER_CONTAINER_IMAGE, DRIVER_IMAGE) - .set(INIT_CONTAINER_DOCKER_IMAGE, INIT_CONTAINER_IMAGE) -======= - .set(DRIVER_DOCKER_IMAGE, DRIVER_IMAGE) - .set(INIT_CONTAINER_IMAGE, INIT_CONTAINER_IMAGE) ->>>>>>> Addressed the second round of comments:resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/DriverConfigOrchestratorSuite.scala + .set(INIT_CONTAINER_IMAGE, IC_IMAGE) .set("spark.jars", "hdfs://localhost:9000/var/apps/jars/jar1.jar") val mainAppResource = JavaMainAppResource("local:///var/apps/jars/main.jar") val orchestrator = new DriverConfigOrchestrator( diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/k8s/SparkPodInitContainerSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/k8s/SparkPodInitContainerSuite.scala index 15956cf71e89..37f18c113f0c 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/k8s/SparkPodInitContainerSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/k8s/SparkPodInitContainerSuite.scala @@ -68,7 +68,6 @@ class SparkPodInitContainerSuite Mockito.verify(fileFetcher).fetchFile("http://localhost:9000/jar1.jar", downloadJarsDir) Mockito.verify(fileFetcher).fetchFile("hdfs://localhost:9000/jar2.jar", downloadJarsDir) Mockito.verify(fileFetcher).fetchFile("http://localhost:9000/file.txt", downloadFilesDir) - } private def getSparkConfForRemoteFileDownloads: SparkConf = { From 1f65417b234bddc9ff16a5a52d6082cda9a8bea4 Mon Sep 17 00:00:00 2001 From: Yinan Li Date: Mon, 18 Dec 2017 08:23:14 -0800 Subject: [PATCH 05/14] More review comments --- .../rest/k8s/SparkConfPropertiesParser.scala | 4 +-- .../DriverConfigOrchestratorSuite.scala | 4 +-- ...riverInitContainerBootstrapStepSuite.scala | 28 +++++++++---------- 3 files changed, 18 insertions(+), 18 deletions(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/SparkConfPropertiesParser.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/SparkConfPropertiesParser.scala index 176166c1fee0..d50ba0f5f796 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/SparkConfPropertiesParser.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/SparkConfPropertiesParser.scala @@ -33,8 +33,8 @@ private[spark] object SparkConfPropertiesParser { val sparkConf = new SparkConf(true) if (!propertiesFile.isFile) { - throw new IllegalArgumentException(s"Server properties file given at" + - s" ${propertiesFile.getAbsoluteFile} does not exist or is not a file.") + throw new IllegalArgumentException("Server properties file given at " + + s"${propertiesFile.getAbsoluteFile} does not exist or is not a file.") } val properties = new Properties diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/DriverConfigOrchestratorSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/DriverConfigOrchestratorSuite.scala index 513b2de068d2..8fc25d17d381 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/DriverConfigOrchestratorSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/DriverConfigOrchestratorSuite.scala @@ -125,8 +125,8 @@ class DriverConfigOrchestratorSuite extends SparkFunSuite { } private def validateStepTypes( - orchestrator: DriverConfigOrchestrator, - types: Class[_ <: DriverConfigurationStep]*): Unit = { + orchestrator: DriverConfigOrchestrator, + types: Class[_ <: DriverConfigurationStep]*): Unit = { val steps = orchestrator.getAllConfigurationSteps() assert(steps.size === types.size) assert(steps.map(_.getClass) === types) diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/DriverInitContainerBootstrapStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/DriverInitContainerBootstrapStepSuite.scala index 36461754772a..b3076f6e80df 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/DriverInitContainerBootstrapStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/DriverInitContainerBootstrapStepSuite.scala @@ -42,8 +42,8 @@ class DriverInitContainerBootstrapStepSuite extends SparkFunSuite { driverSparkConf = new SparkConf(false), otherKubernetesResources = Seq.empty[HasMetadata]) val initContainerSteps = Seq( - FirstTestInitContainerConfigurationStep$, - SecondTestInitContainerConfigurationStep$) + FirstTestInitContainerConfigurationStep, + SecondTestInitContainerConfigurationStep) val bootstrapStep = new DriverInitContainerBootstrapStep( initContainerSteps, CONFIG_MAP_NAME, @@ -52,17 +52,17 @@ class DriverInitContainerBootstrapStepSuite extends SparkFunSuite { val preparedDriverSpec = bootstrapStep.configureDriver(baseDriverSpec) assert(preparedDriverSpec.driverPod.getMetadata.getLabels.asScala === - FirstTestInitContainerConfigurationStep$.additionalLabels) + FirstTestInitContainerConfigurationStep.additionalLabels) val additionalDriverEnv = preparedDriverSpec.driverContainer.getEnv.asScala assert(additionalDriverEnv.size === 1) assert(additionalDriverEnv.head.getName === - FirstTestInitContainerConfigurationStep$.additionalMainContainerEnvKey) + FirstTestInitContainerConfigurationStep.additionalMainContainerEnvKey) assert(additionalDriverEnv.head.getValue === - FirstTestInitContainerConfigurationStep$.additionalMainContainerEnvValue) + FirstTestInitContainerConfigurationStep.additionalMainContainerEnvValue) assert(preparedDriverSpec.otherKubernetesResources.size === 2) assert(preparedDriverSpec.otherKubernetesResources.contains( - FirstTestInitContainerConfigurationStep$.additionalKubernetesResource)) + FirstTestInitContainerConfigurationStep.additionalKubernetesResource)) assert(preparedDriverSpec.otherKubernetesResources.exists { case configMap: ConfigMap => val hasMatchingName = configMap.getMetadata.getName == CONFIG_MAP_NAME @@ -75,8 +75,8 @@ class DriverInitContainerBootstrapStepSuite extends SparkFunSuite { } val initContainerPropertiesMap = Maps.fromProperties(initContainerProperties).asScala val expectedInitContainerProperties = Map( - SecondTestInitContainerConfigurationStep$.additionalInitContainerPropertyKey -> - SecondTestInitContainerConfigurationStep$.additionalInitContainerPropertyValue) + SecondTestInitContainerConfigurationStep.additionalInitContainerPropertyKey -> + SecondTestInitContainerConfigurationStep.additionalInitContainerPropertyValue) val hasMatchingProperties = initContainerPropertiesMap == expectedInitContainerProperties hasMatchingName && hasCorrectNumberOfEntries && hasMatchingProperties case _ => false @@ -87,20 +87,20 @@ class DriverInitContainerBootstrapStepSuite extends SparkFunSuite { val initContainerEnv = initContainers.get(0).getEnv.asScala assert(initContainerEnv.size === 1) assert(initContainerEnv.head.getName === - SecondTestInitContainerConfigurationStep$.additionalInitContainerEnvKey) + SecondTestInitContainerConfigurationStep.additionalInitContainerEnvKey) assert(initContainerEnv.head.getValue === - SecondTestInitContainerConfigurationStep$.additionalInitContainerEnvValue) + SecondTestInitContainerConfigurationStep.additionalInitContainerEnvValue) val expectedSparkConf = Map( INIT_CONTAINER_CONFIG_MAP_NAME.key -> CONFIG_MAP_NAME, INIT_CONTAINER_CONFIG_MAP_KEY_CONF.key -> CONFIG_MAP_KEY, - SecondTestInitContainerConfigurationStep$.additionalDriverSparkConfKey -> - SecondTestInitContainerConfigurationStep$.additionalDriverSparkConfValue) + SecondTestInitContainerConfigurationStep.additionalDriverSparkConfKey -> + SecondTestInitContainerConfigurationStep.additionalDriverSparkConfValue) assert(preparedDriverSpec.driverSparkConf.getAll.toMap === expectedSparkConf) } } -private object FirstTestInitContainerConfigurationStep$ extends InitContainerConfigurationStep { +private object FirstTestInitContainerConfigurationStep extends InitContainerConfigurationStep { val additionalLabels = Map("additionalLabelkey" -> "additionalLabelValue") val additionalMainContainerEnvKey = "TEST_ENV_MAIN_KEY" @@ -132,7 +132,7 @@ private object FirstTestInitContainerConfigurationStep$ extends InitContainerCon } } -private object SecondTestInitContainerConfigurationStep$ extends InitContainerConfigurationStep { +private object SecondTestInitContainerConfigurationStep extends InitContainerConfigurationStep { val additionalInitContainerEnvKey = "TEST_ENV_INIT_KEY" val additionalInitContainerEnvValue = "TEST_ENV_INIT_VALUE" val additionalInitContainerPropertyKey = "spark.initcontainer.testkey" From 109ad800de609f05ead87f9d2b1a99b683bfc335 Mon Sep 17 00:00:00 2001 From: Yinan Li Date: Mon, 18 Dec 2017 20:52:08 -0800 Subject: [PATCH 06/14] Shorten variable names --- .../deploy/k8s/InitContainerBootstrap.scala | 46 +++++++--------- .../k8s/submit/DriverConfigOrchestrator.scala | 34 ++++++------ .../submit/KubernetesClientApplication.scala | 25 ++++----- .../steps/BaseDriverConfigurationStep.scala | 55 +++++++++---------- .../steps/DependencyResolutionStep.scala | 19 ++++--- .../DriverInitContainerBootstrapStep.scala | 34 ++++++------ .../submit/steps/DriverMountSecretsStep.scala | 12 ++-- .../steps/DriverServiceBootstrapStep.scala | 17 +++--- .../BaseInitContainerConfigurationStep.scala | 25 ++++----- .../InitContainerConfigOrchestrator.scala | 36 ++++++------ .../InitContainerConfigurationStep.scala | 2 +- .../InitContainerMountSecretsStep.scala | 19 +++---- .../initcontainer/InitContainerSpec.scala | 8 +-- .../cluster/k8s/ExecutorPodFactory.scala | 36 ++++++------ .../DriverConfigOrchestratorSuite.scala | 2 +- ...riverInitContainerBootstrapStepSuite.scala | 6 +- ...eInitContainerConfigurationStepSuite.scala | 2 +- ...InitContainerConfigOrchestratorSuite.scala | 6 +- 18 files changed, 188 insertions(+), 196 deletions(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/InitContainerBootstrap.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/InitContainerBootstrap.scala index 60adb05821ac..af04d0377067 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/InitContainerBootstrap.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/InitContainerBootstrap.scala @@ -32,25 +32,23 @@ private[spark] trait InitContainerBootstrap { /** * Bootstraps an init-container that downloads dependencies to be used by a main container. */ - def bootstrapInitContainer( - originalPodWithInitContainer: PodWithDetachedInitContainer) - : PodWithDetachedInitContainer + def bootstrapInitContainer(original: PodWithDetachedInitContainer): PodWithDetachedInitContainer } private[spark] class InitContainerBootstrapImpl( initContainerImage: String, - dockerImagePullPolicy: String, + imagePullPolicy: String, jarsDownloadPath: String, filesDownloadPath: String, downloadTimeoutMinutes: Long, - initContainerConfigMapName: String, - initContainerConfigMapKey: String, + configMapName: String, + configMapKey: String, sparkRole: String, sparkConf: SparkConf) extends InitContainerBootstrap { override def bootstrapInitContainer( - podWithDetachedInitContainer: PodWithDetachedInitContainer): PodWithDetachedInitContainer = { + original: PodWithDetachedInitContainer): PodWithDetachedInitContainer = { val sharedVolumeMounts = Seq[VolumeMount]( new VolumeMountBuilder() .withName(INIT_CONTAINER_DOWNLOAD_JARS_VOLUME_NAME) @@ -61,25 +59,23 @@ private[spark] class InitContainerBootstrapImpl( .withMountPath(filesDownloadPath) .build()) - val initContainerCustomEnvVarKeyPrefix = sparkRole match { + val customEnvVarKeyPrefix = sparkRole match { case SPARK_POD_DRIVER_ROLE => KUBERNETES_DRIVER_ENV_KEY case SPARK_POD_EXECUTOR_ROLE => "spark.executorEnv." case _ => throw new SparkException(s"$sparkRole is not a valid Spark pod role") } - val initContainerCustomEnvVars = sparkConf.getAllWithPrefix(initContainerCustomEnvVarKeyPrefix) - .toSeq - .map { env => - new EnvVarBuilder() - .withName(env._1) - .withValue(env._2) - .build() - } + val customEnvVars = sparkConf.getAllWithPrefix(customEnvVarKeyPrefix).toSeq.map { env => + new EnvVarBuilder() + .withName(env._1) + .withValue(env._2) + .build() + } - val initContainer = new ContainerBuilder(podWithDetachedInitContainer.initContainer) + val initContainer = new ContainerBuilder(original.initContainer) .withName("spark-init") .withImage(initContainerImage) - .withImagePullPolicy(dockerImagePullPolicy) - .addAllToEnv(initContainerCustomEnvVars.asJava) + .withImagePullPolicy(imagePullPolicy) + .addAllToEnv(customEnvVars.asJava) .addNewVolumeMount() .withName(INIT_CONTAINER_PROPERTIES_FILE_VOLUME) .withMountPath(INIT_CONTAINER_PROPERTIES_FILE_DIR) @@ -88,14 +84,14 @@ private[spark] class InitContainerBootstrapImpl( .addToArgs(INIT_CONTAINER_PROPERTIES_FILE_PATH) .build() - val podWithBasicVolumes = new PodBuilder(podWithDetachedInitContainer.pod) + val podWithBasicVolumes = new PodBuilder(original.pod) .editSpec() .addNewVolume() .withName(INIT_CONTAINER_PROPERTIES_FILE_VOLUME) .withNewConfigMap() - .withName(initContainerConfigMapName) + .withName(configMapName) .addNewItem() - .withKey(initContainerConfigMapKey) + .withKey(configMapKey) .withPath(INIT_CONTAINER_PROPERTIES_FILE_NAME) .endItem() .endConfigMap() @@ -111,8 +107,8 @@ private[spark] class InitContainerBootstrapImpl( .endSpec() .build() - val mainContainerWithMountedFiles = new ContainerBuilder( - podWithDetachedInitContainer.mainContainer) + val mainContainer = new ContainerBuilder( + original.mainContainer) .addToVolumeMounts(sharedVolumeMounts: _*) .addNewEnv() .withName(ENV_MOUNTED_FILES_DIR) @@ -123,6 +119,6 @@ private[spark] class InitContainerBootstrapImpl( PodWithDetachedInitContainer( podWithBasicVolumes, initContainer, - mainContainerWithMountedFiles) + mainContainer) } } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/DriverConfigOrchestrator.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/DriverConfigOrchestrator.scala index 65470a89b233..7732a88b0806 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/DriverConfigOrchestrator.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/DriverConfigOrchestrator.scala @@ -41,7 +41,7 @@ private[spark] class DriverConfigOrchestrator( appName: String, mainClass: String, appArgs: Array[String], - submissionSparkConf: SparkConf) { + sparkConf: SparkConf) { // The resource name prefix is derived from the Spark application name, making it easy to connect // the names of the Kubernetes resources from e.g. kubectl or the Kubernetes dashboard to the @@ -51,14 +51,14 @@ private[spark] class DriverConfigOrchestrator( s"$appName-$uuid".toLowerCase.replaceAll("\\.", "-") } - private val imagePullPolicy = submissionSparkConf.get(CONTAINER_IMAGE_PULL_POLICY) + private val imagePullPolicy = sparkConf.get(CONTAINER_IMAGE_PULL_POLICY) private val initContainerConfigMapName = s"$kubernetesResourceNamePrefix-init-config" - private val jarsDownloadPath = submissionSparkConf.get(JARS_DOWNLOAD_LOCATION) - private val filesDownloadPath = submissionSparkConf.get(FILES_DOWNLOAD_LOCATION) + private val jarsDownloadPath = sparkConf.get(JARS_DOWNLOAD_LOCATION) + private val filesDownloadPath = sparkConf.get(FILES_DOWNLOAD_LOCATION) - def getAllConfigurationSteps(): Seq[DriverConfigurationStep] = { + def getAllConfigurationSteps: Seq[DriverConfigurationStep] = { val driverCustomLabels = ConfigurationUtils.parsePrefixedKeyValuePairs( - submissionSparkConf, + sparkConf, KUBERNETES_DRIVER_LABEL_PREFIX) require(!driverCustomLabels.contains(SPARK_APP_ID_LABEL), "Label with key " + s"$SPARK_APP_ID_LABEL is not allowed as it is reserved for Spark bookkeeping " + @@ -67,8 +67,8 @@ private[spark] class DriverConfigOrchestrator( s"$SPARK_ROLE_LABEL is not allowed as it is reserved for Spark bookkeeping " + "operations.") - val driverSecretNamesToMountPaths = ConfigurationUtils.parsePrefixedKeyValuePairs( - submissionSparkConf, + val secretNamesToMountPaths = ConfigurationUtils.parsePrefixedKeyValuePairs( + sparkConf, KUBERNETES_DRIVER_SECRETS_PREFIX) val allDriverLabels = driverCustomLabels ++ Map( @@ -83,16 +83,16 @@ private[spark] class DriverConfigOrchestrator( appName, mainClass, appArgs, - submissionSparkConf) + sparkConf) val driverAddressStep = new DriverServiceBootstrapStep( kubernetesResourceNamePrefix, allDriverLabels, - submissionSparkConf, + sparkConf, new SystemClock) val kubernetesCredentialsStep = new DriverKubernetesCredentialsStep( - submissionSparkConf, kubernetesResourceNamePrefix) + sparkConf, kubernetesResourceNamePrefix) val additionalMainAppJar = if (mainAppResource.nonEmpty) { val mayBeResource = mainAppResource.get match { @@ -105,11 +105,11 @@ private[spark] class DriverConfigOrchestrator( None } - val sparkJars = submissionSparkConf.getOption("spark.jars") + val sparkJars = sparkConf.getOption("spark.jars") .map(_.split(",")) .getOrElse(Array.empty[String]) ++ additionalMainAppJar.toSeq - val sparkFiles = submissionSparkConf.getOption("spark.files") + val sparkFiles = sparkConf.getOption("spark.files") .map(_.split(",")) .getOrElse(Array.empty[String]) @@ -136,9 +136,9 @@ private[spark] class DriverConfigOrchestrator( allDriverLabels, initContainerConfigMapName, INIT_CONTAINER_PROPERTIES_FILE_NAME, - submissionSparkConf) + sparkConf) val bootstrapStep = new DriverInitContainerBootstrapStep( - orchestrator.getAllConfigurationSteps(), + orchestrator.getAllConfigurationSteps, initContainerConfigMapName, INIT_CONTAINER_PROPERTIES_FILE_NAME) @@ -147,8 +147,8 @@ private[spark] class DriverConfigOrchestrator( None } - val mayBeMountSecretsStep = if (driverSecretNamesToMountPaths.nonEmpty) { - val mountSecretsBootstrap = new MountSecretsBootstrapImpl(driverSecretNamesToMountPaths) + val mayBeMountSecretsStep = if (secretNamesToMountPaths.nonEmpty) { + val mountSecretsBootstrap = new MountSecretsBootstrapImpl(secretNamesToMountPaths) Some(new DriverMountSecretsStep(mountSecretsBootstrap)) } else { None diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesClientApplication.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesClientApplication.scala index 0bbdaa0933f3..1b9f4e04b6fb 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesClientApplication.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesClientApplication.scala @@ -80,22 +80,22 @@ private[spark] object ClientArguments { * spark.kubernetes.submission.waitAppCompletion is true. * * @param submissionSteps steps that collectively configure the driver - * @param submissionSparkConf the submission client Spark configuration + * @param sparkConf the submission client Spark configuration * @param kubernetesClient the client to talk to the Kubernetes API server * @param waitForAppCompletion a flag indicating whether the client should wait for the application * to complete * @param appName the application name - * @param loggingPodStatusWatcher a watcher that monitors and logs the application status + * @param watcher a watcher that monitors and logs the application status */ private[spark] class Client( submissionSteps: Seq[DriverConfigurationStep], - submissionSparkConf: SparkConf, + sparkConf: SparkConf, kubernetesClient: KubernetesClient, waitForAppCompletion: Boolean, appName: String, - loggingPodStatusWatcher: LoggingPodStatusWatcher) extends Logging { + watcher: LoggingPodStatusWatcher) extends Logging { - private val driverJavaOptions = submissionSparkConf.get( + private val driverJavaOptions = sparkConf.get( org.apache.spark.internal.config.DRIVER_JAVA_OPTIONS) /** @@ -104,7 +104,7 @@ private[spark] class Client( * will be used to build the Driver Container, Driver Pod, and Kubernetes Resources */ def run(): Unit = { - var currentDriverSpec = KubernetesDriverSpec.initialSpec(submissionSparkConf) + var currentDriverSpec = KubernetesDriverSpec.initialSpec(sparkConf) // submissionSteps contain steps necessary to take, to resolve varying // client arguments that are passed in, created by orchestrator for (nextStep <- submissionSteps) { @@ -141,7 +141,7 @@ private[spark] class Client( kubernetesClient .pods() .withName(resolvedDriverPod.getMetadata.getName) - .watch(loggingPodStatusWatcher)) { _ => + .watch(watcher)) { _ => val createdDriverPod = kubernetesClient.pods().create(resolvedDriverPod) try { if (currentDriverSpec.otherKubernetesResources.nonEmpty) { @@ -157,7 +157,7 @@ private[spark] class Client( if (waitForAppCompletion) { logInfo(s"Waiting for application $appName to finish...") - loggingPodStatusWatcher.awaitCompletion() + watcher.awaitCompletion() logInfo(s"Application $appName finished.") } else { logInfo(s"Deployed Spark application $appName into Kubernetes.") @@ -207,10 +207,9 @@ private[spark] class KubernetesClientApplication extends SparkApplication { val master = sparkConf.get("spark.master").substring("k8s://".length) val loggingInterval = if (waitForAppCompletion) Some(sparkConf.get(REPORT_INTERVAL)) else None - val loggingPodStatusWatcher = new LoggingPodStatusWatcherImpl( - kubernetesAppId, loggingInterval) + val watcher = new LoggingPodStatusWatcherImpl(kubernetesAppId, loggingInterval) - val configurationStepsOrchestrator = new DriverConfigOrchestrator( + val orchestrator = new DriverConfigOrchestrator( namespace, kubernetesAppId, launchTime, @@ -228,12 +227,12 @@ private[spark] class KubernetesClientApplication extends SparkApplication { None, None)) { kubernetesClient => val client = new Client( - configurationStepsOrchestrator.getAllConfigurationSteps(), + orchestrator.getAllConfigurationSteps, sparkConf, kubernetesClient, waitForAppCompletion, appName, - loggingPodStatusWatcher) + watcher) client.run() } } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/BaseDriverConfigurationStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/BaseDriverConfigurationStep.scala index c335fcce4036..e6fe596b8b8e 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/BaseDriverConfigurationStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/BaseDriverConfigurationStep.scala @@ -32,39 +32,36 @@ import org.apache.spark.internal.config.{DRIVER_CLASS_PATH, DRIVER_MEMORY, DRIVE */ private[spark] class BaseDriverConfigurationStep( kubernetesAppId: String, - kubernetesResourceNamePrefix: String, + resourceNamePrefix: String, driverLabels: Map[String, String], imagePullPolicy: String, appName: String, mainClass: String, appArgs: Array[String], - submissionSparkConf: SparkConf) extends DriverConfigurationStep { + sparkConf: SparkConf) extends DriverConfigurationStep { - private val kubernetesDriverPodName = submissionSparkConf.get(KUBERNETES_DRIVER_POD_NAME) - .getOrElse(s"$kubernetesResourceNamePrefix-driver") + private val driverPodName = sparkConf + .get(KUBERNETES_DRIVER_POD_NAME) + .getOrElse(s"$resourceNamePrefix-driver") - private val driverExtraClasspath = submissionSparkConf.get( - DRIVER_CLASS_PATH) + private val driverExtraClasspath = sparkConf.get(DRIVER_CLASS_PATH) - private val driverContainerImage = submissionSparkConf + private val driverContainerImage = sparkConf .get(DRIVER_CONTAINER_IMAGE) .getOrElse(throw new SparkException("Must specify the driver container image")) // CPU settings - private val driverCpuCores = submissionSparkConf.getOption("spark.driver.cores").getOrElse("1") - private val driverLimitCores = submissionSparkConf.get(KUBERNETES_DRIVER_LIMIT_CORES) + private val driverCpuCores = sparkConf.getOption("spark.driver.cores").getOrElse("1") + private val driverLimitCores = sparkConf.get(KUBERNETES_DRIVER_LIMIT_CORES) // Memory settings - private val driverMemoryMiB = submissionSparkConf.get( - DRIVER_MEMORY) - private val driverMemoryString = submissionSparkConf.get( - DRIVER_MEMORY.key, - DRIVER_MEMORY.defaultValueString) - private val memoryOverheadMiB = submissionSparkConf + private val driverMemoryMiB = sparkConf.get(DRIVER_MEMORY) + private val driverMemoryString = sparkConf.get( + DRIVER_MEMORY.key, DRIVER_MEMORY.defaultValueString) + private val memoryOverheadMiB = sparkConf .get(DRIVER_MEMORY_OVERHEAD) - .getOrElse(math.max((MEMORY_OVERHEAD_FACTOR * driverMemoryMiB).toInt, - MEMORY_OVERHEAD_MIN_MIB)) - private val driverContainerMemoryWithOverheadMiB = driverMemoryMiB + memoryOverheadMiB + .getOrElse(math.max((MEMORY_OVERHEAD_FACTOR * driverMemoryMiB).toInt, MEMORY_OVERHEAD_MIN_MIB)) + private val driverMemoryWithOverheadMiB = driverMemoryMiB + memoryOverheadMiB override def configureDriver(driverSpec: KubernetesDriverSpec): KubernetesDriverSpec = { val driverExtraClasspathEnv = driverExtraClasspath.map { classPath => @@ -74,15 +71,13 @@ private[spark] class BaseDriverConfigurationStep( .build() } - val driverCustomAnnotations = ConfigurationUtils - .parsePrefixedKeyValuePairs( - submissionSparkConf, - KUBERNETES_DRIVER_ANNOTATION_PREFIX) + val driverCustomAnnotations = ConfigurationUtils.parsePrefixedKeyValuePairs( + sparkConf, KUBERNETES_DRIVER_ANNOTATION_PREFIX) require(!driverCustomAnnotations.contains(SPARK_APP_NAME_ANNOTATION), s"Annotation with key $SPARK_APP_NAME_ANNOTATION is not allowed as it is reserved for" + " Spark bookkeeping operations.") - val driverCustomEnvs = submissionSparkConf.getAllWithPrefix(KUBERNETES_DRIVER_ENV_KEY).toSeq + val driverCustomEnvs = sparkConf.getAllWithPrefix(KUBERNETES_DRIVER_ENV_KEY).toSeq .map { env => new EnvVarBuilder() .withName(env._1) @@ -90,10 +85,10 @@ private[spark] class BaseDriverConfigurationStep( .build() } - val allDriverAnnotations = driverCustomAnnotations ++ Map(SPARK_APP_NAME_ANNOTATION -> appName) + val driverAnnotations = driverCustomAnnotations ++ Map(SPARK_APP_NAME_ANNOTATION -> appName) val nodeSelector = ConfigurationUtils.parsePrefixedKeyValuePairs( - submissionSparkConf, KUBERNETES_NODE_SELECTOR_PREFIX) + sparkConf, KUBERNETES_NODE_SELECTOR_PREFIX) val driverCpuQuantity = new QuantityBuilder(false) .withAmount(driverCpuCores) @@ -102,7 +97,7 @@ private[spark] class BaseDriverConfigurationStep( .withAmount(s"${driverMemoryMiB}Mi") .build() val driverMemoryLimitQuantity = new QuantityBuilder(false) - .withAmount(s"${driverContainerMemoryWithOverheadMiB}Mi") + .withAmount(s"${driverMemoryWithOverheadMiB}Mi") .build() val maybeCpuLimitQuantity = driverLimitCores.map { limitCores => ("cpu", new QuantityBuilder(false).withAmount(limitCores).build()) @@ -142,9 +137,9 @@ private[spark] class BaseDriverConfigurationStep( val baseDriverPod = new PodBuilder(driverSpec.driverPod) .editOrNewMetadata() - .withName(kubernetesDriverPodName) + .withName(driverPodName) .addToLabels(driverLabels.asJava) - .addToAnnotations(allDriverAnnotations.asJava) + .addToAnnotations(driverAnnotations.asJava) .endMetadata() .withNewSpec() .withRestartPolicy("Never") @@ -153,9 +148,9 @@ private[spark] class BaseDriverConfigurationStep( .build() val resolvedSparkConf = driverSpec.driverSparkConf.clone() - .setIfMissing(KUBERNETES_DRIVER_POD_NAME, kubernetesDriverPodName) + .setIfMissing(KUBERNETES_DRIVER_POD_NAME, driverPodName) .set("spark.app.id", kubernetesAppId) - .set(KUBERNETES_EXECUTOR_POD_NAME_PREFIX, kubernetesResourceNamePrefix) + .set(KUBERNETES_EXECUTOR_POD_NAME_PREFIX, resourceNamePrefix) driverSpec.copy( driverPod = baseDriverPod, diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DependencyResolutionStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DependencyResolutionStep.scala index 44e0ecffc0e9..cb00ed0001f1 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DependencyResolutionStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DependencyResolutionStep.scala @@ -31,21 +31,23 @@ private[spark] class DependencyResolutionStep( sparkJars: Seq[String], sparkFiles: Seq[String], jarsDownloadPath: String, - localFilesDownloadPath: String) extends DriverConfigurationStep { + filesDownloadPath: String) extends DriverConfigurationStep { override def configureDriver(driverSpec: KubernetesDriverSpec): KubernetesDriverSpec = { val resolvedSparkJars = KubernetesFileUtils.resolveFileUris(sparkJars, jarsDownloadPath) val resolvedSparkFiles = KubernetesFileUtils.resolveFileUris( - sparkFiles, localFilesDownloadPath) - val sparkConfResolvedSparkDependencies = driverSpec.driverSparkConf.clone() + sparkFiles, filesDownloadPath) + + val sparkConf = driverSpec.driverSparkConf.clone() if (resolvedSparkJars.nonEmpty) { - sparkConfResolvedSparkDependencies.set("spark.jars", resolvedSparkJars.mkString(",")) + sparkConf.set("spark.jars", resolvedSparkJars.mkString(",")) } if (resolvedSparkFiles.nonEmpty) { - sparkConfResolvedSparkDependencies.set("spark.files", resolvedSparkFiles.mkString(",")) + sparkConf.set("spark.files", resolvedSparkFiles.mkString(",")) } + val resolvedClasspath = KubernetesFileUtils.resolveFilePaths(sparkJars, jarsDownloadPath) - val driverContainerWithResolvedClasspath = if (resolvedClasspath.nonEmpty) { + val resolvedDriverContainer = if (resolvedClasspath.nonEmpty) { new ContainerBuilder(driverSpec.driverContainer) .addNewEnv() .withName(ENV_MOUNTED_CLASSPATH) @@ -55,8 +57,9 @@ private[spark] class DependencyResolutionStep( } else { driverSpec.driverContainer } + driverSpec.copy( - driverContainer = driverContainerWithResolvedClasspath, - driverSparkConf = sparkConfResolvedSparkDependencies) + driverContainer = resolvedDriverContainer, + driverSparkConf = sparkConf) } } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DriverInitContainerBootstrapStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DriverInitContainerBootstrapStep.scala index 98a132723326..6ce51220139e 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DriverInitContainerBootstrapStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DriverInitContainerBootstrapStep.scala @@ -31,42 +31,42 @@ import org.apache.spark.deploy.k8s.submit.steps.initcontainer.{InitContainerConf * configuration properties for the init-container. */ private[spark] class DriverInitContainerBootstrapStep( - initContainerConfigurationSteps: Seq[InitContainerConfigurationStep], - initContainerConfigMapName: String, - initContainerConfigMapKey: String) + steps: Seq[InitContainerConfigurationStep], + configMapName: String, + configMapKey: String) extends DriverConfigurationStep { override def configureDriver(driverSpec: KubernetesDriverSpec): KubernetesDriverSpec = { - var currentInitContainerSpec = InitContainerSpec( - initContainerProperties = Map.empty[String, String], + var initContainerSpec = InitContainerSpec( + properties = Map.empty[String, String], driverSparkConf = Map.empty[String, String], initContainer = new ContainerBuilder().build(), driverContainer = driverSpec.driverContainer, driverPod = driverSpec.driverPod, - initContainerDependentResources = Seq.empty[HasMetadata]) - for (nextStep <- initContainerConfigurationSteps) { - currentInitContainerSpec = nextStep.configureInitContainer(currentInitContainerSpec) + dependentResources = Seq.empty[HasMetadata]) + for (nextStep <- steps) { + initContainerSpec = nextStep.configureInitContainer(initContainerSpec) } val configMap = buildConfigMap( - initContainerConfigMapName, - initContainerConfigMapKey, - currentInitContainerSpec.initContainerProperties) + configMapName, + configMapKey, + initContainerSpec.properties) val resolvedDriverSparkConf = driverSpec.driverSparkConf .clone() - .set(INIT_CONTAINER_CONFIG_MAP_NAME, initContainerConfigMapName) - .set(INIT_CONTAINER_CONFIG_MAP_KEY_CONF, initContainerConfigMapKey) - .setAll(currentInitContainerSpec.driverSparkConf) + .set(INIT_CONTAINER_CONFIG_MAP_NAME, configMapName) + .set(INIT_CONTAINER_CONFIG_MAP_KEY_CONF, configMapKey) + .setAll(initContainerSpec.driverSparkConf) val resolvedDriverPod = InitContainerUtil.appendInitContainer( - currentInitContainerSpec.driverPod, currentInitContainerSpec.initContainer) + initContainerSpec.driverPod, initContainerSpec.initContainer) driverSpec.copy( driverPod = resolvedDriverPod, - driverContainer = currentInitContainerSpec.driverContainer, + driverContainer = initContainerSpec.driverContainer, driverSparkConf = resolvedDriverSparkConf, otherKubernetesResources = driverSpec.otherKubernetesResources ++ - currentInitContainerSpec.initContainerDependentResources ++ + initContainerSpec.dependentResources ++ Seq(configMap)) } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DriverMountSecretsStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DriverMountSecretsStep.scala index 9eaaf32d5f58..f872e0f4b65d 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DriverMountSecretsStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DriverMountSecretsStep.scala @@ -22,17 +22,17 @@ import org.apache.spark.deploy.k8s.submit.KubernetesDriverSpec /** * A driver configuration step for mounting user-specified secrets onto user-specified paths. * - * @param mountSecretsBootstrap a utility actually handling mounting of the secrets. + * @param bootstrap a utility actually handling mounting of the secrets. */ private[spark] class DriverMountSecretsStep( - mountSecretsBootstrap: MountSecretsBootstrap) extends DriverConfigurationStep { + bootstrap: MountSecretsBootstrap) extends DriverConfigurationStep { override def configureDriver(driverSpec: KubernetesDriverSpec): KubernetesDriverSpec = { - val (driverPodWithSecretsMounted, driverContainerWithSecretsMounted) = - mountSecretsBootstrap.mountSecrets(driverSpec.driverPod, driverSpec.driverContainer) + val (pod, container) = bootstrap.mountSecrets( + driverSpec.driverPod, driverSpec.driverContainer) driverSpec.copy( - driverPod = driverPodWithSecretsMounted, - driverContainer = driverContainerWithSecretsMounted + driverPod = pod, + driverContainer = container ) } } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DriverServiceBootstrapStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DriverServiceBootstrapStep.scala index 696d11f15ed9..eb594e4f16ec 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DriverServiceBootstrapStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DriverServiceBootstrapStep.scala @@ -32,21 +32,22 @@ import org.apache.spark.util.Clock * ports should correspond to the ports that the executor will reach the pod at for RPC. */ private[spark] class DriverServiceBootstrapStep( - kubernetesResourceNamePrefix: String, + resourceNamePrefix: String, driverLabels: Map[String, String], - submissionSparkConf: SparkConf, + sparkConf: SparkConf, clock: Clock) extends DriverConfigurationStep with Logging { + import DriverServiceBootstrapStep._ override def configureDriver(driverSpec: KubernetesDriverSpec): KubernetesDriverSpec = { - require(submissionSparkConf.getOption(DRIVER_BIND_ADDRESS_KEY).isEmpty, + require(sparkConf.getOption(DRIVER_BIND_ADDRESS_KEY).isEmpty, s"$DRIVER_BIND_ADDRESS_KEY is not supported in Kubernetes mode, as the driver's bind " + "address is managed and set to the driver pod's IP address.") - require(submissionSparkConf.getOption(DRIVER_HOST_KEY).isEmpty, + require(sparkConf.getOption(DRIVER_HOST_KEY).isEmpty, s"$DRIVER_HOST_KEY is not supported in Kubernetes mode, as the driver's hostname will be " + "managed via a Kubernetes service.") - val preferredServiceName = s"$kubernetesResourceNamePrefix$DRIVER_SVC_POSTFIX" + val preferredServiceName = s"$resourceNamePrefix$DRIVER_SVC_POSTFIX" val resolvedServiceName = if (preferredServiceName.length <= MAX_SERVICE_NAME_LENGTH) { preferredServiceName } else { @@ -58,8 +59,8 @@ private[spark] class DriverServiceBootstrapStep( shorterServiceName } - val driverPort = submissionSparkConf.getInt("spark.driver.port", DEFAULT_DRIVER_PORT) - val driverBlockManagerPort = submissionSparkConf.getInt( + val driverPort = sparkConf.getInt("spark.driver.port", DEFAULT_DRIVER_PORT) + val driverBlockManagerPort = sparkConf.getInt( org.apache.spark.internal.config.DRIVER_BLOCK_MANAGER_PORT.key, DEFAULT_BLOCKMANAGER_PORT) val driverService = new ServiceBuilder() .withNewMetadata() @@ -81,7 +82,7 @@ private[spark] class DriverServiceBootstrapStep( .endSpec() .build() - val namespace = submissionSparkConf.get(KUBERNETES_NAMESPACE) + val namespace = sparkConf.get(KUBERNETES_NAMESPACE) val driverHostname = s"${driverService.getMetadata.getName}.$namespace.svc.cluster.local" val resolvedSparkConf = driverSpec.driverSparkConf.clone() .set(DRIVER_HOST_KEY, driverHostname) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/BaseInitContainerConfigurationStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/BaseInitContainerConfigurationStep.scala index 5c2108777c06..4be6ca288688 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/BaseInitContainerConfigurationStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/BaseInitContainerConfigurationStep.scala @@ -25,10 +25,10 @@ private[spark] class BaseInitContainerConfigurationStep( sparkFiles: Seq[String], jarsDownloadPath: String, filesDownloadPath: String, - initContainerBootstrap: InitContainerBootstrap) + bootstrap: InitContainerBootstrap) extends InitContainerConfigurationStep { - override def configureInitContainer(initContainerSpec: InitContainerSpec): InitContainerSpec = { + override def configureInitContainer(spec: InitContainerSpec): InitContainerSpec = { val remoteJarsToDownload = KubernetesFileUtils.getOnlyRemoteFiles(sparkJars) val remoteFilesToDownload = KubernetesFileUtils.getOnlyRemoteFiles(sparkFiles) val remoteJarsConf = if (remoteJarsToDownload.nonEmpty) { @@ -47,17 +47,16 @@ private[spark] class BaseInitContainerConfigurationStep( FILES_DOWNLOAD_LOCATION.key -> filesDownloadPath) ++ remoteJarsConf ++ remoteFilesConf - val bootstrappedPodAndInitContainer = - initContainerBootstrap.bootstrapInitContainer( - PodWithDetachedInitContainer( - initContainerSpec.driverPod, - initContainerSpec.initContainer, - initContainerSpec.driverContainer)) + val bootstrapped = bootstrap.bootstrapInitContainer( + PodWithDetachedInitContainer( + spec.driverPod, + spec.initContainer, + spec.driverContainer)) - initContainerSpec.copy( - initContainer = bootstrappedPodAndInitContainer.initContainer, - driverContainer = bootstrappedPodAndInitContainer.mainContainer, - driverPod = bootstrappedPodAndInitContainer.pod, - initContainerProperties = baseInitContainerConfig) + spec.copy( + initContainer = bootstrapped.initContainer, + driverContainer = bootstrapped.mainContainer, + driverPod = bootstrapped.pod, + properties = baseInitContainerConfig) } } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerConfigOrchestrator.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerConfigOrchestrator.scala index f90b46b38277..f3ea757fbe8b 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerConfigOrchestrator.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerConfigOrchestrator.scala @@ -27,57 +27,57 @@ import org.apache.spark.deploy.k8s.Constants._ */ private[spark] class InitContainerConfigOrchestrator( namespace: String, - kubernetesResourceNamePrefix: String, + resourceNamePrefix: String, sparkJars: Seq[String], sparkFiles: Seq[String], jarsDownloadPath: String, filesDownloadPath: String, - dockerImagePullPolicy: String, + imagePullPolicy: String, driverLabels: Map[String, String], - initContainerConfigMapName: String, - initContainerConfigMapKey: String, - submissionSparkConf: SparkConf) { + configMapName: String, + configMapKey: String, + sparkConf: SparkConf) { - private val initContainerImage = submissionSparkConf + private val initContainerImage = sparkConf .get(INIT_CONTAINER_IMAGE) .getOrElse(throw new SparkException( "Must specify the init-container image when there are remote dependencies")) - private val downloadTimeoutMinutes = submissionSparkConf.get(INIT_CONTAINER_MOUNT_TIMEOUT) + private val downloadTimeoutMinutes = sparkConf.get(INIT_CONTAINER_MOUNT_TIMEOUT) - def getAllConfigurationSteps(): Seq[InitContainerConfigurationStep] = { + def getAllConfigurationSteps: Seq[InitContainerConfigurationStep] = { val initContainerBootstrap = new InitContainerBootstrapImpl( initContainerImage, - dockerImagePullPolicy, + imagePullPolicy, jarsDownloadPath, filesDownloadPath, downloadTimeoutMinutes, - initContainerConfigMapName, - initContainerConfigMapKey, + configMapName, + configMapKey, SPARK_POD_DRIVER_ROLE, - submissionSparkConf) - val baseInitContainerStep = new BaseInitContainerConfigurationStep( + sparkConf) + val baseStep = new BaseInitContainerConfigurationStep( sparkJars, sparkFiles, jarsDownloadPath, filesDownloadPath, initContainerBootstrap) - val driverSecretNamesToMountPaths = ConfigurationUtils.parsePrefixedKeyValuePairs( - submissionSparkConf, + val secretNamesToMountPaths = ConfigurationUtils.parsePrefixedKeyValuePairs( + sparkConf, KUBERNETES_DRIVER_SECRETS_PREFIX) // Mount user-specified driver secrets also into the driver's init-container. The // init-container may need credentials in the secrets to be able to download remote // dependencies. The driver's main container and its init-container share the secrets // because the init-container is sort of an implementation details and this sharing // avoids introducing a dedicated configuration property just for the init-container. - val maybeMountSecretsStep = if (driverSecretNamesToMountPaths.nonEmpty) { - val mountSecretsBootstrap = new MountSecretsBootstrapImpl(driverSecretNamesToMountPaths) + val maybeMountSecretsStep = if (secretNamesToMountPaths.nonEmpty) { + val mountSecretsBootstrap = new MountSecretsBootstrapImpl(secretNamesToMountPaths) Some(new InitContainerMountSecretsStep(mountSecretsBootstrap)) } else { None } - Seq(baseInitContainerStep) ++ + Seq(baseStep) ++ maybeMountSecretsStep.toSeq } } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerConfigurationStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerConfigurationStep.scala index 0cca50539ec8..5a744cb86524 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerConfigurationStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerConfigurationStep.scala @@ -21,5 +21,5 @@ package org.apache.spark.deploy.k8s.submit.steps.initcontainer */ private[spark] trait InitContainerConfigurationStep { - def configureInitContainer(initContainerSpec: InitContainerSpec): InitContainerSpec + def configureInitContainer(spec: InitContainerSpec): InitContainerSpec } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerMountSecretsStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerMountSecretsStep.scala index 6eea1b5c6c93..c0e7bb20cce8 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerMountSecretsStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerMountSecretsStep.scala @@ -22,19 +22,18 @@ import org.apache.spark.deploy.k8s.MountSecretsBootstrap * An init-container configuration step for mounting user-specified secrets onto user-specified * paths. * - * @param mountSecretsBootstrap a utility actually handling mounting of the secrets + * @param bootstrap a utility actually handling mounting of the secrets */ private[spark] class InitContainerMountSecretsStep( - mountSecretsBootstrap: MountSecretsBootstrap) extends InitContainerConfigurationStep { + bootstrap: MountSecretsBootstrap) extends InitContainerConfigurationStep { - override def configureInitContainer(initContainerSpec: InitContainerSpec) : InitContainerSpec = { - val (podWithSecretsMounted, initContainerWithSecretsMounted) = - mountSecretsBootstrap.mountSecrets( - initContainerSpec.driverPod, - initContainerSpec.initContainer) - initContainerSpec.copy( - driverPod = podWithSecretsMounted, - initContainer = initContainerWithSecretsMounted + override def configureInitContainer(spec: InitContainerSpec) : InitContainerSpec = { + val (driverPod, initContainer) = bootstrap.mountSecrets( + spec.driverPod, + spec.initContainer) + spec.copy( + driverPod = driverPod, + initContainer = initContainer ) } } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerSpec.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerSpec.scala index 73fcf19c3936..b52c343f0c0e 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerSpec.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerSpec.scala @@ -21,17 +21,17 @@ import io.fabric8.kubernetes.api.model.{Container, HasMetadata, Pod} /** * Represents a specification of the init-container for the driver pod. * - * @param initContainerProperties properties that should be set on the init-container + * @param properties properties that should be set on the init-container * @param driverSparkConf Spark configuration properties that will be carried back to the driver * @param initContainer the init-container object * @param driverContainer the driver container object * @param driverPod the driver pod object - * @param initContainerDependentResources resources the init-container depends on to work + * @param dependentResources resources the init-container depends on to work */ private[spark] case class InitContainerSpec( - initContainerProperties: Map[String, String], + properties: Map[String, String], driverSparkConf: Map[String, String], initContainer: Container, driverContainer: Container, driverPod: Pod, - initContainerDependentResources: Seq[HasMetadata]) + dependentResources: Seq[HasMetadata]) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala index 32a02ddd4107..55eda3b39986 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala @@ -48,8 +48,8 @@ private[spark] trait ExecutorPodFactory { private[spark] class ExecutorPodFactoryImpl( sparkConf: SparkConf, mountSecretsBootstrap: Option[MountSecretsBootstrap], - executorInitContainerBootstrap: Option[InitContainerBootstrap], - executorInitContainerMountSecretsBootstrap: Option[MountSecretsBootstrap]) + initContainerBootstrap: Option[InitContainerBootstrap], + initContainerMountSecretsBootstrap: Option[MountSecretsBootstrap]) extends ExecutorPodFactory { private val executorExtraClasspath = sparkConf.get(EXECUTOR_CLASS_PATH) @@ -203,7 +203,7 @@ private[spark] class ExecutorPodFactoryImpl( .endSpec() .build() - val containerWithExecutorLimitCores = executorLimitCores.map { limitCores => + val containerWithLimitCores = executorLimitCores.map { limitCores => val executorCpuLimitQuantity = new QuantityBuilder(false) .withAmount(limitCores) .build() @@ -214,33 +214,33 @@ private[spark] class ExecutorPodFactoryImpl( .build() }.getOrElse(executorContainer) - val (withMaybeSecretsMountedPod, withMaybeSecretsMountedContainer) = + val (maybeSecretsMountedPod, maybeSecretsMountedContainer) = mountSecretsBootstrap.map { bootstrap => - bootstrap.mountSecrets(executorPod, containerWithExecutorLimitCores) - }.getOrElse((executorPod, containerWithExecutorLimitCores)) + bootstrap.mountSecrets(executorPod, containerWithLimitCores) + }.getOrElse((executorPod, containerWithLimitCores)) - val (executorPodWithInitContainer, initBootstrappedExecutorContainer) = - executorInitContainerBootstrap.map { bootstrap => + val (bootstrappedPod, bootstrappedContainer) = + initContainerBootstrap.map { bootstrap => val podWithInitContainer = bootstrap.bootstrapInitContainer( PodWithDetachedInitContainer( - withMaybeSecretsMountedPod, + maybeSecretsMountedPod, new ContainerBuilder().build(), - withMaybeSecretsMountedContainer)) + maybeSecretsMountedContainer)) - val (mayBePodWithSecretsMountedToInitContainer, mayBeInitContainerWithSecretsMounted) = - executorInitContainerMountSecretsBootstrap.map { bootstrap => + val (pod, mayBeSecretsMountedInitContainer) = + initContainerMountSecretsBootstrap.map { bootstrap => bootstrap.mountSecrets(podWithInitContainer.pod, podWithInitContainer.initContainer) }.getOrElse((podWithInitContainer.pod, podWithInitContainer.initContainer)) - val podWithAttachedInitContainer = InitContainerUtil.appendInitContainer( - mayBePodWithSecretsMountedToInitContainer, mayBeInitContainerWithSecretsMounted) + val bootstrappedPod = InitContainerUtil.appendInitContainer( + pod, mayBeSecretsMountedInitContainer) - (podWithAttachedInitContainer, podWithInitContainer.mainContainer) - }.getOrElse((withMaybeSecretsMountedPod, withMaybeSecretsMountedContainer)) + (bootstrappedPod, podWithInitContainer.mainContainer) + }.getOrElse((maybeSecretsMountedPod, maybeSecretsMountedContainer)) - new PodBuilder(executorPodWithInitContainer) + new PodBuilder(bootstrappedPod) .editSpec() - .addToContainers(initBootstrappedExecutorContainer) + .addToContainers(bootstrappedContainer) .endSpec() .build() } diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/DriverConfigOrchestratorSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/DriverConfigOrchestratorSuite.scala index 8fc25d17d381..53dfe21410e9 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/DriverConfigOrchestratorSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/DriverConfigOrchestratorSuite.scala @@ -127,7 +127,7 @@ class DriverConfigOrchestratorSuite extends SparkFunSuite { private def validateStepTypes( orchestrator: DriverConfigOrchestrator, types: Class[_ <: DriverConfigurationStep]*): Unit = { - val steps = orchestrator.getAllConfigurationSteps() + val steps = orchestrator.getAllConfigurationSteps assert(steps.size === types.size) assert(steps.map(_.getClass) === types) } diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/DriverInitContainerBootstrapStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/DriverInitContainerBootstrapStepSuite.scala index b3076f6e80df..f89f9d873b99 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/DriverInitContainerBootstrapStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/DriverInitContainerBootstrapStepSuite.scala @@ -127,7 +127,7 @@ private object FirstTestInitContainerConfigurationStep extends InitContainerConf initContainerSpec.copy( driverPod = driverPod, driverContainer = mainContainer, - initContainerDependentResources = initContainerSpec.initContainerDependentResources ++ + dependentResources = initContainerSpec.dependentResources ++ Seq(additionalKubernetesResource)) } } @@ -147,13 +147,13 @@ private object SecondTestInitContainerConfigurationStep extends InitContainerCon .withValue(additionalInitContainerEnvValue) .endEnv() .build() - val initContainerProperties = initContainerSpec.initContainerProperties ++ + val initContainerProperties = initContainerSpec.properties ++ Map(additionalInitContainerPropertyKey -> additionalInitContainerPropertyValue) val driverSparkConf = initContainerSpec.driverSparkConf ++ Map(additionalDriverSparkConfKey -> additionalDriverSparkConfValue) initContainerSpec.copy( initContainer = initContainer, - initContainerProperties = initContainerProperties, + properties = initContainerProperties, driverSparkConf = driverSparkConf) } } diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/BaseInitContainerConfigurationStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/BaseInitContainerConfigurationStepSuite.scala index 55a6a0624fd2..5af3f47aaca1 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/BaseInitContainerConfigurationStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/BaseInitContainerConfigurationStepSuite.scala @@ -89,7 +89,7 @@ class BaseInitContainerConfigurationStepSuite extends SparkFunSuite with BeforeA new Pod, Seq.empty[HasMetadata]) val returnContainerSpec = baseInitStep.configureInitContainer(initContainerSpec) - assert(expectedDriverSparkConf === returnContainerSpec.initContainerProperties) + assert(expectedDriverSparkConf === returnContainerSpec.properties) assert(returnContainerSpec.initContainer.getName == INIT_CONTAINER_NAME) assert(returnContainerSpec.driverContainer.getName == DRIVER_CONTAINER_NAME) assert(returnContainerSpec.driverPod.getMetadata.getLabels.asScala === POD_LABEL) diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerConfigOrchestratorSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerConfigOrchestratorSuite.scala index 68302d256cbe..96abecb807aa 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerConfigOrchestratorSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerConfigOrchestratorSuite.scala @@ -66,8 +66,8 @@ class InitContainerConfigOrchestratorSuite extends SparkFunSuite { INIT_CONTAINER_CONFIG_MAP_KEY, sparkConf) val initSteps : Seq[InitContainerConfigurationStep] = - orchestrator.getAllConfigurationSteps() - assert(initSteps.length == 1) + orchestrator.getAllConfigurationSteps + assert(initSteps.lengthCompare(1) == 0) assert(initSteps.head.isInstanceOf[BaseInitContainerConfigurationStep]) } @@ -90,7 +90,7 @@ class InitContainerConfigOrchestratorSuite extends SparkFunSuite { INIT_CONTAINER_CONFIG_MAP_KEY, sparkConf) val initSteps : Seq[InitContainerConfigurationStep] = - orchestrator.getAllConfigurationSteps() + orchestrator.getAllConfigurationSteps assert(initSteps.length === 2) assert(initSteps.head.isInstanceOf[BaseInitContainerConfigurationStep]) assert(initSteps(1).isInstanceOf[InitContainerMountSecretsStep]) From c21fdcf277f031d8a2d754df22e18f5d81881eaf Mon Sep 17 00:00:00 2001 From: Yinan Li Date: Mon, 18 Dec 2017 21:04:47 -0800 Subject: [PATCH 07/14] Removed traits that have only a single implementation --- .../deploy/k8s/InitContainerBootstrap.scala | 17 +++++-------- .../deploy/k8s/MountSecretsBootstrap.scala | 10 ++------ .../k8s/submit/DriverConfigOrchestrator.scala | 5 ++-- .../InitContainerConfigOrchestrator.scala | 7 +++--- .../cluster/k8s/ExecutorPodFactory.scala | 24 +++++-------------- .../k8s/KubernetesClusterManager.scala | 10 ++++---- .../steps/DriverMountSecretsStepSuite.scala | 4 ++-- .../InitContainerMountSecretsStepSuite.scala | 4 ++-- .../cluster/k8s/ExecutorPodFactorySuite.scala | 18 +++++++------- 9 files changed, 37 insertions(+), 62 deletions(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/InitContainerBootstrap.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/InitContainerBootstrap.scala index af04d0377067..5e2c4caa9bac 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/InitContainerBootstrap.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/InitContainerBootstrap.scala @@ -28,14 +28,7 @@ import org.apache.spark.deploy.k8s.Constants._ * 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 InitContainerBootstrap { - /** - * Bootstraps an init-container that downloads dependencies to be used by a main container. - */ - def bootstrapInitContainer(original: PodWithDetachedInitContainer): PodWithDetachedInitContainer -} - -private[spark] class InitContainerBootstrapImpl( +private[spark] class InitContainerBootstrap( initContainerImage: String, imagePullPolicy: String, jarsDownloadPath: String, @@ -44,10 +37,12 @@ private[spark] class InitContainerBootstrapImpl( configMapName: String, configMapKey: String, sparkRole: String, - sparkConf: SparkConf) - extends InitContainerBootstrap { + sparkConf: SparkConf) { - override def bootstrapInitContainer( + /** + * Bootstraps an init-container that downloads dependencies to be used by a main container. + */ + def bootstrapInitContainer( original: PodWithDetachedInitContainer): PodWithDetachedInitContainer = { val sharedVolumeMounts = Seq[VolumeMount]( new VolumeMountBuilder() diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/MountSecretsBootstrap.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/MountSecretsBootstrap.scala index a1f159ac1031..3bad08d3907d 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/MountSecretsBootstrap.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/MountSecretsBootstrap.scala @@ -21,7 +21,7 @@ import io.fabric8.kubernetes.api.model.{Container, ContainerBuilder, Pod, PodBui /** * Bootstraps a driver or executor container or an init-container with needed secrets mounted. */ -private[spark] trait MountSecretsBootstrap { +private[spark] class MountSecretsBootstrap(secretNamesToMountPaths: Map[String, String]) { /** * Mounts Kubernetes secrets as secret volumes into the given container in the given pod. @@ -30,13 +30,7 @@ private[spark] trait MountSecretsBootstrap { * @param container the container into which the secret volumes are being mounted. * @return the updated pod and container with the secrets mounted. */ - def mountSecrets(pod: Pod, container: Container): (Pod, Container) -} - -private[spark] class MountSecretsBootstrapImpl( - secretNamesToMountPaths: Map[String, String]) extends MountSecretsBootstrap { - - override def mountSecrets(pod: Pod, container: Container): (Pod, Container) = { + def mountSecrets(pod: Pod, container: Container): (Pod, Container) = { var podBuilder = new PodBuilder(pod) secretNamesToMountPaths.keys.foreach { name => podBuilder = podBuilder diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/DriverConfigOrchestrator.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/DriverConfigOrchestrator.scala index 7732a88b0806..8bd2d500bfbf 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/DriverConfigOrchestrator.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/DriverConfigOrchestrator.scala @@ -21,7 +21,7 @@ import java.util.UUID import com.google.common.primitives.Longs import org.apache.spark.SparkConf -import org.apache.spark.deploy.k8s.{ConfigurationUtils, MountSecretsBootstrapImpl} +import org.apache.spark.deploy.k8s.{ConfigurationUtils, MountSecretsBootstrap} import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.deploy.k8s.submit.steps._ @@ -148,8 +148,7 @@ private[spark] class DriverConfigOrchestrator( } val mayBeMountSecretsStep = if (secretNamesToMountPaths.nonEmpty) { - val mountSecretsBootstrap = new MountSecretsBootstrapImpl(secretNamesToMountPaths) - Some(new DriverMountSecretsStep(mountSecretsBootstrap)) + Some(new DriverMountSecretsStep(new MountSecretsBootstrap(secretNamesToMountPaths))) } else { None } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerConfigOrchestrator.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerConfigOrchestrator.scala index f3ea757fbe8b..f2205ee1e62c 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerConfigOrchestrator.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerConfigOrchestrator.scala @@ -17,7 +17,7 @@ package org.apache.spark.deploy.k8s.submit.steps.initcontainer import org.apache.spark.{SparkConf, SparkException} -import org.apache.spark.deploy.k8s.{ConfigurationUtils, InitContainerBootstrapImpl, MountSecretsBootstrapImpl} +import org.apache.spark.deploy.k8s.{ConfigurationUtils, InitContainerBootstrap, MountSecretsBootstrap} import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ @@ -45,7 +45,7 @@ private[spark] class InitContainerConfigOrchestrator( private val downloadTimeoutMinutes = sparkConf.get(INIT_CONTAINER_MOUNT_TIMEOUT) def getAllConfigurationSteps: Seq[InitContainerConfigurationStep] = { - val initContainerBootstrap = new InitContainerBootstrapImpl( + val initContainerBootstrap = new InitContainerBootstrap( initContainerImage, imagePullPolicy, jarsDownloadPath, @@ -71,8 +71,7 @@ private[spark] class InitContainerConfigOrchestrator( // because the init-container is sort of an implementation details and this sharing // avoids introducing a dedicated configuration property just for the init-container. val maybeMountSecretsStep = if (secretNamesToMountPaths.nonEmpty) { - val mountSecretsBootstrap = new MountSecretsBootstrapImpl(secretNamesToMountPaths) - Some(new InitContainerMountSecretsStep(mountSecretsBootstrap)) + Some(new InitContainerMountSecretsStep(new MountSecretsBootstrap(secretNamesToMountPaths))) } else { None } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala index 55eda3b39986..b76f77717f6e 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala @@ -31,26 +31,11 @@ import org.apache.spark.util.Utils /** * A factory class for configuring and creating executor pods. */ -private[spark] trait ExecutorPodFactory { - - /** - * Configure and construct an executor pod with the given parameters. - */ - def createExecutorPod( - executorId: String, - applicationId: String, - driverUrl: String, - executorEnvs: Seq[(String, String)], - driverPod: Pod, - nodeToLocalTaskCount: Map[String, Int]): Pod -} - -private[spark] class ExecutorPodFactoryImpl( +private[spark] class ExecutorPodFactory( sparkConf: SparkConf, mountSecretsBootstrap: Option[MountSecretsBootstrap], initContainerBootstrap: Option[InitContainerBootstrap], - initContainerMountSecretsBootstrap: Option[MountSecretsBootstrap]) - extends ExecutorPodFactory { + initContainerMountSecretsBootstrap: Option[MountSecretsBootstrap]) { private val executorExtraClasspath = sparkConf.get(EXECUTOR_CLASS_PATH) @@ -99,7 +84,10 @@ private[spark] class ExecutorPodFactoryImpl( private val executorCores = sparkConf.getDouble("spark.executor.cores", 1) private val executorLimitCores = sparkConf.get(KUBERNETES_EXECUTOR_LIMIT_CORES) - override def createExecutorPod( + /** + * Configure and construct an executor pod with the given parameters. + */ + def createExecutorPod( executorId: String, applicationId: String, driverUrl: String, diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala index db4b38ecb95b..ed95a70195c7 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala @@ -21,7 +21,7 @@ import java.io.File import io.fabric8.kubernetes.client.Config import org.apache.spark.{SparkContext, SparkException} -import org.apache.spark.deploy.k8s.{ConfigurationUtils, InitContainerBootstrapImpl, MountSecretsBootstrapImpl, SparkKubernetesClientFactory} +import org.apache.spark.deploy.k8s.{ConfigurationUtils, InitContainerBootstrap, MountSecretsBootstrap, SparkKubernetesClientFactory} import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.internal.Logging @@ -69,7 +69,7 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit .get(INIT_CONTAINER_IMAGE) .getOrElse(throw new SparkException( "Must specify the init-container image when there are remote dependencies")) - new InitContainerBootstrapImpl( + new InitContainerBootstrap( initContainerImage, sparkConf.get(CONTAINER_IMAGE_PULL_POLICY), sparkConf.get(JARS_DOWNLOAD_LOCATION), @@ -84,7 +84,7 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit val executorSecretNamesToMountPaths = ConfigurationUtils.parsePrefixedKeyValuePairs( sparkConf, KUBERNETES_EXECUTOR_SECRETS_PREFIX) val mayBeMountSecretBootstrap = if (executorSecretNamesToMountPaths.nonEmpty) { - Some(new MountSecretsBootstrapImpl(executorSecretNamesToMountPaths)) + Some(new MountSecretsBootstrap(executorSecretNamesToMountPaths)) } else { None } @@ -95,7 +95,7 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit // avoids introducing a dedicated configuration property just for the init-container. val mayBeInitContainerMountSecretsBootstrap = if (maybeInitContainerBootstrap.nonEmpty && executorSecretNamesToMountPaths.nonEmpty) { - Some(new MountSecretsBootstrapImpl(executorSecretNamesToMountPaths)) + Some(new MountSecretsBootstrap(executorSecretNamesToMountPaths)) } else { None } @@ -108,7 +108,7 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit Some(new File(Config.KUBERNETES_SERVICE_ACCOUNT_TOKEN_PATH)), Some(new File(Config.KUBERNETES_SERVICE_ACCOUNT_CA_CRT_PATH))) - val executorPodFactory = new ExecutorPodFactoryImpl( + val executorPodFactory = new ExecutorPodFactory( sparkConf, mayBeMountSecretBootstrap, maybeInitContainerBootstrap, diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/DriverMountSecretsStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/DriverMountSecretsStepSuite.scala index 9a40bb859143..9f2c39939a7c 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/DriverMountSecretsStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/DriverMountSecretsStepSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.deploy.k8s.submit.steps import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.deploy.k8s.MountSecretsBootstrapImpl +import org.apache.spark.deploy.k8s.MountSecretsBootstrap import org.apache.spark.deploy.k8s.submit.{KubernetesDriverSpec, SecretVolumeUtils} class DriverMountSecretsStepSuite extends SparkFunSuite { @@ -32,7 +32,7 @@ class DriverMountSecretsStepSuite extends SparkFunSuite { SECRET_FOO -> SECRET_MOUNT_PATH, SECRET_BAR -> SECRET_MOUNT_PATH) - val mountSecretsBootstrap = new MountSecretsBootstrapImpl(secretNamesToMountPaths) + val mountSecretsBootstrap = new MountSecretsBootstrap(secretNamesToMountPaths) val mountSecretsStep = new DriverMountSecretsStep(mountSecretsBootstrap) val configuredDriverSpec = mountSecretsStep.configureDriver(baseDriverSpec) val driverPodWithSecretsMounted = configuredDriverSpec.driverPod diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerMountSecretsStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerMountSecretsStepSuite.scala index ee6c9c4745a4..db49bd303cd0 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerMountSecretsStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerMountSecretsStepSuite.scala @@ -19,7 +19,7 @@ package org.apache.spark.deploy.k8s.submit.steps.initcontainer import io.fabric8.kubernetes.api.model.{ContainerBuilder, PodBuilder} import org.apache.spark.SparkFunSuite -import org.apache.spark.deploy.k8s.MountSecretsBootstrapImpl +import org.apache.spark.deploy.k8s.MountSecretsBootstrap import org.apache.spark.deploy.k8s.submit.SecretVolumeUtils class InitContainerMountSecretsStepSuite extends SparkFunSuite { @@ -40,7 +40,7 @@ class InitContainerMountSecretsStepSuite extends SparkFunSuite { SECRET_FOO -> SECRET_MOUNT_PATH, SECRET_BAR -> SECRET_MOUNT_PATH) - val mountSecretsBootstrap = new MountSecretsBootstrapImpl(secretNamesToMountPaths) + val mountSecretsBootstrap = new MountSecretsBootstrap(secretNamesToMountPaths) val initContainerMountSecretsStep = new InitContainerMountSecretsStep(mountSecretsBootstrap) val configuredInitContainerSpec = initContainerMountSecretsStep.configureInitContainer( baseInitContainerSpec) diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactorySuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactorySuite.scala index 96915ac7a930..7121a802c69c 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactorySuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactorySuite.scala @@ -25,7 +25,7 @@ import org.mockito.Mockito._ import org.scalatest.{BeforeAndAfter, BeforeAndAfterEach} import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.deploy.k8s.{InitContainerBootstrap, MountSecretsBootstrapImpl, PodWithDetachedInitContainer} +import org.apache.spark.deploy.k8s.{InitContainerBootstrap, MountSecretsBootstrap, PodWithDetachedInitContainer} import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ @@ -58,7 +58,7 @@ class ExecutorPodFactorySuite extends SparkFunSuite with BeforeAndAfter with Bef } test("basic executor pod has reasonable defaults") { - val factory = new ExecutorPodFactoryImpl(baseConf, None, None, None) + val factory = new ExecutorPodFactory(baseConf, None, None, None) val executor = factory.createExecutorPod( "1", "dummy", "dummy", Seq[(String, String)](), driverPod, Map[String, Int]()) @@ -89,7 +89,7 @@ class ExecutorPodFactorySuite extends SparkFunSuite with BeforeAndAfter with Bef conf.set(KUBERNETES_EXECUTOR_POD_NAME_PREFIX, "loremipsumdolorsitametvimatelitrefficiendisuscipianturvixlegeresple") - val factory = new ExecutorPodFactoryImpl(conf, None, None, None) + val factory = new ExecutorPodFactory(conf, None, None, None) val executor = factory.createExecutorPod( "1", "dummy", "dummy", Seq[(String, String)](), driverPod, Map[String, Int]()) @@ -101,7 +101,7 @@ class ExecutorPodFactorySuite extends SparkFunSuite with BeforeAndAfter with Bef conf.set(org.apache.spark.internal.config.EXECUTOR_JAVA_OPTIONS, "foo=bar") conf.set(org.apache.spark.internal.config.EXECUTOR_CLASS_PATH, "bar=baz") - val factory = new ExecutorPodFactoryImpl(conf, None, None, None) + val factory = new ExecutorPodFactory(conf, None, None, None) val executor = factory.createExecutorPod( "1", "dummy", "dummy", Seq[(String, String)]("qux" -> "quux"), driverPod, Map[String, Int]()) @@ -115,8 +115,8 @@ class ExecutorPodFactorySuite extends SparkFunSuite with BeforeAndAfter with Bef test("executor secrets get mounted") { val conf = baseConf.clone() - val secretsBootstrap = new MountSecretsBootstrapImpl(Map("secret1" -> "/var/secret1")) - val factory = new ExecutorPodFactoryImpl( + val secretsBootstrap = new MountSecretsBootstrap(Map("secret1" -> "/var/secret1")) + val factory = new ExecutorPodFactory( conf, Some(secretsBootstrap), None, @@ -144,7 +144,7 @@ class ExecutorPodFactorySuite extends SparkFunSuite with BeforeAndAfter with Bef when(initContainerBootstrap.bootstrapInitContainer( any(classOf[PodWithDetachedInitContainer]))).thenAnswer(AdditionalAnswers.returnsFirstArg()) - val factory = new ExecutorPodFactoryImpl( + val factory = new ExecutorPodFactory( conf, None, Some(initContainerBootstrap), @@ -161,9 +161,9 @@ class ExecutorPodFactorySuite extends SparkFunSuite with BeforeAndAfter with Bef val initContainerBootstrap = mock(classOf[InitContainerBootstrap]) when(initContainerBootstrap.bootstrapInitContainer( any(classOf[PodWithDetachedInitContainer]))).thenAnswer(AdditionalAnswers.returnsFirstArg()) - val secretsBootstrap = new MountSecretsBootstrapImpl(Map("secret1" -> "/var/secret1")) + val secretsBootstrap = new MountSecretsBootstrap(Map("secret1" -> "/var/secret1")) - val factory = new ExecutorPodFactoryImpl( + val factory = new ExecutorPodFactory( conf, None, Some(initContainerBootstrap), From a3cd71d86fb8047ed0284ce6a245bf9e12179211 Mon Sep 17 00:00:00 2001 From: Yinan Li Date: Tue, 19 Dec 2017 09:27:48 -0800 Subject: [PATCH 08/14] Remove unused class arguments --- .../deploy/k8s/InitContainerBootstrap.scala | 1 - .../k8s/submit/DriverConfigOrchestrator.scala | 13 ++++++------- .../k8s/submit/KubernetesClientApplication.scala | 1 - .../BaseInitContainerConfigurationStep.scala | 2 +- .../InitContainerConfigOrchestrator.scala | 11 ++++------- .../cluster/k8s/KubernetesClusterManager.scala | 1 - .../submit/DriverConfigOrchestratorSuite.scala | 5 ----- ...BaseInitContainerConfigurationStepSuite.scala | 2 -- .../InitContainerConfigOrchestratorSuite.scala | 16 ---------------- 9 files changed, 11 insertions(+), 41 deletions(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/InitContainerBootstrap.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/InitContainerBootstrap.scala index 5e2c4caa9bac..6887cf8f43ec 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/InitContainerBootstrap.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/InitContainerBootstrap.scala @@ -33,7 +33,6 @@ private[spark] class InitContainerBootstrap( imagePullPolicy: String, jarsDownloadPath: String, filesDownloadPath: String, - downloadTimeoutMinutes: Long, configMapName: String, configMapKey: String, sparkRole: String, diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/DriverConfigOrchestrator.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/DriverConfigOrchestrator.scala index 8bd2d500bfbf..1076946e8193 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/DriverConfigOrchestrator.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/DriverConfigOrchestrator.scala @@ -31,10 +31,12 @@ import org.apache.spark.util.SystemClock import org.apache.spark.util.Utils /** - * Constructs the complete list of driver configuration steps to run to deploy the Spark driver. + * Figures out and returns the complete ordered list of needed DriverConfigurationSteps to + * configure the Spark driver pod. The returned steps will be applied one by one in the given + * order to produce a final KubernetesDriverSpec that is used in KubernetesClientApplication + * to construct and create the driver pod. */ private[spark] class DriverConfigOrchestrator( - namespace: String, kubernetesAppId: String, launchTime: Long, mainAppResource: Option[MainAppResource], @@ -85,7 +87,7 @@ private[spark] class DriverConfigOrchestrator( appArgs, sparkConf) - val driverAddressStep = new DriverServiceBootstrapStep( + val serviceBootstrapStep = new DriverServiceBootstrapStep( kubernetesResourceNamePrefix, allDriverLabels, sparkConf, @@ -126,14 +128,11 @@ private[spark] class DriverConfigOrchestrator( val mayBeInitContainerBootstrapStep = if (areAnyFilesNonContainerLocal(sparkJars ++ sparkFiles)) { val orchestrator = new InitContainerConfigOrchestrator( - namespace, - kubernetesResourceNamePrefix, sparkJars, sparkFiles, jarsDownloadPath, filesDownloadPath, imagePullPolicy, - allDriverLabels, initContainerConfigMapName, INIT_CONTAINER_PROPERTIES_FILE_NAME, sparkConf) @@ -155,7 +154,7 @@ private[spark] class DriverConfigOrchestrator( Seq( initialSubmissionStep, - driverAddressStep, + serviceBootstrapStep, kubernetesCredentialsStep) ++ maybeDependencyResolutionStep.toSeq ++ mayBeInitContainerBootstrapStep.toSeq ++ diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesClientApplication.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesClientApplication.scala index 1b9f4e04b6fb..5884348cb3e4 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesClientApplication.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesClientApplication.scala @@ -210,7 +210,6 @@ private[spark] class KubernetesClientApplication extends SparkApplication { val watcher = new LoggingPodStatusWatcherImpl(kubernetesAppId, loggingInterval) val orchestrator = new DriverConfigOrchestrator( - namespace, kubernetesAppId, launchTime, clientArguments.mainAppResource, diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/BaseInitContainerConfigurationStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/BaseInitContainerConfigurationStep.scala index 4be6ca288688..83616ab2fea4 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/BaseInitContainerConfigurationStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/BaseInitContainerConfigurationStep.scala @@ -57,6 +57,6 @@ private[spark] class BaseInitContainerConfigurationStep( initContainer = bootstrapped.initContainer, driverContainer = bootstrapped.mainContainer, driverPod = bootstrapped.pod, - properties = baseInitContainerConfig) + properties = spec.properties ++ baseInitContainerConfig) } } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerConfigOrchestrator.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerConfigOrchestrator.scala index f2205ee1e62c..d217dfdcfc50 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerConfigOrchestrator.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerConfigOrchestrator.scala @@ -22,18 +22,17 @@ import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ /** - * Returns the complete ordered list of steps required to configure the init-container. This is - * only used when there are remote application dependencies to localize. + * Figures out and returns the complete ordered list of InitContainerConfigurationSteps required to + * configure an init-container. The returned steps will be applied in the given order to produce a + * final InitContainerSpec that is used to construct an init-container. This is only used when an + * init-container is needed, i.e., when there are remote application dependencies to localize. */ private[spark] class InitContainerConfigOrchestrator( - namespace: String, - resourceNamePrefix: String, sparkJars: Seq[String], sparkFiles: Seq[String], jarsDownloadPath: String, filesDownloadPath: String, imagePullPolicy: String, - driverLabels: Map[String, String], configMapName: String, configMapKey: String, sparkConf: SparkConf) { @@ -42,7 +41,6 @@ private[spark] class InitContainerConfigOrchestrator( .get(INIT_CONTAINER_IMAGE) .getOrElse(throw new SparkException( "Must specify the init-container image when there are remote dependencies")) - private val downloadTimeoutMinutes = sparkConf.get(INIT_CONTAINER_MOUNT_TIMEOUT) def getAllConfigurationSteps: Seq[InitContainerConfigurationStep] = { val initContainerBootstrap = new InitContainerBootstrap( @@ -50,7 +48,6 @@ private[spark] class InitContainerConfigOrchestrator( imagePullPolicy, jarsDownloadPath, filesDownloadPath, - downloadTimeoutMinutes, configMapName, configMapKey, SPARK_POD_DRIVER_ROLE, diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala index ed95a70195c7..c07580d9e220 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala @@ -74,7 +74,6 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit sparkConf.get(CONTAINER_IMAGE_PULL_POLICY), sparkConf.get(JARS_DOWNLOAD_LOCATION), sparkConf.get(FILES_DOWNLOAD_LOCATION), - sparkConf.get(INIT_CONTAINER_MOUNT_TIMEOUT), configMap, configMapKey, SPARK_POD_EXECUTOR_ROLE, diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/DriverConfigOrchestratorSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/DriverConfigOrchestratorSuite.scala index 53dfe21410e9..c03838999a18 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/DriverConfigOrchestratorSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/DriverConfigOrchestratorSuite.scala @@ -22,7 +22,6 @@ import org.apache.spark.deploy.k8s.submit.steps._ class DriverConfigOrchestratorSuite extends SparkFunSuite { - private val NAMESPACE = "default" private val DRIVER_IMAGE = "driver-image" private val IC_IMAGE = "init-container-image" private val APP_ID = "spark-app-id" @@ -39,7 +38,6 @@ class DriverConfigOrchestratorSuite extends SparkFunSuite { .set(DRIVER_CONTAINER_IMAGE, DRIVER_IMAGE) val mainAppResource = JavaMainAppResource("local:///var/apps/jars/main.jar") val orchestrator = new DriverConfigOrchestrator( - NAMESPACE, APP_ID, LAUNCH_TIME, Some(mainAppResource), @@ -60,7 +58,6 @@ class DriverConfigOrchestratorSuite extends SparkFunSuite { val sparkConf = new SparkConf(false) .set(DRIVER_CONTAINER_IMAGE, DRIVER_IMAGE) val orchestrator = new DriverConfigOrchestrator( - NAMESPACE, APP_ID, LAUNCH_TIME, Option.empty, @@ -83,7 +80,6 @@ class DriverConfigOrchestratorSuite extends SparkFunSuite { .set("spark.jars", "hdfs://localhost:9000/var/apps/jars/jar1.jar") val mainAppResource = JavaMainAppResource("local:///var/apps/jars/main.jar") val orchestrator = new DriverConfigOrchestrator( - NAMESPACE, APP_ID, LAUNCH_TIME, Some(mainAppResource), @@ -107,7 +103,6 @@ class DriverConfigOrchestratorSuite extends SparkFunSuite { .set(s"$KUBERNETES_DRIVER_SECRETS_PREFIX$SECRET_BAR", SECRET_MOUNT_PATH) val mainAppResource = JavaMainAppResource("local:///var/apps/jars/main.jar") val orchestrator = new DriverConfigOrchestrator( - NAMESPACE, APP_ID, LAUNCH_TIME, Some(mainAppResource), diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/BaseInitContainerConfigurationStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/BaseInitContainerConfigurationStepSuite.scala index 5af3f47aaca1..05f1a54d45b5 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/BaseInitContainerConfigurationStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/BaseInitContainerConfigurationStepSuite.scala @@ -38,8 +38,6 @@ class BaseInitContainerConfigurationStepSuite extends SparkFunSuite with BeforeA "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" diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerConfigOrchestratorSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerConfigOrchestratorSuite.scala index 96abecb807aa..ddd4bf67dc27 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerConfigOrchestratorSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerConfigOrchestratorSuite.scala @@ -22,9 +22,7 @@ import org.apache.spark.deploy.k8s.Constants._ class InitContainerConfigOrchestratorSuite extends SparkFunSuite { - private val NAMESPACE = "namespace" private val DOCKER_IMAGE = "init-container" - 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( @@ -32,16 +30,8 @@ class InitContainerConfigOrchestratorSuite extends SparkFunSuite { 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 SECRET_FOO = "foo" @@ -54,14 +44,11 @@ class InitContainerConfigOrchestratorSuite extends SparkFunSuite { .set(s"$KUBERNETES_DRIVER_LABEL_PREFIX$CUSTOM_LABEL_KEY", CUSTOM_LABEL_VALUE) val orchestrator = new InitContainerConfigOrchestrator( - NAMESPACE, - APP_RESOURCE_PREFIX, SPARK_JARS.take(1), 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) @@ -78,14 +65,11 @@ class InitContainerConfigOrchestratorSuite extends SparkFunSuite { .set(s"$KUBERNETES_DRIVER_SECRETS_PREFIX$SECRET_BAR", SECRET_MOUNT_PATH) val orchestrator = new InitContainerConfigOrchestrator( - NAMESPACE, - APP_RESOURCE_PREFIX, SPARK_JARS.take(1), 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) From 23c5cd9afe8765c322ec10c3d9f5057aeca2e7b7 Mon Sep 17 00:00:00 2001 From: Yinan Li Date: Tue, 19 Dec 2017 10:31:54 -0800 Subject: [PATCH 09/14] Improved documentation --- .../spark/deploy/k8s/InitContainerBootstrap.scala | 5 +++-- .../k8s/submit/DriverConfigOrchestrator.scala | 4 +++- .../submit/steps/BaseDriverConfigurationStep.scala | 2 +- .../k8s/submit/steps/DriverConfigurationStep.scala | 2 +- .../steps/DriverInitContainerBootstrapStep.scala | 6 ++++-- .../BaseInitContainerConfigurationStep.scala | 5 +++++ .../InitContainerConfigOrchestrator.scala | 7 ++++--- .../InitContainerConfigurationStep.scala | 2 +- .../scheduler/cluster/k8s/ExecutorPodFactory.scala | 13 ++++++++++++- 9 files changed, 34 insertions(+), 12 deletions(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/InitContainerBootstrap.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/InitContainerBootstrap.scala index 6887cf8f43ec..501708823d7c 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/InitContainerBootstrap.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/InitContainerBootstrap.scala @@ -25,8 +25,9 @@ import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ /** - * 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. + * Bootstraps an init-container for downloading remote dependencies. This is separated out from + * the init-container steps API because this component can be used to bootstrap init-containers + * for both the driver and executors. */ private[spark] class InitContainerBootstrap( initContainerImage: String, diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/DriverConfigOrchestrator.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/DriverConfigOrchestrator.scala index 1076946e8193..69b3f313049f 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/DriverConfigOrchestrator.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/DriverConfigOrchestrator.scala @@ -34,7 +34,9 @@ import org.apache.spark.util.Utils * Figures out and returns the complete ordered list of needed DriverConfigurationSteps to * configure the Spark driver pod. The returned steps will be applied one by one in the given * order to produce a final KubernetesDriverSpec that is used in KubernetesClientApplication - * to construct and create the driver pod. + * to construct and create the driver pod. It uses the InitContainerConfigOrchestrator to + * configure the driver init-container if one is needed, i.e., when there are remote dependencies + * to localize. */ private[spark] class DriverConfigOrchestrator( kubernetesAppId: String, diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/BaseDriverConfigurationStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/BaseDriverConfigurationStep.scala index e6fe596b8b8e..b1de775011ca 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/BaseDriverConfigurationStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/BaseDriverConfigurationStep.scala @@ -28,7 +28,7 @@ import org.apache.spark.deploy.k8s.submit.KubernetesDriverSpec import org.apache.spark.internal.config.{DRIVER_CLASS_PATH, DRIVER_MEMORY, DRIVER_MEMORY_OVERHEAD} /** - * Represents the initial setup required for the driver. + * Performs basic configuration for the driver pod. */ private[spark] class BaseDriverConfigurationStep( kubernetesAppId: String, diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DriverConfigurationStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DriverConfigurationStep.scala index c99c0436cf25..17614e040e58 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DriverConfigurationStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DriverConfigurationStep.scala @@ -19,7 +19,7 @@ package org.apache.spark.deploy.k8s.submit.steps import org.apache.spark.deploy.k8s.submit.KubernetesDriverSpec /** - * Represents a step in preparing the Kubernetes driver. + * Represents a step in configuring the Spark driver pod. */ private[spark] trait DriverConfigurationStep { diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DriverInitContainerBootstrapStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DriverInitContainerBootstrapStep.scala index 6ce51220139e..40bc1bd1c945 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DriverInitContainerBootstrapStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DriverInitContainerBootstrapStep.scala @@ -26,8 +26,10 @@ import org.apache.spark.deploy.k8s.submit.{InitContainerUtil, KubernetesDriverSp import org.apache.spark.deploy.k8s.submit.steps.initcontainer.{InitContainerConfigurationStep, InitContainerSpec} /** - * Configures the init-container that bootstraps dependencies into the driver pod, including - * building a ConfigMap that will be mounted into the init-container. The ConfigMap carries + * Configures the driver init-container that localizes remote dependencies into the driver pod. + * It applies the given InitContainerConfigurationSteps in the given order to produce a final + * InitContainerSpec that is then used to configure the driver pod with the init-container attached. + * It also builds a ConfigMap that will be mounted into the init-container. The ConfigMap carries * configuration properties for the init-container. */ private[spark] class DriverInitContainerBootstrapStep( diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/BaseInitContainerConfigurationStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/BaseInitContainerConfigurationStep.scala index 83616ab2fea4..8ca3e023ef98 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/BaseInitContainerConfigurationStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/BaseInitContainerConfigurationStep.scala @@ -20,6 +20,10 @@ import org.apache.spark.deploy.k8s.{InitContainerBootstrap, PodWithDetachedInitC import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.submit.KubernetesFileUtils +/** + * Performs basic configuration for the driver init-container with most of the work delegated to + * the given InitContainerBootstrap. + */ private[spark] class BaseInitContainerConfigurationStep( sparkJars: Seq[String], sparkFiles: Seq[String], @@ -47,6 +51,7 @@ private[spark] class BaseInitContainerConfigurationStep( FILES_DOWNLOAD_LOCATION.key -> filesDownloadPath) ++ remoteJarsConf ++ remoteFilesConf + val bootstrapped = bootstrap.bootstrapInitContainer( PodWithDetachedInitContainer( spec.driverPod, diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerConfigOrchestrator.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerConfigOrchestrator.scala index d217dfdcfc50..8c50c515618b 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerConfigOrchestrator.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerConfigOrchestrator.scala @@ -23,9 +23,10 @@ import org.apache.spark.deploy.k8s.Constants._ /** * Figures out and returns the complete ordered list of InitContainerConfigurationSteps required to - * configure an init-container. The returned steps will be applied in the given order to produce a - * final InitContainerSpec that is used to construct an init-container. This is only used when an - * init-container is needed, i.e., when there are remote application dependencies to localize. + * configure the driver init-container. The returned steps will be applied in the given order to + * produce a final InitContainerSpec that is used to construct the driver init-container in + * DriverInitContainerBootstrapStep. This class is only used when an init-container is needed, i.e., + * when there are remote application dependencies to localize. */ private[spark] class InitContainerConfigOrchestrator( sparkJars: Seq[String], diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerConfigurationStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerConfigurationStep.scala index 5a744cb86524..0372ad527095 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerConfigurationStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerConfigurationStep.scala @@ -17,7 +17,7 @@ package org.apache.spark.deploy.k8s.submit.steps.initcontainer /** - * Represents a step in configuring the init-container for the driver and executors. + * Represents a step in configuring the driver init-container. */ private[spark] trait InitContainerConfigurationStep { diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala index b76f77717f6e..9e239d6b832d 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala @@ -29,7 +29,18 @@ import org.apache.spark.internal.config.{EXECUTOR_CLASS_PATH, EXECUTOR_JAVA_OPTI import org.apache.spark.util.Utils /** - * A factory class for configuring and creating executor pods. + * A factory class for bootstrapping and creating executor pods with the given bootstrapping + * components. + * + * @param sparkConf Spark configuration + * @param mountSecretsBootstrap an optional component for mounting user-specified secrets onto + * user-specified paths into the executor container + * @param initContainerBootstrap an optional component for bootstrapping the executor init-container + * if one is needed, i.e., when there are remote dependencies to + * localize + * @param initContainerMountSecretsBootstrap an optional component for mounting user-specified + * secrets onto user-specified paths into the executor + * init-container */ private[spark] class ExecutorPodFactory( sparkConf: SparkConf, From 2ec15c489a7a0429a0b3064ea50f61c75020685b Mon Sep 17 00:00:00 2001 From: Yinan Li Date: Tue, 19 Dec 2017 16:52:46 -0800 Subject: [PATCH 10/14] Addressed latest round of comments --- .../org/apache/spark/deploy/k8s/Config.scala | 7 ++ .../spark/deploy/k8s/ConfigurationUtils.scala | 41 ------------ .../deploy/k8s/InitContainerBootstrap.scala | 26 ++++---- ...sFileUtils.scala => KubernetesUtils.scala} | 40 +++++++++++- .../deploy/k8s/MountSecretsBootstrap.scala | 13 ++-- .../k8s/SparkKubernetesClientFactory.scala | 2 +- .../k8s/submit/DriverConfigOrchestrator.scala | 65 +++++++++---------- .../deploy/k8s/submit/InitContainerUtil.scala | 37 ----------- ...ala => BasicDriverConfigurationStep.scala} | 8 +-- .../steps/DependencyResolutionStep.scala | 9 +-- .../DriverInitContainerBootstrapStep.scala | 5 +- ...BasicInitContainerConfigurationStep.scala} | 8 +-- .../InitContainerConfigOrchestrator.scala | 15 ++--- .../rest/k8s/SparkConfPropertiesParser.scala | 50 -------------- .../rest/k8s/SparkPodInitContainer.scala | 21 +++--- .../cluster/k8s/ExecutorPodFactory.scala | 11 ++-- .../k8s/KubernetesClusterManager.scala | 28 ++++---- .../DriverConfigOrchestratorSuite.scala | 8 +-- ...> BasicDriverConfigurationStepSuite.scala} | 4 +- ...InitContainerConfigurationStepSuite.scala} | 4 +- ...InitContainerConfigOrchestratorSuite.scala | 4 +- 21 files changed, 161 insertions(+), 245 deletions(-) delete mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/ConfigurationUtils.scala rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/{submit/KubernetesFileUtils.scala => KubernetesUtils.scala} (65%) delete mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/InitContainerUtil.scala rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/{BaseDriverConfigurationStep.scala => BasicDriverConfigurationStep.scala} (95%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/{BaseInitContainerConfigurationStep.scala => BasicInitContainerConfigurationStep.scala} (89%) delete mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/SparkConfPropertiesParser.scala rename resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/{BaseDriverConfigurationStepSuite.scala => BasicDriverConfigurationStepSuite.scala} (97%) rename resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/{BaseInitContainerConfigurationStepSuite.scala => BasicInitContainerConfigurationStepSuite.scala} (96%) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala index c34028a2280d..0655be0ad80e 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala @@ -159,6 +159,13 @@ private[spark] object Config extends Logging { .timeConf(TimeUnit.MINUTES) .createWithDefault(5) + val INIT_CONTAINER_MAX_THREAD_POOL_SIZE = + ConfigBuilder("spark.kubernetes.initContainer.maxThreadPoolSize") + .doc("Maximum size of the thread pool in the init-container for downloading remote " + + "dependencies.") + .intConf + .createWithDefault(5) + val INIT_CONTAINER_REMOTE_JARS = ConfigBuilder("spark.kubernetes.initContainer.remoteJars") .doc("Comma-separated list of jar URIs to download in the init-container. This is " + diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/ConfigurationUtils.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/ConfigurationUtils.scala deleted file mode 100644 index 01717479fddd..000000000000 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/ConfigurationUtils.scala +++ /dev/null @@ -1,41 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.deploy.k8s - -import org.apache.spark.SparkConf - -private[spark] object ConfigurationUtils { - - /** - * Extract and parse Spark configuration properties with a given name prefix and - * return the result as a Map. Keys must not have more than one value. - * - * @param sparkConf Spark configuration - * @param prefix the given property name prefix - * @return a Map storing the configuration property keys and values - */ - def parsePrefixedKeyValuePairs( - sparkConf: SparkConf, - prefix: String): Map[String, String] = { - sparkConf.getAllWithPrefix(prefix).toMap - } - - def requireNandDefined(opt1: Option[_], opt2: Option[_], errMessage: String): Unit = { - opt1.foreach { _ => require(opt2.isEmpty, errMessage) } - } -} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/InitContainerBootstrap.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/InitContainerBootstrap.scala index 501708823d7c..dfeccf9e2bd1 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/InitContainerBootstrap.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/InitContainerBootstrap.scala @@ -59,11 +59,12 @@ private[spark] class InitContainerBootstrap( case SPARK_POD_EXECUTOR_ROLE => "spark.executorEnv." case _ => throw new SparkException(s"$sparkRole is not a valid Spark pod role") } - val customEnvVars = sparkConf.getAllWithPrefix(customEnvVarKeyPrefix).toSeq.map { env => - new EnvVarBuilder() - .withName(env._1) - .withValue(env._2) - .build() + val customEnvVars = sparkConf.getAllWithPrefix(customEnvVarKeyPrefix).toSeq.map { + case (key, value) => + new EnvVarBuilder() + .withName(key) + .withValue(value) + .build() } val initContainer = new ContainerBuilder(original.initContainer) @@ -102,14 +103,13 @@ private[spark] class InitContainerBootstrap( .endSpec() .build() - val mainContainer = new ContainerBuilder( - original.mainContainer) - .addToVolumeMounts(sharedVolumeMounts: _*) - .addNewEnv() - .withName(ENV_MOUNTED_FILES_DIR) - .withValue(filesDownloadPath) - .endEnv() - .build() + val mainContainer = new ContainerBuilder(original.mainContainer) + .addToVolumeMounts(sharedVolumeMounts: _*) + .addNewEnv() + .withName(ENV_MOUNTED_FILES_DIR) + .withValue(filesDownloadPath) + .endEnv() + .build() PodWithDetachedInitContainer( podWithBasicVolumes, diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesFileUtils.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesUtils.scala similarity index 65% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesFileUtils.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesUtils.scala index 8a467b9384dd..55a6ba397a26 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesFileUtils.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesUtils.scala @@ -14,13 +14,49 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.k8s.submit +package org.apache.spark.deploy.k8s import java.io.File +import io.fabric8.kubernetes.api.model.{Container, Pod, PodBuilder} + +import org.apache.spark.SparkConf import org.apache.spark.util.Utils -private[spark] object KubernetesFileUtils { +private[spark] object KubernetesUtils { + + /** + * Extract and parse Spark configuration properties with a given name prefix and + * return the result as a Map. Keys must not have more than one value. + * + * @param sparkConf Spark configuration + * @param prefix the given property name prefix + * @return a Map storing the configuration property keys and values + */ + def parsePrefixedKeyValuePairs( + sparkConf: SparkConf, + prefix: String): Map[String, String] = { + sparkConf.getAllWithPrefix(prefix).toMap + } + + def requireNandDefined(opt1: Option[_], opt2: Option[_], errMessage: String): Unit = { + opt1.foreach { _ => require(opt2.isEmpty, errMessage) } + } + + /** + * Append the given init-container to a pod's list of init-containers.. + * + * @param originalPodSpec original specification of the pod + * @param initContainer the init-container to add to the pod + * @return the pod with the init-container added to the list of InitContainers + */ + def appendInitContainer(originalPodSpec: Pod, initContainer: Container): Pod = { + new PodBuilder(originalPodSpec) + .editOrNewSpec() + .addToInitContainers(initContainer) + .endSpec() + .build() + } /** * For the given collection of file URIs, resolves them as follows: diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/MountSecretsBootstrap.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/MountSecretsBootstrap.scala index 3bad08d3907d..b876d183e28d 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/MountSecretsBootstrap.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/MountSecretsBootstrap.scala @@ -45,12 +45,13 @@ private[spark] class MountSecretsBootstrap(secretNamesToMountPaths: Map[String, } var containerBuilder = new ContainerBuilder(container) - secretNamesToMountPaths.foreach { namePath => - containerBuilder = containerBuilder - .addNewVolumeMount() - .withName(secretVolumeName(namePath._1)) - .withMountPath(namePath._2) - .endVolumeMount() + secretNamesToMountPaths.foreach { + case (name, path) => + containerBuilder = containerBuilder + .addNewVolumeMount() + .withName(secretVolumeName(name)) + .withMountPath(path) + .endVolumeMount() } (podBuilder.build(), containerBuilder.build()) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkKubernetesClientFactory.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkKubernetesClientFactory.scala index 1e3f055e0576..c47e78cbf19e 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkKubernetesClientFactory.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkKubernetesClientFactory.scala @@ -48,7 +48,7 @@ private[spark] object SparkKubernetesClientFactory { .map(new File(_)) .orElse(defaultServiceAccountToken) val oauthTokenValue = sparkConf.getOption(oauthTokenConf) - ConfigurationUtils.requireNandDefined( + KubernetesUtils.requireNandDefined( oauthTokenFile, oauthTokenValue, s"Cannot specify OAuth token through both a file $oauthTokenFileConf and a " + diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/DriverConfigOrchestrator.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/DriverConfigOrchestrator.scala index 69b3f313049f..b1e17ff6cb4a 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/DriverConfigOrchestrator.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/DriverConfigOrchestrator.scala @@ -21,7 +21,7 @@ import java.util.UUID import com.google.common.primitives.Longs import org.apache.spark.SparkConf -import org.apache.spark.deploy.k8s.{ConfigurationUtils, MountSecretsBootstrap} +import org.apache.spark.deploy.k8s.{KubernetesUtils, MountSecretsBootstrap} import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.deploy.k8s.submit.steps._ @@ -61,7 +61,7 @@ private[spark] class DriverConfigOrchestrator( private val filesDownloadPath = sparkConf.get(FILES_DOWNLOAD_LOCATION) def getAllConfigurationSteps: Seq[DriverConfigurationStep] = { - val driverCustomLabels = ConfigurationUtils.parsePrefixedKeyValuePairs( + val driverCustomLabels = KubernetesUtils.parsePrefixedKeyValuePairs( sparkConf, KUBERNETES_DRIVER_LABEL_PREFIX) require(!driverCustomLabels.contains(SPARK_APP_ID_LABEL), "Label with key " + @@ -71,7 +71,7 @@ private[spark] class DriverConfigOrchestrator( s"$SPARK_ROLE_LABEL is not allowed as it is reserved for Spark bookkeeping " + "operations.") - val secretNamesToMountPaths = ConfigurationUtils.parsePrefixedKeyValuePairs( + val secretNamesToMountPaths = KubernetesUtils.parsePrefixedKeyValuePairs( sparkConf, KUBERNETES_DRIVER_SECRETS_PREFIX) @@ -79,7 +79,7 @@ private[spark] class DriverConfigOrchestrator( SPARK_APP_ID_LABEL -> kubernetesAppId, SPARK_ROLE_LABEL -> SPARK_POD_DRIVER_ROLE) - val initialSubmissionStep = new BaseDriverConfigurationStep( + val initialSubmissionStep = new BasicDriverConfigurationStep( kubernetesAppId, kubernetesResourceNamePrefix, allDriverLabels, @@ -117,50 +117,49 @@ private[spark] class DriverConfigOrchestrator( .map(_.split(",")) .getOrElse(Array.empty[String]) - val maybeDependencyResolutionStep = if (sparkJars.nonEmpty || sparkFiles.nonEmpty) { - Some(new DependencyResolutionStep( + val dependencyResolutionStep = if (sparkJars.nonEmpty || sparkFiles.nonEmpty) { + Seq(new DependencyResolutionStep( sparkJars, sparkFiles, jarsDownloadPath, filesDownloadPath)) } else { - None + Nil } - val mayBeInitContainerBootstrapStep = - if (areAnyFilesNonContainerLocal(sparkJars ++ sparkFiles)) { - val orchestrator = new InitContainerConfigOrchestrator( - sparkJars, - sparkFiles, - jarsDownloadPath, - filesDownloadPath, - imagePullPolicy, - initContainerConfigMapName, - INIT_CONTAINER_PROPERTIES_FILE_NAME, - sparkConf) - val bootstrapStep = new DriverInitContainerBootstrapStep( - orchestrator.getAllConfigurationSteps, - initContainerConfigMapName, - INIT_CONTAINER_PROPERTIES_FILE_NAME) - - Some(bootstrapStep) - } else { - None - } + val initContainerBootstrapStep = if (areAnyFilesNonContainerLocal(sparkJars ++ sparkFiles)) { + val orchestrator = new InitContainerConfigOrchestrator( + sparkJars, + sparkFiles, + jarsDownloadPath, + filesDownloadPath, + imagePullPolicy, + initContainerConfigMapName, + INIT_CONTAINER_PROPERTIES_FILE_NAME, + sparkConf) + val bootstrapStep = new DriverInitContainerBootstrapStep( + orchestrator.getAllConfigurationSteps, + initContainerConfigMapName, + INIT_CONTAINER_PROPERTIES_FILE_NAME) + + Seq(bootstrapStep) + } else { + Nil + } - val mayBeMountSecretsStep = if (secretNamesToMountPaths.nonEmpty) { - Some(new DriverMountSecretsStep(new MountSecretsBootstrap(secretNamesToMountPaths))) + val mountSecretsStep = if (secretNamesToMountPaths.nonEmpty) { + Seq(new DriverMountSecretsStep(new MountSecretsBootstrap(secretNamesToMountPaths))) } else { - None + Nil } Seq( initialSubmissionStep, serviceBootstrapStep, kubernetesCredentialsStep) ++ - maybeDependencyResolutionStep.toSeq ++ - mayBeInitContainerBootstrapStep.toSeq ++ - mayBeMountSecretsStep.toSeq + dependencyResolutionStep ++ + initContainerBootstrapStep ++ + mountSecretsStep } private def areAnyFilesNonContainerLocal(files: Seq[String]): Boolean = { diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/InitContainerUtil.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/InitContainerUtil.scala deleted file mode 100644 index 3b156138ecba..000000000000 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/InitContainerUtil.scala +++ /dev/null @@ -1,37 +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.k8s.submit - -import io.fabric8.kubernetes.api.model.{Container, Pod, PodBuilder} - -private[spark] object InitContainerUtil { - - /** - * Append (add to the list of InitContainers) a given init-container to a pod. - * - * @param originalPodSpec original specification of the pod - * @param initContainer the init-container to add to the pod - * @return the pod with the init-container added to the list of InitContainers - */ - def appendInitContainer(originalPodSpec: Pod, initContainer: Container): Pod = { - new PodBuilder(originalPodSpec) - .editOrNewSpec() - .addToInitContainers(initContainer) - .endSpec() - .build() - } -} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/BaseDriverConfigurationStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/BasicDriverConfigurationStep.scala similarity index 95% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/BaseDriverConfigurationStep.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/BasicDriverConfigurationStep.scala index b1de775011ca..b7a69a7dfd47 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/BaseDriverConfigurationStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/BasicDriverConfigurationStep.scala @@ -22,15 +22,15 @@ import io.fabric8.kubernetes.api.model.{ContainerBuilder, EnvVarBuilder, EnvVarS import org.apache.spark.{SparkConf, SparkException} import org.apache.spark.deploy.k8s.Config._ -import org.apache.spark.deploy.k8s.ConfigurationUtils import org.apache.spark.deploy.k8s.Constants._ +import org.apache.spark.deploy.k8s.KubernetesUtils import org.apache.spark.deploy.k8s.submit.KubernetesDriverSpec import org.apache.spark.internal.config.{DRIVER_CLASS_PATH, DRIVER_MEMORY, DRIVER_MEMORY_OVERHEAD} /** * Performs basic configuration for the driver pod. */ -private[spark] class BaseDriverConfigurationStep( +private[spark] class BasicDriverConfigurationStep( kubernetesAppId: String, resourceNamePrefix: String, driverLabels: Map[String, String], @@ -71,7 +71,7 @@ private[spark] class BaseDriverConfigurationStep( .build() } - val driverCustomAnnotations = ConfigurationUtils.parsePrefixedKeyValuePairs( + val driverCustomAnnotations = KubernetesUtils.parsePrefixedKeyValuePairs( sparkConf, KUBERNETES_DRIVER_ANNOTATION_PREFIX) require(!driverCustomAnnotations.contains(SPARK_APP_NAME_ANNOTATION), s"Annotation with key $SPARK_APP_NAME_ANNOTATION is not allowed as it is reserved for" + @@ -87,7 +87,7 @@ private[spark] class BaseDriverConfigurationStep( val driverAnnotations = driverCustomAnnotations ++ Map(SPARK_APP_NAME_ANNOTATION -> appName) - val nodeSelector = ConfigurationUtils.parsePrefixedKeyValuePairs( + val nodeSelector = KubernetesUtils.parsePrefixedKeyValuePairs( sparkConf, KUBERNETES_NODE_SELECTOR_PREFIX) val driverCpuQuantity = new QuantityBuilder(false) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DependencyResolutionStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DependencyResolutionStep.scala index cb00ed0001f1..2a987dfe7b11 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DependencyResolutionStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DependencyResolutionStep.scala @@ -21,7 +21,8 @@ import java.io.File import io.fabric8.kubernetes.api.model.ContainerBuilder import org.apache.spark.deploy.k8s.Constants._ -import org.apache.spark.deploy.k8s.submit.{KubernetesDriverSpec, KubernetesFileUtils} +import org.apache.spark.deploy.k8s.KubernetesUtils +import org.apache.spark.deploy.k8s.submit.KubernetesDriverSpec /** * Step that configures the classpath, spark.jars, and spark.files for the driver given that the @@ -34,8 +35,8 @@ private[spark] class DependencyResolutionStep( filesDownloadPath: String) extends DriverConfigurationStep { override def configureDriver(driverSpec: KubernetesDriverSpec): KubernetesDriverSpec = { - val resolvedSparkJars = KubernetesFileUtils.resolveFileUris(sparkJars, jarsDownloadPath) - val resolvedSparkFiles = KubernetesFileUtils.resolveFileUris( + val resolvedSparkJars = KubernetesUtils.resolveFileUris(sparkJars, jarsDownloadPath) + val resolvedSparkFiles = KubernetesUtils.resolveFileUris( sparkFiles, filesDownloadPath) val sparkConf = driverSpec.driverSparkConf.clone() @@ -46,7 +47,7 @@ private[spark] class DependencyResolutionStep( sparkConf.set("spark.files", resolvedSparkFiles.mkString(",")) } - val resolvedClasspath = KubernetesFileUtils.resolveFilePaths(sparkJars, jarsDownloadPath) + val resolvedClasspath = KubernetesUtils.resolveFilePaths(sparkJars, jarsDownloadPath) val resolvedDriverContainer = if (resolvedClasspath.nonEmpty) { new ContainerBuilder(driverSpec.driverContainer) .addNewEnv() diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DriverInitContainerBootstrapStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DriverInitContainerBootstrapStep.scala index 40bc1bd1c945..9fb3dafdda54 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DriverInitContainerBootstrapStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DriverInitContainerBootstrapStep.scala @@ -22,7 +22,8 @@ import java.util.Properties import io.fabric8.kubernetes.api.model.{ConfigMap, ConfigMapBuilder, ContainerBuilder, HasMetadata} import org.apache.spark.deploy.k8s.Config._ -import org.apache.spark.deploy.k8s.submit.{InitContainerUtil, KubernetesDriverSpec} +import org.apache.spark.deploy.k8s.KubernetesUtils +import org.apache.spark.deploy.k8s.submit.KubernetesDriverSpec import org.apache.spark.deploy.k8s.submit.steps.initcontainer.{InitContainerConfigurationStep, InitContainerSpec} /** @@ -59,7 +60,7 @@ private[spark] class DriverInitContainerBootstrapStep( .set(INIT_CONTAINER_CONFIG_MAP_NAME, configMapName) .set(INIT_CONTAINER_CONFIG_MAP_KEY_CONF, configMapKey) .setAll(initContainerSpec.driverSparkConf) - val resolvedDriverPod = InitContainerUtil.appendInitContainer( + val resolvedDriverPod = KubernetesUtils.appendInitContainer( initContainerSpec.driverPod, initContainerSpec.initContainer) driverSpec.copy( diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/BaseInitContainerConfigurationStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/BasicInitContainerConfigurationStep.scala similarity index 89% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/BaseInitContainerConfigurationStep.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/BasicInitContainerConfigurationStep.scala index 8ca3e023ef98..01469853dacc 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/BaseInitContainerConfigurationStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/BasicInitContainerConfigurationStep.scala @@ -18,13 +18,13 @@ package org.apache.spark.deploy.k8s.submit.steps.initcontainer import org.apache.spark.deploy.k8s.{InitContainerBootstrap, PodWithDetachedInitContainer} import org.apache.spark.deploy.k8s.Config._ -import org.apache.spark.deploy.k8s.submit.KubernetesFileUtils +import org.apache.spark.deploy.k8s.KubernetesUtils /** * Performs basic configuration for the driver init-container with most of the work delegated to * the given InitContainerBootstrap. */ -private[spark] class BaseInitContainerConfigurationStep( +private[spark] class BasicInitContainerConfigurationStep( sparkJars: Seq[String], sparkFiles: Seq[String], jarsDownloadPath: String, @@ -33,8 +33,8 @@ private[spark] class BaseInitContainerConfigurationStep( extends InitContainerConfigurationStep { override def configureInitContainer(spec: InitContainerSpec): InitContainerSpec = { - val remoteJarsToDownload = KubernetesFileUtils.getOnlyRemoteFiles(sparkJars) - val remoteFilesToDownload = KubernetesFileUtils.getOnlyRemoteFiles(sparkFiles) + val remoteJarsToDownload = KubernetesUtils.getOnlyRemoteFiles(sparkJars) + val remoteFilesToDownload = KubernetesUtils.getOnlyRemoteFiles(sparkFiles) val remoteJarsConf = if (remoteJarsToDownload.nonEmpty) { Map(INIT_CONTAINER_REMOTE_JARS.key -> remoteJarsToDownload.mkString(",")) } else { diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerConfigOrchestrator.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerConfigOrchestrator.scala index 8c50c515618b..f2c29c7ce107 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerConfigOrchestrator.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerConfigOrchestrator.scala @@ -17,7 +17,7 @@ package org.apache.spark.deploy.k8s.submit.steps.initcontainer import org.apache.spark.{SparkConf, SparkException} -import org.apache.spark.deploy.k8s.{ConfigurationUtils, InitContainerBootstrap, MountSecretsBootstrap} +import org.apache.spark.deploy.k8s.{InitContainerBootstrap, KubernetesUtils, MountSecretsBootstrap} import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ @@ -53,14 +53,14 @@ private[spark] class InitContainerConfigOrchestrator( configMapKey, SPARK_POD_DRIVER_ROLE, sparkConf) - val baseStep = new BaseInitContainerConfigurationStep( + val baseStep = new BasicInitContainerConfigurationStep( sparkJars, sparkFiles, jarsDownloadPath, filesDownloadPath, initContainerBootstrap) - val secretNamesToMountPaths = ConfigurationUtils.parsePrefixedKeyValuePairs( + val secretNamesToMountPaths = KubernetesUtils.parsePrefixedKeyValuePairs( sparkConf, KUBERNETES_DRIVER_SECRETS_PREFIX) // Mount user-specified driver secrets also into the driver's init-container. The @@ -68,13 +68,12 @@ private[spark] class InitContainerConfigOrchestrator( // dependencies. The driver's main container and its init-container share the secrets // because the init-container is sort of an implementation details and this sharing // avoids introducing a dedicated configuration property just for the init-container. - val maybeMountSecretsStep = if (secretNamesToMountPaths.nonEmpty) { - Some(new InitContainerMountSecretsStep(new MountSecretsBootstrap(secretNamesToMountPaths))) + val mountSecretsStep = if (secretNamesToMountPaths.nonEmpty) { + Seq(new InitContainerMountSecretsStep(new MountSecretsBootstrap(secretNamesToMountPaths))) } else { - None + Nil } - Seq(baseStep) ++ - maybeMountSecretsStep.toSeq + Seq(baseStep) ++ mountSecretsStep } } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/SparkConfPropertiesParser.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/SparkConfPropertiesParser.scala deleted file mode 100644 index d50ba0f5f796..000000000000 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/SparkConfPropertiesParser.scala +++ /dev/null @@ -1,50 +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.rest.k8s - -import java.io.{File, FileInputStream} -import java.util.Properties - -import scala.collection.JavaConverters._ - -import com.google.common.collect.Maps - -import org.apache.spark.SparkConf -import org.apache.spark.internal.config.{ConfigReader, SparkConfigProvider} -import org.apache.spark.util.Utils - -private[spark] object SparkConfPropertiesParser { - - def getSparkConfFromPropertiesFile(propertiesFile: File): SparkConf = { - val sparkConf = new SparkConf(true) - - if (!propertiesFile.isFile) { - throw new IllegalArgumentException("Server properties file given at " + - s"${propertiesFile.getAbsoluteFile} does not exist or is not a file.") - } - - val properties = new Properties - Utils.tryWithResource(new FileInputStream(propertiesFile))(properties.load) - val propertiesMap = Maps.fromProperties(properties) - val configReader = new ConfigReader(new SparkConfigProvider(propertiesMap)) - propertiesMap.asScala.keys.foreach { key => - configReader.get(key).foreach(sparkConf.set(key, _)) - } - - sparkConf - } -} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/SparkPodInitContainer.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/SparkPodInitContainer.scala index 49759ff0ff47..11759501f9ee 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/SparkPodInitContainer.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/SparkPodInitContainer.scala @@ -39,8 +39,9 @@ private[spark] class SparkPodInitContainer( sparkConf: SparkConf, fileFetcher: FileFetcher) extends Logging { + private val maxThreadPoolSize = sparkConf.get(INIT_CONTAINER_MAX_THREAD_POOL_SIZE) private implicit val downloadExecutor = ExecutionContext.fromExecutorService( - ThreadUtils.newDaemonCachedThreadPool("download-executor")) + ThreadUtils.newDaemonCachedThreadPool("download-executor", maxThreadPoolSize)) private val jarsDownloadDir = new File( sparkConf.get(JARS_DOWNLOAD_LOCATION)) @@ -75,12 +76,13 @@ private[spark] class SparkPodInitContainer( filesCommaSeparated: Option[String], downloadDir: File, errMessageOnDestinationNotADirectory: String): Unit = { - if (filesCommaSeparated.isDefined) { + filesCommaSeparated.foreach { files => require(downloadDir.isDirectory, errMessageOnDestinationNotADirectory) - } - filesCommaSeparated.map(_.split(",")).toSeq.flatten.foreach { file => - Future[Unit] { - fileFetcher.fetchFile(file, downloadDir) + Utils.stringToSeq(files).foreach { file => + Future[Unit] { + fileFetcher.fetchFile(file, downloadDir) + } + } } } @@ -104,10 +106,9 @@ object SparkPodInitContainer extends Logging { def main(args: Array[String]): Unit = { logInfo("Starting init-container to download Spark application dependencies.") - val sparkConf = if (args.nonEmpty) { - SparkConfPropertiesParser.getSparkConfFromPropertiesFile(new File(args(0))) - } else { - new SparkConf(true) + val sparkConf = new SparkConf(true) + if (args.nonEmpty) { + Utils.loadDefaultSparkProperties(sparkConf, args(0)) } val securityManager = new SparkSecurityManager(sparkConf) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala index 9e239d6b832d..ba5d891f4c77 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala @@ -21,10 +21,9 @@ import scala.collection.JavaConverters._ import io.fabric8.kubernetes.api.model._ import org.apache.spark.{SparkConf, SparkException} -import org.apache.spark.deploy.k8s.{ConfigurationUtils, InitContainerBootstrap, MountSecretsBootstrap, PodWithDetachedInitContainer} +import org.apache.spark.deploy.k8s.{InitContainerBootstrap, KubernetesUtils, MountSecretsBootstrap, PodWithDetachedInitContainer} import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ -import org.apache.spark.deploy.k8s.submit.InitContainerUtil import org.apache.spark.internal.config.{EXECUTOR_CLASS_PATH, EXECUTOR_JAVA_OPTIONS, EXECUTOR_MEMORY, EXECUTOR_MEMORY_OVERHEAD} import org.apache.spark.util.Utils @@ -50,7 +49,7 @@ private[spark] class ExecutorPodFactory( private val executorExtraClasspath = sparkConf.get(EXECUTOR_CLASS_PATH) - private val executorLabels = ConfigurationUtils.parsePrefixedKeyValuePairs( + private val executorLabels = KubernetesUtils.parsePrefixedKeyValuePairs( sparkConf, KUBERNETES_EXECUTOR_LABEL_PREFIX) require( @@ -65,11 +64,11 @@ private[spark] class ExecutorPodFactory( s"Custom executor labels cannot contain $SPARK_ROLE_LABEL as it is reserved for Spark.") private val executorAnnotations = - ConfigurationUtils.parsePrefixedKeyValuePairs( + KubernetesUtils.parsePrefixedKeyValuePairs( sparkConf, KUBERNETES_EXECUTOR_ANNOTATION_PREFIX) private val nodeSelector = - ConfigurationUtils.parsePrefixedKeyValuePairs( + KubernetesUtils.parsePrefixedKeyValuePairs( sparkConf, KUBERNETES_NODE_SELECTOR_PREFIX) @@ -231,7 +230,7 @@ private[spark] class ExecutorPodFactory( bootstrap.mountSecrets(podWithInitContainer.pod, podWithInitContainer.initContainer) }.getOrElse((podWithInitContainer.pod, podWithInitContainer.initContainer)) - val bootstrappedPod = InitContainerUtil.appendInitContainer( + val bootstrappedPod = KubernetesUtils.appendInitContainer( pod, mayBeSecretsMountedInitContainer) (bootstrappedPod, podWithInitContainer.mainContainer) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala index c07580d9e220..15161de81d9d 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala @@ -21,7 +21,7 @@ import java.io.File import io.fabric8.kubernetes.client.Config import org.apache.spark.{SparkContext, SparkException} -import org.apache.spark.deploy.k8s.{ConfigurationUtils, InitContainerBootstrap, MountSecretsBootstrap, SparkKubernetesClientFactory} +import org.apache.spark.deploy.k8s.{InitContainerBootstrap, KubernetesUtils, MountSecretsBootstrap, SparkKubernetesClientFactory} import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.internal.Logging @@ -45,15 +45,15 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit masterURL: String, scheduler: TaskScheduler): SchedulerBackend = { val sparkConf = sc.getConf - val maybeInitContainerConfigMap = sparkConf.get(INIT_CONTAINER_CONFIG_MAP_NAME) - val maybeInitContainerConfigMapKey = sparkConf.get(INIT_CONTAINER_CONFIG_MAP_KEY_CONF) + val initContainerConfigMap = sparkConf.get(INIT_CONTAINER_CONFIG_MAP_NAME) + val initContainerConfigMapKey = sparkConf.get(INIT_CONTAINER_CONFIG_MAP_KEY_CONF) - if (maybeInitContainerConfigMap.isEmpty) { + if (initContainerConfigMap.isEmpty) { logWarning("The executor's init-container config map was not specified. Executors will " + "therefore not attempt to fetch remote or submitted dependencies.") } - if (maybeInitContainerConfigMapKey.isEmpty) { + if (initContainerConfigMapKey.isEmpty) { logWarning("The executor's init-container config map key was not specified. Executors will " + "therefore not attempt to fetch remote or submitted dependencies.") } @@ -61,9 +61,9 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit // Only set up the bootstrap if they've provided both the config map key and the config map // name. The config map might not be provided if init-containers aren't being used to // bootstrap dependencies. - val maybeInitContainerBootstrap = for { - configMap <- maybeInitContainerConfigMap - configMapKey <- maybeInitContainerConfigMapKey + val initContainerBootstrap = for { + configMap <- initContainerConfigMap + configMapKey <- initContainerConfigMapKey } yield { val initContainerImage = sparkConf .get(INIT_CONTAINER_IMAGE) @@ -80,9 +80,9 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit sparkConf) } - val executorSecretNamesToMountPaths = ConfigurationUtils.parsePrefixedKeyValuePairs( + val executorSecretNamesToMountPaths = KubernetesUtils.parsePrefixedKeyValuePairs( sparkConf, KUBERNETES_EXECUTOR_SECRETS_PREFIX) - val mayBeMountSecretBootstrap = if (executorSecretNamesToMountPaths.nonEmpty) { + val mountSecretBootstrap = if (executorSecretNamesToMountPaths.nonEmpty) { Some(new MountSecretsBootstrap(executorSecretNamesToMountPaths)) } else { None @@ -92,7 +92,7 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit // dependencies. The executor's main container and its init-container share the secrets // because the init-container is sort of an implementation details and this sharing // avoids introducing a dedicated configuration property just for the init-container. - val mayBeInitContainerMountSecretsBootstrap = if (maybeInitContainerBootstrap.nonEmpty && + val initContainerMountSecretsBootstrap = if (initContainerBootstrap.nonEmpty && executorSecretNamesToMountPaths.nonEmpty) { Some(new MountSecretsBootstrap(executorSecretNamesToMountPaths)) } else { @@ -109,9 +109,9 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit val executorPodFactory = new ExecutorPodFactory( sparkConf, - mayBeMountSecretBootstrap, - maybeInitContainerBootstrap, - mayBeInitContainerMountSecretsBootstrap) + mountSecretBootstrap, + initContainerBootstrap, + initContainerMountSecretsBootstrap) val allocatorExecutor = ThreadUtils .newDaemonSingleThreadScheduledExecutor("kubernetes-pod-allocator") diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/DriverConfigOrchestratorSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/DriverConfigOrchestratorSuite.scala index c03838999a18..f193b1f4d366 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/DriverConfigOrchestratorSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/DriverConfigOrchestratorSuite.scala @@ -47,7 +47,7 @@ class DriverConfigOrchestratorSuite extends SparkFunSuite { sparkConf) validateStepTypes( orchestrator, - classOf[BaseDriverConfigurationStep], + classOf[BasicDriverConfigurationStep], classOf[DriverServiceBootstrapStep], classOf[DriverKubernetesCredentialsStep], classOf[DependencyResolutionStep] @@ -67,7 +67,7 @@ class DriverConfigOrchestratorSuite extends SparkFunSuite { sparkConf) validateStepTypes( orchestrator, - classOf[BaseDriverConfigurationStep], + classOf[BasicDriverConfigurationStep], classOf[DriverServiceBootstrapStep], classOf[DriverKubernetesCredentialsStep] ) @@ -89,7 +89,7 @@ class DriverConfigOrchestratorSuite extends SparkFunSuite { sparkConf) validateStepTypes( orchestrator, - classOf[BaseDriverConfigurationStep], + classOf[BasicDriverConfigurationStep], classOf[DriverServiceBootstrapStep], classOf[DriverKubernetesCredentialsStep], classOf[DependencyResolutionStep], @@ -112,7 +112,7 @@ class DriverConfigOrchestratorSuite extends SparkFunSuite { sparkConf) validateStepTypes( orchestrator, - classOf[BaseDriverConfigurationStep], + classOf[BasicDriverConfigurationStep], classOf[DriverServiceBootstrapStep], classOf[DriverKubernetesCredentialsStep], classOf[DependencyResolutionStep], diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/BaseDriverConfigurationStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/BasicDriverConfigurationStepSuite.scala similarity index 97% rename from resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/BaseDriverConfigurationStepSuite.scala rename to resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/BasicDriverConfigurationStepSuite.scala index f7c1b3142cf7..e864c6a16eeb 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/BaseDriverConfigurationStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/BasicDriverConfigurationStepSuite.scala @@ -25,7 +25,7 @@ import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.deploy.k8s.submit.KubernetesDriverSpec -class BaseDriverConfigurationStepSuite extends SparkFunSuite { +class BasicDriverConfigurationStepSuite extends SparkFunSuite { private val APP_ID = "spark-app-id" private val RESOURCE_NAME_PREFIX = "spark" @@ -52,7 +52,7 @@ class BaseDriverConfigurationStepSuite extends SparkFunSuite { .set(s"$KUBERNETES_DRIVER_ENV_KEY$DRIVER_CUSTOM_ENV_KEY1", "customDriverEnv1") .set(s"$KUBERNETES_DRIVER_ENV_KEY$DRIVER_CUSTOM_ENV_KEY2", "customDriverEnv2") - val submissionStep = new BaseDriverConfigurationStep( + val submissionStep = new BasicDriverConfigurationStep( APP_ID, RESOURCE_NAME_PREFIX, DRIVER_LABELS, diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/BaseInitContainerConfigurationStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/BasicInitContainerConfigurationStepSuite.scala similarity index 96% rename from resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/BaseInitContainerConfigurationStepSuite.scala rename to resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/BasicInitContainerConfigurationStepSuite.scala index 05f1a54d45b5..ecc23907e565 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/BaseInitContainerConfigurationStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/BasicInitContainerConfigurationStepSuite.scala @@ -30,7 +30,7 @@ import org.apache.spark.SparkFunSuite import org.apache.spark.deploy.k8s.{InitContainerBootstrap, PodWithDetachedInitContainer} import org.apache.spark.deploy.k8s.Config._ -class BaseInitContainerConfigurationStepSuite extends SparkFunSuite with BeforeAndAfter { +class BasicInitContainerConfigurationStepSuite extends SparkFunSuite with BeforeAndAfter { private val SPARK_JARS = Seq( "hdfs://localhost:9000/app/jars/jar1.jar", "file:///app/jars/jar2.jar") @@ -68,7 +68,7 @@ class BaseInitContainerConfigurationStepSuite extends SparkFunSuite with BeforeA } test("Test of additionalDriverSparkConf with mix of remote files and jars") { - val baseInitStep = new BaseInitContainerConfigurationStep( + val baseInitStep = new BasicInitContainerConfigurationStep( SPARK_JARS, SPARK_FILES, JARS_DOWNLOAD_PATH, diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerConfigOrchestratorSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerConfigOrchestratorSuite.scala index ddd4bf67dc27..591671975232 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerConfigOrchestratorSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerConfigOrchestratorSuite.scala @@ -55,7 +55,7 @@ class InitContainerConfigOrchestratorSuite extends SparkFunSuite { val initSteps : Seq[InitContainerConfigurationStep] = orchestrator.getAllConfigurationSteps assert(initSteps.lengthCompare(1) == 0) - assert(initSteps.head.isInstanceOf[BaseInitContainerConfigurationStep]) + assert(initSteps.head.isInstanceOf[BasicInitContainerConfigurationStep]) } test("including step to mount user-specified secrets") { @@ -76,7 +76,7 @@ class InitContainerConfigOrchestratorSuite extends SparkFunSuite { val initSteps : Seq[InitContainerConfigurationStep] = orchestrator.getAllConfigurationSteps assert(initSteps.length === 2) - assert(initSteps.head.isInstanceOf[BaseInitContainerConfigurationStep]) + assert(initSteps.head.isInstanceOf[BasicInitContainerConfigurationStep]) assert(initSteps(1).isInstanceOf[InitContainerMountSecretsStep]) } } From 5d1f889d9b930ac06f6902ae4aa80575a6dc872d Mon Sep 17 00:00:00 2001 From: Yinan Li Date: Wed, 20 Dec 2017 16:27:38 -0800 Subject: [PATCH 11/14] Addressed more comments --- .../apache/spark/deploy/k8s/KubernetesUtils.scala | 2 +- .../spark/deploy/k8s/MountSecretsBootstrap.scala | 13 ++++++------- .../k8s/submit/steps/DependencyResolutionStep.scala | 3 +-- .../deploy/rest/k8s/SparkPodInitContainer.scala | 7 ++----- .../DriverInitContainerBootstrapStepSuite.scala | 1 + .../submit/steps/DriverMountSecretsStepSuite.scala | 12 +++++++----- .../BasicInitContainerConfigurationStepSuite.scala | 6 +++--- .../InitContainerConfigOrchestratorSuite.scala | 8 +++----- .../InitContainerMountSecretsStepSuite.scala | 2 +- .../rest/k8s/SparkPodInitContainerSuite.scala | 9 ++------- 10 files changed, 27 insertions(+), 36 deletions(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesUtils.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesUtils.scala index 55a6ba397a26..37331d8bbf9b 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesUtils.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesUtils.scala @@ -44,7 +44,7 @@ private[spark] object KubernetesUtils { } /** - * Append the given init-container to a pod's list of init-containers.. + * Append the given init-container to a pod's list of init-containers. * * @param originalPodSpec original specification of the pod * @param initContainer the init-container to add to the pod diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/MountSecretsBootstrap.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/MountSecretsBootstrap.scala index b876d183e28d..8286546ce064 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/MountSecretsBootstrap.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/MountSecretsBootstrap.scala @@ -45,13 +45,12 @@ private[spark] class MountSecretsBootstrap(secretNamesToMountPaths: Map[String, } var containerBuilder = new ContainerBuilder(container) - secretNamesToMountPaths.foreach { - case (name, path) => - containerBuilder = containerBuilder - .addNewVolumeMount() - .withName(secretVolumeName(name)) - .withMountPath(path) - .endVolumeMount() + secretNamesToMountPaths.foreach { case (name, path) => + containerBuilder = containerBuilder + .addNewVolumeMount() + .withName(secretVolumeName(name)) + .withMountPath(path) + .endVolumeMount() } (podBuilder.build(), containerBuilder.build()) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DependencyResolutionStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DependencyResolutionStep.scala index 2a987dfe7b11..d4b83235b4e3 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DependencyResolutionStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DependencyResolutionStep.scala @@ -36,8 +36,7 @@ private[spark] class DependencyResolutionStep( override def configureDriver(driverSpec: KubernetesDriverSpec): KubernetesDriverSpec = { val resolvedSparkJars = KubernetesUtils.resolveFileUris(sparkJars, jarsDownloadPath) - val resolvedSparkFiles = KubernetesUtils.resolveFileUris( - sparkFiles, filesDownloadPath) + val resolvedSparkFiles = KubernetesUtils.resolveFileUris(sparkFiles, filesDownloadPath) val sparkConf = driverSpec.driverSparkConf.clone() if (resolvedSparkJars.nonEmpty) { diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/SparkPodInitContainer.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/SparkPodInitContainer.scala index 11759501f9ee..55c2d27d8d7d 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/SparkPodInitContainer.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/SparkPodInitContainer.scala @@ -43,10 +43,8 @@ private[spark] class SparkPodInitContainer( private implicit val downloadExecutor = ExecutionContext.fromExecutorService( ThreadUtils.newDaemonCachedThreadPool("download-executor", maxThreadPoolSize)) - private val jarsDownloadDir = new File( - sparkConf.get(JARS_DOWNLOAD_LOCATION)) - private val filesDownloadDir = new File( - sparkConf.get(FILES_DOWNLOAD_LOCATION)) + private val jarsDownloadDir = new File(sparkConf.get(JARS_DOWNLOAD_LOCATION)) + private val filesDownloadDir = new File(sparkConf.get(FILES_DOWNLOAD_LOCATION)) private val remoteJars = sparkConf.get(INIT_CONTAINER_REMOTE_JARS) private val remoteFiles = sparkConf.get(INIT_CONTAINER_REMOTE_FILES) @@ -82,7 +80,6 @@ private[spark] class SparkPodInitContainer( Future[Unit] { fileFetcher.fetchFile(file, downloadDir) } - } } } diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/DriverInitContainerBootstrapStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/DriverInitContainerBootstrapStepSuite.scala index f89f9d873b99..758871e2ba35 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/DriverInitContainerBootstrapStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/DriverInitContainerBootstrapStepSuite.scala @@ -79,6 +79,7 @@ class DriverInitContainerBootstrapStepSuite extends SparkFunSuite { SecondTestInitContainerConfigurationStep.additionalInitContainerPropertyValue) val hasMatchingProperties = initContainerPropertiesMap == expectedInitContainerProperties hasMatchingName && hasCorrectNumberOfEntries && hasMatchingProperties + case _ => false }) diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/DriverMountSecretsStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/DriverMountSecretsStepSuite.scala index 9f2c39939a7c..9ec0cb55de5a 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/DriverMountSecretsStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/DriverMountSecretsStepSuite.scala @@ -26,7 +26,7 @@ class DriverMountSecretsStepSuite extends SparkFunSuite { private val SECRET_BAR = "bar" private val SECRET_MOUNT_PATH = "/etc/secrets/driver" - test("Mounts all given secrets") { + test("mounts all given secrets") { val baseDriverSpec = KubernetesDriverSpec.initialSpec(new SparkConf(false)) val secretNamesToMountPaths = Map( SECRET_FOO -> SECRET_MOUNT_PATH, @@ -38,10 +38,12 @@ class DriverMountSecretsStepSuite extends SparkFunSuite { val driverPodWithSecretsMounted = configuredDriverSpec.driverPod val driverContainerWithSecretsMounted = configuredDriverSpec.driverContainer - Seq(s"$SECRET_FOO-volume", s"$SECRET_BAR-volume").foreach(volumeName => - assert(SecretVolumeUtils.podHasVolume(driverPodWithSecretsMounted, volumeName))) - Seq(s"$SECRET_FOO-volume", s"$SECRET_BAR-volume").foreach(volumeName => + Seq(s"$SECRET_FOO-volume", s"$SECRET_BAR-volume").foreach { volumeName => + assert(SecretVolumeUtils.podHasVolume(driverPodWithSecretsMounted, volumeName)) + } + Seq(s"$SECRET_FOO-volume", s"$SECRET_BAR-volume").foreach { volumeName => assert(SecretVolumeUtils.containerHasVolume( - driverContainerWithSecretsMounted, volumeName, SECRET_MOUNT_PATH))) + driverContainerWithSecretsMounted, volumeName, SECRET_MOUNT_PATH)) + } } } diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/BasicInitContainerConfigurationStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/BasicInitContainerConfigurationStepSuite.scala index ecc23907e565..4553f9f6b1d4 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/BasicInitContainerConfigurationStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/BasicInitContainerConfigurationStepSuite.scala @@ -67,7 +67,7 @@ class BasicInitContainerConfigurationStepSuite extends SparkFunSuite with Before )}}) } - test("Test of additionalDriverSparkConf with mix of remote files and jars") { + test("additionalDriverSparkConf with mix of remote files and jars") { val baseInitStep = new BasicInitContainerConfigurationStep( SPARK_JARS, SPARK_FILES, @@ -88,8 +88,8 @@ class BasicInitContainerConfigurationStepSuite extends SparkFunSuite with Before Seq.empty[HasMetadata]) val returnContainerSpec = baseInitStep.configureInitContainer(initContainerSpec) assert(expectedDriverSparkConf === returnContainerSpec.properties) - assert(returnContainerSpec.initContainer.getName == INIT_CONTAINER_NAME) - assert(returnContainerSpec.driverContainer.getName == DRIVER_CONTAINER_NAME) + assert(returnContainerSpec.initContainer.getName === INIT_CONTAINER_NAME) + assert(returnContainerSpec.driverContainer.getName === DRIVER_CONTAINER_NAME) assert(returnContainerSpec.driverPod.getMetadata.getLabels.asScala === POD_LABEL) } } diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerConfigOrchestratorSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerConfigOrchestratorSuite.scala index 591671975232..20f2e5bc15df 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerConfigOrchestratorSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerConfigOrchestratorSuite.scala @@ -38,7 +38,7 @@ class InitContainerConfigOrchestratorSuite extends SparkFunSuite { private val SECRET_BAR = "bar" private val SECRET_MOUNT_PATH = "/etc/secrets/init-container" - test ("including basic configuration step") { + test("including basic configuration step") { val sparkConf = new SparkConf(true) .set(INIT_CONTAINER_IMAGE, DOCKER_IMAGE) .set(s"$KUBERNETES_DRIVER_LABEL_PREFIX$CUSTOM_LABEL_KEY", CUSTOM_LABEL_VALUE) @@ -52,8 +52,7 @@ class InitContainerConfigOrchestratorSuite extends SparkFunSuite { INIT_CONTAINER_CONFIG_MAP_NAME, INIT_CONTAINER_CONFIG_MAP_KEY, sparkConf) - val initSteps : Seq[InitContainerConfigurationStep] = - orchestrator.getAllConfigurationSteps + val initSteps = orchestrator.getAllConfigurationSteps assert(initSteps.lengthCompare(1) == 0) assert(initSteps.head.isInstanceOf[BasicInitContainerConfigurationStep]) } @@ -73,8 +72,7 @@ class InitContainerConfigOrchestratorSuite extends SparkFunSuite { INIT_CONTAINER_CONFIG_MAP_NAME, INIT_CONTAINER_CONFIG_MAP_KEY, sparkConf) - val initSteps : Seq[InitContainerConfigurationStep] = - orchestrator.getAllConfigurationSteps + val initSteps = orchestrator.getAllConfigurationSteps assert(initSteps.length === 2) assert(initSteps.head.isInstanceOf[BasicInitContainerConfigurationStep]) assert(initSteps(1).isInstanceOf[InitContainerMountSecretsStep]) diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerMountSecretsStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerMountSecretsStepSuite.scala index db49bd303cd0..eab4e1765945 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerMountSecretsStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerMountSecretsStepSuite.scala @@ -28,7 +28,7 @@ class InitContainerMountSecretsStepSuite extends SparkFunSuite { private val SECRET_BAR = "bar" private val SECRET_MOUNT_PATH = "/etc/secrets/init-container" - test("Mounts all given secrets") { + test("mounts all given secrets") { val baseInitContainerSpec = InitContainerSpec( Map.empty, Map.empty, diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/k8s/SparkPodInitContainerSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/k8s/SparkPodInitContainerSuite.scala index 37f18c113f0c..6c557ec4a7c9 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/k8s/SparkPodInitContainerSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/k8s/SparkPodInitContainerSuite.scala @@ -29,10 +29,7 @@ import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.util.Utils -class SparkPodInitContainerSuite - extends SparkFunSuite with BeforeAndAfter { - - import SparkPodInitContainerSuite.createTempFile +class SparkPodInitContainerSuite extends SparkFunSuite with BeforeAndAfter { private val DOWNLOAD_JARS_SECRET_LOCATION = createTempFile("txt") private val DOWNLOAD_FILES_SECRET_LOCATION = createTempFile("txt") @@ -79,10 +76,8 @@ class SparkPodInitContainerSuite .set(JARS_DOWNLOAD_LOCATION, downloadJarsDir.getAbsolutePath) .set(FILES_DOWNLOAD_LOCATION, downloadFilesDir.getAbsolutePath) } -} -private object SparkPodInitContainerSuite { - def createTempFile(extension: String): String = { + 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) From 9d9c841b3528e0806280a58a0a8acaa456aa6e44 Mon Sep 17 00:00:00 2001 From: Yinan Li Date: Fri, 22 Dec 2017 14:51:52 -0800 Subject: [PATCH 12/14] Updated names of two configuration properties --- .../scala/org/apache/spark/deploy/k8s/Config.scala | 13 ++++++------- 1 file changed, 6 insertions(+), 7 deletions(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala index 0655be0ad80e..e50a2ac4ffcf 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala @@ -152,17 +152,16 @@ private[spark] object Config extends Logging { .createOptional val INIT_CONTAINER_MOUNT_TIMEOUT = - ConfigBuilder("spark.kubernetes.mountDependencies.mountTimeout") - .doc("Timeout before aborting the attempt to download and unpack local dependencies from " + - "remote locations and the resource staging server when initializing the driver and " + - "executor pods.") + ConfigBuilder("spark.kubernetes.mountDependencies.timeout") + .doc("Timeout before aborting the attempt to download and unpack dependencies from remote " + + "locations into the driver and executor pods.") .timeConf(TimeUnit.MINUTES) .createWithDefault(5) val INIT_CONTAINER_MAX_THREAD_POOL_SIZE = - ConfigBuilder("spark.kubernetes.initContainer.maxThreadPoolSize") - .doc("Maximum size of the thread pool in the init-container for downloading remote " + - "dependencies.") + ConfigBuilder("spark.kubernetes.mountDependencies.maxSimultaneousDownloads") + .doc("Maximum number of remote dependencies to download simultaneously in a driver or " + + "executor pod.") .intConf .createWithDefault(5) From c51bc560bb2ae0d5ea8d914e84d7485d333f497e Mon Sep 17 00:00:00 2001 From: Yinan Li Date: Mon, 25 Dec 2017 10:00:59 -0800 Subject: [PATCH 13/14] Addressed more comments --- .../src/main/scala/org/apache/spark/deploy/k8s/Config.scala | 4 ++-- .../spark/deploy/k8s/submit/DriverConfigOrchestrator.scala | 4 ++-- .../apache/spark/deploy/rest/k8s/SparkPodInitContainer.scala | 4 ++-- .../scheduler/cluster/k8s/KubernetesClusterManager.scala | 4 ++-- 4 files changed, 8 insertions(+), 8 deletions(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala index e50a2ac4ffcf..e5d79d9a9d9d 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala @@ -155,8 +155,8 @@ private[spark] object Config extends Logging { ConfigBuilder("spark.kubernetes.mountDependencies.timeout") .doc("Timeout before aborting the attempt to download and unpack dependencies from remote " + "locations into the driver and executor pods.") - .timeConf(TimeUnit.MINUTES) - .createWithDefault(5) + .timeConf(TimeUnit.SECONDS) + .createWithDefault(300) val INIT_CONTAINER_MAX_THREAD_POOL_SIZE = ConfigBuilder("spark.kubernetes.mountDependencies.maxSimultaneousDownloads") diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/DriverConfigOrchestrator.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/DriverConfigOrchestrator.scala index b1e17ff6cb4a..00c9c4ee4917 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/DriverConfigOrchestrator.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/DriverConfigOrchestrator.scala @@ -127,7 +127,7 @@ private[spark] class DriverConfigOrchestrator( Nil } - val initContainerBootstrapStep = if (areAnyFilesNonContainerLocal(sparkJars ++ sparkFiles)) { + val initContainerBootstrapStep = if (existNonContainerLocalFiles(sparkJars ++ sparkFiles)) { val orchestrator = new InitContainerConfigOrchestrator( sparkJars, sparkFiles, @@ -162,7 +162,7 @@ private[spark] class DriverConfigOrchestrator( mountSecretsStep } - private def areAnyFilesNonContainerLocal(files: Seq[String]): Boolean = { + private def existNonContainerLocalFiles(files: Seq[String]): Boolean = { files.exists { uri => Utils.resolveURI(uri).getScheme != "local" } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/SparkPodInitContainer.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/SparkPodInitContainer.scala index 55c2d27d8d7d..4a4b628aedbb 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/SparkPodInitContainer.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/SparkPodInitContainer.scala @@ -73,9 +73,9 @@ private[spark] class SparkPodInitContainer( private def downloadFiles( filesCommaSeparated: Option[String], downloadDir: File, - errMessageOnDestinationNotADirectory: String): Unit = { + errMessage: String): Unit = { filesCommaSeparated.foreach { files => - require(downloadDir.isDirectory, errMessageOnDestinationNotADirectory) + require(downloadDir.isDirectory, errMessage) Utils.stringToSeq(files).foreach { file => Future[Unit] { fileFetcher.fetchFile(file, downloadDir) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala index 15161de81d9d..a942db6ae02d 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala @@ -49,12 +49,12 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit val initContainerConfigMapKey = sparkConf.get(INIT_CONTAINER_CONFIG_MAP_KEY_CONF) if (initContainerConfigMap.isEmpty) { - logWarning("The executor's init-container config map was not specified. Executors will " + + logWarning("The executor's init-container config map is not specified. Executors will " + "therefore not attempt to fetch remote or submitted dependencies.") } if (initContainerConfigMapKey.isEmpty) { - logWarning("The executor's init-container config map key was not specified. Executors will " + + logWarning("The executor's init-container config map key is not specified. Executors will " + "therefore not attempt to fetch remote or submitted dependencies.") } From 28343fb50310826bc9962e785f25d1af9b3c3f4a Mon Sep 17 00:00:00 2001 From: Yinan Li Date: Tue, 26 Dec 2017 00:18:13 -0800 Subject: [PATCH 14/14] Addressed one more comment --- .../kubernetes/docker/src/main/dockerfiles/driver/Dockerfile | 2 +- .../kubernetes/docker/src/main/dockerfiles/executor/Dockerfile | 2 +- .../docker/src/main/dockerfiles/init-container/Dockerfile | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/resource-managers/kubernetes/docker/src/main/dockerfiles/driver/Dockerfile b/resource-managers/kubernetes/docker/src/main/dockerfiles/driver/Dockerfile index db32ffed2a37..45fbcd9cd0de 100644 --- a/resource-managers/kubernetes/docker/src/main/dockerfiles/driver/Dockerfile +++ b/resource-managers/kubernetes/docker/src/main/dockerfiles/driver/Dockerfile @@ -22,7 +22,7 @@ FROM spark-base # If this docker file is being used in the context of building your images from a Spark # distribution, the docker build command should be invoked from the top level directory # of the Spark distribution. E.g.: -# docker build -t spark-driver:latest -f kubernetes/dockerfiles/spark-base/Dockerfile . +# docker build -t spark-driver:latest -f kubernetes/dockerfiles/driver/Dockerfile . COPY examples /opt/spark/examples diff --git a/resource-managers/kubernetes/docker/src/main/dockerfiles/executor/Dockerfile b/resource-managers/kubernetes/docker/src/main/dockerfiles/executor/Dockerfile index c25f4b82978a..0f806cf7e148 100644 --- a/resource-managers/kubernetes/docker/src/main/dockerfiles/executor/Dockerfile +++ b/resource-managers/kubernetes/docker/src/main/dockerfiles/executor/Dockerfile @@ -22,7 +22,7 @@ FROM spark-base # If this docker file is being used in the context of building your images from a Spark # distribution, the docker build command should be invoked from the top level directory # of the Spark distribution. E.g.: -# docker build -t spark-executor:latest -f kubernetes/dockerfiles/spark-base/Dockerfile . +# docker build -t spark-executor:latest -f kubernetes/dockerfiles/executor/Dockerfile . COPY examples /opt/spark/examples diff --git a/resource-managers/kubernetes/docker/src/main/dockerfiles/init-container/Dockerfile b/resource-managers/kubernetes/docker/src/main/dockerfiles/init-container/Dockerfile index 2f13c26090b3..055493188fcb 100644 --- a/resource-managers/kubernetes/docker/src/main/dockerfiles/init-container/Dockerfile +++ b/resource-managers/kubernetes/docker/src/main/dockerfiles/init-container/Dockerfile @@ -19,6 +19,6 @@ FROM spark-base # If this docker file is being used in the context of building your images from a Spark distribution, the docker build # command should be invoked from the top level directory of the Spark distribution. E.g.: -# docker build -t spark-init:latest -f dockerfiles/init-container/Dockerfile . +# docker build -t spark-init:latest -f kubernetes/dockerfiles/init-container/Dockerfile . ENTRYPOINT [ "/opt/entrypoint.sh", "/opt/spark/bin/spark-class", "org.apache.spark.deploy.rest.k8s.SparkPodInitContainer" ]