From ea0f8bce23148a32e9d3e2e5ed763621a25a7330 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Fri, 2 Nov 2018 16:58:53 -0700 Subject: [PATCH 1/4] [SPARK-25876][k8s] Simplify kubernetes configuration types. There are a few issues with the current configuration types used in the kubernetes backend: - they use type parameters for role-specific specialization, which makes type signatures really noisy throughout the code base. - they break encapsulation by forcing the code that creates the config object to remove the configuration from SparkConf before creating the k8s-specific wrapper. - they don't provide an easy way for tests to have default values for fields they do not use. This change fixes those problems by: - creating a base config type with role-specific specialization using inheritance - encapsulating the logic of parsing SparkConf into k8s-specific views inside the k8s config classes - providing some helper code for tests to easily override just the part of the configs they want. Most of the change relates to the above, especially cleaning up the tests. While doing that, I also madke some smaller changes elsewhere: - removed unnecessary type parameters in KubernetesVolumeSpec - simplified the error detection logic in KubernetesVolumeUtils; all the call sites would just throw the first exception collected by that class, since they all called "get" on the "Try" object. Now the unnecessary wrapping is gone and the exception is just thrown where it occurs. - removed a lot of unnecessary mocking from tests. - changed the kerberos-related code so that less logic needs to live in the driver builder. In spirit it should be part of the upcoming work in this series of cleanups, but it made parts of this change simpler. Tested with existing unit tests and integration tests. --- .../org/apache/spark/deploy/k8s/Config.scala | 17 +- .../spark/deploy/k8s/KubernetesConf.scala | 302 ++++++++---------- .../deploy/k8s/KubernetesVolumeSpec.scala | 10 +- .../deploy/k8s/KubernetesVolumeUtils.scala | 54 +--- .../k8s/features/BasicDriverFeatureStep.scala | 24 +- .../features/BasicExecutorFeatureStep.scala | 29 +- .../features/DriverCommandFeatureStep.scala | 8 +- ...iverKubernetesCredentialsFeatureStep.scala | 6 +- .../features/DriverServiceFeatureStep.scala | 10 +- .../k8s/features/EnvSecretsFeatureStep.scala | 11 +- .../HadoopConfExecutorFeatureStep.scala | 14 +- .../HadoopSparkUserExecutorFeatureStep.scala | 17 +- .../KerberosConfDriverFeatureStep.scala | 99 ++++-- .../KerberosConfExecutorFeatureStep.scala | 21 +- .../k8s/features/LocalDirsFeatureStep.scala | 9 +- .../features/MountSecretsFeatureStep.scala | 13 +- .../features/MountVolumesFeatureStep.scala | 11 +- .../features/PodTemplateConfigMapStep.scala | 5 +- .../hadooputils/HadoopKerberosLogin.scala | 64 ---- ...bernetesHadoopDelegationTokenManager.scala | 37 --- .../submit/KubernetesClientApplication.scala | 61 +--- .../k8s/submit/KubernetesDriverBuilder.scala | 51 ++- .../k8s/KubernetesExecutorBuilder.scala | 36 +-- .../deploy/k8s/KubernetesConfSuite.scala | 71 ++-- .../spark/deploy/k8s/KubernetesTestConf.scala | 134 ++++++++ .../k8s/KubernetesVolumeUtilsSuite.scala | 28 +- .../BasicDriverFeatureStepSuite.scala | 127 ++------ .../BasicExecutorFeatureStepSuite.scala | 103 ++---- .../DriverCommandFeatureStepSuite.scala | 29 +- ...ubernetesCredentialsFeatureStepSuite.scala | 69 +--- .../DriverServiceFeatureStepSuite.scala | 193 ++++------- .../features/EnvSecretsFeatureStepSuite.scala | 32 +- .../features/LocalDirsFeatureStepSuite.scala | 46 +-- .../MountSecretsFeatureStepSuite.scala | 21 +- .../MountVolumesFeatureStepSuite.scala | 31 +- .../PodTemplateConfigMapStepSuite.scala | 28 +- .../spark/deploy/k8s/submit/ClientSuite.scala | 50 +-- .../submit/KubernetesDriverBuilderSuite.scala | 185 ++--------- .../k8s/ExecutorPodsAllocatorSuite.scala | 43 +-- .../k8s/KubernetesExecutorBuilderSuite.scala | 114 ++----- 40 files changed, 744 insertions(+), 1469 deletions(-) delete mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadooputils/HadoopKerberosLogin.scala delete mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/security/KubernetesHadoopDelegationTokenManager.scala create mode 100644 resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/KubernetesTestConf.scala 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 a32bd93bb65b..aac9e270da14 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 @@ -60,7 +60,8 @@ private[spark] object Config extends Logging { .doc("Comma separated list of the Kubernetes secrets used " + "to access private image registries.") .stringConf - .createOptional + .toSequence + .createWithDefault(Nil) val KUBERNETES_AUTH_DRIVER_CONF_PREFIX = "spark.kubernetes.authenticate.driver" @@ -112,16 +113,16 @@ private[spark] object Config extends Logging { .stringConf .createOptional - val KUBERNETES_EXECUTOR_POD_NAME_PREFIX = - ConfigBuilder("spark.kubernetes.executor.podNamePrefix") - .doc("Prefix to use in front of the executor pod names.") + // For testing only. + val KUBERNETES_DRIVER_POD_NAME_PREFIX = + ConfigBuilder("spark.kubernetes.driver.resourceNamePrefix") .internal() .stringConf - .createWithDefault("spark") + .createOptional - val KUBERNETES_PYSPARK_PY_FILES = - ConfigBuilder("spark.kubernetes.python.pyFiles") - .doc("The PyFiles that are distributed via client arguments") + val KUBERNETES_EXECUTOR_POD_NAME_PREFIX = + ConfigBuilder("spark.kubernetes.executor.podNamePrefix") + .doc("Prefix to use in front of the executor pod names.") .internal() .stringConf .createOptional diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesConf.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesConf.scala index ebb81540bbbb..a06c21b47f15 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesConf.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesConf.scala @@ -16,93 +16,53 @@ */ package org.apache.spark.deploy.k8s -import scala.collection.mutable +import java.util.Locale import io.fabric8.kubernetes.api.model.{LocalObjectReference, LocalObjectReferenceBuilder, Pod} -import org.apache.hadoop.conf.Configuration import org.apache.spark.SparkConf import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ -import org.apache.spark.deploy.k8s.security.KubernetesHadoopDelegationTokenManager import org.apache.spark.deploy.k8s.submit._ -import org.apache.spark.deploy.k8s.submit.KubernetesClientApplication._ import org.apache.spark.internal.config.ConfigEntry import org.apache.spark.util.Utils - -private[spark] sealed trait KubernetesRoleSpecificConf - -/* - * Structure containing metadata for Kubernetes logic that builds a Spark driver. - */ -private[spark] case class KubernetesDriverSpecificConf( - mainAppResource: MainAppResource, - mainClass: String, - appName: String, - appArgs: Seq[String], - pyFiles: Seq[String] = Nil) extends KubernetesRoleSpecificConf { - - require(mainAppResource != null, "Main resource must be provided.") - -} - -/* - * Structure containing metadata for Kubernetes logic that builds a Spark executor. - */ -private[spark] case class KubernetesExecutorSpecificConf( - executorId: String, - driverPod: Option[Pod]) - extends KubernetesRoleSpecificConf - -/* - * Structure containing metadata for HADOOP_CONF_DIR customization - */ -private[spark] case class HadoopConfSpec( - hadoopConfDir: Option[String], - hadoopConfigMapName: Option[String]) - /** * Structure containing metadata for Kubernetes logic to build Spark pods. */ -private[spark] case class KubernetesConf[T <: KubernetesRoleSpecificConf]( - sparkConf: SparkConf, - roleSpecificConf: T, - appResourceNamePrefix: String, - appId: String, - roleLabels: Map[String, String], - roleAnnotations: Map[String, String], - roleSecretNamesToMountPaths: Map[String, String], - roleSecretEnvNamesToKeyRefs: Map[String, String], - roleEnvs: Map[String, String], - roleVolumes: Iterable[KubernetesVolumeSpec[_ <: KubernetesVolumeSpecificConf]], - hadoopConfSpec: Option[HadoopConfSpec]) { +private[spark] abstract class KubernetesConf(val sparkConf: SparkConf) { - def hadoopConfigMapName: String = s"$appResourceNamePrefix-hadoop-config" + val resourceNamePrefix: String + def labels: Map[String, String] + def environment: Map[String, String] + def annotations: Map[String, String] + def secretEnvNamesToKeyRefs: Map[String, String] + def secretNamesToMountPaths: Map[String, String] + def volumes: Seq[KubernetesVolumeSpec] - def krbConfigMapName: String = s"$appResourceNamePrefix-krb5-file" + def appName: String = get("spark.app.name", "spark") - def tokenManager(conf: SparkConf, hConf: Configuration): KubernetesHadoopDelegationTokenManager = - new KubernetesHadoopDelegationTokenManager(conf, hConf) + def hadoopConfigMapName: String = s"$resourceNamePrefix-hadoop-config" - def namespace(): String = sparkConf.get(KUBERNETES_NAMESPACE) + def krbConfigMapName: String = s"$resourceNamePrefix-krb5-file" - def imagePullPolicy(): String = sparkConf.get(CONTAINER_IMAGE_PULL_POLICY) + def namespace: String = get(KUBERNETES_NAMESPACE) - def imagePullSecrets(): Seq[LocalObjectReference] = { + def imagePullPolicy: String = get(CONTAINER_IMAGE_PULL_POLICY) + + def imagePullSecrets: Seq[LocalObjectReference] = { sparkConf .get(IMAGE_PULL_SECRETS) - .map(_.split(",")) - .getOrElse(Array.empty[String]) - .map(_.trim) .map { secret => new LocalObjectReferenceBuilder().withName(secret).build() } } - def nodeSelector(): Map[String, String] = + def nodeSelector: Map[String, String] = KubernetesUtils.parsePrefixedKeyValuePairs(sparkConf, KUBERNETES_NODE_SELECTOR_PREFIX) + def contains(config: ConfigEntry[_]): Boolean = sparkConf.contains(config) + def get[T](config: ConfigEntry[T]): T = sparkConf.get(config) def get(conf: String): String = sparkConf.get(conf) @@ -112,125 +72,139 @@ private[spark] case class KubernetesConf[T <: KubernetesRoleSpecificConf]( def getOption(key: String): Option[String] = sparkConf.getOption(key) } +private[spark] class KubernetesDriverConf( + sparkConf: SparkConf, + val appId: String, + val mainAppResource: MainAppResource, + val mainClass: String, + val appArgs: Array[String], + val pyFiles: Seq[String]) + extends KubernetesConf(sparkConf) { + + override val resourceNamePrefix: String = { + val custom = if (Utils.isTesting) get(KUBERNETES_DRIVER_POD_NAME_PREFIX) else None + custom.getOrElse(KubernetesConf.getResourceNamePrefix(appName)) + } + + override def labels: Map[String, String] = { + val presetLabels = Map( + SPARK_APP_ID_LABEL -> appId, + SPARK_ROLE_LABEL -> SPARK_POD_DRIVER_ROLE) + val driverCustomLabels = KubernetesUtils.parsePrefixedKeyValuePairs( + sparkConf, KUBERNETES_DRIVER_LABEL_PREFIX) + + presetLabels.keys.foreach { key => + require( + !driverCustomLabels.contains(key), + s"Label with key $key is not allowed as it is reserved for Spark bookkeeping operations.") + } + + driverCustomLabels ++ presetLabels + } + + override def environment: Map[String, String] = { + KubernetesUtils.parsePrefixedKeyValuePairs(sparkConf, KUBERNETES_DRIVER_ENV_PREFIX) + } + + override def annotations: Map[String, String] = { + KubernetesUtils.parsePrefixedKeyValuePairs(sparkConf, KUBERNETES_DRIVER_ANNOTATION_PREFIX) + } + + override def secretNamesToMountPaths: Map[String, String] = { + KubernetesUtils.parsePrefixedKeyValuePairs(sparkConf, KUBERNETES_DRIVER_SECRETS_PREFIX) + } + + override def secretEnvNamesToKeyRefs: Map[String, String] = { + KubernetesUtils.parsePrefixedKeyValuePairs(sparkConf, KUBERNETES_DRIVER_SECRET_KEY_REF_PREFIX) + } + + override def volumes: Seq[KubernetesVolumeSpec] = { + KubernetesVolumeUtils.parseVolumesWithPrefix(sparkConf, KUBERNETES_DRIVER_VOLUMES_PREFIX) + } +} + +private[spark] class KubernetesExecutorConf( + sparkConf: SparkConf, + val appId: String, + val executorId: String, + val driverPod: Option[Pod]) + extends KubernetesConf(sparkConf) { + + override val resourceNamePrefix: String = { + get(KUBERNETES_EXECUTOR_POD_NAME_PREFIX).getOrElse( + KubernetesConf.getResourceNamePrefix(appName)) + } + + override def labels: Map[String, String] = { + val presetLabels = Map( + SPARK_EXECUTOR_ID_LABEL -> executorId, + SPARK_APP_ID_LABEL -> appId, + SPARK_ROLE_LABEL -> SPARK_POD_EXECUTOR_ROLE) + + val executorCustomLabels = KubernetesUtils.parsePrefixedKeyValuePairs( + sparkConf, KUBERNETES_EXECUTOR_LABEL_PREFIX) + + presetLabels.keys.foreach { key => + require( + !executorCustomLabels.contains(key), + s"Custom executor labels cannot contain $key as it is reserved for Spark.") + } + + executorCustomLabels ++ presetLabels + } + + override def environment: Map[String, String] = sparkConf.getExecutorEnv.toMap + + override def annotations: Map[String, String] = { + KubernetesUtils.parsePrefixedKeyValuePairs(sparkConf, KUBERNETES_EXECUTOR_ANNOTATION_PREFIX) + } + + override def secretNamesToMountPaths: Map[String, String] = { + KubernetesUtils.parsePrefixedKeyValuePairs(sparkConf, KUBERNETES_EXECUTOR_SECRETS_PREFIX) + } + + override def secretEnvNamesToKeyRefs: Map[String, String] = { + KubernetesUtils.parsePrefixedKeyValuePairs(sparkConf, KUBERNETES_EXECUTOR_SECRET_KEY_REF_PREFIX) + } + + override def volumes: Seq[KubernetesVolumeSpec] = { + KubernetesVolumeUtils.parseVolumesWithPrefix(sparkConf, KUBERNETES_EXECUTOR_VOLUMES_PREFIX) + } + +} + private[spark] object KubernetesConf { def createDriverConf( sparkConf: SparkConf, - appName: String, - appResourceNamePrefix: String, appId: String, mainAppResource: MainAppResource, mainClass: String, appArgs: Array[String], - maybePyFiles: Option[String], - hadoopConfDir: Option[String]): KubernetesConf[KubernetesDriverSpecificConf] = { - val driverCustomLabels = KubernetesUtils.parsePrefixedKeyValuePairs( - 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 " + - "operations.") - require(!driverCustomLabels.contains(SPARK_ROLE_LABEL), "Label with key " + - s"$SPARK_ROLE_LABEL is not allowed as it is reserved for Spark bookkeeping " + - "operations.") - val driverLabels = driverCustomLabels ++ Map( - SPARK_APP_ID_LABEL -> appId, - SPARK_ROLE_LABEL -> SPARK_POD_DRIVER_ROLE) - val driverAnnotations = KubernetesUtils.parsePrefixedKeyValuePairs( - sparkConf, KUBERNETES_DRIVER_ANNOTATION_PREFIX) - val driverSecretNamesToMountPaths = KubernetesUtils.parsePrefixedKeyValuePairs( - sparkConf, KUBERNETES_DRIVER_SECRETS_PREFIX) - val driverSecretEnvNamesToKeyRefs = KubernetesUtils.parsePrefixedKeyValuePairs( - sparkConf, KUBERNETES_DRIVER_SECRET_KEY_REF_PREFIX) - val driverEnvs = KubernetesUtils.parsePrefixedKeyValuePairs( - sparkConf, KUBERNETES_DRIVER_ENV_PREFIX) - val driverVolumes = KubernetesVolumeUtils.parseVolumesWithPrefix( - sparkConf, KUBERNETES_DRIVER_VOLUMES_PREFIX).map(_.get) - // Also parse executor volumes in order to verify configuration - // before the driver pod is created - KubernetesVolumeUtils.parseVolumesWithPrefix( - sparkConf, KUBERNETES_EXECUTOR_VOLUMES_PREFIX).map(_.get) - - val hadoopConfigMapName = sparkConf.get(KUBERNETES_HADOOP_CONF_CONFIG_MAP) - KubernetesUtils.requireNandDefined( - hadoopConfDir, - hadoopConfigMapName, - "Do not specify both the `HADOOP_CONF_DIR` in your ENV and the ConfigMap " + - "as the creation of an additional ConfigMap, when one is already specified is extraneous" ) - val hadoopConfSpec = - if (hadoopConfDir.isDefined || hadoopConfigMapName.isDefined) { - Some(HadoopConfSpec(hadoopConfDir, hadoopConfigMapName)) - } else { - None - } - val pyFiles = maybePyFiles.map(Utils.stringToSeq).getOrElse(Nil) + maybePyFiles: Option[String]): KubernetesDriverConf = { + // Parse executor volumes in order to verify configuration before the driver pod is created. + KubernetesVolumeUtils.parseVolumesWithPrefix(sparkConf, KUBERNETES_EXECUTOR_VOLUMES_PREFIX) - - KubernetesConf( - sparkConf.clone(), - KubernetesDriverSpecificConf(mainAppResource, mainClass, appName, appArgs, pyFiles), - appResourceNamePrefix, - appId, - driverLabels, - driverAnnotations, - driverSecretNamesToMountPaths, - driverSecretEnvNamesToKeyRefs, - driverEnvs, - driverVolumes, - hadoopConfSpec) + val pyFiles = maybePyFiles.map(Utils.stringToSeq).getOrElse(Nil) + new KubernetesDriverConf(sparkConf.clone(), appId, mainAppResource, mainClass, appArgs, + pyFiles) } def createExecutorConf( sparkConf: SparkConf, executorId: String, appId: String, - driverPod: Option[Pod]): KubernetesConf[KubernetesExecutorSpecificConf] = { - val executorCustomLabels = KubernetesUtils.parsePrefixedKeyValuePairs( - sparkConf, KUBERNETES_EXECUTOR_LABEL_PREFIX) - require( - !executorCustomLabels.contains(SPARK_APP_ID_LABEL), - s"Custom executor labels cannot contain $SPARK_APP_ID_LABEL as it is reserved for Spark.") - require( - !executorCustomLabels.contains(SPARK_EXECUTOR_ID_LABEL), - s"Custom executor labels cannot contain $SPARK_EXECUTOR_ID_LABEL as it is reserved for" + - " Spark.") - require( - !executorCustomLabels.contains(SPARK_ROLE_LABEL), - s"Custom executor labels cannot contain $SPARK_ROLE_LABEL as it is reserved for Spark.") - val executorLabels = Map( - SPARK_EXECUTOR_ID_LABEL -> executorId, - SPARK_APP_ID_LABEL -> appId, - SPARK_ROLE_LABEL -> SPARK_POD_EXECUTOR_ROLE) ++ - executorCustomLabels - val executorAnnotations = KubernetesUtils.parsePrefixedKeyValuePairs( - sparkConf, KUBERNETES_EXECUTOR_ANNOTATION_PREFIX) - val executorMountSecrets = KubernetesUtils.parsePrefixedKeyValuePairs( - sparkConf, KUBERNETES_EXECUTOR_SECRETS_PREFIX) - val executorEnvSecrets = KubernetesUtils.parsePrefixedKeyValuePairs( - sparkConf, KUBERNETES_EXECUTOR_SECRET_KEY_REF_PREFIX) - val executorEnv = sparkConf.getExecutorEnv.toMap - val executorVolumes = KubernetesVolumeUtils.parseVolumesWithPrefix( - sparkConf, KUBERNETES_EXECUTOR_VOLUMES_PREFIX).map(_.get) - - // If no prefix is defined then we are in pure client mode - // (not the one used by cluster mode inside the container) - val appResourceNamePrefix = { - if (sparkConf.getOption(KUBERNETES_EXECUTOR_POD_NAME_PREFIX.key).isEmpty) { - getResourceNamePrefix(getAppName(sparkConf)) - } else { - sparkConf.get(KUBERNETES_EXECUTOR_POD_NAME_PREFIX) - } - } + driverPod: Option[Pod]): KubernetesExecutorConf = { + new KubernetesExecutorConf(sparkConf.clone(), appId, executorId, driverPod) + } - KubernetesConf( - sparkConf.clone(), - KubernetesExecutorSpecificConf(executorId, driverPod), - appResourceNamePrefix, - appId, - executorLabels, - executorAnnotations, - executorMountSecrets, - executorEnvSecrets, - executorEnv, - executorVolumes, - None) + def getResourceNamePrefix(appName: String): String = { + val launchTime = System.currentTimeMillis() + s"$appName-$launchTime" + .trim + .toLowerCase(Locale.ROOT) + .replaceAll("\\s+", "-") + .replaceAll("\\.", "-") + .replaceAll("[^a-z0-9\\-]", "") + .replaceAll("-+", "-") } } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesVolumeSpec.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesVolumeSpec.scala index b1762d1efe2e..a653b57e7458 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesVolumeSpec.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesVolumeSpec.scala @@ -18,12 +18,10 @@ package org.apache.spark.deploy.k8s private[spark] sealed trait KubernetesVolumeSpecificConf -private[spark] case class KubernetesHostPathVolumeConf( - hostPath: String) +private[spark] case class KubernetesHostPathVolumeConf(hostPath: String) extends KubernetesVolumeSpecificConf -private[spark] case class KubernetesPVCVolumeConf( - claimName: String) +private[spark] case class KubernetesPVCVolumeConf(claimName: String) extends KubernetesVolumeSpecificConf private[spark] case class KubernetesEmptyDirVolumeConf( @@ -31,8 +29,8 @@ private[spark] case class KubernetesEmptyDirVolumeConf( sizeLimit: Option[String]) extends KubernetesVolumeSpecificConf -private[spark] case class KubernetesVolumeSpec[T <: KubernetesVolumeSpecificConf]( +private[spark] case class KubernetesVolumeSpec( volumeName: String, mountPath: String, mountReadOnly: Boolean, - volumeConf: T) + volumeConf: KubernetesVolumeSpecificConf) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesVolumeUtils.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesVolumeUtils.scala index 713df5fffc3a..be656601aad0 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesVolumeUtils.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesVolumeUtils.scala @@ -16,10 +16,6 @@ */ package org.apache.spark.deploy.k8s -import java.util.NoSuchElementException - -import scala.util.{Failure, Success, Try} - import org.apache.spark.SparkConf import org.apache.spark.deploy.k8s.Config._ @@ -31,25 +27,18 @@ private[spark] object KubernetesVolumeUtils { * @param prefix the given property name prefix * @return a Map storing with volume name as key and spec as value */ - def parseVolumesWithPrefix( - sparkConf: SparkConf, - prefix: String): Iterable[Try[KubernetesVolumeSpec[_ <: KubernetesVolumeSpecificConf]]] = { + def parseVolumesWithPrefix(sparkConf: SparkConf, prefix: String): Seq[KubernetesVolumeSpec] = { val properties = sparkConf.getAllWithPrefix(prefix).toMap getVolumeTypesAndNames(properties).map { case (volumeType, volumeName) => val pathKey = s"$volumeType.$volumeName.$KUBERNETES_VOLUMES_MOUNT_PATH_KEY" val readOnlyKey = s"$volumeType.$volumeName.$KUBERNETES_VOLUMES_MOUNT_READONLY_KEY" - - for { - path <- properties.getTry(pathKey) - volumeConf <- parseVolumeSpecificConf(properties, volumeType, volumeName) - } yield KubernetesVolumeSpec( + KubernetesVolumeSpec( volumeName = volumeName, - mountPath = path, + mountPath = properties(pathKey), mountReadOnly = properties.get(readOnlyKey).exists(_.toBoolean), - volumeConf = volumeConf - ) - } + volumeConf = parseVolumeSpecificConf(properties, volumeType, volumeName)) + }.toSeq } /** @@ -59,9 +48,7 @@ private[spark] object KubernetesVolumeUtils { * @param properties flat mapping of property names to values * @return Set[(volumeType, volumeName)] */ - private def getVolumeTypesAndNames( - properties: Map[String, String] - ): Set[(String, String)] = { + private def getVolumeTypesAndNames(properties: Map[String, String]): Set[(String, String)] = { properties.keys.flatMap { k => k.split('.').toList match { case tpe :: name :: _ => Some((tpe, name)) @@ -71,40 +58,25 @@ private[spark] object KubernetesVolumeUtils { } private def parseVolumeSpecificConf( - options: Map[String, String], - volumeType: String, - volumeName: String): Try[KubernetesVolumeSpecificConf] = { + options: Map[String, String], + volumeType: String, + volumeName: String): KubernetesVolumeSpecificConf = { volumeType match { case KUBERNETES_VOLUMES_HOSTPATH_TYPE => val pathKey = s"$volumeType.$volumeName.$KUBERNETES_VOLUMES_OPTIONS_PATH_KEY" - for { - path <- options.getTry(pathKey) - } yield KubernetesHostPathVolumeConf(path) + KubernetesHostPathVolumeConf(options(pathKey)) case KUBERNETES_VOLUMES_PVC_TYPE => val claimNameKey = s"$volumeType.$volumeName.$KUBERNETES_VOLUMES_OPTIONS_CLAIM_NAME_KEY" - for { - claimName <- options.getTry(claimNameKey) - } yield KubernetesPVCVolumeConf(claimName) + KubernetesPVCVolumeConf(options(claimNameKey)) case KUBERNETES_VOLUMES_EMPTYDIR_TYPE => val mediumKey = s"$volumeType.$volumeName.$KUBERNETES_VOLUMES_OPTIONS_MEDIUM_KEY" val sizeLimitKey = s"$volumeType.$volumeName.$KUBERNETES_VOLUMES_OPTIONS_SIZE_LIMIT_KEY" - Success(KubernetesEmptyDirVolumeConf(options.get(mediumKey), options.get(sizeLimitKey))) + KubernetesEmptyDirVolumeConf(options.get(mediumKey), options.get(sizeLimitKey)) case _ => - Failure(new RuntimeException(s"Kubernetes Volume type `$volumeType` is not supported")) - } - } - - /** - * Convenience wrapper to accumulate key lookup errors - */ - implicit private class MapOps[A, B](m: Map[A, B]) { - def getTry(key: A): Try[B] = { - m - .get(key) - .fold[Try[B]](Failure(new NoSuchElementException(key.toString)))(Success(_)) + throw new IllegalArgumentException(s"Kubernetes Volume type `$volumeType` is not supported") } } } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStep.scala index 5ddf73cb16a6..d8cf3653d322 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStep.scala @@ -30,13 +30,12 @@ import org.apache.spark.internal.config._ import org.apache.spark.ui.SparkUI import org.apache.spark.util.Utils -private[spark] class BasicDriverFeatureStep( - conf: KubernetesConf[KubernetesDriverSpecificConf]) +private[spark] class BasicDriverFeatureStep(conf: KubernetesDriverConf) extends KubernetesFeatureConfigStep { private val driverPodName = conf .get(KUBERNETES_DRIVER_POD_NAME) - .getOrElse(s"${conf.appResourceNamePrefix}-driver") + .getOrElse(s"${conf.resourceNamePrefix}-driver") private val driverContainerImage = conf .get(DRIVER_CONTAINER_IMAGE) @@ -52,8 +51,8 @@ private[spark] class BasicDriverFeatureStep( // The memory overhead factor to use. If the user has not set it, then use a different // value for non-JVM apps. This value is propagated to executors. private val overheadFactor = - if (conf.roleSpecificConf.mainAppResource.isInstanceOf[NonJVMResource]) { - if (conf.sparkConf.contains(MEMORY_OVERHEAD_FACTOR)) { + if (conf.mainAppResource.isInstanceOf[NonJVMResource]) { + if (conf.contains(MEMORY_OVERHEAD_FACTOR)) { conf.get(MEMORY_OVERHEAD_FACTOR) } else { NON_JVM_MEMORY_OVERHEAD_FACTOR @@ -68,8 +67,7 @@ private[spark] class BasicDriverFeatureStep( private val driverMemoryWithOverheadMiB = driverMemoryMiB + memoryOverheadMiB override def configurePod(pod: SparkPod): SparkPod = { - val driverCustomEnvs = conf.roleEnvs - .toSeq + val driverCustomEnvs = conf.environment.toSeq .map { env => new EnvVarBuilder() .withName(env._1) @@ -96,7 +94,7 @@ private[spark] class BasicDriverFeatureStep( val driverContainer = new ContainerBuilder(pod.container) .withName(Option(pod.container.getName).getOrElse(DEFAULT_DRIVER_CONTAINER_NAME)) .withImage(driverContainerImage) - .withImagePullPolicy(conf.imagePullPolicy()) + .withImagePullPolicy(conf.imagePullPolicy) .addNewPort() .withName(DRIVER_PORT_NAME) .withContainerPort(driverPort) @@ -130,13 +128,13 @@ private[spark] class BasicDriverFeatureStep( val driverPod = new PodBuilder(pod.pod) .editOrNewMetadata() .withName(driverPodName) - .addToLabels(conf.roleLabels.asJava) - .addToAnnotations(conf.roleAnnotations.asJava) + .addToLabels(conf.labels.asJava) + .addToAnnotations(conf.annotations.asJava) .endMetadata() .editOrNewSpec() .withRestartPolicy("Never") - .addToNodeSelector(conf.nodeSelector().asJava) - .addToImagePullSecrets(conf.imagePullSecrets(): _*) + .addToNodeSelector(conf.nodeSelector.asJava) + .addToImagePullSecrets(conf.imagePullSecrets: _*) .endSpec() .build() @@ -147,7 +145,7 @@ private[spark] class BasicDriverFeatureStep( val additionalProps = mutable.Map( KUBERNETES_DRIVER_POD_NAME.key -> driverPodName, "spark.app.id" -> conf.appId, - KUBERNETES_EXECUTOR_POD_NAME_PREFIX.key -> conf.appResourceNamePrefix, + KUBERNETES_EXECUTOR_POD_NAME_PREFIX.key -> conf.resourceNamePrefix, KUBERNETES_DRIVER_SUBMIT_CHECK.key -> "true", MEMORY_OVERHEAD_FACTOR.key -> overheadFactor.toString) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStep.scala index 7f397e6e84fa..8bf315248388 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStep.scala @@ -29,8 +29,7 @@ import org.apache.spark.rpc.RpcEndpointAddress import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend import org.apache.spark.util.Utils -private[spark] class BasicExecutorFeatureStep( - kubernetesConf: KubernetesConf[KubernetesExecutorSpecificConf]) +private[spark] class BasicExecutorFeatureStep(kubernetesConf: KubernetesExecutorConf) extends KubernetesFeatureConfigStep { // Consider moving some of these fields to KubernetesConf or KubernetesExecutorSpecificConf @@ -42,7 +41,7 @@ private[spark] class BasicExecutorFeatureStep( .sparkConf .getInt("spark.blockmanager.port", DEFAULT_BLOCKMANAGER_PORT) - private val executorPodNamePrefix = kubernetesConf.appResourceNamePrefix + private val executorPodNamePrefix = kubernetesConf.resourceNamePrefix private val driverUrl = RpcEndpointAddress( kubernetesConf.get("spark.driver.host"), @@ -76,7 +75,7 @@ private[spark] class BasicExecutorFeatureStep( private val executorLimitCores = kubernetesConf.get(KUBERNETES_EXECUTOR_LIMIT_CORES) override def configurePod(pod: SparkPod): SparkPod = { - val name = s"$executorPodNamePrefix-exec-${kubernetesConf.roleSpecificConf.executorId}" + val name = s"$executorPodNamePrefix-exec-${kubernetesConf.executorId}" // hostname must be no longer than 63 characters, so take the last 63 characters of the pod // name as the hostname. This preserves uniqueness since the end of name contains @@ -98,7 +97,7 @@ private[spark] class BasicExecutorFeatureStep( .get(EXECUTOR_JAVA_OPTIONS) .map { opts => val subsOpts = Utils.substituteAppNExecIds(opts, kubernetesConf.appId, - kubernetesConf.roleSpecificConf.executorId) + kubernetesConf.executorId) val delimitedOpts = Utils.splitCommandString(subsOpts) delimitedOpts.zipWithIndex.map { case (opt, index) => @@ -112,8 +111,8 @@ private[spark] class BasicExecutorFeatureStep( (ENV_APPLICATION_ID, kubernetesConf.appId), // This is to set the SPARK_CONF_DIR to be /opt/spark/conf (ENV_SPARK_CONF_DIR, SPARK_CONF_DIR_INTERNAL), - (ENV_EXECUTOR_ID, kubernetesConf.roleSpecificConf.executorId)) ++ - kubernetesConf.roleEnvs) + (ENV_EXECUTOR_ID, kubernetesConf.executorId)) ++ + kubernetesConf.environment) .map(env => new EnvVarBuilder() .withName(env._1) .withValue(env._2) @@ -138,7 +137,7 @@ private[spark] class BasicExecutorFeatureStep( val executorContainer = new ContainerBuilder(pod.container) .withName(Option(pod.container.getName).getOrElse(DEFAULT_EXECUTOR_CONTAINER_NAME)) .withImage(executorContainerImage) - .withImagePullPolicy(kubernetesConf.imagePullPolicy()) + .withImagePullPolicy(kubernetesConf.imagePullPolicy) .editOrNewResources() .addToRequests("memory", executorMemoryQuantity) .addToLimits("memory", executorMemoryQuantity) @@ -158,27 +157,27 @@ private[spark] class BasicExecutorFeatureStep( .endResources() .build() }.getOrElse(executorContainer) - val driverPod = kubernetesConf.roleSpecificConf.driverPod - val ownerReference = driverPod.map(pod => + val ownerReference = kubernetesConf.driverPod.map { pod => new OwnerReferenceBuilder() .withController(true) .withApiVersion(pod.getApiVersion) .withKind(pod.getKind) .withName(pod.getMetadata.getName) .withUid(pod.getMetadata.getUid) - .build()) + .build() + } val executorPod = new PodBuilder(pod.pod) .editOrNewMetadata() .withName(name) - .addToLabels(kubernetesConf.roleLabels.asJava) - .addToAnnotations(kubernetesConf.roleAnnotations.asJava) + .addToLabels(kubernetesConf.labels.asJava) + .addToAnnotations(kubernetesConf.annotations.asJava) .addToOwnerReferences(ownerReference.toSeq: _*) .endMetadata() .editOrNewSpec() .withHostname(hostname) .withRestartPolicy("Never") - .addToNodeSelector(kubernetesConf.nodeSelector().asJava) - .addToImagePullSecrets(kubernetesConf.imagePullSecrets(): _*) + .addToNodeSelector(kubernetesConf.nodeSelector.asJava) + .addToImagePullSecrets(kubernetesConf.imagePullSecrets: _*) .endSpec() .build() diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/DriverCommandFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/DriverCommandFeatureStep.scala index 8b8f0d01d49f..102c3ebd5af7 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/DriverCommandFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/DriverCommandFeatureStep.scala @@ -32,11 +32,9 @@ import org.apache.spark.util.Utils * Creates the driver command for running the user app, and propagates needed configuration so * executors can also find the app code. */ -private[spark] class DriverCommandFeatureStep(conf: KubernetesConf[KubernetesDriverSpecificConf]) +private[spark] class DriverCommandFeatureStep(driverConf: KubernetesDriverConf) extends KubernetesFeatureConfigStep { - private val driverConf = conf.roleSpecificConf - override def configurePod(pod: SparkPod): SparkPod = { driverConf.mainAppResource match { case JavaMainAppResource(_) => @@ -85,7 +83,7 @@ private[spark] class DriverCommandFeatureStep(conf: KubernetesConf[KubernetesDri val pythonEnvs = Seq(new EnvVarBuilder() .withName(ENV_PYSPARK_MAJOR_PYTHON_VERSION) - .withValue(conf.sparkConf.get(PYSPARK_MAJOR_PYTHON_VERSION)) + .withValue(driverConf.sparkConf.get(PYSPARK_MAJOR_PYTHON_VERSION)) .build()) ++ maybePythonFiles @@ -124,7 +122,7 @@ private[spark] class DriverCommandFeatureStep(conf: KubernetesConf[KubernetesDri } private def mergeFileList(key: String, filesToAdd: Seq[String]): Map[String, String] = { - val existing = Utils.stringToSeq(conf.sparkConf.get(key, "")) + val existing = Utils.stringToSeq(driverConf.sparkConf.get(key, "")) Map(key -> (existing ++ filesToAdd).distinct.mkString(",")) } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/DriverKubernetesCredentialsFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/DriverKubernetesCredentialsFeatureStep.scala index ff5ad6673b30..795ca49a3c87 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/DriverKubernetesCredentialsFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/DriverKubernetesCredentialsFeatureStep.scala @@ -28,7 +28,7 @@ import org.apache.spark.deploy.k8s.{KubernetesConf, SparkPod} import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ -private[spark] class DriverKubernetesCredentialsFeatureStep(kubernetesConf: KubernetesConf[_]) +private[spark] class DriverKubernetesCredentialsFeatureStep(kubernetesConf: KubernetesConf) extends KubernetesFeatureConfigStep { // TODO clean up this class, and credentials in general. See also SparkKubernetesClientFactory. // We should use a struct to hold all creds-related fields. A lot of the code is very repetitive. @@ -66,7 +66,7 @@ private[spark] class DriverKubernetesCredentialsFeatureStep(kubernetesConf: Kube clientCertDataBase64.isDefined private val driverCredentialsSecretName = - s"${kubernetesConf.appResourceNamePrefix}-kubernetes-credentials" + s"${kubernetesConf.resourceNamePrefix}-kubernetes-credentials" override def configurePod(pod: SparkPod): SparkPod = { if (!shouldMountSecret) { @@ -122,7 +122,7 @@ private[spark] class DriverKubernetesCredentialsFeatureStep(kubernetesConf: Kube val redactedTokens = kubernetesConf.sparkConf.getAll .filter(_._1.endsWith(OAUTH_TOKEN_CONF_SUFFIX)) .toMap - .mapValues( _ => "") + .map { case (k, v) => (k, "") } redactedTokens ++ resolvedMountedCaCertFile.map { file => Map( diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/DriverServiceFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/DriverServiceFeatureStep.scala index f2d7bbd08f30..42305457f4ff 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/DriverServiceFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/DriverServiceFeatureStep.scala @@ -20,13 +20,13 @@ import scala.collection.JavaConverters._ import io.fabric8.kubernetes.api.model.{HasMetadata, ServiceBuilder} -import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesDriverSpecificConf, SparkPod} +import org.apache.spark.deploy.k8s.{KubernetesDriverConf, SparkPod} import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.internal.Logging import org.apache.spark.util.{Clock, SystemClock} private[spark] class DriverServiceFeatureStep( - kubernetesConf: KubernetesConf[KubernetesDriverSpecificConf], + kubernetesConf: KubernetesDriverConf, clock: Clock = new SystemClock) extends KubernetesFeatureConfigStep with Logging { import DriverServiceFeatureStep._ @@ -38,7 +38,7 @@ private[spark] class DriverServiceFeatureStep( s"$DRIVER_HOST_KEY is not supported in Kubernetes mode, as the driver's hostname will be " + "managed via a Kubernetes service.") - private val preferredServiceName = s"${kubernetesConf.appResourceNamePrefix}$DRIVER_SVC_POSTFIX" + private val preferredServiceName = s"${kubernetesConf.resourceNamePrefix}$DRIVER_SVC_POSTFIX" private val resolvedServiceName = if (preferredServiceName.length <= MAX_SERVICE_NAME_LENGTH) { preferredServiceName } else { @@ -58,7 +58,7 @@ private[spark] class DriverServiceFeatureStep( override def configurePod(pod: SparkPod): SparkPod = pod override def getAdditionalPodSystemProperties(): Map[String, String] = { - val driverHostname = s"$resolvedServiceName.${kubernetesConf.namespace()}.svc" + val driverHostname = s"$resolvedServiceName.${kubernetesConf.namespace}.svc" Map(DRIVER_HOST_KEY -> driverHostname, "spark.driver.port" -> driverPort.toString, org.apache.spark.internal.config.DRIVER_BLOCK_MANAGER_PORT.key -> @@ -72,7 +72,7 @@ private[spark] class DriverServiceFeatureStep( .endMetadata() .withNewSpec() .withClusterIP("None") - .withSelector(kubernetesConf.roleLabels.asJava) + .withSelector(kubernetesConf.labels.asJava) .addNewPort() .withName(DRIVER_PORT_NAME) .withPort(driverPort) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/EnvSecretsFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/EnvSecretsFeatureStep.scala index 03ff7d48420f..d78f04dcc40e 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/EnvSecretsFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/EnvSecretsFeatureStep.scala @@ -20,14 +20,13 @@ import scala.collection.JavaConverters._ import io.fabric8.kubernetes.api.model.{ContainerBuilder, EnvVarBuilder, HasMetadata} -import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesRoleSpecificConf, SparkPod} +import org.apache.spark.deploy.k8s.{KubernetesConf, SparkPod} -private[spark] class EnvSecretsFeatureStep( - kubernetesConf: KubernetesConf[_ <: KubernetesRoleSpecificConf]) +private[spark] class EnvSecretsFeatureStep(kubernetesConf: KubernetesConf) extends KubernetesFeatureConfigStep { override def configurePod(pod: SparkPod): SparkPod = { val addedEnvSecrets = kubernetesConf - .roleSecretEnvNamesToKeyRefs + .secretEnvNamesToKeyRefs .map{ case (envName, keyRef) => // Keyref parts val keyRefParts = keyRef.split(":") @@ -50,8 +49,4 @@ private[spark] class EnvSecretsFeatureStep( .build() SparkPod(pod.pod, containerWithEnvVars) } - - override def getAdditionalPodSystemProperties(): Map[String, String] = Map.empty - - override def getAdditionalKubernetesResources(): Seq[HasMetadata] = Seq.empty } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopConfExecutorFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopConfExecutorFeatureStep.scala index fd09de2a918a..08b50e862861 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopConfExecutorFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopConfExecutorFeatureStep.scala @@ -16,9 +16,7 @@ */ package org.apache.spark.deploy.k8s.features -import io.fabric8.kubernetes.api.model.HasMetadata - -import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesExecutorSpecificConf, SparkPod} +import org.apache.spark.deploy.k8s.{KubernetesExecutorConf, SparkPod} import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.deploy.k8s.features.hadooputils.HadoopBootstrapUtil import org.apache.spark.internal.Logging @@ -28,21 +26,15 @@ import org.apache.spark.internal.Logging * containing Hadoop config files mounted as volumes and an ENV variable * pointed to the mounted file directory. */ -private[spark] class HadoopConfExecutorFeatureStep( - kubernetesConf: KubernetesConf[KubernetesExecutorSpecificConf]) +private[spark] class HadoopConfExecutorFeatureStep(conf: KubernetesExecutorConf) extends KubernetesFeatureConfigStep with Logging { override def configurePod(pod: SparkPod): SparkPod = { - val sparkConf = kubernetesConf.sparkConf - val hadoopConfDirCMapName = sparkConf.getOption(HADOOP_CONFIG_MAP_NAME) + val hadoopConfDirCMapName = conf.sparkConf.getOption(HADOOP_CONFIG_MAP_NAME) require(hadoopConfDirCMapName.isDefined, "Ensure that the env `HADOOP_CONF_DIR` is defined either in the client or " + " using pre-existing ConfigMaps") logInfo("HADOOP_CONF_DIR defined") HadoopBootstrapUtil.bootstrapHadoopConfDir(None, None, hadoopConfDirCMapName, pod) } - - override def getAdditionalPodSystemProperties(): Map[String, String] = Map.empty - - override def getAdditionalKubernetesResources(): Seq[HasMetadata] = Seq.empty } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopSparkUserExecutorFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopSparkUserExecutorFeatureStep.scala index 5b6a6d5a7db4..96c9f3964976 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopSparkUserExecutorFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopSparkUserExecutorFeatureStep.scala @@ -16,28 +16,19 @@ */ package org.apache.spark.deploy.k8s.features -import io.fabric8.kubernetes.api.model.HasMetadata - -import org.apache.spark.deploy.k8s.{KubernetesConf, SparkPod} +import org.apache.spark.deploy.k8s.{KubernetesExecutorConf, SparkPod} import org.apache.spark.deploy.k8s.Constants._ -import org.apache.spark.deploy.k8s.KubernetesExecutorSpecificConf import org.apache.spark.deploy.k8s.features.hadooputils.HadoopBootstrapUtil -import org.apache.spark.internal.Logging /** * This step is responsible for setting ENV_SPARK_USER when HADOOP_FILES are detected * however, this step would not be run if Kerberos is enabled, as Kerberos sets SPARK_USER */ -private[spark] class HadoopSparkUserExecutorFeatureStep( - kubernetesConf: KubernetesConf[KubernetesExecutorSpecificConf]) - extends KubernetesFeatureConfigStep with Logging { +private[spark] class HadoopSparkUserExecutorFeatureStep(conf: KubernetesExecutorConf) + extends KubernetesFeatureConfigStep { override def configurePod(pod: SparkPod): SparkPod = { - val sparkUserName = kubernetesConf.sparkConf.get(KERBEROS_SPARK_USER_NAME) + val sparkUserName = conf.sparkConf.get(KERBEROS_SPARK_USER_NAME) HadoopBootstrapUtil.bootstrapSparkUserPod(sparkUserName, pod) } - - override def getAdditionalPodSystemProperties(): Map[String, String] = Map.empty - - override def getAdditionalKubernetesResources(): Seq[HasMetadata] = Seq.empty } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/KerberosConfDriverFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/KerberosConfDriverFeatureStep.scala index ce47933b7f70..7fac6866411f 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/KerberosConfDriverFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/KerberosConfDriverFeatureStep.scala @@ -16,40 +16,45 @@ */ package org.apache.spark.deploy.k8s.features -import io.fabric8.kubernetes.api.model.HasMetadata +import io.fabric8.kubernetes.api.model.{HasMetadata, Secret, SecretBuilder} +import org.apache.commons.codec.binary.Base64 +import org.apache.hadoop.security.{Credentials, UserGroupInformation} import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesUtils, SparkPod} +import org.apache.spark.deploy.k8s.{KubernetesDriverConf, KubernetesUtils, SparkPod} import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ -import org.apache.spark.deploy.k8s.KubernetesDriverSpecificConf import org.apache.spark.deploy.k8s.features.hadooputils._ -import org.apache.spark.internal.Logging +import org.apache.spark.deploy.security.HadoopDelegationTokenManager /** * Runs the necessary Hadoop-based logic based on Kerberos configs and the presence of the * HADOOP_CONF_DIR. This runs various bootstrap methods defined in HadoopBootstrapUtil. */ -private[spark] class KerberosConfDriverFeatureStep( - kubernetesConf: KubernetesConf[KubernetesDriverSpecificConf]) - extends KubernetesFeatureConfigStep with Logging { +private[spark] class KerberosConfDriverFeatureStep(kubernetesConf: KubernetesDriverConf) + extends KubernetesFeatureConfigStep { - require(kubernetesConf.hadoopConfSpec.isDefined, - "Ensure that HADOOP_CONF_DIR is defined either via env or a pre-defined ConfigMap") - private val hadoopConfDirSpec = kubernetesConf.hadoopConfSpec.get private val conf = kubernetesConf.sparkConf + + private val hadoopConfDir = Option(kubernetesConf.sparkConf.getenv(ENV_HADOOP_CONF_DIR)) + private val hadoopConfigMapName = kubernetesConf.sparkConf.get(KUBERNETES_HADOOP_CONF_CONFIG_MAP) + KubernetesUtils.requireNandDefined( + hadoopConfDir, + hadoopConfigMapName, + "Do not specify both the `HADOOP_CONF_DIR` in your ENV and the ConfigMap " + + "as the creation of an additional ConfigMap, when one is already specified is extraneous") + private val principal = conf.get(org.apache.spark.internal.config.PRINCIPAL) private val keytab = conf.get(org.apache.spark.internal.config.KEYTAB) private val existingSecretName = conf.get(KUBERNETES_KERBEROS_DT_SECRET_NAME) private val existingSecretItemKey = conf.get(KUBERNETES_KERBEROS_DT_SECRET_ITEM_KEY) private val krb5File = conf.get(KUBERNETES_KERBEROS_KRB5_FILE) private val krb5CMap = conf.get(KUBERNETES_KERBEROS_KRB5_CONFIG_MAP) - private val kubeTokenManager = kubernetesConf.tokenManager(conf, - SparkHadoopUtil.get.newConfiguration(conf)) + private val hadoopConf = SparkHadoopUtil.get.newConfiguration(kubernetesConf.sparkConf) + private val tokenManager = new HadoopDelegationTokenManager(kubernetesConf.sparkConf, hadoopConf) private val isKerberosEnabled = - (hadoopConfDirSpec.hadoopConfDir.isDefined && kubeTokenManager.isSecurityEnabled) || - (hadoopConfDirSpec.hadoopConfigMapName.isDefined && - (krb5File.isDefined || krb5CMap.isDefined)) + (hadoopConfDir.isDefined && UserGroupInformation.isSecurityEnabled) || + (hadoopConfigMapName.isDefined && (krb5File.isDefined || krb5CMap.isDefined)) require(keytab.isEmpty || isKerberosEnabled, "You must enable Kerberos support if you are specifying a Kerberos Keytab") @@ -76,11 +81,11 @@ private[spark] class KerberosConfDriverFeatureStep( "If a secret storing a Kerberos Delegation Token is specified you must also" + " specify the item-key where the data is stored") - private val hadoopConfigurationFiles = hadoopConfDirSpec.hadoopConfDir.map { hConfDir => + private val hadoopConfigurationFiles = hadoopConfDir.map { hConfDir => HadoopBootstrapUtil.getHadoopConfFiles(hConfDir) } private val newHadoopConfigMapName = - if (hadoopConfDirSpec.hadoopConfigMapName.isEmpty) { + if (hadoopConfigMapName.isEmpty) { Some(kubernetesConf.hadoopConfigMapName) } else { None @@ -95,23 +100,24 @@ private[spark] class KerberosConfDriverFeatureStep( dtSecret = None, dtSecretName = secretName, dtSecretItemKey = secretItemKey, - jobUserName = kubeTokenManager.getCurrentUser.getShortUserName) + jobUserName = UserGroupInformation.getCurrentUser.getShortUserName) }).orElse( if (isKerberosEnabled) { - Some(HadoopKerberosLogin.buildSpec( - conf, - kubernetesConf.appResourceNamePrefix, - kubeTokenManager)) + Some(buildKerberosSpec()) } else { None } ) override def configurePod(pod: SparkPod): SparkPod = { + if (!isKerberosEnabled) { + return pod + } + val hadoopBasedSparkPod = HadoopBootstrapUtil.bootstrapHadoopConfDir( - hadoopConfDirSpec.hadoopConfDir, + hadoopConfDir, newHadoopConfigMapName, - hadoopConfDirSpec.hadoopConfigMapName, + hadoopConfigMapName, pod) kerberosConfSpec.map { hSpec => HadoopBootstrapUtil.bootstrapKerberosPod( @@ -124,11 +130,15 @@ private[spark] class KerberosConfDriverFeatureStep( hadoopBasedSparkPod) }.getOrElse( HadoopBootstrapUtil.bootstrapSparkUserPod( - kubeTokenManager.getCurrentUser.getShortUserName, + UserGroupInformation.getCurrentUser.getShortUserName, hadoopBasedSparkPod)) } override def getAdditionalPodSystemProperties(): Map[String, String] = { + if (!isKerberosEnabled) { + return Map.empty + } + val resolvedConfValues = kerberosConfSpec.map { hSpec => Map(KERBEROS_DT_SECRET_NAME -> hSpec.dtSecretName, KERBEROS_DT_SECRET_KEY -> hSpec.dtSecretItemKey, @@ -136,13 +146,16 @@ private[spark] class KerberosConfDriverFeatureStep( KRB5_CONFIG_MAP_NAME -> krb5CMap.getOrElse(kubernetesConf.krbConfigMapName)) }.getOrElse( Map(KERBEROS_SPARK_USER_NAME -> - kubeTokenManager.getCurrentUser.getShortUserName)) + UserGroupInformation.getCurrentUser.getShortUserName)) Map(HADOOP_CONFIG_MAP_NAME -> - hadoopConfDirSpec.hadoopConfigMapName.getOrElse( - kubernetesConf.hadoopConfigMapName)) ++ resolvedConfValues + hadoopConfigMapName.getOrElse(kubernetesConf.hadoopConfigMapName)) ++ resolvedConfValues } override def getAdditionalKubernetesResources(): Seq[HasMetadata] = { + if (!isKerberosEnabled) { + return Seq.empty + } + val hadoopConfConfigMap = for { hName <- newHadoopConfigMapName hFiles <- hadoopConfigurationFiles @@ -162,4 +175,34 @@ private[spark] class KerberosConfDriverFeatureStep( krb5ConfigMap.toSeq ++ kerberosDTSecret.toSeq } + + private def buildKerberosSpec(): KerberosConfigSpec = { + // The JobUserUGI will be taken fom the Local Ticket Cache or via keytab+principal + // The login happens in the SparkSubmit so login logic is not necessary to include + val jobUserUGI = UserGroupInformation.getCurrentUser + val creds = jobUserUGI.getCredentials + tokenManager.obtainDelegationTokens(creds) + val tokenData = SparkHadoopUtil.get.serialize(creds) + require(tokenData.nonEmpty, "Did not obtain any delegation tokens") + val newSecretName = + s"${kubernetesConf.resourceNamePrefix}-$KERBEROS_DELEGEGATION_TOKEN_SECRET_NAME" + val secretDT = + new SecretBuilder() + .withNewMetadata() + .withName(newSecretName) + .endMetadata() + .addToData(KERBEROS_SECRET_KEY, Base64.encodeBase64String(tokenData)) + .build() + KerberosConfigSpec( + dtSecret = Some(secretDT), + dtSecretName = newSecretName, + dtSecretItemKey = KERBEROS_SECRET_KEY, + jobUserName = jobUserUGI.getShortUserName) + } + + private case class KerberosConfigSpec( + dtSecret: Option[Secret], + dtSecretName: String, + dtSecretItemKey: String, + jobUserName: String) } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/KerberosConfExecutorFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/KerberosConfExecutorFeatureStep.scala index 06a88b6c229f..32bb6a5d2bcb 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/KerberosConfExecutorFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/KerberosConfExecutorFeatureStep.scala @@ -16,38 +16,29 @@ */ package org.apache.spark.deploy.k8s.features -import io.fabric8.kubernetes.api.model.HasMetadata - -import org.apache.spark.deploy.k8s.{KubernetesConf, SparkPod} +import org.apache.spark.deploy.k8s.{KubernetesExecutorConf, SparkPod} import org.apache.spark.deploy.k8s.Constants._ -import org.apache.spark.deploy.k8s.KubernetesExecutorSpecificConf import org.apache.spark.deploy.k8s.features.hadooputils.HadoopBootstrapUtil import org.apache.spark.internal.Logging /** * This step is responsible for mounting the DT secret for the executors */ -private[spark] class KerberosConfExecutorFeatureStep( - kubernetesConf: KubernetesConf[KubernetesExecutorSpecificConf]) +private[spark] class KerberosConfExecutorFeatureStep(conf: KubernetesExecutorConf) extends KubernetesFeatureConfigStep with Logging { - private val sparkConf = kubernetesConf.sparkConf - private val maybeKrb5CMap = sparkConf.getOption(KRB5_CONFIG_MAP_NAME) + private val maybeKrb5CMap = conf.getOption(KRB5_CONFIG_MAP_NAME) require(maybeKrb5CMap.isDefined, "HADOOP_CONF_DIR ConfigMap not found") override def configurePod(pod: SparkPod): SparkPod = { logInfo(s"Mounting Resources for Kerberos") HadoopBootstrapUtil.bootstrapKerberosPod( - sparkConf.get(KERBEROS_DT_SECRET_NAME), - sparkConf.get(KERBEROS_DT_SECRET_KEY), - sparkConf.get(KERBEROS_SPARK_USER_NAME), + conf.get(KERBEROS_DT_SECRET_NAME), + conf.get(KERBEROS_DT_SECRET_KEY), + conf.get(KERBEROS_SPARK_USER_NAME), None, None, maybeKrb5CMap, pod) } - - override def getAdditionalPodSystemProperties(): Map[String, String] = Map.empty - - override def getAdditionalKubernetesResources(): Seq[HasMetadata] = Seq.empty[HasMetadata] } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/LocalDirsFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/LocalDirsFeatureStep.scala index be386e119d46..19ed2df5551d 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/LocalDirsFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/LocalDirsFeatureStep.scala @@ -16,16 +16,15 @@ */ package org.apache.spark.deploy.k8s.features -import java.nio.file.Paths import java.util.UUID import io.fabric8.kubernetes.api.model.{ContainerBuilder, HasMetadata, PodBuilder, VolumeBuilder, VolumeMountBuilder} -import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesDriverSpecificConf, KubernetesRoleSpecificConf, SparkPod} +import org.apache.spark.deploy.k8s.{KubernetesConf, SparkPod} import org.apache.spark.deploy.k8s.Config._ private[spark] class LocalDirsFeatureStep( - conf: KubernetesConf[_ <: KubernetesRoleSpecificConf], + conf: KubernetesConf, defaultLocalDir: String = s"/var/data/spark-${UUID.randomUUID}") extends KubernetesFeatureConfigStep { @@ -73,8 +72,4 @@ private[spark] class LocalDirsFeatureStep( .build() SparkPod(podWithLocalDirVolumes, containerWithLocalDirVolumeMounts) } - - override def getAdditionalPodSystemProperties(): Map[String, String] = Map.empty - - override def getAdditionalKubernetesResources(): Seq[HasMetadata] = Seq.empty } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/MountSecretsFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/MountSecretsFeatureStep.scala index 97fa9499b2ed..f4e1a3a32672 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/MountSecretsFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/MountSecretsFeatureStep.scala @@ -18,14 +18,13 @@ package org.apache.spark.deploy.k8s.features import io.fabric8.kubernetes.api.model.{ContainerBuilder, HasMetadata, PodBuilder, VolumeBuilder, VolumeMountBuilder} -import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesRoleSpecificConf, SparkPod} +import org.apache.spark.deploy.k8s.{KubernetesConf, SparkPod} -private[spark] class MountSecretsFeatureStep( - kubernetesConf: KubernetesConf[_ <: KubernetesRoleSpecificConf]) +private[spark] class MountSecretsFeatureStep(kubernetesConf: KubernetesConf) extends KubernetesFeatureConfigStep { override def configurePod(pod: SparkPod): SparkPod = { val addedVolumes = kubernetesConf - .roleSecretNamesToMountPaths + .secretNamesToMountPaths .keys .map(secretName => new VolumeBuilder() @@ -40,7 +39,7 @@ private[spark] class MountSecretsFeatureStep( .endSpec() .build() val addedVolumeMounts = kubernetesConf - .roleSecretNamesToMountPaths + .secretNamesToMountPaths .map { case (secretName, mountPath) => new VolumeMountBuilder() @@ -54,9 +53,5 @@ private[spark] class MountSecretsFeatureStep( SparkPod(podWithVolumes, containerWithMounts) } - override def getAdditionalPodSystemProperties(): Map[String, String] = Map.empty - - override def getAdditionalKubernetesResources(): Seq[HasMetadata] = Seq.empty - private def secretVolumeName(secretName: String): String = s"$secretName-volume" } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/MountVolumesFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/MountVolumesFeatureStep.scala index e60259c4a9b5..7d9cdbe531e7 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/MountVolumesFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/MountVolumesFeatureStep.scala @@ -20,12 +20,11 @@ import io.fabric8.kubernetes.api.model._ import org.apache.spark.deploy.k8s._ -private[spark] class MountVolumesFeatureStep( - kubernetesConf: KubernetesConf[_ <: KubernetesRoleSpecificConf]) +private[spark] class MountVolumesFeatureStep(conf: KubernetesConf) extends KubernetesFeatureConfigStep { override def configurePod(pod: SparkPod): SparkPod = { - val (volumeMounts, volumes) = constructVolumes(kubernetesConf.roleVolumes).unzip + val (volumeMounts, volumes) = constructVolumes(conf.volumes).unzip val podWithVolumes = new PodBuilder(pod.pod) .editSpec() @@ -40,12 +39,8 @@ private[spark] class MountVolumesFeatureStep( SparkPod(podWithVolumes, containerWithVolumeMounts) } - override def getAdditionalPodSystemProperties(): Map[String, String] = Map.empty - - override def getAdditionalKubernetesResources(): Seq[HasMetadata] = Seq.empty - private def constructVolumes( - volumeSpecs: Iterable[KubernetesVolumeSpec[_ <: KubernetesVolumeSpecificConf]] + volumeSpecs: Iterable[KubernetesVolumeSpec] ): Iterable[(VolumeMount, Volume)] = { volumeSpecs.map { spec => val volumeMount = new VolumeMountBuilder() diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/PodTemplateConfigMapStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/PodTemplateConfigMapStep.scala index 28e2d1726ae2..09dcf93a54f8 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/PodTemplateConfigMapStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/PodTemplateConfigMapStep.scala @@ -22,12 +22,11 @@ import java.nio.charset.StandardCharsets import com.google.common.io.Files import io.fabric8.kubernetes.api.model.{ConfigMapBuilder, ContainerBuilder, HasMetadata, PodBuilder} -import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesRoleSpecificConf, SparkPod} +import org.apache.spark.deploy.k8s.{KubernetesConf, SparkPod} import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ -private[spark] class PodTemplateConfigMapStep( - conf: KubernetesConf[_ <: KubernetesRoleSpecificConf]) +private[spark] class PodTemplateConfigMapStep(conf: KubernetesConf) extends KubernetesFeatureConfigStep { def configurePod(pod: SparkPod): SparkPod = { val podWithVolume = new PodBuilder(pod.pod) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadooputils/HadoopKerberosLogin.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadooputils/HadoopKerberosLogin.scala deleted file mode 100644 index 0022d8f242a7..000000000000 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadooputils/HadoopKerberosLogin.scala +++ /dev/null @@ -1,64 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.spark.deploy.k8s.features.hadooputils - -import io.fabric8.kubernetes.api.model.SecretBuilder -import org.apache.commons.codec.binary.Base64 - -import org.apache.spark.SparkConf -import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.deploy.k8s.Constants._ -import org.apache.spark.deploy.k8s.security.KubernetesHadoopDelegationTokenManager - -/** - * This logic does all the heavy lifting for Delegation Token creation. This step - * assumes that the job user has either specified a principal and keytab or ran - * $kinit before running spark-submit. By running UGI.getCurrentUser we are able - * to obtain the current user, either signed in via $kinit or keytab. With the - * Job User principal you then retrieve the delegation token from the NameNode - * and store values in DelegationToken. Lastly, the class puts the data into - * a secret. All this is defined in a KerberosConfigSpec. - */ -private[spark] object HadoopKerberosLogin { - def buildSpec( - submissionSparkConf: SparkConf, - kubernetesResourceNamePrefix: String, - tokenManager: KubernetesHadoopDelegationTokenManager): KerberosConfigSpec = { - // The JobUserUGI will be taken fom the Local Ticket Cache or via keytab+principal - // The login happens in the SparkSubmit so login logic is not necessary to include - val jobUserUGI = tokenManager.getCurrentUser - val originalCredentials = jobUserUGI.getCredentials - tokenManager.obtainDelegationTokens(originalCredentials) - - val tokenData = SparkHadoopUtil.get.serialize(originalCredentials) - - val initialTokenDataKeyName = KERBEROS_SECRET_KEY - val newSecretName = s"$kubernetesResourceNamePrefix-$KERBEROS_DELEGEGATION_TOKEN_SECRET_NAME" - val secretDT = - new SecretBuilder() - .withNewMetadata() - .withName(newSecretName) - .endMetadata() - .addToData(initialTokenDataKeyName, Base64.encodeBase64String(tokenData)) - .build() - KerberosConfigSpec( - dtSecret = Some(secretDT), - dtSecretName = newSecretName, - dtSecretItemKey = initialTokenDataKeyName, - jobUserName = jobUserUGI.getShortUserName) - } -} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/security/KubernetesHadoopDelegationTokenManager.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/security/KubernetesHadoopDelegationTokenManager.scala deleted file mode 100644 index 3e98d5811d83..000000000000 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/security/KubernetesHadoopDelegationTokenManager.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.security - -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.security.UserGroupInformation - -import org.apache.spark.SparkConf -import org.apache.spark.deploy.security.HadoopDelegationTokenManager - -/** - * Adds Kubernetes-specific functionality to HadoopDelegationTokenManager. - */ -private[spark] class KubernetesHadoopDelegationTokenManager( - _sparkConf: SparkConf, - _hadoopConf: Configuration) - extends HadoopDelegationTokenManager(_sparkConf, _hadoopConf) { - - def getCurrentUser: UserGroupInformation = UserGroupInformation.getCurrentUser - def isSecurityEnabled: Boolean = UserGroupInformation.isSecurityEnabled - -} 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 543d6b16d6ae..70a93c968795 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 @@ -17,19 +17,17 @@ package org.apache.spark.deploy.k8s.submit import java.io.StringWriter -import java.util.{Collections, Locale, Properties, UUID} import java.util.{Collections, UUID} import java.util.Properties import io.fabric8.kubernetes.api.model._ import io.fabric8.kubernetes.client.KubernetesClient -import org.apache.hadoop.security.UserGroupInformation import scala.collection.mutable import scala.util.control.NonFatal import org.apache.spark.SparkConf import org.apache.spark.deploy.SparkApplication -import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesDriverSpecificConf, KubernetesUtils, SparkKubernetesClientFactory} +import org.apache.spark.deploy.k8s._ import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.internal.Logging @@ -47,8 +45,7 @@ private[spark] case class ClientArguments( mainAppResource: MainAppResource, mainClass: String, driverArgs: Array[String], - maybePyFiles: Option[String], - hadoopConfigDir: Option[String]) + maybePyFiles: Option[String]) private[spark] object ClientArguments { @@ -82,8 +79,7 @@ private[spark] object ClientArguments { mainAppResource, mainClass.get, driverArgs.toArray, - maybePyFiles, - sys.env.get(ENV_HADOOP_CONF_DIR)) + maybePyFiles) } } @@ -92,27 +88,24 @@ private[spark] object ClientArguments { * watcher that monitors and logs the application status. Waits for the application to terminate if * spark.kubernetes.submission.waitAppCompletion is true. * + * @param conf The kubernetes driver config. * @param builder Responsible for building the base driver pod based on a composition of * implemented features. - * @param kubernetesConf application 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 watcher a watcher that monitors and logs the application status */ private[spark] class Client( + conf: KubernetesDriverConf, builder: KubernetesDriverBuilder, - kubernetesConf: KubernetesConf[KubernetesDriverSpecificConf], kubernetesClient: KubernetesClient, waitForAppCompletion: Boolean, - appName: String, - watcher: LoggingPodStatusWatcher, - kubernetesResourceNamePrefix: String) extends Logging { + watcher: LoggingPodStatusWatcher) extends Logging { def run(): Unit = { - val resolvedDriverSpec = builder.buildFromFeatures(kubernetesConf) - val configMapName = s"$kubernetesResourceNamePrefix-driver-conf-map" + val resolvedDriverSpec = builder.buildFromFeatures(conf) + val configMapName = s"${conf.resourceNamePrefix}-driver-conf-map" val configMap = buildConfigMap(configMapName, resolvedDriverSpec.systemProperties) // The include of the ENV_VAR for "SPARK_CONF_DIR" is to allow for the // Spark command builder to pickup on the Java Options present in the ConfigMap @@ -155,11 +148,11 @@ private[spark] class Client( } if (waitForAppCompletion) { - logInfo(s"Waiting for application $appName to finish...") + logInfo(s"Waiting for application ${conf.appName} to finish...") watcher.awaitCompletion() - logInfo(s"Application $appName finished.") + logInfo(s"Application ${conf.appName} finished.") } else { - logInfo(s"Deployed Spark application $appName into Kubernetes.") + logInfo(s"Deployed Spark application ${conf.appName} into Kubernetes.") } } } @@ -216,19 +209,13 @@ private[spark] class KubernetesClientApplication extends SparkApplication { // a unique app ID (captured by spark.app.id) in the format below. val kubernetesAppId = s"spark-${UUID.randomUUID().toString.replaceAll("-", "")}" val waitForAppCompletion = sparkConf.get(WAIT_FOR_APP_COMPLETION) - val kubernetesResourceNamePrefix = KubernetesClientApplication.getResourceNamePrefix(appName) - sparkConf.set(KUBERNETES_PYSPARK_PY_FILES, clientArguments.maybePyFiles.getOrElse("")) val kubernetesConf = KubernetesConf.createDriverConf( sparkConf, - appName, - kubernetesResourceNamePrefix, kubernetesAppId, clientArguments.mainAppResource, clientArguments.mainClass, clientArguments.driverArgs, - clientArguments.maybePyFiles, - clientArguments.hadoopConfigDir) - val namespace = kubernetesConf.namespace() + clientArguments.maybePyFiles) // The master URL has been checked for validity already in SparkSubmit. // We just need to get rid of the "k8s://" prefix here. val master = KubernetesUtils.parseMasterUrl(sparkConf.get("spark.master")) @@ -238,36 +225,18 @@ private[spark] class KubernetesClientApplication extends SparkApplication { Utils.tryWithResource(SparkKubernetesClientFactory.createKubernetesClient( master, - Some(namespace), + Some(kubernetesConf.namespace), KUBERNETES_AUTH_SUBMISSION_CONF_PREFIX, sparkConf, None, None)) { kubernetesClient => val client = new Client( - KubernetesDriverBuilder(kubernetesClient, kubernetesConf.sparkConf), kubernetesConf, + KubernetesDriverBuilder(kubernetesClient, kubernetesConf.sparkConf), kubernetesClient, waitForAppCompletion, - appName, - watcher, - kubernetesResourceNamePrefix) + watcher) client.run() } } } - -private[spark] object KubernetesClientApplication { - - def getAppName(conf: SparkConf): String = conf.getOption("spark.app.name").getOrElse("spark") - - def getResourceNamePrefix(appName: String): String = { - val launchTime = System.currentTimeMillis() - s"$appName-$launchTime" - .trim - .toLowerCase(Locale.ROOT) - .replaceAll("\\s+", "-") - .replaceAll("\\.", "-") - .replaceAll("[^a-z0-9\\-]", "") - .replaceAll("-+", "-") - } -} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesDriverBuilder.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesDriverBuilder.scala index be4daec3b1bb..ab91f33ce2ba 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesDriverBuilder.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesDriverBuilder.scala @@ -21,57 +21,46 @@ import java.io.File import io.fabric8.kubernetes.client.KubernetesClient import org.apache.spark.SparkConf -import org.apache.spark.deploy.k8s.{Config, KubernetesConf, KubernetesDriverSpec, KubernetesDriverSpecificConf, KubernetesRoleSpecificConf, KubernetesUtils, SparkPod} +import org.apache.spark.deploy.k8s._ import org.apache.spark.deploy.k8s.features._ private[spark] class KubernetesDriverBuilder( - provideBasicStep: (KubernetesConf[KubernetesDriverSpecificConf]) => BasicDriverFeatureStep = + provideBasicStep: (KubernetesDriverConf => BasicDriverFeatureStep) = new BasicDriverFeatureStep(_), - provideCredentialsStep: (KubernetesConf[KubernetesDriverSpecificConf]) - => DriverKubernetesCredentialsFeatureStep = + provideCredentialsStep: (KubernetesDriverConf => DriverKubernetesCredentialsFeatureStep) = new DriverKubernetesCredentialsFeatureStep(_), - provideServiceStep: (KubernetesConf[KubernetesDriverSpecificConf]) => DriverServiceFeatureStep = + provideServiceStep: (KubernetesDriverConf => DriverServiceFeatureStep) = new DriverServiceFeatureStep(_), - provideSecretsStep: (KubernetesConf[_ <: KubernetesRoleSpecificConf] - => MountSecretsFeatureStep) = + provideSecretsStep: (KubernetesConf => MountSecretsFeatureStep) = new MountSecretsFeatureStep(_), - provideEnvSecretsStep: (KubernetesConf[_ <: KubernetesRoleSpecificConf] - => EnvSecretsFeatureStep) = + provideEnvSecretsStep: (KubernetesConf => EnvSecretsFeatureStep) = new EnvSecretsFeatureStep(_), - provideLocalDirsStep: (KubernetesConf[_ <: KubernetesRoleSpecificConf]) - => LocalDirsFeatureStep = + provideLocalDirsStep: (KubernetesConf => LocalDirsFeatureStep) = new LocalDirsFeatureStep(_), - provideVolumesStep: (KubernetesConf[_ <: KubernetesRoleSpecificConf] - => MountVolumesFeatureStep) = + provideVolumesStep: (KubernetesConf => MountVolumesFeatureStep) = new MountVolumesFeatureStep(_), - provideDriverCommandStep: ( - KubernetesConf[KubernetesDriverSpecificConf] - => DriverCommandFeatureStep) = + provideDriverCommandStep: (KubernetesDriverConf => DriverCommandFeatureStep) = new DriverCommandFeatureStep(_), - provideHadoopGlobalStep: ( - KubernetesConf[KubernetesDriverSpecificConf] - => KerberosConfDriverFeatureStep) = - new KerberosConfDriverFeatureStep(_), - providePodTemplateConfigMapStep: (KubernetesConf[_ <: KubernetesRoleSpecificConf] - => PodTemplateConfigMapStep) = - new PodTemplateConfigMapStep(_), + provideHadoopGlobalStep: (KubernetesDriverConf => KerberosConfDriverFeatureStep) = + new KerberosConfDriverFeatureStep(_), + providePodTemplateConfigMapStep: (KubernetesConf => PodTemplateConfigMapStep) = + new PodTemplateConfigMapStep(_), provideInitialPod: () => SparkPod = SparkPod.initialPod) { - def buildFromFeatures( - kubernetesConf: KubernetesConf[KubernetesDriverSpecificConf]): KubernetesDriverSpec = { + def buildFromFeatures(kubernetesConf: KubernetesDriverConf): KubernetesDriverSpec = { val baseFeatures = Seq( provideBasicStep(kubernetesConf), provideCredentialsStep(kubernetesConf), provideServiceStep(kubernetesConf), provideLocalDirsStep(kubernetesConf)) - val secretFeature = if (kubernetesConf.roleSecretNamesToMountPaths.nonEmpty) { + val secretFeature = if (kubernetesConf.secretNamesToMountPaths.nonEmpty) { Seq(provideSecretsStep(kubernetesConf)) } else Nil - val envSecretFeature = if (kubernetesConf.roleSecretEnvNamesToKeyRefs.nonEmpty) { + val envSecretFeature = if (kubernetesConf.secretEnvNamesToKeyRefs.nonEmpty) { Seq(provideEnvSecretsStep(kubernetesConf)) } else Nil - val volumesFeature = if (kubernetesConf.roleVolumes.nonEmpty) { + val volumesFeature = if (kubernetesConf.volumes.nonEmpty) { Seq(provideVolumesStep(kubernetesConf)) } else Nil val podTemplateFeature = if ( @@ -81,14 +70,12 @@ private[spark] class KubernetesDriverBuilder( val driverCommandStep = provideDriverCommandStep(kubernetesConf) - val maybeHadoopConfigStep = - kubernetesConf.hadoopConfSpec.map { _ => - provideHadoopGlobalStep(kubernetesConf)} + val hadoopConfigStep = Some(provideHadoopGlobalStep(kubernetesConf)) val allFeatures: Seq[KubernetesFeatureConfigStep] = baseFeatures ++ Seq(driverCommandStep) ++ secretFeature ++ envSecretFeature ++ volumesFeature ++ - maybeHadoopConfigStep.toSeq ++ podTemplateFeature + hadoopConfigStep ++ podTemplateFeature var spec = KubernetesDriverSpec( provideInitialPod(), diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesExecutorBuilder.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesExecutorBuilder.scala index 089f84dec277..37baac9849d5 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesExecutorBuilder.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesExecutorBuilder.scala @@ -26,50 +26,38 @@ import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.deploy.k8s.features._ private[spark] class KubernetesExecutorBuilder( - provideBasicStep: (KubernetesConf [KubernetesExecutorSpecificConf]) - => BasicExecutorFeatureStep = + provideBasicStep: (KubernetesExecutorConf => BasicExecutorFeatureStep) = new BasicExecutorFeatureStep(_), - provideSecretsStep: (KubernetesConf[_ <: KubernetesRoleSpecificConf]) - => MountSecretsFeatureStep = + provideSecretsStep: (KubernetesConf => MountSecretsFeatureStep) = new MountSecretsFeatureStep(_), - provideEnvSecretsStep: - (KubernetesConf[_ <: KubernetesRoleSpecificConf] => EnvSecretsFeatureStep) = + provideEnvSecretsStep: (KubernetesConf => EnvSecretsFeatureStep) = new EnvSecretsFeatureStep(_), - provideLocalDirsStep: (KubernetesConf[_ <: KubernetesRoleSpecificConf]) - => LocalDirsFeatureStep = + provideLocalDirsStep: (KubernetesConf => LocalDirsFeatureStep) = new LocalDirsFeatureStep(_), - provideVolumesStep: (KubernetesConf[_ <: KubernetesRoleSpecificConf] - => MountVolumesFeatureStep) = + provideVolumesStep: (KubernetesConf => MountVolumesFeatureStep) = new MountVolumesFeatureStep(_), - provideHadoopConfStep: ( - KubernetesConf[KubernetesExecutorSpecificConf] - => HadoopConfExecutorFeatureStep) = + provideHadoopConfStep: (KubernetesExecutorConf => HadoopConfExecutorFeatureStep) = new HadoopConfExecutorFeatureStep(_), - provideKerberosConfStep: ( - KubernetesConf[KubernetesExecutorSpecificConf] - => KerberosConfExecutorFeatureStep) = + provideKerberosConfStep: (KubernetesExecutorConf => KerberosConfExecutorFeatureStep) = new KerberosConfExecutorFeatureStep(_), - provideHadoopSparkUserStep: ( - KubernetesConf[KubernetesExecutorSpecificConf] - => HadoopSparkUserExecutorFeatureStep) = + provideHadoopSparkUserStep: (KubernetesExecutorConf => HadoopSparkUserExecutorFeatureStep) = new HadoopSparkUserExecutorFeatureStep(_), provideInitialPod: () => SparkPod = SparkPod.initialPod) { - def buildFromFeatures( - kubernetesConf: KubernetesConf[KubernetesExecutorSpecificConf]): SparkPod = { + def buildFromFeatures(kubernetesConf: KubernetesExecutorConf): SparkPod = { val sparkConf = kubernetesConf.sparkConf val maybeHadoopConfigMap = sparkConf.getOption(HADOOP_CONFIG_MAP_NAME) val maybeDTSecretName = sparkConf.getOption(KERBEROS_DT_SECRET_NAME) val maybeDTDataItem = sparkConf.getOption(KERBEROS_DT_SECRET_KEY) val baseFeatures = Seq(provideBasicStep(kubernetesConf), provideLocalDirsStep(kubernetesConf)) - val secretFeature = if (kubernetesConf.roleSecretNamesToMountPaths.nonEmpty) { + val secretFeature = if (kubernetesConf.secretNamesToMountPaths.nonEmpty) { Seq(provideSecretsStep(kubernetesConf)) } else Nil - val secretEnvFeature = if (kubernetesConf.roleSecretEnvNamesToKeyRefs.nonEmpty) { + val secretEnvFeature = if (kubernetesConf.secretEnvNamesToKeyRefs.nonEmpty) { Seq(provideEnvSecretsStep(kubernetesConf)) } else Nil - val volumesFeature = if (kubernetesConf.roleVolumes.nonEmpty) { + val volumesFeature = if (kubernetesConf.volumes.nonEmpty) { Seq(provideVolumesStep(kubernetesConf)) } else Nil diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/KubernetesConfSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/KubernetesConfSuite.scala index 41ca8d186c17..f4d40b0b3590 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/KubernetesConfSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/KubernetesConfSuite.scala @@ -26,10 +26,6 @@ import org.apache.spark.deploy.k8s.submit._ class KubernetesConfSuite extends SparkFunSuite { - private val APP_NAME = "test-app" - private val RESOURCE_NAME_PREFIX = "prefix" - private val APP_ID = "test-id" - private val MAIN_CLASS = "test-class" private val APP_ARGS = Array("arg1", "arg2") private val CUSTOM_LABELS = Map( "customLabel1Key" -> "customLabel1Value", @@ -49,26 +45,6 @@ class KubernetesConfSuite extends SparkFunSuite { private val DRIVER_POD = new PodBuilder().build() private val EXECUTOR_ID = "executor-id" - test("Basic driver translated fields.") { - val sparkConf = new SparkConf(false) - val conf = KubernetesConf.createDriverConf( - sparkConf, - APP_NAME, - RESOURCE_NAME_PREFIX, - APP_ID, - mainAppResource = JavaMainAppResource(None), - MAIN_CLASS, - APP_ARGS, - maybePyFiles = None, - hadoopConfDir = None) - assert(conf.appId === APP_ID) - assert(conf.sparkConf.getAll.toMap === sparkConf.getAll.toMap) - assert(conf.appResourceNamePrefix === RESOURCE_NAME_PREFIX) - assert(conf.roleSpecificConf.appName === APP_NAME) - assert(conf.roleSpecificConf.mainClass === MAIN_CLASS) - assert(conf.roleSpecificConf.appArgs === APP_ARGS) - } - test("Resolve driver labels, annotations, secret mount paths, envs, and memory overhead") { val sparkConf = new SparkConf(false) .set(MEMORY_OVERHEAD_FACTOR, 0.3) @@ -90,22 +66,19 @@ class KubernetesConfSuite extends SparkFunSuite { val conf = KubernetesConf.createDriverConf( sparkConf, - APP_NAME, - RESOURCE_NAME_PREFIX, - APP_ID, - mainAppResource = JavaMainAppResource(None), - MAIN_CLASS, + KubernetesTestConf.APP_ID, + JavaMainAppResource(None), + KubernetesTestConf.MAIN_CLASS, APP_ARGS, - maybePyFiles = None, - hadoopConfDir = None) - assert(conf.roleLabels === Map( - SPARK_APP_ID_LABEL -> APP_ID, + None) + assert(conf.labels === Map( + SPARK_APP_ID_LABEL -> KubernetesTestConf.APP_ID, SPARK_ROLE_LABEL -> SPARK_POD_DRIVER_ROLE) ++ CUSTOM_LABELS) - assert(conf.roleAnnotations === CUSTOM_ANNOTATIONS) - assert(conf.roleSecretNamesToMountPaths === SECRET_NAMES_TO_MOUNT_PATHS) - assert(conf.roleSecretEnvNamesToKeyRefs === SECRET_ENV_VARS) - assert(conf.roleEnvs === CUSTOM_ENVS) + assert(conf.annotations === CUSTOM_ANNOTATIONS) + assert(conf.secretNamesToMountPaths === SECRET_NAMES_TO_MOUNT_PATHS) + assert(conf.secretEnvNamesToKeyRefs === SECRET_ENV_VARS) + assert(conf.environment === CUSTOM_ENVS) assert(conf.sparkConf.get(MEMORY_OVERHEAD_FACTOR) === 0.3) } @@ -113,20 +86,20 @@ class KubernetesConfSuite extends SparkFunSuite { val conf = KubernetesConf.createExecutorConf( new SparkConf(false), EXECUTOR_ID, - APP_ID, + KubernetesTestConf.APP_ID, Some(DRIVER_POD)) - assert(conf.roleSpecificConf.executorId === EXECUTOR_ID) - assert(conf.roleSpecificConf.driverPod.get === DRIVER_POD) + assert(conf.executorId === EXECUTOR_ID) + assert(conf.driverPod.get === DRIVER_POD) } test("Image pull secrets.") { val conf = KubernetesConf.createExecutorConf( new SparkConf(false) - .set(IMAGE_PULL_SECRETS, "my-secret-1,my-secret-2 "), + .set(IMAGE_PULL_SECRETS, Seq("my-secret-1", "my-secret-2 ")), EXECUTOR_ID, - APP_ID, + KubernetesTestConf.APP_ID, Some(DRIVER_POD)) - assert(conf.imagePullSecrets() === + assert(conf.imagePullSecrets === Seq( new LocalObjectReferenceBuilder().withName("my-secret-1").build(), new LocalObjectReferenceBuilder().withName("my-secret-2").build())) @@ -150,14 +123,14 @@ class KubernetesConfSuite extends SparkFunSuite { val conf = KubernetesConf.createExecutorConf( sparkConf, EXECUTOR_ID, - APP_ID, + KubernetesTestConf.APP_ID, Some(DRIVER_POD)) - assert(conf.roleLabels === Map( + assert(conf.labels === Map( SPARK_EXECUTOR_ID_LABEL -> EXECUTOR_ID, - SPARK_APP_ID_LABEL -> APP_ID, + SPARK_APP_ID_LABEL -> KubernetesTestConf.APP_ID, SPARK_ROLE_LABEL -> SPARK_POD_EXECUTOR_ROLE) ++ CUSTOM_LABELS) - assert(conf.roleAnnotations === CUSTOM_ANNOTATIONS) - assert(conf.roleSecretNamesToMountPaths === SECRET_NAMES_TO_MOUNT_PATHS) - assert(conf.roleSecretEnvNamesToKeyRefs === SECRET_ENV_VARS) + assert(conf.annotations === CUSTOM_ANNOTATIONS) + assert(conf.secretNamesToMountPaths === SECRET_NAMES_TO_MOUNT_PATHS) + assert(conf.secretEnvNamesToKeyRefs === SECRET_ENV_VARS) } } diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/KubernetesTestConf.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/KubernetesTestConf.scala new file mode 100644 index 000000000000..cfa9f150c168 --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/KubernetesTestConf.scala @@ -0,0 +1,134 @@ +/* + * 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.Pod + +import org.apache.spark.SparkConf +import org.apache.spark.deploy.k8s.Config._ +import org.apache.spark.deploy.k8s.Constants._ +import org.apache.spark.deploy.k8s.submit.{JavaMainAppResource, MainAppResource} + +/** + * Builder methods for KubernetesConf that allow easy control over what to return for a few + * properties. For use with tests instead of having to mock specific properties. + */ +object KubernetesTestConf { + + val APP_ID = "appId" + val MAIN_CLASS = "mainClass" + val RESOURCE_PREFIX = "prefix" + val EXECUTOR_ID = "1" + + private val DEFAULT_CONF = new SparkConf(false) + + // scalastyle:off argcount + def createDriverConf( + sparkConf: SparkConf = DEFAULT_CONF, + appId: String = APP_ID, + mainAppResource: MainAppResource = JavaMainAppResource(None), + mainClass: String = MAIN_CLASS, + appArgs: Array[String] = Array.empty, + pyFiles: Seq[String] = Nil, + resourceNamePrefix: Option[String] = None, + labels: Map[String, String] = Map.empty, + environment: Map[String, String] = Map.empty, + annotations: Map[String, String] = Map.empty, + secretEnvNamesToKeyRefs: Map[String, String] = Map.empty, + secretNamesToMountPaths: Map[String, String] = Map.empty, + volumes: Seq[KubernetesVolumeSpec] = Seq.empty): KubernetesDriverConf = { + val conf = sparkConf.clone() + + resourceNamePrefix.foreach { prefix => + conf.set(KUBERNETES_DRIVER_POD_NAME_PREFIX, prefix) + } + setPrefixedConfigs(conf, KUBERNETES_DRIVER_LABEL_PREFIX, labels) + setPrefixedConfigs(conf, KUBERNETES_DRIVER_ENV_PREFIX, environment) + setPrefixedConfigs(conf, KUBERNETES_DRIVER_ANNOTATION_PREFIX, annotations) + setPrefixedConfigs(conf, KUBERNETES_DRIVER_SECRETS_PREFIX, secretNamesToMountPaths) + setPrefixedConfigs(conf, KUBERNETES_DRIVER_SECRET_KEY_REF_PREFIX, secretEnvNamesToKeyRefs) + setVolumeSpecs(conf, KUBERNETES_DRIVER_VOLUMES_PREFIX, volumes) + + new KubernetesDriverConf(conf, appId, mainAppResource, mainClass, appArgs, pyFiles) + } + // scalastyle:on argcount + + def createExecutorConf( + sparkConf: SparkConf = DEFAULT_CONF, + driverPod: Option[Pod] = None, + labels: Map[String, String] = Map.empty, + environment: Map[String, String] = Map.empty, + annotations: Map[String, String] = Map.empty, + secretEnvNamesToKeyRefs: Map[String, String] = Map.empty, + secretNamesToMountPaths: Map[String, String] = Map.empty, + volumes: Seq[KubernetesVolumeSpec] = Seq.empty): KubernetesExecutorConf = { + val conf = sparkConf.clone() + + setPrefixedConfigs(conf, KUBERNETES_EXECUTOR_LABEL_PREFIX, labels) + setPrefixedConfigs(conf, "spark.executorEnv.", environment) + setPrefixedConfigs(conf, KUBERNETES_EXECUTOR_ANNOTATION_PREFIX, annotations) + setPrefixedConfigs(conf, KUBERNETES_EXECUTOR_SECRETS_PREFIX, secretNamesToMountPaths) + setPrefixedConfigs(conf, KUBERNETES_EXECUTOR_SECRET_KEY_REF_PREFIX, secretEnvNamesToKeyRefs) + setVolumeSpecs(conf, KUBERNETES_EXECUTOR_VOLUMES_PREFIX, volumes) + + new KubernetesExecutorConf(conf, APP_ID, EXECUTOR_ID, driverPod) + } + + private def setPrefixedConfigs( + conf: SparkConf, + prefix: String, + values: Map[String, String]): Unit = { + values.foreach { case (k, v) => + conf.set(s"${prefix}$k", v) + } + } + + private def setVolumeSpecs( + conf: SparkConf, + prefix: String, + volumes: Seq[KubernetesVolumeSpec]): Unit = { + def key(vtype: String, vname: String, subkey: String): String = { + s"${prefix}$vtype.$vname.$subkey" + } + + volumes.foreach { case spec => + val (vtype, configs) = spec.volumeConf match { + case KubernetesHostPathVolumeConf(path) => + (KUBERNETES_VOLUMES_HOSTPATH_TYPE, + Map(KUBERNETES_VOLUMES_OPTIONS_PATH_KEY -> path)) + + case KubernetesPVCVolumeConf(claimName) => + (KUBERNETES_VOLUMES_PVC_TYPE, + Map(KUBERNETES_VOLUMES_OPTIONS_CLAIM_NAME_KEY -> claimName)) + + case KubernetesEmptyDirVolumeConf(medium, sizeLimit) => + val mconf = medium.map { m => (KUBERNETES_VOLUMES_OPTIONS_MEDIUM_KEY, m) }.toMap + val lconf = sizeLimit.map { l => (KUBERNETES_VOLUMES_OPTIONS_SIZE_LIMIT_KEY, l) }.toMap + (KUBERNETES_VOLUMES_EMPTYDIR_TYPE, mconf ++ lconf) + } + + conf.set(key(vtype, spec.volumeName, KUBERNETES_VOLUMES_MOUNT_PATH_KEY), spec.mountPath) + conf.set(key(vtype, spec.volumeName, KUBERNETES_VOLUMES_MOUNT_READONLY_KEY), + spec.mountReadOnly.toString) + configs.foreach { case (k, v) => + conf.set(key(vtype, spec.volumeName, k), v) + } + } + } + +} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/KubernetesVolumeUtilsSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/KubernetesVolumeUtilsSuite.scala index d795d159773a..58ab1fcf334f 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/KubernetesVolumeUtilsSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/KubernetesVolumeUtilsSuite.scala @@ -25,7 +25,7 @@ class KubernetesVolumeUtilsSuite extends SparkFunSuite { sparkConf.set("test.hostPath.volumeName.mount.readOnly", "true") sparkConf.set("test.hostPath.volumeName.options.path", "/hostPath") - val volumeSpec = KubernetesVolumeUtils.parseVolumesWithPrefix(sparkConf, "test.").head.get + val volumeSpec = KubernetesVolumeUtils.parseVolumesWithPrefix(sparkConf, "test.").head assert(volumeSpec.volumeName === "volumeName") assert(volumeSpec.mountPath === "/path") assert(volumeSpec.mountReadOnly === true) @@ -39,7 +39,7 @@ class KubernetesVolumeUtilsSuite extends SparkFunSuite { sparkConf.set("test.persistentVolumeClaim.volumeName.mount.readOnly", "true") sparkConf.set("test.persistentVolumeClaim.volumeName.options.claimName", "claimeName") - val volumeSpec = KubernetesVolumeUtils.parseVolumesWithPrefix(sparkConf, "test.").head.get + val volumeSpec = KubernetesVolumeUtils.parseVolumesWithPrefix(sparkConf, "test.").head assert(volumeSpec.volumeName === "volumeName") assert(volumeSpec.mountPath === "/path") assert(volumeSpec.mountReadOnly === true) @@ -54,7 +54,7 @@ class KubernetesVolumeUtilsSuite extends SparkFunSuite { sparkConf.set("test.emptyDir.volumeName.options.medium", "medium") sparkConf.set("test.emptyDir.volumeName.options.sizeLimit", "5G") - val volumeSpec = KubernetesVolumeUtils.parseVolumesWithPrefix(sparkConf, "test.").head.get + val volumeSpec = KubernetesVolumeUtils.parseVolumesWithPrefix(sparkConf, "test.").head assert(volumeSpec.volumeName === "volumeName") assert(volumeSpec.mountPath === "/path") assert(volumeSpec.mountReadOnly === true) @@ -67,7 +67,7 @@ class KubernetesVolumeUtilsSuite extends SparkFunSuite { sparkConf.set("test.emptyDir.volumeName.mount.path", "/path") sparkConf.set("test.emptyDir.volumeName.mount.readOnly", "true") - val volumeSpec = KubernetesVolumeUtils.parseVolumesWithPrefix(sparkConf, "test.").head.get + val volumeSpec = KubernetesVolumeUtils.parseVolumesWithPrefix(sparkConf, "test.").head assert(volumeSpec.volumeName === "volumeName") assert(volumeSpec.mountPath === "/path") assert(volumeSpec.mountReadOnly === true) @@ -80,27 +80,29 @@ class KubernetesVolumeUtilsSuite extends SparkFunSuite { sparkConf.set("test.hostPath.volumeName.mount.path", "/path") sparkConf.set("test.hostPath.volumeName.options.path", "/hostPath") - val volumeSpec = KubernetesVolumeUtils.parseVolumesWithPrefix(sparkConf, "test.").head.get + val volumeSpec = KubernetesVolumeUtils.parseVolumesWithPrefix(sparkConf, "test.").head assert(volumeSpec.mountReadOnly === false) } - test("Gracefully fails on missing mount key") { + test("Fails on missing mount key") { val sparkConf = new SparkConf(false) sparkConf.set("test.emptyDir.volumeName.mnt.path", "/path") - val volumeSpec = KubernetesVolumeUtils.parseVolumesWithPrefix(sparkConf, "test.").head - assert(volumeSpec.isFailure === true) - assert(volumeSpec.failed.get.getMessage === "emptyDir.volumeName.mount.path") + val e = intercept[NoSuchElementException] { + KubernetesVolumeUtils.parseVolumesWithPrefix(sparkConf, "test.") + } + assert(e.getMessage.contains("emptyDir.volumeName.mount.path")) } - test("Gracefully fails on missing option key") { + test("Fails on missing option key") { val sparkConf = new SparkConf(false) sparkConf.set("test.hostPath.volumeName.mount.path", "/path") sparkConf.set("test.hostPath.volumeName.mount.readOnly", "true") sparkConf.set("test.hostPath.volumeName.options.pth", "/hostPath") - val volumeSpec = KubernetesVolumeUtils.parseVolumesWithPrefix(sparkConf, "test.").head - assert(volumeSpec.isFailure === true) - assert(volumeSpec.failed.get.getMessage === "hostPath.volumeName.options.path") + val e = intercept[NoSuchElementException] { + KubernetesVolumeUtils.parseVolumesWithPrefix(sparkConf, "test.") + } + assert(e.getMessage.contains("hostPath.volumeName.options.path")) } } diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStepSuite.scala index 1e7dfbeffdb2..e4951bc1e69e 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStepSuite.scala @@ -21,7 +21,7 @@ import scala.collection.JavaConverters._ import io.fabric8.kubernetes.api.model.{ContainerPort, ContainerPortBuilder, LocalObjectReferenceBuilder} import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesDriverSpecificConf, SparkPod} +import org.apache.spark.deploy.k8s.{KubernetesTestConf, SparkPod} import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.deploy.k8s.submit._ @@ -30,32 +30,17 @@ import org.apache.spark.ui.SparkUI class BasicDriverFeatureStepSuite extends SparkFunSuite { - private val APP_ID = "spark-app-id" - private val RESOURCE_NAME_PREFIX = "spark" private val DRIVER_LABELS = Map("labelkey" -> "labelvalue") private val CONTAINER_IMAGE_PULL_POLICY = "IfNotPresent" - private val APP_NAME = "spark-test" - private val MAIN_CLASS = "org.apache.spark.examples.SparkPi" - private val PY_MAIN_CLASS = "org.apache.spark.deploy.PythonRunner" - private val APP_ARGS = Array("arg1", "arg2", "\"arg 3\"") - private val CUSTOM_ANNOTATION_KEY = "customAnnotation" - private val CUSTOM_ANNOTATION_VALUE = "customAnnotationValue" - private val DRIVER_ANNOTATIONS = Map(CUSTOM_ANNOTATION_KEY -> CUSTOM_ANNOTATION_VALUE) - private val DRIVER_CUSTOM_ENV1 = "customDriverEnv1" - private val DRIVER_CUSTOM_ENV2 = "customDriverEnv2" + private val DRIVER_ANNOTATIONS = Map("customAnnotation" -> "customAnnotationValue") private val DRIVER_ENVS = Map( - DRIVER_CUSTOM_ENV1 -> DRIVER_CUSTOM_ENV1, - DRIVER_CUSTOM_ENV2 -> DRIVER_CUSTOM_ENV2) + "customDriverEnv1" -> "customDriverEnv2", + "customDriverEnv2" -> "customDriverEnv2") private val TEST_IMAGE_PULL_SECRETS = Seq("my-secret-1", "my-secret-2") private val TEST_IMAGE_PULL_SECRET_OBJECTS = TEST_IMAGE_PULL_SECRETS.map { secret => new LocalObjectReferenceBuilder().withName(secret).build() } - private val emptyDriverSpecificConf = KubernetesDriverSpecificConf( - JavaMainAppResource(None), - APP_NAME, - MAIN_CLASS, - APP_ARGS) test("Check the pod respects all configurations from the user.") { val sparkConf = new SparkConf() @@ -65,19 +50,12 @@ class BasicDriverFeatureStepSuite extends SparkFunSuite { .set(DRIVER_MEMORY.key, "256M") .set(DRIVER_MEMORY_OVERHEAD, 200L) .set(CONTAINER_IMAGE, "spark-driver:latest") - .set(IMAGE_PULL_SECRETS, TEST_IMAGE_PULL_SECRETS.mkString(",")) - val kubernetesConf = KubernetesConf( - sparkConf, - emptyDriverSpecificConf, - RESOURCE_NAME_PREFIX, - APP_ID, - DRIVER_LABELS, - DRIVER_ANNOTATIONS, - Map.empty, - Map.empty, - DRIVER_ENVS, - Nil, - hadoopConfSpec = None) + .set(IMAGE_PULL_SECRETS, TEST_IMAGE_PULL_SECRETS) + val kubernetesConf = KubernetesTestConf.createDriverConf( + sparkConf = sparkConf, + labels = DRIVER_LABELS, + environment = DRIVER_ENVS, + annotations = DRIVER_ANNOTATIONS) val featureStep = new BasicDriverFeatureStep(kubernetesConf) val basePod = SparkPod.initialPod() @@ -99,10 +77,11 @@ class BasicDriverFeatureStepSuite extends SparkFunSuite { val envs = configuredPod.container .getEnv .asScala - .map(env => (env.getName, env.getValue)) + .map { env => (env.getName, env.getValue) } .toMap - assert(envs(DRIVER_CUSTOM_ENV1) === DRIVER_ENVS(DRIVER_CUSTOM_ENV1)) - assert(envs(DRIVER_CUSTOM_ENV2) === DRIVER_ENVS(DRIVER_CUSTOM_ENV2)) + DRIVER_ENVS.foreach { case (k, v) => + assert(envs(v) === v) + } assert(configuredPod.pod.getSpec().getImagePullSecrets.asScala === TEST_IMAGE_PULL_SECRET_OBJECTS) @@ -122,13 +101,15 @@ class BasicDriverFeatureStepSuite extends SparkFunSuite { val driverPodMetadata = configuredPod.pod.getMetadata assert(driverPodMetadata.getName === "spark-driver-pod") - assert(driverPodMetadata.getLabels.asScala === DRIVER_LABELS) + DRIVER_LABELS.foreach { case (k, v) => + assert(driverPodMetadata.getLabels.get(k) === v) + } assert(driverPodMetadata.getAnnotations.asScala === DRIVER_ANNOTATIONS) assert(configuredPod.pod.getSpec.getRestartPolicy === "Never") val expectedSparkConf = Map( KUBERNETES_DRIVER_POD_NAME.key -> "spark-driver-pod", - "spark.app.id" -> APP_ID, - KUBERNETES_EXECUTOR_POD_NAME_PREFIX.key -> RESOURCE_NAME_PREFIX, + "spark.app.id" -> KubernetesTestConf.APP_ID, + KUBERNETES_EXECUTOR_POD_NAME_PREFIX.key -> kubernetesConf.resourceNamePrefix, "spark.kubernetes.submitInDriver" -> "true", MEMORY_OVERHEAD_FACTOR.key -> MEMORY_OVERHEAD_FACTOR.defaultValue.get.toString) assert(featureStep.getAdditionalPodSystemProperties() === expectedSparkConf) @@ -141,39 +122,10 @@ class BasicDriverFeatureStepSuite extends SparkFunSuite { val pythonSparkConf = new SparkConf() .set(DRIVER_MEMORY.key, "4g") .set(CONTAINER_IMAGE, "spark-driver-py:latest") - val javaKubernetesConf = KubernetesConf( - javaSparkConf, - KubernetesDriverSpecificConf( - JavaMainAppResource(None), - APP_NAME, - PY_MAIN_CLASS, - APP_ARGS), - RESOURCE_NAME_PREFIX, - APP_ID, - DRIVER_LABELS, - DRIVER_ANNOTATIONS, - Map.empty, - Map.empty, - DRIVER_ENVS, - Nil, - hadoopConfSpec = None) - - val pythonKubernetesConf = KubernetesConf( - pythonSparkConf, - KubernetesDriverSpecificConf( - PythonMainAppResource(""), - APP_NAME, - PY_MAIN_CLASS, - APP_ARGS), - RESOURCE_NAME_PREFIX, - APP_ID, - DRIVER_LABELS, - DRIVER_ANNOTATIONS, - Map.empty, - Map.empty, - DRIVER_ENVS, - Nil, - hadoopConfSpec = None) + val javaKubernetesConf = KubernetesTestConf.createDriverConf(sparkConf = javaSparkConf) + val pythonKubernetesConf = KubernetesTestConf.createDriverConf( + sparkConf = pythonSparkConf, + mainAppResource = PythonMainAppResource("")) val javaFeatureStep = new BasicDriverFeatureStep(javaKubernetesConf) val pythonFeatureStep = new BasicDriverFeatureStep(pythonKubernetesConf) val basePod = SparkPod.initialPod() @@ -191,25 +143,14 @@ class BasicDriverFeatureStepSuite extends SparkFunSuite { .setJars(allJars) .set("spark.files", allFiles.mkString(",")) .set(CONTAINER_IMAGE, "spark-driver:latest") - val kubernetesConf = KubernetesConf( - sparkConf, - emptyDriverSpecificConf, - RESOURCE_NAME_PREFIX, - APP_ID, - DRIVER_LABELS, - DRIVER_ANNOTATIONS, - Map.empty, - Map.empty, - DRIVER_ENVS, - Nil, - hadoopConfSpec = None) + val kubernetesConf = KubernetesTestConf.createDriverConf(sparkConf = sparkConf) val step = new BasicDriverFeatureStep(kubernetesConf) val additionalProperties = step.getAdditionalPodSystemProperties() val expectedSparkConf = Map( KUBERNETES_DRIVER_POD_NAME.key -> "spark-driver-pod", - "spark.app.id" -> APP_ID, - KUBERNETES_EXECUTOR_POD_NAME_PREFIX.key -> RESOURCE_NAME_PREFIX, + "spark.app.id" -> KubernetesTestConf.APP_ID, + KUBERNETES_EXECUTOR_POD_NAME_PREFIX.key -> kubernetesConf.resourceNamePrefix, "spark.kubernetes.submitInDriver" -> "true", "spark.jars" -> "/opt/spark/jar1.jar,hdfs:///opt/spark/jar2.jar", "spark.files" -> "https://localhost:9000/file1.txt,/opt/spark/file2.txt", @@ -234,19 +175,9 @@ class BasicDriverFeatureStepSuite extends SparkFunSuite { .set(CONTAINER_IMAGE, "spark-driver:latest") .set(DRIVER_MEMORY.key, s"${driverMem.toInt}m") factor.foreach { value => sparkConf.set(MEMORY_OVERHEAD_FACTOR, value) } - val driverConf = emptyDriverSpecificConf.copy(mainAppResource = resource) - val conf = KubernetesConf( - sparkConf, - driverConf, - RESOURCE_NAME_PREFIX, - APP_ID, - DRIVER_LABELS, - DRIVER_ANNOTATIONS, - Map.empty, - Map.empty, - DRIVER_ENVS, - Nil, - hadoopConfSpec = None) + val conf = KubernetesTestConf.createDriverConf( + sparkConf = sparkConf, + mainAppResource = resource) val step = new BasicDriverFeatureStep(conf) val pod = step.configurePod(SparkPod.initialPod()) val mem = pod.container.getResources.getRequests.get("memory").getAmount() diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStepSuite.scala index e9a16aab6ccc..d6003c977937 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStepSuite.scala @@ -19,20 +19,18 @@ package org.apache.spark.deploy.k8s.features import scala.collection.JavaConverters._ import io.fabric8.kubernetes.api.model._ -import org.mockito.MockitoAnnotations -import org.scalatest.{BeforeAndAfter, BeforeAndAfterEach} +import org.scalatest.BeforeAndAfter import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesExecutorSpecificConf, SparkPod} +import org.apache.spark.deploy.k8s.{KubernetesExecutorConf, KubernetesTestConf, SparkPod} import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ +import org.apache.spark.internal.config._ import org.apache.spark.rpc.RpcEndpointAddress import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend -class BasicExecutorFeatureStepSuite - extends SparkFunSuite with BeforeAndAfter with BeforeAndAfterEach { +class BasicExecutorFeatureStepSuite extends SparkFunSuite with BeforeAndAfter { - private val APP_ID = "app-id" private val DRIVER_HOSTNAME = "localhost" private val DRIVER_PORT = 7098 private val DRIVER_ADDRESS = RpcEndpointAddress( @@ -45,7 +43,6 @@ class BasicExecutorFeatureStepSuite private val RESOURCE_NAME_PREFIX = "base" private val EXECUTOR_IMAGE = "executor-image" private val LABELS = Map("label1key" -> "label1value") - private val ANNOTATIONS = Map("annotation1key" -> "annotation1value") private val TEST_IMAGE_PULL_SECRETS = Seq("my-1secret-1", "my-secret-2") private val TEST_IMAGE_PULL_SECRET_OBJECTS = TEST_IMAGE_PULL_SECRETS.map { secret => @@ -66,37 +63,35 @@ class BasicExecutorFeatureStepSuite private var baseConf: SparkConf = _ before { - MockitoAnnotations.initMocks(this) baseConf = new SparkConf() .set(KUBERNETES_DRIVER_POD_NAME, DRIVER_POD_NAME) .set(KUBERNETES_EXECUTOR_POD_NAME_PREFIX, RESOURCE_NAME_PREFIX) .set(CONTAINER_IMAGE, EXECUTOR_IMAGE) .set(KUBERNETES_DRIVER_SUBMIT_CHECK, true) - .set("spark.driver.host", DRIVER_HOSTNAME) + .set(DRIVER_HOST_ADDRESS, DRIVER_HOSTNAME) .set("spark.driver.port", DRIVER_PORT.toString) - .set(IMAGE_PULL_SECRETS, TEST_IMAGE_PULL_SECRETS.mkString(",")) + .set(IMAGE_PULL_SECRETS, TEST_IMAGE_PULL_SECRETS) .set("spark.kubernetes.resource.type", "java") } + private def newExecutorConf( + environment: Map[String, String] = Map.empty): KubernetesExecutorConf = { + KubernetesTestConf.createExecutorConf( + sparkConf = baseConf, + driverPod = Some(DRIVER_POD), + labels = LABELS, + environment = environment) + } + test("basic executor pod has reasonable defaults") { - val step = new BasicExecutorFeatureStep( - KubernetesConf( - baseConf, - KubernetesExecutorSpecificConf("1", Some(DRIVER_POD)), - RESOURCE_NAME_PREFIX, - APP_ID, - LABELS, - ANNOTATIONS, - Map.empty, - Map.empty, - Map.empty, - Nil, - hadoopConfSpec = None)) + val step = new BasicExecutorFeatureStep(newExecutorConf()) val executor = step.configurePod(SparkPod.initialPod()) // The executor pod name and default labels. assert(executor.pod.getMetadata.getName === s"$RESOURCE_NAME_PREFIX-exec-1") - assert(executor.pod.getMetadata.getLabels.asScala === LABELS) + LABELS.foreach { case (k, v) => + assert(executor.pod.getMetadata.getLabels.get(k) === v) + } assert(executor.pod.getSpec.getImagePullSecrets.asScala === TEST_IMAGE_PULL_SECRET_OBJECTS) // There is exactly 1 container with no volume mounts and default memory limits. @@ -116,43 +111,18 @@ class BasicExecutorFeatureStepSuite } test("executor pod hostnames get truncated to 63 characters") { - val conf = baseConf.clone() val longPodNamePrefix = "loremipsumdolorsitametvimatelitrefficiendisuscipianturvixlegeresple" - val step = new BasicExecutorFeatureStep( - KubernetesConf( - conf, - KubernetesExecutorSpecificConf("1", Some(DRIVER_POD)), - longPodNamePrefix, - APP_ID, - LABELS, - ANNOTATIONS, - Map.empty, - Map.empty, - Map.empty, - Nil, - hadoopConfSpec = None)) + baseConf.set(KUBERNETES_EXECUTOR_POD_NAME_PREFIX, longPodNamePrefix) + val step = new BasicExecutorFeatureStep(newExecutorConf()) assert(step.configurePod(SparkPod.initialPod()).pod.getSpec.getHostname.length === 63) } test("classpath and extra java options get translated into environment variables") { - val conf = baseConf.clone() - 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 step = new BasicExecutorFeatureStep( - KubernetesConf( - conf, - KubernetesExecutorSpecificConf("1", Some(DRIVER_POD)), - RESOURCE_NAME_PREFIX, - APP_ID, - LABELS, - ANNOTATIONS, - Map.empty, - Map.empty, - Map("qux" -> "quux"), - Nil, - hadoopConfSpec = None)) + baseConf.set(EXECUTOR_JAVA_OPTIONS, "foo=bar") + baseConf.set(EXECUTOR_CLASS_PATH, "bar=baz") + val kconf = newExecutorConf(environment = Map("qux" -> "quux")) + val step = new BasicExecutorFeatureStep(kconf) val executor = step.configurePod(SparkPod.initialPod()) checkEnv(executor, @@ -163,23 +133,10 @@ class BasicExecutorFeatureStepSuite } test("test executor pyspark memory") { - val conf = baseConf.clone() - conf.set("spark.kubernetes.resource.type", "python") - conf.set(org.apache.spark.internal.config.PYSPARK_EXECUTOR_MEMORY, 42L) - - val step = new BasicExecutorFeatureStep( - KubernetesConf( - conf, - KubernetesExecutorSpecificConf("1", Some(DRIVER_POD)), - RESOURCE_NAME_PREFIX, - APP_ID, - LABELS, - ANNOTATIONS, - Map.empty, - Map.empty, - Map.empty, - Nil, - hadoopConfSpec = None)) + baseConf.set("spark.kubernetes.resource.type", "python") + baseConf.set(PYSPARK_EXECUTOR_MEMORY, 42L) + + val step = new BasicExecutorFeatureStep(newExecutorConf()) val executor = step.configurePod(SparkPod.initialPod()) // This is checking that basic executor + executorMemory = 1408 + 42 = 1450 assert(executor.container.getResources.getRequests.get("memory").getAmount === "1450Mi") @@ -199,7 +156,7 @@ class BasicExecutorFeatureStepSuite ENV_DRIVER_URL -> DRIVER_ADDRESS.toString, ENV_EXECUTOR_CORES -> "1", ENV_EXECUTOR_MEMORY -> "1g", - ENV_APPLICATION_ID -> APP_ID, + ENV_APPLICATION_ID -> KubernetesTestConf.APP_ID, ENV_SPARK_CONF_DIR -> SPARK_CONF_DIR_INTERNAL, ENV_EXECUTOR_POD_IP -> null) ++ additionalEnvVars diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/DriverCommandFeatureStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/DriverCommandFeatureStepSuite.scala index 30672952aaf6..f74ac928028c 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/DriverCommandFeatureStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/DriverCommandFeatureStepSuite.scala @@ -27,8 +27,6 @@ import org.apache.spark.util.Utils class DriverCommandFeatureStepSuite extends SparkFunSuite { - private val MAIN_CLASS = "mainClass" - test("java resource") { val mainResource = "local:///main.jar" val spec = applyFeatureStep( @@ -37,7 +35,7 @@ class DriverCommandFeatureStepSuite extends SparkFunSuite { assert(spec.pod.container.getArgs.asScala === List( "driver", "--properties-file", SPARK_CONF_PATH, - "--class", MAIN_CLASS, + "--class", KubernetesTestConf.MAIN_CLASS, "spark-internal", "5", "7")) val jars = Utils.stringToSeq(spec.systemProperties("spark.jars")) @@ -55,7 +53,7 @@ class DriverCommandFeatureStepSuite extends SparkFunSuite { assert(spec.pod.container.getArgs.asScala === List( "driver", "--properties-file", SPARK_CONF_PATH, - "--class", MAIN_CLASS, + "--class", KubernetesTestConf.MAIN_CLASS, "/main.py")) val envs = spec.pod.container.getEnv.asScala .map { env => (env.getName, env.getValue) } @@ -86,7 +84,7 @@ class DriverCommandFeatureStepSuite extends SparkFunSuite { assert(spec.pod.container.getArgs.asScala === List( "driver", "--properties-file", SPARK_CONF_PATH, - "--class", MAIN_CLASS, + "--class", KubernetesTestConf.MAIN_CLASS, "/main.py", "5", "7", "9")) val envs = spec.pod.container.getEnv.asScala @@ -112,7 +110,7 @@ class DriverCommandFeatureStepSuite extends SparkFunSuite { assert(spec.pod.container.getArgs.asScala === List( "driver", "--properties-file", SPARK_CONF_PATH, - "--class", MAIN_CLASS, + "--class", KubernetesTestConf.MAIN_CLASS, "/main.R", "5", "7", "9")) } @@ -121,20 +119,11 @@ class DriverCommandFeatureStepSuite extends SparkFunSuite { conf: SparkConf = new SparkConf(false), appArgs: Array[String] = Array(), pyFiles: Seq[String] = Nil): KubernetesDriverSpec = { - val driverConf = new KubernetesDriverSpecificConf( - resource, MAIN_CLASS, "appName", appArgs, pyFiles = pyFiles) - val kubernetesConf = KubernetesConf( - conf, - driverConf, - "resource-prefix", - "appId", - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Nil, - hadoopConfSpec = None) + val kubernetesConf = KubernetesTestConf.createDriverConf( + sparkConf = conf, + mainAppResource = resource, + appArgs = appArgs, + pyFiles = pyFiles) val step = new DriverCommandFeatureStep(kubernetesConf) val pod = step.configurePod(SparkPod.initialPod()) val props = step.getAdditionalPodSystemProperties() diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/DriverKubernetesCredentialsFeatureStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/DriverKubernetesCredentialsFeatureStepSuite.scala index 36c6616a87b0..7d8e9296a6cb 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/DriverKubernetesCredentialsFeatureStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/DriverKubernetesCredentialsFeatureStepSuite.scala @@ -18,51 +18,25 @@ package org.apache.spark.deploy.k8s.features import java.io.File +import scala.collection.JavaConverters._ + import com.google.common.base.Charsets import com.google.common.io.{BaseEncoding, Files} -import io.fabric8.kubernetes.api.model.{ContainerBuilder, HasMetadata, PodBuilder, Secret} -import org.mockito.{Mock, MockitoAnnotations} -import org.scalatest.BeforeAndAfter -import scala.collection.JavaConverters._ +import io.fabric8.kubernetes.api.model.Secret import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesDriverSpecificConf, SparkPod} +import org.apache.spark.deploy.k8s.{KubernetesTestConf, SparkPod} import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.util.Utils -class DriverKubernetesCredentialsFeatureStepSuite extends SparkFunSuite with BeforeAndAfter { +class DriverKubernetesCredentialsFeatureStepSuite extends SparkFunSuite { - private val KUBERNETES_RESOURCE_NAME_PREFIX = "spark" - private val APP_ID = "k8s-app" - private var credentialsTempDirectory: File = _ + private val credentialsTempDirectory = Utils.createTempDir() private val BASE_DRIVER_POD = SparkPod.initialPod() - @Mock - private var driverSpecificConf: KubernetesDriverSpecificConf = _ - - before { - MockitoAnnotations.initMocks(this) - credentialsTempDirectory = Utils.createTempDir() - } - - after { - credentialsTempDirectory.delete() - } - test("Don't set any credentials") { - val kubernetesConf = KubernetesConf( - new SparkConf(false), - driverSpecificConf, - KUBERNETES_RESOURCE_NAME_PREFIX, - APP_ID, - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Nil, - hadoopConfSpec = None) + val kubernetesConf = KubernetesTestConf.createDriverConf() val kubernetesCredentialsStep = new DriverKubernetesCredentialsFeatureStep(kubernetesConf) assert(kubernetesCredentialsStep.configurePod(BASE_DRIVER_POD) === BASE_DRIVER_POD) assert(kubernetesCredentialsStep.getAdditionalPodSystemProperties().isEmpty) @@ -83,19 +57,7 @@ class DriverKubernetesCredentialsFeatureStepSuite extends SparkFunSuite with Bef .set( s"$KUBERNETES_AUTH_DRIVER_MOUNTED_CONF_PREFIX.$CA_CERT_FILE_CONF_SUFFIX", "/mnt/secrets/my-ca.pem") - val kubernetesConf = KubernetesConf( - submissionSparkConf, - driverSpecificConf, - KUBERNETES_RESOURCE_NAME_PREFIX, - APP_ID, - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Nil, - hadoopConfSpec = None) - + val kubernetesConf = KubernetesTestConf.createDriverConf(sparkConf = submissionSparkConf) val kubernetesCredentialsStep = new DriverKubernetesCredentialsFeatureStep(kubernetesConf) assert(kubernetesCredentialsStep.configurePod(BASE_DRIVER_POD) === BASE_DRIVER_POD) assert(kubernetesCredentialsStep.getAdditionalKubernetesResources().isEmpty) @@ -122,18 +84,7 @@ class DriverKubernetesCredentialsFeatureStepSuite extends SparkFunSuite with Bef .set( s"$KUBERNETES_AUTH_DRIVER_CONF_PREFIX.$CA_CERT_FILE_CONF_SUFFIX", caCertFile.getAbsolutePath) - val kubernetesConf = KubernetesConf( - submissionSparkConf, - driverSpecificConf, - KUBERNETES_RESOURCE_NAME_PREFIX, - APP_ID, - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Nil, - hadoopConfSpec = None) + val kubernetesConf = KubernetesTestConf.createDriverConf(sparkConf = submissionSparkConf) val kubernetesCredentialsStep = new DriverKubernetesCredentialsFeatureStep(kubernetesConf) val resolvedProperties = kubernetesCredentialsStep.getAdditionalPodSystemProperties() val expectedSparkConf = Map( @@ -153,7 +104,7 @@ class DriverKubernetesCredentialsFeatureStepSuite extends SparkFunSuite with Bef .head .asInstanceOf[Secret] assert(credentialsSecret.getMetadata.getName === - s"$KUBERNETES_RESOURCE_NAME_PREFIX-kubernetes-credentials") + s"${kubernetesConf.resourceNamePrefix}-kubernetes-credentials") val decodedSecretData = credentialsSecret.getData.asScala.map { data => (data._1, new String(BaseEncoding.base64().decode(data._2), Charsets.UTF_8)) } diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/DriverServiceFeatureStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/DriverServiceFeatureStepSuite.scala index 3c46667c3042..045278939dff 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/DriverServiceFeatureStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/DriverServiceFeatureStepSuite.scala @@ -16,24 +16,19 @@ */ package org.apache.spark.deploy.k8s.features -import io.fabric8.kubernetes.api.model.Service -import org.mockito.{Mock, MockitoAnnotations} -import org.mockito.Mockito.when -import org.scalatest.BeforeAndAfter import scala.collection.JavaConverters._ +import io.fabric8.kubernetes.api.model.Service + import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesDriverSpecificConf, SparkPod} +import org.apache.spark.deploy.k8s.{KubernetesTestConf, SparkPod} import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.deploy.k8s.submit.JavaMainAppResource -import org.apache.spark.util.Clock - -class DriverServiceFeatureStepSuite extends SparkFunSuite with BeforeAndAfter { +import org.apache.spark.internal.config._ +import org.apache.spark.util.ManualClock - private val SHORT_RESOURCE_NAME_PREFIX = - "a" * (DriverServiceFeatureStep.MAX_SERVICE_NAME_LENGTH - - DriverServiceFeatureStep.DRIVER_SVC_POSTFIX.length) +class DriverServiceFeatureStepSuite extends SparkFunSuite { private val LONG_RESOURCE_NAME_PREFIX = "a" * (DriverServiceFeatureStep.MAX_SERVICE_NAME_LENGTH - @@ -42,34 +37,14 @@ class DriverServiceFeatureStepSuite extends SparkFunSuite with BeforeAndAfter { "label1key" -> "label1value", "label2key" -> "label2value") - @Mock - private var clock: Clock = _ - - private var sparkConf: SparkConf = _ - - before { - MockitoAnnotations.initMocks(this) - sparkConf = new SparkConf(false) - } - test("Headless service has a port for the driver RPC and the block manager.") { - sparkConf = sparkConf + val sparkConf = new SparkConf(false) .set("spark.driver.port", "9000") - .set(org.apache.spark.internal.config.DRIVER_BLOCK_MANAGER_PORT, 8080) - val configurationStep = new DriverServiceFeatureStep( - KubernetesConf( - sparkConf, - KubernetesDriverSpecificConf( - JavaMainAppResource(None), "main", "app", Seq.empty), - SHORT_RESOURCE_NAME_PREFIX, - "app-id", - DRIVER_LABELS, - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Nil, - hadoopConfSpec = None)) + .set(DRIVER_BLOCK_MANAGER_PORT, 8080) + val kconf = KubernetesTestConf.createDriverConf( + sparkConf = sparkConf, + labels = DRIVER_LABELS) + val configurationStep = new DriverServiceFeatureStep(kconf) assert(configurationStep.configurePod(SparkPod.initialPod()) === SparkPod.initialPod()) assert(configurationStep.getAdditionalKubernetesResources().size === 1) assert(configurationStep.getAdditionalKubernetesResources().head.isInstanceOf[Service]) @@ -80,50 +55,28 @@ class DriverServiceFeatureStepSuite extends SparkFunSuite with BeforeAndAfter { verifyService( 9000, 8080, - s"$SHORT_RESOURCE_NAME_PREFIX${DriverServiceFeatureStep.DRIVER_SVC_POSTFIX}", + s"${kconf.resourceNamePrefix}${DriverServiceFeatureStep.DRIVER_SVC_POSTFIX}", driverService) } test("Hostname and ports are set according to the service name.") { - val configurationStep = new DriverServiceFeatureStep( - KubernetesConf( - sparkConf - .set("spark.driver.port", "9000") - .set(org.apache.spark.internal.config.DRIVER_BLOCK_MANAGER_PORT, 8080) - .set(KUBERNETES_NAMESPACE, "my-namespace"), - KubernetesDriverSpecificConf( - JavaMainAppResource(None), "main", "app", Seq.empty), - SHORT_RESOURCE_NAME_PREFIX, - "app-id", - DRIVER_LABELS, - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Nil, - hadoopConfSpec = None)) - val expectedServiceName = SHORT_RESOURCE_NAME_PREFIX + - DriverServiceFeatureStep.DRIVER_SVC_POSTFIX + val sparkConf = new SparkConf(false) + .set("spark.driver.port", "9000") + .set(DRIVER_BLOCK_MANAGER_PORT, 8080) + .set(KUBERNETES_NAMESPACE, "my-namespace") + val kconf = KubernetesTestConf.createDriverConf( + sparkConf = sparkConf, + labels = DRIVER_LABELS) + val configurationStep = new DriverServiceFeatureStep(kconf) + val expectedServiceName = kconf.resourceNamePrefix + DriverServiceFeatureStep.DRIVER_SVC_POSTFIX val expectedHostName = s"$expectedServiceName.my-namespace.svc" val additionalProps = configurationStep.getAdditionalPodSystemProperties() verifySparkConfHostNames(additionalProps, expectedHostName) } test("Ports should resolve to defaults in SparkConf and in the service.") { - val configurationStep = new DriverServiceFeatureStep( - KubernetesConf( - sparkConf, - KubernetesDriverSpecificConf( - JavaMainAppResource(None), "main", "app", Seq.empty), - SHORT_RESOURCE_NAME_PREFIX, - "app-id", - DRIVER_LABELS, - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Nil, - hadoopConfSpec = None)) + val kconf = KubernetesTestConf.createDriverConf(labels = DRIVER_LABELS) + val configurationStep = new DriverServiceFeatureStep(kconf) val resolvedService = configurationStep .getAdditionalKubernetesResources() .head @@ -131,30 +84,23 @@ class DriverServiceFeatureStepSuite extends SparkFunSuite with BeforeAndAfter { verifyService( DEFAULT_DRIVER_PORT, DEFAULT_BLOCKMANAGER_PORT, - s"$SHORT_RESOURCE_NAME_PREFIX${DriverServiceFeatureStep.DRIVER_SVC_POSTFIX}", + s"${kconf.resourceNamePrefix}${DriverServiceFeatureStep.DRIVER_SVC_POSTFIX}", resolvedService) val additionalProps = configurationStep.getAdditionalPodSystemProperties() assert(additionalProps("spark.driver.port") === DEFAULT_DRIVER_PORT.toString) - assert(additionalProps(org.apache.spark.internal.config.DRIVER_BLOCK_MANAGER_PORT.key) - === DEFAULT_BLOCKMANAGER_PORT.toString) + assert(additionalProps(DRIVER_BLOCK_MANAGER_PORT.key) === DEFAULT_BLOCKMANAGER_PORT.toString) } test("Long prefixes should switch to using a generated name.") { - when(clock.getTimeMillis()).thenReturn(10000) + val clock = new ManualClock() + clock.setTime(10000) + val sparkConf = new SparkConf(false) + .set(KUBERNETES_NAMESPACE, "my-namespace") val configurationStep = new DriverServiceFeatureStep( - KubernetesConf( - sparkConf.set(KUBERNETES_NAMESPACE, "my-namespace"), - KubernetesDriverSpecificConf( - JavaMainAppResource(None), "main", "app", Seq.empty), - LONG_RESOURCE_NAME_PREFIX, - "app-id", - DRIVER_LABELS, - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Nil, - hadoopConfSpec = None), + KubernetesTestConf.createDriverConf( + sparkConf = sparkConf, + resourceNamePrefix = Some(LONG_RESOURCE_NAME_PREFIX), + labels = DRIVER_LABELS), clock) val driverService = configurationStep .getAdditionalKubernetesResources() @@ -168,56 +114,27 @@ class DriverServiceFeatureStepSuite extends SparkFunSuite with BeforeAndAfter { } test("Disallow bind address and driver host to be set explicitly.") { - try { - new DriverServiceFeatureStep( - KubernetesConf( - sparkConf.set(org.apache.spark.internal.config.DRIVER_BIND_ADDRESS, "host"), - KubernetesDriverSpecificConf( - JavaMainAppResource(None), "main", "app", Seq.empty), - LONG_RESOURCE_NAME_PREFIX, - "app-id", - DRIVER_LABELS, - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Nil, - hadoopConfSpec = None), - clock) - fail("The driver bind address should not be allowed.") - } catch { - case e: Throwable => - assert(e.getMessage === - s"requirement failed: ${DriverServiceFeatureStep.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.") + val sparkConf = new SparkConf(false) + .set(DRIVER_BIND_ADDRESS, "host") + .set("spark.app.name", LONG_RESOURCE_NAME_PREFIX) + val e1 = intercept[IllegalArgumentException] { + new DriverServiceFeatureStep(KubernetesTestConf.createDriverConf(sparkConf = sparkConf)) } - sparkConf.remove(org.apache.spark.internal.config.DRIVER_BIND_ADDRESS) - sparkConf.set(org.apache.spark.internal.config.DRIVER_HOST_ADDRESS, "host") - try { - new DriverServiceFeatureStep( - KubernetesConf( - sparkConf, - KubernetesDriverSpecificConf( - JavaMainAppResource(None), "main", "app", Seq.empty), - LONG_RESOURCE_NAME_PREFIX, - "app-id", - DRIVER_LABELS, - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Nil, - hadoopConfSpec = None), - clock) - fail("The driver host address should not be allowed.") - } catch { - case e: Throwable => - assert(e.getMessage === - s"requirement failed: ${DriverServiceFeatureStep.DRIVER_HOST_KEY} is" + - " not supported in Kubernetes mode, as the driver's hostname will be managed via" + - " a Kubernetes service.") + assert(e1.getMessage === + s"requirement failed: ${DriverServiceFeatureStep.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.") + + sparkConf.remove(DRIVER_BIND_ADDRESS) + sparkConf.set(DRIVER_HOST_ADDRESS, "host") + + val e2 = intercept[IllegalArgumentException] { + new DriverServiceFeatureStep(KubernetesTestConf.createDriverConf(sparkConf = sparkConf)) } + assert(e2.getMessage === + s"requirement failed: ${DriverServiceFeatureStep.DRIVER_HOST_KEY} is" + + " not supported in Kubernetes mode, as the driver's hostname will be managed via" + + " a Kubernetes service.") } private def verifyService( @@ -227,7 +144,9 @@ class DriverServiceFeatureStepSuite extends SparkFunSuite with BeforeAndAfter { service: Service): Unit = { assert(service.getMetadata.getName === expectedServiceName) assert(service.getSpec.getClusterIP === "None") - assert(service.getSpec.getSelector.asScala === DRIVER_LABELS) + DRIVER_LABELS.foreach { case (k, v) => + assert(service.getSpec.getSelector.get(k) === v) + } assert(service.getSpec.getPorts.size() === 2) val driverServicePorts = service.getSpec.getPorts.asScala assert(driverServicePorts.head.getName === DRIVER_PORT_NAME) diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/EnvSecretsFeatureStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/EnvSecretsFeatureStepSuite.scala index 3d253079c3ce..045552611106 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/EnvSecretsFeatureStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/EnvSecretsFeatureStepSuite.scala @@ -16,12 +16,12 @@ */ package org.apache.spark.deploy.k8s.features -import io.fabric8.kubernetes.api.model.PodBuilder +import scala.collection.JavaConverters._ -import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.SparkFunSuite import org.apache.spark.deploy.k8s._ -class EnvSecretsFeatureStepSuite extends SparkFunSuite{ +class EnvSecretsFeatureStepSuite extends SparkFunSuite { private val KEY_REF_NAME_FOO = "foo" private val KEY_REF_NAME_BAR = "bar" private val KEY_REF_KEY_FOO = "key_foo" @@ -34,28 +34,14 @@ class EnvSecretsFeatureStepSuite extends SparkFunSuite{ val envVarsToKeys = Map( ENV_NAME_BAR -> s"${KEY_REF_NAME_BAR}:${KEY_REF_KEY_BAR}", ENV_NAME_FOO -> s"${KEY_REF_NAME_FOO}:${KEY_REF_KEY_FOO}") - val sparkConf = new SparkConf(false) - val kubernetesConf = KubernetesConf( - sparkConf, - KubernetesExecutorSpecificConf("1", Some(new PodBuilder().build())), - "resource-name-prefix", - "app-id", - Map.empty, - Map.empty, - Map.empty, - envVarsToKeys, - Map.empty, - Nil, - hadoopConfSpec = None) + val kubernetesConf = KubernetesTestConf.createDriverConf( + secretEnvNamesToKeyRefs = envVarsToKeys) val step = new EnvSecretsFeatureStep(kubernetesConf) - val driverContainerWithEnvSecrets = step.configurePod(baseDriverPod).container - - val expectedVars = - Seq(s"${ENV_NAME_BAR}", s"${ENV_NAME_FOO}") - - expectedVars.foreach { envName => - assert(KubernetesFeaturesTestUtils.containerHasEnvVar(driverContainerWithEnvSecrets, envName)) + val container = step.configurePod(baseDriverPod).container + val containerEnvKeys = container.getEnv.asScala.map { v => v.getName }.toSet + envVarsToKeys.keys.foreach { envName => + assert(containerEnvKeys.contains(envName)) } } } diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/LocalDirsFeatureStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/LocalDirsFeatureStepSuite.scala index 894d824999aa..8f34ce5c6b94 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/LocalDirsFeatureStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/LocalDirsFeatureStepSuite.scala @@ -17,45 +17,19 @@ package org.apache.spark.deploy.k8s.features import io.fabric8.kubernetes.api.model.{EnvVarBuilder, VolumeBuilder, VolumeMountBuilder} -import org.mockito.Mockito -import org.scalatest._ -import org.scalatest.BeforeAndAfter import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesDriverSpecificConf, KubernetesRoleSpecificConf, SparkPod} +import org.apache.spark.deploy.k8s.{KubernetesTestConf, SparkPod} import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.submit.JavaMainAppResource +import org.apache.spark.util.SparkConfWithEnv -class LocalDirsFeatureStepSuite extends SparkFunSuite with BeforeAndAfter { +class LocalDirsFeatureStepSuite extends SparkFunSuite { private val defaultLocalDir = "/var/data/default-local-dir" - private var sparkConf: SparkConf = _ - private var kubernetesConf: KubernetesConf[_ <: KubernetesRoleSpecificConf] = _ - - before { - val realSparkConf = new SparkConf(false) - sparkConf = Mockito.spy(realSparkConf) - kubernetesConf = KubernetesConf( - sparkConf, - KubernetesDriverSpecificConf( - JavaMainAppResource(None), - "app-name", - "main", - Seq.empty), - "resource", - "app-id", - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Nil, - hadoopConfSpec = None) - } test("Resolve to default local dir if neither env nor configuration are set") { - Mockito.doReturn(null).when(sparkConf).get("spark.local.dir") - Mockito.doReturn(null).when(sparkConf).getenv("SPARK_LOCAL_DIRS") - val stepUnderTest = new LocalDirsFeatureStep(kubernetesConf, defaultLocalDir) + val stepUnderTest = new LocalDirsFeatureStep(KubernetesTestConf.createDriverConf(), + defaultLocalDir) val configuredPod = stepUnderTest.configurePod(SparkPod.initialPod()) assert(configuredPod.pod.getSpec.getVolumes.size === 1) assert(configuredPod.pod.getSpec.getVolumes.get(0) === @@ -79,8 +53,9 @@ class LocalDirsFeatureStepSuite extends SparkFunSuite with BeforeAndAfter { } test("Use configured local dirs split on comma if provided.") { - Mockito.doReturn("/var/data/my-local-dir-1,/var/data/my-local-dir-2") - .when(sparkConf).getenv("SPARK_LOCAL_DIRS") + val sparkConf = new SparkConfWithEnv(Map( + "SPARK_LOCAL_DIRS" -> "/var/data/my-local-dir-1,/var/data/my-local-dir-2")) + val kubernetesConf = KubernetesTestConf.createDriverConf(sparkConf = sparkConf) val stepUnderTest = new LocalDirsFeatureStep(kubernetesConf, defaultLocalDir) val configuredPod = stepUnderTest.configurePod(SparkPod.initialPod()) assert(configuredPod.pod.getSpec.getVolumes.size === 2) @@ -116,9 +91,8 @@ class LocalDirsFeatureStepSuite extends SparkFunSuite with BeforeAndAfter { } test("Use tmpfs to back default local dir") { - Mockito.doReturn(null).when(sparkConf).get("spark.local.dir") - Mockito.doReturn(null).when(sparkConf).getenv("SPARK_LOCAL_DIRS") - Mockito.doReturn(true).when(sparkConf).get(KUBERNETES_LOCAL_DIRS_TMPFS) + val sparkConf = new SparkConf(false).set(KUBERNETES_LOCAL_DIRS_TMPFS, true) + val kubernetesConf = KubernetesTestConf.createDriverConf(sparkConf = sparkConf) val stepUnderTest = new LocalDirsFeatureStep(kubernetesConf, defaultLocalDir) val configuredPod = stepUnderTest.configurePod(SparkPod.initialPod()) assert(configuredPod.pod.getSpec.getVolumes.size === 1) diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/MountSecretsFeatureStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/MountSecretsFeatureStepSuite.scala index 1555f6a9c652..22f6d26c4d0d 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/MountSecretsFeatureStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/MountSecretsFeatureStepSuite.scala @@ -16,10 +16,8 @@ */ package org.apache.spark.deploy.k8s.features -import io.fabric8.kubernetes.api.model.PodBuilder - -import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesExecutorSpecificConf, SecretVolumeUtils, SparkPod} +import org.apache.spark.SparkFunSuite +import org.apache.spark.deploy.k8s.{KubernetesTestConf, SecretVolumeUtils, SparkPod} class MountSecretsFeatureStepSuite extends SparkFunSuite { @@ -32,19 +30,8 @@ class MountSecretsFeatureStepSuite extends SparkFunSuite { val secretNamesToMountPaths = Map( SECRET_FOO -> SECRET_MOUNT_PATH, SECRET_BAR -> SECRET_MOUNT_PATH) - val sparkConf = new SparkConf(false) - val kubernetesConf = KubernetesConf( - sparkConf, - KubernetesExecutorSpecificConf("1", Some(new PodBuilder().build())), - "resource-name-prefix", - "app-id", - Map.empty, - Map.empty, - secretNamesToMountPaths, - Map.empty, - Map.empty, - Nil, - hadoopConfSpec = None) + val kubernetesConf = KubernetesTestConf.createExecutorConf( + secretNamesToMountPaths = secretNamesToMountPaths) val step = new MountSecretsFeatureStep(kubernetesConf) val driverPodWithSecretsMounted = step.configurePod(baseDriverPod).pod diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/MountVolumesFeatureStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/MountVolumesFeatureStepSuite.scala index 2a957460ca8e..e25229aa4392 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/MountVolumesFeatureStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/MountVolumesFeatureStepSuite.scala @@ -18,27 +18,8 @@ package org.apache.spark.deploy.k8s.features import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.deploy.k8s._ -import org.apache.spark.deploy.k8s.submit.JavaMainAppResource class MountVolumesFeatureStepSuite extends SparkFunSuite { - private val sparkConf = new SparkConf(false) - private val emptyKubernetesConf = KubernetesConf( - sparkConf = sparkConf, - roleSpecificConf = KubernetesDriverSpecificConf( - JavaMainAppResource(None), - "app-name", - "main", - Seq.empty), - appResourceNamePrefix = "resource", - appId = "app-id", - roleLabels = Map.empty, - roleAnnotations = Map.empty, - roleSecretNamesToMountPaths = Map.empty, - roleSecretEnvNamesToKeyRefs = Map.empty, - roleEnvs = Map.empty, - roleVolumes = Nil, - hadoopConfSpec = None) - test("Mounts hostPath volumes") { val volumeConf = KubernetesVolumeSpec( "testVolume", @@ -46,7 +27,7 @@ class MountVolumesFeatureStepSuite extends SparkFunSuite { false, KubernetesHostPathVolumeConf("/hostPath/tmp") ) - val kubernetesConf = emptyKubernetesConf.copy(roleVolumes = volumeConf :: Nil) + val kubernetesConf = KubernetesTestConf.createDriverConf(volumes = Seq(volumeConf)) val step = new MountVolumesFeatureStep(kubernetesConf) val configuredPod = step.configurePod(SparkPod.initialPod()) @@ -65,7 +46,7 @@ class MountVolumesFeatureStepSuite extends SparkFunSuite { true, KubernetesPVCVolumeConf("pvcClaim") ) - val kubernetesConf = emptyKubernetesConf.copy(roleVolumes = volumeConf :: Nil) + val kubernetesConf = KubernetesTestConf.createDriverConf(volumes = Seq(volumeConf)) val step = new MountVolumesFeatureStep(kubernetesConf) val configuredPod = step.configurePod(SparkPod.initialPod()) @@ -86,7 +67,7 @@ class MountVolumesFeatureStepSuite extends SparkFunSuite { false, KubernetesEmptyDirVolumeConf(Some("Memory"), Some("6G")) ) - val kubernetesConf = emptyKubernetesConf.copy(roleVolumes = volumeConf :: Nil) + val kubernetesConf = KubernetesTestConf.createDriverConf(volumes = Seq(volumeConf)) val step = new MountVolumesFeatureStep(kubernetesConf) val configuredPod = step.configurePod(SparkPod.initialPod()) @@ -107,7 +88,7 @@ class MountVolumesFeatureStepSuite extends SparkFunSuite { false, KubernetesEmptyDirVolumeConf(None, None) ) - val kubernetesConf = emptyKubernetesConf.copy(roleVolumes = volumeConf :: Nil) + val kubernetesConf = KubernetesTestConf.createDriverConf(volumes = Seq(volumeConf)) val step = new MountVolumesFeatureStep(kubernetesConf) val configuredPod = step.configurePod(SparkPod.initialPod()) @@ -134,8 +115,8 @@ class MountVolumesFeatureStepSuite extends SparkFunSuite { true, KubernetesPVCVolumeConf("pvcClaim") ) - val volumesConf = hpVolumeConf :: pvcVolumeConf :: Nil - val kubernetesConf = emptyKubernetesConf.copy(roleVolumes = volumesConf) + val kubernetesConf = KubernetesTestConf.createDriverConf( + volumes = Seq(hpVolumeConf, pvcVolumeConf)) val step = new MountVolumesFeatureStep(kubernetesConf) val configuredPod = step.configurePod(SparkPod.initialPod()) diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/PodTemplateConfigMapStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/PodTemplateConfigMapStepSuite.scala index 370948c9502e..7295b82ca479 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/PodTemplateConfigMapStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/PodTemplateConfigMapStepSuite.scala @@ -20,40 +20,22 @@ import java.io.{File, PrintWriter} import java.nio.file.Files import io.fabric8.kubernetes.api.model.ConfigMap -import org.mockito.Mockito import org.scalatest.BeforeAndAfter import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.deploy.k8s._ -import org.apache.spark.deploy.k8s.submit.JavaMainAppResource class PodTemplateConfigMapStepSuite extends SparkFunSuite with BeforeAndAfter { - private var sparkConf: SparkConf = _ - private var kubernetesConf : KubernetesConf[_ <: KubernetesRoleSpecificConf] = _ + private var kubernetesConf : KubernetesConf = _ private var templateFile: File = _ before { - sparkConf = Mockito.mock(classOf[SparkConf]) - kubernetesConf = KubernetesConf( - sparkConf, - KubernetesDriverSpecificConf( - JavaMainAppResource(None), - "app-name", - "main", - Seq.empty), - "resource", - "app-id", - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Nil, - Option.empty) templateFile = Files.createTempFile("pod-template", "yml").toFile templateFile.deleteOnExit() - Mockito.doReturn(Option(templateFile.getAbsolutePath)).when(sparkConf) - .get(Config.KUBERNETES_EXECUTOR_PODTEMPLATE_FILE) + + val sparkConf = new SparkConf(false) + .set(Config.KUBERNETES_EXECUTOR_PODTEMPLATE_FILE, templateFile.getAbsolutePath) + kubernetesConf = KubernetesTestConf.createDriverConf(sparkConf = sparkConf) } test("Mounts executor template volume if config specified") { diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/ClientSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/ClientSuite.scala index 81e3822389f3..e9c05fef6f5d 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/ClientSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/ClientSuite.scala @@ -18,17 +18,16 @@ package org.apache.spark.deploy.k8s.submit import io.fabric8.kubernetes.api.model._ import io.fabric8.kubernetes.client.{KubernetesClient, Watch} -import io.fabric8.kubernetes.client.dsl.{MixedOperation, NamespaceListVisitFromServerGetDeleteRecreateWaitApplicable, PodResource} +import io.fabric8.kubernetes.client.dsl.PodResource import org.mockito.{ArgumentCaptor, Mock, MockitoAnnotations} import org.mockito.Mockito.{doReturn, verify, when} import org.scalatest.BeforeAndAfter import org.scalatest.mockito.MockitoSugar._ -import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesDriverSpec, KubernetesDriverSpecificConf, SparkPod} +import org.apache.spark.SparkFunSuite +import org.apache.spark.deploy.k8s._ import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.deploy.k8s.Fabric8Aliases._ -import org.apache.spark.deploy.k8s.submit.JavaMainAppResource class ClientSuite extends SparkFunSuite with BeforeAndAfter { @@ -38,10 +37,6 @@ class ClientSuite extends SparkFunSuite with BeforeAndAfter { private val KUBERNETES_RESOURCE_PREFIX = "resource-example" private val POD_NAME = "driver" private val CONTAINER_NAME = "container" - private val APP_ID = "app-id" - private val APP_NAME = "app" - private val MAIN_CLASS = "main" - private val APP_ARGS = Seq("arg1", "arg2") private val RESOLVED_JAVA_OPTIONS = Map( "conf1key" -> "conf1value", "conf2key" -> "conf2value") @@ -123,28 +118,15 @@ class ClientSuite extends SparkFunSuite with BeforeAndAfter { @Mock private var resourceList: RESOURCE_LIST = _ - private var kubernetesConf: KubernetesConf[KubernetesDriverSpecificConf] = _ - - private var sparkConf: SparkConf = _ + private var kconf: KubernetesDriverConf = _ private var createdPodArgumentCaptor: ArgumentCaptor[Pod] = _ private var createdResourcesArgumentCaptor: ArgumentCaptor[HasMetadata] = _ before { MockitoAnnotations.initMocks(this) - sparkConf = new SparkConf(false) - kubernetesConf = KubernetesConf[KubernetesDriverSpecificConf]( - sparkConf, - KubernetesDriverSpecificConf(JavaMainAppResource(None), MAIN_CLASS, APP_NAME, APP_ARGS), - KUBERNETES_RESOURCE_PREFIX, - APP_ID, - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Nil, - hadoopConfSpec = None) - when(driverBuilder.buildFromFeatures(kubernetesConf)).thenReturn(BUILT_KUBERNETES_SPEC) + kconf = KubernetesTestConf.createDriverConf( + resourceNamePrefix = Some(KUBERNETES_RESOURCE_PREFIX)) + when(driverBuilder.buildFromFeatures(kconf)).thenReturn(BUILT_KUBERNETES_SPEC) when(kubernetesClient.pods()).thenReturn(podOperations) when(podOperations.withName(POD_NAME)).thenReturn(namedPods) @@ -159,26 +141,22 @@ class ClientSuite extends SparkFunSuite with BeforeAndAfter { test("The client should configure the pod using the builder.") { val submissionClient = new Client( + kconf, driverBuilder, - kubernetesConf, kubernetesClient, false, - "spark", - loggingPodStatusWatcher, - KUBERNETES_RESOURCE_PREFIX) + loggingPodStatusWatcher) submissionClient.run() verify(podOperations).create(FULL_EXPECTED_POD) } test("The client should create Kubernetes resources") { val submissionClient = new Client( + kconf, driverBuilder, - kubernetesConf, kubernetesClient, false, - "spark", - loggingPodStatusWatcher, - KUBERNETES_RESOURCE_PREFIX) + loggingPodStatusWatcher) submissionClient.run() val otherCreatedResources = createdResourcesArgumentCaptor.getAllValues assert(otherCreatedResources.size === 2) @@ -198,13 +176,11 @@ class ClientSuite extends SparkFunSuite with BeforeAndAfter { test("Waiting for app completion should stall on the watcher") { val submissionClient = new Client( + kconf, driverBuilder, - kubernetesConf, kubernetesClient, true, - "spark", - loggingPodStatusWatcher, - KUBERNETES_RESOURCE_PREFIX) + loggingPodStatusWatcher) submissionClient.run() verify(loggingPodStatusWatcher).awaitCompletion() } diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/KubernetesDriverBuilderSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/KubernetesDriverBuilderSuite.scala index fe900fda6e54..ea207dc5b4db 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/KubernetesDriverBuilderSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/KubernetesDriverBuilderSuite.scala @@ -83,48 +83,21 @@ class KubernetesDriverBuilderSuite extends SparkFunSuite { _ => templateVolumeStep) test("Apply fundamental steps all the time.") { - val conf = KubernetesConf( - new SparkConf(false), - KubernetesDriverSpecificConf( - JavaMainAppResource(Some("example.jar")), - "test-app", - "main", - Seq.empty), - "prefix", - "appId", - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Nil, - hadoopConfSpec = None) + val conf = KubernetesTestConf.createDriverConf() validateStepTypesApplied( builderUnderTest.buildFromFeatures(conf), BASIC_STEP_TYPE, CREDENTIALS_STEP_TYPE, SERVICE_STEP_TYPE, LOCAL_DIRS_STEP_TYPE, - DRIVER_CMD_STEP_TYPE) + DRIVER_CMD_STEP_TYPE, + HADOOP_GLOBAL_STEP_TYPE) } test("Apply secrets step if secrets are present.") { - val conf = KubernetesConf( - new SparkConf(false), - KubernetesDriverSpecificConf( - JavaMainAppResource(None), - "test-app", - "main", - Seq.empty), - "prefix", - "appId", - Map.empty, - Map.empty, - Map("secret" -> "secretMountPath"), - Map("EnvName" -> "SecretName:secretKey"), - Map.empty, - Nil, - hadoopConfSpec = None) + val conf = KubernetesTestConf.createDriverConf( + secretEnvNamesToKeyRefs = Map("EnvName" -> "SecretName:secretKey"), + secretNamesToMountPaths = Map("secret" -> "secretMountPath")) validateStepTypesApplied( builderUnderTest.buildFromFeatures(conf), BASIC_STEP_TYPE, @@ -133,7 +106,8 @@ class KubernetesDriverBuilderSuite extends SparkFunSuite { LOCAL_DIRS_STEP_TYPE, SECRETS_STEP_TYPE, ENV_SECRETS_STEP_TYPE, - DRIVER_CMD_STEP_TYPE) + DRIVER_CMD_STEP_TYPE, + HADOOP_GLOBAL_STEP_TYPE) } test("Apply volumes step if mounts are present.") { @@ -142,22 +116,8 @@ class KubernetesDriverBuilderSuite extends SparkFunSuite { "/tmp", false, KubernetesHostPathVolumeConf("/path")) - val conf = KubernetesConf( - new SparkConf(false), - KubernetesDriverSpecificConf( - JavaMainAppResource(None), - "test-app", - "main", - Seq.empty), - "prefix", - "appId", - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Map.empty, - volumeSpec :: Nil, - hadoopConfSpec = None) + val conf = KubernetesTestConf.createDriverConf( + volumes = Seq(volumeSpec)) validateStepTypesApplied( builderUnderTest.buildFromFeatures(conf), BASIC_STEP_TYPE, @@ -165,89 +125,14 @@ class KubernetesDriverBuilderSuite extends SparkFunSuite { SERVICE_STEP_TYPE, LOCAL_DIRS_STEP_TYPE, MOUNT_VOLUMES_STEP_TYPE, - DRIVER_CMD_STEP_TYPE) - } - - test("Apply template volume step if executor template is present.") { - val sparkConf = spy(new SparkConf(false)) - doReturn(Option("filename")).when(sparkConf) - .get(KUBERNETES_EXECUTOR_PODTEMPLATE_FILE) - val conf = KubernetesConf( - sparkConf, - KubernetesDriverSpecificConf( - JavaMainAppResource(Some("example.jar")), - "test-app", - "main", - Seq.empty), - "prefix", - "appId", - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Nil, - Option.empty) - validateStepTypesApplied( - builderUnderTest.buildFromFeatures(conf), - BASIC_STEP_TYPE, - CREDENTIALS_STEP_TYPE, - SERVICE_STEP_TYPE, - LOCAL_DIRS_STEP_TYPE, - DRIVER_CMD_STEP_TYPE, - TEMPLATE_VOLUME_STEP_TYPE) - } - - test("Apply HadoopSteps if HADOOP_CONF_DIR is defined.") { - val conf = KubernetesConf( - new SparkConf(false), - KubernetesDriverSpecificConf( - JavaMainAppResource(None), - "test-app", - "main", - Seq.empty), - "prefix", - "appId", - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Nil, - hadoopConfSpec = Some( - HadoopConfSpec( - Some("/var/hadoop-conf"), - None))) - validateStepTypesApplied( - builderUnderTest.buildFromFeatures(conf), - BASIC_STEP_TYPE, - CREDENTIALS_STEP_TYPE, - SERVICE_STEP_TYPE, - LOCAL_DIRS_STEP_TYPE, DRIVER_CMD_STEP_TYPE, HADOOP_GLOBAL_STEP_TYPE) } - test("Apply HadoopSteps if HADOOP_CONF ConfigMap is defined.") { - val conf = KubernetesConf( - new SparkConf(false), - KubernetesDriverSpecificConf( - JavaMainAppResource(None), - "test-app", - "main", - Seq.empty), - "prefix", - "appId", - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Nil, - hadoopConfSpec = Some( - HadoopConfSpec( - None, - Some("pre-defined-configMapName")))) + test("Apply template volume step if executor template is present.") { + val sparkConf = new SparkConf(false) + .set(KUBERNETES_EXECUTOR_PODTEMPLATE_FILE, "filename") + val conf = KubernetesTestConf.createDriverConf(sparkConf = sparkConf) validateStepTypesApplied( builderUnderTest.buildFromFeatures(conf), BASIC_STEP_TYPE, @@ -255,12 +140,16 @@ class KubernetesDriverBuilderSuite extends SparkFunSuite { SERVICE_STEP_TYPE, LOCAL_DIRS_STEP_TYPE, DRIVER_CMD_STEP_TYPE, - HADOOP_GLOBAL_STEP_TYPE) + HADOOP_GLOBAL_STEP_TYPE, + TEMPLATE_VOLUME_STEP_TYPE) } private def validateStepTypesApplied(resolvedSpec: KubernetesDriverSpec, stepTypes: String*) : Unit = { - assert(resolvedSpec.systemProperties.size === stepTypes.size) + val addedProperties = resolvedSpec.systemProperties + .filter { case (k, _) => !k.startsWith("spark.") } + .toMap + assert(addedProperties.keys.toSet === stepTypes.toSet) stepTypes.foreach { stepType => assert(resolvedSpec.pod.pod.getMetadata.getLabels.get(stepType) === stepType) assert(resolvedSpec.driverKubernetesResources.containsSlice( @@ -280,22 +169,7 @@ class KubernetesDriverBuilderSuite extends SparkFunSuite { val sparkConf = new SparkConf(false) .set(CONTAINER_IMAGE, "spark-driver:latest") .set(KUBERNETES_DRIVER_PODTEMPLATE_FILE, "template-file.yaml") - val kubernetesConf = new KubernetesConf( - sparkConf, - KubernetesDriverSpecificConf( - JavaMainAppResource(Some("example.jar")), - "test-app", - "main", - Seq.empty), - "prefix", - "appId", - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Nil, - Option.empty) + val kubernetesConf = KubernetesTestConf.createDriverConf(sparkConf = sparkConf) val driverSpec = KubernetesDriverBuilder .apply(kubernetesClient, sparkConf) .buildFromFeatures(kubernetesConf) @@ -312,22 +186,7 @@ class KubernetesDriverBuilderSuite extends SparkFunSuite { val sparkConf = new SparkConf(false) .set(CONTAINER_IMAGE, "spark-driver:latest") .set(KUBERNETES_DRIVER_PODTEMPLATE_FILE, "template-file.yaml") - val kubernetesConf = new KubernetesConf( - sparkConf, - KubernetesDriverSpecificConf( - JavaMainAppResource(Some("example.jar")), - "test-app", - "main", - Seq.empty), - "prefix", - "appId", - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Nil, - Option.empty) + val kubernetesConf = KubernetesTestConf.createDriverConf(sparkConf = sparkConf) val exception = intercept[SparkException] { KubernetesDriverBuilder .apply(kubernetesClient, sparkConf) diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala index b336774838bc..ddf9f67a0727 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala @@ -27,7 +27,7 @@ import org.mockito.stubbing.Answer import org.scalatest.BeforeAndAfter import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesExecutorSpecificConf, SparkPod} +import org.apache.spark.deploy.k8s.{KubernetesExecutorConf, KubernetesTestConf, SparkPod} import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.deploy.k8s.Fabric8Aliases._ @@ -79,7 +79,7 @@ class ExecutorPodsAllocatorSuite extends SparkFunSuite with BeforeAndAfter { when(kubernetesClient.pods()).thenReturn(podOperations) when(podOperations.withName(driverPodName)).thenReturn(driverPodOperations) when(driverPodOperations.get).thenReturn(driverPod) - when(executorBuilder.buildFromFeatures(kubernetesConfWithCorrectFields())) + when(executorBuilder.buildFromFeatures(any(classOf[KubernetesExecutorConf]))) .thenAnswer(executorPodAnswer()) snapshotsStore = new DeterministicExecutorPodsSnapshotsStore() waitForExecutorPodsClock = new ManualClock(0L) @@ -147,44 +147,9 @@ class ExecutorPodsAllocatorSuite extends SparkFunSuite with BeforeAndAfter { private def executorPodAnswer(): Answer[SparkPod] = { new Answer[SparkPod] { override def answer(invocation: InvocationOnMock): SparkPod = { - val k8sConf = invocation.getArgumentAt( - 0, classOf[KubernetesConf[KubernetesExecutorSpecificConf]]) - executorPodWithId(k8sConf.roleSpecificConf.executorId.toInt) + val k8sConf = invocation.getArgumentAt(0, classOf[KubernetesExecutorConf]) + executorPodWithId(k8sConf.executorId.toInt) } } } - - private def kubernetesConfWithCorrectFields(): KubernetesConf[KubernetesExecutorSpecificConf] = - Matchers.argThat(new ArgumentMatcher[KubernetesConf[KubernetesExecutorSpecificConf]] { - override def matches(argument: scala.Any): Boolean = { - if (!argument.isInstanceOf[KubernetesConf[KubernetesExecutorSpecificConf]]) { - false - } else { - val k8sConf = argument.asInstanceOf[KubernetesConf[KubernetesExecutorSpecificConf]] - val executorSpecificConf = k8sConf.roleSpecificConf - // TODO: HADOOP_CONF_DIR - val expectedK8sConf = KubernetesConf.createExecutorConf( - conf, - executorSpecificConf.executorId, - TEST_SPARK_APP_ID, - Some(driverPod)) - - // Set prefixes to a common string since KUBERNETES_EXECUTOR_POD_NAME_PREFIX - // has not be set for the tests and thus KubernetesConf will use a random - // string for the prefix, based on the app name, and this comparison here will fail. - val k8sConfCopy = k8sConf - .copy(appResourceNamePrefix = "") - .copy(sparkConf = conf) - val expectedK8sConfCopy = expectedK8sConf - .copy(appResourceNamePrefix = "") - .copy(sparkConf = conf) - - k8sConf.sparkConf.getAll.toMap == conf.getAll.toMap && - // Since KubernetesConf.createExecutorConf clones the SparkConf object, force - // deep equality comparison for the SparkConf object and use object equality - // comparison on all other fields. - k8sConfCopy == expectedK8sConfCopy - } - } - }) } diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesExecutorBuilderSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesExecutorBuilderSuite.scala index 1fea08c37ccc..54ab6933429c 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesExecutorBuilderSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesExecutorBuilderSuite.scala @@ -16,6 +16,8 @@ */ package org.apache.spark.scheduler.cluster.k8s +import scala.collection.JavaConverters._ + import io.fabric8.kubernetes.api.model.{Config => _, _} import io.fabric8.kubernetes.client.KubernetesClient import org.mockito.Mockito.{mock, never, verify} @@ -25,6 +27,7 @@ import org.apache.spark.deploy.k8s._ import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.deploy.k8s.features._ import org.apache.spark.deploy.k8s.submit.PodBuilderSuiteUtils +import org.apache.spark.util.SparkConfWithEnv class KubernetesExecutorBuilderSuite extends SparkFunSuite { private val BASIC_STEP_TYPE = "basic" @@ -64,37 +67,15 @@ class KubernetesExecutorBuilderSuite extends SparkFunSuite { _ => hadoopSparkUser) test("Basic steps are consistently applied.") { - val conf = KubernetesConf( - new SparkConf(false), - KubernetesExecutorSpecificConf( - "executor-id", Some(new PodBuilder().build())), - "prefix", - "appId", - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Nil, - None) + val conf = KubernetesTestConf.createExecutorConf() validateStepTypesApplied( builderUnderTest.buildFromFeatures(conf), BASIC_STEP_TYPE, LOCAL_DIRS_STEP_TYPE) } test("Apply secrets step if secrets are present.") { - val conf = KubernetesConf( - new SparkConf(false), - KubernetesExecutorSpecificConf( - "executor-id", Some(new PodBuilder().build())), - "prefix", - "appId", - Map.empty, - Map.empty, - Map("secret" -> "secretMountPath"), - Map("secret-name" -> "secret-key"), - Map.empty, - Nil, - None) + val conf = KubernetesTestConf.createExecutorConf( + secretEnvNamesToKeyRefs = Map("secret-name" -> "secret-key"), + secretNamesToMountPaths = Map("secret" -> "secretMountPath")) validateStepTypesApplied( builderUnderTest.buildFromFeatures(conf), BASIC_STEP_TYPE, @@ -109,19 +90,8 @@ class KubernetesExecutorBuilderSuite extends SparkFunSuite { "/tmp", false, KubernetesHostPathVolumeConf("/checkpoint")) - val conf = KubernetesConf( - new SparkConf(false), - KubernetesExecutorSpecificConf( - "executor-id", Some(new PodBuilder().build())), - "prefix", - "appId", - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Map.empty, - volumeSpec :: Nil, - None) + val conf = KubernetesTestConf.createExecutorConf( + volumes = Seq(volumeSpec)) validateStepTypesApplied( builderUnderTest.buildFromFeatures(conf), BASIC_STEP_TYPE, @@ -131,25 +101,10 @@ class KubernetesExecutorBuilderSuite extends SparkFunSuite { test("Apply basicHadoop step if HADOOP_CONF_DIR is defined") { // HADOOP_DELEGATION_TOKEN - val HADOOP_CREDS_PREFIX = "spark.security.credentials." - val HADOOPFS_PROVIDER = s"$HADOOP_CREDS_PREFIX.hadoopfs.enabled" - val conf = KubernetesConf( - new SparkConf(false) + val conf = KubernetesTestConf.createExecutorConf( + sparkConf = new SparkConfWithEnv(Map("HADOOP_CONF_DIR" -> "/var/hadoop-conf")) .set(HADOOP_CONFIG_MAP_NAME, "hadoop-conf-map-name") - .set(KRB5_CONFIG_MAP_NAME, "krb5-conf-map-name") - .set(KERBEROS_SPARK_USER_NAME, "spark-user") - .set(HADOOPFS_PROVIDER, "true"), - KubernetesExecutorSpecificConf( - "executor-id", Some(new PodBuilder().build())), - "prefix", - "appId", - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Nil, - Some(HadoopConfSpec(Some("/var/hadoop-conf"), None))) + .set(KRB5_CONFIG_MAP_NAME, "krb5-conf-map-name")) validateStepTypesApplied( builderUnderTest.buildFromFeatures(conf), BASIC_STEP_TYPE, @@ -159,24 +114,13 @@ class KubernetesExecutorBuilderSuite extends SparkFunSuite { } test("Apply kerberos step if DT secrets created") { - val conf = KubernetesConf( - new SparkConf(false) + val conf = KubernetesTestConf.createExecutorConf( + sparkConf = new SparkConf(false) .set(HADOOP_CONFIG_MAP_NAME, "hadoop-conf-map-name") .set(KRB5_CONFIG_MAP_NAME, "krb5-conf-map-name") .set(KERBEROS_SPARK_USER_NAME, "spark-user") .set(KERBEROS_DT_SECRET_NAME, "dt-secret") - .set(KERBEROS_DT_SECRET_KEY, "dt-key"), - KubernetesExecutorSpecificConf( - "executor-id", Some(new PodBuilder().build())), - "prefix", - "appId", - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Nil, - Some(HadoopConfSpec(None, Some("pre-defined-onfigMapName")))) + .set(KERBEROS_DT_SECRET_KEY, "dt-key" )) validateStepTypesApplied( builderUnderTest.buildFromFeatures(conf), BASIC_STEP_TYPE, @@ -186,10 +130,7 @@ class KubernetesExecutorBuilderSuite extends SparkFunSuite { } private def validateStepTypesApplied(resolvedPod: SparkPod, stepTypes: String*): Unit = { - assert(resolvedPod.pod.getMetadata.getLabels.size === stepTypes.size) - stepTypes.foreach { stepType => - assert(resolvedPod.pod.getMetadata.getLabels.get(stepType) === stepType) - } + assert(resolvedPod.pod.getMetadata.getLabels.asScala.keys.toSet === stepTypes.toSet) } test("Starts with empty executor pod if template is not specified") { @@ -204,25 +145,14 @@ class KubernetesExecutorBuilderSuite extends SparkFunSuite { .set("spark.driver.host", "https://driver.host.com") .set(Config.CONTAINER_IMAGE, "spark-executor:latest") .set(Config.KUBERNETES_EXECUTOR_PODTEMPLATE_FILE, "template-file.yaml") - val kubernetesConf = KubernetesConf( - sparkConf, - KubernetesExecutorSpecificConf( - "executor-id", Some(new PodBuilder() - .withNewMetadata() + val kubernetesConf = KubernetesTestConf.createExecutorConf( + sparkConf = sparkConf, + driverPod = Some(new PodBuilder() + .withNewMetadata() .withName("driver") .endMetadata() - .build())), - "prefix", - "appId", - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Nil, - Option.empty) - val sparkPod = KubernetesExecutorBuilder - .apply(kubernetesClient, sparkConf) + .build())) + val sparkPod = KubernetesExecutorBuilder(kubernetesClient, sparkConf) .buildFromFeatures(kubernetesConf) PodBuilderSuiteUtils.verifyPodWithSupportedFeatures(sparkPod) } From 396fcc859b0904cbcdb4405886ed048249bae268 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Mon, 26 Nov 2018 14:59:49 -0800 Subject: [PATCH 2/4] Fix code after merge. --- .../spark/deploy/k8s/KubernetesTestConf.scala | 4 +++ .../k8s/KubernetesVolumeUtilsSuite.scala | 2 +- .../MountVolumesFeatureStepSuite.scala | 25 ++++++++++--------- .../submit/KubernetesDriverBuilderSuite.scala | 20 ++------------- 4 files changed, 20 insertions(+), 31 deletions(-) diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/KubernetesTestConf.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/KubernetesTestConf.scala index cfa9f150c168..1d77a6d18152 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/KubernetesTestConf.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/KubernetesTestConf.scala @@ -123,6 +123,10 @@ object KubernetesTestConf { } conf.set(key(vtype, spec.volumeName, KUBERNETES_VOLUMES_MOUNT_PATH_KEY), spec.mountPath) + if (spec.mountSubPath.nonEmpty) { + conf.set(key(vtype, spec.volumeName, KUBERNETES_VOLUMES_MOUNT_SUBPATH_KEY), + spec.mountSubPath) + } conf.set(key(vtype, spec.volumeName, KUBERNETES_VOLUMES_MOUNT_READONLY_KEY), spec.mountReadOnly.toString) configs.foreach { case (k, v) => diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/KubernetesVolumeUtilsSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/KubernetesVolumeUtilsSuite.scala index 1e050fe92d5b..c0790898e097 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/KubernetesVolumeUtilsSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/KubernetesVolumeUtilsSuite.scala @@ -39,7 +39,7 @@ class KubernetesVolumeUtilsSuite extends SparkFunSuite { sparkConf.set("test.emptyDir.volumeName.mount.readOnly", "true") sparkConf.set("test.emptyDir.volumeName.mount.subPath", "subPath") - val volumeSpec = KubernetesVolumeUtils.parseVolumesWithPrefix(sparkConf, "test.").head.get + val volumeSpec = KubernetesVolumeUtils.parseVolumesWithPrefix(sparkConf, "test.").head assert(volumeSpec.volumeName === "volumeName") assert(volumeSpec.mountPath === "/path") assert(volumeSpec.mountSubPath === "subPath") diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/MountVolumesFeatureStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/MountVolumesFeatureStepSuite.scala index a2440d20714b..e6f1dd640e3e 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/MountVolumesFeatureStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/MountVolumesFeatureStepSuite.scala @@ -16,6 +16,8 @@ */ package org.apache.spark.deploy.k8s.features +import scala.collection.JavaConverters._ + import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.deploy.k8s._ @@ -138,7 +140,7 @@ class MountVolumesFeatureStepSuite extends SparkFunSuite { false, KubernetesEmptyDirVolumeConf(None, None) ) - val kubernetesConf = emptyKubernetesConf.copy(roleVolumes = volumeConf :: Nil) + val kubernetesConf = KubernetesTestConf.createDriverConf(volumes = Seq(volumeConf)) val step = new MountVolumesFeatureStep(kubernetesConf) val configuredPod = step.configurePod(SparkPod.initialPod()) @@ -157,7 +159,7 @@ class MountVolumesFeatureStepSuite extends SparkFunSuite { true, KubernetesPVCVolumeConf("pvcClaim") ) - val kubernetesConf = emptyKubernetesConf.copy(roleVolumes = volumeConf :: Nil) + val kubernetesConf = KubernetesTestConf.createDriverConf(volumes = Seq(volumeConf)) val step = new MountVolumesFeatureStep(kubernetesConf) val configuredPod = step.configurePod(SparkPod.initialPod()) @@ -187,19 +189,18 @@ class MountVolumesFeatureStepSuite extends SparkFunSuite { true, KubernetesEmptyDirVolumeConf(None, None) ) - val kubernetesConf = emptyKubernetesConf.copy( - roleVolumes = emptyDirSpec :: pvcSpec :: Nil) + val kubernetesConf = KubernetesTestConf.createDriverConf(volumes = Seq(emptyDirSpec, pvcSpec)) val step = new MountVolumesFeatureStep(kubernetesConf) val configuredPod = step.configurePod(SparkPod.initialPod()) assert(configuredPod.pod.getSpec.getVolumes.size() === 2) - val mounts = configuredPod.container.getVolumeMounts - assert(mounts.size() === 2) - assert(mounts.get(0).getName === "testEmptyDir") - assert(mounts.get(0).getMountPath === "/tmp/foo") - assert(mounts.get(0).getSubPath === "foo") - assert(mounts.get(1).getName === "testPVC") - assert(mounts.get(1).getMountPath === "/tmp/bar") - assert(mounts.get(1).getSubPath === "bar") + val mounts = configuredPod.container.getVolumeMounts.asScala.sortBy(_.getName()) + assert(mounts.size === 2) + assert(mounts(0).getName === "testEmptyDir") + assert(mounts(0).getMountPath === "/tmp/foo") + assert(mounts(0).getSubPath === "foo") + assert(mounts(1).getName === "testPVC") + assert(mounts(1).getMountPath === "/tmp/bar") + assert(mounts(1).getSubPath === "bar") } } diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/KubernetesDriverBuilderSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/KubernetesDriverBuilderSuite.scala index a8a6902b3ed4..0f91c27276d3 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/KubernetesDriverBuilderSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/KubernetesDriverBuilderSuite.scala @@ -117,22 +117,7 @@ class KubernetesDriverBuilderSuite extends SparkFunSuite { "", false, KubernetesHostPathVolumeConf("/path")) - val conf = KubernetesConf( - new SparkConf(false), - KubernetesDriverSpecificConf( - JavaMainAppResource(None), - "test-app", - "main", - Seq.empty), - "prefix", - "appId", - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Map.empty, - volumeSpec :: Nil, - hadoopConfSpec = None) + val conf = KubernetesTestConf.createDriverConf(volumes = Seq(volumeSpec)) validateStepTypesApplied( builderUnderTest.buildFromFeatures(conf), BASIC_STEP_TYPE, @@ -150,8 +135,7 @@ class KubernetesDriverBuilderSuite extends SparkFunSuite { "foo", false, KubernetesHostPathVolumeConf("/path")) - val conf = KubernetesTestConf.createDriverConf( - volumes = Seq(volumeSpec)) + val conf = KubernetesTestConf.createDriverConf(volumes = Seq(volumeSpec)) validateStepTypesApplied( builderUnderTest.buildFromFeatures(conf), BASIC_STEP_TYPE, From 2bed04b84be9fad94701f05a24cb670a2a653510 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Mon, 26 Nov 2018 15:04:06 -0800 Subject: [PATCH 3/4] Feedback. --- .../features/DriverCommandFeatureStep.scala | 20 +++++++++---------- .../HadoopConfExecutorFeatureStep.scala | 2 +- .../HadoopSparkUserExecutorFeatureStep.scala | 2 +- .../KerberosConfDriverFeatureStep.scala | 16 +++++++-------- 4 files changed, 19 insertions(+), 21 deletions(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/DriverCommandFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/DriverCommandFeatureStep.scala index 102c3ebd5af7..76b4ec98d494 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/DriverCommandFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/DriverCommandFeatureStep.scala @@ -32,11 +32,11 @@ import org.apache.spark.util.Utils * Creates the driver command for running the user app, and propagates needed configuration so * executors can also find the app code. */ -private[spark] class DriverCommandFeatureStep(driverConf: KubernetesDriverConf) +private[spark] class DriverCommandFeatureStep(conf: KubernetesDriverConf) extends KubernetesFeatureConfigStep { override def configurePod(pod: SparkPod): SparkPod = { - driverConf.mainAppResource match { + conf.mainAppResource match { case JavaMainAppResource(_) => configureForJava(pod) @@ -49,7 +49,7 @@ private[spark] class DriverCommandFeatureStep(driverConf: KubernetesDriverConf) } override def getAdditionalPodSystemProperties(): Map[String, String] = { - driverConf.mainAppResource match { + conf.mainAppResource match { case JavaMainAppResource(res) => res.map(additionalJavaProperties).getOrElse(Map.empty) @@ -69,10 +69,10 @@ private[spark] class DriverCommandFeatureStep(driverConf: KubernetesDriverConf) } private def configureForPython(pod: SparkPod, res: String): SparkPod = { - val maybePythonFiles = if (driverConf.pyFiles.nonEmpty) { + val maybePythonFiles = if (conf.pyFiles.nonEmpty) { // Delineation by ":" is to append the PySpark Files to the PYTHONPATH // of the respective PySpark pod - val resolved = KubernetesUtils.resolveFileUrisAndPath(driverConf.pyFiles) + val resolved = KubernetesUtils.resolveFileUrisAndPath(conf.pyFiles) Some(new EnvVarBuilder() .withName(ENV_PYSPARK_FILES) .withValue(resolved.mkString(":")) @@ -83,7 +83,7 @@ private[spark] class DriverCommandFeatureStep(driverConf: KubernetesDriverConf) val pythonEnvs = Seq(new EnvVarBuilder() .withName(ENV_PYSPARK_MAJOR_PYTHON_VERSION) - .withValue(driverConf.sparkConf.get(PYSPARK_MAJOR_PYTHON_VERSION)) + .withValue(conf.get(PYSPARK_MAJOR_PYTHON_VERSION)) .build()) ++ maybePythonFiles @@ -103,9 +103,9 @@ private[spark] class DriverCommandFeatureStep(driverConf: KubernetesDriverConf) new ContainerBuilder(pod.container) .addToArgs("driver") .addToArgs("--properties-file", SPARK_CONF_PATH) - .addToArgs("--class", driverConf.mainClass) + .addToArgs("--class", conf.mainClass) .addToArgs(resource) - .addToArgs(driverConf.appArgs: _*) + .addToArgs(conf.appArgs: _*) } private def additionalJavaProperties(resource: String): Map[String, String] = { @@ -114,7 +114,7 @@ private[spark] class DriverCommandFeatureStep(driverConf: KubernetesDriverConf) private def additionalPythonProperties(resource: String): Map[String, String] = { resourceType(APP_RESOURCE_TYPE_PYTHON) ++ - mergeFileList("spark.files", Seq(resource) ++ driverConf.pyFiles) + mergeFileList("spark.files", Seq(resource) ++ conf.pyFiles) } private def additionalRProperties(resource: String): Map[String, String] = { @@ -122,7 +122,7 @@ private[spark] class DriverCommandFeatureStep(driverConf: KubernetesDriverConf) } private def mergeFileList(key: String, filesToAdd: Seq[String]): Map[String, String] = { - val existing = Utils.stringToSeq(driverConf.sparkConf.get(key, "")) + val existing = Utils.stringToSeq(conf.get(key, "")) Map(key -> (existing ++ filesToAdd).distinct.mkString(",")) } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopConfExecutorFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopConfExecutorFeatureStep.scala index 08b50e862861..bca66759d586 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopConfExecutorFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopConfExecutorFeatureStep.scala @@ -30,7 +30,7 @@ private[spark] class HadoopConfExecutorFeatureStep(conf: KubernetesExecutorConf) extends KubernetesFeatureConfigStep with Logging { override def configurePod(pod: SparkPod): SparkPod = { - val hadoopConfDirCMapName = conf.sparkConf.getOption(HADOOP_CONFIG_MAP_NAME) + val hadoopConfDirCMapName = conf.getOption(HADOOP_CONFIG_MAP_NAME) require(hadoopConfDirCMapName.isDefined, "Ensure that the env `HADOOP_CONF_DIR` is defined either in the client or " + " using pre-existing ConfigMaps") diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopSparkUserExecutorFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopSparkUserExecutorFeatureStep.scala index 96c9f3964976..e34211076319 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopSparkUserExecutorFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopSparkUserExecutorFeatureStep.scala @@ -28,7 +28,7 @@ private[spark] class HadoopSparkUserExecutorFeatureStep(conf: KubernetesExecutor extends KubernetesFeatureConfigStep { override def configurePod(pod: SparkPod): SparkPod = { - val sparkUserName = conf.sparkConf.get(KERBEROS_SPARK_USER_NAME) + val sparkUserName = conf.get(KERBEROS_SPARK_USER_NAME) HadoopBootstrapUtil.bootstrapSparkUserPod(sparkUserName, pod) } } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/KerberosConfDriverFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/KerberosConfDriverFeatureStep.scala index 7fac6866411f..c6d5a866fa7b 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/KerberosConfDriverFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/KerberosConfDriverFeatureStep.scala @@ -34,22 +34,20 @@ import org.apache.spark.deploy.security.HadoopDelegationTokenManager private[spark] class KerberosConfDriverFeatureStep(kubernetesConf: KubernetesDriverConf) extends KubernetesFeatureConfigStep { - private val conf = kubernetesConf.sparkConf - private val hadoopConfDir = Option(kubernetesConf.sparkConf.getenv(ENV_HADOOP_CONF_DIR)) - private val hadoopConfigMapName = kubernetesConf.sparkConf.get(KUBERNETES_HADOOP_CONF_CONFIG_MAP) + private val hadoopConfigMapName = kubernetesConf.get(KUBERNETES_HADOOP_CONF_CONFIG_MAP) KubernetesUtils.requireNandDefined( hadoopConfDir, hadoopConfigMapName, "Do not specify both the `HADOOP_CONF_DIR` in your ENV and the ConfigMap " + "as the creation of an additional ConfigMap, when one is already specified is extraneous") - private val principal = conf.get(org.apache.spark.internal.config.PRINCIPAL) - private val keytab = conf.get(org.apache.spark.internal.config.KEYTAB) - private val existingSecretName = conf.get(KUBERNETES_KERBEROS_DT_SECRET_NAME) - private val existingSecretItemKey = conf.get(KUBERNETES_KERBEROS_DT_SECRET_ITEM_KEY) - private val krb5File = conf.get(KUBERNETES_KERBEROS_KRB5_FILE) - private val krb5CMap = conf.get(KUBERNETES_KERBEROS_KRB5_CONFIG_MAP) + private val principal = kubernetesConf.get(org.apache.spark.internal.config.PRINCIPAL) + private val keytab = kubernetesConf.get(org.apache.spark.internal.config.KEYTAB) + private val existingSecretName = kubernetesConf.get(KUBERNETES_KERBEROS_DT_SECRET_NAME) + private val existingSecretItemKey = kubernetesConf.get(KUBERNETES_KERBEROS_DT_SECRET_ITEM_KEY) + private val krb5File = kubernetesConf.get(KUBERNETES_KERBEROS_KRB5_FILE) + private val krb5CMap = kubernetesConf.get(KUBERNETES_KERBEROS_KRB5_CONFIG_MAP) private val hadoopConf = SparkHadoopUtil.get.newConfiguration(kubernetesConf.sparkConf) private val tokenManager = new HadoopDelegationTokenManager(kubernetesConf.sparkConf, hadoopConf) private val isKerberosEnabled = From 516ae6816a8521e944fa7471f2c069dbfc93ecfe Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Mon, 26 Nov 2018 15:05:21 -0800 Subject: [PATCH 4/4] Another test fix. --- .../spark/deploy/k8s/submit/KubernetesDriverBuilderSuite.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/KubernetesDriverBuilderSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/KubernetesDriverBuilderSuite.scala index 0f91c27276d3..7e7dc4763c2e 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/KubernetesDriverBuilderSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/KubernetesDriverBuilderSuite.scala @@ -125,7 +125,8 @@ class KubernetesDriverBuilderSuite extends SparkFunSuite { SERVICE_STEP_TYPE, LOCAL_DIRS_STEP_TYPE, MOUNT_VOLUMES_STEP_TYPE, - DRIVER_CMD_STEP_TYPE) + DRIVER_CMD_STEP_TYPE, + HADOOP_GLOBAL_STEP_TYPE) } test("Apply volumes step if a mount subpath is present.") {