From 73f2777c7ef1b925d323599952c792b5605d5c72 Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Fri, 29 Jun 2018 11:41:18 +0300 Subject: [PATCH 01/25] initial Driver logic for Hadoop and Kerberos Support --- .../org/apache/spark/deploy/SparkSubmit.scala | 7 +- .../org/apache/spark/deploy/k8s/Config.scala | 45 +++++++ .../apache/spark/deploy/k8s/Constants.scala | 31 +++++ .../spark/deploy/k8s/KubernetesConf.scala | 29 ++++- .../k8s/features/HadoopConfFeatureStep.scala | 79 ++++++++++++ .../features/HadoopGlobalFeatureStep.scala | 116 ++++++++++++++++++ .../k8s/features/OptionRequirements.scala | 40 ++++++ .../hadoopsteps/HadoopConfSparkUserStep.scala | 43 +++++++ .../hadoopsteps/HadoopConfigMounterStep.scala | 76 ++++++++++++ .../hadoopsteps/HadoopConfigSpec.scala | 43 +++++++ .../hadoopsteps/HadoopConfigurationStep.scala | 26 ++++ .../HadoopKerberosKeytabResolverStep.scala | 105 ++++++++++++++++ .../HadoopKerberosSecretResolverStep.scala | 41 +++++++ .../hadoopsteps/HadoopStepsOrchestrator.scala | 89 ++++++++++++++ ...bernetesHadoopDelegationTokenManager.scala | 81 ++++++++++++ .../submit/KubernetesClientApplication.scala | 9 +- .../k8s/submit/KubernetesDriverBuilder.scala | 22 +++- .../cluster/k8s/ExecutorPodsAllocator.scala | 4 +- .../deploy/k8s/KubernetesConfSuite.scala | 27 ++-- .../BasicDriverFeatureStepSuite.scala | 16 ++- .../BasicExecutorFeatureStepSuite.scala | 9 +- ...ubernetesCredentialsFeatureStepSuite.scala | 9 +- .../DriverServiceFeatureStepSuite.scala | 18 ++- .../features/EnvSecretsFeatureStepSuite.scala | 3 +- .../features/LocalDirsFeatureStepSuite.scala | 3 +- .../MountSecretsFeatureStepSuite.scala | 3 +- .../bindings/JavaDriverFeatureStepSuite.scala | 3 +- .../PythonDriverFeatureStepSuite.scala | 6 +- .../spark/deploy/k8s/submit/ClientSuite.scala | 3 +- .../submit/KubernetesDriverBuilderSuite.scala | 56 ++++++++- .../k8s/ExecutorPodsAllocatorSuite.scala | 4 +- .../k8s/KubernetesExecutorBuilderSuite.scala | 6 +- 32 files changed, 995 insertions(+), 57 deletions(-) create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopConfFeatureStep.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopGlobalFeatureStep.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/OptionRequirements.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopConfSparkUserStep.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopConfigMounterStep.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopConfigSpec.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopConfigurationStep.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopKerberosKeytabResolverStep.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopKerberosSecretResolverStep.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopStepsOrchestrator.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/security/KubernetesHadoopDelegationTokenManager.scala diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index e83d82f847c6..5efbf389c8ad 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -336,7 +336,7 @@ private[spark] class SparkSubmit extends Logging { val targetDir = Utils.createTempDir() // assure a keytab is available from any place in a JVM - if (clusterManager == YARN || clusterManager == LOCAL || isMesosClient) { + if (clusterManager == YARN || clusterManager == LOCAL || isMesosClient || isKubernetesCluster) { if (args.principal != null) { if (args.keytab != null) { require(new File(args.keytab).exists(), s"Keytab file: ${args.keytab} does not exist") @@ -644,7 +644,8 @@ private[spark] class SparkSubmit extends Logging { } } - if (clusterManager == MESOS && UserGroupInformation.isSecurityEnabled) { + if ((clusterManager == MESOS || clusterManager == KUBERNETES) + && UserGroupInformation.isSecurityEnabled) { setRMPrincipal(sparkConf) } @@ -755,7 +756,7 @@ private[spark] class SparkSubmit extends Logging { // [SPARK-20328]. HadoopRDD calls into a Hadoop library that fetches delegation tokens with // renewer set to the YARN ResourceManager. Since YARN isn't configured in Mesos mode, we - // must trick it into thinking we're YARN. + // must trick it into thinking we're YARN. Same is on for Kubernetes. private def setRMPrincipal(sparkConf: SparkConf): Unit = { val shortUserName = UserGroupInformation.getCurrentUser.getShortUserName val key = s"spark.hadoop.${YarnConfiguration.RM_PRINCIPAL}" 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 bf33179ae3da..900e0d650d30 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 @@ -211,6 +211,51 @@ private[spark] object Config extends Logging { "Ensure that major Python version is either Python2 or Python3") .createWithDefault("2") + val KUBERNETES_KERBEROS_SUPPORT = + ConfigBuilder("spark.kubernetes.kerberos.enabled") + .doc("Specify whether your job is a job that will require a Delegation Token to access HDFS") + .booleanConf + .createWithDefault(false) + + val KUBERNETES_KERBEROS_KEYTAB = + ConfigBuilder("spark.kubernetes.kerberos.keytab") + .doc("Specify the location of keytab " + + "for Kerberos in order to access Secure HDFS") + .stringConf + .createOptional + + val KUBERNETES_KERBEROS_PRINCIPAL = + ConfigBuilder("spark.kubernetes.kerberos.principal") + .doc("Specify the principal " + + "for Kerberos in order to access Secure HDFS") + .stringConf + .createOptional + + val KUBERNETES_KERBEROS_RENEWER_PRINCIPAL = + ConfigBuilder("spark.kubernetes.kerberos.renewer.principal") + .doc("Specify the principal " + + "you wish to renew and retrieve your Kerberos values with") + .stringConf + .createOptional + + val KUBERNETES_KERBEROS_DT_SECRET_NAME = + ConfigBuilder("spark.kubernetes.kerberos.tokensecret.name") + .doc("Specify the name of the secret where " + + "your existing delegation token is stored. This removes the need " + + "for the job user to provide any keytab for launching a job") + .stringConf + .createOptional + + val KUBERNETES_KERBEROS_DT_SECRET_ITEM_KEY = + ConfigBuilder("spark.kubernetes.kerberos.tokensecret.itemkey") + .doc("Specify the item key of the data where " + + "your existing delegation token is stored. This removes the need " + + "for the job user to provide any keytab for launching a job") + .stringConf + .createOptional + + + val KUBERNETES_AUTH_SUBMISSION_CONF_PREFIX = "spark.kubernetes.authenticate.submission" diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Constants.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Constants.scala index 69bd03d1eda6..1e21a68a6358 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Constants.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Constants.scala @@ -65,11 +65,13 @@ private[spark] object Constants { val ENV_CLASSPATH = "SPARK_CLASSPATH" val ENV_DRIVER_BIND_ADDRESS = "SPARK_DRIVER_BIND_ADDRESS" val ENV_SPARK_CONF_DIR = "SPARK_CONF_DIR" + val ENV_SPARK_USER = "SPARK_USER" // Spark app configs for containers val SPARK_CONF_VOLUME = "spark-conf-volume" val SPARK_CONF_DIR_INTERNAL = "/opt/spark/conf" val SPARK_CONF_FILE_NAME = "spark.properties" val SPARK_CONF_PATH = s"$SPARK_CONF_DIR_INTERNAL/$SPARK_CONF_FILE_NAME" + val ENV_HADOOP_TOKEN_FILE_LOCATION = "HADOOP_TOKEN_FILE_LOCATION" // BINDINGS val ENV_PYSPARK_PRIMARY = "PYSPARK_PRIMARY" @@ -81,4 +83,33 @@ private[spark] object Constants { val KUBERNETES_MASTER_INTERNAL_URL = "https://kubernetes.default.svc" val DRIVER_CONTAINER_NAME = "spark-kubernetes-driver" val MEMORY_OVERHEAD_MIN_MIB = 384L + + // Hadoop Configuration + val HADOOP_FILE_VOLUME = "hadoop-properties" + val HADOOP_CONF_DIR_PATH = "/etc/hadoop/conf" + val ENV_HADOOP_CONF_DIR = "HADOOP_CONF_DIR" + val HADOOP_CONF_DIR_LOC = "spark.kubernetes.hadoop.conf.dir" + val HADOOP_CONFIG_MAP_SPARK_CONF_NAME = + "spark.kubernetes.hadoop.executor.hadoopConfigMapName" + + // Kerberos Configuration + val KERBEROS_DELEGEGATION_TOKEN_SECRET_NAME = + "spark.kubernetes.kerberos.delegation-token-secret-name" + val KERBEROS_KEYTAB_SECRET_NAME = + "spark.kubernetes.kerberos.key-tab-secret-name" + val KERBEROS_KEYTAB_SECRET_KEY = + "spark.kubernetes.kerberos.key-tab-secret-key" + val KERBEROS_SECRET_LABEL_PREFIX = + "hadoop-tokens" + val SPARK_HADOOP_PREFIX = "spark.hadoop." + val HADOOP_SECURITY_AUTHENTICATION = + SPARK_HADOOP_PREFIX + "hadoop.security.authentication" + + // Kerberos Token-Refresh Server + val KERBEROS_REFRESH_LABEL_KEY = "refresh-hadoop-tokens" + val KERBEROS_REFRESH_LABEL_VALUE = "yes" + + // Hadoop credentials secrets for the Spark app. + private[spark] val SPARK_APP_HADOOP_CREDENTIALS_BASE_DIR = "/mnt/secrets/hadoop-credentials" + private[spark] val SPARK_APP_HADOOP_SECRET_VOLUME_NAME = "hadoop-secret" } 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 b0ccaa36b01e..34f1979d9c9f 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 @@ -23,6 +23,8 @@ import io.fabric8.kubernetes.api.model.{LocalObjectReference, LocalObjectReferen 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.features.hadoopsteps.HadoopStepsOrchestrator +import org.apache.spark.deploy.k8s.security.KubernetesHadoopDelegationTokenManager import org.apache.spark.deploy.k8s.submit._ import org.apache.spark.internal.config.ConfigEntry @@ -59,7 +61,20 @@ private[spark] case class KubernetesConf[T <: KubernetesRoleSpecificConf]( roleSecretNamesToMountPaths: Map[String, String], roleSecretEnvNamesToKeyRefs: Map[String, String], roleEnvs: Map[String, String], - sparkFiles: Seq[String]) { + sparkFiles: Seq[String], + hadoopConfDir: Option[String]) { + + def getHadoopConfigMapName: String = s"$appResourceNamePrefix-hadoop-config" + + def getHadoopStepsOrchestrator : Option[HadoopStepsOrchestrator] = hadoopConfDir.map { + hConf => new HadoopStepsOrchestrator( + sparkConf, + appResourceNamePrefix, + hConf, + getHadoopConfigMapName)} + + def getTokenManager : KubernetesHadoopDelegationTokenManager = + new KubernetesHadoopDelegationTokenManager def namespace(): String = sparkConf.get(KUBERNETES_NAMESPACE) @@ -111,7 +126,8 @@ private[spark] object KubernetesConf { mainAppResource: Option[MainAppResource], mainClass: String, appArgs: Array[String], - maybePyFiles: Option[String]): KubernetesConf[KubernetesDriverSpecificConf] = { + maybePyFiles: Option[String], + hadoopConfDir: Option[String]): KubernetesConf[KubernetesDriverSpecificConf] = { val sparkConfWithMainAppJar = sparkConf.clone() val additionalFiles = mutable.ArrayBuffer.empty[String] mainAppResource.foreach { @@ -171,14 +187,16 @@ private[spark] object KubernetesConf { driverSecretNamesToMountPaths, driverSecretEnvNamesToKeyRefs, driverEnvs, - sparkFiles) + sparkFiles, + hadoopConfDir) } def createExecutorConf( sparkConf: SparkConf, executorId: String, appId: String, - driverPod: Pod): KubernetesConf[KubernetesExecutorSpecificConf] = { + driverPod: Pod, + hadoopConfDir: Option[String]): KubernetesConf[KubernetesExecutorSpecificConf] = { val executorCustomLabels = KubernetesUtils.parsePrefixedKeyValuePairs( sparkConf, KUBERNETES_EXECUTOR_LABEL_PREFIX) require( @@ -214,6 +232,7 @@ private[spark] object KubernetesConf { executorMountSecrets, executorEnvSecrets, executorEnv, - Seq.empty[String]) + Seq.empty[String], + hadoopConfDir) } } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopConfFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopConfFeatureStep.scala new file mode 100644 index 000000000000..1192a20a584d --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopConfFeatureStep.scala @@ -0,0 +1,79 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.k8s.features + +import scala.collection.JavaConverters._ + +import io.fabric8.kubernetes.api.model.{ContainerBuilder, HasMetadata, KeyToPathBuilder, PodBuilder} + +import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesRoleSpecificConf, SparkPod} +import org.apache.spark.deploy.k8s.Constants._ +import org.apache.spark.internal.Logging + + /** + * This step is responsible for bootstraping the container with ConfigMaps + * containing Hadoop config files mounted as volumes and an ENV variable + * pointed to the mounted file directory. This is run by both the driver + * and executor, as they both require Hadoop config files. + */ +private[spark] class HadoopConfFeatureStep( + kubernetesConf: KubernetesConf[_ <: KubernetesRoleSpecificConf]) + extends KubernetesFeatureConfigStep with Logging{ + + override def configurePod(pod: SparkPod): SparkPod = { + require(kubernetesConf.hadoopConfDir.isDefined, "Ensure that HADOOP_CONF_DIR is defined") + logInfo("HADOOP_CONF_DIR defined. Mounting Hadoop specific files") + val kubeTokenManager = kubernetesConf.getTokenManager + val hadoopConfigFiles = + kubeTokenManager.getHadoopConfFiles(kubernetesConf.hadoopConfDir.get) + val keyPaths = hadoopConfigFiles.map { file => + val fileStringPath = file.toPath.getFileName.toString + new KeyToPathBuilder() + .withKey(fileStringPath) + .withPath(fileStringPath) + .build() } + + val hadoopSupportedPod = new PodBuilder(pod.pod) + .editSpec() + .addNewVolume() + .withName(HADOOP_FILE_VOLUME) + .withNewConfigMap() + .withName(kubernetesConf.getHadoopConfigMapName) + .withItems(keyPaths.asJava) + .endConfigMap() + .endVolume() + .endSpec() + .build() + + val hadoopSupportedContainer = new ContainerBuilder(pod.container) + .addNewVolumeMount() + .withName(HADOOP_FILE_VOLUME) + .withMountPath(HADOOP_CONF_DIR_PATH) + .endVolumeMount() + .addNewEnv() + .withName(ENV_HADOOP_CONF_DIR) + .withValue(HADOOP_CONF_DIR_PATH) + .endEnv() + .build() + + SparkPod(hadoopSupportedPod, hadoopSupportedContainer) + } + + 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/HadoopGlobalFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopGlobalFeatureStep.scala new file mode 100644 index 000000000000..1a02f94d6aca --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopGlobalFeatureStep.scala @@ -0,0 +1,116 @@ +/* + * 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 + +import scala.collection.JavaConverters._ + +import io.fabric8.kubernetes.api.model.{ConfigMapBuilder, ContainerBuilder, HasMetadata, PodBuilder} + +import org.apache.spark.deploy.k8s.{KubernetesConf, SparkPod} +import org.apache.spark.deploy.k8s.Constants._ +import org.apache.spark.deploy.k8s.KubernetesRoleSpecificConf +import org.apache.spark.deploy.k8s.features.hadoopsteps.{HadoopConfigSpec, HadoopConfigurationStep} +import org.apache.spark.internal.Logging + + /** + * This is the main method that runs the hadoopConfigurationSteps defined + * by the HadoopStepsOrchestrator. These steps are run to modify the + * SparkPod and Kubernetes Resources using the additive method of the feature steps + */ +private[spark] class HadoopGlobalFeatureStep( + kubernetesConf: KubernetesConf[_ <: KubernetesRoleSpecificConf]) + extends KubernetesFeatureConfigStep with Logging { + private val hadoopTestOrchestrator = + kubernetesConf.getHadoopStepsOrchestrator + require(hadoopTestOrchestrator.isDefined, "Ensure that HADOOP_CONF_DIR is defined") + private val hadoopSteps = + hadoopTestOrchestrator + .map(hto => hto.getHadoopSteps(kubernetesConf.getTokenManager)) + .getOrElse(Seq.empty[HadoopConfigurationStep]) + + var currentHadoopSpec = HadoopConfigSpec( + podVolumes = Seq.empty, + containerEnvs = Seq.empty, + containerVMs = Seq.empty, + configMapProperties = Map.empty[String, String], + dtSecret = None, + dtSecretName = KERBEROS_DELEGEGATION_TOKEN_SECRET_NAME, + dtSecretItemKey = None, + jobUserName = None) + + for (nextStep <- hadoopSteps) { + currentHadoopSpec = nextStep.configureHadoopSpec(currentHadoopSpec) + } + + override def configurePod(pod: SparkPod): SparkPod = { + val hadoopBasedPod = new PodBuilder(pod.pod) + .editSpec() + .addAllToVolumes(currentHadoopSpec.podVolumes.asJava) + .endSpec() + .build() + + val hadoopBasedContainer = new ContainerBuilder(pod.container) + .addAllToEnv(currentHadoopSpec.containerEnvs.asJava) + .addAllToVolumeMounts(currentHadoopSpec.containerVMs.asJava) + .build() + + val maybeKerberosModification = + for { + secretItemKey <- currentHadoopSpec.dtSecretItemKey + userName <- currentHadoopSpec.jobUserName + } yield { + val kerberizedPod = new PodBuilder(hadoopBasedPod) + .editOrNewSpec() + .addNewVolume() + .withName(SPARK_APP_HADOOP_SECRET_VOLUME_NAME) + .withNewSecret() + .withSecretName(currentHadoopSpec.dtSecretName) + .endSecret() + .endVolume() + .endSpec() + .build() + val kerberizedContainer = new ContainerBuilder(hadoopBasedContainer) + .addNewVolumeMount() + .withName(SPARK_APP_HADOOP_SECRET_VOLUME_NAME) + .withMountPath(SPARK_APP_HADOOP_CREDENTIALS_BASE_DIR) + .endVolumeMount() + .addNewEnv() + .withName(ENV_HADOOP_TOKEN_FILE_LOCATION) + .withValue(s"$SPARK_APP_HADOOP_CREDENTIALS_BASE_DIR/$secretItemKey") + .endEnv() + .addNewEnv() + .withName(ENV_SPARK_USER) + .withValue(userName) + .endEnv() + .build() + SparkPod(kerberizedPod, kerberizedContainer) } + maybeKerberosModification.getOrElse(SparkPod(hadoopBasedPod, hadoopBasedContainer)) + } + + override def getAdditionalPodSystemProperties(): Map[String, String] = Map.empty + + override def getAdditionalKubernetesResources(): Seq[HasMetadata] = { + val configMap = + new ConfigMapBuilder() + .withNewMetadata() + .withName(kubernetesConf.getHadoopConfigMapName) + .endMetadata() + .addToData(currentHadoopSpec.configMapProperties.asJava) + .build() + Seq(configMap) ++ currentHadoopSpec.dtSecret.toSeq + } +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/OptionRequirements.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/OptionRequirements.scala new file mode 100644 index 000000000000..c08cfe4e0783 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/OptionRequirements.scala @@ -0,0 +1,40 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.k8s.features + +private[spark] object OptionRequirements { + + def requireBothOrNeitherDefined( + opt1: Option[_], + opt2: Option[_], + errMessageWhenFirstIsMissing: String, + errMessageWhenSecondIsMissing: String): Unit = { + requireSecondIfFirstIsDefined(opt1, opt2, errMessageWhenSecondIsMissing) + requireSecondIfFirstIsDefined(opt2, opt1, errMessageWhenFirstIsMissing) + } + + def requireSecondIfFirstIsDefined( + opt1: Option[_], opt2: Option[_], errMessageWhenSecondIsMissing: String): Unit = { + opt1.foreach { _ => + require(opt2.isDefined, errMessageWhenSecondIsMissing) + } + } + + def requireNandDefined(opt1: Option[_], opt2: Option[_], errMessage: String): Unit = { + opt1.foreach { _ => require(opt2.isEmpty, errMessage) } + } +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopConfSparkUserStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopConfSparkUserStep.scala new file mode 100644 index 000000000000..bb3e19118b6c --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopConfSparkUserStep.scala @@ -0,0 +1,43 @@ +/* + * 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.hadoopsteps + +import io.fabric8.kubernetes.api.model.EnvVarBuilder + +import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesRoleSpecificConf} +import org.apache.spark.deploy.k8s.Constants._ +import org.apache.spark.deploy.k8s.security.KubernetesHadoopDelegationTokenManager + + /** + * 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 HadoopConfSparkUserStep( + tokenManager: KubernetesHadoopDelegationTokenManager) + extends HadoopConfigurationStep { + + override def configureHadoopSpec(hSpec: HadoopConfigSpec) : HadoopConfigSpec = { + val sparkUserEnv = + new EnvVarBuilder() + .withName(ENV_SPARK_USER) + .withValue(tokenManager.getShortUserName) + .build() + hSpec.copy( + containerEnvs = hSpec.containerEnvs :+ sparkUserEnv + ) + } +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopConfigMounterStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopConfigMounterStep.scala new file mode 100644 index 000000000000..83f673750eed --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopConfigMounterStep.scala @@ -0,0 +1,76 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.k8s.features.hadoopsteps + +import java.io.File + +import scala.collection.JavaConverters._ + +import com.google.common.base.Charsets +import com.google.common.io.Files +import io.fabric8.kubernetes.api.model.{EnvVarBuilder, KeyToPathBuilder, VolumeBuilder, VolumeMountBuilder} + +import org.apache.spark.deploy.k8s.Constants._ +import org.apache.spark.internal.Logging + + /** + * This step is responsible for creating ConfigMaps containing Hadoop config files + */ +private[spark] class HadoopConfigMounterStep( + hadoopConfConfigMapName: String, + hadoopConfigFiles: Seq[File]) + extends HadoopConfigurationStep with Logging{ + + override def configureHadoopSpec(hSpec: HadoopConfigSpec) : HadoopConfigSpec = { + logInfo("HADOOP_CONF_DIR defined. Mounting Hadoop specific files") + val keyPaths = hadoopConfigFiles.map { file => + val fileStringPath = file.toPath.getFileName.toString + new KeyToPathBuilder() + .withKey(fileStringPath) + .withPath(fileStringPath) + .build() } + + val hadoopFileVolume = + new VolumeBuilder() + .withName(HADOOP_FILE_VOLUME) + .withNewConfigMap() + .withName(hadoopConfConfigMapName) + .withItems(keyPaths.asJava) + .endConfigMap() + .build() + + val hadoopEnvVals = + new EnvVarBuilder() + .withName(ENV_HADOOP_CONF_DIR) + .withValue(HADOOP_CONF_DIR_PATH) + .build() + + val hadoopVolumeMount = + new VolumeMountBuilder() + .withName(HADOOP_FILE_VOLUME) + .withMountPath(HADOOP_CONF_DIR_PATH) + .build() + + hSpec.copy( + podVolumes = hSpec.podVolumes :+ hadoopFileVolume, + containerEnvs = hSpec.containerEnvs :+ hadoopEnvVals, + containerVMs = hSpec.containerVMs :+ hadoopVolumeMount, + configMapProperties = hSpec.configMapProperties ++ + hadoopConfigFiles.map(file => + (file.toPath.getFileName.toString, Files.toString(file, Charsets.UTF_8))).toMap) + } +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopConfigSpec.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopConfigSpec.scala new file mode 100644 index 000000000000..8f6207e27be8 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopConfigSpec.scala @@ -0,0 +1,43 @@ +/* + * 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.hadoopsteps + +import io.fabric8.kubernetes.api.model.{EnvVar, Secret, Volume, VolumeMount} + + /** + * Represents a given configuration of the hadoop configuration logic, informing the + * HadoopConfigBootstrapStep of how the driver should be configured. This includes: + *

+ * - Volumes that need to mounted onto the pod + * - Environmental variables that need to be launched with the container + * - Volume Mounts that need to mounted with the container + * - The properties that will be stored into the config map which have (key, value) + * pairs of (path, data) + * - The secret containing a DT, either previously specified or built on the fly + * - The name of the secret where the DT will be stored + * - The data item-key on the secret which correlates with where the current DT data is stored + * - The Job User's username + */ +private[spark] case class HadoopConfigSpec( + podVolumes: Seq[Volume], + containerEnvs: Seq[EnvVar], + containerVMs: Seq[VolumeMount], + configMapProperties: Map[String, String], + dtSecret: Option[Secret], + dtSecretName: String, + dtSecretItemKey: Option[String], + jobUserName: Option[String]) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopConfigurationStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopConfigurationStep.scala new file mode 100644 index 000000000000..b1e9e5f22033 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopConfigurationStep.scala @@ -0,0 +1,26 @@ +/* + * 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.hadoopsteps + +private[spark] trait HadoopConfigurationStep { + /** + * Apply modifications on the given HadoopConfSpec in accordance to this feature. + * Note that we should return a HadoopConfSpec that keeps all of the properties of the + * passed HadoopConfSpec object. + */ + def configureHadoopSpec(hadoopConfSpec: HadoopConfigSpec) : HadoopConfigSpec +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopKerberosKeytabResolverStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopKerberosKeytabResolverStep.scala new file mode 100644 index 000000000000..b8c9a85a4554 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopKerberosKeytabResolverStep.scala @@ -0,0 +1,105 @@ +/* + * 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.hadoopsteps + +import java.io._ +import java.security.PrivilegedExceptionAction + +import scala.collection.JavaConverters._ + +import io.fabric8.kubernetes.api.model.SecretBuilder +import org.apache.commons.codec.binary.Base64 + +import org.apache.spark.{SparkConf, SparkException} +import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.deploy.k8s.Constants._ +import org.apache.spark.deploy.k8s.security.KubernetesHadoopDelegationTokenManager +import org.apache.spark.deploy.security.HadoopDelegationTokenManager +import org.apache.spark.internal.Logging + + /** + * This step does all the heavy lifting for Delegation Token logic. This step + * assumes that the job user has either specified a principal and keytab or ran + * $kinit before running spark-submit. With a TGT stored locally, by running + * UGI.getCurrentUser you are able to obtain the current user, alternatively + * you can run UGI.loginUserFromKeytabAndReturnUGI and by running .doAs run + * as the logged into user instead of the current user. 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 + * appended to the current HadoopSpec which in turn will append to the current + * DriverSpec. + */ +private[spark] class HadoopKerberosKeytabResolverStep( + submissionSparkConf: SparkConf, + kubernetesResourceNamePrefix : String, + maybePrincipal: Option[String], + maybeKeytab: Option[File], + maybeRenewerPrincipal: Option[String], + tokenManager: KubernetesHadoopDelegationTokenManager) + extends HadoopConfigurationStep with Logging { + + override def configureHadoopSpec(hSpec: HadoopConfigSpec): HadoopConfigSpec = { + val hadoopConf = SparkHadoopUtil.get.newConfiguration(submissionSparkConf) + if (!tokenManager.isSecurityEnabled) { + throw new SparkException("Hadoop not configured with Kerberos") + } + val maybeJobUserUGI = + for { + principal <- maybePrincipal + keytab <- maybeKeytab + } yield { + logDebug("Logged into KDC with keytab using Job User UGI") + tokenManager.loginUserFromKeytabAndReturnUGI( + principal, + keytab.toURI.toString) + } + // In the case that keytab is not specified we will read from Local Ticket Cache + val jobUserUGI = maybeJobUserUGI.getOrElse(tokenManager.getCurrentUser) + // It is necessary to run as jobUserUGI because logged in user != Current User + val (tokenData, renewalInterval) = jobUserUGI.doAs( + new PrivilegedExceptionAction[(Array[Byte], Long)] { + override def run(): (Array[Byte], Long) = { + val originalCredentials = jobUserUGI.getCredentials + val hadoopTokenManager: HadoopDelegationTokenManager = + new HadoopDelegationTokenManager(submissionSparkConf, hadoopConf) + tokenManager.getDelegationTokens( + originalCredentials, + submissionSparkConf, + hadoopConf, + hadoopTokenManager) + }}) + if (tokenData.isEmpty) throw new SparkException(s"Did not obtain any delegation tokens") + val currentTime = tokenManager.getCurrentTime + val initialTokenDataKeyName = s"$KERBEROS_SECRET_LABEL_PREFIX-$currentTime-$renewalInterval" + val uniqueSecretName = + s"$kubernetesResourceNamePrefix-$KERBEROS_DELEGEGATION_TOKEN_SECRET_NAME.$currentTime" + val secretDT = + new SecretBuilder() + .withNewMetadata() + .withName(uniqueSecretName) + .withLabels(Map(KERBEROS_REFRESH_LABEL_KEY -> KERBEROS_REFRESH_LABEL_VALUE).asJava) + .endMetadata() + .addToData(initialTokenDataKeyName, Base64.encodeBase64String(tokenData)) + .build() + + hSpec.copy( + dtSecret = Some(secretDT), + dtSecretName = uniqueSecretName, + dtSecretItemKey = Some(initialTokenDataKeyName), + jobUserName = Some(jobUserUGI.getShortUserName)) + } +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopKerberosSecretResolverStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopKerberosSecretResolverStep.scala new file mode 100644 index 000000000000..ff3c75494bb3 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopKerberosSecretResolverStep.scala @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.k8s.features.hadoopsteps + +import org.apache.spark.deploy.k8s.security.KubernetesHadoopDelegationTokenManager +import org.apache.spark.internal.Logging + + /** + * This step assumes that you have already done all the heavy lifting in retrieving a + * delegation token and storing the following data in a secret before running this job. + * This step requires that you just specify the secret name and data item-key corresponding + * to the data where the delegation token is stored. + */ +private[spark] class HadoopKerberosSecretResolverStep( + tokenSecretName: String, + tokenItemKeyName: String, + tokenManager: KubernetesHadoopDelegationTokenManager) + extends HadoopConfigurationStep with Logging { + + override def configureHadoopSpec(hSpec: HadoopConfigSpec): HadoopConfigSpec = { + hSpec.copy( + dtSecret = None, + dtSecretName = tokenSecretName, + dtSecretItemKey = Some(tokenItemKeyName), + jobUserName = Some(tokenManager.getCurrentUser.getShortUserName)) + } +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopStepsOrchestrator.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopStepsOrchestrator.scala new file mode 100644 index 000000000000..f235bff2b2d5 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopStepsOrchestrator.scala @@ -0,0 +1,89 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.k8s.features.hadoopsteps + +import java.io.File + +import org.apache.spark.SparkConf +import org.apache.spark.deploy.k8s.Config._ +import org.apache.spark.deploy.k8s.features.OptionRequirements +import org.apache.spark.deploy.k8s.security.KubernetesHadoopDelegationTokenManager +import org.apache.spark.internal.Logging + +private[spark] class HadoopStepsOrchestrator( + conf: SparkConf, + kubernetesResourceNamePrefix: String, + hadoopConfDir: String, + hadoopConfigMapName: String) extends Logging { + + private val isKerberosEnabled = conf.get(KUBERNETES_KERBEROS_SUPPORT) + private val maybePrincipal = conf.get(KUBERNETES_KERBEROS_PRINCIPAL) + private val maybeKeytab = conf.get(KUBERNETES_KERBEROS_KEYTAB) + .map(k => new File(k)) + private val maybeExistingSecretName = conf.get(KUBERNETES_KERBEROS_DT_SECRET_NAME) + private val maybeExistingSecretItemKey = + conf.get(KUBERNETES_KERBEROS_DT_SECRET_ITEM_KEY) + private val maybeRenewerPrincipal = + conf.get(KUBERNETES_KERBEROS_RENEWER_PRINCIPAL) + + require(maybeKeytab.forall( _ => isKerberosEnabled ), + "You must enable Kerberos support if you are specifying a Kerberos Keytab") + + require(maybeExistingSecretName.forall( _ => isKerberosEnabled ), + "You must enable Kerberos support if you are specifying a Kerberos Secret") + + OptionRequirements.requireBothOrNeitherDefined( + maybeKeytab, + maybePrincipal, + "If a Kerberos keytab is specified you must also specify a Kerberos principal", + "If a Kerberos principal is specified you must also specify a Kerberos keytab") + + OptionRequirements.requireBothOrNeitherDefined( + maybeExistingSecretName, + maybeExistingSecretItemKey, + "If a secret storing a Kerberos Delegation Token is specified you must also" + + " specify the label where the data is stored", + "If a secret data item-key where the data of the Kerberos Delegation Token is specified" + + " you must also specify the name of the secret") + + def getHadoopSteps(kubeTokenManager: KubernetesHadoopDelegationTokenManager): + Seq[HadoopConfigurationStep] = { + val hadoopConfigurationFiles = kubeTokenManager.getHadoopConfFiles(hadoopConfDir) + logInfo(s"Hadoop Conf directory: $hadoopConfDir") + val hadoopConfMounterStep = new HadoopConfigMounterStep( + hadoopConfigMapName, hadoopConfigurationFiles) + val maybeKerberosStep = + if (isKerberosEnabled) { + val maybeExistingSecretStep = for { + secretName <- maybeExistingSecretName + secretItemKey <- maybeExistingSecretItemKey + } yield { + new HadoopKerberosSecretResolverStep(secretName, secretItemKey, kubeTokenManager) + } + maybeExistingSecretStep.getOrElse(new HadoopKerberosKeytabResolverStep( + conf, + kubernetesResourceNamePrefix, + maybePrincipal, + maybeKeytab, + maybeRenewerPrincipal, + kubeTokenManager)) + } else { + new HadoopConfSparkUserStep(kubeTokenManager) + } + Seq(hadoopConfMounterStep) :+ maybeKerberosStep + } +} 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 new file mode 100644 index 000000000000..987e8b208708 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/security/KubernetesHadoopDelegationTokenManager.scala @@ -0,0 +1,81 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.deploy.k8s.security + +import java.io.File + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.FileSystem +import org.apache.hadoop.security.{Credentials, UserGroupInformation} + +import org.apache.spark.SparkConf +import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.deploy.security.HadoopDelegationTokenManager +import org.apache.spark.internal.Logging +import org.apache.spark.util.{Clock, SystemClock} + + /** + * The KubernetesHadoopDelegationTokenManager fetches and updates Hadoop delegation tokens + * on the behalf of the Kubernetes submission client. It is modeled after the YARN + * AMCredentialRenewer, renewals in Kubernetes happen in a seperate microservice that will + * automatically update the Tokens via Kubernetes Secrets. The principal difference is that + * instead of writing the new credentials to HDFS and incrementing the timestamp of the file, + * the new credentials (called Tokens when they are serialized) are stored in Secrets accessible + * to the driver and executors, when new Tokens are received they overwrite the current Secrets. + */ +private[spark] class KubernetesHadoopDelegationTokenManager extends Logging { + + // HadoopUGI Util methods + private val clock: Clock = new SystemClock() + def getCurrentUser: UserGroupInformation = UserGroupInformation.getCurrentUser + def getShortUserName : String = getCurrentUser.getShortUserName + def getFileSystem(hadoopConf: Configuration) : FileSystem = FileSystem.get(hadoopConf) + def isSecurityEnabled: Boolean = UserGroupInformation.isSecurityEnabled + def loginUserFromKeytabAndReturnUGI(principal: String, keytab: String): UserGroupInformation = + UserGroupInformation.loginUserFromKeytabAndReturnUGI(principal, keytab) + def getCurrentTime: Long = clock.getTimeMillis() + def serializeCreds(creds: Credentials): Array[Byte] = SparkHadoopUtil.get.serialize(creds) + def nextRT(rt: Long, conf: SparkConf): Long = SparkHadoopUtil.nextCredentialRenewalTime(rt, conf) + + // Grab files in the HADOOP_CONF directory + def getHadoopConfFiles(path: String) : Seq[File] = { + val dir = new File(path) + if (dir.isDirectory) { + dir.listFiles.flatMap { file => Some(file).filter(_.isFile) }.toSeq + } else { + Seq.empty[File] + } + } + + // Principle method in charge of retrieving new Delegation Tokens + def getDelegationTokens( + creds: Credentials, + conf: SparkConf, + hadoopConf: Configuration, + tokenManager: HadoopDelegationTokenManager): (Array[Byte], Long) = { + try { + val rt = tokenManager.obtainDelegationTokens(hadoopConf, creds) + logDebug(s"Initialized tokens: ${SparkHadoopUtil.get.dumpTokens(creds)}") + (serializeCreds(creds), nextRT(rt, conf)) + } catch { + case e: Exception => + logError(s"Failed to fetch Hadoop delegation tokens $e") + throw e + } + } +} 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 eaff47205dbb..a8c4708392f3 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 @@ -45,7 +45,8 @@ private[spark] case class ClientArguments( mainAppResource: Option[MainAppResource], mainClass: String, driverArgs: Array[String], - maybePyFiles: Option[String]) + maybePyFiles: Option[String], + hadoopConfigDir: Option[String]) private[spark] object ClientArguments { @@ -77,7 +78,8 @@ private[spark] object ClientArguments { mainAppResource, mainClass.get, driverArgs.toArray, - maybePyFiles) + maybePyFiles, + sys.env.get(ENV_HADOOP_CONF_DIR)) } } @@ -223,7 +225,8 @@ private[spark] class KubernetesClientApplication extends SparkApplication { clientArguments.mainAppResource, clientArguments.mainClass, clientArguments.driverArgs, - clientArguments.maybePyFiles) + clientArguments.maybePyFiles, + clientArguments.hadoopConfigDir) val builder = new KubernetesDriverBuilder val namespace = kubernetesConf.namespace() // The master URL has been checked for validity already in SparkSubmit. 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 5762d8245f77..c9ec2cd8ddf7 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 @@ -17,7 +17,7 @@ package org.apache.spark.deploy.k8s.submit import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesDriverSpec, KubernetesDriverSpecificConf, KubernetesRoleSpecificConf} -import org.apache.spark.deploy.k8s.features.{BasicDriverFeatureStep, DriverKubernetesCredentialsFeatureStep, DriverServiceFeatureStep, EnvSecretsFeatureStep, KubernetesFeatureConfigStep, LocalDirsFeatureStep, MountSecretsFeatureStep} +import org.apache.spark.deploy.k8s.features._ import org.apache.spark.deploy.k8s.features.bindings.{JavaDriverFeatureStep, PythonDriverFeatureStep} private[spark] class KubernetesDriverBuilder( @@ -44,7 +44,15 @@ private[spark] class KubernetesDriverBuilder( providePythonStep: ( KubernetesConf[KubernetesDriverSpecificConf] => PythonDriverFeatureStep) = - new PythonDriverFeatureStep(_)) { + new PythonDriverFeatureStep(_), + provideHadoopConfStep: ( + KubernetesConf[KubernetesDriverSpecificConf] + => HadoopConfFeatureStep) = + new HadoopConfFeatureStep(_), + provideHadoopGlobalStep: ( + KubernetesConf[KubernetesDriverSpecificConf] + => HadoopGlobalFeatureStep) = + new HadoopGlobalFeatureStep(_)) { def buildFromFeatures( kubernetesConf: KubernetesConf[KubernetesDriverSpecificConf]): KubernetesDriverSpec = { @@ -66,10 +74,18 @@ private[spark] class KubernetesDriverBuilder( case PythonMainAppResource(_) => providePythonStep(kubernetesConf)}.getOrElse(provideJavaStep(kubernetesConf)) + val maybeHadoopConfigSteps = + kubernetesConf.hadoopConfDir.map { _ => + Seq( + provideHadoopConfStep(kubernetesConf), + provideHadoopGlobalStep(kubernetesConf)) + }.getOrElse(Seq.empty[KubernetesFeatureConfigStep]) + val allFeatures: Seq[KubernetesFeatureConfigStep] = (baseFeatures :+ bindingsStep) ++ maybeRoleSecretNamesStep.toSeq ++ - maybeProvideSecretsStep.toSeq + maybeProvideSecretsStep.toSeq ++ + maybeHadoopConfigSteps var spec = KubernetesDriverSpec.initialSpec(kubernetesConf.sparkConf.getAll.toMap) for (feature <- allFeatures) { diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala index 5a143ad3600f..f9690fe5bafc 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala @@ -118,11 +118,13 @@ private[spark] class ExecutorPodsAllocator( logInfo(s"Going to request $numExecutorsToAllocate executors from Kubernetes.") for ( _ <- 0 until numExecutorsToAllocate) { val newExecutorId = EXECUTOR_ID_COUNTER.incrementAndGet() + // TODO: HADOOP_CONF_DIR val executorConf = KubernetesConf.createExecutorConf( conf, newExecutorId.toString, applicationId, - driverPod) + driverPod, + None) val executorPod = executorBuilder.buildFromFeatures(executorConf) val podWithAttachedContainer = new PodBuilder(executorPod.pod) .editOrNewSpec() 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 661f94243592..896957fa0e5e 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 @@ -59,7 +59,8 @@ class KubernetesConfSuite extends SparkFunSuite { mainAppResource = None, MAIN_CLASS, APP_ARGS, - maybePyFiles = None) + maybePyFiles = None, + hadoopConfDir = None) assert(conf.appId === APP_ID) assert(conf.sparkConf.getAll.toMap === sparkConf.getAll.toMap) assert(conf.appResourceNamePrefix === RESOURCE_NAME_PREFIX) @@ -81,7 +82,8 @@ class KubernetesConfSuite extends SparkFunSuite { mainAppJar, MAIN_CLASS, APP_ARGS, - maybePyFiles = None) + maybePyFiles = None, + hadoopConfDir = None) assert(kubernetesConfWithMainJar.sparkConf.get("spark.jars") .split(",") === Array("local:///opt/spark/jar1.jar", "local:///opt/spark/main.jar")) @@ -93,7 +95,8 @@ class KubernetesConfSuite extends SparkFunSuite { mainAppResource = None, MAIN_CLASS, APP_ARGS, - maybePyFiles = None) + maybePyFiles = None, + hadoopConfDir = None) assert(kubernetesConfWithoutMainJar.sparkConf.get("spark.jars").split(",") === Array("local:///opt/spark/jar1.jar")) assert(kubernetesConfWithoutMainJar.sparkConf.get(MEMORY_OVERHEAD_FACTOR) === 0.1) @@ -114,7 +117,8 @@ class KubernetesConfSuite extends SparkFunSuite { mainAppResource, MAIN_CLASS, APP_ARGS, - Some(inputPyFiles.mkString(","))) + Some(inputPyFiles.mkString(",")), + hadoopConfDir = None) assert(kubernetesConfWithMainResource.sparkConf.get("spark.jars").split(",") === Array("local:///opt/spark/jar1.jar")) assert(kubernetesConfWithMainResource.sparkConf.get(MEMORY_OVERHEAD_FACTOR) === 0.4) @@ -136,7 +140,8 @@ class KubernetesConfSuite extends SparkFunSuite { mainAppResource, MAIN_CLASS, APP_ARGS, - None) + maybePyFiles = None, + hadoopConfDir = None) assert(conf.sparkConf.get(MEMORY_OVERHEAD_FACTOR) === 0.3) } @@ -167,7 +172,8 @@ class KubernetesConfSuite extends SparkFunSuite { mainAppResource = None, MAIN_CLASS, APP_ARGS, - maybePyFiles = None) + maybePyFiles = None, + hadoopConfDir = None) assert(conf.roleLabels === Map( SPARK_APP_ID_LABEL -> APP_ID, SPARK_ROLE_LABEL -> SPARK_POD_DRIVER_ROLE) ++ @@ -184,7 +190,8 @@ class KubernetesConfSuite extends SparkFunSuite { new SparkConf(false), EXECUTOR_ID, APP_ID, - DRIVER_POD) + DRIVER_POD, + None) assert(conf.roleSpecificConf.executorId === EXECUTOR_ID) assert(conf.roleSpecificConf.driverPod === DRIVER_POD) } @@ -195,7 +202,8 @@ class KubernetesConfSuite extends SparkFunSuite { .set(IMAGE_PULL_SECRETS, "my-secret-1,my-secret-2 "), EXECUTOR_ID, APP_ID, - DRIVER_POD) + DRIVER_POD, + None) assert(conf.imagePullSecrets() === Seq( new LocalObjectReferenceBuilder().withName("my-secret-1").build(), @@ -221,7 +229,8 @@ class KubernetesConfSuite extends SparkFunSuite { sparkConf, EXECUTOR_ID, APP_ID, - DRIVER_POD) + DRIVER_POD, + None) assert(conf.roleLabels === Map( SPARK_EXECUTOR_ID_LABEL -> EXECUTOR_ID, SPARK_APP_ID_LABEL -> APP_ID, 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 04b909db9d9f..5611faac0104 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 @@ -74,7 +74,8 @@ class BasicDriverFeatureStepSuite extends SparkFunSuite { Map.empty, Map.empty, DRIVER_ENVS, - Seq.empty[String]) + Seq.empty[String], + hadoopConfDir = None) val featureStep = new BasicDriverFeatureStep(kubernetesConf) val basePod = SparkPod.initialPod() @@ -128,7 +129,7 @@ class BasicDriverFeatureStepSuite extends SparkFunSuite { .set(CONTAINER_IMAGE, "spark-driver:latest") val pythonSparkConf = new SparkConf() .set(org.apache.spark.internal.config.DRIVER_MEMORY.key, "4g") - .set(CONTAINER_IMAGE, "spark-driver:latest") + .set(CONTAINER_IMAGE, "spark-driver-py:latest") val javaKubernetesConf = KubernetesConf( javaSparkConf, KubernetesDriverSpecificConf( @@ -143,7 +144,8 @@ class BasicDriverFeatureStepSuite extends SparkFunSuite { Map.empty, Map.empty, DRIVER_ENVS, - Seq.empty[String]) + Seq.empty[String], + hadoopConfDir = None) val pythonKubernetesConf = KubernetesConf( pythonSparkConf, KubernetesDriverSpecificConf( @@ -158,12 +160,15 @@ class BasicDriverFeatureStepSuite extends SparkFunSuite { Map.empty, Map.empty, DRIVER_ENVS, - Seq.empty[String]) + Seq.empty[String], + hadoopConfDir = None) val javaFeatureStep = new BasicDriverFeatureStep(javaKubernetesConf) val pythonFeatureStep = new BasicDriverFeatureStep(pythonKubernetesConf) val basePod = SparkPod.initialPod() val configuredJavaPod = javaFeatureStep.configurePod(basePod) val configuredPythonPod = pythonFeatureStep.configurePod(basePod) + assert(configuredJavaPod.container.getImage === "spark-driver:latest") + assert(configuredPythonPod.container.getImage === "spark-driver-py:latest") } test("Additional system properties resolve jars and set cluster-mode confs.") { @@ -188,7 +193,8 @@ class BasicDriverFeatureStepSuite extends SparkFunSuite { Map.empty, Map.empty, DRIVER_ENVS, - allFiles) + allFiles, + hadoopConfDir = None) val step = new BasicDriverFeatureStep(kubernetesConf) val additionalProperties = step.getAdditionalPodSystemProperties() val expectedSparkConf = Map( 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 f06030aa55c0..df7a114b96dd 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 @@ -89,7 +89,8 @@ class BasicExecutorFeatureStepSuite Map.empty, Map.empty, Map.empty, - Seq.empty[String])) + Seq.empty[String], + hadoopConfDir = None)) val executor = step.configurePod(SparkPod.initialPod()) // The executor pod name and default labels. @@ -128,7 +129,8 @@ class BasicExecutorFeatureStepSuite Map.empty, Map.empty, Map.empty, - Seq.empty[String])) + Seq.empty[String], + hadoopConfDir = None)) assert(step.configurePod(SparkPod.initialPod()).pod.getSpec.getHostname.length === 63) } @@ -148,7 +150,8 @@ class BasicExecutorFeatureStepSuite Map.empty, Map.empty, Map("qux" -> "quux"), - Seq.empty[String])) + Seq.empty[String], + hadoopConfDir = None)) val executor = step.configurePod(SparkPod.initialPod()) checkEnv(executor, 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 7cea83591f3e..ef31a5f64b90 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 @@ -61,7 +61,8 @@ class DriverKubernetesCredentialsFeatureStepSuite extends SparkFunSuite with Bef Map.empty, Map.empty, Map.empty, - Seq.empty[String]) + Seq.empty[String], + hadoopConfDir = None) val kubernetesCredentialsStep = new DriverKubernetesCredentialsFeatureStep(kubernetesConf) assert(kubernetesCredentialsStep.configurePod(BASE_DRIVER_POD) === BASE_DRIVER_POD) assert(kubernetesCredentialsStep.getAdditionalPodSystemProperties().isEmpty) @@ -92,7 +93,8 @@ class DriverKubernetesCredentialsFeatureStepSuite extends SparkFunSuite with Bef Map.empty, Map.empty, Map.empty, - Seq.empty[String]) + Seq.empty[String], + hadoopConfDir = None) val kubernetesCredentialsStep = new DriverKubernetesCredentialsFeatureStep(kubernetesConf) assert(kubernetesCredentialsStep.configurePod(BASE_DRIVER_POD) === BASE_DRIVER_POD) @@ -130,7 +132,8 @@ class DriverKubernetesCredentialsFeatureStepSuite extends SparkFunSuite with Bef Map.empty, Map.empty, Map.empty, - Seq.empty[String]) + Seq.empty[String], + hadoopConfDir = None) val kubernetesCredentialsStep = new DriverKubernetesCredentialsFeatureStep(kubernetesConf) val resolvedProperties = kubernetesCredentialsStep.getAdditionalPodSystemProperties() val expectedSparkConf = Map( 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 77d38bf19cd1..43c1d9baa7c4 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 @@ -67,7 +67,8 @@ class DriverServiceFeatureStepSuite extends SparkFunSuite with BeforeAndAfter { Map.empty, Map.empty, Map.empty, - Seq.empty[String])) + Seq.empty[String], + hadoopConfDir = None)) assert(configurationStep.configurePod(SparkPod.initialPod()) === SparkPod.initialPod()) assert(configurationStep.getAdditionalKubernetesResources().size === 1) assert(configurationStep.getAdditionalKubernetesResources().head.isInstanceOf[Service]) @@ -98,7 +99,8 @@ class DriverServiceFeatureStepSuite extends SparkFunSuite with BeforeAndAfter { Map.empty, Map.empty, Map.empty, - Seq.empty[String])) + Seq.empty[String], + hadoopConfDir = None)) val expectedServiceName = SHORT_RESOURCE_NAME_PREFIX + DriverServiceFeatureStep.DRIVER_SVC_POSTFIX val expectedHostName = s"$expectedServiceName.my-namespace.svc" @@ -119,7 +121,8 @@ class DriverServiceFeatureStepSuite extends SparkFunSuite with BeforeAndAfter { Map.empty, Map.empty, Map.empty, - Seq.empty[String])) + Seq.empty[String], + hadoopConfDir = None)) val resolvedService = configurationStep .getAdditionalKubernetesResources() .head @@ -149,7 +152,8 @@ class DriverServiceFeatureStepSuite extends SparkFunSuite with BeforeAndAfter { Map.empty, Map.empty, Map.empty, - Seq.empty[String]), + Seq.empty[String], + hadoopConfDir = None), clock) val driverService = configurationStep .getAdditionalKubernetesResources() @@ -176,7 +180,8 @@ class DriverServiceFeatureStepSuite extends SparkFunSuite with BeforeAndAfter { Map.empty, Map.empty, Map.empty, - Seq.empty[String]), + Seq.empty[String], + hadoopConfDir = None), clock) fail("The driver bind address should not be allowed.") } catch { @@ -201,7 +206,8 @@ class DriverServiceFeatureStepSuite extends SparkFunSuite with BeforeAndAfter { Map.empty, Map.empty, Map.empty, - Seq.empty[String]), + Seq.empty[String], + hadoopConfDir = None), clock) fail("The driver host address should not be allowed.") } catch { 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 af6b35eae484..4489baa2e163 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 @@ -45,7 +45,8 @@ class EnvSecretsFeatureStepSuite extends SparkFunSuite{ Map.empty, envVarsToKeys, Map.empty, - Seq.empty[String]) + Seq.empty[String], + hadoopConfDir = None) val step = new EnvSecretsFeatureStep(kubernetesConf) val driverContainerWithEnvSecrets = step.configurePod(baseDriverPod).container 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 bd6ce4b42fc8..71650e122c3f 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 @@ -45,7 +45,8 @@ class LocalDirsFeatureStepSuite extends SparkFunSuite with BeforeAndAfter { Map.empty, Map.empty, Map.empty, - Seq.empty[String]) + Seq.empty[String], + hadoopConfDir = None) } test("Resolve to default local dir if neither env nor configuration are set") { 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 eff75b8a15da..b83525d927e0 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 @@ -43,7 +43,8 @@ class MountSecretsFeatureStepSuite extends SparkFunSuite { secretNamesToMountPaths, Map.empty, Map.empty, - Seq.empty[String]) + Seq.empty[String], + hadoopConfDir = None) 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/bindings/JavaDriverFeatureStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/bindings/JavaDriverFeatureStepSuite.scala index 0f2bf2fa1d9b..12d1254374db 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/bindings/JavaDriverFeatureStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/bindings/JavaDriverFeatureStepSuite.scala @@ -42,7 +42,8 @@ class JavaDriverFeatureStepSuite extends SparkFunSuite { roleSecretNamesToMountPaths = Map.empty, roleSecretEnvNamesToKeyRefs = Map.empty, roleEnvs = Map.empty, - sparkFiles = Seq.empty[String]) + sparkFiles = Seq.empty[String], + hadoopConfDir = None) val step = new JavaDriverFeatureStep(kubernetesConf) val driverPod = step.configurePod(baseDriverPod).pod diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/bindings/PythonDriverFeatureStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/bindings/PythonDriverFeatureStepSuite.scala index a1f9a5d9e264..e0953370374e 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/bindings/PythonDriverFeatureStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/bindings/PythonDriverFeatureStepSuite.scala @@ -52,7 +52,8 @@ class PythonDriverFeatureStepSuite extends SparkFunSuite { roleSecretNamesToMountPaths = Map.empty, roleSecretEnvNamesToKeyRefs = Map.empty, roleEnvs = Map.empty, - sparkFiles = Seq.empty[String]) + sparkFiles = Seq.empty[String], + hadoopConfDir = None) val step = new PythonDriverFeatureStep(kubernetesConf) val driverPod = step.configurePod(baseDriverPod).pod @@ -88,7 +89,8 @@ class PythonDriverFeatureStepSuite extends SparkFunSuite { roleSecretNamesToMountPaths = Map.empty, roleSecretEnvNamesToKeyRefs = Map.empty, roleEnvs = Map.empty, - sparkFiles = Seq.empty[String]) + sparkFiles = Seq.empty[String], + hadoopConfDir = None) val step = new PythonDriverFeatureStep(kubernetesConf) val driverContainerwithPySpark = step.configurePod(baseDriverPod).container val args = driverContainerwithPySpark 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 d045d9ae89c0..d5b97386dbd8 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 @@ -141,7 +141,8 @@ class ClientSuite extends SparkFunSuite with BeforeAndAfter { Map.empty, Map.empty, Map.empty, - Seq.empty[String]) + Seq.empty[String], + hadoopConfDir = None) when(driverBuilder.buildFromFeatures(kubernetesConf)).thenReturn(BUILT_KUBERNETES_SPEC) when(kubernetesClient.pods()).thenReturn(podOperations) when(podOperations.withName(POD_NAME)).thenReturn(namedPods) 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 4e8c30054343..d04704f74152 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 @@ -18,7 +18,7 @@ package org.apache.spark.deploy.k8s.submit import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesDriverSpec, KubernetesDriverSpecificConf} -import org.apache.spark.deploy.k8s.features.{BasicDriverFeatureStep, DriverKubernetesCredentialsFeatureStep, DriverServiceFeatureStep, EnvSecretsFeatureStep, KubernetesFeaturesTestUtils, LocalDirsFeatureStep, MountSecretsFeatureStep} +import org.apache.spark.deploy.k8s.features.{BasicDriverFeatureStep, DriverKubernetesCredentialsFeatureStep, DriverServiceFeatureStep, EnvSecretsFeatureStep, HadoopConfFeatureStep, HadoopGlobalFeatureStep, KubernetesFeaturesTestUtils, LocalDirsFeatureStep, MountSecretsFeatureStep} import org.apache.spark.deploy.k8s.features.bindings.{JavaDriverFeatureStep, PythonDriverFeatureStep} class KubernetesDriverBuilderSuite extends SparkFunSuite { @@ -31,6 +31,8 @@ class KubernetesDriverBuilderSuite extends SparkFunSuite { private val JAVA_STEP_TYPE = "java-bindings" private val PYSPARK_STEP_TYPE = "pyspark-bindings" private val ENV_SECRETS_STEP_TYPE = "env-secrets" + private val HADOOP_CONF_STEP_TYPE = "hadoop-conf" + private val HADOOP_GLOBAL_STEP_TYPE = "hadoop-global" private val basicFeatureStep = KubernetesFeaturesTestUtils.getMockConfigStepForStepType( BASIC_STEP_TYPE, classOf[BasicDriverFeatureStep]) @@ -56,6 +58,13 @@ class KubernetesDriverBuilderSuite extends SparkFunSuite { private val envSecretsStep = KubernetesFeaturesTestUtils.getMockConfigStepForStepType( ENV_SECRETS_STEP_TYPE, classOf[EnvSecretsFeatureStep]) + private val hadoopConfStep = KubernetesFeaturesTestUtils.getMockConfigStepForStepType( + HADOOP_CONF_STEP_TYPE, classOf[HadoopConfFeatureStep]) + + private val hadoopGlobalStep = KubernetesFeaturesTestUtils.getMockConfigStepForStepType( + HADOOP_GLOBAL_STEP_TYPE, classOf[HadoopGlobalFeatureStep]) + + private val builderUnderTest: KubernetesDriverBuilder = new KubernetesDriverBuilder( _ => basicFeatureStep, @@ -65,7 +74,9 @@ class KubernetesDriverBuilderSuite extends SparkFunSuite { _ => envSecretsStep, _ => localDirsStep, _ => javaStep, - _ => pythonStep) + _ => pythonStep, + _ => hadoopConfStep, + _ => hadoopGlobalStep) test("Apply fundamental steps all the time.") { val conf = KubernetesConf( @@ -82,7 +93,8 @@ class KubernetesDriverBuilderSuite extends SparkFunSuite { Map.empty, Map.empty, Map.empty, - Seq.empty[String]) + Seq.empty[String], + hadoopConfDir = None) validateStepTypesApplied( builderUnderTest.buildFromFeatures(conf), BASIC_STEP_TYPE, @@ -107,7 +119,8 @@ class KubernetesDriverBuilderSuite extends SparkFunSuite { Map("secret" -> "secretMountPath"), Map("EnvName" -> "SecretName:secretKey"), Map.empty, - Seq.empty[String]) + Seq.empty[String], + hadoopConfDir = None) validateStepTypesApplied( builderUnderTest.buildFromFeatures(conf), BASIC_STEP_TYPE, @@ -134,7 +147,8 @@ class KubernetesDriverBuilderSuite extends SparkFunSuite { Map.empty, Map.empty, Map.empty, - Seq.empty[String]) + Seq.empty[String], + hadoopConfDir = None) validateStepTypesApplied( builderUnderTest.buildFromFeatures(conf), BASIC_STEP_TYPE, @@ -159,7 +173,8 @@ class KubernetesDriverBuilderSuite extends SparkFunSuite { Map.empty, Map.empty, Map.empty, - Seq.empty[String]) + Seq.empty[String], + hadoopConfDir = None) validateStepTypesApplied( builderUnderTest.buildFromFeatures(conf), BASIC_STEP_TYPE, @@ -169,6 +184,35 @@ class KubernetesDriverBuilderSuite extends SparkFunSuite { PYSPARK_STEP_TYPE) } + test("Apply HadoopSteps if HADOOP_CONF_DIR is defined.") { + val conf = KubernetesConf( + new SparkConf(false), + KubernetesDriverSpecificConf( + None, + "test-app", + "main", + Seq.empty), + "prefix", + "appId", + Map.empty, + Map.empty, + Map.empty, + Map.empty, + Map.empty, + Seq.empty[String], + hadoopConfDir = Some("/var/hadoop-conf")) + validateStepTypesApplied( + builderUnderTest.buildFromFeatures(conf), + BASIC_STEP_TYPE, + CREDENTIALS_STEP_TYPE, + SERVICE_STEP_TYPE, + LOCAL_DIRS_STEP_TYPE, + JAVA_STEP_TYPE, + HADOOP_CONF_STEP_TYPE, + HADOOP_GLOBAL_STEP_TYPE) + } + + private def validateStepTypesApplied(resolvedSpec: KubernetesDriverSpec, stepTypes: String*) : Unit = { assert(resolvedSpec.systemProperties.size === stepTypes.size) 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 0c19f5946b75..e835535c796a 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 @@ -162,11 +162,13 @@ class ExecutorPodsAllocatorSuite extends SparkFunSuite with BeforeAndAfter { } 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, - driverPod) + driverPod, + None) 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 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 a6bc8bce3292..204e90552164 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 @@ -55,7 +55,8 @@ class KubernetesExecutorBuilderSuite extends SparkFunSuite { Map.empty, Map.empty, Map.empty, - Seq.empty[String]) + Seq.empty[String], + None) validateStepTypesApplied( builderUnderTest.buildFromFeatures(conf), BASIC_STEP_TYPE, LOCAL_DIRS_STEP_TYPE) } @@ -72,7 +73,8 @@ class KubernetesExecutorBuilderSuite extends SparkFunSuite { Map("secret" -> "secretMountPath"), Map("secret-name" -> "secret-key"), Map.empty, - Seq.empty[String]) + Seq.empty[String], + None) validateStepTypesApplied( builderUnderTest.buildFromFeatures(conf), BASIC_STEP_TYPE, From 6069be5159c97834f72a9b3bf473dcdbd063fad9 Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Sun, 1 Jul 2018 13:19:24 +0300 Subject: [PATCH 02/25] add executors... still need to refactor to use sparkConf exclusivley --- .../apache/spark/deploy/k8s/Constants.scala | 2 + .../k8s/features/HadoopConfFeatureStep.scala | 2 + ...la => HadoopGlobalFeatureDriverStep.scala} | 64 +++++++------- ... HadoopSparkUserExecutorFeatureStep.scala} | 36 ++++---- .../KerberosConfExecutorFeatureStep.scala | 50 +++++++++++ .../hadoopsteps/HadoopBootstrapUtil.scala | 86 +++++++++++++++++++ .../hadoopsteps/HadoopStepsOrchestrator.scala | 12 ++- .../k8s/KubernetesExecutorBuilder.scala | 41 ++++++++- 8 files changed, 235 insertions(+), 58 deletions(-) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/{HadoopGlobalFeatureStep.scala => HadoopGlobalFeatureDriverStep.scala} (67%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/{hadoopsteps/HadoopConfSparkUserStep.scala => HadoopSparkUserExecutorFeatureStep.scala} (53%) create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/KerberosConfExecutorFeatureStep.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopBootstrapUtil.scala diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Constants.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Constants.scala index 1e21a68a6358..a6b2d0794196 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Constants.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Constants.scala @@ -99,6 +99,8 @@ private[spark] object Constants { "spark.kubernetes.kerberos.key-tab-secret-name" val KERBEROS_KEYTAB_SECRET_KEY = "spark.kubernetes.kerberos.key-tab-secret-key" + val KERBEROS_SPARK_USER_NAME = + "spark.kubernetes.kerberos.spark-user-name" val KERBEROS_SECRET_LABEL_PREFIX = "hadoop-tokens" val SPARK_HADOOP_PREFIX = "spark.hadoop." diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopConfFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopConfFeatureStep.scala index 1192a20a584d..bb4ddf09d99b 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopConfFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopConfFeatureStep.scala @@ -38,6 +38,8 @@ private[spark] class HadoopConfFeatureStep( require(kubernetesConf.hadoopConfDir.isDefined, "Ensure that HADOOP_CONF_DIR is defined") logInfo("HADOOP_CONF_DIR defined. Mounting Hadoop specific files") val kubeTokenManager = kubernetesConf.getTokenManager + // TODO: For executors they should be taking from + // sparkConf move logic into a bootstrap val hadoopConfigFiles = kubeTokenManager.getHadoopConfFiles(kubernetesConf.hadoopConfDir.get) val keyPaths = hadoopConfigFiles.map { file => diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopGlobalFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopGlobalFeatureDriverStep.scala similarity index 67% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopGlobalFeatureStep.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopGlobalFeatureDriverStep.scala index 1a02f94d6aca..f830bb5141cf 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopGlobalFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopGlobalFeatureDriverStep.scala @@ -22,8 +22,8 @@ import io.fabric8.kubernetes.api.model.{ConfigMapBuilder, ContainerBuilder, HasM import org.apache.spark.deploy.k8s.{KubernetesConf, SparkPod} import org.apache.spark.deploy.k8s.Constants._ -import org.apache.spark.deploy.k8s.KubernetesRoleSpecificConf -import org.apache.spark.deploy.k8s.features.hadoopsteps.{HadoopConfigSpec, HadoopConfigurationStep} +import org.apache.spark.deploy.k8s.KubernetesDriverSpecificConf +import org.apache.spark.deploy.k8s.features.hadoopsteps.{HadoopBootstrapUtil, HadoopConfigSpec, HadoopConfigurationStep} import org.apache.spark.internal.Logging /** @@ -32,11 +32,12 @@ import org.apache.spark.internal.Logging * SparkPod and Kubernetes Resources using the additive method of the feature steps */ private[spark] class HadoopGlobalFeatureStep( - kubernetesConf: KubernetesConf[_ <: KubernetesRoleSpecificConf]) + kubernetesConf: KubernetesConf[KubernetesDriverSpecificConf]) extends KubernetesFeatureConfigStep with Logging { private val hadoopTestOrchestrator = kubernetesConf.getHadoopStepsOrchestrator - require(hadoopTestOrchestrator.isDefined, "Ensure that HADOOP_CONF_DIR is defined") + require(kubernetesConf.hadoopConfDir.isDefined && + hadoopTestOrchestrator.isDefined, "Ensure that HADOOP_CONF_DIR is defined") private val hadoopSteps = hadoopTestOrchestrator .map(hto => hto.getHadoopSteps(kubernetesConf.getTokenManager)) @@ -73,35 +74,36 @@ private[spark] class HadoopGlobalFeatureStep( secretItemKey <- currentHadoopSpec.dtSecretItemKey userName <- currentHadoopSpec.jobUserName } yield { - val kerberizedPod = new PodBuilder(hadoopBasedPod) - .editOrNewSpec() - .addNewVolume() - .withName(SPARK_APP_HADOOP_SECRET_VOLUME_NAME) - .withNewSecret() - .withSecretName(currentHadoopSpec.dtSecretName) - .endSecret() - .endVolume() - .endSpec() - .build() - val kerberizedContainer = new ContainerBuilder(hadoopBasedContainer) - .addNewVolumeMount() - .withName(SPARK_APP_HADOOP_SECRET_VOLUME_NAME) - .withMountPath(SPARK_APP_HADOOP_CREDENTIALS_BASE_DIR) - .endVolumeMount() - .addNewEnv() - .withName(ENV_HADOOP_TOKEN_FILE_LOCATION) - .withValue(s"$SPARK_APP_HADOOP_CREDENTIALS_BASE_DIR/$secretItemKey") - .endEnv() - .addNewEnv() - .withName(ENV_SPARK_USER) - .withValue(userName) - .endEnv() - .build() - SparkPod(kerberizedPod, kerberizedContainer) } - maybeKerberosModification.getOrElse(SparkPod(hadoopBasedPod, hadoopBasedContainer)) + HadoopBootstrapUtil.bootstrapKerberosPod( + currentHadoopSpec.dtSecretName, + secretItemKey, + userName, + SparkPod(hadoopBasedPod, hadoopBasedContainer)) + } + maybeKerberosModification.getOrElse( + HadoopBootstrapUtil.bootstrapSparkUserPod( + kubernetesConf.getTokenManager.getCurrentUser.getShortUserName, + SparkPod(hadoopBasedPod, hadoopBasedContainer))) + } + + override def getAdditionalPodSystemProperties(): Map[String, String] = { + val maybeKerberosConfValues = + for { + secretItemKey <- currentHadoopSpec.dtSecretItemKey + userName <- currentHadoopSpec.jobUserName + } yield { + Map(KERBEROS_KEYTAB_SECRET_NAME -> currentHadoopSpec.dtSecretName, + KERBEROS_KEYTAB_SECRET_KEY -> secretItemKey, + KERBEROS_SPARK_USER_NAME -> userName) + } + val resolvedConfValues = maybeKerberosConfValues.getOrElse( + Map(KERBEROS_SPARK_USER_NAME -> + kubernetesConf.getTokenManager.getCurrentUser.getShortUserName) + ) + Map(HADOOP_CONFIG_MAP_SPARK_CONF_NAME -> kubernetesConf.getHadoopConfigMapName, + HADOOP_CONF_DIR_LOC -> kubernetesConf.hadoopConfDir.get) ++ resolvedConfValues } - override def getAdditionalPodSystemProperties(): Map[String, String] = Map.empty override def getAdditionalKubernetesResources(): Seq[HasMetadata] = { val configMap = diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopConfSparkUserStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopSparkUserExecutorFeatureStep.scala similarity index 53% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopConfSparkUserStep.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopSparkUserExecutorFeatureStep.scala index bb3e19118b6c..d2c4c01d5ca3 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopConfSparkUserStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopSparkUserExecutorFeatureStep.scala @@ -14,30 +14,32 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.k8s.features.hadoopsteps +package org.apache.spark.deploy.k8s.features -import io.fabric8.kubernetes.api.model.EnvVarBuilder +import io.fabric8.kubernetes.api.model.HasMetadata -import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesRoleSpecificConf} +import org.apache.spark.deploy.k8s.{KubernetesConf, SparkPod} import org.apache.spark.deploy.k8s.Constants._ -import org.apache.spark.deploy.k8s.security.KubernetesHadoopDelegationTokenManager +import org.apache.spark.deploy.k8s.KubernetesExecutorSpecificConf +import org.apache.spark.deploy.k8s.features.hadoopsteps.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 HadoopConfSparkUserStep( - tokenManager: KubernetesHadoopDelegationTokenManager) - extends HadoopConfigurationStep { + private[spark] class HadoopSparkUserExecutorFeatureStep( + kubernetesConf: KubernetesConf[KubernetesExecutorSpecificConf]) + extends KubernetesFeatureConfigStep with Logging{ - override def configureHadoopSpec(hSpec: HadoopConfigSpec) : HadoopConfigSpec = { - val sparkUserEnv = - new EnvVarBuilder() - .withName(ENV_SPARK_USER) - .withValue(tokenManager.getShortUserName) - .build() - hSpec.copy( - containerEnvs = hSpec.containerEnvs :+ sparkUserEnv - ) - } + override def configurePod(pod: SparkPod): SparkPod = { + val sparkUserName = kubernetesConf.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/KerberosConfExecutorFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/KerberosConfExecutorFeatureStep.scala new file mode 100644 index 000000000000..945ff397791e --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/KerberosConfExecutorFeatureStep.scala @@ -0,0 +1,50 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.k8s.features + +import io.fabric8.kubernetes.api.model.HasMetadata + +import org.apache.spark.deploy.k8s.{KubernetesConf, SparkPod} +import org.apache.spark.deploy.k8s.Constants._ +import org.apache.spark.deploy.k8s.KubernetesExecutorSpecificConf +import org.apache.spark.deploy.k8s.features.hadoopsteps.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]) + extends KubernetesFeatureConfigStep with Logging{ + + override def configurePod(pod: SparkPod): SparkPod = { + val sparkConf = kubernetesConf.sparkConf + val dTSecretName = sparkConf.get(KERBEROS_KEYTAB_SECRET_NAME) + val dTDataItemKey = sparkConf.get(KERBEROS_KEYTAB_SECRET_KEY) + val sparkUserName = sparkConf.get(KERBEROS_SPARK_USER_NAME) + logInfo(s"Mounting HDFS DT from Secret $dTSecretName for Secure HDFS") + HadoopBootstrapUtil.bootstrapKerberosPod( + dTSecretName, + dTDataItemKey, + 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/hadoopsteps/HadoopBootstrapUtil.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopBootstrapUtil.scala new file mode 100644 index 000000000000..f31fe0abd904 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopBootstrapUtil.scala @@ -0,0 +1,86 @@ +/* + * 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.hadoopsteps + +import io.fabric8.kubernetes.api.model.ContainerBuilder +import io.fabric8.kubernetes.api.model.PodBuilder + +import org.apache.spark.deploy.k8s.Constants._ +import org.apache.spark.deploy.k8s.SparkPod + +private[spark] object HadoopBootstrapUtil { + + /** + * Mounting the DT secret for both the Driver and the executors + * + * @param dtSecretName Name of the secret that stores the Delegation Token + * @param dtSecretItemKey Name of the Item Key storing the Delegation Token + * @param userName Name of the SparkUser to set SPARK_USER + * @param pod Input pod to be appended to + * @return a modified SparkPod + */ + def bootstrapKerberosPod( + dtSecretName: String, + dtSecretItemKey: String, + userName: String, + pod: SparkPod) : SparkPod = { + val kerberizedPod = new PodBuilder(pod.pod) + .editOrNewSpec() + .addNewVolume() + .withName(SPARK_APP_HADOOP_SECRET_VOLUME_NAME) + .withNewSecret() + .withSecretName(dtSecretName) + .endSecret() + .endVolume() + .endSpec() + .build() + val kerberizedContainer = new ContainerBuilder(pod.container) + .addNewVolumeMount() + .withName(SPARK_APP_HADOOP_SECRET_VOLUME_NAME) + .withMountPath(SPARK_APP_HADOOP_CREDENTIALS_BASE_DIR) + .endVolumeMount() + .addNewEnv() + .withName(ENV_HADOOP_TOKEN_FILE_LOCATION) + .withValue(s"$SPARK_APP_HADOOP_CREDENTIALS_BASE_DIR/$dtSecretItemKey") + .endEnv() + .addNewEnv() + .withName(ENV_SPARK_USER) + .withValue(userName) + .endEnv() + .build() + SparkPod(kerberizedPod, kerberizedContainer) + } + + /** + * setting ENV_SPARK_USER when HADOOP_FILES are detected + * + * @param sparkUserName Name of the SPARK_USER + * @param pod Input pod to be appended to + * @return a modified SparkPod + */ + def bootstrapSparkUserPod( + sparkUserName: String, + pod: SparkPod) : SparkPod = { + val envModifiedContainer = new ContainerBuilder(pod.container) + .addNewEnv() + .withName(ENV_SPARK_USER) + .withValue(sparkUserName) + .endEnv() + .build() + SparkPod(pod.pod, envModifiedContainer) + } +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopStepsOrchestrator.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopStepsOrchestrator.scala index f235bff2b2d5..142c119a599f 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopStepsOrchestrator.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopStepsOrchestrator.scala @@ -72,18 +72,16 @@ private[spark] class HadoopStepsOrchestrator( secretName <- maybeExistingSecretName secretItemKey <- maybeExistingSecretItemKey } yield { - new HadoopKerberosSecretResolverStep(secretName, secretItemKey, kubeTokenManager) + Some(new HadoopKerberosSecretResolverStep(secretName, secretItemKey, kubeTokenManager)) } - maybeExistingSecretStep.getOrElse(new HadoopKerberosKeytabResolverStep( + maybeExistingSecretStep.getOrElse(Some(new HadoopKerberosKeytabResolverStep( conf, kubernetesResourceNamePrefix, maybePrincipal, maybeKeytab, maybeRenewerPrincipal, - kubeTokenManager)) - } else { - new HadoopConfSparkUserStep(kubeTokenManager) - } - Seq(hadoopConfMounterStep) :+ maybeKerberosStep + kubeTokenManager))) + } else None + Seq(hadoopConfMounterStep) ++ maybeKerberosStep.toSeq } } 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 769a0a5a6304..b43ec86103c8 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 @@ -17,7 +17,8 @@ package org.apache.spark.scheduler.cluster.k8s import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesExecutorSpecificConf, KubernetesRoleSpecificConf, SparkPod} -import org.apache.spark.deploy.k8s.features.{BasicExecutorFeatureStep, EnvSecretsFeatureStep, KubernetesFeatureConfigStep, LocalDirsFeatureStep, MountSecretsFeatureStep} +import org.apache.spark.deploy.k8s.Constants._ +import org.apache.spark.deploy.k8s.features._ private[spark] class KubernetesExecutorBuilder( provideBasicStep: (KubernetesConf[KubernetesExecutorSpecificConf]) => BasicExecutorFeatureStep = @@ -30,10 +31,28 @@ private[spark] class KubernetesExecutorBuilder( new EnvSecretsFeatureStep(_), provideLocalDirsStep: (KubernetesConf[_ <: KubernetesRoleSpecificConf]) => LocalDirsFeatureStep = - new LocalDirsFeatureStep(_)) { + new LocalDirsFeatureStep(_), + provideHadoopConfStep: ( + KubernetesConf[KubernetesExecutorSpecificConf] + => HadoopConfFeatureStep) = + new HadoopConfFeatureStep(_), + provideKerberosConfStep: ( + KubernetesConf[KubernetesExecutorSpecificConf] + => KerberosConfExecutorFeatureStep) = + new KerberosConfExecutorFeatureStep(_), + provideHadoopSparkUserStep: ( + KubernetesConf[KubernetesExecutorSpecificConf] + => HadoopSparkUserExecutorFeatureStep) = + new HadoopSparkUserExecutorFeatureStep(_)) { def buildFromFeatures( kubernetesConf: KubernetesConf[KubernetesExecutorSpecificConf]): SparkPod = { + val sparkConf = kubernetesConf.sparkConf + val maybeHadoopConfigMap = sparkConf.getOption(HADOOP_CONFIG_MAP_SPARK_CONF_NAME) + val maybeDTSecretName = sparkConf.getOption(KERBEROS_KEYTAB_SECRET_NAME) + val maybeDTDataItem = sparkConf.getOption(KERBEROS_KEYTAB_SECRET_KEY) + val maybeSparkUserName = sparkConf.getOption(KERBEROS_SPARK_USER_NAME) + val baseFeatures = Seq( provideBasicStep(kubernetesConf), provideLocalDirsStep(kubernetesConf)) @@ -44,10 +63,26 @@ private[spark] class KubernetesExecutorBuilder( val maybeProvideSecretsStep = if (kubernetesConf.roleSecretEnvNamesToKeyRefs.nonEmpty) { Some(provideEnvSecretsStep(kubernetesConf)) } else None + val maybeHadoopConfFeatureSteps = if (maybeHadoopConfigMap.isDefined) { + val maybeKerberosStep = + for { + _ <- maybeDTSecretName + _ <- maybeDTDataItem + } yield { + provideKerberosConfStep(kubernetesConf) + } + val maybeSparkUserStep = maybeSparkUserName.map {_ => + provideHadoopSparkUserStep(kubernetesConf)} + Seq(provideHadoopConfStep(kubernetesConf)) ++ + maybeKerberosStep.toSeq ++ + maybeSparkUserStep.toSeq + } else Seq.empty[KubernetesFeatureConfigStep] + val allFeatures: Seq[KubernetesFeatureConfigStep] = baseFeatures ++ maybeRoleSecretNamesStep.toSeq ++ - maybeProvideSecretsStep.toSeq + maybeProvideSecretsStep.toSeq ++ + maybeHadoopConfFeatureSteps var executorPod = SparkPod.initialPod() for (feature <- allFeatures) { From 000120f7b714bdabf87c77233c5b534a15b3904d Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Mon, 2 Jul 2018 07:40:23 +0300 Subject: [PATCH 03/25] refactored executor logic preparing for e2e testing --- .../apache/spark/deploy/k8s/Constants.scala | 4 +- .../spark/deploy/k8s/KubernetesConf.scala | 5 +- ...la => HadoopConfExecutorFeatureStep.scala} | 53 +++---------- .../HadoopGlobalFeatureDriverStep.scala | 13 +++- .../hadoopsteps/HadoopBootstrapUtil.scala | 55 +++++++++++++- .../submit/KubernetesClientApplication.scala | 9 ++- .../k8s/submit/KubernetesDriverBuilder.scala | 17 ++--- .../cluster/k8s/ExecutorPodsAllocator.scala | 4 +- .../k8s/KubernetesExecutorBuilder.scala | 14 ++-- .../deploy/k8s/KubernetesConfSuite.scala | 9 +-- .../submit/KubernetesDriverBuilderSuite.scala | 10 +-- .../k8s/ExecutorPodsAllocatorSuite.scala | 3 +- .../k8s/KubernetesExecutorBuilderSuite.scala | 76 ++++++++++++++++++- 13 files changed, 178 insertions(+), 94 deletions(-) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/{HadoopConfFeatureStep.scala => HadoopConfExecutorFeatureStep.scala} (53%) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Constants.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Constants.scala index a6b2d0794196..0817274c2263 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Constants.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Constants.scala @@ -112,6 +112,6 @@ private[spark] object Constants { val KERBEROS_REFRESH_LABEL_VALUE = "yes" // Hadoop credentials secrets for the Spark app. - private[spark] val SPARK_APP_HADOOP_CREDENTIALS_BASE_DIR = "/mnt/secrets/hadoop-credentials" - private[spark] val SPARK_APP_HADOOP_SECRET_VOLUME_NAME = "hadoop-secret" + val SPARK_APP_HADOOP_CREDENTIALS_BASE_DIR = "/mnt/secrets/hadoop-credentials" + val SPARK_APP_HADOOP_SECRET_VOLUME_NAME = "hadoop-secret" } 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 34f1979d9c9f..97b5333a0be2 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 @@ -195,8 +195,7 @@ private[spark] object KubernetesConf { sparkConf: SparkConf, executorId: String, appId: String, - driverPod: Pod, - hadoopConfDir: Option[String]): KubernetesConf[KubernetesExecutorSpecificConf] = { + driverPod: Pod): KubernetesConf[KubernetesExecutorSpecificConf] = { val executorCustomLabels = KubernetesUtils.parsePrefixedKeyValuePairs( sparkConf, KUBERNETES_EXECUTOR_LABEL_PREFIX) require( @@ -233,6 +232,6 @@ private[spark] object KubernetesConf { executorEnvSecrets, executorEnv, Seq.empty[String], - hadoopConfDir) + None) } } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopConfFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopConfExecutorFeatureStep.scala similarity index 53% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopConfFeatureStep.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopConfExecutorFeatureStep.scala index bb4ddf09d99b..b1c9948a6c65 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopConfFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopConfExecutorFeatureStep.scala @@ -16,12 +16,11 @@ */ package org.apache.spark.deploy.k8s.features -import scala.collection.JavaConverters._ - -import io.fabric8.kubernetes.api.model.{ContainerBuilder, HasMetadata, KeyToPathBuilder, PodBuilder} +import io.fabric8.kubernetes.api.model.HasMetadata import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesRoleSpecificConf, SparkPod} import org.apache.spark.deploy.k8s.Constants._ +import org.apache.spark.deploy.k8s.features.hadoopsteps.HadoopBootstrapUtil import org.apache.spark.internal.Logging /** @@ -30,49 +29,21 @@ import org.apache.spark.internal.Logging * pointed to the mounted file directory. This is run by both the driver * and executor, as they both require Hadoop config files. */ -private[spark] class HadoopConfFeatureStep( +private[spark] class HadoopConfExecutorFeatureStep( kubernetesConf: KubernetesConf[_ <: KubernetesRoleSpecificConf]) extends KubernetesFeatureConfigStep with Logging{ override def configurePod(pod: SparkPod): SparkPod = { - require(kubernetesConf.hadoopConfDir.isDefined, "Ensure that HADOOP_CONF_DIR is defined") + val maybeHadoopConfDir = kubernetesConf.sparkConf.getOption(HADOOP_CONF_DIR_LOC) + val maybeHadoopConfigMap = kubernetesConf.sparkConf.getOption(HADOOP_CONFIG_MAP_SPARK_CONF_NAME) + require(maybeHadoopConfDir.isDefined && maybeHadoopConfigMap.isDefined, + "Ensure that HADOOP_CONF_DIR is defined") logInfo("HADOOP_CONF_DIR defined. Mounting Hadoop specific files") - val kubeTokenManager = kubernetesConf.getTokenManager - // TODO: For executors they should be taking from - // sparkConf move logic into a bootstrap - val hadoopConfigFiles = - kubeTokenManager.getHadoopConfFiles(kubernetesConf.hadoopConfDir.get) - val keyPaths = hadoopConfigFiles.map { file => - val fileStringPath = file.toPath.getFileName.toString - new KeyToPathBuilder() - .withKey(fileStringPath) - .withPath(fileStringPath) - .build() } - - val hadoopSupportedPod = new PodBuilder(pod.pod) - .editSpec() - .addNewVolume() - .withName(HADOOP_FILE_VOLUME) - .withNewConfigMap() - .withName(kubernetesConf.getHadoopConfigMapName) - .withItems(keyPaths.asJava) - .endConfigMap() - .endVolume() - .endSpec() - .build() - - val hadoopSupportedContainer = new ContainerBuilder(pod.container) - .addNewVolumeMount() - .withName(HADOOP_FILE_VOLUME) - .withMountPath(HADOOP_CONF_DIR_PATH) - .endVolumeMount() - .addNewEnv() - .withName(ENV_HADOOP_CONF_DIR) - .withValue(HADOOP_CONF_DIR_PATH) - .endEnv() - .build() - - SparkPod(hadoopSupportedPod, hadoopSupportedContainer) + HadoopBootstrapUtil.bootstrapHadoopConfDir( + maybeHadoopConfDir.get, + maybeHadoopConfigMap.get, + kubernetesConf.getTokenManager, + pod) } override def getAdditionalPodSystemProperties(): Map[String, String] = Map.empty diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopGlobalFeatureDriverStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopGlobalFeatureDriverStep.scala index f830bb5141cf..ebf2b7b22bff 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopGlobalFeatureDriverStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopGlobalFeatureDriverStep.scala @@ -31,7 +31,7 @@ import org.apache.spark.internal.Logging * by the HadoopStepsOrchestrator. These steps are run to modify the * SparkPod and Kubernetes Resources using the additive method of the feature steps */ -private[spark] class HadoopGlobalFeatureStep( +private[spark] class HadoopGlobalFeatureDriverStep( kubernetesConf: KubernetesConf[KubernetesDriverSpecificConf]) extends KubernetesFeatureConfigStep with Logging { private val hadoopTestOrchestrator = @@ -69,6 +69,12 @@ private[spark] class HadoopGlobalFeatureStep( .addAllToVolumeMounts(currentHadoopSpec.containerVMs.asJava) .build() + val hadoopBasedSparkPod = HadoopBootstrapUtil.bootstrapHadoopConfDir( + kubernetesConf.hadoopConfDir.get, + kubernetesConf.getHadoopConfigMapName, + kubernetesConf.getTokenManager, + SparkPod(hadoopBasedPod, hadoopBasedContainer)) + val maybeKerberosModification = for { secretItemKey <- currentHadoopSpec.dtSecretItemKey @@ -78,12 +84,12 @@ private[spark] class HadoopGlobalFeatureStep( currentHadoopSpec.dtSecretName, secretItemKey, userName, - SparkPod(hadoopBasedPod, hadoopBasedContainer)) + hadoopBasedSparkPod) } maybeKerberosModification.getOrElse( HadoopBootstrapUtil.bootstrapSparkUserPod( kubernetesConf.getTokenManager.getCurrentUser.getShortUserName, - SparkPod(hadoopBasedPod, hadoopBasedContainer))) + hadoopBasedSparkPod)) } override def getAdditionalPodSystemProperties(): Map[String, String] = { @@ -104,7 +110,6 @@ private[spark] class HadoopGlobalFeatureStep( HADOOP_CONF_DIR_LOC -> kubernetesConf.hadoopConfDir.get) ++ resolvedConfValues } - override def getAdditionalKubernetesResources(): Seq[HasMetadata] = { val configMap = new ConfigMapBuilder() diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopBootstrapUtil.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopBootstrapUtil.scala index f31fe0abd904..76d2602b88b6 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopBootstrapUtil.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopBootstrapUtil.scala @@ -16,11 +16,13 @@ */ package org.apache.spark.deploy.k8s.features.hadoopsteps -import io.fabric8.kubernetes.api.model.ContainerBuilder -import io.fabric8.kubernetes.api.model.PodBuilder +import scala.collection.JavaConverters._ + +import io.fabric8.kubernetes.api.model.{ContainerBuilder, KeyToPathBuilder, PodBuilder} import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.deploy.k8s.SparkPod +import org.apache.spark.deploy.k8s.security.KubernetesHadoopDelegationTokenManager private[spark] object HadoopBootstrapUtil { @@ -83,4 +85,53 @@ private[spark] object HadoopBootstrapUtil { .build() SparkPod(pod.pod, envModifiedContainer) } + + /** + * bootstraping the container with ConfigMaps that store + * Hadoop conifiguration files + * + * @param hadoopConfDir location of HADOOP_CONF_DIR + * @param hadoopConfigMapName name of the configMap for HADOOP_CONF_DIR + * @param kubeTokenManager KubernetesHadoopDelegationTokenManager + * @param pod Input pod to be appended to + * @return a modified SparkPod + */ + def bootstrapHadoopConfDir( + hadoopConfDir: String, + hadoopConfigMapName: String, + kubeTokenManager: KubernetesHadoopDelegationTokenManager, + pod: SparkPod) : SparkPod = { + val hadoopConfigFiles = + kubeTokenManager.getHadoopConfFiles(hadoopConfDir) + val keyPaths = hadoopConfigFiles.map { file => + val fileStringPath = file.toPath.getFileName.toString + new KeyToPathBuilder() + .withKey(fileStringPath) + .withPath(fileStringPath) + .build() } + + val hadoopSupportedPod = new PodBuilder(pod.pod) + .editSpec() + .addNewVolume() + .withName(HADOOP_FILE_VOLUME) + .withNewConfigMap() + .withName(hadoopConfigMapName) + .withItems(keyPaths.asJava) + .endConfigMap() + .endVolume() + .endSpec() + .build() + + val hadoopSupportedContainer = new ContainerBuilder(pod.container) + .addNewVolumeMount() + .withName(HADOOP_FILE_VOLUME) + .withMountPath(HADOOP_CONF_DIR_PATH) + .endVolumeMount() + .addNewEnv() + .withName(ENV_HADOOP_CONF_DIR) + .withValue(HADOOP_CONF_DIR_PATH) + .endEnv() + .build() + SparkPod(hadoopSupportedPod, hadoopSupportedContainer) + } } 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 a8c4708392f3..75c46dbd78a5 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 @@ -109,7 +109,14 @@ private[spark] class Client( def run(): Unit = { val resolvedDriverSpec = builder.buildFromFeatures(kubernetesConf) val configMapName = s"$kubernetesResourceNamePrefix-driver-conf-map" - val configMap = buildConfigMap(configMapName, resolvedDriverSpec.systemProperties) + val isKerberosEnabled = kubernetesConf.sparkConf.get(KUBERNETES_KERBEROS_SUPPORT) + // HADOOP_SECURITY_AUTHENTICATION is defined as simple for the driver and executors as + // they need only the delegation token to access secure HDFS, no need to sign in to Kerberos + val maybeSimpleAuthentication = + if (isKerberosEnabled) Some((s"-D$HADOOP_SECURITY_AUTHENTICATION", "simple")) else None + val configMap = + buildConfigMap(configMapName, + resolvedDriverSpec.systemProperties ++ maybeSimpleAuthentication) // 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 val resolvedDriverContainer = new ContainerBuilder(resolvedDriverSpec.pod.container) 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 c9ec2cd8ddf7..3e8e56df05d2 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 @@ -45,14 +45,10 @@ private[spark] class KubernetesDriverBuilder( KubernetesConf[KubernetesDriverSpecificConf] => PythonDriverFeatureStep) = new PythonDriverFeatureStep(_), - provideHadoopConfStep: ( - KubernetesConf[KubernetesDriverSpecificConf] - => HadoopConfFeatureStep) = - new HadoopConfFeatureStep(_), provideHadoopGlobalStep: ( KubernetesConf[KubernetesDriverSpecificConf] - => HadoopGlobalFeatureStep) = - new HadoopGlobalFeatureStep(_)) { + => HadoopGlobalFeatureDriverStep) = + new HadoopGlobalFeatureDriverStep(_)) { def buildFromFeatures( kubernetesConf: KubernetesConf[KubernetesDriverSpecificConf]): KubernetesDriverSpec = { @@ -74,18 +70,15 @@ private[spark] class KubernetesDriverBuilder( case PythonMainAppResource(_) => providePythonStep(kubernetesConf)}.getOrElse(provideJavaStep(kubernetesConf)) - val maybeHadoopConfigSteps = + val maybeHadoopConfigStep = kubernetesConf.hadoopConfDir.map { _ => - Seq( - provideHadoopConfStep(kubernetesConf), - provideHadoopGlobalStep(kubernetesConf)) - }.getOrElse(Seq.empty[KubernetesFeatureConfigStep]) + provideHadoopGlobalStep(kubernetesConf)} val allFeatures: Seq[KubernetesFeatureConfigStep] = (baseFeatures :+ bindingsStep) ++ maybeRoleSecretNamesStep.toSeq ++ maybeProvideSecretsStep.toSeq ++ - maybeHadoopConfigSteps + maybeHadoopConfigStep.toSeq var spec = KubernetesDriverSpec.initialSpec(kubernetesConf.sparkConf.getAll.toMap) for (feature <- allFeatures) { diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala index f9690fe5bafc..5a143ad3600f 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala @@ -118,13 +118,11 @@ private[spark] class ExecutorPodsAllocator( logInfo(s"Going to request $numExecutorsToAllocate executors from Kubernetes.") for ( _ <- 0 until numExecutorsToAllocate) { val newExecutorId = EXECUTOR_ID_COUNTER.incrementAndGet() - // TODO: HADOOP_CONF_DIR val executorConf = KubernetesConf.createExecutorConf( conf, newExecutorId.toString, applicationId, - driverPod, - None) + driverPod) val executorPod = executorBuilder.buildFromFeatures(executorConf) val podWithAttachedContainer = new PodBuilder(executorPod.pod) .editOrNewSpec() 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 b43ec86103c8..4a4c7d25fa61 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 @@ -34,8 +34,8 @@ private[spark] class KubernetesExecutorBuilder( new LocalDirsFeatureStep(_), provideHadoopConfStep: ( KubernetesConf[KubernetesExecutorSpecificConf] - => HadoopConfFeatureStep) = - new HadoopConfFeatureStep(_), + => HadoopConfExecutorFeatureStep) = + new HadoopConfExecutorFeatureStep(_), provideKerberosConfStep: ( KubernetesConf[KubernetesExecutorSpecificConf] => KerberosConfExecutorFeatureStep) = @@ -63,7 +63,7 @@ private[spark] class KubernetesExecutorBuilder( val maybeProvideSecretsStep = if (kubernetesConf.roleSecretEnvNamesToKeyRefs.nonEmpty) { Some(provideEnvSecretsStep(kubernetesConf)) } else None - val maybeHadoopConfFeatureSteps = if (maybeHadoopConfigMap.isDefined) { + val maybeHadoopConfFeatureSteps = maybeHadoopConfigMap.map { _ => val maybeKerberosStep = for { _ <- maybeDTSecretName @@ -71,12 +71,10 @@ private[spark] class KubernetesExecutorBuilder( } yield { provideKerberosConfStep(kubernetesConf) } - val maybeSparkUserStep = maybeSparkUserName.map {_ => - provideHadoopSparkUserStep(kubernetesConf)} Seq(provideHadoopConfStep(kubernetesConf)) ++ - maybeKerberosStep.toSeq ++ - maybeSparkUserStep.toSeq - } else Seq.empty[KubernetesFeatureConfigStep] + maybeKerberosStep :+ maybeKerberosStep.getOrElse( + provideHadoopSparkUserStep(kubernetesConf)) + }.getOrElse(Seq.empty[KubernetesFeatureConfigStep]) val allFeatures: Seq[KubernetesFeatureConfigStep] = baseFeatures ++ 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 896957fa0e5e..793428270785 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 @@ -190,8 +190,7 @@ class KubernetesConfSuite extends SparkFunSuite { new SparkConf(false), EXECUTOR_ID, APP_ID, - DRIVER_POD, - None) + DRIVER_POD) assert(conf.roleSpecificConf.executorId === EXECUTOR_ID) assert(conf.roleSpecificConf.driverPod === DRIVER_POD) } @@ -202,8 +201,7 @@ class KubernetesConfSuite extends SparkFunSuite { .set(IMAGE_PULL_SECRETS, "my-secret-1,my-secret-2 "), EXECUTOR_ID, APP_ID, - DRIVER_POD, - None) + DRIVER_POD) assert(conf.imagePullSecrets() === Seq( new LocalObjectReferenceBuilder().withName("my-secret-1").build(), @@ -229,8 +227,7 @@ class KubernetesConfSuite extends SparkFunSuite { sparkConf, EXECUTOR_ID, APP_ID, - DRIVER_POD, - None) + DRIVER_POD) assert(conf.roleLabels === Map( SPARK_EXECUTOR_ID_LABEL -> EXECUTOR_ID, SPARK_APP_ID_LABEL -> APP_ID, 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 d04704f74152..4210f0078a2f 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 @@ -18,7 +18,7 @@ package org.apache.spark.deploy.k8s.submit import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesDriverSpec, KubernetesDriverSpecificConf} -import org.apache.spark.deploy.k8s.features.{BasicDriverFeatureStep, DriverKubernetesCredentialsFeatureStep, DriverServiceFeatureStep, EnvSecretsFeatureStep, HadoopConfFeatureStep, HadoopGlobalFeatureStep, KubernetesFeaturesTestUtils, LocalDirsFeatureStep, MountSecretsFeatureStep} +import org.apache.spark.deploy.k8s.features.{BasicDriverFeatureStep, DriverKubernetesCredentialsFeatureStep, DriverServiceFeatureStep, EnvSecretsFeatureStep, HadoopGlobalFeatureDriverStep, KubernetesFeaturesTestUtils, LocalDirsFeatureStep, MountSecretsFeatureStep} import org.apache.spark.deploy.k8s.features.bindings.{JavaDriverFeatureStep, PythonDriverFeatureStep} class KubernetesDriverBuilderSuite extends SparkFunSuite { @@ -31,7 +31,6 @@ class KubernetesDriverBuilderSuite extends SparkFunSuite { private val JAVA_STEP_TYPE = "java-bindings" private val PYSPARK_STEP_TYPE = "pyspark-bindings" private val ENV_SECRETS_STEP_TYPE = "env-secrets" - private val HADOOP_CONF_STEP_TYPE = "hadoop-conf" private val HADOOP_GLOBAL_STEP_TYPE = "hadoop-global" private val basicFeatureStep = KubernetesFeaturesTestUtils.getMockConfigStepForStepType( @@ -58,11 +57,8 @@ class KubernetesDriverBuilderSuite extends SparkFunSuite { private val envSecretsStep = KubernetesFeaturesTestUtils.getMockConfigStepForStepType( ENV_SECRETS_STEP_TYPE, classOf[EnvSecretsFeatureStep]) - private val hadoopConfStep = KubernetesFeaturesTestUtils.getMockConfigStepForStepType( - HADOOP_CONF_STEP_TYPE, classOf[HadoopConfFeatureStep]) - private val hadoopGlobalStep = KubernetesFeaturesTestUtils.getMockConfigStepForStepType( - HADOOP_GLOBAL_STEP_TYPE, classOf[HadoopGlobalFeatureStep]) + HADOOP_GLOBAL_STEP_TYPE, classOf[HadoopGlobalFeatureDriverStep]) private val builderUnderTest: KubernetesDriverBuilder = @@ -75,7 +71,6 @@ class KubernetesDriverBuilderSuite extends SparkFunSuite { _ => localDirsStep, _ => javaStep, _ => pythonStep, - _ => hadoopConfStep, _ => hadoopGlobalStep) test("Apply fundamental steps all the time.") { @@ -208,7 +203,6 @@ class KubernetesDriverBuilderSuite extends SparkFunSuite { SERVICE_STEP_TYPE, LOCAL_DIRS_STEP_TYPE, JAVA_STEP_TYPE, - HADOOP_CONF_STEP_TYPE, HADOOP_GLOBAL_STEP_TYPE) } 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 e835535c796a..6c2c1a849d91 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 @@ -167,8 +167,7 @@ class ExecutorPodsAllocatorSuite extends SparkFunSuite with BeforeAndAfter { conf, executorSpecificConf.executorId, TEST_SPARK_APP_ID, - driverPod, - None) + driverPod) 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 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 204e90552164..095f1a0602e0 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 @@ -20,13 +20,18 @@ import io.fabric8.kubernetes.api.model.PodBuilder import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesExecutorSpecificConf, SparkPod} -import org.apache.spark.deploy.k8s.features.{BasicExecutorFeatureStep, EnvSecretsFeatureStep, KubernetesFeaturesTestUtils, LocalDirsFeatureStep, MountSecretsFeatureStep} +import org.apache.spark.deploy.k8s.Config._ +import org.apache.spark.deploy.k8s.Constants._ +import org.apache.spark.deploy.k8s.features._ class KubernetesExecutorBuilderSuite extends SparkFunSuite { private val BASIC_STEP_TYPE = "basic" private val SECRETS_STEP_TYPE = "mount-secrets" private val ENV_SECRETS_STEP_TYPE = "env-secrets" private val LOCAL_DIRS_STEP_TYPE = "local-dirs" + private val HADOOP_CONF_STEP_TYPE = "hadoop-conf-step" + private val HADOOP_SPARK_USER_STEP_TYPE = "hadoop-spark-user" + private val KERBEROS_CONF_STEP_TYPE = "kerberos-step" private val basicFeatureStep = KubernetesFeaturesTestUtils.getMockConfigStepForStepType( BASIC_STEP_TYPE, classOf[BasicExecutorFeatureStep]) @@ -36,12 +41,21 @@ class KubernetesExecutorBuilderSuite extends SparkFunSuite { ENV_SECRETS_STEP_TYPE, classOf[EnvSecretsFeatureStep]) private val localDirsStep = KubernetesFeaturesTestUtils.getMockConfigStepForStepType( LOCAL_DIRS_STEP_TYPE, classOf[LocalDirsFeatureStep]) + private val hadoopConfStep = KubernetesFeaturesTestUtils.getMockConfigStepForStepType( + HADOOP_CONF_STEP_TYPE, classOf[HadoopConfExecutorFeatureStep]) + private val hadoopSparkUser = KubernetesFeaturesTestUtils.getMockConfigStepForStepType( + HADOOP_SPARK_USER_STEP_TYPE, classOf[HadoopSparkUserExecutorFeatureStep]) + private val kerberosConf = KubernetesFeaturesTestUtils.getMockConfigStepForStepType( + KERBEROS_CONF_STEP_TYPE, classOf[KerberosConfExecutorFeatureStep]) private val builderUnderTest = new KubernetesExecutorBuilder( _ => basicFeatureStep, _ => mountSecretsStep, _ => envSecretsStep, - _ => localDirsStep) + _ => localDirsStep, + _ => hadoopConfStep, + _ => kerberosConf, + _ => hadoopSparkUser) test("Basic steps are consistently applied.") { val conf = KubernetesConf( @@ -83,6 +97,64 @@ class KubernetesExecutorBuilderSuite extends SparkFunSuite { ENV_SECRETS_STEP_TYPE) } + 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) + .set(KUBERNETES_KERBEROS_SUPPORT, true) + .set(HADOOP_CONFIG_MAP_SPARK_CONF_NAME, "hadoop-conf-map-loc") + .set(HADOOP_CONF_DIR_LOC, "hadoop-conf-dir-loc") + .set(KERBEROS_SPARK_USER_NAME, "spark-user") + .set(HADOOPFS_PROVIDER, "true"), + KubernetesExecutorSpecificConf( + "executor-id", new PodBuilder().build()), + "prefix", + "appId", + Map.empty, + Map.empty, + Map.empty, + Map.empty, + Map.empty, + Seq.empty[String], + None) + validateStepTypesApplied( + builderUnderTest.buildFromFeatures(conf), + BASIC_STEP_TYPE, + LOCAL_DIRS_STEP_TYPE, + HADOOP_CONF_STEP_TYPE, + HADOOP_SPARK_USER_STEP_TYPE) + } + + test("Apply kerberos step if DT secrets created") { + val conf = KubernetesConf( + new SparkConf(false) + .set(KUBERNETES_KERBEROS_SUPPORT, true) + .set(HADOOP_CONFIG_MAP_SPARK_CONF_NAME, "hadoop-conf-map-loc") + .set(HADOOP_CONF_DIR_LOC, "hadoop-conf-dir-loc") + .set(KERBEROS_SPARK_USER_NAME, "spark-user") + .set(KERBEROS_KEYTAB_SECRET_NAME, "dt-secret") + .set(KERBEROS_KEYTAB_SECRET_KEY, "dt-key"), + KubernetesExecutorSpecificConf( + "executor-id", new PodBuilder().build()), + "prefix", + "appId", + Map.empty, + Map.empty, + Map.empty, + Map.empty, + Map.empty, + Seq.empty[String], + None) + validateStepTypesApplied( + builderUnderTest.buildFromFeatures(conf), + BASIC_STEP_TYPE, + LOCAL_DIRS_STEP_TYPE, + HADOOP_CONF_STEP_TYPE, + KERBEROS_CONF_STEP_TYPE) + } + private def validateStepTypesApplied(resolvedPod: SparkPod, stepTypes: String*): Unit = { assert(resolvedPod.pod.getMetadata.getLabels.size === stepTypes.size) stepTypes.foreach { stepType => From 13b3adc5ffb55fbfd6572089b1f54e8bca393494 Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Sat, 7 Jul 2018 12:14:05 +0200 Subject: [PATCH 04/25] resolved initial comments --- .../src/main/scala/org/apache/spark/deploy/k8s/Config.scala | 6 ------ .../scala/org/apache/spark/deploy/k8s/KubernetesConf.scala | 3 ++- .../spark/deploy/k8s/features/OptionRequirements.scala | 4 ---- .../k8s/features/hadoopsteps/HadoopStepsOrchestrator.scala | 4 ++-- .../deploy/k8s/submit/KubernetesClientApplication.scala | 2 +- .../scheduler/cluster/k8s/KubernetesExecutorBuilder.scala | 1 - .../cluster/k8s/KubernetesExecutorBuilderSuite.scala | 2 -- .../docker/src/main/dockerfiles/spark/entrypoint.sh | 4 ++++ 8 files changed, 9 insertions(+), 17 deletions(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala index 900e0d650d30..7c9f21f34c21 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 @@ -211,12 +211,6 @@ private[spark] object Config extends Logging { "Ensure that major Python version is either Python2 or Python3") .createWithDefault("2") - val KUBERNETES_KERBEROS_SUPPORT = - ConfigBuilder("spark.kubernetes.kerberos.enabled") - .doc("Specify whether your job is a job that will require a Delegation Token to access HDFS") - .booleanConf - .createWithDefault(false) - val KUBERNETES_KERBEROS_KEYTAB = ConfigBuilder("spark.kubernetes.kerberos.keytab") .doc("Specify the location of keytab " + 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 97b5333a0be2..bc3e14a54d9c 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 @@ -71,7 +71,8 @@ private[spark] case class KubernetesConf[T <: KubernetesRoleSpecificConf]( sparkConf, appResourceNamePrefix, hConf, - getHadoopConfigMapName)} + getHadoopConfigMapName, + getTokenManager.isSecurityEnabled)} def getTokenManager : KubernetesHadoopDelegationTokenManager = new KubernetesHadoopDelegationTokenManager diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/OptionRequirements.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/OptionRequirements.scala index c08cfe4e0783..3766384e0ab5 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/OptionRequirements.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/OptionRequirements.scala @@ -33,8 +33,4 @@ private[spark] object OptionRequirements { require(opt2.isDefined, errMessageWhenSecondIsMissing) } } - - def requireNandDefined(opt1: Option[_], opt2: Option[_], errMessage: String): Unit = { - opt1.foreach { _ => require(opt2.isEmpty, errMessage) } - } } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopStepsOrchestrator.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopStepsOrchestrator.scala index 142c119a599f..7f1323140076 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopStepsOrchestrator.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopStepsOrchestrator.scala @@ -28,9 +28,9 @@ private[spark] class HadoopStepsOrchestrator( conf: SparkConf, kubernetesResourceNamePrefix: String, hadoopConfDir: String, - hadoopConfigMapName: String) extends Logging { + hadoopConfigMapName: String, + isKerberosEnabled: Boolean) extends Logging { - private val isKerberosEnabled = conf.get(KUBERNETES_KERBEROS_SUPPORT) private val maybePrincipal = conf.get(KUBERNETES_KERBEROS_PRINCIPAL) private val maybeKeytab = conf.get(KUBERNETES_KERBEROS_KEYTAB) .map(k => new File(k)) 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 75c46dbd78a5..505f51838323 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 @@ -109,7 +109,7 @@ private[spark] class Client( def run(): Unit = { val resolvedDriverSpec = builder.buildFromFeatures(kubernetesConf) val configMapName = s"$kubernetesResourceNamePrefix-driver-conf-map" - val isKerberosEnabled = kubernetesConf.sparkConf.get(KUBERNETES_KERBEROS_SUPPORT) + val isKerberosEnabled = kubernetesConf.getTokenManager.isSecurityEnabled // HADOOP_SECURITY_AUTHENTICATION is defined as simple for the driver and executors as // they need only the delegation token to access secure HDFS, no need to sign in to Kerberos val maybeSimpleAuthentication = 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 4a4c7d25fa61..e267d135ce09 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 @@ -51,7 +51,6 @@ private[spark] class KubernetesExecutorBuilder( val maybeHadoopConfigMap = sparkConf.getOption(HADOOP_CONFIG_MAP_SPARK_CONF_NAME) val maybeDTSecretName = sparkConf.getOption(KERBEROS_KEYTAB_SECRET_NAME) val maybeDTDataItem = sparkConf.getOption(KERBEROS_KEYTAB_SECRET_KEY) - val maybeSparkUserName = sparkConf.getOption(KERBEROS_SPARK_USER_NAME) val baseFeatures = Seq( provideBasicStep(kubernetesConf), 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 095f1a0602e0..73f2cc4d1584 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 @@ -103,7 +103,6 @@ class KubernetesExecutorBuilderSuite extends SparkFunSuite { val HADOOPFS_PROVIDER = s"$HADOOP_CREDS_PREFIX.hadoopfs.enabled" val conf = KubernetesConf( new SparkConf(false) - .set(KUBERNETES_KERBEROS_SUPPORT, true) .set(HADOOP_CONFIG_MAP_SPARK_CONF_NAME, "hadoop-conf-map-loc") .set(HADOOP_CONF_DIR_LOC, "hadoop-conf-dir-loc") .set(KERBEROS_SPARK_USER_NAME, "spark-user") @@ -130,7 +129,6 @@ class KubernetesExecutorBuilderSuite extends SparkFunSuite { test("Apply kerberos step if DT secrets created") { val conf = KubernetesConf( new SparkConf(false) - .set(KUBERNETES_KERBEROS_SUPPORT, true) .set(HADOOP_CONFIG_MAP_SPARK_CONF_NAME, "hadoop-conf-map-loc") .set(HADOOP_CONF_DIR_LOC, "hadoop-conf-dir-loc") .set(KERBEROS_SPARK_USER_NAME, "spark-user") diff --git a/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/entrypoint.sh b/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/entrypoint.sh index 2f4e115e84ec..12218d4efab5 100755 --- a/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/entrypoint.sh +++ b/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/entrypoint.sh @@ -81,6 +81,10 @@ elif [ "$PYSPARK_MAJOR_PYTHON_VERSION" == "3" ]; then export PYSPARK_DRIVER_PYTHON="python3" fi +if ! [ -z ${HADOOP_CONF_DIR+x} ]; then + SPARK_CLASSPATH="$HADOOP_CONF_DIR:$SPARK_CLASSPATH"; +fi + case "$SPARK_K8S_CMD" in driver) CMD=( From c30ad8c4be1d42e7da4992570a656099c073d745 Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Tue, 7 Aug 2018 10:34:01 -0400 Subject: [PATCH 05/25] launching driver with kerberos authentication instead of simple --- .../org/apache/spark/deploy/SparkSubmit.scala | 10 +++- .../org/apache/spark/examples/HdfsTest.scala | 2 + .../org/apache/spark/deploy/k8s/Config.scala | 15 +++++ .../apache/spark/deploy/k8s/Constants.scala | 4 +- .../spark/deploy/k8s/KubernetesConf.scala | 2 + .../HadoopGlobalFeatureDriverStep.scala | 12 +++- .../KerberosConfExecutorFeatureStep.scala | 13 ++++- .../hadoopsteps/HadoopBootstrapUtil.scala | 56 +++++++++++++++++-- .../hadoopsteps/HadoopConfigMounterStep.scala | 31 ---------- .../submit/KubernetesClientApplication.scala | 16 +++--- .../src/main/dockerfiles/spark/Dockerfile | 2 +- 11 files changed, 115 insertions(+), 48 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index ea498d6d2311..292a7092deb2 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -164,7 +164,15 @@ private[spark] class SparkSubmit extends Logging { } } } else { - runMain(childArgs, childClasspath, sparkConf, childMainClass, args.verbose) + if (sparkConf.getOption("spark.kubernetes.kerberos.proxyUser").isDefined) { + // scalastyle:off println + printStream.println("Running as proxy user in k8s cluster mode...") + // scalastyle:on println + SparkHadoopUtil.get.runAsSparkUser( + () => runMain(childArgs, childClasspath, sparkConf, childMainClass, args.verbose)) + } else { + runMain(childArgs, childClasspath, sparkConf, childMainClass, args.verbose) + } } } diff --git a/examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala b/examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala index e1f985ece8c0..c1d610246639 100644 --- a/examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala @@ -41,6 +41,8 @@ object HdfsTest { val end = System.currentTimeMillis() println(s"Iteration $iter took ${end-start} ms") } + println(s"File contents: ${file.map(s => s.toString).collect().mkString(",")}") + println(s"Returned length(s) of: ${file.map(s => s.length).collect().mkString(",")}") spark.stop() } } 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 d989c61c822b..0677ff85afc8 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 @@ -212,6 +212,21 @@ private[spark] object Config extends Logging { "Ensure that major Python version is either Python2 or Python3") .createWithDefault("2") + val KUBERNETES_KERBEROS_PROXY_USER = + ConfigBuilder("spark.kubernetes.kerberos.proxyUser") + .doc("Specify the proxy user " + + "for HadoopUGI login for the Driver + Executors") + .internal() + .stringConf + .createWithDefault("false") + + val KUBERNETES_KERBEROS_KRB5_FILE = + ConfigBuilder("spark.kubernetes.kerberos.krb5location") + .doc("Specify the location of the krb5 file " + + "to be mounted on the driver and executors for Secure HDFS") + .stringConf + .createOptional + val KUBERNETES_KERBEROS_KEYTAB = ConfigBuilder("spark.kubernetes.kerberos.keytab") .doc("Specify the location of keytab " + diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Constants.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Constants.scala index a26da63adacb..08690cd19ce1 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Constants.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Constants.scala @@ -81,7 +81,9 @@ private[spark] object Constants { // Hadoop Configuration val HADOOP_FILE_VOLUME = "hadoop-properties" - val HADOOP_CONF_DIR_PATH = "/etc/hadoop/conf" + val KRB_FILE_VOLUME = "krb5-file" + val HADOOP_CONF_DIR_PATH = "/opt/hadoop/conf" + val KRB_FILE_DIR_PATH = "/etc" val ENV_HADOOP_CONF_DIR = "HADOOP_CONF_DIR" val HADOOP_CONF_DIR_LOC = "spark.kubernetes.hadoop.conf.dir" val HADOOP_CONFIG_MAP_SPARK_CONF_NAME = 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 29504578ce3f..2311d06b069e 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 @@ -67,6 +67,8 @@ private[spark] case class KubernetesConf[T <: KubernetesRoleSpecificConf]( def getHadoopConfigMapName: String = s"$appResourceNamePrefix-hadoop-config" + def getKRBConfigMapName: String = s"$appResourceNamePrefix-krb5-file" + def getHadoopStepsOrchestrator : Option[HadoopStepsOrchestrator] = hadoopConfDir.map { hConf => new HadoopStepsOrchestrator( sparkConf, diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopGlobalFeatureDriverStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopGlobalFeatureDriverStep.scala index ebf2b7b22bff..a96ce12316e9 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopGlobalFeatureDriverStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopGlobalFeatureDriverStep.scala @@ -21,6 +21,7 @@ import scala.collection.JavaConverters._ import io.fabric8.kubernetes.api.model.{ConfigMapBuilder, ContainerBuilder, HasMetadata, PodBuilder} import org.apache.spark.deploy.k8s.{KubernetesConf, SparkPod} +import org.apache.spark.deploy.k8s.Config.{KUBERNETES_KERBEROS_KRB5_FILE, KUBERNETES_KERBEROS_PROXY_USER} import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.deploy.k8s.KubernetesDriverSpecificConf import org.apache.spark.deploy.k8s.features.hadoopsteps.{HadoopBootstrapUtil, HadoopConfigSpec, HadoopConfigurationStep} @@ -79,11 +80,14 @@ private[spark] class HadoopGlobalFeatureDriverStep( for { secretItemKey <- currentHadoopSpec.dtSecretItemKey userName <- currentHadoopSpec.jobUserName + krb5fileLocation <- kubernetesConf.get(KUBERNETES_KERBEROS_KRB5_FILE) } yield { HadoopBootstrapUtil.bootstrapKerberosPod( currentHadoopSpec.dtSecretName, secretItemKey, userName, + krb5fileLocation, + kubernetesConf.getKRBConfigMapName, hadoopBasedSparkPod) } maybeKerberosModification.getOrElse( @@ -111,6 +115,10 @@ private[spark] class HadoopGlobalFeatureDriverStep( } override def getAdditionalKubernetesResources(): Seq[HasMetadata] = { + val krb5ConfigMap = kubernetesConf.get(KUBERNETES_KERBEROS_KRB5_FILE) + .map(fileLocation => HadoopBootstrapUtil.buildkrb5ConfigMap( + kubernetesConf.getKRBConfigMapName, + fileLocation)) val configMap = new ConfigMapBuilder() .withNewMetadata() @@ -118,6 +126,8 @@ private[spark] class HadoopGlobalFeatureDriverStep( .endMetadata() .addToData(currentHadoopSpec.configMapProperties.asJava) .build() - Seq(configMap) ++ currentHadoopSpec.dtSecret.toSeq + Seq(configMap) ++ + krb5ConfigMap.toSeq ++ + currentHadoopSpec.dtSecret.toSeq } } 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 945ff397791e..c5c5745eea60 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 @@ -18,7 +18,9 @@ package org.apache.spark.deploy.k8s.features import io.fabric8.kubernetes.api.model.HasMetadata +import org.apache.spark.SparkException import org.apache.spark.deploy.k8s.{KubernetesConf, SparkPod} +import org.apache.spark.deploy.k8s.Config.KUBERNETES_KERBEROS_KRB5_FILE import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.deploy.k8s.KubernetesExecutorSpecificConf import org.apache.spark.deploy.k8s.features.hadoopsteps.HadoopBootstrapUtil @@ -35,16 +37,25 @@ private[spark] class KerberosConfExecutorFeatureStep( val sparkConf = kubernetesConf.sparkConf val dTSecretName = sparkConf.get(KERBEROS_KEYTAB_SECRET_NAME) val dTDataItemKey = sparkConf.get(KERBEROS_KEYTAB_SECRET_KEY) + val krb5Location = sparkConf.get(KUBERNETES_KERBEROS_KRB5_FILE) + .getOrElse(throw new SparkException("Must specify krb5 file location")) val sparkUserName = sparkConf.get(KERBEROS_SPARK_USER_NAME) logInfo(s"Mounting HDFS DT from Secret $dTSecretName for Secure HDFS") HadoopBootstrapUtil.bootstrapKerberosPod( dTSecretName, dTDataItemKey, sparkUserName, + krb5Location, + kubernetesConf.getKRBConfigMapName, pod) } override def getAdditionalPodSystemProperties(): Map[String, String] = Map.empty - override def getAdditionalKubernetesResources(): Seq[HasMetadata] = Seq.empty + override def getAdditionalKubernetesResources(): Seq[HasMetadata] = { + kubernetesConf.get(KUBERNETES_KERBEROS_KRB5_FILE) + .map(fileLocation => HadoopBootstrapUtil.buildkrb5ConfigMap( + kubernetesConf.getKRBConfigMapName, + fileLocation)).toSeq + } } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopBootstrapUtil.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopBootstrapUtil.scala index 76d2602b88b6..5281f9fa0f01 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopBootstrapUtil.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopBootstrapUtil.scala @@ -16,9 +16,13 @@ */ package org.apache.spark.deploy.k8s.features.hadoopsteps +import java.io.File + import scala.collection.JavaConverters._ -import io.fabric8.kubernetes.api.model.{ContainerBuilder, KeyToPathBuilder, PodBuilder} +import com.google.common.base.Charsets +import com.google.common.io.Files +import io.fabric8.kubernetes.api.model.{ConfigMap, ConfigMapBuilder, ContainerBuilder, KeyToPathBuilder, PodBuilder} import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.deploy.k8s.SparkPod @@ -32,6 +36,8 @@ private[spark] object HadoopBootstrapUtil { * @param dtSecretName Name of the secret that stores the Delegation Token * @param dtSecretItemKey Name of the Item Key storing the Delegation Token * @param userName Name of the SparkUser to set SPARK_USER + * @param fileLocation Location of the krb5 file + * @param krb5ConfName Name of the ConfigMap for Krb5 * @param pod Input pod to be appended to * @return a modified SparkPod */ @@ -39,7 +45,11 @@ private[spark] object HadoopBootstrapUtil { dtSecretName: String, dtSecretItemKey: String, userName: String, + fileLocation: String, + krb5ConfName: String, pod: SparkPod) : SparkPod = { + val krb5File = new File(fileLocation) + val fileStringPath = krb5File.toPath.getFileName.toString val kerberizedPod = new PodBuilder(pod.pod) .editOrNewSpec() .addNewVolume() @@ -48,6 +58,16 @@ private[spark] object HadoopBootstrapUtil { .withSecretName(dtSecretName) .endSecret() .endVolume() + .addNewVolume() + .withName(KRB_FILE_VOLUME) + .withNewConfigMap() + .withName(krb5ConfName) + .withItems(new KeyToPathBuilder() + .withKey(fileStringPath) + .withPath(fileStringPath) + .build()) + .endConfigMap() + .endVolume() .endSpec() .build() val kerberizedContainer = new ContainerBuilder(pod.container) @@ -55,13 +75,21 @@ private[spark] object HadoopBootstrapUtil { .withName(SPARK_APP_HADOOP_SECRET_VOLUME_NAME) .withMountPath(SPARK_APP_HADOOP_CREDENTIALS_BASE_DIR) .endVolumeMount() + .addNewVolumeMount() + .withName(KRB_FILE_VOLUME) + .withMountPath(KRB_FILE_DIR_PATH) + .endVolumeMount() .addNewEnv() .withName(ENV_HADOOP_TOKEN_FILE_LOCATION) .withValue(s"$SPARK_APP_HADOOP_CREDENTIALS_BASE_DIR/$dtSecretItemKey") .endEnv() + // TODO (ifilonenko): This has the correct user as ` userName` however + // since the user to which the keytab has access to might not be on the k8s + // nodes, this atm leaves us with the option to use `root`. Next step is to + // support customization of the UNIX username. .addNewEnv() .withName(ENV_SPARK_USER) - .withValue(userName) + .withValue("root") .endEnv() .build() SparkPod(kerberizedPod, kerberizedContainer) @@ -87,8 +115,8 @@ private[spark] object HadoopBootstrapUtil { } /** - * bootstraping the container with ConfigMaps that store - * Hadoop conifiguration files + * Bootstraping the container with ConfigMaps that store + * Hadoop configuration files * * @param hadoopConfDir location of HADOOP_CONF_DIR * @param hadoopConfigMapName name of the configMap for HADOOP_CONF_DIR @@ -134,4 +162,24 @@ private[spark] object HadoopBootstrapUtil { .build() SparkPod(hadoopSupportedPod, hadoopSupportedContainer) } + /** + * bootstraping the container with ConfigMaps that store + * Hadoop configuration files + * + * @param configMapName name of configMap for krb5 + * @param fileLocation location of krb5 file + * @return a ConfigMap + */ + def buildkrb5ConfigMap( + configMapName: String, + fileLocation: String) : ConfigMap = { + val file = new File(fileLocation) + new ConfigMapBuilder() + .withNewMetadata() + .withName(configMapName) + .endMetadata() + .addToData( + Map(file.toPath.getFileName.toString -> Files.toString(file, Charsets.UTF_8)).asJava) + .build() + } } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopConfigMounterStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopConfigMounterStep.scala index 83f673750eed..86d384204ebc 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopConfigMounterStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopConfigMounterStep.scala @@ -37,38 +37,7 @@ private[spark] class HadoopConfigMounterStep( override def configureHadoopSpec(hSpec: HadoopConfigSpec) : HadoopConfigSpec = { logInfo("HADOOP_CONF_DIR defined. Mounting Hadoop specific files") - val keyPaths = hadoopConfigFiles.map { file => - val fileStringPath = file.toPath.getFileName.toString - new KeyToPathBuilder() - .withKey(fileStringPath) - .withPath(fileStringPath) - .build() } - - val hadoopFileVolume = - new VolumeBuilder() - .withName(HADOOP_FILE_VOLUME) - .withNewConfigMap() - .withName(hadoopConfConfigMapName) - .withItems(keyPaths.asJava) - .endConfigMap() - .build() - - val hadoopEnvVals = - new EnvVarBuilder() - .withName(ENV_HADOOP_CONF_DIR) - .withValue(HADOOP_CONF_DIR_PATH) - .build() - - val hadoopVolumeMount = - new VolumeMountBuilder() - .withName(HADOOP_FILE_VOLUME) - .withMountPath(HADOOP_CONF_DIR_PATH) - .build() - hSpec.copy( - podVolumes = hSpec.podVolumes :+ hadoopFileVolume, - containerEnvs = hSpec.containerEnvs :+ hadoopEnvVals, - containerVMs = hSpec.containerVMs :+ hadoopVolumeMount, configMapProperties = hSpec.configMapProperties ++ hadoopConfigFiles.map(file => (file.toPath.getFileName.toString, Files.toString(file, Charsets.UTF_8))).toMap) 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 21ab55eb1897..8a1553ef23fa 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 @@ -22,6 +22,7 @@ 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 @@ -109,14 +110,8 @@ private[spark] class Client( def run(): Unit = { val resolvedDriverSpec = builder.buildFromFeatures(kubernetesConf) val configMapName = s"$kubernetesResourceNamePrefix-driver-conf-map" - val isKerberosEnabled = kubernetesConf.getTokenManager.isSecurityEnabled - // HADOOP_SECURITY_AUTHENTICATION is defined as simple for the driver and executors as - // they need only the delegation token to access secure HDFS, no need to sign in to Kerberos - val maybeSimpleAuthentication = - if (isKerberosEnabled) Some((s"-D$HADOOP_SECURITY_AUTHENTICATION", "simple")) else None val configMap = - buildConfigMap(configMapName, - resolvedDriverSpec.systemProperties ++ maybeSimpleAuthentication) + 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 val resolvedDriverContainer = new ContainerBuilder(resolvedDriverSpec.pod.container) @@ -204,7 +199,7 @@ private[spark] class Client( /** * Main class and entry point of application submission in KUBERNETES mode. */ -private[spark] class KubernetesClientApplication extends SparkApplication { +private[spark] class KubernetesClientApplication extends SparkApplication with Logging { override def start(args: Array[String], conf: SparkConf): Unit = { val parsedArguments = ClientArguments.fromCommandLineArgs(args) @@ -224,6 +219,11 @@ private[spark] class KubernetesClientApplication extends SparkApplication { s"$appName-$launchTime".toLowerCase.replaceAll("\\.", "-") } sparkConf.set(KUBERNETES_PYSPARK_PY_FILES, clientArguments.maybePyFiles.getOrElse("")) + // Run driver as proxy user for Kerberos login by the HadoopUGI + if (UserGroupInformation.isSecurityEnabled) { + logInfo("Because Kerberos is enabled we should run driver as proxy user") + sparkConf.set(KUBERNETES_KERBEROS_PROXY_USER, "true") + } val kubernetesConf = KubernetesConf.createDriverConf( sparkConf, appName, diff --git a/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/Dockerfile b/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/Dockerfile index 42a670174eae..e0e36f276fae 100644 --- a/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/Dockerfile +++ b/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/Dockerfile @@ -29,7 +29,7 @@ ARG img_path=kubernetes/dockerfiles RUN set -ex && \ apk upgrade --no-cache && \ - apk add --no-cache bash tini libc6-compat && \ + apk add --no-cache bash tini libc6-compat krb5 krb5-libs && \ mkdir -p /opt/spark && \ mkdir -p /opt/spark/work-dir && \ touch /opt/spark/RELEASE && \ From 4a000d2abda968a28f419d21418f61e2f53355fc Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Mon, 20 Aug 2018 17:39:35 -0400 Subject: [PATCH 06/25] fix dockerfile --- .../kubernetes/docker/src/main/dockerfiles/spark/Dockerfile | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/Dockerfile b/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/Dockerfile index e11609036508..eececea49bfd 100644 --- a/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/Dockerfile +++ b/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/Dockerfile @@ -45,7 +45,7 @@ COPY examples /opt/spark/examples COPY data /opt/spark/data RUN if [ -z "$SPARK_USER" ]; then \ addgroup -g 2000 sparkpod && \ - adduser -D -u 1000 -G sparkpod sparkpod \ + adduser -D -u 1000 -G sparkpod sparkpod; \ fi ENV SPARK_HOME /opt/spark From 719b059910adff2368a0e7e0b55ad26329d5030b Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Wed, 29 Aug 2018 00:31:40 -0400 Subject: [PATCH 07/25] non-effective attempt to solve null UnixUsername error --- .../k8s/features/hadoopsteps/HadoopBootstrapUtil.scala | 5 ----- .../docker/src/main/dockerfiles/spark/Dockerfile | 4 ---- .../docker/src/main/dockerfiles/spark/entrypoint.sh | 8 ++++++++ 3 files changed, 8 insertions(+), 9 deletions(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopBootstrapUtil.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopBootstrapUtil.scala index 67a9739c745e..8b51bd7fa5f2 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopBootstrapUtil.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopBootstrapUtil.scala @@ -68,11 +68,6 @@ private[spark] object HadoopBootstrapUtil { .build()) .endConfigMap() .endVolume() - // TODO: (ifilonenko) make configurable PU(G)ID - .editOrNewSecurityContext() - .withRunAsUser(1000L) - .withFsGroup(2000L) - .endSecurityContext() .endSpec() .build() val kerberizedContainer = new ContainerBuilder(pod.container) diff --git a/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/Dockerfile b/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/Dockerfile index eececea49bfd..e0e36f276fae 100644 --- a/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/Dockerfile +++ b/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/Dockerfile @@ -43,10 +43,6 @@ COPY sbin /opt/spark/sbin COPY ${img_path}/spark/entrypoint.sh /opt/ COPY examples /opt/spark/examples COPY data /opt/spark/data -RUN if [ -z "$SPARK_USER" ]; then \ - addgroup -g 2000 sparkpod && \ - adduser -D -u 1000 -G sparkpod sparkpod; \ - fi ENV SPARK_HOME /opt/spark diff --git a/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/entrypoint.sh b/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/entrypoint.sh index 4958b7363fee..0a2ab5292256 100755 --- a/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/entrypoint.sh +++ b/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/entrypoint.sh @@ -87,6 +87,14 @@ if ! [ -z ${HADOOP_CONF_DIR+x} ]; then SPARK_CLASSPATH="$HADOOP_CONF_DIR:$SPARK_CLASSPATH"; fi +if [ -n "$HADOOP_TOKEN_FILE_LOCATION" ]; then \ + touch /etc/group && \ + touch /etc/passwd && \ + addgroup -g 2000 sparkpod && \ + adduser -D -u 1000 -G sparkpod $SPARK_USER && \ + su $SPARK_USER; +fi + case "$SPARK_K8S_CMD" in driver) CMD=( From fb9e810e187576eb0d48cf685c0cd5e56ac3675a Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Wed, 29 Aug 2018 16:01:15 -0400 Subject: [PATCH 08/25] move credential get --- .../features/hadoopsteps/HadoopKerberosKeytabResolverStep.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopKerberosKeytabResolverStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopKerberosKeytabResolverStep.scala index b8c9a85a4554..ed08a448f217 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopKerberosKeytabResolverStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopKerberosKeytabResolverStep.scala @@ -69,11 +69,11 @@ private[spark] class HadoopKerberosKeytabResolverStep( } // In the case that keytab is not specified we will read from Local Ticket Cache val jobUserUGI = maybeJobUserUGI.getOrElse(tokenManager.getCurrentUser) + val originalCredentials = jobUserUGI.getCredentials // It is necessary to run as jobUserUGI because logged in user != Current User val (tokenData, renewalInterval) = jobUserUGI.doAs( new PrivilegedExceptionAction[(Array[Byte], Long)] { override def run(): (Array[Byte], Long) = { - val originalCredentials = jobUserUGI.getCredentials val hadoopTokenManager: HadoopDelegationTokenManager = new HadoopDelegationTokenManager(submissionSparkConf, hadoopConf) tokenManager.getDelegationTokens( From e7935f8b7cfc2874e742f47dd2394aaddf713875 Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Tue, 4 Sep 2018 16:08:05 -0400 Subject: [PATCH 09/25] current working solution --- .../org/apache/spark/deploy/SparkSubmit.scala | 10 +--- docs/security.md | 56 +++++++++++++++++++ .../HadoopGlobalFeatureDriverStep.scala | 16 +----- .../hadoopsteps/HadoopBootstrapUtil.scala | 3 +- .../hadoopsteps/HadoopConfigSpec.scala | 6 -- .../k8s/KubernetesExecutorBuilder.scala | 4 +- .../src/main/dockerfiles/spark/entrypoint.sh | 8 --- 7 files changed, 62 insertions(+), 41 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index b7cea8781687..34118ccdd6fc 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -164,15 +164,7 @@ private[spark] class SparkSubmit extends Logging { } } } else { - if (sparkConf.getOption("spark.kubernetes.kerberos.proxyUser").isDefined) { - // scalastyle:off println - printStream.println("Running as proxy user in k8s cluster mode...") - // scalastyle:on println - SparkHadoopUtil.get.runAsSparkUser( - () => runMain(childArgs, childClasspath, sparkConf, childMainClass, args.verbose)) - } else { - runMain(childArgs, childClasspath, sparkConf, childMainClass, args.verbose) - } + runMain(childArgs, childClasspath, sparkConf, childMainClass, args.verbose) } } diff --git a/docs/security.md b/docs/security.md index c8eec730889c..c184f3de89e7 100644 --- a/docs/security.md +++ b/docs/security.md @@ -722,6 +722,62 @@ with encryption, at least. The Kerberos login will be periodically renewed using the provided credentials, and new delegation tokens for supported will be created. +## Secure Interaction with Kubernetes + +When talking to Hadoop-based services behind Kerberos, it was noted that Spark needs to obtain delegation tokens +so that non-local processes can authenticate. These delegation tokens in Kubernetes are stored in Secrets that are +shared by the Driver and its Executors. As such, there are three ways of submitting a kerberos job: + +1. Submitting with a $kinit that stores a TGT in the Local Ticket Cache: +```bash +/usr/bin/kinit -kt / +/opt/spark/bin/spark-submit \ + --deploy-mode cluster \ + --class org.apache.spark.examples.HdfsTest \ + --master k8s:// \ + --conf spark.executor.instances=1 \ + --conf spark.app.name=spark-hdfs \ + --conf spark.kubernetes.container.image=spark:latest \ + --conf spark.kubernetes.kerberos.krb5location=/etc/krb5.conf \ + local:///opt/spark/examples/jars/spark-examples_2.11-2.4.0-SNAPSHOT.jar \ + +``` +3. Submitting with a local keytab and principle +```bash +/opt/spark/bin/spark-submit \ + --deploy-mode cluster \ + --class org.apache.spark.examples.HdfsTest \ + --master k8s:// \ + --conf spark.executor.instances=1 \ + --conf spark.app.name=spark-hdfs \ + --conf spark.kubernetes.container.image=spark:latest \ + --conf spark.kubernetes.kerberos.keytab= \ + --conf spark.kubernetes.kerberos.principal= \ + --conf spark.kubernetes.kerberos.krb5location=/etc/krb5.conf \ + local:///opt/spark/examples/jars/spark-examples_2.11-2.4.0-SNAPSHOT.jar \ + +``` + +3. Submitting with pre-populated secrets already existing within the namespace +```bash +/opt/spark/bin/spark-submit \ + --deploy-mode cluster \ + --class org.apache.spark.examples.HdfsTest \ + --master k8s:// \ + --conf spark.executor.instances=1 \ + --conf spark.app.name=spark-hdfs \ + --conf spark.kubernetes.container.image=spark:latest \ + --conf spark.kubernetes.kerberos.tokensecret.name= \ + --conf spark.kubernetes.kerberos.tokensecret.itemkey= \ + --conf spark.kubernetes.kerberos.krb5location=/etc/krb5.conf \ + local:///opt/spark/examples/jars/spark-examples_2.11-2.4.0-SNAPSHOT.jar \ + + +``` + + + + # Event Logging diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopGlobalFeatureDriverStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopGlobalFeatureDriverStep.scala index a96ce12316e9..7c437bce350f 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopGlobalFeatureDriverStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopGlobalFeatureDriverStep.scala @@ -45,9 +45,6 @@ private[spark] class HadoopGlobalFeatureDriverStep( .getOrElse(Seq.empty[HadoopConfigurationStep]) var currentHadoopSpec = HadoopConfigSpec( - podVolumes = Seq.empty, - containerEnvs = Seq.empty, - containerVMs = Seq.empty, configMapProperties = Map.empty[String, String], dtSecret = None, dtSecretName = KERBEROS_DELEGEGATION_TOKEN_SECRET_NAME, @@ -59,22 +56,11 @@ private[spark] class HadoopGlobalFeatureDriverStep( } override def configurePod(pod: SparkPod): SparkPod = { - val hadoopBasedPod = new PodBuilder(pod.pod) - .editSpec() - .addAllToVolumes(currentHadoopSpec.podVolumes.asJava) - .endSpec() - .build() - - val hadoopBasedContainer = new ContainerBuilder(pod.container) - .addAllToEnv(currentHadoopSpec.containerEnvs.asJava) - .addAllToVolumeMounts(currentHadoopSpec.containerVMs.asJava) - .build() - val hadoopBasedSparkPod = HadoopBootstrapUtil.bootstrapHadoopConfDir( kubernetesConf.hadoopConfDir.get, kubernetesConf.getHadoopConfigMapName, kubernetesConf.getTokenManager, - SparkPod(hadoopBasedPod, hadoopBasedContainer)) + pod) val maybeKerberosModification = for { diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopBootstrapUtil.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopBootstrapUtil.scala index 8b51bd7fa5f2..54943c412722 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopBootstrapUtil.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopBootstrapUtil.scala @@ -77,7 +77,8 @@ private[spark] object HadoopBootstrapUtil { .endVolumeMount() .addNewVolumeMount() .withName(KRB_FILE_VOLUME) - .withMountPath(KRB_FILE_DIR_PATH) + .withMountPath(KRB_FILE_DIR_PATH + "/krb5.conf") + .withSubPath("krb5.conf") .endVolumeMount() .addNewEnv() .withName(ENV_HADOOP_TOKEN_FILE_LOCATION) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopConfigSpec.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopConfigSpec.scala index 8f6207e27be8..2b10838d8ac7 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopConfigSpec.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopConfigSpec.scala @@ -22,9 +22,6 @@ import io.fabric8.kubernetes.api.model.{EnvVar, Secret, Volume, VolumeMount} * Represents a given configuration of the hadoop configuration logic, informing the * HadoopConfigBootstrapStep of how the driver should be configured. This includes: *

- * - Volumes that need to mounted onto the pod - * - Environmental variables that need to be launched with the container - * - Volume Mounts that need to mounted with the container * - The properties that will be stored into the config map which have (key, value) * pairs of (path, data) * - The secret containing a DT, either previously specified or built on the fly @@ -33,9 +30,6 @@ import io.fabric8.kubernetes.api.model.{EnvVar, Secret, Volume, VolumeMount} * - The Job User's username */ private[spark] case class HadoopConfigSpec( - podVolumes: Seq[Volume], - containerEnvs: Seq[EnvVar], - containerVMs: Seq[VolumeMount], configMapProperties: Map[String, String], dtSecret: Option[Secret], dtSecretName: String, 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 2c417ebe5cca..eab19809c6af 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 @@ -75,8 +75,8 @@ private[spark] class KubernetesExecutorBuilder( } yield { provideKerberosConfStep(kubernetesConf) } - Seq(provideHadoopConfStep(kubernetesConf)) ++ - maybeKerberosStep :+ maybeKerberosStep.getOrElse( + Seq(provideHadoopConfStep(kubernetesConf)) :+ + maybeKerberosStep.getOrElse( provideHadoopSparkUserStep(kubernetesConf)) }.getOrElse(Seq.empty[KubernetesFeatureConfigStep]) diff --git a/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/entrypoint.sh b/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/entrypoint.sh index 0a2ab5292256..4958b7363fee 100755 --- a/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/entrypoint.sh +++ b/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/entrypoint.sh @@ -87,14 +87,6 @@ if ! [ -z ${HADOOP_CONF_DIR+x} ]; then SPARK_CLASSPATH="$HADOOP_CONF_DIR:$SPARK_CLASSPATH"; fi -if [ -n "$HADOOP_TOKEN_FILE_LOCATION" ]; then \ - touch /etc/group && \ - touch /etc/passwd && \ - addgroup -g 2000 sparkpod && \ - adduser -D -u 1000 -G sparkpod $SPARK_USER && \ - su $SPARK_USER; -fi - case "$SPARK_K8S_CMD" in driver) CMD=( From 583a52c1759f6be9dfe9bebf504c16dbcd7578dd Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Fri, 21 Sep 2018 13:52:35 -0400 Subject: [PATCH 10/25] merge conflicts and various additions --- docs/security.md | 5 +- .../org/apache/spark/examples/HdfsTest.scala | 4 +- .../org/apache/spark/deploy/k8s/Config.scala | 27 +--- .../spark/deploy/k8s/KubernetesConf.scala | 9 -- .../spark/deploy/k8s/KubernetesUtils.scala | 9 ++ .../HadoopGlobalFeatureDriverStep.scala | 149 ++++++++++++------ .../k8s/features/OptionRequirements.scala | 36 ----- .../hadoopsteps/HadoopConfigMounterStep.scala | 45 ------ .../hadoopsteps/HadoopConfigurationStep.scala | 26 --- .../HadoopKerberosKeytabResolverStep.scala | 105 ------------ .../hadoopsteps/HadoopKerberosLogin.scala | 101 ++++++++++++ .../HadoopKerberosSecretResolverStep.scala | 41 ----- .../hadoopsteps/HadoopStepsOrchestrator.scala | 87 ---------- ...figSpec.scala => KerberosConfigSpec.scala} | 11 +- .../submit/KubernetesClientApplication.scala | 5 - .../BasicExecutorFeatureStepSuite.scala | 3 +- 16 files changed, 226 insertions(+), 437 deletions(-) delete mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/OptionRequirements.scala delete mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopConfigMounterStep.scala delete mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopConfigurationStep.scala delete mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopKerberosKeytabResolverStep.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopKerberosLogin.scala delete mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopKerberosSecretResolverStep.scala delete mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopStepsOrchestrator.scala rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/{HadoopConfigSpec.scala => KerberosConfigSpec.scala} (79%) diff --git a/docs/security.md b/docs/security.md index ee56e7afa82d..b147aade891a 100644 --- a/docs/security.md +++ b/docs/security.md @@ -728,6 +728,7 @@ When talking to Hadoop-based services behind Kerberos, it was noted that Spark n so that non-local processes can authenticate. These delegation tokens in Kubernetes are stored in Secrets that are shared by the Driver and its Executors. As such, there are three ways of submitting a kerberos job: +In all cases you must define the environment variable: `HADOOP_CONF_DIR`. 1. Submitting with a $kinit that stores a TGT in the Local Ticket Cache: ```bash /usr/bin/kinit -kt / @@ -742,7 +743,7 @@ shared by the Driver and its Executors. As such, there are three ways of submitt local:///opt/spark/examples/jars/spark-examples_2.11-2.4.0-SNAPSHOT.jar \ ``` -3. Submitting with a local keytab and principle +2. Submitting with a local keytab and principal ```bash /opt/spark/bin/spark-submit \ --deploy-mode cluster \ @@ -758,7 +759,7 @@ shared by the Driver and its Executors. As such, there are three ways of submitt ``` -3. Submitting with pre-populated secrets already existing within the namespace +3. Submitting with pre-populated secrets, that contain the delegation token, already existing within the namespace ```bash /opt/spark/bin/spark-submit \ --deploy-mode cluster \ diff --git a/examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala b/examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala index c1d610246639..5ee1806dcb67 100644 --- a/examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala @@ -41,8 +41,8 @@ object HdfsTest { val end = System.currentTimeMillis() println(s"Iteration $iter took ${end-start} ms") } - println(s"File contents: ${file.map(s => s.toString).collect().mkString(",")}") - println(s"Returned length(s) of: ${file.map(s => s.length).collect().mkString(",")}") + println(s"File contents: ${file.map(_.toString).collect().mkString(",")}") + println(s"Returned length(s) of: ${file.map(_.length).collect().mkString(",")}") spark.stop() } } 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 129add8300c2..b9969a01ddbc 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 @@ -224,48 +224,37 @@ private[spark] object Config extends Logging { .checkValue(pv => List("2", "3").contains(pv), "Ensure that major Python version is either Python2 or Python3") .createWithDefault("2") - - val KUBERNETES_KERBEROS_PROXY_USER = - ConfigBuilder("spark.kubernetes.kerberos.proxyUser") - .doc("Specify the proxy user " + - "for HadoopUGI login for the Driver + Executors") - .internal() - .stringConf - .createWithDefault("false") val KUBERNETES_KERBEROS_KRB5_FILE = ConfigBuilder("spark.kubernetes.kerberos.krb5location") - .doc("Specify the location of the krb5 file " + - "to be mounted on the driver and executors for Secure HDFS") + .doc("Specify the location of the krb5 file to be mounted on the driver " + + "and executors for Secure HDFS. Note: the KDC defined needs to be " + + "visible from inside the containers ") .stringConf .createOptional val KUBERNETES_KERBEROS_KEYTAB = ConfigBuilder("spark.kubernetes.kerberos.keytab") - .doc("Specify the location of keytab " + - "for Kerberos in order to access Secure HDFS") + .doc("Specify the location of keytab for Kerberos in order to access Secure HDFS") .stringConf .createOptional val KUBERNETES_KERBEROS_PRINCIPAL = ConfigBuilder("spark.kubernetes.kerberos.principal") - .doc("Specify the principal " + - "for Kerberos in order to access Secure HDFS") + .doc("Specify the principal for Kerberos in order to access Secure HDFS") .stringConf .createOptional val KUBERNETES_KERBEROS_RENEWER_PRINCIPAL = ConfigBuilder("spark.kubernetes.kerberos.renewer.principal") - .doc("Specify the principal " + - "you wish to renew and retrieve your Kerberos values with") + .doc("Specify the principal you wish to renew and retrieve your Kerberos values with") .stringConf .createOptional val KUBERNETES_KERBEROS_DT_SECRET_NAME = ConfigBuilder("spark.kubernetes.kerberos.tokensecret.name") - .doc("Specify the name of the secret where " + - "your existing delegation token is stored. This removes the need " + - "for the job user to provide any keytab for launching a job") + .doc("Specify the name of the secret where your existing delegation token is stored. " + + "This removes the need for the job user to provide any keytab for launching a job") .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 27f4ed98f316..5a54853f1cc4 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 @@ -23,7 +23,6 @@ import io.fabric8.kubernetes.api.model.{LocalObjectReference, LocalObjectReferen 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.features.hadoopsteps.HadoopStepsOrchestrator import org.apache.spark.deploy.k8s.security.KubernetesHadoopDelegationTokenManager import org.apache.spark.deploy.k8s.submit._ import org.apache.spark.deploy.k8s.submit.KubernetesClientApplication._ @@ -70,14 +69,6 @@ private[spark] case class KubernetesConf[T <: KubernetesRoleSpecificConf]( def getKRBConfigMapName: String = s"$appResourceNamePrefix-krb5-file" - def getHadoopStepsOrchestrator : Option[HadoopStepsOrchestrator] = hadoopConfDir.map { - hConf => new HadoopStepsOrchestrator( - sparkConf, - appResourceNamePrefix, - hConf, - getHadoopConfigMapName, - getTokenManager.isSecurityEnabled)} - def getTokenManager : KubernetesHadoopDelegationTokenManager = new KubernetesHadoopDelegationTokenManager diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesUtils.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesUtils.scala index f5fae7cc8c47..f05318095358 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesUtils.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesUtils.scala @@ -39,6 +39,15 @@ private[spark] object KubernetesUtils { sparkConf.getAllWithPrefix(prefix).toMap } + def requireBothOrNeitherDefined( + opt1: Option[_], + opt2: Option[_], + errMessageWhenFirstIsMissing: String, + errMessageWhenSecondIsMissing: String): Unit = { + requireNandDefined(opt1, opt2, errMessageWhenSecondIsMissing) + requireNandDefined(opt2, opt1, errMessageWhenFirstIsMissing) + } + def requireNandDefined(opt1: Option[_], opt2: Option[_], errMessage: String): Unit = { opt1.foreach { _ => require(opt2.isEmpty, errMessage) } } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopGlobalFeatureDriverStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopGlobalFeatureDriverStep.scala index 7c437bce350f..310f6d12c99a 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopGlobalFeatureDriverStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopGlobalFeatureDriverStep.scala @@ -16,15 +16,19 @@ */ package org.apache.spark.deploy.k8s.features +import java.io.File + import scala.collection.JavaConverters._ -import io.fabric8.kubernetes.api.model.{ConfigMapBuilder, ContainerBuilder, HasMetadata, PodBuilder} +import com.google.common.base.Charsets +import com.google.common.io.Files +import io.fabric8.kubernetes.api.model.{ConfigMapBuilder, HasMetadata} -import org.apache.spark.deploy.k8s.{KubernetesConf, SparkPod} -import org.apache.spark.deploy.k8s.Config.{KUBERNETES_KERBEROS_KRB5_FILE, KUBERNETES_KERBEROS_PROXY_USER} +import org.apache.spark.deploy.k8s.{KubernetesConf, 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.hadoopsteps.{HadoopBootstrapUtil, HadoopConfigSpec, HadoopConfigurationStep} +import org.apache.spark.deploy.k8s.features.hadoopsteps._ import org.apache.spark.internal.Logging /** @@ -35,64 +39,98 @@ import org.apache.spark.internal.Logging private[spark] class HadoopGlobalFeatureDriverStep( kubernetesConf: KubernetesConf[KubernetesDriverSpecificConf]) extends KubernetesFeatureConfigStep with Logging { - private val hadoopTestOrchestrator = - kubernetesConf.getHadoopStepsOrchestrator - require(kubernetesConf.hadoopConfDir.isDefined && - hadoopTestOrchestrator.isDefined, "Ensure that HADOOP_CONF_DIR is defined") - private val hadoopSteps = - hadoopTestOrchestrator - .map(hto => hto.getHadoopSteps(kubernetesConf.getTokenManager)) - .getOrElse(Seq.empty[HadoopConfigurationStep]) - var currentHadoopSpec = HadoopConfigSpec( - configMapProperties = Map.empty[String, String], - dtSecret = None, - dtSecretName = KERBEROS_DELEGEGATION_TOKEN_SECRET_NAME, - dtSecretItemKey = None, - jobUserName = None) + private val conf = kubernetesConf.sparkConf + private val maybePrincipal = conf.get(KUBERNETES_KERBEROS_PRINCIPAL) + private val maybeKeytab = conf.get(KUBERNETES_KERBEROS_KEYTAB) + .map(k => new File(k)) + private val maybeExistingSecretName = conf.get(KUBERNETES_KERBEROS_DT_SECRET_NAME) + private val maybeExistingSecretItemKey = + conf.get(KUBERNETES_KERBEROS_DT_SECRET_ITEM_KEY) + private val maybeRenewerPrincipal = + conf.get(KUBERNETES_KERBEROS_RENEWER_PRINCIPAL) + private val kubeTokenManager = kubernetesConf.getTokenManager + private val isKerberosEnabled = kubeTokenManager.isSecurityEnabled + + require(maybeKeytab.forall( _ => isKerberosEnabled ), + "You must enable Kerberos support if you are specifying a Kerberos Keytab") + + require(maybeExistingSecretName.forall( _ => isKerberosEnabled ), + "You must enable Kerberos support if you are specifying a Kerberos Secret") + + KubernetesUtils.requireBothOrNeitherDefined( + maybeKeytab, + maybePrincipal, + "If a Kerberos keytab is specified you must also specify a Kerberos principal", + "If a Kerberos principal is specified you must also specify a Kerberos keytab") + + KubernetesUtils.requireBothOrNeitherDefined( + maybeExistingSecretName, + maybeExistingSecretItemKey, + "If a secret storing a Kerberos Delegation Token is specified you must also" + + " specify the label where the data is stored", + "If a secret data item-key where the data of the Kerberos Delegation Token is specified" + + " you must also specify the name of the secret") - for (nextStep <- hadoopSteps) { - currentHadoopSpec = nextStep.configureHadoopSpec(currentHadoopSpec) - } + // TODO: Using pre-existing configMaps instead of local HADOOP_CONF_DIR + require(kubernetesConf.hadoopConfDir.isDefined, "Ensure that HADOOP_CONF_DIR is defined") + private val hadoopConfDir = kubernetesConf.hadoopConfDir.get + private val hadoopConfigurationFiles = kubeTokenManager.getHadoopConfFiles(hadoopConfDir) + + // Either use pre-existing secret or login to create new Secret with DT stored within + private val hadoopSpec: Option[KerberosConfigSpec] = (for { + secretName <- maybeExistingSecretName + secretItemKey <- maybeExistingSecretItemKey + } yield { + KerberosConfigSpec( + dtSecret = None, + dtSecretName = secretName, + dtSecretItemKey = secretItemKey, + jobUserName = kubeTokenManager.getCurrentUser.getShortUserName) + }).orElse( + for { + _ <- maybePrincipal + _ <- maybeKeytab + _ <- maybeRenewerPrincipal + } yield { + HadoopKerberosLogin.buildSpec( + conf, + kubernetesConf.appResourceNamePrefix, + maybePrincipal, + maybeKeytab, + maybeRenewerPrincipal, + kubeTokenManager) + }) override def configurePod(pod: SparkPod): SparkPod = { val hadoopBasedSparkPod = HadoopBootstrapUtil.bootstrapHadoopConfDir( - kubernetesConf.hadoopConfDir.get, + hadoopConfDir, kubernetesConf.getHadoopConfigMapName, - kubernetesConf.getTokenManager, + kubeTokenManager, pod) - - val maybeKerberosModification = - for { - secretItemKey <- currentHadoopSpec.dtSecretItemKey - userName <- currentHadoopSpec.jobUserName - krb5fileLocation <- kubernetesConf.get(KUBERNETES_KERBEROS_KRB5_FILE) - } yield { - HadoopBootstrapUtil.bootstrapKerberosPod( - currentHadoopSpec.dtSecretName, - secretItemKey, - userName, - krb5fileLocation, - kubernetesConf.getKRBConfigMapName, - hadoopBasedSparkPod) - } - maybeKerberosModification.getOrElse( + (for { + hSpec <- hadoopSpec + krb5fileLocation <- kubernetesConf.get(KUBERNETES_KERBEROS_KRB5_FILE) + } yield { + HadoopBootstrapUtil.bootstrapKerberosPod( + hSpec.dtSecretName, + hSpec.dtSecretItemKey, + hSpec.jobUserName, + krb5fileLocation, + kubernetesConf.getKRBConfigMapName, + hadoopBasedSparkPod) + }).getOrElse( HadoopBootstrapUtil.bootstrapSparkUserPod( - kubernetesConf.getTokenManager.getCurrentUser.getShortUserName, + kubeTokenManager.getCurrentUser.getShortUserName, hadoopBasedSparkPod)) } override def getAdditionalPodSystemProperties(): Map[String, String] = { - val maybeKerberosConfValues = - for { - secretItemKey <- currentHadoopSpec.dtSecretItemKey - userName <- currentHadoopSpec.jobUserName - } yield { - Map(KERBEROS_KEYTAB_SECRET_NAME -> currentHadoopSpec.dtSecretName, - KERBEROS_KEYTAB_SECRET_KEY -> secretItemKey, - KERBEROS_SPARK_USER_NAME -> userName) - } - val resolvedConfValues = maybeKerberosConfValues.getOrElse( + val resolvedConfValues = hadoopSpec.map{hSpec => + Map(KERBEROS_KEYTAB_SECRET_NAME -> hSpec.dtSecretName, + KERBEROS_KEYTAB_SECRET_KEY -> hSpec.dtSecretItemKey, + KERBEROS_SPARK_USER_NAME -> hSpec.jobUserName) + }.getOrElse( Map(KERBEROS_SPARK_USER_NAME -> kubernetesConf.getTokenManager.getCurrentUser.getShortUserName) ) @@ -105,15 +143,22 @@ private[spark] class HadoopGlobalFeatureDriverStep( .map(fileLocation => HadoopBootstrapUtil.buildkrb5ConfigMap( kubernetesConf.getKRBConfigMapName, fileLocation)) + val kerberosDTSecret = for { + hSpec <- hadoopSpec + kDtSecret <- hSpec.dtSecret + } yield { + kDtSecret + } val configMap = new ConfigMapBuilder() .withNewMetadata() .withName(kubernetesConf.getHadoopConfigMapName) .endMetadata() - .addToData(currentHadoopSpec.configMapProperties.asJava) + .addToData(hadoopConfigurationFiles.map(file => + (file.toPath.getFileName.toString, Files.toString(file, Charsets.UTF_8))).toMap.asJava) .build() Seq(configMap) ++ krb5ConfigMap.toSeq ++ - currentHadoopSpec.dtSecret.toSeq + kerberosDTSecret.toSeq } } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/OptionRequirements.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/OptionRequirements.scala deleted file mode 100644 index 3766384e0ab5..000000000000 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/OptionRequirements.scala +++ /dev/null @@ -1,36 +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 - -private[spark] object OptionRequirements { - - def requireBothOrNeitherDefined( - opt1: Option[_], - opt2: Option[_], - errMessageWhenFirstIsMissing: String, - errMessageWhenSecondIsMissing: String): Unit = { - requireSecondIfFirstIsDefined(opt1, opt2, errMessageWhenSecondIsMissing) - requireSecondIfFirstIsDefined(opt2, opt1, errMessageWhenFirstIsMissing) - } - - def requireSecondIfFirstIsDefined( - opt1: Option[_], opt2: Option[_], errMessageWhenSecondIsMissing: String): Unit = { - opt1.foreach { _ => - require(opt2.isDefined, errMessageWhenSecondIsMissing) - } - } -} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopConfigMounterStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopConfigMounterStep.scala deleted file mode 100644 index 86d384204ebc..000000000000 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopConfigMounterStep.scala +++ /dev/null @@ -1,45 +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.hadoopsteps - -import java.io.File - -import scala.collection.JavaConverters._ - -import com.google.common.base.Charsets -import com.google.common.io.Files -import io.fabric8.kubernetes.api.model.{EnvVarBuilder, KeyToPathBuilder, VolumeBuilder, VolumeMountBuilder} - -import org.apache.spark.deploy.k8s.Constants._ -import org.apache.spark.internal.Logging - - /** - * This step is responsible for creating ConfigMaps containing Hadoop config files - */ -private[spark] class HadoopConfigMounterStep( - hadoopConfConfigMapName: String, - hadoopConfigFiles: Seq[File]) - extends HadoopConfigurationStep with Logging{ - - override def configureHadoopSpec(hSpec: HadoopConfigSpec) : HadoopConfigSpec = { - logInfo("HADOOP_CONF_DIR defined. Mounting Hadoop specific files") - hSpec.copy( - configMapProperties = hSpec.configMapProperties ++ - hadoopConfigFiles.map(file => - (file.toPath.getFileName.toString, Files.toString(file, Charsets.UTF_8))).toMap) - } -} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopConfigurationStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopConfigurationStep.scala deleted file mode 100644 index b1e9e5f22033..000000000000 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopConfigurationStep.scala +++ /dev/null @@ -1,26 +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.hadoopsteps - -private[spark] trait HadoopConfigurationStep { - /** - * Apply modifications on the given HadoopConfSpec in accordance to this feature. - * Note that we should return a HadoopConfSpec that keeps all of the properties of the - * passed HadoopConfSpec object. - */ - def configureHadoopSpec(hadoopConfSpec: HadoopConfigSpec) : HadoopConfigSpec -} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopKerberosKeytabResolverStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopKerberosKeytabResolverStep.scala deleted file mode 100644 index ed08a448f217..000000000000 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopKerberosKeytabResolverStep.scala +++ /dev/null @@ -1,105 +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.hadoopsteps - -import java.io._ -import java.security.PrivilegedExceptionAction - -import scala.collection.JavaConverters._ - -import io.fabric8.kubernetes.api.model.SecretBuilder -import org.apache.commons.codec.binary.Base64 - -import org.apache.spark.{SparkConf, SparkException} -import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.deploy.k8s.Constants._ -import org.apache.spark.deploy.k8s.security.KubernetesHadoopDelegationTokenManager -import org.apache.spark.deploy.security.HadoopDelegationTokenManager -import org.apache.spark.internal.Logging - - /** - * This step does all the heavy lifting for Delegation Token logic. This step - * assumes that the job user has either specified a principal and keytab or ran - * $kinit before running spark-submit. With a TGT stored locally, by running - * UGI.getCurrentUser you are able to obtain the current user, alternatively - * you can run UGI.loginUserFromKeytabAndReturnUGI and by running .doAs run - * as the logged into user instead of the current user. 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 - * appended to the current HadoopSpec which in turn will append to the current - * DriverSpec. - */ -private[spark] class HadoopKerberosKeytabResolverStep( - submissionSparkConf: SparkConf, - kubernetesResourceNamePrefix : String, - maybePrincipal: Option[String], - maybeKeytab: Option[File], - maybeRenewerPrincipal: Option[String], - tokenManager: KubernetesHadoopDelegationTokenManager) - extends HadoopConfigurationStep with Logging { - - override def configureHadoopSpec(hSpec: HadoopConfigSpec): HadoopConfigSpec = { - val hadoopConf = SparkHadoopUtil.get.newConfiguration(submissionSparkConf) - if (!tokenManager.isSecurityEnabled) { - throw new SparkException("Hadoop not configured with Kerberos") - } - val maybeJobUserUGI = - for { - principal <- maybePrincipal - keytab <- maybeKeytab - } yield { - logDebug("Logged into KDC with keytab using Job User UGI") - tokenManager.loginUserFromKeytabAndReturnUGI( - principal, - keytab.toURI.toString) - } - // In the case that keytab is not specified we will read from Local Ticket Cache - val jobUserUGI = maybeJobUserUGI.getOrElse(tokenManager.getCurrentUser) - val originalCredentials = jobUserUGI.getCredentials - // It is necessary to run as jobUserUGI because logged in user != Current User - val (tokenData, renewalInterval) = jobUserUGI.doAs( - new PrivilegedExceptionAction[(Array[Byte], Long)] { - override def run(): (Array[Byte], Long) = { - val hadoopTokenManager: HadoopDelegationTokenManager = - new HadoopDelegationTokenManager(submissionSparkConf, hadoopConf) - tokenManager.getDelegationTokens( - originalCredentials, - submissionSparkConf, - hadoopConf, - hadoopTokenManager) - }}) - if (tokenData.isEmpty) throw new SparkException(s"Did not obtain any delegation tokens") - val currentTime = tokenManager.getCurrentTime - val initialTokenDataKeyName = s"$KERBEROS_SECRET_LABEL_PREFIX-$currentTime-$renewalInterval" - val uniqueSecretName = - s"$kubernetesResourceNamePrefix-$KERBEROS_DELEGEGATION_TOKEN_SECRET_NAME.$currentTime" - val secretDT = - new SecretBuilder() - .withNewMetadata() - .withName(uniqueSecretName) - .withLabels(Map(KERBEROS_REFRESH_LABEL_KEY -> KERBEROS_REFRESH_LABEL_VALUE).asJava) - .endMetadata() - .addToData(initialTokenDataKeyName, Base64.encodeBase64String(tokenData)) - .build() - - hSpec.copy( - dtSecret = Some(secretDT), - dtSecretName = uniqueSecretName, - dtSecretItemKey = Some(initialTokenDataKeyName), - jobUserName = Some(jobUserUGI.getShortUserName)) - } -} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopKerberosLogin.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopKerberosLogin.scala new file mode 100644 index 000000000000..e9caf240dbc7 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopKerberosLogin.scala @@ -0,0 +1,101 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.k8s.features.hadoopsteps + +import java.io._ +import java.security.PrivilegedExceptionAction + +import scala.collection.JavaConverters._ + +import io.fabric8.kubernetes.api.model.SecretBuilder +import org.apache.commons.codec.binary.Base64 + +import org.apache.spark.{SparkConf, SparkException} +import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.deploy.k8s.Constants._ +import org.apache.spark.deploy.k8s.security.KubernetesHadoopDelegationTokenManager +import org.apache.spark.deploy.security.HadoopDelegationTokenManager + + /** + * This step does all the heavy lifting for Delegation Token logic. This step + * assumes that the job user has either specified a principal and keytab or ran + * $kinit before running spark-submit. With a TGT stored locally, by running + * UGI.getCurrentUser you are able to obtain the current user, alternatively + * you can run UGI.loginUserFromKeytabAndReturnUGI and by running .doAs run + * as the logged into user instead of the current user. 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 + * appended to the current HadoopSpec which in turn will append to the current + * DriverSpec. + */ +private[spark] object HadoopKerberosLogin { + def buildSpec( + submissionSparkConf: SparkConf, + kubernetesResourceNamePrefix : String, + maybePrincipal: Option[String], + maybeKeytab: Option[File], + maybeRenewerPrincipal: Option[String], + tokenManager: KubernetesHadoopDelegationTokenManager): KerberosConfigSpec = { + val hadoopConf = SparkHadoopUtil.get.newConfiguration(submissionSparkConf) + if (!tokenManager.isSecurityEnabled) { + throw new SparkException("Hadoop not configured with Kerberos") + } + val maybeJobUserUGI = + for { + principal <- maybePrincipal + keytab <- maybeKeytab + } yield { + tokenManager.loginUserFromKeytabAndReturnUGI( + principal, + keytab.toURI.toString) + } + // In the case that keytab is not specified we will read from Local Ticket Cache + val jobUserUGI = maybeJobUserUGI.getOrElse(tokenManager.getCurrentUser) + val originalCredentials = jobUserUGI.getCredentials + // It is necessary to run as jobUserUGI because logged in user != Current User + val (tokenData, renewalInterval) = jobUserUGI.doAs( + new PrivilegedExceptionAction[(Array[Byte], Long)] { + override def run(): (Array[Byte], Long) = { + val hadoopTokenManager: HadoopDelegationTokenManager = + new HadoopDelegationTokenManager(submissionSparkConf, hadoopConf) + tokenManager.getDelegationTokens( + originalCredentials, + submissionSparkConf, + hadoopConf, + hadoopTokenManager) + }}) + if (tokenData.isEmpty) throw new SparkException(s"Did not obtain any delegation tokens") + val currentTime = tokenManager.getCurrentTime + val initialTokenDataKeyName = s"$KERBEROS_SECRET_LABEL_PREFIX-$currentTime-$renewalInterval" + val uniqueSecretName = + s"$kubernetesResourceNamePrefix-$KERBEROS_DELEGEGATION_TOKEN_SECRET_NAME.$currentTime" + val secretDT = + new SecretBuilder() + .withNewMetadata() + .withName(uniqueSecretName) + .withLabels(Map(KERBEROS_REFRESH_LABEL_KEY -> KERBEROS_REFRESH_LABEL_VALUE).asJava) + .endMetadata() + .addToData(initialTokenDataKeyName, Base64.encodeBase64String(tokenData)) + .build() + + KerberosConfigSpec( + dtSecret = Some(secretDT), + dtSecretName = uniqueSecretName, + dtSecretItemKey = initialTokenDataKeyName, + jobUserName = jobUserUGI.getShortUserName) + } +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopKerberosSecretResolverStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopKerberosSecretResolverStep.scala deleted file mode 100644 index ff3c75494bb3..000000000000 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopKerberosSecretResolverStep.scala +++ /dev/null @@ -1,41 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.spark.deploy.k8s.features.hadoopsteps - -import org.apache.spark.deploy.k8s.security.KubernetesHadoopDelegationTokenManager -import org.apache.spark.internal.Logging - - /** - * This step assumes that you have already done all the heavy lifting in retrieving a - * delegation token and storing the following data in a secret before running this job. - * This step requires that you just specify the secret name and data item-key corresponding - * to the data where the delegation token is stored. - */ -private[spark] class HadoopKerberosSecretResolverStep( - tokenSecretName: String, - tokenItemKeyName: String, - tokenManager: KubernetesHadoopDelegationTokenManager) - extends HadoopConfigurationStep with Logging { - - override def configureHadoopSpec(hSpec: HadoopConfigSpec): HadoopConfigSpec = { - hSpec.copy( - dtSecret = None, - dtSecretName = tokenSecretName, - dtSecretItemKey = Some(tokenItemKeyName), - jobUserName = Some(tokenManager.getCurrentUser.getShortUserName)) - } -} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopStepsOrchestrator.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopStepsOrchestrator.scala deleted file mode 100644 index 7f1323140076..000000000000 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopStepsOrchestrator.scala +++ /dev/null @@ -1,87 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.spark.deploy.k8s.features.hadoopsteps - -import java.io.File - -import org.apache.spark.SparkConf -import org.apache.spark.deploy.k8s.Config._ -import org.apache.spark.deploy.k8s.features.OptionRequirements -import org.apache.spark.deploy.k8s.security.KubernetesHadoopDelegationTokenManager -import org.apache.spark.internal.Logging - -private[spark] class HadoopStepsOrchestrator( - conf: SparkConf, - kubernetesResourceNamePrefix: String, - hadoopConfDir: String, - hadoopConfigMapName: String, - isKerberosEnabled: Boolean) extends Logging { - - private val maybePrincipal = conf.get(KUBERNETES_KERBEROS_PRINCIPAL) - private val maybeKeytab = conf.get(KUBERNETES_KERBEROS_KEYTAB) - .map(k => new File(k)) - private val maybeExistingSecretName = conf.get(KUBERNETES_KERBEROS_DT_SECRET_NAME) - private val maybeExistingSecretItemKey = - conf.get(KUBERNETES_KERBEROS_DT_SECRET_ITEM_KEY) - private val maybeRenewerPrincipal = - conf.get(KUBERNETES_KERBEROS_RENEWER_PRINCIPAL) - - require(maybeKeytab.forall( _ => isKerberosEnabled ), - "You must enable Kerberos support if you are specifying a Kerberos Keytab") - - require(maybeExistingSecretName.forall( _ => isKerberosEnabled ), - "You must enable Kerberos support if you are specifying a Kerberos Secret") - - OptionRequirements.requireBothOrNeitherDefined( - maybeKeytab, - maybePrincipal, - "If a Kerberos keytab is specified you must also specify a Kerberos principal", - "If a Kerberos principal is specified you must also specify a Kerberos keytab") - - OptionRequirements.requireBothOrNeitherDefined( - maybeExistingSecretName, - maybeExistingSecretItemKey, - "If a secret storing a Kerberos Delegation Token is specified you must also" + - " specify the label where the data is stored", - "If a secret data item-key where the data of the Kerberos Delegation Token is specified" + - " you must also specify the name of the secret") - - def getHadoopSteps(kubeTokenManager: KubernetesHadoopDelegationTokenManager): - Seq[HadoopConfigurationStep] = { - val hadoopConfigurationFiles = kubeTokenManager.getHadoopConfFiles(hadoopConfDir) - logInfo(s"Hadoop Conf directory: $hadoopConfDir") - val hadoopConfMounterStep = new HadoopConfigMounterStep( - hadoopConfigMapName, hadoopConfigurationFiles) - val maybeKerberosStep = - if (isKerberosEnabled) { - val maybeExistingSecretStep = for { - secretName <- maybeExistingSecretName - secretItemKey <- maybeExistingSecretItemKey - } yield { - Some(new HadoopKerberosSecretResolverStep(secretName, secretItemKey, kubeTokenManager)) - } - maybeExistingSecretStep.getOrElse(Some(new HadoopKerberosKeytabResolverStep( - conf, - kubernetesResourceNamePrefix, - maybePrincipal, - maybeKeytab, - maybeRenewerPrincipal, - kubeTokenManager))) - } else None - Seq(hadoopConfMounterStep) ++ maybeKerberosStep.toSeq - } -} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopConfigSpec.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/KerberosConfigSpec.scala similarity index 79% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopConfigSpec.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/KerberosConfigSpec.scala index 2b10838d8ac7..a9e6e2bfb8cf 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopConfigSpec.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/KerberosConfigSpec.scala @@ -16,22 +16,19 @@ */ package org.apache.spark.deploy.k8s.features.hadoopsteps -import io.fabric8.kubernetes.api.model.{EnvVar, Secret, Volume, VolumeMount} +import io.fabric8.kubernetes.api.model.Secret /** * Represents a given configuration of the hadoop configuration logic, informing the * HadoopConfigBootstrapStep of how the driver should be configured. This includes: *

- * - The properties that will be stored into the config map which have (key, value) - * pairs of (path, data) * - The secret containing a DT, either previously specified or built on the fly * - The name of the secret where the DT will be stored * - The data item-key on the secret which correlates with where the current DT data is stored * - The Job User's username */ -private[spark] case class HadoopConfigSpec( - configMapProperties: Map[String, String], +private[spark] case class KerberosConfigSpec( dtSecret: Option[Secret], dtSecretName: String, - dtSecretItemKey: Option[String], - jobUserName: Option[String]) + dtSecretItemKey: String, + jobUserName: String) 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 16db756bc85e..cb5f728c31d2 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 @@ -218,11 +218,6 @@ private[spark] class KubernetesClientApplication extends SparkApplication with L val waitForAppCompletion = sparkConf.get(WAIT_FOR_APP_COMPLETION) val kubernetesResourceNamePrefix = KubernetesClientApplication.getResourceNamePrefix(appName) sparkConf.set(KUBERNETES_PYSPARK_PY_FILES, clientArguments.maybePyFiles.getOrElse("")) - // Run driver as proxy user for Kerberos login by the HadoopUGI - if (UserGroupInformation.isSecurityEnabled) { - logInfo("Because Kerberos is enabled we should run driver as proxy user") - sparkConf.set(KUBERNETES_KERBEROS_PROXY_USER, "true") - } val kubernetesConf = KubernetesConf.createDriverConf( sparkConf, appName, 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 0b0a4827576b..87bf400e5387 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 @@ -182,7 +182,8 @@ class BasicExecutorFeatureStepSuite Map.empty, Map.empty, Nil, - Seq.empty[String])) + Seq.empty[String], + hadoopConfDir = None)) 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") From 78953e65fc496b1f58adc06e9876ec712241912b Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Fri, 21 Sep 2018 16:18:49 -0400 Subject: [PATCH 11/25] fixes so tests pass --- .../org/apache/spark/deploy/k8s/Config.scala | 11 +- .../spark/deploy/k8s/KubernetesConf.scala | 6 +- .../spark/deploy/k8s/KubernetesUtils.scala | 19 +- .../HadoopConfExecutorFeatureStep.scala | 2 +- .../HadoopGlobalFeatureDriverStep.scala | 218 +++++++++--------- .../KerberosConfExecutorFeatureStep.scala | 4 +- .../hadoopsteps/HadoopBootstrapUtil.scala | 178 +++++++------- .../hadoopsteps/HadoopKerberosLogin.scala | 8 +- .../hadoopsteps/KerberosConfigSpec.scala | 3 +- ...bernetesHadoopDelegationTokenManager.scala | 12 +- .../submit/KubernetesClientApplication.scala | 3 +- .../k8s/submit/KubernetesDriverBuilder.scala | 2 +- .../k8s/KubernetesExecutorBuilder.scala | 2 +- 13 files changed, 226 insertions(+), 242 deletions(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala index b9969a01ddbc..5fd73350f1dc 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 @@ -245,12 +245,6 @@ private[spark] object Config extends Logging { .stringConf .createOptional - val KUBERNETES_KERBEROS_RENEWER_PRINCIPAL = - ConfigBuilder("spark.kubernetes.kerberos.renewer.principal") - .doc("Specify the principal you wish to renew and retrieve your Kerberos values with") - .stringConf - .createOptional - val KUBERNETES_KERBEROS_DT_SECRET_NAME = ConfigBuilder("spark.kubernetes.kerberos.tokensecret.name") .doc("Specify the name of the secret where your existing delegation token is stored. " + @@ -260,9 +254,8 @@ private[spark] object Config extends Logging { val KUBERNETES_KERBEROS_DT_SECRET_ITEM_KEY = ConfigBuilder("spark.kubernetes.kerberos.tokensecret.itemkey") - .doc("Specify the item key of the data where " + - "your existing delegation token is stored. This removes the need " + - "for the job user to provide any keytab for launching a job") + .doc("Specify the item key of the data where your existing delegation token is stored. " + + "This removes the need for the job user to provide any keytab for launching a job") .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 5a54853f1cc4..26306a7d03d4 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 @@ -65,11 +65,11 @@ private[spark] case class KubernetesConf[T <: KubernetesRoleSpecificConf]( sparkFiles: Seq[String], hadoopConfDir: Option[String]) { - def getHadoopConfigMapName: String = s"$appResourceNamePrefix-hadoop-config" + def hadoopConfigMapName: String = s"$appResourceNamePrefix-hadoop-config" - def getKRBConfigMapName: String = s"$appResourceNamePrefix-krb5-file" + def kRBConfigMapName: String = s"$appResourceNamePrefix-krb5-file" - def getTokenManager : KubernetesHadoopDelegationTokenManager = + def tokenManager : KubernetesHadoopDelegationTokenManager = new KubernetesHadoopDelegationTokenManager def namespace(): String = sparkConf.get(KUBERNETES_NAMESPACE) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesUtils.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesUtils.scala index f05318095358..eb445f124187 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesUtils.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesUtils.scala @@ -40,12 +40,19 @@ private[spark] object KubernetesUtils { } def requireBothOrNeitherDefined( - opt1: Option[_], - opt2: Option[_], - errMessageWhenFirstIsMissing: String, - errMessageWhenSecondIsMissing: String): Unit = { - requireNandDefined(opt1, opt2, errMessageWhenSecondIsMissing) - requireNandDefined(opt2, opt1, errMessageWhenFirstIsMissing) + opt1: Option[_], + opt2: Option[_], + errMessageWhenFirstIsMissing: String, + errMessageWhenSecondIsMissing: String): Unit = { + requireSecondIfFirstIsDefined(opt1, opt2, errMessageWhenSecondIsMissing) + requireSecondIfFirstIsDefined(opt2, opt1, errMessageWhenFirstIsMissing) + } + + def requireSecondIfFirstIsDefined( + opt1: Option[_], opt2: Option[_], errMessageWhenSecondIsMissing: String): Unit = { + opt1.foreach { _ => + require(opt2.isDefined, errMessageWhenSecondIsMissing) + } } def requireNandDefined(opt1: Option[_], opt2: Option[_], errMessage: String): Unit = { 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 b1c9948a6c65..94797026f208 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 @@ -42,7 +42,7 @@ private[spark] class HadoopConfExecutorFeatureStep( HadoopBootstrapUtil.bootstrapHadoopConfDir( maybeHadoopConfDir.get, maybeHadoopConfigMap.get, - kubernetesConf.getTokenManager, + kubernetesConf.tokenManager, pod) } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopGlobalFeatureDriverStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopGlobalFeatureDriverStep.scala index 310f6d12c99a..ec7dc56ca9ca 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopGlobalFeatureDriverStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopGlobalFeatureDriverStep.scala @@ -37,128 +37,120 @@ import org.apache.spark.internal.Logging * SparkPod and Kubernetes Resources using the additive method of the feature steps */ private[spark] class HadoopGlobalFeatureDriverStep( - kubernetesConf: KubernetesConf[KubernetesDriverSpecificConf]) - extends KubernetesFeatureConfigStep with Logging { + kubernetesConf: KubernetesConf[KubernetesDriverSpecificConf]) + extends KubernetesFeatureConfigStep with Logging { - private val conf = kubernetesConf.sparkConf - private val maybePrincipal = conf.get(KUBERNETES_KERBEROS_PRINCIPAL) - private val maybeKeytab = conf.get(KUBERNETES_KERBEROS_KEYTAB) - .map(k => new File(k)) - private val maybeExistingSecretName = conf.get(KUBERNETES_KERBEROS_DT_SECRET_NAME) - private val maybeExistingSecretItemKey = - conf.get(KUBERNETES_KERBEROS_DT_SECRET_ITEM_KEY) - private val maybeRenewerPrincipal = - conf.get(KUBERNETES_KERBEROS_RENEWER_PRINCIPAL) - private val kubeTokenManager = kubernetesConf.getTokenManager - private val isKerberosEnabled = kubeTokenManager.isSecurityEnabled + private val conf = kubernetesConf.sparkConf + private val maybePrincipal = conf.get(KUBERNETES_KERBEROS_PRINCIPAL) + private val maybeKeytab = conf.get(KUBERNETES_KERBEROS_KEYTAB) + .map(k => new File(k)) + private val maybeExistingSecretName = conf.get(KUBERNETES_KERBEROS_DT_SECRET_NAME) + private val maybeExistingSecretItemKey = + conf.get(KUBERNETES_KERBEROS_DT_SECRET_ITEM_KEY) + private val kubeTokenManager = kubernetesConf.tokenManager + private val isKerberosEnabled = kubeTokenManager.isSecurityEnabled - require(maybeKeytab.forall( _ => isKerberosEnabled ), - "You must enable Kerberos support if you are specifying a Kerberos Keytab") + require(maybeKeytab.forall( _ => isKerberosEnabled ), + "You must enable Kerberos support if you are specifying a Kerberos Keytab") - require(maybeExistingSecretName.forall( _ => isKerberosEnabled ), - "You must enable Kerberos support if you are specifying a Kerberos Secret") + require(maybeExistingSecretName.forall( _ => isKerberosEnabled ), + "You must enable Kerberos support if you are specifying a Kerberos Secret") - KubernetesUtils.requireBothOrNeitherDefined( - maybeKeytab, - maybePrincipal, - "If a Kerberos keytab is specified you must also specify a Kerberos principal", - "If a Kerberos principal is specified you must also specify a Kerberos keytab") + KubernetesUtils.requireBothOrNeitherDefined( + maybeKeytab, + maybePrincipal, + "If a Kerberos principal is specified you must also specify a Kerberos keytab", + "If a Kerberos keytab is specified you must also specify a Kerberos principal") - KubernetesUtils.requireBothOrNeitherDefined( - maybeExistingSecretName, - maybeExistingSecretItemKey, - "If a secret storing a Kerberos Delegation Token is specified you must also" + - " specify the label where the data is stored", - "If a secret data item-key where the data of the Kerberos Delegation Token is specified" + - " you must also specify the name of the secret") + KubernetesUtils.requireBothOrNeitherDefined( + maybeExistingSecretName, + maybeExistingSecretItemKey, + "If a secret data item-key where the data of the Kerberos Delegation Token is specified" + + " you must also specify the name of the secret", + "If a secret storing a Kerberos Delegation Token is specified you must also" + + " specify the item-key where the data is stored") - // TODO: Using pre-existing configMaps instead of local HADOOP_CONF_DIR - require(kubernetesConf.hadoopConfDir.isDefined, "Ensure that HADOOP_CONF_DIR is defined") - private val hadoopConfDir = kubernetesConf.hadoopConfDir.get - private val hadoopConfigurationFiles = kubeTokenManager.getHadoopConfFiles(hadoopConfDir) + // TODO: Using pre-existing configMaps instead of local HADOOP_CONF_DIR + require(kubernetesConf.hadoopConfDir.isDefined, "Ensure that HADOOP_CONF_DIR is defined") + private val hadoopConfDir = kubernetesConf.hadoopConfDir.get + private val hadoopConfigurationFiles = kubeTokenManager.getHadoopConfFiles(hadoopConfDir) - // Either use pre-existing secret or login to create new Secret with DT stored within - private val hadoopSpec: Option[KerberosConfigSpec] = (for { - secretName <- maybeExistingSecretName - secretItemKey <- maybeExistingSecretItemKey - } yield { - KerberosConfigSpec( - dtSecret = None, - dtSecretName = secretName, - dtSecretItemKey = secretItemKey, - jobUserName = kubeTokenManager.getCurrentUser.getShortUserName) - }).orElse( - for { - _ <- maybePrincipal - _ <- maybeKeytab - _ <- maybeRenewerPrincipal - } yield { - HadoopKerberosLogin.buildSpec( - conf, - kubernetesConf.appResourceNamePrefix, - maybePrincipal, - maybeKeytab, - maybeRenewerPrincipal, - kubeTokenManager) - }) - - override def configurePod(pod: SparkPod): SparkPod = { - val hadoopBasedSparkPod = HadoopBootstrapUtil.bootstrapHadoopConfDir( - hadoopConfDir, - kubernetesConf.getHadoopConfigMapName, - kubeTokenManager, - pod) - (for { - hSpec <- hadoopSpec - krb5fileLocation <- kubernetesConf.get(KUBERNETES_KERBEROS_KRB5_FILE) + // Either use pre-existing secret or login to create new Secret with DT stored within + private val hadoopSpec: Option[KerberosConfigSpec] = (for { + secretName <- maybeExistingSecretName + secretItemKey <- maybeExistingSecretItemKey } yield { - HadoopBootstrapUtil.bootstrapKerberosPod( - hSpec.dtSecretName, - hSpec.dtSecretItemKey, - hSpec.jobUserName, - krb5fileLocation, - kubernetesConf.getKRBConfigMapName, - hadoopBasedSparkPod) - }).getOrElse( - HadoopBootstrapUtil.bootstrapSparkUserPod( - kubeTokenManager.getCurrentUser.getShortUserName, - hadoopBasedSparkPod)) - } + KerberosConfigSpec( + dtSecret = None, + dtSecretName = secretName, + dtSecretItemKey = secretItemKey, + jobUserName = kubeTokenManager.getCurrentUser.getShortUserName) + }).orElse( + if (isKerberosEnabled) { + Some(HadoopKerberosLogin.buildSpec( + conf, + kubernetesConf.appResourceNamePrefix, + maybePrincipal, + maybeKeytab, + kubeTokenManager)) + } else None ) - override def getAdditionalPodSystemProperties(): Map[String, String] = { - val resolvedConfValues = hadoopSpec.map{hSpec => - Map(KERBEROS_KEYTAB_SECRET_NAME -> hSpec.dtSecretName, - KERBEROS_KEYTAB_SECRET_KEY -> hSpec.dtSecretItemKey, - KERBEROS_SPARK_USER_NAME -> hSpec.jobUserName) - }.getOrElse( - Map(KERBEROS_SPARK_USER_NAME -> - kubernetesConf.getTokenManager.getCurrentUser.getShortUserName) - ) - Map(HADOOP_CONFIG_MAP_SPARK_CONF_NAME -> kubernetesConf.getHadoopConfigMapName, - HADOOP_CONF_DIR_LOC -> kubernetesConf.hadoopConfDir.get) ++ resolvedConfValues - } + override def configurePod(pod: SparkPod): SparkPod = { + val hadoopBasedSparkPod = HadoopBootstrapUtil.bootstrapHadoopConfDir( + hadoopConfDir, + kubernetesConf.hadoopConfigMapName, + kubeTokenManager, + pod) + (for { + hSpec <- hadoopSpec + krb5fileLocation <- kubernetesConf.get(KUBERNETES_KERBEROS_KRB5_FILE) + } yield { + HadoopBootstrapUtil.bootstrapKerberosPod( + hSpec.dtSecretName, + hSpec.dtSecretItemKey, + hSpec.jobUserName, + krb5fileLocation, + kubernetesConf.kRBConfigMapName, + hadoopBasedSparkPod) + }).getOrElse( + HadoopBootstrapUtil.bootstrapSparkUserPod( + kubeTokenManager.getCurrentUser.getShortUserName, + hadoopBasedSparkPod)) + } - override def getAdditionalKubernetesResources(): Seq[HasMetadata] = { - val krb5ConfigMap = kubernetesConf.get(KUBERNETES_KERBEROS_KRB5_FILE) - .map(fileLocation => HadoopBootstrapUtil.buildkrb5ConfigMap( - kubernetesConf.getKRBConfigMapName, - fileLocation)) - val kerberosDTSecret = for { - hSpec <- hadoopSpec - kDtSecret <- hSpec.dtSecret - } yield { - kDtSecret + override def getAdditionalPodSystemProperties(): Map[String, String] = { + val resolvedConfValues = hadoopSpec.map{ hSpec => + Map(KERBEROS_KEYTAB_SECRET_NAME -> hSpec.dtSecretName, + KERBEROS_KEYTAB_SECRET_KEY -> hSpec.dtSecretItemKey, + KERBEROS_SPARK_USER_NAME -> hSpec.jobUserName) + }.getOrElse( + Map(KERBEROS_SPARK_USER_NAME -> + kubernetesConf.tokenManager.getCurrentUser.getShortUserName)) + Map(HADOOP_CONFIG_MAP_SPARK_CONF_NAME -> kubernetesConf.hadoopConfigMapName, + HADOOP_CONF_DIR_LOC -> kubernetesConf.hadoopConfDir.get) ++ resolvedConfValues + } + + override def getAdditionalKubernetesResources(): Seq[HasMetadata] = { + val krb5ConfigMap = kubernetesConf.get(KUBERNETES_KERBEROS_KRB5_FILE) + .map(fileLocation => HadoopBootstrapUtil.buildkrb5ConfigMap( + kubernetesConf.kRBConfigMapName, + fileLocation)) + val kerberosDTSecret = for { + hSpec <- hadoopSpec + kDtSecret <- hSpec.dtSecret + } yield { + kDtSecret + } + val configMap = + new ConfigMapBuilder() + .withNewMetadata() + .withName(kubernetesConf.hadoopConfigMapName) + .endMetadata() + .addToData(hadoopConfigurationFiles.map(file => + (file.toPath.getFileName.toString, Files.toString(file, Charsets.UTF_8))).toMap.asJava) + .build() + Seq(configMap) ++ + krb5ConfigMap.toSeq ++ + kerberosDTSecret.toSeq } - val configMap = - new ConfigMapBuilder() - .withNewMetadata() - .withName(kubernetesConf.getHadoopConfigMapName) - .endMetadata() - .addToData(hadoopConfigurationFiles.map(file => - (file.toPath.getFileName.toString, Files.toString(file, Charsets.UTF_8))).toMap.asJava) - .build() - Seq(configMap) ++ - krb5ConfigMap.toSeq ++ - kerberosDTSecret.toSeq - } } 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 c5c5745eea60..48e9f6f5de17 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 @@ -46,7 +46,7 @@ private[spark] class KerberosConfExecutorFeatureStep( dTDataItemKey, sparkUserName, krb5Location, - kubernetesConf.getKRBConfigMapName, + kubernetesConf.kRBConfigMapName, pod) } @@ -55,7 +55,7 @@ private[spark] class KerberosConfExecutorFeatureStep( override def getAdditionalKubernetesResources(): Seq[HasMetadata] = { kubernetesConf.get(KUBERNETES_KERBEROS_KRB5_FILE) .map(fileLocation => HadoopBootstrapUtil.buildkrb5ConfigMap( - kubernetesConf.getKRBConfigMapName, + kubernetesConf.kRBConfigMapName, fileLocation)).toSeq } } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopBootstrapUtil.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopBootstrapUtil.scala index 54943c412722..574d31488e47 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopBootstrapUtil.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopBootstrapUtil.scala @@ -42,53 +42,53 @@ private[spark] object HadoopBootstrapUtil { * @return a modified SparkPod */ def bootstrapKerberosPod( - dtSecretName: String, - dtSecretItemKey: String, - userName: String, - fileLocation: String, - krb5ConfName: String, - pod: SparkPod) : SparkPod = { - val krb5File = new File(fileLocation) - val fileStringPath = krb5File.toPath.getFileName.toString - val kerberizedPod = new PodBuilder(pod.pod) - .editOrNewSpec() - .addNewVolume() - .withName(SPARK_APP_HADOOP_SECRET_VOLUME_NAME) - .withNewSecret() - .withSecretName(dtSecretName) - .endSecret() - .endVolume() - .addNewVolume() - .withName(KRB_FILE_VOLUME) - .withNewConfigMap() - .withName(krb5ConfName) - .withItems(new KeyToPathBuilder() - .withKey(fileStringPath) - .withPath(fileStringPath) - .build()) - .endConfigMap() - .endVolume() - .endSpec() - .build() - val kerberizedContainer = new ContainerBuilder(pod.container) - .addNewVolumeMount() + dtSecretName: String, + dtSecretItemKey: String, + userName: String, + fileLocation: String, + krb5ConfName: String, + pod: SparkPod) : SparkPod = { + val krb5File = new File(fileLocation) + val fileStringPath = krb5File.toPath.getFileName.toString + val kerberizedPod = new PodBuilder(pod.pod) + .editOrNewSpec() + .addNewVolume() .withName(SPARK_APP_HADOOP_SECRET_VOLUME_NAME) - .withMountPath(SPARK_APP_HADOOP_CREDENTIALS_BASE_DIR) - .endVolumeMount() - .addNewVolumeMount() + .withNewSecret() + .withSecretName(dtSecretName) + .endSecret() + .endVolume() + .addNewVolume() .withName(KRB_FILE_VOLUME) - .withMountPath(KRB_FILE_DIR_PATH + "/krb5.conf") - .withSubPath("krb5.conf") - .endVolumeMount() - .addNewEnv() - .withName(ENV_HADOOP_TOKEN_FILE_LOCATION) - .withValue(s"$SPARK_APP_HADOOP_CREDENTIALS_BASE_DIR/$dtSecretItemKey") - .endEnv() - .addNewEnv() - .withName(ENV_SPARK_USER) - .withValue(userName) - .endEnv() - .build() + .withNewConfigMap() + .withName(krb5ConfName) + .withItems(new KeyToPathBuilder() + .withKey(fileStringPath) + .withPath(fileStringPath) + .build()) + .endConfigMap() + .endVolume() + .endSpec() + .build() + val kerberizedContainer = new ContainerBuilder(pod.container) + .addNewVolumeMount() + .withName(SPARK_APP_HADOOP_SECRET_VOLUME_NAME) + .withMountPath(SPARK_APP_HADOOP_CREDENTIALS_BASE_DIR) + .endVolumeMount() + .addNewVolumeMount() + .withName(KRB_FILE_VOLUME) + .withMountPath(KRB_FILE_DIR_PATH + "/krb5.conf") + .withSubPath("krb5.conf") + .endVolumeMount() + .addNewEnv() + .withName(ENV_HADOOP_TOKEN_FILE_LOCATION) + .withValue(s"$SPARK_APP_HADOOP_CREDENTIALS_BASE_DIR/$dtSecretItemKey") + .endEnv() + .addNewEnv() + .withName(ENV_SPARK_USER) + .withValue(userName) + .endEnv() + .build() SparkPod(kerberizedPod, kerberizedContainer) } @@ -103,11 +103,11 @@ private[spark] object HadoopBootstrapUtil { sparkUserName: String, pod: SparkPod) : SparkPod = { val envModifiedContainer = new ContainerBuilder(pod.container) - .addNewEnv() - .withName(ENV_SPARK_USER) - .withValue(sparkUserName) - .endEnv() - .build() + .addNewEnv() + .withName(ENV_SPARK_USER) + .withValue(sparkUserName) + .endEnv() + .build() SparkPod(pod.pod, envModifiedContainer) } @@ -126,47 +126,47 @@ private[spark] object HadoopBootstrapUtil { hadoopConfigMapName: String, kubeTokenManager: KubernetesHadoopDelegationTokenManager, pod: SparkPod) : SparkPod = { - val hadoopConfigFiles = - kubeTokenManager.getHadoopConfFiles(hadoopConfDir) - val keyPaths = hadoopConfigFiles.map { file => - val fileStringPath = file.toPath.getFileName.toString - new KeyToPathBuilder() - .withKey(fileStringPath) - .withPath(fileStringPath) - .build() } + val hadoopConfigFiles = + kubeTokenManager.getHadoopConfFiles(hadoopConfDir) + val keyPaths = hadoopConfigFiles.map { file => + val fileStringPath = file.toPath.getFileName.toString + new KeyToPathBuilder() + .withKey(fileStringPath) + .withPath(fileStringPath) + .build() } - val hadoopSupportedPod = new PodBuilder(pod.pod) - .editSpec() - .addNewVolume() - .withName(HADOOP_FILE_VOLUME) - .withNewConfigMap() - .withName(hadoopConfigMapName) - .withItems(keyPaths.asJava) - .endConfigMap() - .endVolume() - .endSpec() - .build() + val hadoopSupportedPod = new PodBuilder(pod.pod) + .editSpec() + .addNewVolume() + .withName(HADOOP_FILE_VOLUME) + .withNewConfigMap() + .withName(hadoopConfigMapName) + .withItems(keyPaths.asJava) + .endConfigMap() + .endVolume() + .endSpec() + .build() - val hadoopSupportedContainer = new ContainerBuilder(pod.container) - .addNewVolumeMount() - .withName(HADOOP_FILE_VOLUME) - .withMountPath(HADOOP_CONF_DIR_PATH) - .endVolumeMount() - .addNewEnv() - .withName(ENV_HADOOP_CONF_DIR) - .withValue(HADOOP_CONF_DIR_PATH) - .endEnv() - .build() - SparkPod(hadoopSupportedPod, hadoopSupportedContainer) - } - /** - * bootstraping the container with ConfigMaps that store - * Hadoop configuration files - * - * @param configMapName name of configMap for krb5 - * @param fileLocation location of krb5 file - * @return a ConfigMap - */ + val hadoopSupportedContainer = new ContainerBuilder(pod.container) + .addNewVolumeMount() + .withName(HADOOP_FILE_VOLUME) + .withMountPath(HADOOP_CONF_DIR_PATH) + .endVolumeMount() + .addNewEnv() + .withName(ENV_HADOOP_CONF_DIR) + .withValue(HADOOP_CONF_DIR_PATH) + .endEnv() + .build() + SparkPod(hadoopSupportedPod, hadoopSupportedContainer) + } + /** + * bootstraping the container with ConfigMaps that store + * Hadoop configuration files + * + * @param configMapName name of configMap for krb5 + * @param fileLocation location of krb5 file + * @return a ConfigMap + */ def buildkrb5ConfigMap( configMapName: String, fileLocation: String) : ConfigMap = { diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopKerberosLogin.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopKerberosLogin.scala index e9caf240dbc7..0d8814abae32 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopKerberosLogin.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopKerberosLogin.scala @@ -31,7 +31,7 @@ import org.apache.spark.deploy.k8s.security.KubernetesHadoopDelegationTokenManag import org.apache.spark.deploy.security.HadoopDelegationTokenManager /** - * This step does all the heavy lifting for Delegation Token logic. This step + * 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. With a TGT stored locally, by running * UGI.getCurrentUser you are able to obtain the current user, alternatively @@ -39,8 +39,7 @@ import org.apache.spark.deploy.security.HadoopDelegationTokenManager * as the logged into user instead of the current user. 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 - * appended to the current HadoopSpec which in turn will append to the current - * DriverSpec. + * defined in a KerberosConfigSpec */ private[spark] object HadoopKerberosLogin { def buildSpec( @@ -48,7 +47,6 @@ private[spark] object HadoopKerberosLogin { kubernetesResourceNamePrefix : String, maybePrincipal: Option[String], maybeKeytab: Option[File], - maybeRenewerPrincipal: Option[String], tokenManager: KubernetesHadoopDelegationTokenManager): KerberosConfigSpec = { val hadoopConf = SparkHadoopUtil.get.newConfiguration(submissionSparkConf) if (!tokenManager.isSecurityEnabled) { @@ -78,7 +76,7 @@ private[spark] object HadoopKerberosLogin { hadoopConf, hadoopTokenManager) }}) - if (tokenData.isEmpty) throw new SparkException(s"Did not obtain any delegation tokens") + require(tokenData.nonEmpty, "Did not obtain any delegation tokens") val currentTime = tokenManager.getCurrentTime val initialTokenDataKeyName = s"$KERBEROS_SECRET_LABEL_PREFIX-$currentTime-$renewalInterval" val uniqueSecretName = diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/KerberosConfigSpec.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/KerberosConfigSpec.scala index a9e6e2bfb8cf..47111c3c91db 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/KerberosConfigSpec.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/KerberosConfigSpec.scala @@ -19,8 +19,7 @@ package org.apache.spark.deploy.k8s.features.hadoopsteps import io.fabric8.kubernetes.api.model.Secret /** - * Represents a given configuration of the hadoop configuration logic, informing the - * HadoopConfigBootstrapStep of how the driver should be configured. This includes: + * Represents a given configuration of the Kerberos Configuration logic *

* - The secret containing a DT, either previously specified or built on the fly * - The name of the secret where the DT will be stored 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 index 987e8b208708..e384c67e1f51 100644 --- 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 @@ -30,12 +30,9 @@ import org.apache.spark.internal.Logging import org.apache.spark.util.{Clock, SystemClock} /** - * The KubernetesHadoopDelegationTokenManager fetches and updates Hadoop delegation tokens - * on the behalf of the Kubernetes submission client. It is modeled after the YARN - * AMCredentialRenewer, renewals in Kubernetes happen in a seperate microservice that will - * automatically update the Tokens via Kubernetes Secrets. The principal difference is that - * instead of writing the new credentials to HDFS and incrementing the timestamp of the file, - * the new credentials (called Tokens when they are serialized) are stored in Secrets accessible + * The KubernetesHadoopDelegationTokenManager fetches Hadoop delegation tokens + * on the behalf of the Kubernetes submission client. The new credentials + * (called Tokens when they are serialized) are stored in Secrets accessible * to the driver and executors, when new Tokens are received they overwrite the current Secrets. */ private[spark] class KubernetesHadoopDelegationTokenManager extends Logging { @@ -62,7 +59,6 @@ private[spark] class KubernetesHadoopDelegationTokenManager extends Logging { } } - // Principle method in charge of retrieving new Delegation Tokens def getDelegationTokens( creds: Credentials, conf: SparkConf, @@ -70,7 +66,7 @@ private[spark] class KubernetesHadoopDelegationTokenManager extends Logging { tokenManager: HadoopDelegationTokenManager): (Array[Byte], Long) = { try { val rt = tokenManager.obtainDelegationTokens(hadoopConf, creds) - logDebug(s"Initialized tokens: ${SparkHadoopUtil.get.dumpTokens(creds)}") + logDebug(s"Initialized tokens") (serializeCreds(creds), nextRT(rt, conf)) } catch { case e: Exception => 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 cb5f728c31d2..94847462fa43 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 @@ -112,8 +112,7 @@ private[spark] class Client( def run(): Unit = { val resolvedDriverSpec = builder.buildFromFeatures(kubernetesConf) val configMapName = s"$kubernetesResourceNamePrefix-driver-conf-map" - val configMap = - buildConfigMap(configMapName, resolvedDriverSpec.systemProperties) + 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 val resolvedDriverContainer = new ContainerBuilder(resolvedDriverSpec.pod.container) 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 a6240b786bce..a0b422ec1cfb 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 @@ -17,7 +17,7 @@ package org.apache.spark.deploy.k8s.submit import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesDriverSpec, KubernetesDriverSpecificConf, KubernetesRoleSpecificConf} -import org.apache.spark.deploy.k8s.features.{BasicDriverFeatureStep, DriverKubernetesCredentialsFeatureStep, DriverServiceFeatureStep, EnvSecretsFeatureStep, HadoopGlobalFeatureDriverStep, KubernetesFeatureConfigStep, LocalDirsFeatureStep, MountSecretsFeatureStep, MountVolumesFeatureStep} +import org.apache.spark.deploy.k8s.features._ import org.apache.spark.deploy.k8s.features.bindings.{JavaDriverFeatureStep, PythonDriverFeatureStep, RDriverFeatureStep} private[spark] class KubernetesDriverBuilder( 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 eab19809c6af..974213bdf463 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 @@ -18,7 +18,7 @@ package org.apache.spark.scheduler.cluster.k8s import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesExecutorSpecificConf, KubernetesRoleSpecificConf, SparkPod} import org.apache.spark.deploy.k8s.Constants._ -import org.apache.spark.deploy.k8s.features.{BasicExecutorFeatureStep, EnvSecretsFeatureStep, HadoopConfExecutorFeatureStep, HadoopSparkUserExecutorFeatureStep, KerberosConfExecutorFeatureStep, KubernetesFeatureConfigStep, LocalDirsFeatureStep, MountSecretsFeatureStep, MountVolumesFeatureStep} +import org.apache.spark.deploy.k8s.features._ private[spark] class KubernetesExecutorBuilder( provideBasicStep: (KubernetesConf [KubernetesExecutorSpecificConf]) From 73f157f540a87e498b2e2c1b15c5d834355a6492 Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Wed, 26 Sep 2018 12:27:06 -0700 Subject: [PATCH 12/25] refactor to handle login logic being used in spark-submit --- .../org/apache/spark/deploy/k8s/Config.scala | 12 ------- .../HadoopGlobalFeatureDriverStep.scala | 7 ++-- .../hadoopsteps/HadoopKerberosLogin.scala | 35 ++++++------------- 3 files changed, 12 insertions(+), 42 deletions(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala index 5fd73350f1dc..2860350b690f 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 @@ -233,18 +233,6 @@ private[spark] object Config extends Logging { .stringConf .createOptional - val KUBERNETES_KERBEROS_KEYTAB = - ConfigBuilder("spark.kubernetes.kerberos.keytab") - .doc("Specify the location of keytab for Kerberos in order to access Secure HDFS") - .stringConf - .createOptional - - val KUBERNETES_KERBEROS_PRINCIPAL = - ConfigBuilder("spark.kubernetes.kerberos.principal") - .doc("Specify the principal for Kerberos in order to access Secure HDFS") - .stringConf - .createOptional - val KUBERNETES_KERBEROS_DT_SECRET_NAME = ConfigBuilder("spark.kubernetes.kerberos.tokensecret.name") .doc("Specify the name of the secret where your existing delegation token is stored. " + diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopGlobalFeatureDriverStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopGlobalFeatureDriverStep.scala index ec7dc56ca9ca..ae9b8db5071d 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopGlobalFeatureDriverStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopGlobalFeatureDriverStep.scala @@ -41,9 +41,8 @@ private[spark] class HadoopGlobalFeatureDriverStep( extends KubernetesFeatureConfigStep with Logging { private val conf = kubernetesConf.sparkConf - private val maybePrincipal = conf.get(KUBERNETES_KERBEROS_PRINCIPAL) - private val maybeKeytab = conf.get(KUBERNETES_KERBEROS_KEYTAB) - .map(k => new File(k)) + private val maybePrincipal = conf.get(org.apache.spark.internal.config.PRINCIPAL) + private val maybeKeytab = conf.get(org.apache.spark.internal.config.KEYTAB) private val maybeExistingSecretName = conf.get(KUBERNETES_KERBEROS_DT_SECRET_NAME) private val maybeExistingSecretItemKey = conf.get(KUBERNETES_KERBEROS_DT_SECRET_ITEM_KEY) @@ -90,8 +89,6 @@ private[spark] class HadoopGlobalFeatureDriverStep( Some(HadoopKerberosLogin.buildSpec( conf, kubernetesConf.appResourceNamePrefix, - maybePrincipal, - maybeKeytab, kubeTokenManager)) } else None ) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopKerberosLogin.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopKerberosLogin.scala index 0d8814abae32..af6fe1be81cb 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopKerberosLogin.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopKerberosLogin.scala @@ -45,37 +45,22 @@ private[spark] object HadoopKerberosLogin { def buildSpec( submissionSparkConf: SparkConf, kubernetesResourceNamePrefix : String, - maybePrincipal: Option[String], - maybeKeytab: Option[File], tokenManager: KubernetesHadoopDelegationTokenManager): KerberosConfigSpec = { val hadoopConf = SparkHadoopUtil.get.newConfiguration(submissionSparkConf) if (!tokenManager.isSecurityEnabled) { throw new SparkException("Hadoop not configured with Kerberos") } - val maybeJobUserUGI = - for { - principal <- maybePrincipal - keytab <- maybeKeytab - } yield { - tokenManager.loginUserFromKeytabAndReturnUGI( - principal, - keytab.toURI.toString) - } - // In the case that keytab is not specified we will read from Local Ticket Cache - val jobUserUGI = maybeJobUserUGI.getOrElse(tokenManager.getCurrentUser) + // 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 + val jobUserUGI = tokenManager.getCurrentUser val originalCredentials = jobUserUGI.getCredentials - // It is necessary to run as jobUserUGI because logged in user != Current User - val (tokenData, renewalInterval) = jobUserUGI.doAs( - new PrivilegedExceptionAction[(Array[Byte], Long)] { - override def run(): (Array[Byte], Long) = { - val hadoopTokenManager: HadoopDelegationTokenManager = - new HadoopDelegationTokenManager(submissionSparkConf, hadoopConf) - tokenManager.getDelegationTokens( - originalCredentials, - submissionSparkConf, - hadoopConf, - hadoopTokenManager) - }}) + val hadoopTokenManager: HadoopDelegationTokenManager = + new HadoopDelegationTokenManager(submissionSparkConf, hadoopConf) + val (tokenData, renewalInterval) = tokenManager.getDelegationTokens( + originalCredentials, + submissionSparkConf, + hadoopConf, + hadoopTokenManager) require(tokenData.nonEmpty, "Did not obtain any delegation tokens") val currentTime = tokenManager.getCurrentTime val initialTokenDataKeyName = s"$KERBEROS_SECRET_LABEL_PREFIX-$currentTime-$renewalInterval" From 5f52a1aa788fb13b7b2a841cfe04eb2309953cbd Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Wed, 26 Sep 2018 18:21:18 -0700 Subject: [PATCH 13/25] resolve comments and add documentation --- docs/running-on-kubernetes.md | 24 +++++++++++++++++++ docs/security.md | 24 +++++++++++-------- .../org/apache/spark/deploy/k8s/Config.scala | 2 +- .../HadoopConfExecutorFeatureStep.scala | 7 +++--- .../HadoopGlobalFeatureDriverStep.scala | 12 ++++------ .../hadoopsteps/HadoopBootstrapUtil.scala | 16 ++++++------- .../hadoopsteps/HadoopKerberosLogin.scala | 15 ++++-------- .../submit/KubernetesClientApplication.scala | 2 +- .../submit/KubernetesDriverBuilderSuite.scala | 2 +- 9 files changed, 62 insertions(+), 42 deletions(-) diff --git a/docs/running-on-kubernetes.md b/docs/running-on-kubernetes.md index 4ae7acaae231..c2f7f8490d0e 100644 --- a/docs/running-on-kubernetes.md +++ b/docs/running-on-kubernetes.md @@ -820,4 +820,28 @@ specific to Spark on Kubernetes. This sets the major Python version of the docker image used to run the driver and executor containers. Can either be 2 or 3. + + spark.kubernetes.kerberos.krb5location + (none) + + Specify the location of the krb5 file to be mounted on the driver and executors for Kerberos interaction. + It is important to note that for local files, the KDC defined needs to be visible from inside the containers. + + + + spark.kubernetes.kerberos.tokensecret.name + (none) + + Specify the name of the secret where your existing delegation token is stored. This removes the need for the job user + to provide any keytab for launching a job. + + + + spark.kubernetes.kerberos.tokensecret.itemkey + (none) + + Specify the item key of the data where your existing delegation token is stored. This removes the need for the job user + to provide any keytab for launching a job. + + diff --git a/docs/security.md b/docs/security.md index b147aade891a..8c5ffe5ad67a 100644 --- a/docs/security.md +++ b/docs/security.md @@ -729,6 +729,15 @@ so that non-local processes can authenticate. These delegation tokens in Kuberne shared by the Driver and its Executors. As such, there are three ways of submitting a kerberos job: In all cases you must define the environment variable: `HADOOP_CONF_DIR`. +It also important to note that the KDC needs to be visible from inside the containers if the user uses a local +krb5 file. + +If a user wishes to use a remote HADOOP_CONF directory, that contains the Hadoop configuration files, or +a remote krb5 file, this could be achieved by mounting a pre-defined ConfigMap and mounting the volume in the +desired location that you can point to via the appropriate configs. This method is useful for those who wish to not +rebuild their Docker images, but instead point to a ConfigMap that they could modify. This strategy is supported +via the pod-template feature. + 1. Submitting with a $kinit that stores a TGT in the Local Ticket Cache: ```bash /usr/bin/kinit -kt / @@ -740,7 +749,7 @@ In all cases you must define the environment variable: `HADOOP_CONF_DIR`. --conf spark.app.name=spark-hdfs \ --conf spark.kubernetes.container.image=spark:latest \ --conf spark.kubernetes.kerberos.krb5location=/etc/krb5.conf \ - local:///opt/spark/examples/jars/spark-examples_2.11-2.4.0-SNAPSHOT.jar \ + local:///opt/spark/examples/jars/spark-examples_-SNAPSHOT.jar \ ``` 2. Submitting with a local keytab and principal @@ -752,10 +761,10 @@ In all cases you must define the environment variable: `HADOOP_CONF_DIR`. --conf spark.executor.instances=1 \ --conf spark.app.name=spark-hdfs \ --conf spark.kubernetes.container.image=spark:latest \ - --conf spark.kubernetes.kerberos.keytab= \ - --conf spark.kubernetes.kerberos.principal= \ + --conf spark.kerberos.keytab= \ + --conf spark.kerberos.principal= \ --conf spark.kubernetes.kerberos.krb5location=/etc/krb5.conf \ - local:///opt/spark/examples/jars/spark-examples_2.11-2.4.0-SNAPSHOT.jar \ + local:///opt/spark/examples/jars/spark-examples_-SNAPSHOT.jar \ ``` @@ -771,15 +780,10 @@ In all cases you must define the environment variable: `HADOOP_CONF_DIR`. --conf spark.kubernetes.kerberos.tokensecret.name= \ --conf spark.kubernetes.kerberos.tokensecret.itemkey= \ --conf spark.kubernetes.kerberos.krb5location=/etc/krb5.conf \ - local:///opt/spark/examples/jars/spark-examples_2.11-2.4.0-SNAPSHOT.jar \ - + local:///opt/spark/examples/jars/spark-examples_-SNAPSHOT.jar \ ``` - - - - # Event Logging If your applications are using event logging, the directory where the event logs go 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 2860350b690f..48d65b599f70 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 @@ -228,7 +228,7 @@ private[spark] object Config extends Logging { val KUBERNETES_KERBEROS_KRB5_FILE = ConfigBuilder("spark.kubernetes.kerberos.krb5location") .doc("Specify the location of the krb5 file to be mounted on the driver " + - "and executors for Secure HDFS. Note: the KDC defined needs to be " + + "and executors for Secure HDFS. Note: For Local files the KDC defined needs to be " + "visible from inside the containers ") .stringConf .createOptional 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 94797026f208..2d1cb405908d 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 @@ -18,7 +18,7 @@ package org.apache.spark.deploy.k8s.features import io.fabric8.kubernetes.api.model.HasMetadata -import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesRoleSpecificConf, SparkPod} +import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesExecutorSpecificConf, SparkPod} import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.deploy.k8s.features.hadoopsteps.HadoopBootstrapUtil import org.apache.spark.internal.Logging @@ -26,11 +26,10 @@ import org.apache.spark.internal.Logging /** * This step is responsible for bootstraping the container with ConfigMaps * containing Hadoop config files mounted as volumes and an ENV variable - * pointed to the mounted file directory. This is run by both the driver - * and executor, as they both require Hadoop config files. + * pointed to the mounted file directory. */ private[spark] class HadoopConfExecutorFeatureStep( - kubernetesConf: KubernetesConf[_ <: KubernetesRoleSpecificConf]) + kubernetesConf: KubernetesConf[KubernetesExecutorSpecificConf]) extends KubernetesFeatureConfigStep with Logging{ override def configurePod(pod: SparkPod): SparkPod = { diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopGlobalFeatureDriverStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopGlobalFeatureDriverStep.scala index ae9b8db5071d..61b2c3e0aa41 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopGlobalFeatureDriverStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopGlobalFeatureDriverStep.scala @@ -32,9 +32,8 @@ import org.apache.spark.deploy.k8s.features.hadoopsteps._ import org.apache.spark.internal.Logging /** - * This is the main method that runs the hadoopConfigurationSteps defined - * by the HadoopStepsOrchestrator. These steps are run to modify the - * SparkPod and Kubernetes Resources using the additive method of the feature steps + * 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 HadoopGlobalFeatureDriverStep( kubernetesConf: KubernetesConf[KubernetesDriverSpecificConf]) @@ -69,7 +68,6 @@ private[spark] class HadoopGlobalFeatureDriverStep( "If a secret storing a Kerberos Delegation Token is specified you must also" + " specify the item-key where the data is stored") - // TODO: Using pre-existing configMaps instead of local HADOOP_CONF_DIR require(kubernetesConf.hadoopConfDir.isDefined, "Ensure that HADOOP_CONF_DIR is defined") private val hadoopConfDir = kubernetesConf.hadoopConfDir.get private val hadoopConfigurationFiles = kubeTokenManager.getHadoopConfFiles(hadoopConfDir) @@ -118,13 +116,13 @@ private[spark] class HadoopGlobalFeatureDriverStep( override def getAdditionalPodSystemProperties(): Map[String, String] = { val resolvedConfValues = hadoopSpec.map{ hSpec => Map(KERBEROS_KEYTAB_SECRET_NAME -> hSpec.dtSecretName, - KERBEROS_KEYTAB_SECRET_KEY -> hSpec.dtSecretItemKey, - KERBEROS_SPARK_USER_NAME -> hSpec.jobUserName) + KERBEROS_KEYTAB_SECRET_KEY -> hSpec.dtSecretItemKey, + KERBEROS_SPARK_USER_NAME -> hSpec.jobUserName) }.getOrElse( Map(KERBEROS_SPARK_USER_NAME -> kubernetesConf.tokenManager.getCurrentUser.getShortUserName)) Map(HADOOP_CONFIG_MAP_SPARK_CONF_NAME -> kubernetesConf.hadoopConfigMapName, - HADOOP_CONF_DIR_LOC -> kubernetesConf.hadoopConfDir.get) ++ resolvedConfValues + HADOOP_CONF_DIR_LOC -> kubernetesConf.hadoopConfDir.get) ++ resolvedConfValues } override def getAdditionalKubernetesResources(): Seq[HasMetadata] = { diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopBootstrapUtil.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopBootstrapUtil.scala index 574d31488e47..17f392efd184 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopBootstrapUtil.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopBootstrapUtil.scala @@ -100,14 +100,14 @@ private[spark] object HadoopBootstrapUtil { * @return a modified SparkPod */ def bootstrapSparkUserPod( - sparkUserName: String, - pod: SparkPod) : SparkPod = { - val envModifiedContainer = new ContainerBuilder(pod.container) - .addNewEnv() - .withName(ENV_SPARK_USER) - .withValue(sparkUserName) - .endEnv() - .build() + sparkUserName: String, + pod: SparkPod) : SparkPod = { + val envModifiedContainer = new ContainerBuilder(pod.container) + .addNewEnv() + .withName(ENV_SPARK_USER) + .withValue(sparkUserName) + .endEnv() + .build() SparkPod(pod.pod, envModifiedContainer) } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopKerberosLogin.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopKerberosLogin.scala index af6fe1be81cb..beb8872b02df 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopKerberosLogin.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopKerberosLogin.scala @@ -16,9 +16,6 @@ */ package org.apache.spark.deploy.k8s.features.hadoopsteps -import java.io._ -import java.security.PrivilegedExceptionAction - import scala.collection.JavaConverters._ import io.fabric8.kubernetes.api.model.SecretBuilder @@ -33,13 +30,11 @@ import org.apache.spark.deploy.security.HadoopDelegationTokenManager /** * 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. With a TGT stored locally, by running - * UGI.getCurrentUser you are able to obtain the current user, alternatively - * you can run UGI.loginUserFromKeytabAndReturnUGI and by running .doAs run - * as the logged into user instead of the current user. 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 + * $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( 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 94847462fa43..e495e519baf6 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 @@ -200,7 +200,7 @@ private[spark] class Client( /** * Main class and entry point of application submission in KUBERNETES mode. */ -private[spark] class KubernetesClientApplication extends SparkApplication with Logging { +private[spark] class KubernetesClientApplication extends SparkApplication { override def start(args: Array[String], conf: SparkConf): Unit = { val parsedArguments = ClientArguments.fromCommandLineArgs(args) 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 48c86c88bd12..bd3ba5992efa 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 @@ -18,7 +18,7 @@ package org.apache.spark.deploy.k8s.submit import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.deploy.k8s._ -import org.apache.spark.deploy.k8s.features.{BasicDriverFeatureStep, DriverKubernetesCredentialsFeatureStep, DriverServiceFeatureStep, EnvSecretsFeatureStep, HadoopGlobalFeatureDriverStep, KubernetesFeaturesTestUtils, LocalDirsFeatureStep, MountSecretsFeatureStep, MountVolumesFeatureStep} +import org.apache.spark.deploy.k8s.features._ import org.apache.spark.deploy.k8s.features.bindings.{JavaDriverFeatureStep, PythonDriverFeatureStep, RDriverFeatureStep} class KubernetesDriverBuilderSuite extends SparkFunSuite { From 6548ef9932bc49854d0a2968a153a926623b44aa Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Sat, 6 Oct 2018 22:17:12 +0100 Subject: [PATCH 14/25] resolved comments --- .../org/apache/spark/deploy/SparkSubmit.scala | 8 +- docs/running-on-kubernetes.md | 16 ++- docs/security.md | 21 ++-- .../org/apache/spark/deploy/k8s/Config.scala | 18 ++- .../apache/spark/deploy/k8s/Constants.scala | 4 +- .../spark/deploy/k8s/KubernetesUtils.scala | 4 +- .../HadoopConfExecutorFeatureStep.scala | 22 ++-- .../HadoopSparkUserExecutorFeatureStep.scala | 8 +- ...la => KerberosConfDriverFeatureStep.scala} | 44 +++++--- .../KerberosConfExecutorFeatureStep.scala | 30 ++--- .../HadoopBootstrapUtil.scala | 106 ++++++++++++------ .../HadoopKerberosLogin.scala | 5 +- .../KerberosConfigSpec.scala | 2 +- ...bernetesHadoopDelegationTokenManager.scala | 12 -- .../k8s/submit/KubernetesDriverBuilder.scala | 4 +- .../k8s/KubernetesExecutorBuilder.scala | 2 +- .../submit/KubernetesDriverBuilderSuite.scala | 2 +- .../k8s/KubernetesExecutorBuilderSuite.scala | 4 +- 18 files changed, 182 insertions(+), 130 deletions(-) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/{HadoopGlobalFeatureDriverStep.scala => KerberosConfDriverFeatureStep.scala} (82%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/{hadoopsteps => hadooputils}/HadoopBootstrapUtil.scala (67%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/{hadoopsteps => hadooputils}/HadoopKerberosLogin.scala (94%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/{hadoopsteps => hadooputils}/KerberosConfigSpec.scala (95%) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index 76a474857a5c..9d753192f76c 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -647,8 +647,8 @@ private[spark] class SparkSubmit extends Logging { } if ((clusterManager == MESOS || clusterManager == KUBERNETES) - && UserGroupInformation.isSecurityEnabled) { - setRMPrincipal(sparkConf) + && UserGroupInformation.isSecurityEnabled) { + setRMPrincipal(sparkConf) } // In yarn-cluster mode, use yarn.Client as a wrapper around the user class @@ -763,8 +763,8 @@ private[spark] class SparkSubmit extends Logging { } // [SPARK-20328]. HadoopRDD calls into a Hadoop library that fetches delegation tokens with - // renewer set to the YARN ResourceManager. Since YARN isn't configured in Mesos mode, we - // must trick it into thinking we're YARN. Same is on for Kubernetes. + // renewer set to the YARN ResourceManager. Since YARN isn't configured in Mesos or Kubernetes + // mode, we must trick it into thinking we're YARN. private def setRMPrincipal(sparkConf: SparkConf): Unit = { val shortUserName = UserGroupInformation.getCurrentUser.getShortUserName val key = s"spark.hadoop.${YarnConfiguration.RM_PRINCIPAL}" diff --git a/docs/running-on-kubernetes.md b/docs/running-on-kubernetes.md index c2f7f8490d0e..df84f98ef3fc 100644 --- a/docs/running-on-kubernetes.md +++ b/docs/running-on-kubernetes.md @@ -821,15 +821,23 @@ specific to Spark on Kubernetes. - spark.kubernetes.kerberos.krb5location + spark.kubernetes.kerberos.krb5.location (none) - Specify the location of the krb5 file to be mounted on the driver and executors for Kerberos interaction. + Specify the local location of the krb5 file to be mounted on the driver and executors for Kerberos interaction. It is important to note that for local files, the KDC defined needs to be visible from inside the containers. - spark.kubernetes.kerberos.tokensecret.name + spark.kubernetes.kerberos.krb5.configMapName + (none) + + Specify the name of the ConfigMap, containing the krb5 file, to be mounted on the driver and executors + for Kerberos interaction. The KDC defined needs to be visible from inside the containers. + + + + spark.kubernetes.kerberos.tokenSecret.name (none) Specify the name of the secret where your existing delegation token is stored. This removes the need for the job user @@ -837,7 +845,7 @@ specific to Spark on Kubernetes. - spark.kubernetes.kerberos.tokensecret.itemkey + spark.kubernetes.kerberos.tokenSecret.itemKey (none) Specify the item key of the data where your existing delegation token is stored. This removes the need for the job user diff --git a/docs/security.md b/docs/security.md index 8c5ffe5ad67a..244bddd83566 100644 --- a/docs/security.md +++ b/docs/security.md @@ -728,14 +728,15 @@ When talking to Hadoop-based services behind Kerberos, it was noted that Spark n so that non-local processes can authenticate. These delegation tokens in Kubernetes are stored in Secrets that are shared by the Driver and its Executors. As such, there are three ways of submitting a kerberos job: -In all cases you must define the environment variable: `HADOOP_CONF_DIR`. +In all cases you must define the environment variable: `HADOOP_CONF_DIR` as well as either +`spark.kubernetes.kerberos.krb5.location` or `spark.kubernetes.kerberos.krb5.configMapName`. + It also important to note that the KDC needs to be visible from inside the containers if the user uses a local krb5 file. -If a user wishes to use a remote HADOOP_CONF directory, that contains the Hadoop configuration files, or -a remote krb5 file, this could be achieved by mounting a pre-defined ConfigMap and mounting the volume in the -desired location that you can point to via the appropriate configs. This method is useful for those who wish to not -rebuild their Docker images, but instead point to a ConfigMap that they could modify. This strategy is supported +If a user wishes to use a remote HADOOP_CONF directory, that contains the Hadoop configuration files, this could be achieved by mounting a pre-defined ConfigMap in the desired location +that you can point to via the appropriate configs. This method is useful for those who wish to not rebuild +their Docker images, but instead point to a ConfigMap that they could modify. This strategy is supported via the pod-template feature. 1. Submitting with a $kinit that stores a TGT in the Local Ticket Cache: @@ -748,7 +749,7 @@ via the pod-template feature. --conf spark.executor.instances=1 \ --conf spark.app.name=spark-hdfs \ --conf spark.kubernetes.container.image=spark:latest \ - --conf spark.kubernetes.kerberos.krb5location=/etc/krb5.conf \ + --conf spark.kubernetes.kerberos.krb5.locationn=/etc/krb5.conf \ local:///opt/spark/examples/jars/spark-examples_-SNAPSHOT.jar \ ``` @@ -763,7 +764,7 @@ via the pod-template feature. --conf spark.kubernetes.container.image=spark:latest \ --conf spark.kerberos.keytab= \ --conf spark.kerberos.principal= \ - --conf spark.kubernetes.kerberos.krb5location=/etc/krb5.conf \ + --conf spark.kubernetes.kerberos.krb5.location=/etc/krb5.conf \ local:///opt/spark/examples/jars/spark-examples_-SNAPSHOT.jar \ ``` @@ -777,9 +778,9 @@ via the pod-template feature. --conf spark.executor.instances=1 \ --conf spark.app.name=spark-hdfs \ --conf spark.kubernetes.container.image=spark:latest \ - --conf spark.kubernetes.kerberos.tokensecret.name= \ - --conf spark.kubernetes.kerberos.tokensecret.itemkey= \ - --conf spark.kubernetes.kerberos.krb5location=/etc/krb5.conf \ + --conf spark.kubernetes.kerberos.tokenSecret.name= \ + --conf spark.kubernetes.kerberos.tokenSecret.itemKey= \ + --conf spark.kubernetes.kerberos.krb5.location=/etc/krb5.conf \ local:///opt/spark/examples/jars/spark-examples_-SNAPSHOT.jar \ ``` 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 48d65b599f70..0286ee7db304 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 @@ -226,22 +226,30 @@ private[spark] object Config extends Logging { .createWithDefault("2") val KUBERNETES_KERBEROS_KRB5_FILE = - ConfigBuilder("spark.kubernetes.kerberos.krb5location") - .doc("Specify the location of the krb5 file to be mounted on the driver " + - "and executors for Secure HDFS. Note: For Local files the KDC defined needs to be " + + ConfigBuilder("spark.kubernetes.kerberos.krb5.location") + .doc("Specify the local location of the krb5 file to be mounted on the driver " + + "and executors for Kerberos. Note: For Local files the KDC defined needs to be " + "visible from inside the containers ") .stringConf .createOptional + val KUBERNETES_KERBEROS_KRB5_CONFIG_MAP = + ConfigBuilder("spark.kubernetes.kerberos.krb5.configMapName") + .doc("Specify the name of the ConfigMap, containing the krb5 file, to be mounted " + + "on the driver and executors for Kerberos. Note: For Local files the KDC defined" + + "needs to be visible from inside the containers ") + .stringConf + .createOptional + val KUBERNETES_KERBEROS_DT_SECRET_NAME = - ConfigBuilder("spark.kubernetes.kerberos.tokensecret.name") + ConfigBuilder("spark.kubernetes.kerberos.tokenSecret.name") .doc("Specify the name of the secret where your existing delegation token is stored. " + "This removes the need for the job user to provide any keytab for launching a job") .stringConf .createOptional val KUBERNETES_KERBEROS_DT_SECRET_ITEM_KEY = - ConfigBuilder("spark.kubernetes.kerberos.tokensecret.itemkey") + ConfigBuilder("spark.kubernetes.kerberos.tokenSecret.itemKey") .doc("Specify the item key of the data where your existing delegation token is stored. " + "This removes the need for the job user to provide any keytab for launching a job") .stringConf diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Constants.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Constants.scala index bbe561229247..b2d8e568d20f 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Constants.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Constants.scala @@ -88,8 +88,8 @@ private[spark] object Constants { val KRB_FILE_DIR_PATH = "/etc" val ENV_HADOOP_CONF_DIR = "HADOOP_CONF_DIR" val HADOOP_CONF_DIR_LOC = "spark.kubernetes.hadoop.conf.dir" - val HADOOP_CONFIG_MAP_SPARK_CONF_NAME = - "spark.kubernetes.hadoop.executor.hadoopConfigMapName" + val HADOOP_CONFIG_MAP_NAME = + "spark.kubernetes.executor.hadoopConfigMapName" // Kerberos Configuration val KERBEROS_DELEGEGATION_TOKEN_SECRET_NAME = diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesUtils.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesUtils.scala index eb445f124187..d9a1aee77f78 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesUtils.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesUtils.scala @@ -49,7 +49,9 @@ private[spark] object KubernetesUtils { } def requireSecondIfFirstIsDefined( - opt1: Option[_], opt2: Option[_], errMessageWhenSecondIsMissing: String): Unit = { + opt1: Option[_], + opt2: Option[_], + errMessageWhenSecondIsMissing: String): Unit = { opt1.foreach { _ => require(opt2.isDefined, errMessageWhenSecondIsMissing) } 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 2d1cb405908d..ebfc24356078 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 @@ -20,7 +20,7 @@ import io.fabric8.kubernetes.api.model.HasMetadata import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesExecutorSpecificConf, SparkPod} import org.apache.spark.deploy.k8s.Constants._ -import org.apache.spark.deploy.k8s.features.hadoopsteps.HadoopBootstrapUtil +import org.apache.spark.deploy.k8s.features.hadooputils.HadoopBootstrapUtil import org.apache.spark.internal.Logging /** @@ -33,16 +33,16 @@ private[spark] class HadoopConfExecutorFeatureStep( extends KubernetesFeatureConfigStep with Logging{ override def configurePod(pod: SparkPod): SparkPod = { - val maybeHadoopConfDir = kubernetesConf.sparkConf.getOption(HADOOP_CONF_DIR_LOC) - val maybeHadoopConfigMap = kubernetesConf.sparkConf.getOption(HADOOP_CONFIG_MAP_SPARK_CONF_NAME) - require(maybeHadoopConfDir.isDefined && maybeHadoopConfigMap.isDefined, - "Ensure that HADOOP_CONF_DIR is defined") - logInfo("HADOOP_CONF_DIR defined. Mounting Hadoop specific files") - HadoopBootstrapUtil.bootstrapHadoopConfDir( - maybeHadoopConfDir.get, - maybeHadoopConfigMap.get, - kubernetesConf.tokenManager, - pod) + val maybeHadoopConfDir = kubernetesConf.sparkConf.getOption(HADOOP_CONF_DIR_LOC) + val maybeHadoopConfigMap = kubernetesConf.sparkConf.getOption(HADOOP_CONFIG_MAP_NAME) + require(maybeHadoopConfDir.isDefined && maybeHadoopConfigMap.isDefined, + "Ensure that HADOOP_CONF_DIR is defined") + logInfo("HADOOP_CONF_DIR defined. Mounting Hadoop specific files") + HadoopBootstrapUtil.bootstrapHadoopConfDir( + maybeHadoopConfDir.get, + maybeHadoopConfigMap.get, + kubernetesConf.tokenManager, + pod) } override def getAdditionalPodSystemProperties(): Map[String, String] = Map.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 d2c4c01d5ca3..60128394583b 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 @@ -21,7 +21,7 @@ import io.fabric8.kubernetes.api.model.HasMetadata import org.apache.spark.deploy.k8s.{KubernetesConf, SparkPod} import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.deploy.k8s.KubernetesExecutorSpecificConf -import org.apache.spark.deploy.k8s.features.hadoopsteps.HadoopBootstrapUtil +import org.apache.spark.deploy.k8s.features.hadooputils.HadoopBootstrapUtil import org.apache.spark.internal.Logging /** @@ -33,10 +33,8 @@ import org.apache.spark.internal.Logging extends KubernetesFeatureConfigStep with Logging{ override def configurePod(pod: SparkPod): SparkPod = { - val sparkUserName = kubernetesConf.sparkConf.get(KERBEROS_SPARK_USER_NAME) - HadoopBootstrapUtil.bootstrapSparkUserPod( - sparkUserName, - pod) + val sparkUserName = kubernetesConf.sparkConf.get(KERBEROS_SPARK_USER_NAME) + HadoopBootstrapUtil.bootstrapSparkUserPod(sparkUserName, pod) } override def getAdditionalPodSystemProperties(): Map[String, String] = Map.empty diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopGlobalFeatureDriverStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/KerberosConfDriverFeatureStep.scala similarity index 82% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopGlobalFeatureDriverStep.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/KerberosConfDriverFeatureStep.scala index 61b2c3e0aa41..4fdd2d144cae 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopGlobalFeatureDriverStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/KerberosConfDriverFeatureStep.scala @@ -16,8 +16,6 @@ */ package org.apache.spark.deploy.k8s.features -import java.io.File - import scala.collection.JavaConverters._ import com.google.common.base.Charsets @@ -28,14 +26,14 @@ import org.apache.spark.deploy.k8s.{KubernetesConf, 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.hadoopsteps._ +import org.apache.spark.deploy.k8s.features.hadooputils._ import org.apache.spark.internal.Logging /** * 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 HadoopGlobalFeatureDriverStep( +private[spark] class KerberosConfDriverFeatureStep( kubernetesConf: KubernetesConf[KubernetesDriverSpecificConf]) extends KubernetesFeatureConfigStep with Logging { @@ -45,15 +43,28 @@ private[spark] class HadoopGlobalFeatureDriverStep( private val maybeExistingSecretName = conf.get(KUBERNETES_KERBEROS_DT_SECRET_NAME) private val maybeExistingSecretItemKey = conf.get(KUBERNETES_KERBEROS_DT_SECRET_ITEM_KEY) + private val maybeKrb5File = + conf.get(KUBERNETES_KERBEROS_KRB5_FILE) + private val maybeKrb5CMap = + conf.get(KUBERNETES_KERBEROS_KRB5_CONFIG_MAP) private val kubeTokenManager = kubernetesConf.tokenManager private val isKerberosEnabled = kubeTokenManager.isSecurityEnabled - require(maybeKeytab.forall( _ => isKerberosEnabled ), + require(maybeKeytab.isEmpty || isKerberosEnabled, "You must enable Kerberos support if you are specifying a Kerberos Keytab") - require(maybeExistingSecretName.forall( _ => isKerberosEnabled ), + require(maybeExistingSecretName.isEmpty || isKerberosEnabled, "You must enable Kerberos support if you are specifying a Kerberos Secret") + require((maybeKrb5File.isEmpty || maybeKrb5CMap.isEmpty) || isKerberosEnabled, + "You must specify either a krb5 file location or a ConfigMap with a krb5 file") + + KubernetesUtils.requireNandDefined( + maybeKrb5File, + maybeKrb5CMap, + "Do not specify both a Krb5 local file and the ConfigMap as the creation" + + "of an additional ConfigMap, when one is already specified, is extraneous") + KubernetesUtils.requireBothOrNeitherDefined( maybeKeytab, maybePrincipal, @@ -70,7 +81,7 @@ private[spark] class HadoopGlobalFeatureDriverStep( require(kubernetesConf.hadoopConfDir.isDefined, "Ensure that HADOOP_CONF_DIR is defined") private val hadoopConfDir = kubernetesConf.hadoopConfDir.get - private val hadoopConfigurationFiles = kubeTokenManager.getHadoopConfFiles(hadoopConfDir) + private val hadoopConfigurationFiles = HadoopBootstrapUtil.getHadoopConfFiles(hadoopConfDir) // Either use pre-existing secret or login to create new Secret with DT stored within private val hadoopSpec: Option[KerberosConfigSpec] = (for { @@ -96,18 +107,16 @@ private[spark] class HadoopGlobalFeatureDriverStep( kubernetesConf.hadoopConfigMapName, kubeTokenManager, pod) - (for { - hSpec <- hadoopSpec - krb5fileLocation <- kubernetesConf.get(KUBERNETES_KERBEROS_KRB5_FILE) - } yield { + hadoopSpec.map { hSpec => HadoopBootstrapUtil.bootstrapKerberosPod( hSpec.dtSecretName, hSpec.dtSecretItemKey, hSpec.jobUserName, - krb5fileLocation, + maybeKrb5File, kubernetesConf.kRBConfigMapName, + maybeKrb5CMap, hadoopBasedSparkPod) - }).getOrElse( + }.getOrElse( HadoopBootstrapUtil.bootstrapSparkUserPod( kubeTokenManager.getCurrentUser.getShortUserName, hadoopBasedSparkPod)) @@ -121,15 +130,16 @@ private[spark] class HadoopGlobalFeatureDriverStep( }.getOrElse( Map(KERBEROS_SPARK_USER_NAME -> kubernetesConf.tokenManager.getCurrentUser.getShortUserName)) - Map(HADOOP_CONFIG_MAP_SPARK_CONF_NAME -> kubernetesConf.hadoopConfigMapName, + Map(HADOOP_CONFIG_MAP_NAME -> kubernetesConf.hadoopConfigMapName, HADOOP_CONF_DIR_LOC -> kubernetesConf.hadoopConfDir.get) ++ resolvedConfValues } override def getAdditionalKubernetesResources(): Seq[HasMetadata] = { - val krb5ConfigMap = kubernetesConf.get(KUBERNETES_KERBEROS_KRB5_FILE) - .map(fileLocation => HadoopBootstrapUtil.buildkrb5ConfigMap( + val krb5ConfigMap = + maybeKrb5File.map { fileLocation => + HadoopBootstrapUtil.buildkrb5ConfigMap( kubernetesConf.kRBConfigMapName, - fileLocation)) + fileLocation)} val kerberosDTSecret = for { hSpec <- hadoopSpec kDtSecret <- hSpec.dtSecret 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 48e9f6f5de17..93db0f6d6310 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 @@ -18,12 +18,11 @@ package org.apache.spark.deploy.k8s.features import io.fabric8.kubernetes.api.model.HasMetadata -import org.apache.spark.SparkException -import org.apache.spark.deploy.k8s.{KubernetesConf, SparkPod} -import org.apache.spark.deploy.k8s.Config.KUBERNETES_KERBEROS_KRB5_FILE +import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesUtils, SparkPod} +import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.deploy.k8s.KubernetesExecutorSpecificConf -import org.apache.spark.deploy.k8s.features.hadoopsteps.HadoopBootstrapUtil +import org.apache.spark.deploy.k8s.features.hadooputils.HadoopBootstrapUtil import org.apache.spark.internal.Logging /** @@ -35,18 +34,21 @@ private[spark] class KerberosConfExecutorFeatureStep( override def configurePod(pod: SparkPod): SparkPod = { val sparkConf = kubernetesConf.sparkConf - val dTSecretName = sparkConf.get(KERBEROS_KEYTAB_SECRET_NAME) - val dTDataItemKey = sparkConf.get(KERBEROS_KEYTAB_SECRET_KEY) - val krb5Location = sparkConf.get(KUBERNETES_KERBEROS_KRB5_FILE) - .getOrElse(throw new SparkException("Must specify krb5 file location")) - val sparkUserName = sparkConf.get(KERBEROS_SPARK_USER_NAME) - logInfo(s"Mounting HDFS DT from Secret $dTSecretName for Secure HDFS") + val maybeKrb5File = sparkConf.get(KUBERNETES_KERBEROS_KRB5_FILE) + val maybeKrb5CMap = sparkConf.get(KUBERNETES_KERBEROS_KRB5_CONFIG_MAP) + KubernetesUtils.requireNandDefined( + maybeKrb5File, + maybeKrb5CMap, + "Do not specify both a Krb5 local file and the ConfigMap as the creation" + + "of an additional ConfigMap, when one is already specified, is extraneous") + logInfo(s"Mounting HDFS DT for Secure HDFS") HadoopBootstrapUtil.bootstrapKerberosPod( - dTSecretName, - dTDataItemKey, - sparkUserName, - krb5Location, + sparkConf.get(KERBEROS_KEYTAB_SECRET_NAME), + sparkConf.get(KERBEROS_KEYTAB_SECRET_KEY), + sparkConf.get(KERBEROS_SPARK_USER_NAME), + maybeKrb5File, kubernetesConf.kRBConfigMapName, + maybeKrb5CMap, pod) } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopBootstrapUtil.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadooputils/HadoopBootstrapUtil.scala similarity index 67% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopBootstrapUtil.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadooputils/HadoopBootstrapUtil.scala index 17f392efd184..b312f910208e 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopBootstrapUtil.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadooputils/HadoopBootstrapUtil.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.k8s.features.hadoopsteps +package org.apache.spark.deploy.k8s.features.hadooputils import java.io.File @@ -22,8 +22,9 @@ import scala.collection.JavaConverters._ import com.google.common.base.Charsets import com.google.common.io.Files -import io.fabric8.kubernetes.api.model.{ConfigMap, ConfigMapBuilder, ContainerBuilder, KeyToPathBuilder, PodBuilder} +import io.fabric8.kubernetes.api.model._ +import org.apache.spark.SparkException import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.deploy.k8s.SparkPod import org.apache.spark.deploy.k8s.security.KubernetesHadoopDelegationTokenManager @@ -36,8 +37,9 @@ private[spark] object HadoopBootstrapUtil { * @param dtSecretName Name of the secret that stores the Delegation Token * @param dtSecretItemKey Name of the Item Key storing the Delegation Token * @param userName Name of the SparkUser to set SPARK_USER - * @param fileLocation Location of the krb5 file - * @param krb5ConfName Name of the ConfigMap for Krb5 + * @param maybeFileLocation Optional Location of the krb5 file + * @param newKrb5ConfName Optiona location of the ConfigMap for Krb5 + * @param oldKrb5ConfName Optional name of ConfigMap for Krb5 * @param pod Input pod to be appended to * @return a modified SparkPod */ @@ -45,11 +47,41 @@ private[spark] object HadoopBootstrapUtil { dtSecretName: String, dtSecretItemKey: String, userName: String, - fileLocation: String, - krb5ConfName: String, + maybeFileLocation: Option[String], + newKrb5ConfName: String, + maybeKrb5ConfName: Option[String], pod: SparkPod) : SparkPod = { - val krb5File = new File(fileLocation) - val fileStringPath = krb5File.toPath.getFileName.toString + + val maybePreConfigMapVolume = maybeKrb5ConfName.map { kconf => + new VolumeBuilder() + .withNewConfigMap() + .withName(kconf) + .endConfigMap() + .build() } + + val maybeCreateConfigMapVolume = maybeFileLocation.map { + fileLocation => + val krb5File = new File(fileLocation) + val fileStringPath = krb5File.toPath.getFileName.toString + new VolumeBuilder() + .withName(KRB_FILE_VOLUME) + .withNewConfigMap() + .withName(newKrb5ConfName) + .withItems(new KeyToPathBuilder() + .withKey(fileStringPath) + .withPath(fileStringPath) + .build()) + .endConfigMap() + .build() } + + // Breaking up Volume Creation for clarity + val configMapVolume = + maybePreConfigMapVolume.getOrElse( + maybeCreateConfigMapVolume.getOrElse( + throw new SparkException( + "Must specify krb5 file locally or via ConfigMap") + )) + val kerberizedPod = new PodBuilder(pod.pod) .editOrNewSpec() .addNewVolume() @@ -58,16 +90,7 @@ private[spark] object HadoopBootstrapUtil { .withSecretName(dtSecretName) .endSecret() .endVolume() - .addNewVolume() - .withName(KRB_FILE_VOLUME) - .withNewConfigMap() - .withName(krb5ConfName) - .withItems(new KeyToPathBuilder() - .withKey(fileStringPath) - .withPath(fileStringPath) - .build()) - .endConfigMap() - .endVolume() + .withVolumes(configMapVolume) .endSpec() .build() val kerberizedContainer = new ContainerBuilder(pod.container) @@ -99,9 +122,7 @@ private[spark] object HadoopBootstrapUtil { * @param pod Input pod to be appended to * @return a modified SparkPod */ - def bootstrapSparkUserPod( - sparkUserName: String, - pod: SparkPod) : SparkPod = { + def bootstrapSparkUserPod(sparkUserName: String, pod: SparkPod) : SparkPod = { val envModifiedContainer = new ContainerBuilder(pod.container) .addNewEnv() .withName(ENV_SPARK_USER) @@ -111,6 +132,21 @@ private[spark] object HadoopBootstrapUtil { SparkPod(pod.pod, envModifiedContainer) } + /** + * Grabbing files in the HADOOP_CONF_DIR + * + * @param path location of HADOOP_CONF_DIR + * @return a list of File object + */ + def getHadoopConfFiles(path: String) : Seq[File] = { + val dir = new File(path) + if (dir.isDirectory) { + dir.listFiles.flatMap { file => Some(file).filter(_.isFile) }.toSeq + } else { + Seq.empty[File] + } + } + /** * Bootstraping the container with ConfigMaps that store * Hadoop configuration files @@ -126,8 +162,7 @@ private[spark] object HadoopBootstrapUtil { hadoopConfigMapName: String, kubeTokenManager: KubernetesHadoopDelegationTokenManager, pod: SparkPod) : SparkPod = { - val hadoopConfigFiles = - kubeTokenManager.getHadoopConfFiles(hadoopConfDir) + val hadoopConfigFiles = getHadoopConfFiles(hadoopConfDir) val keyPaths = hadoopConfigFiles.map { file => val fileStringPath = file.toPath.getFileName.toString new KeyToPathBuilder() @@ -159,24 +194,25 @@ private[spark] object HadoopBootstrapUtil { .build() SparkPod(hadoopSupportedPod, hadoopSupportedContainer) } + /** - * bootstraping the container with ConfigMaps that store - * Hadoop configuration files + * Builds ConfigMap given the file location of the + * krb5.conf file * * @param configMapName name of configMap for krb5 * @param fileLocation location of krb5 file * @return a ConfigMap */ def buildkrb5ConfigMap( - configMapName: String, - fileLocation: String) : ConfigMap = { - val file = new File(fileLocation) - new ConfigMapBuilder() - .withNewMetadata() - .withName(configMapName) - .endMetadata() - .addToData( - Map(file.toPath.getFileName.toString -> Files.toString(file, Charsets.UTF_8)).asJava) - .build() + configMapName: String, + fileLocation: String) : ConfigMap = { + val file = new File(fileLocation) + new ConfigMapBuilder() + .withNewMetadata() + .withName(configMapName) + .endMetadata() + .addToData( + Map(file.toPath.getFileName.toString -> Files.toString(file, Charsets.UTF_8)).asJava) + .build() } } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopKerberosLogin.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadooputils/HadoopKerberosLogin.scala similarity index 94% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopKerberosLogin.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadooputils/HadoopKerberosLogin.scala index beb8872b02df..820b050689a7 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopKerberosLogin.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadooputils/HadoopKerberosLogin.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.k8s.features.hadoopsteps +package org.apache.spark.deploy.k8s.features.hadooputils import scala.collection.JavaConverters._ @@ -49,8 +49,7 @@ private[spark] object HadoopKerberosLogin { // The login happens in the SparkSubmit so login logic is not necessary val jobUserUGI = tokenManager.getCurrentUser val originalCredentials = jobUserUGI.getCredentials - val hadoopTokenManager: HadoopDelegationTokenManager = - new HadoopDelegationTokenManager(submissionSparkConf, hadoopConf) + val hadoopTokenManager = new HadoopDelegationTokenManager(submissionSparkConf, hadoopConf) val (tokenData, renewalInterval) = tokenManager.getDelegationTokens( originalCredentials, submissionSparkConf, diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/KerberosConfigSpec.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadooputils/KerberosConfigSpec.scala similarity index 95% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/KerberosConfigSpec.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadooputils/KerberosConfigSpec.scala index 47111c3c91db..4ae19cbde9ac 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/KerberosConfigSpec.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadooputils/KerberosConfigSpec.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.k8s.features.hadoopsteps +package org.apache.spark.deploy.k8s.features.hadooputils import io.fabric8.kubernetes.api.model.Secret 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 index e384c67e1f51..8539b66f2cba 100644 --- 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 @@ -17,8 +17,6 @@ package org.apache.spark.deploy.k8s.security -import java.io.File - import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.FileSystem import org.apache.hadoop.security.{Credentials, UserGroupInformation} @@ -49,16 +47,6 @@ private[spark] class KubernetesHadoopDelegationTokenManager extends Logging { def serializeCreds(creds: Credentials): Array[Byte] = SparkHadoopUtil.get.serialize(creds) def nextRT(rt: Long, conf: SparkConf): Long = SparkHadoopUtil.nextCredentialRenewalTime(rt, conf) - // Grab files in the HADOOP_CONF directory - def getHadoopConfFiles(path: String) : Seq[File] = { - val dir = new File(path) - if (dir.isDirectory) { - dir.listFiles.flatMap { file => Some(file).filter(_.isFile) }.toSeq - } else { - Seq.empty[File] - } - } - def getDelegationTokens( creds: Credentials, conf: SparkConf, 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 a0b422ec1cfb..09c8a2b0d699 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 @@ -54,8 +54,8 @@ private[spark] class KubernetesDriverBuilder( new JavaDriverFeatureStep(_), provideHadoopGlobalStep: ( KubernetesConf[KubernetesDriverSpecificConf] - => HadoopGlobalFeatureDriverStep) = - new HadoopGlobalFeatureDriverStep(_)) { + => KerberosConfDriverFeatureStep) = + new KerberosConfDriverFeatureStep(_)) { def buildFromFeatures( kubernetesConf: KubernetesConf[KubernetesDriverSpecificConf]): KubernetesDriverSpec = { 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 974213bdf463..638308e7862e 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 @@ -52,7 +52,7 @@ private[spark] class KubernetesExecutorBuilder( def buildFromFeatures( kubernetesConf: KubernetesConf[KubernetesExecutorSpecificConf]): SparkPod = { val sparkConf = kubernetesConf.sparkConf - val maybeHadoopConfigMap = sparkConf.getOption(HADOOP_CONFIG_MAP_SPARK_CONF_NAME) + val maybeHadoopConfigMap = sparkConf.getOption(HADOOP_CONFIG_MAP_NAME) val maybeDTSecretName = sparkConf.getOption(KERBEROS_KEYTAB_SECRET_NAME) val maybeDTDataItem = sparkConf.getOption(KERBEROS_KEYTAB_SECRET_KEY) 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 bd3ba5992efa..ea17f2ca43b3 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 @@ -63,7 +63,7 @@ class KubernetesDriverBuilderSuite extends SparkFunSuite { ENV_SECRETS_STEP_TYPE, classOf[EnvSecretsFeatureStep]) private val hadoopGlobalStep = KubernetesFeaturesTestUtils.getMockConfigStepForStepType( - HADOOP_GLOBAL_STEP_TYPE, classOf[HadoopGlobalFeatureDriverStep]) + HADOOP_GLOBAL_STEP_TYPE, classOf[KerberosConfDriverFeatureStep]) private val mountVolumesStep = KubernetesFeaturesTestUtils.getMockConfigStepForStepType( MOUNT_VOLUMES_STEP_TYPE, classOf[MountVolumesFeatureStep]) 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 0dad9f822201..a14c1b1a5c78 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 @@ -135,7 +135,7 @@ class KubernetesExecutorBuilderSuite extends SparkFunSuite { val HADOOPFS_PROVIDER = s"$HADOOP_CREDS_PREFIX.hadoopfs.enabled" val conf = KubernetesConf( new SparkConf(false) - .set(HADOOP_CONFIG_MAP_SPARK_CONF_NAME, "hadoop-conf-map-loc") + .set(HADOOP_CONFIG_MAP_NAME, "hadoop-conf-map-loc") .set(HADOOP_CONF_DIR_LOC, "hadoop-conf-dir-loc") .set(KERBEROS_SPARK_USER_NAME, "spark-user") .set(HADOOPFS_PROVIDER, "true"), @@ -162,7 +162,7 @@ class KubernetesExecutorBuilderSuite extends SparkFunSuite { test("Apply kerberos step if DT secrets created") { val conf = KubernetesConf( new SparkConf(false) - .set(HADOOP_CONFIG_MAP_SPARK_CONF_NAME, "hadoop-conf-map-loc") + .set(HADOOP_CONFIG_MAP_NAME, "hadoop-conf-map-loc") .set(HADOOP_CONF_DIR_LOC, "hadoop-conf-dir-loc") .set(KERBEROS_SPARK_USER_NAME, "spark-user") .set(KERBEROS_KEYTAB_SECRET_NAME, "dt-secret") From 7f72af5d9bcd36ff7033467a165b54790b9f1b51 Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Sat, 6 Oct 2018 22:52:58 +0100 Subject: [PATCH 15/25] resolved rest of comments --- .../spark/deploy/k8s/KubernetesConf.scala | 7 +++++-- .../HadoopConfExecutorFeatureStep.scala | 1 - .../KerberosConfDriverFeatureStep.scala | 7 ++++--- .../KerberosConfExecutorFeatureStep.scala | 20 +++++++++---------- .../hadooputils/HadoopBootstrapUtil.scala | 5 +---- .../hadooputils/HadoopKerberosLogin.scala | 7 ++----- ...bernetesHadoopDelegationTokenManager.scala | 6 +++--- 7 files changed, 25 insertions(+), 28 deletions(-) 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 26306a7d03d4..7c33e24e814e 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 @@ -19,6 +19,7 @@ package org.apache.spark.deploy.k8s import scala.collection.mutable 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._ @@ -26,6 +27,7 @@ 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.deploy.security.HadoopDelegationTokenManager import org.apache.spark.internal.config.ConfigEntry @@ -69,8 +71,9 @@ private[spark] case class KubernetesConf[T <: KubernetesRoleSpecificConf]( def kRBConfigMapName: String = s"$appResourceNamePrefix-krb5-file" - def tokenManager : KubernetesHadoopDelegationTokenManager = - new KubernetesHadoopDelegationTokenManager + def tokenManager(conf: SparkConf, hConf: Configuration) : KubernetesHadoopDelegationTokenManager = + new KubernetesHadoopDelegationTokenManager( + new HadoopDelegationTokenManager(conf, hConf)) def namespace(): String = sparkConf.get(KUBERNETES_NAMESPACE) 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 ebfc24356078..5a3b57c7e2d5 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 @@ -41,7 +41,6 @@ private[spark] class HadoopConfExecutorFeatureStep( HadoopBootstrapUtil.bootstrapHadoopConfDir( maybeHadoopConfDir.get, maybeHadoopConfigMap.get, - kubernetesConf.tokenManager, 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 4fdd2d144cae..6ff0d11b3274 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 @@ -22,6 +22,7 @@ import com.google.common.base.Charsets import com.google.common.io.Files import io.fabric8.kubernetes.api.model.{ConfigMapBuilder, HasMetadata} +import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesUtils, SparkPod} import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ @@ -47,7 +48,8 @@ private[spark] class KerberosConfDriverFeatureStep( conf.get(KUBERNETES_KERBEROS_KRB5_FILE) private val maybeKrb5CMap = conf.get(KUBERNETES_KERBEROS_KRB5_CONFIG_MAP) - private val kubeTokenManager = kubernetesConf.tokenManager + private val kubeTokenManager = kubernetesConf.tokenManager(conf, + SparkHadoopUtil.get.newConfiguration(conf)) private val isKerberosEnabled = kubeTokenManager.isSecurityEnabled require(maybeKeytab.isEmpty || isKerberosEnabled, @@ -105,7 +107,6 @@ private[spark] class KerberosConfDriverFeatureStep( val hadoopBasedSparkPod = HadoopBootstrapUtil.bootstrapHadoopConfDir( hadoopConfDir, kubernetesConf.hadoopConfigMapName, - kubeTokenManager, pod) hadoopSpec.map { hSpec => HadoopBootstrapUtil.bootstrapKerberosPod( @@ -129,7 +130,7 @@ private[spark] class KerberosConfDriverFeatureStep( KERBEROS_SPARK_USER_NAME -> hSpec.jobUserName) }.getOrElse( Map(KERBEROS_SPARK_USER_NAME -> - kubernetesConf.tokenManager.getCurrentUser.getShortUserName)) + kubeTokenManager.getCurrentUser.getShortUserName)) Map(HADOOP_CONFIG_MAP_NAME -> kubernetesConf.hadoopConfigMapName, HADOOP_CONF_DIR_LOC -> kubernetesConf.hadoopConfDir.get) ++ resolvedConfValues } 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 93db0f6d6310..3e167cedce7b 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 @@ -31,17 +31,17 @@ import org.apache.spark.internal.Logging private[spark] class KerberosConfExecutorFeatureStep( kubernetesConf: KubernetesConf[KubernetesExecutorSpecificConf]) extends KubernetesFeatureConfigStep with Logging{ + private val sparkConf = kubernetesConf.sparkConf + private val maybeKrb5File = sparkConf.get(KUBERNETES_KERBEROS_KRB5_FILE) + private val maybeKrb5CMap = sparkConf.get(KUBERNETES_KERBEROS_KRB5_CONFIG_MAP) + KubernetesUtils.requireNandDefined( + maybeKrb5File, + maybeKrb5CMap, + "Do not specify both a Krb5 local file and the ConfigMap as the creation" + + "of an additional ConfigMap, when one is already specified, is extraneous") override def configurePod(pod: SparkPod): SparkPod = { - val sparkConf = kubernetesConf.sparkConf - val maybeKrb5File = sparkConf.get(KUBERNETES_KERBEROS_KRB5_FILE) - val maybeKrb5CMap = sparkConf.get(KUBERNETES_KERBEROS_KRB5_CONFIG_MAP) - KubernetesUtils.requireNandDefined( - maybeKrb5File, - maybeKrb5CMap, - "Do not specify both a Krb5 local file and the ConfigMap as the creation" + - "of an additional ConfigMap, when one is already specified, is extraneous") - logInfo(s"Mounting HDFS DT for Secure HDFS") + logInfo(s"Mounting Kerberos DT for Kerberos") HadoopBootstrapUtil.bootstrapKerberosPod( sparkConf.get(KERBEROS_KEYTAB_SECRET_NAME), sparkConf.get(KERBEROS_KEYTAB_SECRET_KEY), @@ -55,7 +55,7 @@ private[spark] class KerberosConfExecutorFeatureStep( override def getAdditionalPodSystemProperties(): Map[String, String] = Map.empty override def getAdditionalKubernetesResources(): Seq[HasMetadata] = { - kubernetesConf.get(KUBERNETES_KERBEROS_KRB5_FILE) + maybeKrb5File .map(fileLocation => HadoopBootstrapUtil.buildkrb5ConfigMap( kubernetesConf.kRBConfigMapName, fileLocation)).toSeq diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadooputils/HadoopBootstrapUtil.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadooputils/HadoopBootstrapUtil.scala index b312f910208e..fee6b1a407fe 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadooputils/HadoopBootstrapUtil.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadooputils/HadoopBootstrapUtil.scala @@ -27,7 +27,6 @@ import io.fabric8.kubernetes.api.model._ import org.apache.spark.SparkException import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.deploy.k8s.SparkPod -import org.apache.spark.deploy.k8s.security.KubernetesHadoopDelegationTokenManager private[spark] object HadoopBootstrapUtil { @@ -39,7 +38,7 @@ private[spark] object HadoopBootstrapUtil { * @param userName Name of the SparkUser to set SPARK_USER * @param maybeFileLocation Optional Location of the krb5 file * @param newKrb5ConfName Optiona location of the ConfigMap for Krb5 - * @param oldKrb5ConfName Optional name of ConfigMap for Krb5 + * @param maybeKrb5ConfName Optional name of ConfigMap for Krb5 * @param pod Input pod to be appended to * @return a modified SparkPod */ @@ -153,14 +152,12 @@ private[spark] object HadoopBootstrapUtil { * * @param hadoopConfDir location of HADOOP_CONF_DIR * @param hadoopConfigMapName name of the configMap for HADOOP_CONF_DIR - * @param kubeTokenManager KubernetesHadoopDelegationTokenManager * @param pod Input pod to be appended to * @return a modified SparkPod */ def bootstrapHadoopConfDir( hadoopConfDir: String, hadoopConfigMapName: String, - kubeTokenManager: KubernetesHadoopDelegationTokenManager, pod: SparkPod) : SparkPod = { val hadoopConfigFiles = getHadoopConfFiles(hadoopConfDir) val keyPaths = hadoopConfigFiles.map { file => 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 index 820b050689a7..b944a5d8e540 100644 --- 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 @@ -25,7 +25,6 @@ import org.apache.spark.{SparkConf, SparkException} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.deploy.k8s.security.KubernetesHadoopDelegationTokenManager -import org.apache.spark.deploy.security.HadoopDelegationTokenManager /** * This logic does all the heavy lifting for Delegation Token creation. This step @@ -46,15 +45,13 @@ private[spark] object HadoopKerberosLogin { throw new SparkException("Hadoop not configured with Kerberos") } // 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 + // The login happens in the SparkSubmit so login logic is not necessary to include val jobUserUGI = tokenManager.getCurrentUser val originalCredentials = jobUserUGI.getCredentials - val hadoopTokenManager = new HadoopDelegationTokenManager(submissionSparkConf, hadoopConf) val (tokenData, renewalInterval) = tokenManager.getDelegationTokens( originalCredentials, submissionSparkConf, - hadoopConf, - hadoopTokenManager) + hadoopConf) require(tokenData.nonEmpty, "Did not obtain any delegation tokens") val currentTime = tokenManager.getCurrentTime val initialTokenDataKeyName = s"$KERBEROS_SECRET_LABEL_PREFIX-$currentTime-$renewalInterval" 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 index 8539b66f2cba..2f4b7927939d 100644 --- 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 @@ -33,7 +33,8 @@ import org.apache.spark.util.{Clock, SystemClock} * (called Tokens when they are serialized) are stored in Secrets accessible * to the driver and executors, when new Tokens are received they overwrite the current Secrets. */ -private[spark] class KubernetesHadoopDelegationTokenManager extends Logging { +private[spark] class KubernetesHadoopDelegationTokenManager( + tokenManager: HadoopDelegationTokenManager) extends Logging { // HadoopUGI Util methods private val clock: Clock = new SystemClock() @@ -50,8 +51,7 @@ private[spark] class KubernetesHadoopDelegationTokenManager extends Logging { def getDelegationTokens( creds: Credentials, conf: SparkConf, - hadoopConf: Configuration, - tokenManager: HadoopDelegationTokenManager): (Array[Byte], Long) = { + hadoopConf: Configuration): (Array[Byte], Long) = { try { val rt = tokenManager.obtainDelegationTokens(hadoopConf, creds) logDebug(s"Initialized tokens") From 4ce00a54ddc7083204855c118aa09d8df77e3c40 Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Sat, 6 Oct 2018 23:06:46 +0100 Subject: [PATCH 16/25] small doc addition --- docs/security.md | 25 ++++++++++++++++++++----- 1 file changed, 20 insertions(+), 5 deletions(-) diff --git a/docs/security.md b/docs/security.md index 244bddd83566..8a47fa2e37c7 100644 --- a/docs/security.md +++ b/docs/security.md @@ -726,7 +726,7 @@ tokens for supported will be created. When talking to Hadoop-based services behind Kerberos, it was noted that Spark needs to obtain delegation tokens so that non-local processes can authenticate. These delegation tokens in Kubernetes are stored in Secrets that are -shared by the Driver and its Executors. As such, there are three ways of submitting a kerberos job: +shared by the Driver and its Executors. As such, there are three ways of submitting a Kerberos job: In all cases you must define the environment variable: `HADOOP_CONF_DIR` as well as either `spark.kubernetes.kerberos.krb5.location` or `spark.kubernetes.kerberos.krb5.configMapName`. @@ -734,10 +734,10 @@ In all cases you must define the environment variable: `HADOOP_CONF_DIR` as well It also important to note that the KDC needs to be visible from inside the containers if the user uses a local krb5 file. -If a user wishes to use a remote HADOOP_CONF directory, that contains the Hadoop configuration files, this could be achieved by mounting a pre-defined ConfigMap in the desired location -that you can point to via the appropriate configs. This method is useful for those who wish to not rebuild -their Docker images, but instead point to a ConfigMap that they could modify. This strategy is supported -via the pod-template feature. +If a user wishes to use a remote HADOOP_CONF directory, that contains the Hadoop configuration files, this could be +achieved by mounting a pre-defined ConfigMap in the desired location that you can point to via the appropriate configs. +This method is useful for those who wish to not rebuild their Docker images, but instead point to a ConfigMap that they +could modify. This strategy is supported via the pod-template feature. 1. Submitting with a $kinit that stores a TGT in the Local Ticket Cache: ```bash @@ -785,6 +785,21 @@ via the pod-template feature. ``` +3b. Submitting like in (3) however specifying a pre-created krb5 config map +```bash +/opt/spark/bin/spark-submit \ + --deploy-mode cluster \ + --class org.apache.spark.examples.HdfsTest \ + --master k8s:// \ + --conf spark.executor.instances=1 \ + --conf spark.app.name=spark-hdfs \ + --conf spark.kubernetes.container.image=spark:latest \ + --conf spark.kubernetes.kerberos.tokenSecret.name= \ + --conf spark.kubernetes.kerberos.tokenSecret.itemKey= \ + --conf spark.kubernetes.kerberos.krb5.configMapName= \ + local:///opt/spark/examples/jars/spark-examples_-SNAPSHOT.jar \ + +``` # Event Logging If your applications are using event logging, the directory where the event logs go From 89063fdfa76184bb87bcdf1f4b193f3571200fac Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Sat, 6 Oct 2018 19:13:07 -0700 Subject: [PATCH 17/25] fixes to pass kerberos tests --- .../deploy/k8s/features/HadoopConfExecutorFeatureStep.scala | 5 +++-- .../deploy/k8s/features/KerberosConfDriverFeatureStep.scala | 5 ++--- .../k8s/features/KerberosConfExecutorFeatureStep.scala | 2 +- .../k8s/features/hadooputils/HadoopBootstrapUtil.scala | 4 +++- 4 files changed, 9 insertions(+), 7 deletions(-) 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 5a3b57c7e2d5..7de1e81e08f4 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 @@ -33,8 +33,9 @@ private[spark] class HadoopConfExecutorFeatureStep( extends KubernetesFeatureConfigStep with Logging{ override def configurePod(pod: SparkPod): SparkPod = { - val maybeHadoopConfDir = kubernetesConf.sparkConf.getOption(HADOOP_CONF_DIR_LOC) - val maybeHadoopConfigMap = kubernetesConf.sparkConf.getOption(HADOOP_CONFIG_MAP_NAME) + val sparkConf = kubernetesConf.sparkConf + val maybeHadoopConfDir = sparkConf.getOption(HADOOP_CONF_DIR_LOC) + val maybeHadoopConfigMap = sparkConf.getOption(HADOOP_CONFIG_MAP_NAME) require(maybeHadoopConfDir.isDefined && maybeHadoopConfigMap.isDefined, "Ensure that HADOOP_CONF_DIR is defined") logInfo("HADOOP_CONF_DIR defined. Mounting Hadoop specific files") 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 6ff0d11b3274..8b660587f623 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 @@ -64,7 +64,7 @@ private[spark] class KerberosConfDriverFeatureStep( KubernetesUtils.requireNandDefined( maybeKrb5File, maybeKrb5CMap, - "Do not specify both a Krb5 local file and the ConfigMap as the creation" + + "Do not specify both a Krb5 local file and the ConfigMap as the creation " + "of an additional ConfigMap, when one is already specified, is extraneous") KubernetesUtils.requireBothOrNeitherDefined( @@ -136,8 +136,7 @@ private[spark] class KerberosConfDriverFeatureStep( } override def getAdditionalKubernetesResources(): Seq[HasMetadata] = { - val krb5ConfigMap = - maybeKrb5File.map { fileLocation => + val krb5ConfigMap = maybeKrb5File.map { fileLocation => HadoopBootstrapUtil.buildkrb5ConfigMap( kubernetesConf.kRBConfigMapName, fileLocation)} 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 3e167cedce7b..8dea4c2ca536 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 @@ -37,7 +37,7 @@ private[spark] class KerberosConfExecutorFeatureStep( KubernetesUtils.requireNandDefined( maybeKrb5File, maybeKrb5CMap, - "Do not specify both a Krb5 local file and the ConfigMap as the creation" + + "Do not specify both a Krb5 local file and the ConfigMap as the creation " + "of an additional ConfigMap, when one is already specified, is extraneous") override def configurePod(pod: SparkPod): SparkPod = { diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadooputils/HadoopBootstrapUtil.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadooputils/HadoopBootstrapUtil.scala index fee6b1a407fe..5d7de6d62f5e 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadooputils/HadoopBootstrapUtil.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadooputils/HadoopBootstrapUtil.scala @@ -53,6 +53,7 @@ private[spark] object HadoopBootstrapUtil { val maybePreConfigMapVolume = maybeKrb5ConfName.map { kconf => new VolumeBuilder() + .withName(KRB_FILE_VOLUME) .withNewConfigMap() .withName(kconf) .endConfigMap() @@ -89,7 +90,8 @@ private[spark] object HadoopBootstrapUtil { .withSecretName(dtSecretName) .endSecret() .endVolume() - .withVolumes(configMapVolume) + .addNewVolumeLike(configMapVolume) + .endVolume() .endSpec() .build() val kerberizedContainer = new ContainerBuilder(pod.container) From e30304810c1090c1eaf69cd0302f1621b1f44644 Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Mon, 8 Oct 2018 13:09:40 -0700 Subject: [PATCH 18/25] resolve comments --- docs/running-on-kubernetes.md | 3 ++- docs/security.md | 9 ++++---- .../spark/deploy/k8s/KubernetesUtils.scala | 1 + .../HadoopConfExecutorFeatureStep.scala | 5 ++-- .../KerberosConfDriverFeatureStep.scala | 11 +++------ .../KerberosConfExecutorFeatureStep.scala | 7 +----- .../hadooputils/HadoopBootstrapUtil.scala | 23 ++++++++++++++++++- 7 files changed, 37 insertions(+), 22 deletions(-) diff --git a/docs/running-on-kubernetes.md b/docs/running-on-kubernetes.md index df84f98ef3fc..667f7de65798 100644 --- a/docs/running-on-kubernetes.md +++ b/docs/running-on-kubernetes.md @@ -833,7 +833,8 @@ specific to Spark on Kubernetes. (none) Specify the name of the ConfigMap, containing the krb5 file, to be mounted on the driver and executors - for Kerberos interaction. The KDC defined needs to be visible from inside the containers. + for Kerberos interaction. The KDC defined needs to be visible from inside the containers. The ConfigMap must also + be in the same namespace of the driver and executor pods. diff --git a/docs/security.md b/docs/security.md index 8a47fa2e37c7..406cde56f250 100644 --- a/docs/security.md +++ b/docs/security.md @@ -735,7 +735,8 @@ It also important to note that the KDC needs to be visible from inside the conta krb5 file. If a user wishes to use a remote HADOOP_CONF directory, that contains the Hadoop configuration files, this could be -achieved by mounting a pre-defined ConfigMap in the desired location that you can point to via the appropriate configs. +achieved by setting the environmental variable `HADOOP_CONF_DIR` on the container to be pointed to the path where the +pre-created ConfigMap is mounted. This method is useful for those who wish to not rebuild their Docker images, but instead point to a ConfigMap that they could modify. This strategy is supported via the pod-template feature. @@ -753,7 +754,7 @@ could modify. This strategy is supported via the pod-template feature. local:///opt/spark/examples/jars/spark-examples_-SNAPSHOT.jar \ ``` -2. Submitting with a local keytab and principal +2. Submitting with a local Keytab and Principal ```bash /opt/spark/bin/spark-submit \ --deploy-mode cluster \ @@ -769,7 +770,7 @@ could modify. This strategy is supported via the pod-template feature. ``` -3. Submitting with pre-populated secrets, that contain the delegation token, already existing within the namespace +3. Submitting with pre-populated secrets, that contain the Delegation Token, already existing within the namespace ```bash /opt/spark/bin/spark-submit \ --deploy-mode cluster \ @@ -785,7 +786,7 @@ could modify. This strategy is supported via the pod-template feature. ``` -3b. Submitting like in (3) however specifying a pre-created krb5 config map +3b. Submitting like in (3) however specifying a pre-created krb5 ConfigMap ```bash /opt/spark/bin/spark-submit \ --deploy-mode cluster \ diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesUtils.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesUtils.scala index d9a1aee77f78..8f36fa12aed1 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesUtils.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesUtils.scala @@ -59,6 +59,7 @@ private[spark] object KubernetesUtils { def requireNandDefined(opt1: Option[_], opt2: Option[_], errMessage: String): Unit = { opt1.foreach { _ => require(opt2.isEmpty, errMessage) } + opt2.foreach { _ => require(opt1.isEmpty, errMessage) } } /** 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 7de1e81e08f4..46f0797c2895 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 @@ -37,8 +37,9 @@ private[spark] class HadoopConfExecutorFeatureStep( val maybeHadoopConfDir = sparkConf.getOption(HADOOP_CONF_DIR_LOC) val maybeHadoopConfigMap = sparkConf.getOption(HADOOP_CONFIG_MAP_NAME) require(maybeHadoopConfDir.isDefined && maybeHadoopConfigMap.isDefined, - "Ensure that HADOOP_CONF_DIR is defined") - logInfo("HADOOP_CONF_DIR defined. Mounting Hadoop specific files") + "Ensure that the env `HADOOP_CONF_DIR` is defined either in the client or " + + " using pre-existing ConfigMaps where the `HADOOP_CONF_DIR` should be defined") + logInfo("HADOOP_CONF_DIR defined") HadoopBootstrapUtil.bootstrapHadoopConfDir( maybeHadoopConfDir.get, maybeHadoopConfigMap.get, 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 8b660587f623..b999f2b2b735 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 @@ -146,14 +146,9 @@ private[spark] class KerberosConfDriverFeatureStep( } yield { kDtSecret } - val configMap = - new ConfigMapBuilder() - .withNewMetadata() - .withName(kubernetesConf.hadoopConfigMapName) - .endMetadata() - .addToData(hadoopConfigurationFiles.map(file => - (file.toPath.getFileName.toString, Files.toString(file, Charsets.UTF_8))).toMap.asJava) - .build() + val configMap = HadoopBootstrapUtil.buildHadoopConfigMap( + kubernetesConf.hadoopConfigMapName, + hadoopConfigurationFiles) Seq(configMap) ++ krb5ConfigMap.toSeq ++ kerberosDTSecret.toSeq 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 8dea4c2ca536..c5719360c403 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 @@ -54,10 +54,5 @@ private[spark] class KerberosConfExecutorFeatureStep( override def getAdditionalPodSystemProperties(): Map[String, String] = Map.empty - override def getAdditionalKubernetesResources(): Seq[HasMetadata] = { - maybeKrb5File - .map(fileLocation => HadoopBootstrapUtil.buildkrb5ConfigMap( - kubernetesConf.kRBConfigMapName, - fileLocation)).toSeq - } + override def getAdditionalKubernetesResources(): Seq[HasMetadata] = Seq.empty[HasMetadata] } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadooputils/HadoopBootstrapUtil.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadooputils/HadoopBootstrapUtil.scala index 5d7de6d62f5e..990eacbc2950 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadooputils/HadoopBootstrapUtil.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadooputils/HadoopBootstrapUtil.scala @@ -37,7 +37,7 @@ private[spark] object HadoopBootstrapUtil { * @param dtSecretItemKey Name of the Item Key storing the Delegation Token * @param userName Name of the SparkUser to set SPARK_USER * @param maybeFileLocation Optional Location of the krb5 file - * @param newKrb5ConfName Optiona location of the ConfigMap for Krb5 + * @param newKrb5ConfName Optional location of the ConfigMap for Krb5 * @param maybeKrb5ConfName Optional name of ConfigMap for Krb5 * @param pod Input pod to be appended to * @return a modified SparkPod @@ -214,4 +214,25 @@ private[spark] object HadoopBootstrapUtil { Map(file.toPath.getFileName.toString -> Files.toString(file, Charsets.UTF_8)).asJava) .build() } + + /** + * Builds ConfigMap given the ConfigMap name + * and a list of Hadoop Conf files + * + * @param hadoopConfigMapName name of hadoopConfigMap + * @param hadoopConfFiles list of hadoopFiles + * @return a ConfigMap + */ + def buildHadoopConfigMap( + hadoopConfigMapName: String, + hadoopConfFiles: Seq[File]) : ConfigMap = { + new ConfigMapBuilder() + .withNewMetadata() + .withName(hadoopConfigMapName) + .endMetadata() + .addToData(hadoopConfFiles.map(file => + (file.toPath.getFileName.toString, Files.toString(file, Charsets.UTF_8))).toMap.asJava) + .build() + } + } From 69840a8026d1f8a72eed9a3e6c7073e8bdbf04b0 Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Mon, 8 Oct 2018 19:01:41 -0700 Subject: [PATCH 19/25] resolve comments --- .../org/apache/spark/deploy/SparkSubmit.scala | 4 +- docs/running-on-kubernetes.md | 14 +- docs/security.md | 15 +- .../org/apache/spark/examples/HdfsTest.scala | 2 +- .../org/apache/spark/deploy/k8s/Config.scala | 7 + .../apache/spark/deploy/k8s/Constants.scala | 16 +- .../spark/deploy/k8s/KubernetesConf.scala | 24 +- .../HadoopConfExecutorFeatureStep.scala | 18 +- .../HadoopSparkUserExecutorFeatureStep.scala | 8 +- .../KerberosConfDriverFeatureStep.scala | 164 ++++++++------ .../KerberosConfExecutorFeatureStep.scala | 23 +- .../hadooputils/HadoopBootstrapUtil.scala | 206 ++++++++++-------- .../hadooputils/HadoopKerberosLogin.scala | 16 +- .../k8s/KubernetesExecutorBuilder.scala | 10 +- .../submit/KubernetesDriverBuilderSuite.scala | 36 ++- .../k8s/KubernetesExecutorBuilderSuite.scala | 14 +- 16 files changed, 334 insertions(+), 243 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index 9d753192f76c..ceed31fad5d0 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -647,8 +647,8 @@ private[spark] class SparkSubmit extends Logging { } if ((clusterManager == MESOS || clusterManager == KUBERNETES) - && UserGroupInformation.isSecurityEnabled) { - setRMPrincipal(sparkConf) + && UserGroupInformation.isSecurityEnabled) { + setRMPrincipal(sparkConf) } // In yarn-cluster mode, use yarn.Client as a wrapper around the user class diff --git a/docs/running-on-kubernetes.md b/docs/running-on-kubernetes.md index 667f7de65798..e0019e47648b 100644 --- a/docs/running-on-kubernetes.md +++ b/docs/running-on-kubernetes.md @@ -825,7 +825,7 @@ specific to Spark on Kubernetes. (none) Specify the local location of the krb5 file to be mounted on the driver and executors for Kerberos interaction. - It is important to note that for local files, the KDC defined needs to be visible from inside the containers. + It is important to note that the KDC defined needs to be visible from inside the containers. @@ -837,12 +837,20 @@ specific to Spark on Kubernetes. be in the same namespace of the driver and executor pods. + + spark.kubernetes.hadoop.configMapName + (none) + + Specify the name of the ConfigMap, containing the HADOOP_CONF_DIR files, to be mounted on the driver + and executors for custom Hadoop configuration. + + spark.kubernetes.kerberos.tokenSecret.name (none) Specify the name of the secret where your existing delegation token is stored. This removes the need for the job user - to provide any keytab for launching a job. + to provide any kerberos credentials for launching a job. @@ -850,7 +858,7 @@ specific to Spark on Kubernetes. (none) Specify the item key of the data where your existing delegation token is stored. This removes the need for the job user - to provide any keytab for launching a job. + to provide any kerberos credentials for launching a job. diff --git a/docs/security.md b/docs/security.md index 406cde56f250..b87393e9fa41 100644 --- a/docs/security.md +++ b/docs/security.md @@ -728,17 +728,15 @@ When talking to Hadoop-based services behind Kerberos, it was noted that Spark n so that non-local processes can authenticate. These delegation tokens in Kubernetes are stored in Secrets that are shared by the Driver and its Executors. As such, there are three ways of submitting a Kerberos job: -In all cases you must define the environment variable: `HADOOP_CONF_DIR` as well as either +In all cases you must define the environment variable: `HADOOP_CONF_DIR` or +`spark.kubernetes.hadoop.configMapName` as well as either `spark.kubernetes.kerberos.krb5.location` or `spark.kubernetes.kerberos.krb5.configMapName`. It also important to note that the KDC needs to be visible from inside the containers if the user uses a local krb5 file. If a user wishes to use a remote HADOOP_CONF directory, that contains the Hadoop configuration files, this could be -achieved by setting the environmental variable `HADOOP_CONF_DIR` on the container to be pointed to the path where the -pre-created ConfigMap is mounted. -This method is useful for those who wish to not rebuild their Docker images, but instead point to a ConfigMap that they -could modify. This strategy is supported via the pod-template feature. +achieved by setting `spark.kubernetes.hadoop.configMapName` to a pre-existing ConfigMap. 1. Submitting with a $kinit that stores a TGT in the Local Ticket Cache: ```bash @@ -764,7 +762,7 @@ could modify. This strategy is supported via the pod-template feature. --conf spark.app.name=spark-hdfs \ --conf spark.kubernetes.container.image=spark:latest \ --conf spark.kerberos.keytab= \ - --conf spark.kerberos.principal= \ + --conf spark.kerberos.principal= \ --conf spark.kubernetes.kerberos.krb5.location=/etc/krb5.conf \ local:///opt/spark/examples/jars/spark-examples_-SNAPSHOT.jar \ @@ -786,7 +784,7 @@ could modify. This strategy is supported via the pod-template feature. ``` -3b. Submitting like in (3) however specifying a pre-created krb5 ConfigMap +3b. Submitting like in (3) however specifying a pre-created krb5 ConfigMap and pre-created `HADOOP_CONF_DIR` ConfigMap ```bash /opt/spark/bin/spark-submit \ --deploy-mode cluster \ @@ -797,7 +795,8 @@ could modify. This strategy is supported via the pod-template feature. --conf spark.kubernetes.container.image=spark:latest \ --conf spark.kubernetes.kerberos.tokenSecret.name= \ --conf spark.kubernetes.kerberos.tokenSecret.itemKey= \ - --conf spark.kubernetes.kerberos.krb5.configMapName= \ + --conf spark.kubernetes.hadoop.configMapName= \ + --conf spark.kubernetes.kerberos.krb5.configMapName= \ local:///opt/spark/examples/jars/spark-examples_-SNAPSHOT.jar \ ``` diff --git a/examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala b/examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala index 5ee1806dcb67..c1f71e6e298e 100644 --- a/examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala @@ -41,7 +41,7 @@ object HdfsTest { val end = System.currentTimeMillis() println(s"Iteration $iter took ${end-start} ms") } - println(s"File contents: ${file.map(_.toString).collect().mkString(",")}") + println(s"File contents: ${file.map(_.toString).take(1).mkString(",").slice(0, 10)}") println(s"Returned length(s) of: ${file.map(_.length).collect().mkString(",")}") spark.stop() } 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 0286ee7db304..4115628f6f46 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 @@ -241,6 +241,13 @@ private[spark] object Config extends Logging { .stringConf .createOptional + val KUBERNETES_HADOOP_CONF_CONFIG_MAP = + ConfigBuilder("spark.kubernetes.hadoop.configMapName") + .doc("Specify the name of the ConfigMap, containing the HADOOP_CONF_DIR files, " + + "to be mounted on the driver and executors for custom Hadoop configuration.") + .stringConf + .createOptional + val KUBERNETES_KERBEROS_DT_SECRET_NAME = ConfigBuilder("spark.kubernetes.kerberos.tokenSecret.name") .doc("Specify the name of the secret where your existing delegation token is stored. " + diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Constants.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Constants.scala index b2d8e568d20f..02a39644e387 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Constants.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Constants.scala @@ -87,28 +87,20 @@ private[spark] object Constants { val HADOOP_CONF_DIR_PATH = "/opt/hadoop/conf" val KRB_FILE_DIR_PATH = "/etc" val ENV_HADOOP_CONF_DIR = "HADOOP_CONF_DIR" - val HADOOP_CONF_DIR_LOC = "spark.kubernetes.hadoop.conf.dir" val HADOOP_CONFIG_MAP_NAME = "spark.kubernetes.executor.hadoopConfigMapName" + val KRB5_CONFIG_MAP_NAME = + "spark.kubernetes.executor.krb5ConfigMapName" // Kerberos Configuration - val KERBEROS_DELEGEGATION_TOKEN_SECRET_NAME = - "spark.kubernetes.kerberos.delegation-token-secret-name" + val KERBEROS_DELEGEGATION_TOKEN_SECRET_NAME = "delegation-tokens" val KERBEROS_KEYTAB_SECRET_NAME = "spark.kubernetes.kerberos.key-tab-secret-name" val KERBEROS_KEYTAB_SECRET_KEY = "spark.kubernetes.kerberos.key-tab-secret-key" val KERBEROS_SPARK_USER_NAME = "spark.kubernetes.kerberos.spark-user-name" - val KERBEROS_SECRET_LABEL_PREFIX = - "hadoop-tokens" - val SPARK_HADOOP_PREFIX = "spark.hadoop." - val HADOOP_SECURITY_AUTHENTICATION = - SPARK_HADOOP_PREFIX + "hadoop.security.authentication" - - // Kerberos Token-Refresh Server - val KERBEROS_REFRESH_LABEL_KEY = "refresh-hadoop-tokens" - val KERBEROS_REFRESH_LABEL_VALUE = "yes" + val KERBEROS_SECRET_KEY_PREFIX = "hadoop-tokens" // Hadoop credentials secrets for the Spark app. val SPARK_APP_HADOOP_CREDENTIALS_BASE_DIR = "/mnt/secrets/hadoop-credentials" 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 7c33e24e814e..27b785a2d2cb 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 @@ -50,6 +50,13 @@ private[spark] case class KubernetesExecutorSpecificConf( driverPod: Option[Pod]) extends KubernetesRoleSpecificConf +/* + * Structure containing metadata for HADOOP_CONF_DIR customization + */ +private[spark] case class HadoopConfSpecConf( + hadoopConfDir: Option[String], + hadoopConfigMapName: Option[String]) + /** * Structure containing metadata for Kubernetes logic to build Spark pods. */ @@ -65,7 +72,7 @@ private[spark] case class KubernetesConf[T <: KubernetesRoleSpecificConf]( roleEnvs: Map[String, String], roleVolumes: Iterable[KubernetesVolumeSpec[_ <: KubernetesVolumeSpecificConf]], sparkFiles: Seq[String], - hadoopConfDir: Option[String]) { + hadoopConfDir: Option[HadoopConfSpecConf]) { def hadoopConfigMapName: String = s"$appResourceNamePrefix-hadoop-config" @@ -188,6 +195,19 @@ private[spark] object KubernetesConf { .map(str => str.split(",").toSeq) .getOrElse(Seq.empty[String]) ++ additionalFiles + 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 hadoopConfSpecConf = + if (hadoopConfDir.isDefined || hadoopConfigMapName.isDefined) { + Some(HadoopConfSpecConf(hadoopConfDir, hadoopConfigMapName)) + } else { + None + } + KubernetesConf( sparkConfWithMainAppJar, KubernetesDriverSpecificConf(mainAppResource, mainClass, appName, appArgs), @@ -200,7 +220,7 @@ private[spark] object KubernetesConf { driverEnvs, driverVolumes, sparkFiles, - hadoopConfDir) + hadoopConfSpecConf) } def createExecutorConf( 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 46f0797c2895..89af99567688 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 @@ -29,21 +29,17 @@ import org.apache.spark.internal.Logging * pointed to the mounted file directory. */ private[spark] class HadoopConfExecutorFeatureStep( - kubernetesConf: KubernetesConf[KubernetesExecutorSpecificConf]) - extends KubernetesFeatureConfigStep with Logging{ + kubernetesConf: KubernetesConf[KubernetesExecutorSpecificConf]) + extends KubernetesFeatureConfigStep with Logging { - override def configurePod(pod: SparkPod): SparkPod = { + override def configurePod(pod: SparkPod): SparkPod = { val sparkConf = kubernetesConf.sparkConf - val maybeHadoopConfDir = sparkConf.getOption(HADOOP_CONF_DIR_LOC) - val maybeHadoopConfigMap = sparkConf.getOption(HADOOP_CONFIG_MAP_NAME) - require(maybeHadoopConfDir.isDefined && maybeHadoopConfigMap.isDefined, + val maybeHadoopConfDir = sparkConf.getOption(HADOOP_CONFIG_MAP_NAME) + require(maybeHadoopConfDir.isDefined, "Ensure that the env `HADOOP_CONF_DIR` is defined either in the client or " + - " using pre-existing ConfigMaps where the `HADOOP_CONF_DIR` should be defined") + " using pre-existing ConfigMaps") logInfo("HADOOP_CONF_DIR defined") - HadoopBootstrapUtil.bootstrapHadoopConfDir( - maybeHadoopConfDir.get, - maybeHadoopConfigMap.get, - pod) + HadoopBootstrapUtil.bootstrapHadoopConfDir(None, None, maybeHadoopConfDir, pod) } override def getAdditionalPodSystemProperties(): Map[String, String] = Map.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 60128394583b..a4774190ae11 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 @@ -29,13 +29,13 @@ import org.apache.spark.internal.Logging * 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{ + kubernetesConf: KubernetesConf[KubernetesExecutorSpecificConf]) + extends KubernetesFeatureConfigStep with Logging{ - override def configurePod(pod: SparkPod): SparkPod = { + override def configurePod(pod: SparkPod): SparkPod = { val sparkUserName = kubernetesConf.sparkConf.get(KERBEROS_SPARK_USER_NAME) HadoopBootstrapUtil.bootstrapSparkUserPod(sparkUserName, pod) - } + } override def getAdditionalPodSystemProperties(): Map[String, String] = Map.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 b999f2b2b735..82542e7ef020 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,11 +16,7 @@ */ package org.apache.spark.deploy.k8s.features -import scala.collection.JavaConverters._ - -import com.google.common.base.Charsets -import com.google.common.io.Files -import io.fabric8.kubernetes.api.model.{ConfigMapBuilder, HasMetadata} +import io.fabric8.kubernetes.api.model.HasMetadata import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesUtils, SparkPod} @@ -35,31 +31,37 @@ import org.apache.spark.internal.Logging * HADOOP_CONF_DIR. This runs various bootstrap methods defined in HadoopBootstrapUtil. */ private[spark] class KerberosConfDriverFeatureStep( - kubernetesConf: KubernetesConf[KubernetesDriverSpecificConf]) - extends KubernetesFeatureConfigStep with Logging { - - private val conf = kubernetesConf.sparkConf - private val maybePrincipal = conf.get(org.apache.spark.internal.config.PRINCIPAL) - private val maybeKeytab = conf.get(org.apache.spark.internal.config.KEYTAB) - private val maybeExistingSecretName = conf.get(KUBERNETES_KERBEROS_DT_SECRET_NAME) - private val maybeExistingSecretItemKey = - conf.get(KUBERNETES_KERBEROS_DT_SECRET_ITEM_KEY) - private val maybeKrb5File = + kubernetesConf: KubernetesConf[KubernetesDriverSpecificConf]) + extends KubernetesFeatureConfigStep with Logging { + + require(kubernetesConf.hadoopConfDir.isDefined, + "Ensure that HADOOP_CONF_DIR is defined either via env or a pre-defined ConfigMap") + private val hadoopConfDirSpec = kubernetesConf.hadoopConfDir.get + private val conf = kubernetesConf.sparkConf + private val maybePrincipal = conf.get(org.apache.spark.internal.config.PRINCIPAL) + private val maybeKeytab = conf.get(org.apache.spark.internal.config.KEYTAB) + private val maybeExistingSecretName = conf.get(KUBERNETES_KERBEROS_DT_SECRET_NAME) + private val maybeExistingSecretItemKey = + conf.get(KUBERNETES_KERBEROS_DT_SECRET_ITEM_KEY) + private val maybeKrb5File = conf.get(KUBERNETES_KERBEROS_KRB5_FILE) - private val maybeKrb5CMap = - conf.get(KUBERNETES_KERBEROS_KRB5_CONFIG_MAP) - private val kubeTokenManager = kubernetesConf.tokenManager(conf, - SparkHadoopUtil.get.newConfiguration(conf)) - private val isKerberosEnabled = kubeTokenManager.isSecurityEnabled + private val maybeKrb5CMap = + conf.get(KUBERNETES_KERBEROS_KRB5_CONFIG_MAP) + private val kubeTokenManager = kubernetesConf.tokenManager(conf, + SparkHadoopUtil.get.newConfiguration(conf)) + private val isKerberosEnabled = + (hadoopConfDirSpec.hadoopConfDir.isDefined && kubeTokenManager.isSecurityEnabled) || + (hadoopConfDirSpec.hadoopConfigMapName.isDefined && + (maybeKrb5File.isDefined || maybeKrb5CMap.isDefined)) - require(maybeKeytab.isEmpty || isKerberosEnabled, - "You must enable Kerberos support if you are specifying a Kerberos Keytab") + require(maybeKeytab.isEmpty || isKerberosEnabled, + "You must enable Kerberos support if you are specifying a Kerberos Keytab") - require(maybeExistingSecretName.isEmpty || isKerberosEnabled, - "You must enable Kerberos support if you are specifying a Kerberos Secret") + require(maybeExistingSecretName.isEmpty || isKerberosEnabled, + "You must enable Kerberos support if you are specifying a Kerberos Secret") require((maybeKrb5File.isEmpty || maybeKrb5CMap.isEmpty) || isKerberosEnabled, - "You must specify either a krb5 file location or a ConfigMap with a krb5 file") + "You must specify either a krb5 file location or a ConfigMap with a krb5 file") KubernetesUtils.requireNandDefined( maybeKrb5File, @@ -67,46 +69,56 @@ private[spark] class KerberosConfDriverFeatureStep( "Do not specify both a Krb5 local file and the ConfigMap as the creation " + "of an additional ConfigMap, when one is already specified, is extraneous") - KubernetesUtils.requireBothOrNeitherDefined( - maybeKeytab, - maybePrincipal, - "If a Kerberos principal is specified you must also specify a Kerberos keytab", - "If a Kerberos keytab is specified you must also specify a Kerberos principal") + KubernetesUtils.requireBothOrNeitherDefined( + maybeKeytab, + maybePrincipal, + "If a Kerberos principal is specified you must also specify a Kerberos keytab", + "If a Kerberos keytab is specified you must also specify a Kerberos principal") - KubernetesUtils.requireBothOrNeitherDefined( - maybeExistingSecretName, - maybeExistingSecretItemKey, - "If a secret data item-key where the data of the Kerberos Delegation Token is specified" + + KubernetesUtils.requireBothOrNeitherDefined( + maybeExistingSecretName, + maybeExistingSecretItemKey, + "If a secret data item-key where the data of the Kerberos Delegation Token is specified" + " you must also specify the name of the secret", - "If a secret storing a Kerberos Delegation Token is specified you must also" + + "If a secret storing a Kerberos Delegation Token is specified you must also" + " specify the item-key where the data is stored") - require(kubernetesConf.hadoopConfDir.isDefined, "Ensure that HADOOP_CONF_DIR is defined") - private val hadoopConfDir = kubernetesConf.hadoopConfDir.get - private val hadoopConfigurationFiles = HadoopBootstrapUtil.getHadoopConfFiles(hadoopConfDir) - - // Either use pre-existing secret or login to create new Secret with DT stored within - private val hadoopSpec: Option[KerberosConfigSpec] = (for { + private val hadoopConfigurationFiles = hadoopConfDirSpec.hadoopConfDir.map { hConfDir => + HadoopBootstrapUtil.getHadoopConfFiles(hConfDir) + } + private val newHadoopConfigMapName = + if (hadoopConfDirSpec.hadoopConfigMapName.isEmpty) { + Some(kubernetesConf.hadoopConfigMapName) + } else { + None + } + + // Either use pre-existing secret or login to create new Secret with DT stored within + private val hadoopSpec: Option[KerberosConfigSpec] = (for { secretName <- maybeExistingSecretName secretItemKey <- maybeExistingSecretItemKey - } yield { + } yield { KerberosConfigSpec( dtSecret = None, dtSecretName = secretName, dtSecretItemKey = secretItemKey, jobUserName = kubeTokenManager.getCurrentUser.getShortUserName) - }).orElse( + }).orElse( if (isKerberosEnabled) { Some(HadoopKerberosLogin.buildSpec( conf, kubernetesConf.appResourceNamePrefix, kubeTokenManager)) - } else None ) + } else { + None + } + ) - override def configurePod(pod: SparkPod): SparkPod = { - val hadoopBasedSparkPod = HadoopBootstrapUtil.bootstrapHadoopConfDir( - hadoopConfDir, - kubernetesConf.hadoopConfigMapName, + override def configurePod(pod: SparkPod): SparkPod = { + val hadoopBasedSparkPod = HadoopBootstrapUtil.bootstrapHadoopConfDir( + hadoopConfDirSpec.hadoopConfDir, + newHadoopConfigMapName, + hadoopConfDirSpec.hadoopConfigMapName, pod) hadoopSpec.map { hSpec => HadoopBootstrapUtil.bootstrapKerberosPod( @@ -114,7 +126,7 @@ private[spark] class KerberosConfDriverFeatureStep( hSpec.dtSecretItemKey, hSpec.jobUserName, maybeKrb5File, - kubernetesConf.kRBConfigMapName, + Some(kubernetesConf.kRBConfigMapName), maybeKrb5CMap, hadoopBasedSparkPod) }.getOrElse( @@ -123,34 +135,48 @@ private[spark] class KerberosConfDriverFeatureStep( hadoopBasedSparkPod)) } - override def getAdditionalPodSystemProperties(): Map[String, String] = { - val resolvedConfValues = hadoopSpec.map{ hSpec => - Map(KERBEROS_KEYTAB_SECRET_NAME -> hSpec.dtSecretName, - KERBEROS_KEYTAB_SECRET_KEY -> hSpec.dtSecretItemKey, - KERBEROS_SPARK_USER_NAME -> hSpec.jobUserName) - }.getOrElse( + override def getAdditionalPodSystemProperties(): Map[String, String] = { + val resolvedConfValues = hadoopSpec.map { hSpec => + Map(KERBEROS_KEYTAB_SECRET_NAME -> hSpec.dtSecretName, + KERBEROS_KEYTAB_SECRET_KEY -> hSpec.dtSecretItemKey, + KERBEROS_SPARK_USER_NAME -> hSpec.jobUserName, + KRB5_CONFIG_MAP_NAME -> maybeKrb5CMap.getOrElse(kubernetesConf.kRBConfigMapName)) + }.getOrElse( Map(KERBEROS_SPARK_USER_NAME -> kubeTokenManager.getCurrentUser.getShortUserName)) - Map(HADOOP_CONFIG_MAP_NAME -> kubernetesConf.hadoopConfigMapName, - HADOOP_CONF_DIR_LOC -> kubernetesConf.hadoopConfDir.get) ++ resolvedConfValues + Map(HADOOP_CONFIG_MAP_NAME -> + hadoopConfDirSpec.hadoopConfigMapName.getOrElse( + kubernetesConf.hadoopConfigMapName)) ++ resolvedConfValues } - override def getAdditionalKubernetesResources(): Seq[HasMetadata] = { - val krb5ConfigMap = maybeKrb5File.map { fileLocation => - HadoopBootstrapUtil.buildkrb5ConfigMap( - kubernetesConf.kRBConfigMapName, - fileLocation)} - val kerberosDTSecret = for { + override def getAdditionalKubernetesResources(): Seq[HasMetadata] = { + // HADOOP_CONF_DIR ConfigMap + val hadoopConfConfigMap = for { + hName <- newHadoopConfigMapName + hFiles <- hadoopConfigurationFiles + } yield { + HadoopBootstrapUtil.buildHadoopConfigMap( + hName, + hFiles) + } + + // krb5 ConfigMap + val krb5ConfigMap = maybeKrb5File.map { fileLocation => + HadoopBootstrapUtil.buildkrb5ConfigMap( + kubernetesConf.kRBConfigMapName, + fileLocation) + } + + // Kerberos DT Secret + val kerberosDTSecret = for { hSpec <- hadoopSpec kDtSecret <- hSpec.dtSecret } yield { kDtSecret } - val configMap = HadoopBootstrapUtil.buildHadoopConfigMap( - kubernetesConf.hadoopConfigMapName, - hadoopConfigurationFiles) - Seq(configMap) ++ - krb5ConfigMap.toSeq ++ - kerberosDTSecret.toSeq + + hadoopConfConfigMap.toSeq ++ + krb5ConfigMap.toSeq ++ + kerberosDTSecret.toSeq } } 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 c5719360c403..eb5e956a4825 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 @@ -18,8 +18,7 @@ package org.apache.spark.deploy.k8s.features import io.fabric8.kubernetes.api.model.HasMetadata -import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesUtils, SparkPod} -import org.apache.spark.deploy.k8s.Config._ +import org.apache.spark.deploy.k8s.{KubernetesConf, SparkPod} import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.deploy.k8s.KubernetesExecutorSpecificConf import org.apache.spark.deploy.k8s.features.hadooputils.HadoopBootstrapUtil @@ -29,25 +28,21 @@ 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]) - extends KubernetesFeatureConfigStep with Logging{ + kubernetesConf: KubernetesConf[KubernetesExecutorSpecificConf]) + extends KubernetesFeatureConfigStep with Logging{ + private val sparkConf = kubernetesConf.sparkConf - private val maybeKrb5File = sparkConf.get(KUBERNETES_KERBEROS_KRB5_FILE) - private val maybeKrb5CMap = sparkConf.get(KUBERNETES_KERBEROS_KRB5_CONFIG_MAP) - KubernetesUtils.requireNandDefined( - maybeKrb5File, - maybeKrb5CMap, - "Do not specify both a Krb5 local file and the ConfigMap as the creation " + - "of an additional ConfigMap, when one is already specified, is extraneous") + private val maybeKrb5CMap = sparkConf.getOption(KRB5_CONFIG_MAP_NAME) + require(maybeKrb5CMap.isDefined, "HADOOP_CONF_DIR ConfigMap not found") override def configurePod(pod: SparkPod): SparkPod = { - logInfo(s"Mounting Kerberos DT for Kerberos") + logInfo(s"Mounting Resources for Kerberos") HadoopBootstrapUtil.bootstrapKerberosPod( sparkConf.get(KERBEROS_KEYTAB_SECRET_NAME), sparkConf.get(KERBEROS_KEYTAB_SECRET_KEY), sparkConf.get(KERBEROS_SPARK_USER_NAME), - maybeKrb5File, - kubernetesConf.kRBConfigMapName, + None, + None, maybeKrb5CMap, pod) } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadooputils/HadoopBootstrapUtil.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadooputils/HadoopBootstrapUtil.scala index 990eacbc2950..3a52277551cb 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadooputils/HadoopBootstrapUtil.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadooputils/HadoopBootstrapUtil.scala @@ -17,14 +17,13 @@ package org.apache.spark.deploy.k8s.features.hadooputils import java.io.File +import java.nio.charset.StandardCharsets import scala.collection.JavaConverters._ -import com.google.common.base.Charsets import com.google.common.io.Files import io.fabric8.kubernetes.api.model._ -import org.apache.spark.SparkException import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.deploy.k8s.SparkPod @@ -43,47 +42,46 @@ private[spark] object HadoopBootstrapUtil { * @return a modified SparkPod */ def bootstrapKerberosPod( - dtSecretName: String, - dtSecretItemKey: String, - userName: String, - maybeFileLocation: Option[String], - newKrb5ConfName: String, - maybeKrb5ConfName: Option[String], - pod: SparkPod) : SparkPod = { - - val maybePreConfigMapVolume = maybeKrb5ConfName.map { kconf => - new VolumeBuilder() - .withName(KRB_FILE_VOLUME) - .withNewConfigMap() + dtSecretName: String, + dtSecretItemKey: String, + userName: String, + maybeFileLocation: Option[String], + newKrb5ConfName: Option[String], + maybeKrb5ConfName: Option[String], + pod: SparkPod) : SparkPod = { + + val maybePreConfigMapVolume = maybeKrb5ConfName.map { kconf => + new VolumeBuilder() + .withName(KRB_FILE_VOLUME) + .withNewConfigMap() .withName(kconf) - .endConfigMap() - .build() } - - val maybeCreateConfigMapVolume = maybeFileLocation.map { - fileLocation => - val krb5File = new File(fileLocation) - val fileStringPath = krb5File.toPath.getFileName.toString - new VolumeBuilder() - .withName(KRB_FILE_VOLUME) - .withNewConfigMap() - .withName(newKrb5ConfName) - .withItems(new KeyToPathBuilder() - .withKey(fileStringPath) - .withPath(fileStringPath) - .build()) - .endConfigMap() - .build() } - - // Breaking up Volume Creation for clarity - val configMapVolume = - maybePreConfigMapVolume.getOrElse( - maybeCreateConfigMapVolume.getOrElse( - throw new SparkException( - "Must specify krb5 file locally or via ConfigMap") - )) - - val kerberizedPod = new PodBuilder(pod.pod) - .editOrNewSpec() + .endConfigMap() + .build() } + + val maybeCreateConfigMapVolume = for { + fileLocation <- maybeFileLocation + krb5ConfName <- newKrb5ConfName + } yield { + val krb5File = new File(fileLocation) + val fileStringPath = krb5File.toPath.getFileName.toString + new VolumeBuilder() + .withName(KRB_FILE_VOLUME) + .withNewConfigMap() + .withName(krb5ConfName) + .withItems(new KeyToPathBuilder() + .withKey(fileStringPath) + .withPath(fileStringPath) + .build()) + .endConfigMap() + .build() + } + + // Breaking up Volume Creation for clarity + val configMapVolume = maybePreConfigMapVolume.getOrElse( + maybeCreateConfigMapVolume.get) + + val kerberizedPod = new PodBuilder(pod.pod) + .editOrNewSpec() .addNewVolume() .withName(SPARK_APP_HADOOP_SECRET_VOLUME_NAME) .withNewSecret() @@ -93,8 +91,9 @@ private[spark] object HadoopBootstrapUtil { .addNewVolumeLike(configMapVolume) .endVolume() .endSpec() - .build() - val kerberizedContainer = new ContainerBuilder(pod.container) + .build() + + val kerberizedContainer = new ContainerBuilder(pod.container) .addNewVolumeMount() .withName(SPARK_APP_HADOOP_SECRET_VOLUME_NAME) .withMountPath(SPARK_APP_HADOOP_CREDENTIALS_BASE_DIR) @@ -113,7 +112,7 @@ private[spark] object HadoopBootstrapUtil { .withValue(userName) .endEnv() .build() - SparkPod(kerberizedPod, kerberizedContainer) + SparkPod(kerberizedPod, kerberizedContainer) } /** @@ -124,64 +123,85 @@ private[spark] object HadoopBootstrapUtil { * @return a modified SparkPod */ def bootstrapSparkUserPod(sparkUserName: String, pod: SparkPod) : SparkPod = { - val envModifiedContainer = new ContainerBuilder(pod.container) - .addNewEnv() + val envModifiedContainer = new ContainerBuilder(pod.container) + .addNewEnv() .withName(ENV_SPARK_USER) .withValue(sparkUserName) .endEnv() .build() - SparkPod(pod.pod, envModifiedContainer) + SparkPod(pod.pod, envModifiedContainer) } - /** - * Grabbing files in the HADOOP_CONF_DIR - * - * @param path location of HADOOP_CONF_DIR - * @return a list of File object - */ + /** + * Grabbing files in the HADOOP_CONF_DIR + * + * @param path location of HADOOP_CONF_DIR + * @return a list of File object + */ def getHadoopConfFiles(path: String) : Seq[File] = { val dir = new File(path) if (dir.isDirectory) { - dir.listFiles.flatMap { file => Some(file).filter(_.isFile) }.toSeq + dir.listFiles.filter(_.isFile).toSeq } else { Seq.empty[File] } - } + } /** * Bootstraping the container with ConfigMaps that store * Hadoop configuration files * - * @param hadoopConfDir location of HADOOP_CONF_DIR - * @param hadoopConfigMapName name of the configMap for HADOOP_CONF_DIR + * @param maybeHadoopConfDir directory location of HADOOP_CONF_DIR env + * @param newHadoopConfigMapName name of the new configMap for HADOOP_CONF_DIR + * @param oldHadoopConfigMapName name of the pre-defined configMap for HADOOP_CONF_DIR * @param pod Input pod to be appended to * @return a modified SparkPod */ def bootstrapHadoopConfDir( - hadoopConfDir: String, - hadoopConfigMapName: String, - pod: SparkPod) : SparkPod = { - val hadoopConfigFiles = getHadoopConfFiles(hadoopConfDir) - val keyPaths = hadoopConfigFiles.map { file => - val fileStringPath = file.toPath.getFileName.toString - new KeyToPathBuilder() - .withKey(fileStringPath) - .withPath(fileStringPath) + maybeHadoopConfDir: Option[String], + newHadoopConfigMapName: Option[String], + oldHadoopConfigMapName: Option[String], + pod: SparkPod) : SparkPod = { + val maybePreConfigMapVolume = oldHadoopConfigMapName.map { hConf => + new VolumeBuilder() + .withName(HADOOP_FILE_VOLUME) + .withNewConfigMap() + .withName(hConf) + .endConfigMap() .build() } - val hadoopSupportedPod = new PodBuilder(pod.pod) - .editSpec() - .addNewVolume() - .withName(HADOOP_FILE_VOLUME) - .withNewConfigMap() - .withName(hadoopConfigMapName) + val maybeCreateConfigMapVolume = for { + dirLocation <- maybeHadoopConfDir + hConfName <- newHadoopConfigMapName + } yield { + val hadoopConfigFiles = getHadoopConfFiles(dirLocation) + val keyPaths = hadoopConfigFiles.map { file => + val fileStringPath = file.toPath.getFileName.toString + new KeyToPathBuilder() + .withKey(fileStringPath) + .withPath(fileStringPath) + .build() } + new VolumeBuilder() + .withName(HADOOP_FILE_VOLUME) + .withNewConfigMap() + .withName(hConfName) .withItems(keyPaths.asJava) .endConfigMap() + .build() } + + // Breaking up Volume Creation for clarity + val configMapVolume = maybePreConfigMapVolume.getOrElse( + maybeCreateConfigMapVolume.get) + + val hadoopSupportedPod = new PodBuilder(pod.pod) + .editSpec() + .addNewVolumeLike(configMapVolume) .endVolume() .endSpec() .build() - val hadoopSupportedContainer = new ContainerBuilder(pod.container) + + val hadoopSupportedContainer = new ContainerBuilder(pod.container) .addNewVolumeMount() .withName(HADOOP_FILE_VOLUME) .withMountPath(HADOOP_CONF_DIR_PATH) @@ -194,14 +214,14 @@ private[spark] object HadoopBootstrapUtil { SparkPod(hadoopSupportedPod, hadoopSupportedContainer) } - /** - * Builds ConfigMap given the file location of the - * krb5.conf file - * - * @param configMapName name of configMap for krb5 - * @param fileLocation location of krb5 file - * @return a ConfigMap - */ + /** + * Builds ConfigMap given the file location of the + * krb5.conf file + * + * @param configMapName name of configMap for krb5 + * @param fileLocation location of krb5 file + * @return a ConfigMap + */ def buildkrb5ConfigMap( configMapName: String, fileLocation: String) : ConfigMap = { @@ -211,18 +231,19 @@ private[spark] object HadoopBootstrapUtil { .withName(configMapName) .endMetadata() .addToData( - Map(file.toPath.getFileName.toString -> Files.toString(file, Charsets.UTF_8)).asJava) + Map(file.toPath.getFileName.toString -> + Files.toString(file, StandardCharsets.UTF_8)).asJava) .build() } - /** - * Builds ConfigMap given the ConfigMap name - * and a list of Hadoop Conf files - * - * @param hadoopConfigMapName name of hadoopConfigMap - * @param hadoopConfFiles list of hadoopFiles - * @return a ConfigMap - */ + /** + * Builds ConfigMap given the ConfigMap name + * and a list of Hadoop Conf files + * + * @param hadoopConfigMapName name of hadoopConfigMap + * @param hadoopConfFiles list of hadoopFiles + * @return a ConfigMap + */ def buildHadoopConfigMap( hadoopConfigMapName: String, hadoopConfFiles: Seq[File]) : ConfigMap = { @@ -231,7 +252,8 @@ private[spark] object HadoopBootstrapUtil { .withName(hadoopConfigMapName) .endMetadata() .addToData(hadoopConfFiles.map(file => - (file.toPath.getFileName.toString, Files.toString(file, Charsets.UTF_8))).toMap.asJava) + (file.toPath.getFileName.toString, + Files.toString(file, StandardCharsets.UTF_8))).toMap.asJava) .build() } 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 index b944a5d8e540..ed9c1368e60d 100644 --- 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 @@ -16,12 +16,10 @@ */ package org.apache.spark.deploy.k8s.features.hadooputils -import scala.collection.JavaConverters._ - import io.fabric8.kubernetes.api.model.SecretBuilder import org.apache.commons.codec.binary.Base64 -import org.apache.spark.{SparkConf, SparkException} +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 @@ -41,9 +39,6 @@ private[spark] object HadoopKerberosLogin { kubernetesResourceNamePrefix : String, tokenManager: KubernetesHadoopDelegationTokenManager): KerberosConfigSpec = { val hadoopConf = SparkHadoopUtil.get.newConfiguration(submissionSparkConf) - if (!tokenManager.isSecurityEnabled) { - throw new SparkException("Hadoop not configured with Kerberos") - } // 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 @@ -54,21 +49,20 @@ private[spark] object HadoopKerberosLogin { hadoopConf) require(tokenData.nonEmpty, "Did not obtain any delegation tokens") val currentTime = tokenManager.getCurrentTime - val initialTokenDataKeyName = s"$KERBEROS_SECRET_LABEL_PREFIX-$currentTime-$renewalInterval" - val uniqueSecretName = + val initialTokenDataKeyName = s"$KERBEROS_SECRET_KEY_PREFIX-$currentTime" + val newSecretName = s"$kubernetesResourceNamePrefix-$KERBEROS_DELEGEGATION_TOKEN_SECRET_NAME.$currentTime" val secretDT = new SecretBuilder() .withNewMetadata() - .withName(uniqueSecretName) - .withLabels(Map(KERBEROS_REFRESH_LABEL_KEY -> KERBEROS_REFRESH_LABEL_VALUE).asJava) + .withName(newSecretName) .endMetadata() .addToData(initialTokenDataKeyName, Base64.encodeBase64String(tokenData)) .build() KerberosConfigSpec( dtSecret = Some(secretDT), - dtSecretName = uniqueSecretName, + dtSecretName = newSecretName, dtSecretItemKey = initialTokenDataKeyName, jobUserName = jobUserUGI.getShortUserName) } 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 638308e7862e..4e032689a2d1 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 @@ -69,15 +69,13 @@ private[spark] class KubernetesExecutorBuilder( val maybeHadoopConfFeatureSteps = maybeHadoopConfigMap.map { _ => val maybeKerberosStep = - for { - _ <- maybeDTSecretName - _ <- maybeDTDataItem - } yield { + if (maybeDTSecretName.isDefined && maybeDTDataItem.isDefined) { provideKerberosConfStep(kubernetesConf) + } else { + provideHadoopSparkUserStep(kubernetesConf) } Seq(provideHadoopConfStep(kubernetesConf)) :+ - maybeKerberosStep.getOrElse( - provideHadoopSparkUserStep(kubernetesConf)) + maybeKerberosStep }.getOrElse(Seq.empty[KubernetesFeatureConfigStep]) val allFeatures: Seq[KubernetesFeatureConfigStep] = 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 ea17f2ca43b3..909ad177986e 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 @@ -269,7 +269,41 @@ class KubernetesDriverBuilderSuite extends SparkFunSuite { Map.empty, Nil, Seq.empty[String], - hadoopConfDir = Some("/var/hadoop-conf")) + hadoopConfDir = Some( + HadoopConfSpecConf( + Some("/var/hadoop-conf"), + None))) + validateStepTypesApplied( + builderUnderTest.buildFromFeatures(conf), + BASIC_STEP_TYPE, + CREDENTIALS_STEP_TYPE, + SERVICE_STEP_TYPE, + LOCAL_DIRS_STEP_TYPE, + JAVA_STEP_TYPE, + HADOOP_GLOBAL_STEP_TYPE) + } + + test("Apply HadoopSteps if HADOOP_CONF ConfigMap is defined.") { + val conf = KubernetesConf( + new SparkConf(false), + KubernetesDriverSpecificConf( + None, + "test-app", + "main", + Seq.empty), + "prefix", + "appId", + Map.empty, + Map.empty, + Map.empty, + Map.empty, + Map.empty, + Nil, + Seq.empty[String], + hadoopConfDir = Some( + HadoopConfSpecConf( + None, + Some("pre-defined-configMapName")))) validateStepTypesApplied( builderUnderTest.buildFromFeatures(conf), BASIC_STEP_TYPE, 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 a14c1b1a5c78..e364cec8a333 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 @@ -19,7 +19,7 @@ package org.apache.spark.scheduler.cluster.k8s import io.fabric8.kubernetes.api.model.PodBuilder import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesExecutorSpecificConf, KubernetesHostPathVolumeConf, KubernetesVolumeSpec, SparkPod} +import org.apache.spark.deploy.k8s._ import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.deploy.k8s.features._ @@ -135,8 +135,8 @@ class KubernetesExecutorBuilderSuite extends SparkFunSuite { val HADOOPFS_PROVIDER = s"$HADOOP_CREDS_PREFIX.hadoopfs.enabled" val conf = KubernetesConf( new SparkConf(false) - .set(HADOOP_CONFIG_MAP_NAME, "hadoop-conf-map-loc") - .set(HADOOP_CONF_DIR_LOC, "hadoop-conf-dir-loc") + .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( @@ -150,7 +150,7 @@ class KubernetesExecutorBuilderSuite extends SparkFunSuite { Map.empty, Nil, Seq.empty[String], - Some("hadoop-conf-dir-loc")) + Some(HadoopConfSpecConf(Some("/var/hadoop-conf"), None))) validateStepTypesApplied( builderUnderTest.buildFromFeatures(conf), BASIC_STEP_TYPE, @@ -162,8 +162,8 @@ class KubernetesExecutorBuilderSuite extends SparkFunSuite { test("Apply kerberos step if DT secrets created") { val conf = KubernetesConf( new SparkConf(false) - .set(HADOOP_CONFIG_MAP_NAME, "hadoop-conf-map-loc") - .set(HADOOP_CONF_DIR_LOC, "hadoop-conf-dir-loc") + .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_KEYTAB_SECRET_NAME, "dt-secret") .set(KERBEROS_KEYTAB_SECRET_KEY, "dt-key"), @@ -178,7 +178,7 @@ class KubernetesExecutorBuilderSuite extends SparkFunSuite { Map.empty, Nil, Seq.empty[String], - Some("hadoop-conf-dir-loc")) + Some(HadoopConfSpecConf(None, Some("pre-defined-onfigMapName")))) validateStepTypesApplied( builderUnderTest.buildFromFeatures(conf), BASIC_STEP_TYPE, From 210815496cc2e5ecf8a0e7f86572733994e2f671 Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Mon, 8 Oct 2018 19:09:53 -0700 Subject: [PATCH 20/25] style and indentation --- .../KerberosConfDriverFeatureStep.scala | 6 +- .../hadooputils/HadoopBootstrapUtil.scala | 190 +++++++++--------- 2 files changed, 98 insertions(+), 98 deletions(-) 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 82542e7ef020..5dbc234738ae 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 @@ -155,9 +155,9 @@ private[spark] class KerberosConfDriverFeatureStep( hName <- newHadoopConfigMapName hFiles <- hadoopConfigurationFiles } yield { - HadoopBootstrapUtil.buildHadoopConfigMap( - hName, - hFiles) + HadoopBootstrapUtil.buildHadoopConfigMap( + hName, + hFiles) } // krb5 ConfigMap diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadooputils/HadoopBootstrapUtil.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadooputils/HadoopBootstrapUtil.scala index 3a52277551cb..84a8c2509524 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadooputils/HadoopBootstrapUtil.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadooputils/HadoopBootstrapUtil.scala @@ -42,38 +42,38 @@ private[spark] object HadoopBootstrapUtil { * @return a modified SparkPod */ def bootstrapKerberosPod( - dtSecretName: String, - dtSecretItemKey: String, - userName: String, - maybeFileLocation: Option[String], - newKrb5ConfName: Option[String], - maybeKrb5ConfName: Option[String], - pod: SparkPod) : SparkPod = { + dtSecretName: String, + dtSecretItemKey: String, + userName: String, + maybeFileLocation: Option[String], + newKrb5ConfName: Option[String], + maybeKrb5ConfName: Option[String], + pod: SparkPod) : SparkPod = { - val maybePreConfigMapVolume = maybeKrb5ConfName.map { kconf => - new VolumeBuilder() - .withName(KRB_FILE_VOLUME) - .withNewConfigMap() + val maybePreConfigMapVolume = maybeKrb5ConfName.map { kconf => + new VolumeBuilder() + .withName(KRB_FILE_VOLUME) + .withNewConfigMap() .withName(kconf) - .endConfigMap() - .build() } + .endConfigMap() + .build() } - val maybeCreateConfigMapVolume = for { - fileLocation <- maybeFileLocation - krb5ConfName <- newKrb5ConfName + val maybeCreateConfigMapVolume = for { + fileLocation <- maybeFileLocation + krb5ConfName <- newKrb5ConfName } yield { - val krb5File = new File(fileLocation) - val fileStringPath = krb5File.toPath.getFileName.toString - new VolumeBuilder() - .withName(KRB_FILE_VOLUME) - .withNewConfigMap() - .withName(krb5ConfName) - .withItems(new KeyToPathBuilder() - .withKey(fileStringPath) - .withPath(fileStringPath) - .build()) - .endConfigMap() - .build() + val krb5File = new File(fileLocation) + val fileStringPath = krb5File.toPath.getFileName.toString + new VolumeBuilder() + .withName(KRB_FILE_VOLUME) + .withNewConfigMap() + .withName(krb5ConfName) + .withItems(new KeyToPathBuilder() + .withKey(fileStringPath) + .withPath(fileStringPath) + .build()) + .endConfigMap() + .build() } // Breaking up Volume Creation for clarity @@ -158,61 +158,61 @@ private[spark] object HadoopBootstrapUtil { * @return a modified SparkPod */ def bootstrapHadoopConfDir( - maybeHadoopConfDir: Option[String], - newHadoopConfigMapName: Option[String], - oldHadoopConfigMapName: Option[String], - pod: SparkPod) : SparkPod = { - val maybePreConfigMapVolume = oldHadoopConfigMapName.map { hConf => - new VolumeBuilder() - .withName(HADOOP_FILE_VOLUME) - .withNewConfigMap() - .withName(hConf) - .endConfigMap() - .build() } + maybeHadoopConfDir: Option[String], + newHadoopConfigMapName: Option[String], + oldHadoopConfigMapName: Option[String], + pod: SparkPod) : SparkPod = { + val maybePreConfigMapVolume = oldHadoopConfigMapName.map { hConf => + new VolumeBuilder() + .withName(HADOOP_FILE_VOLUME) + .withNewConfigMap() + .withName(hConf) + .endConfigMap() + .build() } - val maybeCreateConfigMapVolume = for { - dirLocation <- maybeHadoopConfDir - hConfName <- newHadoopConfigMapName + val maybeCreateConfigMapVolume = for { + dirLocation <- maybeHadoopConfDir + hConfName <- newHadoopConfigMapName } yield { - val hadoopConfigFiles = getHadoopConfFiles(dirLocation) - val keyPaths = hadoopConfigFiles.map { file => - val fileStringPath = file.toPath.getFileName.toString - new KeyToPathBuilder() - .withKey(fileStringPath) - .withPath(fileStringPath) - .build() } - new VolumeBuilder() - .withName(HADOOP_FILE_VOLUME) - .withNewConfigMap() - .withName(hConfName) - .withItems(keyPaths.asJava) - .endConfigMap() - .build() } + val hadoopConfigFiles = getHadoopConfFiles(dirLocation) + val keyPaths = hadoopConfigFiles.map { file => + val fileStringPath = file.toPath.getFileName.toString + new KeyToPathBuilder() + .withKey(fileStringPath) + .withPath(fileStringPath) + .build() } + new VolumeBuilder() + .withName(HADOOP_FILE_VOLUME) + .withNewConfigMap() + .withName(hConfName) + .withItems(keyPaths.asJava) + .endConfigMap() + .build() } - // Breaking up Volume Creation for clarity - val configMapVolume = maybePreConfigMapVolume.getOrElse( - maybeCreateConfigMapVolume.get) + // Breaking up Volume Creation for clarity + val configMapVolume = maybePreConfigMapVolume.getOrElse( + maybeCreateConfigMapVolume.get) - val hadoopSupportedPod = new PodBuilder(pod.pod) - .editSpec() - .addNewVolumeLike(configMapVolume) - .endVolume() - .endSpec() + val hadoopSupportedPod = new PodBuilder(pod.pod) + .editSpec() + .addNewVolumeLike(configMapVolume) + .endVolume() + .endSpec() .build() - val hadoopSupportedContainer = new ContainerBuilder(pod.container) - .addNewVolumeMount() - .withName(HADOOP_FILE_VOLUME) - .withMountPath(HADOOP_CONF_DIR_PATH) - .endVolumeMount() - .addNewEnv() - .withName(ENV_HADOOP_CONF_DIR) - .withValue(HADOOP_CONF_DIR_PATH) - .endEnv() - .build() - SparkPod(hadoopSupportedPod, hadoopSupportedContainer) - } + val hadoopSupportedContainer = new ContainerBuilder(pod.container) + .addNewVolumeMount() + .withName(HADOOP_FILE_VOLUME) + .withMountPath(HADOOP_CONF_DIR_PATH) + .endVolumeMount() + .addNewEnv() + .withName(ENV_HADOOP_CONF_DIR) + .withValue(HADOOP_CONF_DIR_PATH) + .endEnv() + .build() + SparkPod(hadoopSupportedPod, hadoopSupportedContainer) + } /** * Builds ConfigMap given the file location of the @@ -223,17 +223,17 @@ private[spark] object HadoopBootstrapUtil { * @return a ConfigMap */ def buildkrb5ConfigMap( - configMapName: String, - fileLocation: String) : ConfigMap = { - val file = new File(fileLocation) - new ConfigMapBuilder() - .withNewMetadata() - .withName(configMapName) - .endMetadata() - .addToData( - Map(file.toPath.getFileName.toString -> - Files.toString(file, StandardCharsets.UTF_8)).asJava) - .build() + configMapName: String, + fileLocation: String) : ConfigMap = { + val file = new File(fileLocation) + new ConfigMapBuilder() + .withNewMetadata() + .withName(configMapName) + .endMetadata() + .addToData( + Map(file.toPath.getFileName.toString -> + Files.toString(file, StandardCharsets.UTF_8)).asJava) + .build() } /** @@ -245,16 +245,16 @@ private[spark] object HadoopBootstrapUtil { * @return a ConfigMap */ def buildHadoopConfigMap( - hadoopConfigMapName: String, - hadoopConfFiles: Seq[File]) : ConfigMap = { - new ConfigMapBuilder() - .withNewMetadata() + hadoopConfigMapName: String, + hadoopConfFiles: Seq[File]) : ConfigMap = { + new ConfigMapBuilder() + .withNewMetadata() .withName(hadoopConfigMapName) .endMetadata() - .addToData(hadoopConfFiles.map(file => - (file.toPath.getFileName.toString, - Files.toString(file, StandardCharsets.UTF_8))).toMap.asJava) - .build() + .addToData(hadoopConfFiles.map(file => + (file.toPath.getFileName.toString, + Files.toString(file, StandardCharsets.UTF_8))).toMap.asJava) + .build() } } From a987a708aceb69ed796ebe2dbd637e209f76c52a Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Tue, 9 Oct 2018 11:42:47 -0700 Subject: [PATCH 21/25] resolving comments --- .../org/apache/spark/deploy/SparkSubmit.scala | 2 +- .../apache/spark/deploy/k8s/Constants.scala | 8 +- .../spark/deploy/k8s/KubernetesConf.scala | 10 +-- .../KerberosConfDriverFeatureStep.scala | 83 +++++++++---------- .../KerberosConfExecutorFeatureStep.scala | 4 +- .../hadooputils/HadoopBootstrapUtil.scala | 38 ++++----- .../k8s/KubernetesExecutorBuilder.scala | 4 +- .../submit/KubernetesDriverBuilderSuite.scala | 4 +- .../k8s/KubernetesExecutorBuilderSuite.scala | 8 +- 9 files changed, 79 insertions(+), 82 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index ceed31fad5d0..64dc86fc850e 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -647,7 +647,7 @@ private[spark] class SparkSubmit extends Logging { } if ((clusterManager == MESOS || clusterManager == KUBERNETES) - && UserGroupInformation.isSecurityEnabled) { + && UserGroupInformation.isSecurityEnabled) { setRMPrincipal(sparkConf) } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Constants.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Constants.scala index 02a39644e387..69516912353b 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Constants.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Constants.scala @@ -94,10 +94,10 @@ private[spark] object Constants { // Kerberos Configuration val KERBEROS_DELEGEGATION_TOKEN_SECRET_NAME = "delegation-tokens" - val KERBEROS_KEYTAB_SECRET_NAME = - "spark.kubernetes.kerberos.key-tab-secret-name" - val KERBEROS_KEYTAB_SECRET_KEY = - "spark.kubernetes.kerberos.key-tab-secret-key" + val KERBEROS_DT_SECRET_NAME = + "spark.kubernetes.kerberos.dt-secret-name" + val KERBEROS_DT_SECRET_KEY = + "spark.kubernetes.kerberos.dt-secret-key" val KERBEROS_SPARK_USER_NAME = "spark.kubernetes.kerberos.spark-user-name" val KERBEROS_SECRET_KEY_PREFIX = "hadoop-tokens" 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 27b785a2d2cb..731240f8e692 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 @@ -53,7 +53,7 @@ private[spark] case class KubernetesExecutorSpecificConf( /* * Structure containing metadata for HADOOP_CONF_DIR customization */ -private[spark] case class HadoopConfSpecConf( +private[spark] case class HadoopConfSpec( hadoopConfDir: Option[String], hadoopConfigMapName: Option[String]) @@ -72,7 +72,7 @@ private[spark] case class KubernetesConf[T <: KubernetesRoleSpecificConf]( roleEnvs: Map[String, String], roleVolumes: Iterable[KubernetesVolumeSpec[_ <: KubernetesVolumeSpecificConf]], sparkFiles: Seq[String], - hadoopConfDir: Option[HadoopConfSpecConf]) { + hadoopConfDir: Option[HadoopConfSpec]) { def hadoopConfigMapName: String = s"$appResourceNamePrefix-hadoop-config" @@ -201,9 +201,9 @@ private[spark] object KubernetesConf { 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 hadoopConfSpecConf = + val hadoopConfSpec = if (hadoopConfDir.isDefined || hadoopConfigMapName.isDefined) { - Some(HadoopConfSpecConf(hadoopConfDir, hadoopConfigMapName)) + Some(HadoopConfSpec(hadoopConfDir, hadoopConfigMapName)) } else { None } @@ -220,7 +220,7 @@ private[spark] object KubernetesConf { driverEnvs, driverVolumes, sparkFiles, - hadoopConfSpecConf) + hadoopConfSpec) } def createExecutorConf( 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 5dbc234738ae..340489e7efc6 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 @@ -38,46 +38,43 @@ private[spark] class KerberosConfDriverFeatureStep( "Ensure that HADOOP_CONF_DIR is defined either via env or a pre-defined ConfigMap") private val hadoopConfDirSpec = kubernetesConf.hadoopConfDir.get private val conf = kubernetesConf.sparkConf - private val maybePrincipal = conf.get(org.apache.spark.internal.config.PRINCIPAL) - private val maybeKeytab = conf.get(org.apache.spark.internal.config.KEYTAB) - private val maybeExistingSecretName = conf.get(KUBERNETES_KERBEROS_DT_SECRET_NAME) - private val maybeExistingSecretItemKey = - conf.get(KUBERNETES_KERBEROS_DT_SECRET_ITEM_KEY) - private val maybeKrb5File = - conf.get(KUBERNETES_KERBEROS_KRB5_FILE) - private val maybeKrb5CMap = - conf.get(KUBERNETES_KERBEROS_KRB5_CONFIG_MAP) + 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 isKerberosEnabled = (hadoopConfDirSpec.hadoopConfDir.isDefined && kubeTokenManager.isSecurityEnabled) || (hadoopConfDirSpec.hadoopConfigMapName.isDefined && - (maybeKrb5File.isDefined || maybeKrb5CMap.isDefined)) + (krb5File.isDefined || krb5CMap.isDefined)) - require(maybeKeytab.isEmpty || isKerberosEnabled, + require(keytab.isEmpty || isKerberosEnabled, "You must enable Kerberos support if you are specifying a Kerberos Keytab") - require(maybeExistingSecretName.isEmpty || isKerberosEnabled, + require(existingSecretName.isEmpty || isKerberosEnabled, "You must enable Kerberos support if you are specifying a Kerberos Secret") - require((maybeKrb5File.isEmpty || maybeKrb5CMap.isEmpty) || isKerberosEnabled, + require((krb5File.isEmpty || krb5CMap.isEmpty) || isKerberosEnabled, "You must specify either a krb5 file location or a ConfigMap with a krb5 file") KubernetesUtils.requireNandDefined( - maybeKrb5File, - maybeKrb5CMap, + krb5File, + krb5CMap, "Do not specify both a Krb5 local file and the ConfigMap as the creation " + "of an additional ConfigMap, when one is already specified, is extraneous") KubernetesUtils.requireBothOrNeitherDefined( - maybeKeytab, - maybePrincipal, + keytab, + principal, "If a Kerberos principal is specified you must also specify a Kerberos keytab", "If a Kerberos keytab is specified you must also specify a Kerberos principal") KubernetesUtils.requireBothOrNeitherDefined( - maybeExistingSecretName, - maybeExistingSecretItemKey, + existingSecretName, + existingSecretItemKey, "If a secret data item-key where the data of the Kerberos Delegation Token is specified" + " you must also specify the name of the secret", "If a secret storing a Kerberos Delegation Token is specified you must also" + @@ -94,21 +91,21 @@ private[spark] class KerberosConfDriverFeatureStep( } // Either use pre-existing secret or login to create new Secret with DT stored within - private val hadoopSpec: Option[KerberosConfigSpec] = (for { - secretName <- maybeExistingSecretName - secretItemKey <- maybeExistingSecretItemKey + private val kerberosConfSpec: Option[KerberosConfigSpec] = (for { + secretName <- existingSecretName + secretItemKey <- existingSecretItemKey } yield { KerberosConfigSpec( - dtSecret = None, - dtSecretName = secretName, - dtSecretItemKey = secretItemKey, - jobUserName = kubeTokenManager.getCurrentUser.getShortUserName) + dtSecret = None, + dtSecretName = secretName, + dtSecretItemKey = secretItemKey, + jobUserName = kubeTokenManager.getCurrentUser.getShortUserName) }).orElse( if (isKerberosEnabled) { Some(HadoopKerberosLogin.buildSpec( - conf, - kubernetesConf.appResourceNamePrefix, - kubeTokenManager)) + conf, + kubernetesConf.appResourceNamePrefix, + kubeTokenManager)) } else { None } @@ -120,15 +117,15 @@ private[spark] class KerberosConfDriverFeatureStep( newHadoopConfigMapName, hadoopConfDirSpec.hadoopConfigMapName, pod) - hadoopSpec.map { hSpec => + kerberosConfSpec.map { hSpec => HadoopBootstrapUtil.bootstrapKerberosPod( - hSpec.dtSecretName, - hSpec.dtSecretItemKey, - hSpec.jobUserName, - maybeKrb5File, - Some(kubernetesConf.kRBConfigMapName), - maybeKrb5CMap, - hadoopBasedSparkPod) + hSpec.dtSecretName, + hSpec.dtSecretItemKey, + hSpec.jobUserName, + krb5File, + Some(kubernetesConf.kRBConfigMapName), + krb5CMap, + hadoopBasedSparkPod) }.getOrElse( HadoopBootstrapUtil.bootstrapSparkUserPod( kubeTokenManager.getCurrentUser.getShortUserName, @@ -136,11 +133,11 @@ private[spark] class KerberosConfDriverFeatureStep( } override def getAdditionalPodSystemProperties(): Map[String, String] = { - val resolvedConfValues = hadoopSpec.map { hSpec => - Map(KERBEROS_KEYTAB_SECRET_NAME -> hSpec.dtSecretName, - KERBEROS_KEYTAB_SECRET_KEY -> hSpec.dtSecretItemKey, + val resolvedConfValues = kerberosConfSpec.map { hSpec => + Map(KERBEROS_DT_SECRET_NAME -> hSpec.dtSecretName, + KERBEROS_DT_SECRET_KEY -> hSpec.dtSecretItemKey, KERBEROS_SPARK_USER_NAME -> hSpec.jobUserName, - KRB5_CONFIG_MAP_NAME -> maybeKrb5CMap.getOrElse(kubernetesConf.kRBConfigMapName)) + KRB5_CONFIG_MAP_NAME -> krb5CMap.getOrElse(kubernetesConf.kRBConfigMapName)) }.getOrElse( Map(KERBEROS_SPARK_USER_NAME -> kubeTokenManager.getCurrentUser.getShortUserName)) @@ -161,7 +158,7 @@ private[spark] class KerberosConfDriverFeatureStep( } // krb5 ConfigMap - val krb5ConfigMap = maybeKrb5File.map { fileLocation => + val krb5ConfigMap = krb5File.map { fileLocation => HadoopBootstrapUtil.buildkrb5ConfigMap( kubernetesConf.kRBConfigMapName, fileLocation) @@ -169,7 +166,7 @@ private[spark] class KerberosConfDriverFeatureStep( // Kerberos DT Secret val kerberosDTSecret = for { - hSpec <- hadoopSpec + hSpec <- kerberosConfSpec kDtSecret <- hSpec.dtSecret } yield { kDtSecret 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 eb5e956a4825..15ec619ecd5d 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 @@ -38,8 +38,8 @@ private[spark] class KerberosConfExecutorFeatureStep( override def configurePod(pod: SparkPod): SparkPod = { logInfo(s"Mounting Resources for Kerberos") HadoopBootstrapUtil.bootstrapKerberosPod( - sparkConf.get(KERBEROS_KEYTAB_SECRET_NAME), - sparkConf.get(KERBEROS_KEYTAB_SECRET_KEY), + sparkConf.get(KERBEROS_DT_SECRET_NAME), + sparkConf.get(KERBEROS_DT_SECRET_KEY), sparkConf.get(KERBEROS_SPARK_USER_NAME), None, None, diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadooputils/HadoopBootstrapUtil.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadooputils/HadoopBootstrapUtil.scala index 84a8c2509524..aba63af4382b 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadooputils/HadoopBootstrapUtil.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadooputils/HadoopBootstrapUtil.scala @@ -35,9 +35,9 @@ private[spark] object HadoopBootstrapUtil { * @param dtSecretName Name of the secret that stores the Delegation Token * @param dtSecretItemKey Name of the Item Key storing the Delegation Token * @param userName Name of the SparkUser to set SPARK_USER - * @param maybeFileLocation Optional Location of the krb5 file + * @param fileLocation Optional Location of the krb5 file * @param newKrb5ConfName Optional location of the ConfigMap for Krb5 - * @param maybeKrb5ConfName Optional name of ConfigMap for Krb5 + * @param existingKrb5ConfName Optional name of ConfigMap for Krb5 * @param pod Input pod to be appended to * @return a modified SparkPod */ @@ -45,12 +45,12 @@ private[spark] object HadoopBootstrapUtil { dtSecretName: String, dtSecretItemKey: String, userName: String, - maybeFileLocation: Option[String], + fileLocation: Option[String], newKrb5ConfName: Option[String], - maybeKrb5ConfName: Option[String], + existingKrb5ConfName: Option[String], pod: SparkPod) : SparkPod = { - val maybePreConfigMapVolume = maybeKrb5ConfName.map { kconf => + val preConfigMapVolume = existingKrb5ConfName.map { kconf => new VolumeBuilder() .withName(KRB_FILE_VOLUME) .withNewConfigMap() @@ -58,11 +58,11 @@ private[spark] object HadoopBootstrapUtil { .endConfigMap() .build() } - val maybeCreateConfigMapVolume = for { - fileLocation <- maybeFileLocation + val ceateConfigMapVolume = for { + fLocation <- fileLocation krb5ConfName <- newKrb5ConfName } yield { - val krb5File = new File(fileLocation) + val krb5File = new File(fLocation) val fileStringPath = krb5File.toPath.getFileName.toString new VolumeBuilder() .withName(KRB_FILE_VOLUME) @@ -77,8 +77,8 @@ private[spark] object HadoopBootstrapUtil { } // Breaking up Volume Creation for clarity - val configMapVolume = maybePreConfigMapVolume.getOrElse( - maybeCreateConfigMapVolume.get) + val configMapVolume = preConfigMapVolume.getOrElse( + ceateConfigMapVolume.get) val kerberizedPod = new PodBuilder(pod.pod) .editOrNewSpec() @@ -151,18 +151,18 @@ private[spark] object HadoopBootstrapUtil { * Bootstraping the container with ConfigMaps that store * Hadoop configuration files * - * @param maybeHadoopConfDir directory location of HADOOP_CONF_DIR env + * @param hadoopConfDir directory location of HADOOP_CONF_DIR env * @param newHadoopConfigMapName name of the new configMap for HADOOP_CONF_DIR - * @param oldHadoopConfigMapName name of the pre-defined configMap for HADOOP_CONF_DIR + * @param existingHadoopConfigMapName name of the pre-defined configMap for HADOOP_CONF_DIR * @param pod Input pod to be appended to * @return a modified SparkPod */ def bootstrapHadoopConfDir( - maybeHadoopConfDir: Option[String], + hadoopConfDir: Option[String], newHadoopConfigMapName: Option[String], - oldHadoopConfigMapName: Option[String], + existingHadoopConfigMapName: Option[String], pod: SparkPod) : SparkPod = { - val maybePreConfigMapVolume = oldHadoopConfigMapName.map { hConf => + val preConfigMapVolume = existingHadoopConfigMapName.map { hConf => new VolumeBuilder() .withName(HADOOP_FILE_VOLUME) .withNewConfigMap() @@ -170,8 +170,8 @@ private[spark] object HadoopBootstrapUtil { .endConfigMap() .build() } - val maybeCreateConfigMapVolume = for { - dirLocation <- maybeHadoopConfDir + val createConfigMapVolume = for { + dirLocation <- hadoopConfDir hConfName <- newHadoopConfigMapName } yield { val hadoopConfigFiles = getHadoopConfFiles(dirLocation) @@ -190,8 +190,8 @@ private[spark] object HadoopBootstrapUtil { .build() } // Breaking up Volume Creation for clarity - val configMapVolume = maybePreConfigMapVolume.getOrElse( - maybeCreateConfigMapVolume.get) + val configMapVolume = preConfigMapVolume.getOrElse( + createConfigMapVolume.get) val hadoopSupportedPod = new PodBuilder(pod.pod) .editSpec() 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 4e032689a2d1..6199a8ae3043 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 @@ -53,8 +53,8 @@ private[spark] class KubernetesExecutorBuilder( kubernetesConf: KubernetesConf[KubernetesExecutorSpecificConf]): SparkPod = { val sparkConf = kubernetesConf.sparkConf val maybeHadoopConfigMap = sparkConf.getOption(HADOOP_CONFIG_MAP_NAME) - val maybeDTSecretName = sparkConf.getOption(KERBEROS_KEYTAB_SECRET_NAME) - val maybeDTDataItem = sparkConf.getOption(KERBEROS_KEYTAB_SECRET_KEY) + 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) { 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 909ad177986e..c919ab02b8c6 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 @@ -270,7 +270,7 @@ class KubernetesDriverBuilderSuite extends SparkFunSuite { Nil, Seq.empty[String], hadoopConfDir = Some( - HadoopConfSpecConf( + HadoopConfSpec( Some("/var/hadoop-conf"), None))) validateStepTypesApplied( @@ -301,7 +301,7 @@ class KubernetesDriverBuilderSuite extends SparkFunSuite { Nil, Seq.empty[String], hadoopConfDir = Some( - HadoopConfSpecConf( + HadoopConfSpec( None, Some("pre-defined-configMapName")))) validateStepTypesApplied( 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 e364cec8a333..b572dac2bf62 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 @@ -150,7 +150,7 @@ class KubernetesExecutorBuilderSuite extends SparkFunSuite { Map.empty, Nil, Seq.empty[String], - Some(HadoopConfSpecConf(Some("/var/hadoop-conf"), None))) + Some(HadoopConfSpec(Some("/var/hadoop-conf"), None))) validateStepTypesApplied( builderUnderTest.buildFromFeatures(conf), BASIC_STEP_TYPE, @@ -165,8 +165,8 @@ class KubernetesExecutorBuilderSuite extends SparkFunSuite { .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_KEYTAB_SECRET_NAME, "dt-secret") - .set(KERBEROS_KEYTAB_SECRET_KEY, "dt-key"), + .set(KERBEROS_DT_SECRET_NAME, "dt-secret") + .set(KERBEROS_DT_SECRET_KEY, "dt-key"), KubernetesExecutorSpecificConf( "executor-id", Some(new PodBuilder().build())), "prefix", @@ -178,7 +178,7 @@ class KubernetesExecutorBuilderSuite extends SparkFunSuite { Map.empty, Nil, Seq.empty[String], - Some(HadoopConfSpecConf(None, Some("pre-defined-onfigMapName")))) + Some(HadoopConfSpec(None, Some("pre-defined-onfigMapName")))) validateStepTypesApplied( builderUnderTest.buildFromFeatures(conf), BASIC_STEP_TYPE, From e2f806386dfd836d12804997c748076083977378 Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Tue, 9 Oct 2018 12:35:27 -0700 Subject: [PATCH 22/25] hopefully final comment resolution --- .../apache/spark/deploy/k8s/KubernetesConf.scala | 2 +- .../features/HadoopConfExecutorFeatureStep.scala | 6 +++--- .../features/KerberosConfDriverFeatureStep.scala | 4 ++-- .../hadooputils/HadoopKerberosLogin.scala | 6 ++---- .../KubernetesHadoopDelegationTokenManager.scala | 3 --- .../k8s/submit/KubernetesDriverBuilder.scala | 2 +- .../features/BasicDriverFeatureStepSuite.scala | 8 ++++---- .../features/BasicExecutorFeatureStepSuite.scala | 8 ++++---- ...erKubernetesCredentialsFeatureStepSuite.scala | 6 +++--- .../features/DriverServiceFeatureStepSuite.scala | 12 ++++++------ .../features/EnvSecretsFeatureStepSuite.scala | 2 +- .../k8s/features/LocalDirsFeatureStepSuite.scala | 2 +- .../features/MountSecretsFeatureStepSuite.scala | 2 +- .../features/MountVolumesFeatureStepSuite.scala | 2 +- .../bindings/JavaDriverFeatureStepSuite.scala | 2 +- .../bindings/PythonDriverFeatureStepSuite.scala | 4 ++-- .../bindings/RDriverFeatureStepSuite.scala | 2 +- .../spark/deploy/k8s/submit/ClientSuite.scala | 2 +- .../submit/KubernetesDriverBuilderSuite.scala | 16 ++++++++-------- 19 files changed, 43 insertions(+), 48 deletions(-) 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 731240f8e692..ab7bc9d274b1 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 @@ -72,7 +72,7 @@ private[spark] case class KubernetesConf[T <: KubernetesRoleSpecificConf]( roleEnvs: Map[String, String], roleVolumes: Iterable[KubernetesVolumeSpec[_ <: KubernetesVolumeSpecificConf]], sparkFiles: Seq[String], - hadoopConfDir: Option[HadoopConfSpec]) { + hadoopConfSpec: Option[HadoopConfSpec]) { def hadoopConfigMapName: String = s"$appResourceNamePrefix-hadoop-config" 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 89af99567688..56d739bd4469 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 @@ -34,12 +34,12 @@ private[spark] class HadoopConfExecutorFeatureStep( override def configurePod(pod: SparkPod): SparkPod = { val sparkConf = kubernetesConf.sparkConf - val maybeHadoopConfDir = sparkConf.getOption(HADOOP_CONFIG_MAP_NAME) - require(maybeHadoopConfDir.isDefined, + val hadoopConfDirCMapName = 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, maybeHadoopConfDir, pod) + HadoopBootstrapUtil.bootstrapHadoopConfDir(None, None, hadoopConfDirCMapName, pod) } override def getAdditionalPodSystemProperties(): Map[String, String] = Map.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 340489e7efc6..f7c5479f69ce 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,9 +34,9 @@ private[spark] class KerberosConfDriverFeatureStep( kubernetesConf: KubernetesConf[KubernetesDriverSpecificConf]) extends KubernetesFeatureConfigStep with Logging { - require(kubernetesConf.hadoopConfDir.isDefined, + require(kubernetesConf.hadoopConfSpec.isDefined, "Ensure that HADOOP_CONF_DIR is defined either via env or a pre-defined ConfigMap") - private val hadoopConfDirSpec = kubernetesConf.hadoopConfDir.get + private val hadoopConfDirSpec = kubernetesConf.hadoopConfSpec.get private val conf = kubernetesConf.sparkConf private val principal = conf.get(org.apache.spark.internal.config.PRINCIPAL) private val keytab = conf.get(org.apache.spark.internal.config.KEYTAB) 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 index ed9c1368e60d..b8bb344d596e 100644 --- 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 @@ -48,10 +48,8 @@ private[spark] object HadoopKerberosLogin { submissionSparkConf, hadoopConf) require(tokenData.nonEmpty, "Did not obtain any delegation tokens") - val currentTime = tokenManager.getCurrentTime - val initialTokenDataKeyName = s"$KERBEROS_SECRET_KEY_PREFIX-$currentTime" - val newSecretName = - s"$kubernetesResourceNamePrefix-$KERBEROS_DELEGEGATION_TOKEN_SECRET_NAME.$currentTime" + val initialTokenDataKeyName = KERBEROS_SECRET_KEY_PREFIX + val newSecretName = s"$kubernetesResourceNamePrefix-$KERBEROS_DELEGEGATION_TOKEN_SECRET_NAME" val secretDT = new SecretBuilder() .withNewMetadata() 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 index 2f4b7927939d..713bb6895567 100644 --- 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 @@ -25,7 +25,6 @@ import org.apache.spark.SparkConf import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.deploy.security.HadoopDelegationTokenManager import org.apache.spark.internal.Logging -import org.apache.spark.util.{Clock, SystemClock} /** * The KubernetesHadoopDelegationTokenManager fetches Hadoop delegation tokens @@ -37,14 +36,12 @@ private[spark] class KubernetesHadoopDelegationTokenManager( tokenManager: HadoopDelegationTokenManager) extends Logging { // HadoopUGI Util methods - private val clock: Clock = new SystemClock() def getCurrentUser: UserGroupInformation = UserGroupInformation.getCurrentUser def getShortUserName : String = getCurrentUser.getShortUserName def getFileSystem(hadoopConf: Configuration) : FileSystem = FileSystem.get(hadoopConf) def isSecurityEnabled: Boolean = UserGroupInformation.isSecurityEnabled def loginUserFromKeytabAndReturnUGI(principal: String, keytab: String): UserGroupInformation = UserGroupInformation.loginUserFromKeytabAndReturnUGI(principal, keytab) - def getCurrentTime: Long = clock.getTimeMillis() def serializeCreds(creds: Credentials): Array[Byte] = SparkHadoopUtil.get.serialize(creds) def nextRT(rt: Long, conf: SparkConf): Long = SparkHadoopUtil.nextCredentialRenewalTime(rt, conf) 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 09c8a2b0d699..b0b53321abd2 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 @@ -85,7 +85,7 @@ private[spark] class KubernetesDriverBuilder( .getOrElse(provideJavaStep(kubernetesConf)) val maybeHadoopConfigStep = - kubernetesConf.hadoopConfDir.map { _ => + kubernetesConf.hadoopConfSpec.map { _ => provideHadoopGlobalStep(kubernetesConf)} val allFeatures: Seq[KubernetesFeatureConfigStep] = 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 48ff9069a229..eebdd157da63 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 @@ -78,7 +78,7 @@ class BasicDriverFeatureStepSuite extends SparkFunSuite { DRIVER_ENVS, Nil, Seq.empty[String], - hadoopConfDir = None) + hadoopConfSpec = None) val featureStep = new BasicDriverFeatureStep(kubernetesConf) val basePod = SparkPod.initialPod() @@ -157,7 +157,7 @@ class BasicDriverFeatureStepSuite extends SparkFunSuite { DRIVER_ENVS, Nil, Seq.empty[String], - hadoopConfDir = None) + hadoopConfSpec = None) val pythonKubernetesConf = KubernetesConf( pythonSparkConf, @@ -175,7 +175,7 @@ class BasicDriverFeatureStepSuite extends SparkFunSuite { DRIVER_ENVS, Nil, Seq.empty[String], - hadoopConfDir = None) + hadoopConfSpec = None) val javaFeatureStep = new BasicDriverFeatureStep(javaKubernetesConf) val pythonFeatureStep = new BasicDriverFeatureStep(pythonKubernetesConf) val basePod = SparkPod.initialPod() @@ -205,7 +205,7 @@ class BasicDriverFeatureStepSuite extends SparkFunSuite { DRIVER_ENVS, Nil, allFiles, - hadoopConfDir = None) + hadoopConfSpec = None) val step = new BasicDriverFeatureStep(kubernetesConf) val additionalProperties = step.getAdditionalPodSystemProperties() 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 87bf400e5387..41f34bd45cd5 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 @@ -92,7 +92,7 @@ class BasicExecutorFeatureStepSuite Map.empty, Nil, Seq.empty[String], - hadoopConfDir = None)) + hadoopConfSpec = None)) val executor = step.configurePod(SparkPod.initialPod()) // The executor pod name and default labels. @@ -133,7 +133,7 @@ class BasicExecutorFeatureStepSuite Map.empty, Nil, Seq.empty[String], - hadoopConfDir = None)) + hadoopConfSpec = None)) assert(step.configurePod(SparkPod.initialPod()).pod.getSpec.getHostname.length === 63) } @@ -155,7 +155,7 @@ class BasicExecutorFeatureStepSuite Map("qux" -> "quux"), Nil, Seq.empty[String], - hadoopConfDir = None)) + hadoopConfSpec = None)) val executor = step.configurePod(SparkPod.initialPod()) checkEnv(executor, @@ -183,7 +183,7 @@ class BasicExecutorFeatureStepSuite Map.empty, Nil, Seq.empty[String], - hadoopConfDir = None)) + hadoopConfSpec = None)) 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") 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 ce6e02cbe439..8675ceb48cf6 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 @@ -63,7 +63,7 @@ class DriverKubernetesCredentialsFeatureStepSuite extends SparkFunSuite with Bef Map.empty, Nil, Seq.empty[String], - hadoopConfDir = None) + hadoopConfSpec = None) val kubernetesCredentialsStep = new DriverKubernetesCredentialsFeatureStep(kubernetesConf) assert(kubernetesCredentialsStep.configurePod(BASE_DRIVER_POD) === BASE_DRIVER_POD) assert(kubernetesCredentialsStep.getAdditionalPodSystemProperties().isEmpty) @@ -96,7 +96,7 @@ class DriverKubernetesCredentialsFeatureStepSuite extends SparkFunSuite with Bef Map.empty, Nil, Seq.empty[String], - hadoopConfDir = None) + hadoopConfSpec = None) val kubernetesCredentialsStep = new DriverKubernetesCredentialsFeatureStep(kubernetesConf) assert(kubernetesCredentialsStep.configurePod(BASE_DRIVER_POD) === BASE_DRIVER_POD) @@ -136,7 +136,7 @@ class DriverKubernetesCredentialsFeatureStepSuite extends SparkFunSuite with Bef Map.empty, Nil, Seq.empty[String], - hadoopConfDir = None) + hadoopConfSpec = None) val kubernetesCredentialsStep = new DriverKubernetesCredentialsFeatureStep(kubernetesConf) val resolvedProperties = kubernetesCredentialsStep.getAdditionalPodSystemProperties() val expectedSparkConf = Map( 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 ad084f43bb71..5c3e80150151 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 @@ -69,7 +69,7 @@ class DriverServiceFeatureStepSuite extends SparkFunSuite with BeforeAndAfter { Map.empty, Nil, Seq.empty[String], - hadoopConfDir = None)) + hadoopConfSpec = None)) assert(configurationStep.configurePod(SparkPod.initialPod()) === SparkPod.initialPod()) assert(configurationStep.getAdditionalKubernetesResources().size === 1) assert(configurationStep.getAdditionalKubernetesResources().head.isInstanceOf[Service]) @@ -102,7 +102,7 @@ class DriverServiceFeatureStepSuite extends SparkFunSuite with BeforeAndAfter { Map.empty, Nil, Seq.empty[String], - hadoopConfDir = None)) + hadoopConfSpec = None)) val expectedServiceName = SHORT_RESOURCE_NAME_PREFIX + DriverServiceFeatureStep.DRIVER_SVC_POSTFIX val expectedHostName = s"$expectedServiceName.my-namespace.svc" @@ -125,7 +125,7 @@ class DriverServiceFeatureStepSuite extends SparkFunSuite with BeforeAndAfter { Map.empty, Nil, Seq.empty[String], - hadoopConfDir = None)) + hadoopConfSpec = None)) val resolvedService = configurationStep .getAdditionalKubernetesResources() .head @@ -157,7 +157,7 @@ class DriverServiceFeatureStepSuite extends SparkFunSuite with BeforeAndAfter { Map.empty, Nil, Seq.empty[String], - hadoopConfDir = None), + hadoopConfSpec = None), clock) val driverService = configurationStep .getAdditionalKubernetesResources() @@ -186,7 +186,7 @@ class DriverServiceFeatureStepSuite extends SparkFunSuite with BeforeAndAfter { Map.empty, Nil, Seq.empty[String], - hadoopConfDir = None), + hadoopConfSpec = None), clock) fail("The driver bind address should not be allowed.") } catch { @@ -213,7 +213,7 @@ class DriverServiceFeatureStepSuite extends SparkFunSuite with BeforeAndAfter { Map.empty, Nil, Seq.empty[String], - hadoopConfDir = None), + hadoopConfSpec = None), clock) fail("The driver host address should not be allowed.") } catch { 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 fd071620ff18..43796b77efdc 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 @@ -47,7 +47,7 @@ class EnvSecretsFeatureStepSuite extends SparkFunSuite{ Map.empty, Nil, Seq.empty[String], - hadoopConfDir = None) + hadoopConfSpec = None) val step = new EnvSecretsFeatureStep(kubernetesConf) val driverContainerWithEnvSecrets = step.configurePod(baseDriverPod).container 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 bf76dd101c54..3a4e60547d7f 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 @@ -49,7 +49,7 @@ class LocalDirsFeatureStepSuite extends SparkFunSuite with BeforeAndAfter { Map.empty, Nil, Seq.empty[String], - hadoopConfDir = None) + hadoopConfSpec = None) } test("Resolve to default local dir if neither env nor configuration are set") { 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 1d5790254f1a..18e3d773f690 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 @@ -45,7 +45,7 @@ class MountSecretsFeatureStepSuite extends SparkFunSuite { Map.empty, Nil, Seq.empty[String], - hadoopConfDir = None) + hadoopConfSpec = None) 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 132629c3c446..0d0a5fb951f6 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 @@ -37,7 +37,7 @@ class MountVolumesFeatureStepSuite extends SparkFunSuite { roleEnvs = Map.empty, roleVolumes = Nil, sparkFiles = Nil, - hadoopConfDir = None) + hadoopConfSpec = None) test("Mounts hostPath volumes") { val volumeConf = KubernetesVolumeSpec( diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/bindings/JavaDriverFeatureStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/bindings/JavaDriverFeatureStepSuite.scala index 13b8a47f6c18..9172e0c3dc40 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/bindings/JavaDriverFeatureStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/bindings/JavaDriverFeatureStepSuite.scala @@ -44,7 +44,7 @@ class JavaDriverFeatureStepSuite extends SparkFunSuite { roleEnvs = Map.empty, roleVolumes = Nil, sparkFiles = Seq.empty[String], - hadoopConfDir = None) + hadoopConfSpec = None) val step = new JavaDriverFeatureStep(kubernetesConf) val driverPod = step.configurePod(baseDriverPod).pod diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/bindings/PythonDriverFeatureStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/bindings/PythonDriverFeatureStepSuite.scala index 98517f3eb45d..2bcc6465b79d 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/bindings/PythonDriverFeatureStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/bindings/PythonDriverFeatureStepSuite.scala @@ -54,7 +54,7 @@ class PythonDriverFeatureStepSuite extends SparkFunSuite { roleEnvs = Map.empty, roleVolumes = Nil, sparkFiles = Seq.empty[String], - hadoopConfDir = None) + hadoopConfSpec = None) val step = new PythonDriverFeatureStep(kubernetesConf) val driverPod = step.configurePod(baseDriverPod).pod @@ -92,7 +92,7 @@ class PythonDriverFeatureStepSuite extends SparkFunSuite { roleEnvs = Map.empty, roleVolumes = Nil, sparkFiles = Seq.empty[String], - hadoopConfDir = None) + hadoopConfSpec = None) val step = new PythonDriverFeatureStep(kubernetesConf) val driverContainerwithPySpark = step.configurePod(baseDriverPod).container val args = driverContainerwithPySpark diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/bindings/RDriverFeatureStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/bindings/RDriverFeatureStepSuite.scala index 8ba74adaf85e..17af6011a17d 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/bindings/RDriverFeatureStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/bindings/RDriverFeatureStepSuite.scala @@ -48,7 +48,7 @@ class RDriverFeatureStepSuite extends SparkFunSuite { roleEnvs = Map.empty, roleVolumes = Seq.empty, sparkFiles = Seq.empty[String], - hadoopConfDir = None) + hadoopConfSpec = None) val step = new RDriverFeatureStep(kubernetesConf) val driverContainerwithR = step.configurePod(baseDriverPod).container 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 3dd7f4781e0c..ae13df39b7a7 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 @@ -143,7 +143,7 @@ class ClientSuite extends SparkFunSuite with BeforeAndAfter { Map.empty, Nil, Seq.empty[String], - hadoopConfDir = None) + hadoopConfSpec = None) when(driverBuilder.buildFromFeatures(kubernetesConf)).thenReturn(BUILT_KUBERNETES_SPEC) when(kubernetesClient.pods()).thenReturn(podOperations) when(podOperations.withName(POD_NAME)).thenReturn(namedPods) 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 c919ab02b8c6..051d7b6994f5 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 @@ -99,7 +99,7 @@ class KubernetesDriverBuilderSuite extends SparkFunSuite { Map.empty, Nil, Seq.empty[String], - hadoopConfDir = None) + hadoopConfSpec = None) validateStepTypesApplied( builderUnderTest.buildFromFeatures(conf), BASIC_STEP_TYPE, @@ -126,7 +126,7 @@ class KubernetesDriverBuilderSuite extends SparkFunSuite { Map.empty, Nil, Seq.empty[String], - hadoopConfDir = None) + hadoopConfSpec = None) validateStepTypesApplied( builderUnderTest.buildFromFeatures(conf), BASIC_STEP_TYPE, @@ -155,7 +155,7 @@ class KubernetesDriverBuilderSuite extends SparkFunSuite { Map.empty, Nil, Seq.empty[String], - hadoopConfDir = None) + hadoopConfSpec = None) validateStepTypesApplied( builderUnderTest.buildFromFeatures(conf), BASIC_STEP_TYPE, @@ -182,7 +182,7 @@ class KubernetesDriverBuilderSuite extends SparkFunSuite { Map.empty, Nil, Seq.empty[String], - hadoopConfDir = None) + hadoopConfSpec = None) validateStepTypesApplied( builderUnderTest.buildFromFeatures(conf), BASIC_STEP_TYPE, @@ -214,7 +214,7 @@ class KubernetesDriverBuilderSuite extends SparkFunSuite { Map.empty, volumeSpec :: Nil, Seq.empty[String], - hadoopConfDir = None) + hadoopConfSpec = None) validateStepTypesApplied( builderUnderTest.buildFromFeatures(conf), BASIC_STEP_TYPE, @@ -242,7 +242,7 @@ class KubernetesDriverBuilderSuite extends SparkFunSuite { Map.empty, Nil, Seq.empty[String], - hadoopConfDir = None) + hadoopConfSpec = None) validateStepTypesApplied( builderUnderTest.buildFromFeatures(conf), BASIC_STEP_TYPE, @@ -269,7 +269,7 @@ class KubernetesDriverBuilderSuite extends SparkFunSuite { Map.empty, Nil, Seq.empty[String], - hadoopConfDir = Some( + hadoopConfSpec = Some( HadoopConfSpec( Some("/var/hadoop-conf"), None))) @@ -300,7 +300,7 @@ class KubernetesDriverBuilderSuite extends SparkFunSuite { Map.empty, Nil, Seq.empty[String], - hadoopConfDir = Some( + hadoopConfSpec = Some( HadoopConfSpec( None, Some("pre-defined-configMapName")))) From f3a0ffb3ba217ed90237a9edb296d1d7648b35ac Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Tue, 9 Oct 2018 17:36:57 -0700 Subject: [PATCH 23/25] style issues --- docs/running-on-kubernetes.md | 10 +- docs/security.md | 19 +- .../org/apache/spark/examples/HdfsTest.scala | 2 +- .../org/apache/spark/deploy/k8s/Config.scala | 14 +- .../apache/spark/deploy/k8s/Constants.scala | 2 +- .../spark/deploy/k8s/KubernetesConf.scala | 7 +- .../HadoopConfExecutorFeatureStep.scala | 20 +- .../HadoopSparkUserExecutorFeatureStep.scala | 14 +- .../KerberosConfDriverFeatureStep.scala | 261 +++++++++--------- .../KerberosConfExecutorFeatureStep.scala | 10 +- .../hadooputils/HadoopBootstrapUtil.scala | 66 +++-- .../hadooputils/HadoopKerberosLogin.scala | 59 ++-- .../hadooputils/KerberosConfigSpec.scala | 8 +- ...bernetesHadoopDelegationTokenManager.scala | 28 +- 14 files changed, 256 insertions(+), 264 deletions(-) diff --git a/docs/running-on-kubernetes.md b/docs/running-on-kubernetes.md index e0019e47648b..9dfb52634e21 100644 --- a/docs/running-on-kubernetes.md +++ b/docs/running-on-kubernetes.md @@ -821,10 +821,10 @@ specific to Spark on Kubernetes. - spark.kubernetes.kerberos.krb5.location + spark.kubernetes.kerberos.krb5.path (none) - Specify the local location of the krb5 file to be mounted on the driver and executors for Kerberos interaction. + Specify the local location of the krb5.conf file to be mounted on the driver and executors for Kerberos interaction. It is important to note that the KDC defined needs to be visible from inside the containers. @@ -832,7 +832,7 @@ specific to Spark on Kubernetes. spark.kubernetes.kerberos.krb5.configMapName (none) - Specify the name of the ConfigMap, containing the krb5 file, to be mounted on the driver and executors + Specify the name of the ConfigMap, containing the krb5.conf file, to be mounted on the driver and executors for Kerberos interaction. The KDC defined needs to be visible from inside the containers. The ConfigMap must also be in the same namespace of the driver and executor pods. @@ -849,7 +849,7 @@ specific to Spark on Kubernetes. spark.kubernetes.kerberos.tokenSecret.name (none) - Specify the name of the secret where your existing delegation token is stored. This removes the need for the job user + Specify the name of the secret where your existing delegation tokens are stored. This removes the need for the job user to provide any kerberos credentials for launching a job. @@ -857,7 +857,7 @@ specific to Spark on Kubernetes. spark.kubernetes.kerberos.tokenSecret.itemKey (none) - Specify the item key of the data where your existing delegation token is stored. This removes the need for the job user + Specify the item key of the data where your existing delegation tokens are stored. This removes the need for the job user to provide any kerberos credentials for launching a job. diff --git a/docs/security.md b/docs/security.md index b87393e9fa41..55d987825173 100644 --- a/docs/security.md +++ b/docs/security.md @@ -730,10 +730,9 @@ shared by the Driver and its Executors. As such, there are three ways of submitt In all cases you must define the environment variable: `HADOOP_CONF_DIR` or `spark.kubernetes.hadoop.configMapName` as well as either -`spark.kubernetes.kerberos.krb5.location` or `spark.kubernetes.kerberos.krb5.configMapName`. +`spark.kubernetes.kerberos.krb5.path` or `spark.kubernetes.kerberos.krb5.configMapName`. -It also important to note that the KDC needs to be visible from inside the containers if the user uses a local -krb5 file. +It also important to note that the KDC needs to be visible from inside the containers. If a user wishes to use a remote HADOOP_CONF directory, that contains the Hadoop configuration files, this could be achieved by setting `spark.kubernetes.hadoop.configMapName` to a pre-existing ConfigMap. @@ -748,8 +747,8 @@ achieved by setting `spark.kubernetes.hadoop.configMapName` to a pre-existing Co --conf spark.executor.instances=1 \ --conf spark.app.name=spark-hdfs \ --conf spark.kubernetes.container.image=spark:latest \ - --conf spark.kubernetes.kerberos.krb5.locationn=/etc/krb5.conf \ - local:///opt/spark/examples/jars/spark-examples_-SNAPSHOT.jar \ + --conf spark.kubernetes.kerberos.krb5.path=/etc/krb5.conf \ + local:///opt/spark/examples/jars/spark-examples_.jar \ ``` 2. Submitting with a local Keytab and Principal @@ -763,8 +762,8 @@ achieved by setting `spark.kubernetes.hadoop.configMapName` to a pre-existing Co --conf spark.kubernetes.container.image=spark:latest \ --conf spark.kerberos.keytab= \ --conf spark.kerberos.principal= \ - --conf spark.kubernetes.kerberos.krb5.location=/etc/krb5.conf \ - local:///opt/spark/examples/jars/spark-examples_-SNAPSHOT.jar \ + --conf spark.kubernetes.kerberos.krb5.path=/etc/krb5.conf \ + local:///opt/spark/examples/jars/spark-examples_.jar \ ``` @@ -779,8 +778,8 @@ achieved by setting `spark.kubernetes.hadoop.configMapName` to a pre-existing Co --conf spark.kubernetes.container.image=spark:latest \ --conf spark.kubernetes.kerberos.tokenSecret.name= \ --conf spark.kubernetes.kerberos.tokenSecret.itemKey= \ - --conf spark.kubernetes.kerberos.krb5.location=/etc/krb5.conf \ - local:///opt/spark/examples/jars/spark-examples_-SNAPSHOT.jar \ + --conf spark.kubernetes.kerberos.krb5.path=/etc/krb5.conf \ + local:///opt/spark/examples/jars/spark-examples_.jar \ ``` @@ -797,7 +796,7 @@ achieved by setting `spark.kubernetes.hadoop.configMapName` to a pre-existing Co --conf spark.kubernetes.kerberos.tokenSecret.itemKey= \ --conf spark.kubernetes.hadoop.configMapName= \ --conf spark.kubernetes.kerberos.krb5.configMapName= \ - local:///opt/spark/examples/jars/spark-examples_-SNAPSHOT.jar \ + local:///opt/spark/examples/jars/spark-examples_.jar \ ``` # Event Logging diff --git a/examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala b/examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala index c1f71e6e298e..08af3306a96f 100644 --- a/examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala @@ -42,7 +42,7 @@ object HdfsTest { println(s"Iteration $iter took ${end-start} ms") } println(s"File contents: ${file.map(_.toString).take(1).mkString(",").slice(0, 10)}") - println(s"Returned length(s) of: ${file.map(_.length).collect().mkString(",")}") + println(s"Returned length(s) of: ${file.map(_.length).sum().toString}") spark.stop() } } 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 4115628f6f46..c2ad80c4755a 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 @@ -226,17 +226,17 @@ private[spark] object Config extends Logging { .createWithDefault("2") val KUBERNETES_KERBEROS_KRB5_FILE = - ConfigBuilder("spark.kubernetes.kerberos.krb5.location") - .doc("Specify the local location of the krb5 file to be mounted on the driver " + - "and executors for Kerberos. Note: For Local files the KDC defined needs to be " + + ConfigBuilder("spark.kubernetes.kerberos.krb5.path") + .doc("Specify the local location of the krb5.conf file to be mounted on the driver " + + "and executors for Kerberos. Note: The KDC defined needs to be " + "visible from inside the containers ") .stringConf .createOptional val KUBERNETES_KERBEROS_KRB5_CONFIG_MAP = ConfigBuilder("spark.kubernetes.kerberos.krb5.configMapName") - .doc("Specify the name of the ConfigMap, containing the krb5 file, to be mounted " + - "on the driver and executors for Kerberos. Note: For Local files the KDC defined" + + .doc("Specify the name of the ConfigMap, containing the krb5.conf file, to be mounted " + + "on the driver and executors for Kerberos. Note: The KDC defined" + "needs to be visible from inside the containers ") .stringConf .createOptional @@ -250,14 +250,14 @@ private[spark] object Config extends Logging { val KUBERNETES_KERBEROS_DT_SECRET_NAME = ConfigBuilder("spark.kubernetes.kerberos.tokenSecret.name") - .doc("Specify the name of the secret where your existing delegation token is stored. " + + .doc("Specify the name of the secret where your existing delegation tokens are stored. " + "This removes the need for the job user to provide any keytab for launching a job") .stringConf .createOptional val KUBERNETES_KERBEROS_DT_SECRET_ITEM_KEY = ConfigBuilder("spark.kubernetes.kerberos.tokenSecret.itemKey") - .doc("Specify the item key of the data where your existing delegation token is stored. " + + .doc("Specify the item key of the data where your existing delegation tokens are stored. " + "This removes the need for the job user to provide any keytab for launching a job") .stringConf .createOptional diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Constants.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Constants.scala index 69516912353b..172a9054bb4f 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Constants.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Constants.scala @@ -100,7 +100,7 @@ private[spark] object Constants { "spark.kubernetes.kerberos.dt-secret-key" val KERBEROS_SPARK_USER_NAME = "spark.kubernetes.kerberos.spark-user-name" - val KERBEROS_SECRET_KEY_PREFIX = "hadoop-tokens" + val KERBEROS_SECRET_KEY = "hadoop-tokens" // Hadoop credentials secrets for the Spark app. val SPARK_APP_HADOOP_CREDENTIALS_BASE_DIR = "/mnt/secrets/hadoop-credentials" 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 ab7bc9d274b1..3e30ab2c8353 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 @@ -76,11 +76,10 @@ private[spark] case class KubernetesConf[T <: KubernetesRoleSpecificConf]( def hadoopConfigMapName: String = s"$appResourceNamePrefix-hadoop-config" - def kRBConfigMapName: String = s"$appResourceNamePrefix-krb5-file" + def krbConfigMapName: String = s"$appResourceNamePrefix-krb5-file" - def tokenManager(conf: SparkConf, hConf: Configuration) : KubernetesHadoopDelegationTokenManager = - new KubernetesHadoopDelegationTokenManager( - new HadoopDelegationTokenManager(conf, hConf)) + def tokenManager(conf: SparkConf, hConf: Configuration): KubernetesHadoopDelegationTokenManager = + new KubernetesHadoopDelegationTokenManager(new HadoopDelegationTokenManager(conf, hConf)) def namespace(): String = sparkConf.get(KUBERNETES_NAMESPACE) 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 56d739bd4469..09198e90502b 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 @@ -29,17 +29,17 @@ import org.apache.spark.internal.Logging * pointed to the mounted file directory. */ private[spark] class HadoopConfExecutorFeatureStep( - kubernetesConf: KubernetesConf[KubernetesExecutorSpecificConf]) - extends KubernetesFeatureConfigStep with Logging { + kubernetesConf: KubernetesConf[KubernetesExecutorSpecificConf]) + extends KubernetesFeatureConfigStep with Logging { - override def configurePod(pod: SparkPod): SparkPod = { - val sparkConf = kubernetesConf.sparkConf - val hadoopConfDirCMapName = 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 configurePod(pod: SparkPod): SparkPod = { + val sparkConf = kubernetesConf.sparkConf + val hadoopConfDirCMapName = 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 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 a4774190ae11..65dac0099109 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,16 +28,16 @@ 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( +private[spark] class HadoopSparkUserExecutorFeatureStep( kubernetesConf: KubernetesConf[KubernetesExecutorSpecificConf]) - extends KubernetesFeatureConfigStep with Logging{ + extends KubernetesFeatureConfigStep with Logging { - override def configurePod(pod: SparkPod): SparkPod = { - val sparkUserName = kubernetesConf.sparkConf.get(KERBEROS_SPARK_USER_NAME) + override def configurePod(pod: SparkPod): SparkPod = { + val sparkUserName = kubernetesConf.sparkConf.get(KERBEROS_SPARK_USER_NAME) HadoopBootstrapUtil.bootstrapSparkUserPod(sparkUserName, pod) - } + } - override def getAdditionalPodSystemProperties(): Map[String, String] = Map.empty + override def getAdditionalPodSystemProperties(): Map[String, String] = Map.empty - override def getAdditionalKubernetesResources(): Seq[HasMetadata] = Seq.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 f7c5479f69ce..172310c0b0a6 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 @@ -31,149 +31,138 @@ import org.apache.spark.internal.Logging * HADOOP_CONF_DIR. This runs various bootstrap methods defined in HadoopBootstrapUtil. */ private[spark] class KerberosConfDriverFeatureStep( - kubernetesConf: KubernetesConf[KubernetesDriverSpecificConf]) - extends KubernetesFeatureConfigStep with Logging { + kubernetesConf: KubernetesConf[KubernetesDriverSpecificConf]) + extends KubernetesFeatureConfigStep with Logging { - require(kubernetesConf.hadoopConfSpec.isDefined, + 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 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 isKerberosEnabled = - (hadoopConfDirSpec.hadoopConfDir.isDefined && kubeTokenManager.isSecurityEnabled) || - (hadoopConfDirSpec.hadoopConfigMapName.isDefined && - (krb5File.isDefined || krb5CMap.isDefined)) - - require(keytab.isEmpty || isKerberosEnabled, - "You must enable Kerberos support if you are specifying a Kerberos Keytab") - - require(existingSecretName.isEmpty || isKerberosEnabled, - "You must enable Kerberos support if you are specifying a Kerberos Secret") - - require((krb5File.isEmpty || krb5CMap.isEmpty) || isKerberosEnabled, - "You must specify either a krb5 file location or a ConfigMap with a krb5 file") - - KubernetesUtils.requireNandDefined( - krb5File, - krb5CMap, - "Do not specify both a Krb5 local file and the ConfigMap as the creation " + + private val hadoopConfDirSpec = kubernetesConf.hadoopConfSpec.get + private val conf = kubernetesConf.sparkConf + 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 isKerberosEnabled = + (hadoopConfDirSpec.hadoopConfDir.isDefined && kubeTokenManager.isSecurityEnabled) || + (hadoopConfDirSpec.hadoopConfigMapName.isDefined && + (krb5File.isDefined || krb5CMap.isDefined)) + require(keytab.isEmpty || isKerberosEnabled, + "You must enable Kerberos support if you are specifying a Kerberos Keytab") + + require(existingSecretName.isEmpty || isKerberosEnabled, + "You must enable Kerberos support if you are specifying a Kerberos Secret") + + KubernetesUtils.requireNandDefined( + krb5File, + krb5CMap, + "Do not specify both a Krb5 local file and the ConfigMap as the creation " + "of an additional ConfigMap, when one is already specified, is extraneous") - KubernetesUtils.requireBothOrNeitherDefined( - keytab, - principal, - "If a Kerberos principal is specified you must also specify a Kerberos keytab", - "If a Kerberos keytab is specified you must also specify a Kerberos principal") - - KubernetesUtils.requireBothOrNeitherDefined( - existingSecretName, - existingSecretItemKey, - "If a secret data item-key where the data of the Kerberos Delegation Token is specified" + - " you must also specify the name of the secret", - "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 => - HadoopBootstrapUtil.getHadoopConfFiles(hConfDir) - } - private val newHadoopConfigMapName = - if (hadoopConfDirSpec.hadoopConfigMapName.isEmpty) { - Some(kubernetesConf.hadoopConfigMapName) - } else { - None - } - - // Either use pre-existing secret or login to create new Secret with DT stored within - private val kerberosConfSpec: Option[KerberosConfigSpec] = (for { - secretName <- existingSecretName - secretItemKey <- existingSecretItemKey - } yield { - KerberosConfigSpec( - dtSecret = None, - dtSecretName = secretName, - dtSecretItemKey = secretItemKey, - jobUserName = kubeTokenManager.getCurrentUser.getShortUserName) - }).orElse( - if (isKerberosEnabled) { - Some(HadoopKerberosLogin.buildSpec( - conf, - kubernetesConf.appResourceNamePrefix, - kubeTokenManager)) - } else { - None - } - ) - - override def configurePod(pod: SparkPod): SparkPod = { - val hadoopBasedSparkPod = HadoopBootstrapUtil.bootstrapHadoopConfDir( - hadoopConfDirSpec.hadoopConfDir, - newHadoopConfigMapName, - hadoopConfDirSpec.hadoopConfigMapName, - pod) - kerberosConfSpec.map { hSpec => - HadoopBootstrapUtil.bootstrapKerberosPod( - hSpec.dtSecretName, - hSpec.dtSecretItemKey, - hSpec.jobUserName, - krb5File, - Some(kubernetesConf.kRBConfigMapName), - krb5CMap, - hadoopBasedSparkPod) - }.getOrElse( - HadoopBootstrapUtil.bootstrapSparkUserPod( - kubeTokenManager.getCurrentUser.getShortUserName, - hadoopBasedSparkPod)) + KubernetesUtils.requireBothOrNeitherDefined( + keytab, + principal, + "If a Kerberos principal is specified you must also specify a Kerberos keytab", + "If a Kerberos keytab is specified you must also specify a Kerberos principal") + + KubernetesUtils.requireBothOrNeitherDefined( + existingSecretName, + existingSecretItemKey, + "If a secret data item-key where the data of the Kerberos Delegation Token is specified" + + " you must also specify the name of the secret", + "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 => + HadoopBootstrapUtil.getHadoopConfFiles(hConfDir) + } + private val newHadoopConfigMapName = + if (hadoopConfDirSpec.hadoopConfigMapName.isEmpty) { + Some(kubernetesConf.hadoopConfigMapName) + } else { + None } - override def getAdditionalPodSystemProperties(): Map[String, String] = { - val resolvedConfValues = kerberosConfSpec.map { hSpec => - Map(KERBEROS_DT_SECRET_NAME -> hSpec.dtSecretName, - KERBEROS_DT_SECRET_KEY -> hSpec.dtSecretItemKey, - KERBEROS_SPARK_USER_NAME -> hSpec.jobUserName, - KRB5_CONFIG_MAP_NAME -> krb5CMap.getOrElse(kubernetesConf.kRBConfigMapName)) - }.getOrElse( - Map(KERBEROS_SPARK_USER_NAME -> - kubeTokenManager.getCurrentUser.getShortUserName)) - Map(HADOOP_CONFIG_MAP_NAME -> - hadoopConfDirSpec.hadoopConfigMapName.getOrElse( - kubernetesConf.hadoopConfigMapName)) ++ resolvedConfValues + // Either use pre-existing secret or login to create new Secret with DT stored within + private val kerberosConfSpec: Option[KerberosConfigSpec] = (for { + secretName <- existingSecretName + secretItemKey <- existingSecretItemKey + } yield { + KerberosConfigSpec( + dtSecret = None, + dtSecretName = secretName, + dtSecretItemKey = secretItemKey, + jobUserName = kubeTokenManager.getCurrentUser.getShortUserName) + }).orElse( + if (isKerberosEnabled) { + Some(HadoopKerberosLogin.buildSpec( + conf, + kubernetesConf.appResourceNamePrefix, + kubeTokenManager)) + } else { + None + } + ) + + override def configurePod(pod: SparkPod): SparkPod = { + val hadoopBasedSparkPod = HadoopBootstrapUtil.bootstrapHadoopConfDir( + hadoopConfDirSpec.hadoopConfDir, + newHadoopConfigMapName, + hadoopConfDirSpec.hadoopConfigMapName, + pod) + kerberosConfSpec.map { hSpec => + HadoopBootstrapUtil.bootstrapKerberosPod( + hSpec.dtSecretName, + hSpec.dtSecretItemKey, + hSpec.jobUserName, + krb5File, + Some(kubernetesConf.krbConfigMapName), + krb5CMap, + hadoopBasedSparkPod) + }.getOrElse( + HadoopBootstrapUtil.bootstrapSparkUserPod( + kubeTokenManager.getCurrentUser.getShortUserName, + hadoopBasedSparkPod)) + } + + override def getAdditionalPodSystemProperties(): Map[String, String] = { + val resolvedConfValues = kerberosConfSpec.map { hSpec => + Map(KERBEROS_DT_SECRET_NAME -> hSpec.dtSecretName, + KERBEROS_DT_SECRET_KEY -> hSpec.dtSecretItemKey, + KERBEROS_SPARK_USER_NAME -> hSpec.jobUserName, + KRB5_CONFIG_MAP_NAME -> krb5CMap.getOrElse(kubernetesConf.krbConfigMapName)) + }.getOrElse( + Map(KERBEROS_SPARK_USER_NAME -> + kubeTokenManager.getCurrentUser.getShortUserName)) + Map(HADOOP_CONFIG_MAP_NAME -> + hadoopConfDirSpec.hadoopConfigMapName.getOrElse( + kubernetesConf.hadoopConfigMapName)) ++ resolvedConfValues + } + + override def getAdditionalKubernetesResources(): Seq[HasMetadata] = { + // HADOOP_CONF_DIR ConfigMap + val hadoopConfConfigMap = for { + hName <- newHadoopConfigMapName + hFiles <- hadoopConfigurationFiles + } yield { + HadoopBootstrapUtil.buildHadoopConfigMap(hName, hFiles) } - override def getAdditionalKubernetesResources(): Seq[HasMetadata] = { - // HADOOP_CONF_DIR ConfigMap - val hadoopConfConfigMap = for { - hName <- newHadoopConfigMapName - hFiles <- hadoopConfigurationFiles - } yield { - HadoopBootstrapUtil.buildHadoopConfigMap( - hName, - hFiles) - } - - // krb5 ConfigMap - val krb5ConfigMap = krb5File.map { fileLocation => - HadoopBootstrapUtil.buildkrb5ConfigMap( - kubernetesConf.kRBConfigMapName, - fileLocation) - } - - // Kerberos DT Secret - val kerberosDTSecret = for { - hSpec <- kerberosConfSpec - kDtSecret <- hSpec.dtSecret - } yield { - kDtSecret - } - - hadoopConfConfigMap.toSeq ++ - krb5ConfigMap.toSeq ++ - kerberosDTSecret.toSeq + // krb5 ConfigMap + val krb5ConfigMap = krb5File.map { fileLocation => + HadoopBootstrapUtil.buildkrb5ConfigMap( + kubernetesConf.krbConfigMapName, + fileLocation) } + + // Kerberos DT Secret + val kerberosDTSecret = kerberosConfSpec.flatMap(_.dtSecret) + + hadoopConfConfigMap.toSeq ++ + krb5ConfigMap.toSeq ++ + kerberosDTSecret.toSeq + } } 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 15ec619ecd5d..bff922296dcc 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 @@ -28,12 +28,12 @@ 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]) - extends KubernetesFeatureConfigStep with Logging{ + kubernetesConf: KubernetesConf[KubernetesExecutorSpecificConf]) + extends KubernetesFeatureConfigStep with Logging { - private val sparkConf = kubernetesConf.sparkConf - private val maybeKrb5CMap = sparkConf.getOption(KRB5_CONFIG_MAP_NAME) - require(maybeKrb5CMap.isDefined, "HADOOP_CONF_DIR ConfigMap not found") + private val sparkConf = kubernetesConf.sparkConf + private val maybeKrb5CMap = sparkConf.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") diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadooputils/HadoopBootstrapUtil.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadooputils/HadoopBootstrapUtil.scala index aba63af4382b..06f503f9cd05 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadooputils/HadoopBootstrapUtil.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadooputils/HadoopBootstrapUtil.scala @@ -24,6 +24,7 @@ import scala.collection.JavaConverters._ import com.google.common.io.Files import io.fabric8.kubernetes.api.model._ +import org.apache.spark.SparkException import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.deploy.k8s.SparkPod @@ -42,13 +43,13 @@ private[spark] object HadoopBootstrapUtil { * @return a modified SparkPod */ def bootstrapKerberosPod( - dtSecretName: String, - dtSecretItemKey: String, - userName: String, - fileLocation: Option[String], - newKrb5ConfName: Option[String], - existingKrb5ConfName: Option[String], - pod: SparkPod) : SparkPod = { + dtSecretName: String, + dtSecretItemKey: String, + userName: String, + fileLocation: Option[String], + newKrb5ConfName: Option[String], + existingKrb5ConfName: Option[String], + pod: SparkPod): SparkPod = { val preConfigMapVolume = existingKrb5ConfName.map { kconf => new VolumeBuilder() @@ -56,12 +57,13 @@ private[spark] object HadoopBootstrapUtil { .withNewConfigMap() .withName(kconf) .endConfigMap() - .build() } + .build() + } - val ceateConfigMapVolume = for { + val createConfigMapVolume = for { fLocation <- fileLocation krb5ConfName <- newKrb5ConfName - } yield { + } yield { val krb5File = new File(fLocation) val fileStringPath = krb5File.toPath.getFileName.toString new VolumeBuilder() @@ -78,7 +80,10 @@ private[spark] object HadoopBootstrapUtil { // Breaking up Volume Creation for clarity val configMapVolume = preConfigMapVolume.getOrElse( - ceateConfigMapVolume.get) + createConfigMapVolume.getOrElse( + throw new SparkException("Must specify krb5.conf file locally or via ConfigMap") + ) + ) val kerberizedPod = new PodBuilder(pod.pod) .editOrNewSpec() @@ -122,7 +127,7 @@ private[spark] object HadoopBootstrapUtil { * @param pod Input pod to be appended to * @return a modified SparkPod */ - def bootstrapSparkUserPod(sparkUserName: String, pod: SparkPod) : SparkPod = { + def bootstrapSparkUserPod(sparkUserName: String, pod: SparkPod): SparkPod = { val envModifiedContainer = new ContainerBuilder(pod.container) .addNewEnv() .withName(ENV_SPARK_USER) @@ -138,14 +143,14 @@ private[spark] object HadoopBootstrapUtil { * @param path location of HADOOP_CONF_DIR * @return a list of File object */ - def getHadoopConfFiles(path: String) : Seq[File] = { + def getHadoopConfFiles(path: String): Seq[File] = { val dir = new File(path) if (dir.isDirectory) { dir.listFiles.filter(_.isFile).toSeq } else { Seq.empty[File] } - } + } /** * Bootstraping the container with ConfigMaps that store @@ -158,10 +163,10 @@ private[spark] object HadoopBootstrapUtil { * @return a modified SparkPod */ def bootstrapHadoopConfDir( - hadoopConfDir: Option[String], - newHadoopConfigMapName: Option[String], - existingHadoopConfigMapName: Option[String], - pod: SparkPod) : SparkPod = { + hadoopConfDir: Option[String], + newHadoopConfigMapName: Option[String], + existingHadoopConfigMapName: Option[String], + pod: SparkPod): SparkPod = { val preConfigMapVolume = existingHadoopConfigMapName.map { hConf => new VolumeBuilder() .withName(HADOOP_FILE_VOLUME) @@ -173,25 +178,26 @@ private[spark] object HadoopBootstrapUtil { val createConfigMapVolume = for { dirLocation <- hadoopConfDir hConfName <- newHadoopConfigMapName - } yield { + } yield { val hadoopConfigFiles = getHadoopConfFiles(dirLocation) val keyPaths = hadoopConfigFiles.map { file => val fileStringPath = file.toPath.getFileName.toString new KeyToPathBuilder() .withKey(fileStringPath) .withPath(fileStringPath) - .build() } + .build() + } new VolumeBuilder() .withName(HADOOP_FILE_VOLUME) .withNewConfigMap() .withName(hConfName) .withItems(keyPaths.asJava) .endConfigMap() - .build() } + .build() + } // Breaking up Volume Creation for clarity - val configMapVolume = preConfigMapVolume.getOrElse( - createConfigMapVolume.get) + val configMapVolume = preConfigMapVolume.getOrElse(createConfigMapVolume.get) val hadoopSupportedPod = new PodBuilder(pod.pod) .editSpec() @@ -200,7 +206,6 @@ private[spark] object HadoopBootstrapUtil { .endSpec() .build() - val hadoopSupportedContainer = new ContainerBuilder(pod.container) .addNewVolumeMount() .withName(HADOOP_FILE_VOLUME) @@ -223,8 +228,8 @@ private[spark] object HadoopBootstrapUtil { * @return a ConfigMap */ def buildkrb5ConfigMap( - configMapName: String, - fileLocation: String) : ConfigMap = { + configMapName: String, + fileLocation: String): ConfigMap = { val file = new File(fileLocation) new ConfigMapBuilder() .withNewMetadata() @@ -245,15 +250,16 @@ private[spark] object HadoopBootstrapUtil { * @return a ConfigMap */ def buildHadoopConfigMap( - hadoopConfigMapName: String, - hadoopConfFiles: Seq[File]) : ConfigMap = { + hadoopConfigMapName: String, + hadoopConfFiles: Seq[File]): ConfigMap = { new ConfigMapBuilder() .withNewMetadata() .withName(hadoopConfigMapName) .endMetadata() - .addToData(hadoopConfFiles.map(file => + .addToData(hadoopConfFiles.map { file => (file.toPath.getFileName.toString, - Files.toString(file, StandardCharsets.UTF_8))).toMap.asJava) + Files.toString(file, StandardCharsets.UTF_8)) + }.toMap.asJava) .build() } 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 index b8bb344d596e..525e685fbe70 100644 --- 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 @@ -34,34 +34,33 @@ import org.apache.spark.deploy.k8s.security.KubernetesHadoopDelegationTokenManag * a secret. All this is defined in a KerberosConfigSpec. */ private[spark] object HadoopKerberosLogin { - def buildSpec( - submissionSparkConf: SparkConf, - kubernetesResourceNamePrefix : String, - tokenManager: KubernetesHadoopDelegationTokenManager): KerberosConfigSpec = { - val hadoopConf = SparkHadoopUtil.get.newConfiguration(submissionSparkConf) - // 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 - val (tokenData, renewalInterval) = tokenManager.getDelegationTokens( - originalCredentials, - submissionSparkConf, - hadoopConf) - require(tokenData.nonEmpty, "Did not obtain any delegation tokens") - val initialTokenDataKeyName = KERBEROS_SECRET_KEY_PREFIX - 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) - } + def buildSpec( + submissionSparkConf: SparkConf, + kubernetesResourceNamePrefix : String, + tokenManager: KubernetesHadoopDelegationTokenManager): KerberosConfigSpec = { + val hadoopConf = SparkHadoopUtil.get.newConfiguration(submissionSparkConf) + // 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 + val (tokenData, renewalInterval) = tokenManager.getDelegationTokens( + originalCredentials, + submissionSparkConf, + hadoopConf) + require(tokenData.nonEmpty, "Did not obtain any delegation tokens") + 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/features/hadooputils/KerberosConfigSpec.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadooputils/KerberosConfigSpec.scala index 4ae19cbde9ac..847dbc6f7353 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadooputils/KerberosConfigSpec.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadooputils/KerberosConfigSpec.scala @@ -27,7 +27,7 @@ import io.fabric8.kubernetes.api.model.Secret * - The Job User's username */ private[spark] case class KerberosConfigSpec( - dtSecret: Option[Secret], - dtSecretName: String, - dtSecretItemKey: String, - jobUserName: String) + dtSecret: Option[Secret], + dtSecretName: String, + dtSecretItemKey: String, + jobUserName: String) 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 index 713bb6895567..8d5fcebfb259 100644 --- 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 @@ -33,22 +33,22 @@ import org.apache.spark.internal.Logging * to the driver and executors, when new Tokens are received they overwrite the current Secrets. */ private[spark] class KubernetesHadoopDelegationTokenManager( - tokenManager: HadoopDelegationTokenManager) extends Logging { + tokenManager: HadoopDelegationTokenManager) extends Logging { - // HadoopUGI Util methods - def getCurrentUser: UserGroupInformation = UserGroupInformation.getCurrentUser - def getShortUserName : String = getCurrentUser.getShortUserName - def getFileSystem(hadoopConf: Configuration) : FileSystem = FileSystem.get(hadoopConf) - def isSecurityEnabled: Boolean = UserGroupInformation.isSecurityEnabled - def loginUserFromKeytabAndReturnUGI(principal: String, keytab: String): UserGroupInformation = - UserGroupInformation.loginUserFromKeytabAndReturnUGI(principal, keytab) - def serializeCreds(creds: Credentials): Array[Byte] = SparkHadoopUtil.get.serialize(creds) - def nextRT(rt: Long, conf: SparkConf): Long = SparkHadoopUtil.nextCredentialRenewalTime(rt, conf) + // HadoopUGI Util methods + def getCurrentUser: UserGroupInformation = UserGroupInformation.getCurrentUser + def getShortUserName : String = getCurrentUser.getShortUserName + def getFileSystem(hadoopConf: Configuration) : FileSystem = FileSystem.get(hadoopConf) + def isSecurityEnabled: Boolean = UserGroupInformation.isSecurityEnabled + def loginUserFromKeytabAndReturnUGI(principal: String, keytab: String): UserGroupInformation = + UserGroupInformation.loginUserFromKeytabAndReturnUGI(principal, keytab) + def serializeCreds(creds: Credentials): Array[Byte] = SparkHadoopUtil.get.serialize(creds) + def nextRT(rt: Long, conf: SparkConf): Long = SparkHadoopUtil.nextCredentialRenewalTime(rt, conf) - def getDelegationTokens( - creds: Credentials, - conf: SparkConf, - hadoopConf: Configuration): (Array[Byte], Long) = { + def getDelegationTokens( + creds: Credentials, + conf: SparkConf, + hadoopConf: Configuration): (Array[Byte], Long) = { try { val rt = tokenManager.obtainDelegationTokens(hadoopConf, creds) logDebug(s"Initialized tokens") From a9589203ef9262414c474febadaf9e809d28ac8c Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Tue, 9 Oct 2018 19:10:01 -0700 Subject: [PATCH 24/25] included new ability to bake krb5.conf into your docker images and not fail --- .../hadooputils/HadoopBootstrapUtil.scala | 67 ++++++++++++------- 1 file changed, 42 insertions(+), 25 deletions(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadooputils/HadoopBootstrapUtil.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadooputils/HadoopBootstrapUtil.scala index 06f503f9cd05..2303ff06fee2 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadooputils/HadoopBootstrapUtil.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadooputils/HadoopBootstrapUtil.scala @@ -24,11 +24,11 @@ import scala.collection.JavaConverters._ import com.google.common.io.Files import io.fabric8.kubernetes.api.model._ -import org.apache.spark.SparkException import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.deploy.k8s.SparkPod +import org.apache.spark.internal.Logging -private[spark] object HadoopBootstrapUtil { +private[spark] object HadoopBootstrapUtil extends Logging { /** * Mounting the DT secret for both the Driver and the executors @@ -78,36 +78,39 @@ private[spark] object HadoopBootstrapUtil { .build() } - // Breaking up Volume Creation for clarity - val configMapVolume = preConfigMapVolume.getOrElse( - createConfigMapVolume.getOrElse( - throw new SparkException("Must specify krb5.conf file locally or via ConfigMap") - ) - ) + // Breaking up Volume creation for clarity + val configMapVolume = preConfigMapVolume.orElse(createConfigMapVolume) + if (configMapVolume.isEmpty) { + logInfo("You have not specified a krb5.conf file locally or via a ConfigMap. " + + "Make sure that you have the krb5.conf locally on the Driver and Executor images") + } - val kerberizedPod = new PodBuilder(pod.pod) - .editOrNewSpec() + val kerberizedPodWithDTSecret = new PodBuilder(pod.pod) + .editOrNewSpec() .addNewVolume() .withName(SPARK_APP_HADOOP_SECRET_VOLUME_NAME) .withNewSecret() .withSecretName(dtSecretName) .endSecret() .endVolume() - .addNewVolumeLike(configMapVolume) - .endVolume() .endSpec() + .build() + + // Optionally add the krb5.conf ConfigMap + val kerberizedPod = configMapVolume.map { cmVolume => + new PodBuilder(kerberizedPodWithDTSecret) + .editSpec() + .addNewVolumeLike(cmVolume) + .endVolume() + .endSpec() .build() + }.getOrElse(kerberizedPodWithDTSecret) - val kerberizedContainer = new ContainerBuilder(pod.container) + val kerberizedContainerWithMounts = new ContainerBuilder(pod.container) .addNewVolumeMount() .withName(SPARK_APP_HADOOP_SECRET_VOLUME_NAME) .withMountPath(SPARK_APP_HADOOP_CREDENTIALS_BASE_DIR) .endVolumeMount() - .addNewVolumeMount() - .withName(KRB_FILE_VOLUME) - .withMountPath(KRB_FILE_DIR_PATH + "/krb5.conf") - .withSubPath("krb5.conf") - .endVolumeMount() .addNewEnv() .withName(ENV_HADOOP_TOKEN_FILE_LOCATION) .withValue(s"$SPARK_APP_HADOOP_CREDENTIALS_BASE_DIR/$dtSecretItemKey") @@ -117,7 +120,22 @@ private[spark] object HadoopBootstrapUtil { .withValue(userName) .endEnv() .build() - SparkPod(kerberizedPod, kerberizedContainer) + + // Optionally add the krb5.conf Volume Mount + val kerberizedContainer = + if (configMapVolume.isDefined) { + new ContainerBuilder(kerberizedContainerWithMounts) + .addNewVolumeMount() + .withName(KRB_FILE_VOLUME) + .withMountPath(KRB_FILE_DIR_PATH + "/krb5.conf") + .withSubPath("krb5.conf") + .endVolumeMount() + .build() + } else { + kerberizedContainerWithMounts + } + + SparkPod(kerberizedPod, kerberizedContainer) } /** @@ -130,9 +148,9 @@ private[spark] object HadoopBootstrapUtil { def bootstrapSparkUserPod(sparkUserName: String, pod: SparkPod): SparkPod = { val envModifiedContainer = new ContainerBuilder(pod.container) .addNewEnv() - .withName(ENV_SPARK_USER) - .withValue(sparkUserName) - .endEnv() + .withName(ENV_SPARK_USER) + .withValue(sparkUserName) + .endEnv() .build() SparkPod(pod.pod, envModifiedContainer) } @@ -235,9 +253,8 @@ private[spark] object HadoopBootstrapUtil { .withNewMetadata() .withName(configMapName) .endMetadata() - .addToData( - Map(file.toPath.getFileName.toString -> - Files.toString(file, StandardCharsets.UTF_8)).asJava) + .addToData(Map(file.toPath.getFileName.toString -> + Files.toString(file, StandardCharsets.UTF_8)).asJava) .build() } From dd95fcab754e71e9465f4e46818c3cef09e86c8b Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Wed, 10 Oct 2018 12:05:46 -0700 Subject: [PATCH 25/25] style check --- docs/security.md | 3 +- .../HadoopConfExecutorFeatureStep.scala | 12 +- .../HadoopSparkUserExecutorFeatureStep.scala | 12 +- .../KerberosConfDriverFeatureStep.scala | 13 +- .../KerberosConfExecutorFeatureStep.scala | 8 +- .../hadooputils/HadoopBootstrapUtil.scala | 114 +++++++++--------- .../hadooputils/HadoopKerberosLogin.scala | 20 +-- .../hadooputils/KerberosConfigSpec.scala | 16 +-- ...bernetesHadoopDelegationTokenManager.scala | 16 +-- 9 files changed, 105 insertions(+), 109 deletions(-) diff --git a/docs/security.md b/docs/security.md index 55d987825173..ffae683df625 100644 --- a/docs/security.md +++ b/docs/security.md @@ -729,8 +729,7 @@ so that non-local processes can authenticate. These delegation tokens in Kuberne shared by the Driver and its Executors. As such, there are three ways of submitting a Kerberos job: In all cases you must define the environment variable: `HADOOP_CONF_DIR` or -`spark.kubernetes.hadoop.configMapName` as well as either -`spark.kubernetes.kerberos.krb5.path` or `spark.kubernetes.kerberos.krb5.configMapName`. +`spark.kubernetes.hadoop.configMapName.` It also important to note that the KDC needs to be visible from inside the containers. 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 09198e90502b..fd09de2a918a 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 @@ -23,14 +23,14 @@ import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.deploy.k8s.features.hadooputils.HadoopBootstrapUtil import org.apache.spark.internal.Logging - /** - * This step is responsible for bootstraping the container with ConfigMaps - * containing Hadoop config files mounted as volumes and an ENV variable - * pointed to the mounted file directory. - */ +/** + * This step is responsible for bootstraping the container with ConfigMaps + * containing Hadoop config files mounted as volumes and an ENV variable + * pointed to the mounted file directory. + */ private[spark] class HadoopConfExecutorFeatureStep( kubernetesConf: KubernetesConf[KubernetesExecutorSpecificConf]) - extends KubernetesFeatureConfigStep with Logging { + extends KubernetesFeatureConfigStep with Logging { override def configurePod(pod: SparkPod): SparkPod = { val sparkConf = kubernetesConf.sparkConf 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 65dac0099109..5b6a6d5a7db4 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 @@ -24,17 +24,17 @@ 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 - */ +/** + * 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 { + extends KubernetesFeatureConfigStep with Logging { override def configurePod(pod: SparkPod): SparkPod = { val sparkUserName = kubernetesConf.sparkConf.get(KERBEROS_SPARK_USER_NAME) - HadoopBootstrapUtil.bootstrapSparkUserPod(sparkUserName, pod) + HadoopBootstrapUtil.bootstrapSparkUserPod(sparkUserName, pod) } override def getAdditionalPodSystemProperties(): Map[String, String] = Map.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 172310c0b0a6..ce47933b7f70 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 @@ -26,13 +26,13 @@ import org.apache.spark.deploy.k8s.KubernetesDriverSpecificConf import org.apache.spark.deploy.k8s.features.hadooputils._ import org.apache.spark.internal.Logging - /** - * 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. - */ +/** + * 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 { + extends KubernetesFeatureConfigStep with Logging { require(kubernetesConf.hadoopConfSpec.isDefined, "Ensure that HADOOP_CONF_DIR is defined either via env or a pre-defined ConfigMap") @@ -143,7 +143,6 @@ private[spark] class KerberosConfDriverFeatureStep( } override def getAdditionalKubernetesResources(): Seq[HasMetadata] = { - // HADOOP_CONF_DIR ConfigMap val hadoopConfConfigMap = for { hName <- newHadoopConfigMapName hFiles <- hadoopConfigurationFiles @@ -151,14 +150,12 @@ private[spark] class KerberosConfDriverFeatureStep( HadoopBootstrapUtil.buildHadoopConfigMap(hName, hFiles) } - // krb5 ConfigMap val krb5ConfigMap = krb5File.map { fileLocation => HadoopBootstrapUtil.buildkrb5ConfigMap( kubernetesConf.krbConfigMapName, fileLocation) } - // Kerberos DT Secret val kerberosDTSecret = kerberosConfSpec.flatMap(_.dtSecret) hadoopConfConfigMap.toSeq ++ 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 bff922296dcc..06a88b6c229f 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 @@ -24,12 +24,12 @@ 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 - */ +/** + * This step is responsible for mounting the DT secret for the executors + */ private[spark] class KerberosConfExecutorFeatureStep( kubernetesConf: KubernetesConf[KubernetesExecutorSpecificConf]) - extends KubernetesFeatureConfigStep with Logging { + extends KubernetesFeatureConfigStep with Logging { private val sparkConf = kubernetesConf.sparkConf private val maybeKrb5CMap = sparkConf.getOption(KRB5_CONFIG_MAP_NAME) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadooputils/HadoopBootstrapUtil.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadooputils/HadoopBootstrapUtil.scala index 2303ff06fee2..5bee766caf2b 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadooputils/HadoopBootstrapUtil.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadooputils/HadoopBootstrapUtil.scala @@ -30,18 +30,18 @@ import org.apache.spark.internal.Logging private[spark] object HadoopBootstrapUtil extends Logging { - /** - * Mounting the DT secret for both the Driver and the executors - * - * @param dtSecretName Name of the secret that stores the Delegation Token - * @param dtSecretItemKey Name of the Item Key storing the Delegation Token - * @param userName Name of the SparkUser to set SPARK_USER - * @param fileLocation Optional Location of the krb5 file - * @param newKrb5ConfName Optional location of the ConfigMap for Krb5 - * @param existingKrb5ConfName Optional name of ConfigMap for Krb5 - * @param pod Input pod to be appended to - * @return a modified SparkPod - */ + /** + * Mounting the DT secret for both the Driver and the executors + * + * @param dtSecretName Name of the secret that stores the Delegation Token + * @param dtSecretItemKey Name of the Item Key storing the Delegation Token + * @param userName Name of the SparkUser to set SPARK_USER + * @param fileLocation Optional Location of the krb5 file + * @param newKrb5ConfName Optional location of the ConfigMap for Krb5 + * @param existingKrb5ConfName Optional name of ConfigMap for Krb5 + * @param pod Input pod to be appended to + * @return a modified SparkPod + */ def bootstrapKerberosPod( dtSecretName: String, dtSecretItemKey: String, @@ -138,29 +138,29 @@ private[spark] object HadoopBootstrapUtil extends Logging { SparkPod(kerberizedPod, kerberizedContainer) } - /** - * setting ENV_SPARK_USER when HADOOP_FILES are detected - * - * @param sparkUserName Name of the SPARK_USER - * @param pod Input pod to be appended to - * @return a modified SparkPod - */ + /** + * setting ENV_SPARK_USER when HADOOP_FILES are detected + * + * @param sparkUserName Name of the SPARK_USER + * @param pod Input pod to be appended to + * @return a modified SparkPod + */ def bootstrapSparkUserPod(sparkUserName: String, pod: SparkPod): SparkPod = { - val envModifiedContainer = new ContainerBuilder(pod.container) - .addNewEnv() - .withName(ENV_SPARK_USER) - .withValue(sparkUserName) - .endEnv() + val envModifiedContainer = new ContainerBuilder(pod.container) + .addNewEnv() + .withName(ENV_SPARK_USER) + .withValue(sparkUserName) + .endEnv() .build() - SparkPod(pod.pod, envModifiedContainer) + SparkPod(pod.pod, envModifiedContainer) } - /** - * Grabbing files in the HADOOP_CONF_DIR - * - * @param path location of HADOOP_CONF_DIR - * @return a list of File object - */ + /** + * Grabbing files in the HADOOP_CONF_DIR + * + * @param path location of HADOOP_CONF_DIR + * @return a list of File object + */ def getHadoopConfFiles(path: String): Seq[File] = { val dir = new File(path) if (dir.isDirectory) { @@ -170,16 +170,16 @@ private[spark] object HadoopBootstrapUtil extends Logging { } } - /** - * Bootstraping the container with ConfigMaps that store - * Hadoop configuration files - * - * @param hadoopConfDir directory location of HADOOP_CONF_DIR env - * @param newHadoopConfigMapName name of the new configMap for HADOOP_CONF_DIR - * @param existingHadoopConfigMapName name of the pre-defined configMap for HADOOP_CONF_DIR - * @param pod Input pod to be appended to - * @return a modified SparkPod - */ + /** + * Bootstraping the container with ConfigMaps that store + * Hadoop configuration files + * + * @param hadoopConfDir directory location of HADOOP_CONF_DIR env + * @param newHadoopConfigMapName name of the new configMap for HADOOP_CONF_DIR + * @param existingHadoopConfigMapName name of the pre-defined configMap for HADOOP_CONF_DIR + * @param pod Input pod to be appended to + * @return a modified SparkPod + */ def bootstrapHadoopConfDir( hadoopConfDir: Option[String], newHadoopConfigMapName: Option[String], @@ -237,14 +237,14 @@ private[spark] object HadoopBootstrapUtil extends Logging { SparkPod(hadoopSupportedPod, hadoopSupportedContainer) } - /** - * Builds ConfigMap given the file location of the - * krb5.conf file - * - * @param configMapName name of configMap for krb5 - * @param fileLocation location of krb5 file - * @return a ConfigMap - */ + /** + * Builds ConfigMap given the file location of the + * krb5.conf file + * + * @param configMapName name of configMap for krb5 + * @param fileLocation location of krb5 file + * @return a ConfigMap + */ def buildkrb5ConfigMap( configMapName: String, fileLocation: String): ConfigMap = { @@ -258,14 +258,14 @@ private[spark] object HadoopBootstrapUtil extends Logging { .build() } - /** - * Builds ConfigMap given the ConfigMap name - * and a list of Hadoop Conf files - * - * @param hadoopConfigMapName name of hadoopConfigMap - * @param hadoopConfFiles list of hadoopFiles - * @return a ConfigMap - */ + /** + * Builds ConfigMap given the ConfigMap name + * and a list of Hadoop Conf files + * + * @param hadoopConfigMapName name of hadoopConfigMap + * @param hadoopConfFiles list of hadoopFiles + * @return a ConfigMap + */ def buildHadoopConfigMap( hadoopConfigMapName: String, hadoopConfFiles: Seq[File]): ConfigMap = { 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 index 525e685fbe70..67a58491e442 100644 --- 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 @@ -24,19 +24,19 @@ 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. - */ +/** + * 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, + kubernetesResourceNamePrefix: String, tokenManager: KubernetesHadoopDelegationTokenManager): KerberosConfigSpec = { val hadoopConf = SparkHadoopUtil.get.newConfiguration(submissionSparkConf) // The JobUserUGI will be taken fom the Local Ticket Cache or via keytab+principal diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadooputils/KerberosConfigSpec.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadooputils/KerberosConfigSpec.scala index 847dbc6f7353..7f7ef216cf48 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadooputils/KerberosConfigSpec.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadooputils/KerberosConfigSpec.scala @@ -18,14 +18,14 @@ package org.apache.spark.deploy.k8s.features.hadooputils import io.fabric8.kubernetes.api.model.Secret - /** - * Represents a given configuration of the Kerberos Configuration logic - *

- * - The secret containing a DT, either previously specified or built on the fly - * - The name of the secret where the DT will be stored - * - The data item-key on the secret which correlates with where the current DT data is stored - * - The Job User's username - */ +/** + * Represents a given configuration of the Kerberos Configuration logic + *

+ * - The secret containing a DT, either previously specified or built on the fly + * - The name of the secret where the DT will be stored + * - The data item-key on the secret which correlates with where the current DT data is stored + * - The Job User's username + */ private[spark] case class KerberosConfigSpec( dtSecret: Option[Secret], dtSecretName: String, 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 index 8d5fcebfb259..135e2c482bbb 100644 --- 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 @@ -26,19 +26,19 @@ import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.deploy.security.HadoopDelegationTokenManager import org.apache.spark.internal.Logging - /** - * The KubernetesHadoopDelegationTokenManager fetches Hadoop delegation tokens - * on the behalf of the Kubernetes submission client. The new credentials - * (called Tokens when they are serialized) are stored in Secrets accessible - * to the driver and executors, when new Tokens are received they overwrite the current Secrets. - */ +/** + * The KubernetesHadoopDelegationTokenManager fetches Hadoop delegation tokens + * on the behalf of the Kubernetes submission client. The new credentials + * (called Tokens when they are serialized) are stored in Secrets accessible + * to the driver and executors, when new Tokens are received they overwrite the current Secrets. + */ private[spark] class KubernetesHadoopDelegationTokenManager( tokenManager: HadoopDelegationTokenManager) extends Logging { // HadoopUGI Util methods def getCurrentUser: UserGroupInformation = UserGroupInformation.getCurrentUser - def getShortUserName : String = getCurrentUser.getShortUserName - def getFileSystem(hadoopConf: Configuration) : FileSystem = FileSystem.get(hadoopConf) + def getShortUserName: String = getCurrentUser.getShortUserName + def getFileSystem(hadoopConf: Configuration): FileSystem = FileSystem.get(hadoopConf) def isSecurityEnabled: Boolean = UserGroupInformation.isSecurityEnabled def loginUserFromKeytabAndReturnUGI(principal: String, keytab: String): UserGroupInformation = UserGroupInformation.loginUserFromKeytabAndReturnUGI(principal, keytab)