From 7612bf55c7ca2d515813d04f0818cfe597a7c629 Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Thu, 2 Nov 2017 20:25:49 -0400 Subject: [PATCH 01/14] first stage of PR #514 of just logic --- .../org/apache/spark/deploy/SparkSubmit.scala | 2 +- docs/running-on-kubernetes.md | 56 +++++++- .../deploy/k8s/HadoopConfBootstrap.scala | 86 ++++++++++++ .../spark/deploy/k8s/HadoopUGIUtil.scala | 79 +++++++++++ .../k8s/KerberosTokenConfBootstrap.scala | 76 +++++++++++ .../org/apache/spark/deploy/k8s/config.scala | 49 +++++++ .../apache/spark/deploy/k8s/constants.scala | 27 ++++ .../spark/deploy/k8s/submit/Client.scala | 16 ++- ...DriverConfigurationStepsOrchestrator.scala | 13 +- .../k8s/submit/PodWithMainContainer.scala | 28 ++++ .../HadoopConfigBootstrapStep.scala | 68 ++++++++++ .../hadoopsteps/HadoopConfMounterStep.scala | 56 ++++++++ .../hadoopsteps/HadoopConfigSpec.scala | 42 ++++++ .../hadoopsteps/HadoopConfigurationStep.scala | 25 ++++ .../HadoopKerberosKeytabResolverStep.scala | 125 ++++++++++++++++++ .../HadoopKerberosSecretResolverStep.scala | 56 ++++++++ .../hadoopsteps/HadoopStepsOrchestrator.scala | 104 +++++++++++++++ .../cluster/k8s/ExecutorPodFactory.scala | 50 +++++-- .../k8s/KubernetesClusterManager.scala | 45 ++++++- 19 files changed, 981 insertions(+), 22 deletions(-) create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/HadoopConfBootstrap.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/HadoopUGIUtil.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KerberosTokenConfBootstrap.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/PodWithMainContainer.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/HadoopConfigBootstrapStep.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopConfMounterStep.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopConfigSpec.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopConfigurationStep.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopKerberosKeytabResolverStep.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopKerberosSecretResolverStep.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopStepsOrchestrator.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 85b76013ba5f3..4aced4e23ba77 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -573,7 +573,7 @@ object SparkSubmit extends CommandLineUtils { } // assure a keytab is available from any place in a JVM - if (clusterManager == YARN || clusterManager == LOCAL) { + if (clusterManager == YARN || clusterManager == KUBERNETES || clusterManager == LOCAL) { if (args.principal != null) { require(args.keytab != null, "Keytab must be specified when principal is specified") if (!new File(args.keytab).exists()) { diff --git a/docs/running-on-kubernetes.md b/docs/running-on-kubernetes.md index 99d356044b146..5920988b9facc 100644 --- a/docs/running-on-kubernetes.md +++ b/docs/running-on-kubernetes.md @@ -752,6 +752,61 @@ from the other deployment modes. See the [configuration page](configuration.html + spark.kubernetes.kerberos.enabled + false + + Specify whether your job requires a Kerberos Authentication to access HDFS. By default, we + will assume that you will not require secure HDFS access. + + + + spark.kubernetes.kerberos.keytab + (none) + + Assuming you have set spark.kubernetes.kerberos.enabled to be true. This will let you specify + the location of your Kerberos keytab to be used in order to access Secure HDFS. This is optional as you + may login by running kinit before running the spark-submit, and the submission client + will look within your local TGT cache to resolve this. + + + + spark.kubernetes.kerberos.principal + (none) + + Assuming you have set spark.kubernetes.kerberos.enabled to be true. This will let you specify + your Kerberos principal that you wish to use to access Secure HDFS. This is optional as you + may login by running kinit before running the spark-submit, and the submission client + will look within your local TGT cache to resolve this. + + + + spark.kubernetes.kerberos.rewewer.principal + (none) + + Assuming you have set spark.kubernetes.kerberos.enabled to be true. This will let you specify + the principal that you wish to use to handle renewing of Delegation Tokens. This is optional as you + we will set the principal to be the job users principal by default. + + + + spark.kubernetes.kerberos.tokensecret.name + (none) + + Assuming you have set spark.kubernetes.kerberos.enabled to be true. This will let you specify + the name of the secret where your existing delegation token data is stored. You must also specify the + item key spark.kubernetes.kerberos.tokensecret.itemkey where your data is stored on the secret. + This is optional in the case that you want to use pre-existing secret, otherwise a new secret will be automatically + created. + + + + spark.kubernetes.kerberos.tokensecret.itemkey + spark.kubernetes.kerberos.dt.label + + Assuming you have set spark.kubernetes.kerberos.enabled to be true. This will let you specify + the data item key name within the pre-specified secret where the data of your existing delegation token data is stored. + We have a default value of spark.kubernetes.kerberos.tokensecret.itemkey should you not include it. But + you should always include this if you are proposing a pre-existing secret contain the delegation token data. spark.executorEnv.[EnvironmentVariableName] (none) @@ -791,4 +846,3 @@ from the other deployment modes. See the [configuration page](configuration.html Running Spark on Kubernetes is currently an experimental feature. Some restrictions on the current implementation that should be lifted in the future include: * Applications can only run in cluster mode. -* Only Scala and Java applications can be run. diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/HadoopConfBootstrap.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/HadoopConfBootstrap.scala new file mode 100644 index 0000000000000..228740f9fe2c0 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/HadoopConfBootstrap.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 + +import java.io.File + +import scala.collection.JavaConverters._ + +import io.fabric8.kubernetes.api.model.{ContainerBuilder, KeyToPathBuilder, PodBuilder} + +import org.apache.spark.deploy.k8s.constants._ +import org.apache.spark.internal.Logging + +/** + * This is separated out from the HadoopConf steps API because this component can be reused to + * set up the Hadoop Configuration for executors as well. + */ +private[spark] trait HadoopConfBootstrap { + /** + * Bootstraps a main container with the ConfigMaps containing Hadoop config files + * mounted as volumes and an ENV variable pointing to the mounted file. + */ + def bootstrapMainContainerAndVolumes( + originalPodWithMainContainer: PodWithMainContainer) + : PodWithMainContainer +} + +private[spark] class HadoopConfBootstrapImpl( + hadoopConfConfigMapName: String, + hadoopConfigFiles: Seq[File], + hadoopUGI: HadoopUGIUtil) extends HadoopConfBootstrap with Logging{ + + override def bootstrapMainContainerAndVolumes( + originalPodWithMainContainer: PodWithMainContainer) + : PodWithMainContainer = { + logInfo("HADOOP_CONF_DIR defined. Mounting HDFS specific .xml files") + val keyPaths = hadoopConfigFiles.map(file => + new KeyToPathBuilder() + .withKey(file.toPath.getFileName.toString) + .withPath(file.toPath.getFileName.toString) + .build()) + val hadoopSupportedPod = new PodBuilder(originalPodWithMainContainer.pod) + .editSpec() + .addNewVolume() + .withName(HADOOP_FILE_VOLUME) + .withNewConfigMap() + .withName(hadoopConfConfigMapName) + .withItems(keyPaths.asJava) + .endConfigMap() + .endVolume() + .endSpec() + .build() + val mainContainerWithMountedHadoopConf = new ContainerBuilder( + originalPodWithMainContainer.mainContainer) + .addNewVolumeMount() + .withName(HADOOP_FILE_VOLUME) + .withMountPath(HADOOP_CONF_DIR_PATH) + .endVolumeMount() + .addNewEnv() + .withName(ENV_HADOOP_CONF_DIR) + .withValue(HADOOP_CONF_DIR_PATH) + .endEnv() + .addNewEnv() + .withName(ENV_SPARK_USER) + .withValue(hadoopUGI.getShortName) + .endEnv() + .build() + originalPodWithMainContainer.copy( + pod = hadoopSupportedPod, + mainContainer = mainContainerWithMountedHadoopConf) + } +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/HadoopUGIUtil.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/HadoopUGIUtil.scala new file mode 100644 index 0000000000000..e9837f3258611 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/HadoopUGIUtil.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 + +import java.io.{ByteArrayInputStream, ByteArrayOutputStream, DataInputStream, DataOutputStream} + +import scala.util.Try + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.FileSystem +import org.apache.hadoop.security.{Credentials, UserGroupInformation} +import org.apache.hadoop.security.token.{Token, TokenIdentifier} +import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenIdentifier + + +// Function of this class is merely for mocking reasons +private[spark] class HadoopUGIUtil{ + def getCurrentUser: UserGroupInformation = UserGroupInformation.getCurrentUser + + def getShortName: String = getCurrentUser.getShortUserName + + def isSecurityEnabled: Boolean = UserGroupInformation.isSecurityEnabled + + def loginUserFromKeytabAndReturnUGI(principal: String, keytab: String): UserGroupInformation = + UserGroupInformation.loginUserFromKeytabAndReturnUGI(principal, keytab) + + def dfsAddDelegationToken(hadoopConf: Configuration, renewer: String, creds: Credentials) + : Iterable[Token[_ <: TokenIdentifier]] = + FileSystem.get(hadoopConf).addDelegationTokens(renewer, creds) + + def getCurrentTime: Long = System.currentTimeMillis() + + // Functions that should be in Core with Rebase to 2.3 + @deprecated("Moved to core in 2.2", "2.2") + def getTokenRenewalInterval( + renewedTokens: Iterable[Token[_ <: TokenIdentifier]], + hadoopConf: Configuration): Option[Long] = { + val renewIntervals = renewedTokens.filter { + _.decodeIdentifier().isInstanceOf[AbstractDelegationTokenIdentifier] + }.flatMap { token => + Try { + val newExpiration = token.renew(hadoopConf) + val identifier = token.decodeIdentifier().asInstanceOf[AbstractDelegationTokenIdentifier] + val interval = newExpiration - identifier.getIssueDate + interval + }.toOption + } + renewIntervals.reduceLeftOption(_ min _) + } + + @deprecated("Moved to core in 2.2", "2.2") + def serialize(creds: Credentials): Array[Byte] = { + val byteStream = new ByteArrayOutputStream + val dataStream = new DataOutputStream(byteStream) + creds.writeTokenStorageToStream(dataStream) + byteStream.toByteArray + } + + @deprecated("Moved to core in 2.2", "2.2") + def deserialize(tokenBytes: Array[Byte]): Credentials = { + val creds = new Credentials() + creds.readTokenStorageStream(new DataInputStream(new ByteArrayInputStream(tokenBytes))) + creds + } +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KerberosTokenConfBootstrap.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KerberosTokenConfBootstrap.scala new file mode 100644 index 0000000000000..dfe46e7106e87 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KerberosTokenConfBootstrap.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 + +import io.fabric8.kubernetes.api.model.{ContainerBuilder, PodBuilder} + +import org.apache.spark.deploy.k8s.constants._ +import org.apache.spark.internal.Logging + + + /** + * This is separated out from the HadoopConf steps API because this component can be reused to + * mounted the DT secret for executors as well. + */ +private[spark] trait KerberosTokenConfBootstrap { + // Bootstraps a main container with the Secret mounted as volumes and an ENV variable + // pointing to the mounted file containing the DT for Secure HDFS interaction + def bootstrapMainContainerAndVolumes( + originalPodWithMainContainer: PodWithMainContainer) + : PodWithMainContainer +} + +private[spark] class KerberosTokenConfBootstrapImpl( + secretName: String, + secretItemKey: String, + userName: String) extends KerberosTokenConfBootstrap with Logging{ + + + override def bootstrapMainContainerAndVolumes( + originalPodWithMainContainer: PodWithMainContainer) + : PodWithMainContainer = { + logInfo("Mounting HDFS DT from Secret for Secure HDFS") + val dtMountedPod = new PodBuilder(originalPodWithMainContainer.pod) + .editOrNewSpec() + .addNewVolume() + .withName(SPARK_APP_HADOOP_SECRET_VOLUME_NAME) + .withNewSecret() + .withSecretName(secretName) + .endSecret() + .endVolume() + .endSpec() + .build() + val mainContainerWithMountedKerberos = new ContainerBuilder( + originalPodWithMainContainer.mainContainer) + .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() + originalPodWithMainContainer.copy( + pod = dtMountedPod, + mainContainer = mainContainerWithMountedKerberos) + } +} \ No newline at end of file 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 0e35e04ff5803..739d205c0070e 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 @@ -157,6 +157,12 @@ package object config extends Logging { .stringConf .createOptional + private[spark] val KUBERNETES_SHUFFLE_DIR = + ConfigBuilder("spark.kubernetes.shuffle.dir") + .doc("Path to the shared shuffle directories.") + .stringConf + .createOptional + private[spark] val KUBERNETES_SHUFFLE_APISERVER_URI = ConfigBuilder("spark.kubernetes.shuffle.apiServer.url") .doc("URL to the Kubernetes API server that the shuffle service will monitor for Spark pods.") @@ -496,6 +502,49 @@ package object config extends Logging { private[spark] val KUBERNETES_NODE_SELECTOR_PREFIX = "spark.kubernetes.node.selector." + private[spark] 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) + + private[spark] 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 + + private[spark] val KUBERNETES_KERBEROS_PRINCIPAL = + ConfigBuilder("spark.kubernetes.kerberos.principal") + .doc("Specify the principal" + + " for Kerberos in order to access Secure HDFS") + .stringConf + .createOptional + + private[spark] val KUBERNETES_KERBEROS_RENEWER_PRINCIPAL = + ConfigBuilder("spark.kubernetes.kerberos.rewnewer.principal") + .doc("Specify the principal" + + " you wish to renew and retrieve your Kerberos values with") + .stringConf + .createOptional + + private[spark] 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 + + private[spark] 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 + private[spark] def resolveK8sMaster(rawMasterString: String): String = { if (!rawMasterString.startsWith("k8s://")) { throw new IllegalArgumentException("Master URL should start with k8s:// in Kubernetes mode.") 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 26cdcaa7f67c8..8f705ea86bc9c 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 @@ -43,6 +43,10 @@ package object constants { s"$DRIVER_CREDENTIALS_SECRETS_BASE_DIR/$DRIVER_CREDENTIALS_OAUTH_TOKEN_SECRET_NAME" private[spark] val DRIVER_CREDENTIALS_SECRET_VOLUME_NAME = "kubernetes-credentials" + // 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" + // Default and fixed ports private[spark] val DEFAULT_DRIVER_PORT = 7078 private[spark] val DEFAULT_BLOCKMANAGER_PORT = 7079 @@ -73,8 +77,11 @@ package object constants { private[spark] val ENV_R_FILE = "R_FILE" private[spark] val ENV_JAVA_OPT_PREFIX = "SPARK_JAVA_OPT_" private[spark] val ENV_MOUNTED_FILES_FROM_SECRET_DIR = "SPARK_MOUNTED_FILES_FROM_SECRET_DIR" + private[spark] val ENV_HADOOP_TOKEN_FILE_LOCATION = "HADOOP_TOKEN_FILE_LOCATION" + private[spark] val ENV_SPARK_USER = "SPARK_USER" // Bootstrapping dependencies with the init-container + private[spark] val INIT_CONTAINER_ANNOTATION = "pod.beta.kubernetes.io/init-containers" private[spark] val INIT_CONTAINER_SECRET_VOLUME_MOUNT_PATH = "/mnt/secrets/spark-init" private[spark] val INIT_CONTAINER_SUBMITTED_JARS_SECRET_KEY = @@ -94,6 +101,26 @@ package object constants { private[spark] val DEFAULT_SHUFFLE_MOUNT_NAME = "shuffle" private[spark] val INIT_CONTAINER_SECRET_VOLUME_NAME = "spark-init-secret" + // Hadoop Configuration + private[spark] val HADOOP_FILE_VOLUME = "hadoop-properties" + private[spark] val HADOOP_CONF_DIR_PATH = "/etc/hadoop/conf" + private[spark] val ENV_HADOOP_CONF_DIR = "HADOOP_CONF_DIR" + private[spark] val HADOOP_CONF_DIR_LOC = "spark.kubernetes.hadoop.conf.dir" + private[spark] val HADOOP_CONFIG_MAP_SPARK_CONF_NAME = + "spark.kubernetes.hadoop.executor.hadoopconfigmapname" + + // Kerberos Configuration + private[spark] val HADOOP_KERBEROS_SECRET_NAME = + "spark.kubernetes.kerberos.dt" + private[spark] val HADOOP_KERBEROS_CONF_SECRET = + "spark.kubernetes.kerberos.secretname" + private[spark] val HADOOP_KERBEROS_CONF_ITEM_KEY = + "spark.kubernetes.kerberos.itemkeyname" + private[spark] val KERBEROS_SECRET_LABEL_PREFIX = + "hadoop-tokens" + private[spark] val SPARK_HADOOP_PREFIX = "spark.hadoop." + private[spark] val HADOOP_SECURITY_AUTHENTICATION = + SPARK_HADOOP_PREFIX + "hadoop.security.authentication" // Bootstrapping dependencies via a secret private[spark] val MOUNTED_SMALL_FILES_SECRET_MOUNT_PATH = "/etc/spark-submitted-files" diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/Client.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/Client.scala index 6ed497130429f..a2b26b237ce01 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/Client.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/Client.scala @@ -81,6 +81,9 @@ private[spark] class Client( private val driverJavaOptions = submissionSparkConf.get( org.apache.spark.internal.config.DRIVER_JAVA_OPTIONS) + private val isKerberosEnabled = submissionSparkConf.get(KUBERNETES_KERBEROS_SUPPORT) + private val maybeSimpleAuthentication = + if (isKerberosEnabled) Some(s"-D$HADOOP_SECURITY_AUTHENTICATION=simple") else None /** * Run command that initalizes a DriverSpec that will be updated after each @@ -101,7 +104,8 @@ private[spark] class Client( .getAll .map { case (confKey, confValue) => s"-D$confKey=$confValue" - } ++ driverJavaOptions.map(Utils.splitCommandString).getOrElse(Seq.empty) + } ++ driverJavaOptions.map(Utils.splitCommandString).getOrElse(Seq.empty) ++ + maybeSimpleAuthentication val driverJavaOptsEnvs: Seq[EnvVar] = resolvedDriverJavaOpts.zipWithIndex.map { case (option, index) => new EnvVarBuilder() .withName(s"$ENV_JAVA_OPT_PREFIX$index") @@ -155,7 +159,9 @@ private[spark] class Client( } private[spark] object Client { - def run(sparkConf: SparkConf, clientArguments: ClientArguments): Unit = { + def run(sparkConf: SparkConf, + clientArguments: ClientArguments, + hadoopConfDir: Option[String]): Unit = { val namespace = sparkConf.get(KUBERNETES_NAMESPACE) val kubernetesAppId = s"spark-${UUID.randomUUID().toString.replaceAll("-", "")}" val launchTime = System.currentTimeMillis() @@ -174,6 +180,7 @@ private[spark] object Client { clientArguments.mainClass, clientArguments.driverArgs, clientArguments.otherPyFiles, + hadoopConfDir, sparkConf) Utils.tryWithResource(SparkKubernetesClientFactory.createKubernetesClient( master, @@ -201,6 +208,9 @@ private[spark] object Client { def main(args: Array[String]): Unit = { val parsedArguments = ClientArguments.fromCommandLineArgs(args) val sparkConf = new SparkConf() - run(sparkConf, parsedArguments) + // hadoopConfDir is passed into Client#run() to allow for us to + // test this env variable within the integration test environment + val hadoopConfDir = sys.env.get("HADOOP_CONF_DIR") + run(sparkConf, parsedArguments, hadoopConfDir) } } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestrator.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestrator.scala index 232aca4f2f55c..61a4fd8842683 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestrator.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestrator.scala @@ -20,7 +20,8 @@ import org.apache.spark.SparkConf import org.apache.spark.deploy.k8s.ConfigurationUtils import org.apache.spark.deploy.k8s.config._ import org.apache.spark.deploy.k8s.constants._ -import org.apache.spark.deploy.k8s.submit.submitsteps.{BaseDriverConfigurationStep, DependencyResolutionStep, DriverConfigurationStep, DriverKubernetesCredentialsStep, DriverServiceBootstrapStep, InitContainerBootstrapStep, MountSecretsStep, MountSmallLocalFilesStep, PythonStep, RStep} +import org.apache.spark.deploy.k8s.submit.submitsteps.{BaseDriverConfigurationStep, DependencyResolutionStep, DriverConfigurationStep, DriverKubernetesCredentialsStep, DriverServiceBootstrapStep, HadoopConfigBootstrapStep, InitContainerBootstrapStep, MountSecretsStep, MountSmallLocalFilesStep, PythonStep, RStep} +import org.apache.spark.deploy.k8s.submit.submitsteps.hadoopsteps.HadoopStepsOrchestrator import org.apache.spark.deploy.k8s.submit.submitsteps.initcontainer.InitContainerConfigurationStepsOrchestrator import org.apache.spark.deploy.k8s.submit.submitsteps.LocalDirectoryMountConfigurationStep import org.apache.spark.launcher.SparkLauncher @@ -38,6 +39,7 @@ private[spark] class DriverConfigurationStepsOrchestrator( mainClass: String, appArgs: Array[String], additionalPythonFiles: Seq[String], + hadoopConfDir: Option[String], submissionSparkConf: SparkConf) { // The resource name prefix is derived from the application name, making it easy to connect the @@ -52,6 +54,7 @@ private[spark] class DriverConfigurationStepsOrchestrator( private val filesDownloadPath = submissionSparkConf.get(INIT_CONTAINER_FILES_DOWNLOAD_LOCATION) private val dockerImagePullPolicy = submissionSparkConf.get(DOCKER_IMAGE_PULL_POLICY) private val initContainerConfigMapName = s"$kubernetesResourceNamePrefix-init-config" + private val hadoopConfigMapName = s"$kubernetesResourceNamePrefix-hadoop-config" def getAllConfigurationSteps(): Seq[DriverConfigurationStep] = { val additionalMainAppJar = mainAppResource match { @@ -114,6 +117,13 @@ private[spark] class DriverConfigurationStepsOrchestrator( val localDirectoryMountConfigurationStep = new LocalDirectoryMountConfigurationStep( submissionSparkConf) + val hadoopConfigSteps = + hadoopConfDir.map { conf => + val hadoopStepsOrchestrator = + new HadoopStepsOrchestrator(namespace, hadoopConfigMapName, submissionSparkConf, conf) + val hadoopConfSteps = hadoopStepsOrchestrator.getHadoopSteps() + Some(new HadoopConfigBootstrapStep(hadoopConfSteps, hadoopConfigMapName))} + .getOrElse(Option.empty[DriverConfigurationStep]) val resourceStep = mainAppResource match { case PythonMainAppResource(mainPyResource) => Option(new PythonStep(mainPyResource, additionalPythonFiles, filesDownloadPath)) @@ -196,6 +206,7 @@ private[spark] class DriverConfigurationStepsOrchestrator( dependencyResolutionStep, localDirectoryMountConfigurationStep) ++ submittedDependenciesBootstrapSteps ++ + hadoopConfigSteps.toSeq ++ resourceStep.toSeq ++ mountSecretsStep.toSeq } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/PodWithMainContainer.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/PodWithMainContainer.scala new file mode 100644 index 0000000000000..42f3343acee7d --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/PodWithMainContainer.scala @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.k8s + +import io.fabric8.kubernetes.api.model.{Container, Pod} + + /** + * The purpose of this case class is so that we can package together + * the driver pod with its container so we can bootstrap and modify + * the class instead of each component seperately + */ +private[spark] case class PodWithMainContainer( + pod: Pod, + mainContainer: Container) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/HadoopConfigBootstrapStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/HadoopConfigBootstrapStep.scala new file mode 100644 index 0000000000000..66c513be985b7 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/HadoopConfigBootstrapStep.scala @@ -0,0 +1,68 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.k8s.submit.submitsteps + +import scala.collection.JavaConverters._ + +import io.fabric8.kubernetes.api.model.ConfigMapBuilder + +import org.apache.spark.deploy.k8s.constants._ +import org.apache.spark.deploy.k8s.submit.submitsteps.hadoopsteps.{HadoopConfigSpec, HadoopConfigurationStep} + + /** + * This class configures the driverSpec with hadoop configuration logic which includes + * volume mounts, config maps, and environment variable manipulation. The steps are + * resolved with the orchestrator and they are run modifying the HadoopSpec with each + * step. The final HadoopSpec's contents will be appended to the driverSpec. + */ +private[spark] class HadoopConfigBootstrapStep( + hadoopConfigurationSteps: Seq[HadoopConfigurationStep], + hadoopConfigMapName: String ) + extends DriverConfigurationStep { + + override def configureDriver(driverSpec: KubernetesDriverSpec): KubernetesDriverSpec = { + var currentHadoopSpec = HadoopConfigSpec( + driverPod = driverSpec.driverPod, + driverContainer = driverSpec.driverContainer, + configMapProperties = Map.empty[String, String], + additionalDriverSparkConf = Map.empty[String, String], + dtSecret = None, + dtSecretName = HADOOP_KERBEROS_SECRET_NAME, + dtSecretItemKey = "") + for (nextStep <- hadoopConfigurationSteps) { + currentHadoopSpec = nextStep.configureContainers(currentHadoopSpec) + } + val configMap = + new ConfigMapBuilder() + .withNewMetadata() + .withName(hadoopConfigMapName) + .endMetadata() + .addToData(currentHadoopSpec.configMapProperties.asJava) + .build() + val executorSparkConf = driverSpec.driverSparkConf.clone() + .set(HADOOP_CONFIG_MAP_SPARK_CONF_NAME, hadoopConfigMapName) + .setAll(currentHadoopSpec.additionalDriverSparkConf) + driverSpec.copy( + driverPod = currentHadoopSpec.driverPod, + driverContainer = currentHadoopSpec.driverContainer, + driverSparkConf = executorSparkConf, + otherKubernetesResources = + driverSpec.otherKubernetesResources ++ + Seq(configMap) ++ currentHadoopSpec.dtSecret.toSeq + ) + } +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopConfMounterStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopConfMounterStep.scala new file mode 100644 index 0000000000000..0f00eca4abf75 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopConfMounterStep.scala @@ -0,0 +1,56 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.k8s.submit.submitsteps.hadoopsteps + +import java.io.File + +import org.apache.commons.io.FileUtils.readFileToString + +import org.apache.spark.deploy.k8s.{HadoopConfBootstrap, PodWithMainContainer} +import org.apache.spark.deploy.k8s.constants._ + + /** + * This step is responsible for taking the contents from each file in + * HADOOP_CONF_DIR, grabbing its contents as a string and storing each of them + * as a key-value pair in a configmap. Each key-value pair will be stored + * as a file, via Volume Mounts, later. The HADOOP_CONF_DIR_LOC is passed into the + * SchedulerBackend via sparkConf. + */ +private[spark] class HadoopConfMounterStep( + hadoopConfigMapName: String, + hadoopConfigurationFiles: Seq[File], + hadoopConfBootstrapConf: HadoopConfBootstrap, + hadoopConfDir: String) + extends HadoopConfigurationStep { + + override def configureContainers(hadoopConfigSpec: HadoopConfigSpec): HadoopConfigSpec = { + val bootstrappedPodAndMainContainer = + hadoopConfBootstrapConf.bootstrapMainContainerAndVolumes( + PodWithMainContainer( + hadoopConfigSpec.driverPod, + hadoopConfigSpec.driverContainer)) + hadoopConfigSpec.copy( + driverPod = bootstrappedPodAndMainContainer.pod, + driverContainer = bootstrappedPodAndMainContainer.mainContainer, + configMapProperties = + hadoopConfigurationFiles.map(file => + (file.toPath.getFileName.toString, readFileToString(file))).toMap, + additionalDriverSparkConf = hadoopConfigSpec.additionalDriverSparkConf ++ + Map(HADOOP_CONF_DIR_LOC -> hadoopConfDir) + ) + } +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopConfigSpec.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopConfigSpec.scala new file mode 100644 index 0000000000000..b38cae25dca26 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopConfigSpec.scala @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.k8s.submit.submitsteps.hadoopsteps + +import io.fabric8.kubernetes.api.model.{Container, Pod, Secret} + + /** + * Represents a given configuration of the hadoop configuration logic, informing the + * HadoopConfigBootstrapStep of how the driver should be configured. This includes: + *

+ * - What Spark properties should be set on the driver's SparkConf for the executors + * - The spec of the main container so that it can be modified to share volumes + * - The spec of the driver pod EXCEPT for the addition of the given hadoop configs (e.g. volumes + * the hadoop logic needs) + * - 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 + */ +private[spark] case class HadoopConfigSpec( + additionalDriverSparkConf: Map[String, String], + driverPod: Pod, + driverContainer: Container, + configMapProperties: Map[String, String], + dtSecret: Option[Secret], + dtSecretName: String, + dtSecretItemKey: String) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopConfigurationStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopConfigurationStep.scala new file mode 100644 index 0000000000000..b08b180ce8531 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopConfigurationStep.scala @@ -0,0 +1,25 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.k8s.submit.submitsteps.hadoopsteps + + /** + * Represents a step in preparing the driver with Hadoop Configuration logic. + */ +private[spark] trait HadoopConfigurationStep { + + def configureContainers(hadoopConfigSpec: HadoopConfigSpec): HadoopConfigSpec +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopKerberosKeytabResolverStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopKerberosKeytabResolverStep.scala new file mode 100644 index 0000000000000..8e9fe1e366ca6 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopKerberosKeytabResolverStep.scala @@ -0,0 +1,125 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.k8s.submit.submitsteps.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.hadoop.fs.FileSystem +import org.apache.hadoop.security.Credentials +import org.apache.hadoop.security.token.{Token, TokenIdentifier} + +import org.apache.spark.SparkConf +import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.deploy.k8s.{HadoopUGIUtil, KerberosTokenConfBootstrapImpl, PodWithMainContainer} +import org.apache.spark.deploy.k8s.constants._ +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, + maybePrincipal: Option[String], + maybeKeytab: Option[File], + maybeRenewerPrincipal: Option[String], + hadoopUGI: HadoopUGIUtil) extends HadoopConfigurationStep with Logging{ + private var originalCredentials: Credentials = _ + private var dfs : FileSystem = _ + private var renewer: String = _ + private var credentials: Credentials = _ + private var tokens: Iterable[Token[_ <: TokenIdentifier]] = _ + + override def configureContainers(hadoopConfigSpec: HadoopConfigSpec): HadoopConfigSpec = { + val hadoopConf = SparkHadoopUtil.get.newConfiguration(submissionSparkConf) + if (hadoopUGI.isSecurityEnabled) logDebug("Hadoop not configured with Kerberos") + val maybeJobUserUGI = + for { + principal <- maybePrincipal + keytab <- maybeKeytab + } yield { + // Not necessary with [Spark-16742] + // Reliant on [Spark-20328] for changing to YARN principal + submissionSparkConf.set("spark.yarn.principal", principal) + submissionSparkConf.set("spark.yarn.keytab", keytab.toURI.toString) + logDebug("Logged into KDC with keytab using Job User UGI") + hadoopUGI.loginUserFromKeytabAndReturnUGI( + principal, + keytab.toURI.toString) + } + // In the case that keytab is not specified we will read from Local Ticket Cache + val jobUserUGI = maybeJobUserUGI.getOrElse(hadoopUGI.getCurrentUser) + // It is necessary to run as jobUserUGI because logged in user != Current User + jobUserUGI.doAs(new PrivilegedExceptionAction[Void] { + override def run(): Void = { + originalCredentials = jobUserUGI.getCredentials + // TODO: This is not necessary with [Spark-20328] since we would be using + // Spark core providers to handle delegation token renewal + renewer = maybeRenewerPrincipal.getOrElse(jobUserUGI.getShortUserName) + credentials = new Credentials(originalCredentials) + hadoopUGI.dfsAddDelegationToken(hadoopConf, renewer, credentials) + tokens = credentials.getAllTokens.asScala + null + }}) + if (tokens.isEmpty) logDebug("Did not obtain any Delegation Tokens") + val data = hadoopUGI.serialize(credentials) + val renewalInterval = + hadoopUGI.getTokenRenewalInterval(tokens, hadoopConf).getOrElse(Long.MaxValue) + val currentTime: Long = hadoopUGI.getCurrentTime + val initialTokenDataKeyName = s"$KERBEROS_SECRET_LABEL_PREFIX-$currentTime-$renewalInterval" + val uniqueSecretName = s"$HADOOP_KERBEROS_SECRET_NAME.$currentTime" + val secretDT = + new SecretBuilder() + .withNewMetadata() + .withName(uniqueSecretName) + .withLabels(Map("refresh-hadoop-tokens" -> "yes").asJava) + .endMetadata() + .addToData(initialTokenDataKeyName, Base64.encodeBase64String(data)) + .build() + val bootstrapKerberos = new KerberosTokenConfBootstrapImpl( + uniqueSecretName, + initialTokenDataKeyName, + jobUserUGI.getShortUserName) + val withKerberosEnvPod = bootstrapKerberos.bootstrapMainContainerAndVolumes( + PodWithMainContainer( + hadoopConfigSpec.driverPod, + hadoopConfigSpec.driverContainer)) + hadoopConfigSpec.copy( + additionalDriverSparkConf = + hadoopConfigSpec.additionalDriverSparkConf ++ Map( + HADOOP_KERBEROS_CONF_ITEM_KEY -> initialTokenDataKeyName, + HADOOP_KERBEROS_CONF_SECRET -> uniqueSecretName), + driverPod = withKerberosEnvPod.pod, + driverContainer = withKerberosEnvPod.mainContainer, + dtSecret = Some(secretDT), + dtSecretName = uniqueSecretName, + dtSecretItemKey = initialTokenDataKeyName) + } +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopKerberosSecretResolverStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopKerberosSecretResolverStep.scala new file mode 100644 index 0000000000000..9d60f932bc736 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopKerberosSecretResolverStep.scala @@ -0,0 +1,56 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.k8s.submit.submitsteps.hadoopsteps + +import org.apache.hadoop.security.UserGroupInformation + +import org.apache.spark.SparkConf +import org.apache.spark.deploy.k8s.{KerberosTokenConfBootstrapImpl, PodWithMainContainer} +import org.apache.spark.deploy.k8s.constants._ + + /** + * 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( + submissionSparkConf: SparkConf, + tokenSecretName: String, + tokenItemKeyName: String) extends HadoopConfigurationStep { + + override def configureContainers(hadoopConfigSpec: HadoopConfigSpec): HadoopConfigSpec = { + val bootstrapKerberos = new KerberosTokenConfBootstrapImpl( + tokenSecretName, + tokenItemKeyName, + UserGroupInformation.getCurrentUser.getShortUserName) + val withKerberosEnvPod = bootstrapKerberos.bootstrapMainContainerAndVolumes( + PodWithMainContainer( + hadoopConfigSpec.driverPod, + hadoopConfigSpec.driverContainer)) + hadoopConfigSpec.copy( + driverPod = withKerberosEnvPod.pod, + driverContainer = withKerberosEnvPod.mainContainer, + additionalDriverSparkConf = + hadoopConfigSpec.additionalDriverSparkConf ++ Map( + HADOOP_KERBEROS_CONF_ITEM_KEY -> tokenItemKeyName, + HADOOP_KERBEROS_CONF_SECRET -> tokenSecretName), + dtSecret = None, + dtSecretName = tokenSecretName, + dtSecretItemKey = tokenItemKeyName) + } +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopStepsOrchestrator.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopStepsOrchestrator.scala new file mode 100644 index 0000000000000..0c79d8c5fac63 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopStepsOrchestrator.scala @@ -0,0 +1,104 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.k8s.submit.submitsteps.hadoopsteps + +import java.io.File + +import org.apache.spark.SparkConf +import org.apache.spark.deploy.k8s.{HadoopConfBootstrapImpl, HadoopUGIUtil, OptionRequirements} +import org.apache.spark.deploy.k8s.config._ +import org.apache.spark.internal.Logging + + /** + * Returns the complete ordered list of steps required to configure the hadoop configurations. + */ +private[spark] class HadoopStepsOrchestrator( + namespace: String, + hadoopConfigMapName: String, + submissionSparkConf: SparkConf, + hadoopConfDir: String) extends Logging{ + private val isKerberosEnabled = submissionSparkConf.get(KUBERNETES_KERBEROS_SUPPORT) + private val maybePrincipal = submissionSparkConf.get(KUBERNETES_KERBEROS_PRINCIPAL) + private val maybeKeytab = submissionSparkConf.get(KUBERNETES_KERBEROS_KEYTAB) + .map(k => new File(k)) + private val maybeExistingSecret = submissionSparkConf.get(KUBERNETES_KERBEROS_DT_SECRET_NAME) + private val maybeExistingSecretItemKey = + submissionSparkConf.get(KUBERNETES_KERBEROS_DT_SECRET_ITEM_KEY) + private val maybeRenewerPrincipal = + submissionSparkConf.get(KUBERNETES_KERBEROS_RENEWER_PRINCIPAL) + private val hadoopConfigurationFiles = getHadoopConfFiles(hadoopConfDir) + private val hadoopUGI = new HadoopUGIUtil + logInfo(s"Hadoop Conf directory: $hadoopConfDir") + + require(maybeKeytab.forall( _ => isKerberosEnabled ), + "You must enable Kerberos support if you are specifying a Kerberos Keytab") + + require(maybeExistingSecret.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( + maybeExistingSecret, + 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(): Seq[HadoopConfigurationStep] = { + val hadoopConfBootstrapImpl = new HadoopConfBootstrapImpl( + hadoopConfigMapName, + hadoopConfigurationFiles, + hadoopUGI) + val hadoopConfMounterStep = new HadoopConfMounterStep( + hadoopConfigMapName, + hadoopConfigurationFiles, + hadoopConfBootstrapImpl, + hadoopConfDir) + val maybeKerberosStep = + if (isKerberosEnabled) { + maybeExistingSecret.map(existingSecretName => Some(new HadoopKerberosSecretResolverStep( + submissionSparkConf, + existingSecretName, + maybeExistingSecretItemKey.get))).getOrElse(Some( + new HadoopKerberosKeytabResolverStep( + submissionSparkConf, + maybePrincipal, + maybeKeytab, + maybeRenewerPrincipal, + hadoopUGI))) + } else { + Option.empty[HadoopConfigurationStep] + } + Seq(hadoopConfMounterStep) ++ maybeKerberosStep.toSeq + } + + private def getHadoopConfFiles(path: String) : Seq[File] = { + def isFile(file: File) = if (file.isFile) Some(file) else None + val dir = new File(path) + if (dir.isDirectory) { + dir.listFiles.flatMap { file => isFile(file) }.toSeq + } else { + Seq.empty[File] + } + } +} \ No newline at end of file diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala index 98a0d879b6a58..89426de34faf3 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala @@ -16,11 +16,12 @@ */ package org.apache.spark.scheduler.cluster.k8s -import io.fabric8.kubernetes.api.model.{ContainerBuilder, ContainerPortBuilder, EnvVar, EnvVarBuilder, EnvVarSourceBuilder, Pod, PodBuilder, QuantityBuilder, VolumeBuilder, VolumeMountBuilder} import scala.collection.JavaConverters._ +import io.fabric8.kubernetes.api.model.{ContainerBuilder, ContainerPortBuilder, EnvVar, EnvVarBuilder, EnvVarSourceBuilder, Pod, PodBuilder, QuantityBuilder, VolumeBuilder, VolumeMountBuilder} + import org.apache.spark.{SparkConf, SparkException} -import org.apache.spark.deploy.k8s.{ConfigurationUtils, InitContainerResourceStagingServerSecretPlugin, PodWithDetachedInitContainer, SparkPodInitContainerBootstrap} +import org.apache.spark.deploy.k8s.{ConfigurationUtils, HadoopConfBootstrap, InitContainerResourceStagingServerSecretPlugin, KerberosTokenConfBootstrap, PodWithDetachedInitContainer, PodWithMainContainer, SparkPodInitContainerBootstrap} import org.apache.spark.deploy.k8s.config._ import org.apache.spark.deploy.k8s.constants._ import org.apache.spark.deploy.k8s.submit.{InitContainerUtil, MountSecretsBootstrap, MountSmallFilesBootstrap} @@ -45,7 +46,10 @@ private[spark] class ExecutorPodFactoryImpl( mountSmallFilesBootstrap: Option[MountSmallFilesBootstrap], executorInitContainerBootstrap: Option[SparkPodInitContainerBootstrap], executorMountInitContainerSecretPlugin: Option[InitContainerResourceStagingServerSecretPlugin], - executorLocalDirVolumeProvider: ExecutorLocalDirVolumeProvider) + executorLocalDirVolumeProvider: ExecutorLocalDirVolumeProvider, + shuffleManager: Option[KubernetesExternalShuffleManager], + hadoopBootStrap: Option[HadoopConfBootstrap], + kerberosBootstrap: Option[KerberosTokenConfBootstrap]) extends ExecutorPodFactory { import ExecutorPodFactoryImpl._ @@ -54,6 +58,9 @@ private[spark] class ExecutorPodFactoryImpl( org.apache.spark.internal.config.EXECUTOR_CLASS_PATH) private val executorJarsDownloadDir = sparkConf.get(INIT_CONTAINER_JARS_DOWNLOAD_LOCATION) + private val isKerberosEnabled = sparkConf.get(KUBERNETES_KERBEROS_SUPPORT) + private val maybeSimpleAuthentication = + if (isKerberosEnabled) Some(s"-D$HADOOP_SECURITY_AUTHENTICATION=simple") else None private val executorLabels = ConfigurationUtils.parsePrefixedKeyValuePairs( sparkConf, KUBERNETES_EXECUTOR_LABEL_PREFIX, @@ -135,15 +142,16 @@ private[spark] class ExecutorPodFactoryImpl( .withValue(cp) .build() } - val executorExtraJavaOptionsEnv = sparkConf - .get(org.apache.spark.internal.config.EXECUTOR_JAVA_OPTIONS) - .map { opts => - val delimitedOpts = Utils.splitCommandString(opts) - delimitedOpts.zipWithIndex.map { - case (opt, index) => - new EnvVarBuilder().withName(s"$ENV_JAVA_OPT_PREFIX$index").withValue(opt).build() - } - }.getOrElse(Seq.empty[EnvVar]) + val executorExtraJavaOptions = Option(( + sparkConf.get(org.apache.spark.internal.config.EXECUTOR_JAVA_OPTIONS) + ++ maybeSimpleAuthentication).mkString(" ")).filter(str => !str.isEmpty) + val executorExtraJavaOptionsEnv = executorExtraJavaOptions.map { opts => + val delimitedOpts = Utils.splitCommandString(opts) + delimitedOpts.zipWithIndex.map { + case (opt, index) => + new EnvVarBuilder().withName(s"$ENV_JAVA_OPT_PREFIX$index").withValue(opt).build() + } + }.getOrElse(Seq.empty[EnvVar]) val executorEnv = (Seq( (ENV_EXECUTOR_PORT, executorPort.toString), (ENV_DRIVER_URL, driverUrl), @@ -260,10 +268,24 @@ private[spark] class ExecutorPodFactoryImpl( val executorPodWithNodeAffinity = nodeAffinityExecutorPodModifier.addNodeAffinityAnnotationIfUseful( executorPodWithInitContainer, nodeToLocalTaskCount) + val (executorHadoopConfPod, executorHadoopConfContainer) = + hadoopBootStrap.map { bootstrap => + val podWithMainContainer = bootstrap.bootstrapMainContainerAndVolumes( + PodWithMainContainer(executorPodWithNodeAffinity, initBootstrappedExecutorContainer) + ) + (podWithMainContainer.pod, podWithMainContainer.mainContainer) + }.getOrElse(executorPodWithNodeAffinity, initBootstrappedExecutorContainer) - new PodBuilder(executorPodWithNodeAffinity) + val (executorKerberosPod, executorKerberosContainer) = + kerberosBootstrap.map { bootstrap => + val podWithMainContainer = bootstrap.bootstrapMainContainerAndVolumes( + PodWithMainContainer(executorHadoopConfPod, executorHadoopConfContainer)) + (podWithMainContainer.pod, podWithMainContainer.mainContainer) + }.getOrElse((executorHadoopConfPod, executorHadoopConfContainer)) + val resolvedExecutorPod = new PodBuilder(executorKerberosPod) + new PodBuilder(resolvedExecutorPod) .editSpec() - .addToContainers(initBootstrappedExecutorContainer) + .addToContainers(executorKerberosContainer) .endSpec() .build() } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala index 60260e2931c29..06372caeebf84 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala @@ -21,7 +21,7 @@ import java.io.File import io.fabric8.kubernetes.client.Config import org.apache.spark.SparkContext -import org.apache.spark.deploy.k8s.{ConfigurationUtils, InitContainerResourceStagingServerSecretPluginImpl, SparkKubernetesClientFactory, SparkPodInitContainerBootstrapImpl} +import org.apache.spark.deploy.k8s.{ConfigurationUtils, HadoopConfBootstrapImpl, HadoopUGIUtil, InitContainerResourceStagingServerSecretPluginImpl, KerberosTokenConfBootstrapImpl, SparkKubernetesClientFactory, SparkPodInitContainerBootstrapImpl} import org.apache.spark.deploy.k8s.config._ import org.apache.spark.deploy.k8s.constants._ import org.apache.spark.deploy.k8s.submit.{MountSecretsBootstrapImpl, MountSmallFilesBootstrapImpl} @@ -44,6 +44,10 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit override def createSchedulerBackend(sc: SparkContext, masterURL: String, scheduler: TaskScheduler) : SchedulerBackend = { val sparkConf = sc.getConf + val maybeHadoopConfigMap = sparkConf.getOption(HADOOP_CONFIG_MAP_SPARK_CONF_NAME) + val maybeHadoopConfDir = sparkConf.getOption(HADOOP_CONF_DIR_LOC) + val maybeDTSecretName = sparkConf.getOption(HADOOP_KERBEROS_CONF_SECRET) + val maybeDTDataItem = sparkConf.getOption(HADOOP_KERBEROS_CONF_ITEM_KEY) val maybeInitContainerConfigMap = sparkConf.get(EXECUTOR_INIT_CONTAINER_CONFIG_MAP) val maybeInitContainerConfigMapKey = sparkConf.get(EXECUTOR_INIT_CONTAINER_CONFIG_MAP_KEY) val maybeSubmittedFilesSecret = sparkConf.get(EXECUTOR_SUBMITTED_SMALL_FILES_SECRET) @@ -81,6 +85,27 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit configMapKey) } + val hadoopBootStrap = for { + hadoopConfigMap <- maybeHadoopConfigMap + } yield { + val hadoopUtil = new HadoopUGIUtil + val hadoopConfigurations = maybeHadoopConfDir.map( + conf_dir => getHadoopConfFiles(conf_dir)).getOrElse(Array.empty[File]) + new HadoopConfBootstrapImpl( + hadoopConfigMap, + hadoopConfigurations, + hadoopUtil + ) + } + val kerberosBootstrap = for { + secretName <- maybeDTSecretName + secretItemKey <- maybeDTDataItem + } yield { + new KerberosTokenConfBootstrapImpl( + secretName, + secretItemKey, + Utils.getCurrentUserName) + } val mountSmallFilesBootstrap = for { secretName <- maybeSubmittedFilesSecret secretMountPath <- maybeSubmittedFilesSecretMountPath @@ -105,6 +130,10 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit " therefore not attempt to fetch remote or submitted dependencies.") } + if (maybeHadoopConfigMap.isEmpty) { + logWarning("The executor's hadoop config map key was not specified. Executors will" + + " therefore not attempt to fetch hadoop configuration files.") + } val kubernetesClient = SparkKubernetesClientFactory.createKubernetesClient( KUBERNETES_MASTER_INTERNAL_URL, Some(sparkConf.get(KUBERNETES_NAMESPACE)), @@ -134,7 +163,10 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit mountSmallFilesBootstrap, executorInitContainerBootstrap, executorInitContainerSecretVolumePlugin, - executorLocalDirVolumeProvider) + executorLocalDirVolumeProvider, + kubernetesShuffleManager, + hadoopBootStrap, + kerberosBootstrap) val allocatorExecutor = ThreadUtils .newDaemonSingleThreadScheduledExecutor("kubernetes-pod-allocator") val requestExecutorsService = ThreadUtils.newDaemonCachedThreadPool( @@ -152,4 +184,13 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit override def initialize(scheduler: TaskScheduler, backend: SchedulerBackend): Unit = { scheduler.asInstanceOf[TaskSchedulerImpl].initialize(backend) } + private def getHadoopConfFiles(path: String) : Array[File] = { + def isFile(file: File) = if (file.isFile) Some(file) else None + val dir = new File(path) + if (dir.isDirectory) { + dir.listFiles.flatMap { file => isFile(file) } + } else { + Array.empty[File] + } + } } From 50f47d030f459ee2643642ecd69de664e70d8084 Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Thu, 2 Nov 2017 21:00:22 -0400 Subject: [PATCH 02/14] fixing build and unit test issues --- .../spark/deploy/k8s/HadoopUGIUtil.scala | 6 +-- .../k8s/KerberosTokenConfBootstrap.scala | 2 +- .../hadoopsteps/HadoopStepsOrchestrator.scala | 2 +- .../cluster/k8s/ExecutorPodFactory.scala | 1 - .../k8s/KubernetesClusterManager.scala | 1 - ...rConfigurationStepsOrchestratorSuite.scala | 42 ++++++++++++++++--- .../cluster/k8s/ExecutorPodFactorySuite.scala | 28 +++++++++---- 7 files changed, 62 insertions(+), 20 deletions(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/HadoopUGIUtil.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/HadoopUGIUtil.scala index e9837f3258611..47f1fa5c9631a 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/HadoopUGIUtil.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/HadoopUGIUtil.scala @@ -45,7 +45,7 @@ private[spark] class HadoopUGIUtil{ def getCurrentTime: Long = System.currentTimeMillis() // Functions that should be in Core with Rebase to 2.3 - @deprecated("Moved to core in 2.2", "2.2") + @deprecated("Moved to core in 2.3", "2.3") def getTokenRenewalInterval( renewedTokens: Iterable[Token[_ <: TokenIdentifier]], hadoopConf: Configuration): Option[Long] = { @@ -62,7 +62,7 @@ private[spark] class HadoopUGIUtil{ renewIntervals.reduceLeftOption(_ min _) } - @deprecated("Moved to core in 2.2", "2.2") + @deprecated("Moved to core in 2.3", "2.3") def serialize(creds: Credentials): Array[Byte] = { val byteStream = new ByteArrayOutputStream val dataStream = new DataOutputStream(byteStream) @@ -70,7 +70,7 @@ private[spark] class HadoopUGIUtil{ byteStream.toByteArray } - @deprecated("Moved to core in 2.2", "2.2") + @deprecated("Moved to core in 2.3", "2.3") def deserialize(tokenBytes: Array[Byte]): Credentials = { val creds = new Credentials() creds.readTokenStorageStream(new DataInputStream(new ByteArrayInputStream(tokenBytes))) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KerberosTokenConfBootstrap.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KerberosTokenConfBootstrap.scala index dfe46e7106e87..9b8f83dc3f891 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KerberosTokenConfBootstrap.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KerberosTokenConfBootstrap.scala @@ -73,4 +73,4 @@ private[spark] class KerberosTokenConfBootstrapImpl( pod = dtMountedPod, mainContainer = mainContainerWithMountedKerberos) } -} \ No newline at end of file +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopStepsOrchestrator.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopStepsOrchestrator.scala index 0c79d8c5fac63..8f052662ba600 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopStepsOrchestrator.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopStepsOrchestrator.scala @@ -101,4 +101,4 @@ private[spark] class HadoopStepsOrchestrator( Seq.empty[File] } } -} \ No newline at end of file +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala index 89426de34faf3..78e2ac48809d8 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala @@ -47,7 +47,6 @@ private[spark] class ExecutorPodFactoryImpl( executorInitContainerBootstrap: Option[SparkPodInitContainerBootstrap], executorMountInitContainerSecretPlugin: Option[InitContainerResourceStagingServerSecretPlugin], executorLocalDirVolumeProvider: ExecutorLocalDirVolumeProvider, - shuffleManager: Option[KubernetesExternalShuffleManager], hadoopBootStrap: Option[HadoopConfBootstrap], kerberosBootstrap: Option[KerberosTokenConfBootstrap]) extends ExecutorPodFactory { diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala index 06372caeebf84..d624564924cd6 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala @@ -164,7 +164,6 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit executorInitContainerBootstrap, executorInitContainerSecretVolumePlugin, executorLocalDirVolumeProvider, - kubernetesShuffleManager, hadoopBootStrap, kerberosBootstrap) val allocatorExecutor = ThreadUtils diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestratorSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestratorSuite.scala index 7af6613fcc9b3..64ab0b10f49ff 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestratorSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestratorSuite.scala @@ -18,7 +18,7 @@ package org.apache.spark.deploy.k8s.submit import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.deploy.k8s.config._ -import org.apache.spark.deploy.k8s.submit.submitsteps.{BaseDriverConfigurationStep, DependencyResolutionStep, DriverConfigurationStep, DriverKubernetesCredentialsStep, DriverServiceBootstrapStep, InitContainerBootstrapStep, LocalDirectoryMountConfigurationStep, MountSecretsStep, MountSmallLocalFilesStep, PythonStep, RStep} +import org.apache.spark.deploy.k8s.submit.submitsteps.{BaseDriverConfigurationStep, DependencyResolutionStep, DriverConfigurationStep, DriverKubernetesCredentialsStep, DriverServiceBootstrapStep, HadoopConfigBootstrapStep, InitContainerBootstrapStep, LocalDirectoryMountConfigurationStep, MountSecretsStep, MountSmallLocalFilesStep, PythonStep, RStep} private[spark] class DriverConfigurationStepsOrchestratorSuite extends SparkFunSuite { @@ -45,7 +45,8 @@ private[spark] class DriverConfigurationStepsOrchestratorSuite extends SparkFunS APP_NAME, MAIN_CLASS, APP_ARGS, - Seq.empty[String], + ADDITIONAL_PYTHON_FILES, + None, sparkConf) validateStepTypes( orchestrator, @@ -69,7 +70,8 @@ private[spark] class DriverConfigurationStepsOrchestratorSuite extends SparkFunS APP_NAME, MAIN_CLASS, APP_ARGS, - Seq.empty[String], + ADDITIONAL_PYTHON_FILES, + None, sparkConf) validateStepTypes( orchestrator, @@ -93,6 +95,7 @@ private[spark] class DriverConfigurationStepsOrchestratorSuite extends SparkFunS MAIN_CLASS, APP_ARGS, ADDITIONAL_PYTHON_FILES, + None, sparkConf) validateStepTypes( orchestrator, @@ -116,6 +119,7 @@ private[spark] class DriverConfigurationStepsOrchestratorSuite extends SparkFunS MAIN_CLASS, APP_ARGS, Seq.empty[String], + None, sparkConf) validateStepTypes( orchestrator, @@ -127,7 +131,6 @@ private[spark] class DriverConfigurationStepsOrchestratorSuite extends SparkFunS classOf[RStep]) } - test("Only local files without a resource staging server.") { val sparkConf = new SparkConf(false).set("spark.files", "/var/spark/file1.txt") val mainAppResource = JavaMainAppResource("local:///var/apps/jars/main.jar") @@ -139,7 +142,8 @@ private[spark] class DriverConfigurationStepsOrchestratorSuite extends SparkFunS APP_NAME, MAIN_CLASS, APP_ARGS, - Seq.empty[String], + ADDITIONAL_PYTHON_FILES, + None, sparkConf) validateStepTypes( orchestrator, @@ -164,7 +168,8 @@ private[spark] class DriverConfigurationStepsOrchestratorSuite extends SparkFunS APP_NAME, MAIN_CLASS, APP_ARGS, - Seq.empty[String], + ADDITIONAL_PYTHON_FILES, + None, sparkConf) validateStepTypes( orchestrator, @@ -175,6 +180,31 @@ private[spark] class DriverConfigurationStepsOrchestratorSuite extends SparkFunS classOf[LocalDirectoryMountConfigurationStep], classOf[MountSecretsStep]) } + test("Submission steps with hdfs interaction and HADOOP_CONF_DIR defined") { + val sparkConf = new SparkConf(false) + val mainAppResource = JavaMainAppResource("local:///var/apps/jars/main.jar") + val hadoopConf = Some("/etc/hadoop/conf") + val orchestrator = new DriverConfigurationStepsOrchestrator( + NAMESPACE, + APP_ID, + LAUNCH_TIME, + mainAppResource, + APP_NAME, + MAIN_CLASS, + APP_ARGS, + ADDITIONAL_PYTHON_FILES, + hadoopConf, + sparkConf) + val steps = orchestrator.getAllConfigurationSteps() + validateStepTypes( + orchestrator, + classOf[BaseDriverConfigurationStep], + classOf[DriverServiceBootstrapStep], + classOf[DriverKubernetesCredentialsStep], + classOf[DependencyResolutionStep], + classOf[LocalDirectoryMountConfigurationStep], + classOf[HadoopConfigBootstrapStep]) + } private def validateStepTypes( orchestrator: DriverConfigurationStepsOrchestrator, diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactorySuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactorySuite.scala index 71204a5aa1deb..b77cd2b655594 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactorySuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactorySuite.scala @@ -77,7 +77,9 @@ class ExecutorPodFactorySuite extends SparkFunSuite with BeforeAndAfter with Bef None, None, None, - executorLocalDirVolumeProvider) + executorLocalDirVolumeProvider, + None, + None) val executor = factory.createExecutorPod( "1", "dummy", "dummy", Seq[(String, String)](), driverPod, Map[String, Int]()) @@ -117,7 +119,9 @@ class ExecutorPodFactorySuite extends SparkFunSuite with BeforeAndAfter with Bef None, None, None, - executorLocalDirVolumeProvider) + executorLocalDirVolumeProvider, + None, + None) val executor = factory.createExecutorPod( "1", "dummy", "dummy", Seq[(String, String)](), driverPod, Map[String, Int]()) @@ -138,7 +142,9 @@ class ExecutorPodFactorySuite extends SparkFunSuite with BeforeAndAfter with Bef None, None, None, - executorLocalDirVolumeProvider) + executorLocalDirVolumeProvider, + None, + None) val executor = factory.createExecutorPod( "1", "dummy", "dummy", Seq[(String, String)](), driverPod, Map[String, Int]()) @@ -172,7 +178,9 @@ class ExecutorPodFactorySuite extends SparkFunSuite with BeforeAndAfter with Bef None, Some(initContainerBootstrap), None, - executorLocalDirVolumeProvider) + executorLocalDirVolumeProvider, + None, + None) val executor = factory.createExecutorPod( "1", "dummy", "dummy", Seq[(String, String)](), driverPod, Map[String, Int]()) @@ -202,7 +210,9 @@ class ExecutorPodFactorySuite extends SparkFunSuite with BeforeAndAfter with Bef None, None, None, - executorLocalDirVolumeProvider) + executorLocalDirVolumeProvider, + None, + None) val executor = factory.createExecutorPod( "1", "dummy", "dummy", Seq[(String, String)](), driverPod, Map[String, Int]()) assert(executor.getSpec.getVolumes.size === 1) @@ -223,7 +233,9 @@ class ExecutorPodFactorySuite extends SparkFunSuite with BeforeAndAfter with Bef Some(smallFiles), None, None, - executorLocalDirVolumeProvider) + executorLocalDirVolumeProvider, + None, + None) val executor = factory.createExecutorPod( "1", "dummy", "dummy", Seq[(String, String)](), driverPod, Map[String, Int]()) @@ -256,7 +268,9 @@ class ExecutorPodFactorySuite extends SparkFunSuite with BeforeAndAfter with Bef None, None, None, - executorLocalDirVolumeProvider) + executorLocalDirVolumeProvider, + None, + None) val executor = factory.createExecutorPod( "1", "dummy", "dummy", Seq[(String, String)]("qux" -> "quux"), driverPod, Map[String, Int]()) From 87df4a78c26c2caf96fef3f8691d21b16294ffea Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Thu, 2 Nov 2017 22:43:28 -0400 Subject: [PATCH 03/14] fixed integration tests --- .../spark/deploy/k8s/integrationtest/KubernetesSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala index 9d48d488bf967..f761d547eb513 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala @@ -367,7 +367,7 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { mainClass = mainClass, driverArgs = appArgs, otherPyFiles = otherPyFiles) - Client.run(sparkConf, clientArguments) + Client.run(sparkConf, clientArguments, None) val driverPod = kubernetesTestComponents.kubernetesClient .pods() .withLabel("spark-app-locator", APP_LOCATOR_LABEL) From 67856a5ed45a37fcf4a7a5d14fdbe1a95df1c695 Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Thu, 2 Nov 2017 23:06:48 -0400 Subject: [PATCH 04/14] fixed issue with executorPodFactory unit tests --- .../spark/scheduler/cluster/k8s/ExecutorPodFactory.scala | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala index 78e2ac48809d8..16ed1eead7167 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala @@ -270,8 +270,7 @@ private[spark] class ExecutorPodFactoryImpl( val (executorHadoopConfPod, executorHadoopConfContainer) = hadoopBootStrap.map { bootstrap => val podWithMainContainer = bootstrap.bootstrapMainContainerAndVolumes( - PodWithMainContainer(executorPodWithNodeAffinity, initBootstrappedExecutorContainer) - ) + PodWithMainContainer(executorPodWithNodeAffinity, initBootstrappedExecutorContainer)) (podWithMainContainer.pod, podWithMainContainer.mainContainer) }.getOrElse(executorPodWithNodeAffinity, initBootstrappedExecutorContainer) @@ -281,8 +280,7 @@ private[spark] class ExecutorPodFactoryImpl( PodWithMainContainer(executorHadoopConfPod, executorHadoopConfContainer)) (podWithMainContainer.pod, podWithMainContainer.mainContainer) }.getOrElse((executorHadoopConfPod, executorHadoopConfContainer)) - val resolvedExecutorPod = new PodBuilder(executorKerberosPod) - new PodBuilder(resolvedExecutorPod) + new PodBuilder(executorKerberosPod) .editSpec() .addToContainers(executorKerberosContainer) .endSpec() From 7cdae311c98fcd0cc350546016a2ed9af1bdc264 Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Fri, 3 Nov 2017 14:00:04 -0400 Subject: [PATCH 05/14] first series of PR comments --- docs/running-on-kubernetes.md | 2 +- .../org/apache/spark/deploy/k8s/HadoopConfBootstrap.scala | 6 +++--- .../scala/org/apache/spark/deploy/k8s/HadoopUGIUtil.scala | 1 + .../main/scala/org/apache/spark/deploy/k8s/constants.scala | 3 +-- .../hadoopsteps/HadoopKerberosKeytabResolverStep.scala | 4 ++-- 5 files changed, 8 insertions(+), 8 deletions(-) diff --git a/docs/running-on-kubernetes.md b/docs/running-on-kubernetes.md index 5920988b9facc..13efe88eb34cb 100644 --- a/docs/running-on-kubernetes.md +++ b/docs/running-on-kubernetes.md @@ -784,7 +784,7 @@ from the other deployment modes. See the [configuration page](configuration.html (none) Assuming you have set spark.kubernetes.kerberos.enabled to be true. This will let you specify - the principal that you wish to use to handle renewing of Delegation Tokens. This is optional as you + the principal that you wish to use to handle renewing of Delegation Tokens. This is optional as we will set the principal to be the job users principal by default. diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/HadoopConfBootstrap.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/HadoopConfBootstrap.scala index 228740f9fe2c0..a49bfa7032429 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/HadoopConfBootstrap.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/HadoopConfBootstrap.scala @@ -57,9 +57,9 @@ private[spark] class HadoopConfBootstrapImpl( .editSpec() .addNewVolume() .withName(HADOOP_FILE_VOLUME) - .withNewConfigMap() - .withName(hadoopConfConfigMapName) - .withItems(keyPaths.asJava) + .withNewConfigMap() + .withName(hadoopConfConfigMapName) + .withItems(keyPaths.asJava) .endConfigMap() .endVolume() .endSpec() diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/HadoopUGIUtil.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/HadoopUGIUtil.scala index 47f1fa5c9631a..c5e2c8fcb2f27 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/HadoopUGIUtil.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/HadoopUGIUtil.scala @@ -67,6 +67,7 @@ private[spark] class HadoopUGIUtil{ val byteStream = new ByteArrayOutputStream val dataStream = new DataOutputStream(byteStream) creds.writeTokenStorageToStream(dataStream) + dataStream.close() byteStream.toByteArray } 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 8f705ea86bc9c..df1b8ee4a2026 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,6 @@ package object constants { private[spark] val ENV_SPARK_USER = "SPARK_USER" // Bootstrapping dependencies with the init-container - private[spark] val INIT_CONTAINER_ANNOTATION = "pod.beta.kubernetes.io/init-containers" private[spark] val INIT_CONTAINER_SECRET_VOLUME_MOUNT_PATH = "/mnt/secrets/spark-init" private[spark] val INIT_CONTAINER_SUBMITTED_JARS_SECRET_KEY = @@ -107,7 +106,7 @@ package object constants { private[spark] val ENV_HADOOP_CONF_DIR = "HADOOP_CONF_DIR" private[spark] val HADOOP_CONF_DIR_LOC = "spark.kubernetes.hadoop.conf.dir" private[spark] val HADOOP_CONFIG_MAP_SPARK_CONF_NAME = - "spark.kubernetes.hadoop.executor.hadoopconfigmapname" + "spark.kubernetes.hadoop.executor.hadoopConfigMapName" // Kerberos Configuration private[spark] val HADOOP_KERBEROS_SECRET_NAME = diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopKerberosKeytabResolverStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopKerberosKeytabResolverStep.scala index 8e9fe1e366ca6..90cc4e04b8f26 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopKerberosKeytabResolverStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopKerberosKeytabResolverStep.scala @@ -50,7 +50,7 @@ private[spark] class HadoopKerberosKeytabResolverStep( maybePrincipal: Option[String], maybeKeytab: Option[File], maybeRenewerPrincipal: Option[String], - hadoopUGI: HadoopUGIUtil) extends HadoopConfigurationStep with Logging{ + hadoopUGI: HadoopUGIUtil) extends HadoopConfigurationStep with Logging { private var originalCredentials: Credentials = _ private var dfs : FileSystem = _ private var renewer: String = _ @@ -59,7 +59,7 @@ private[spark] class HadoopKerberosKeytabResolverStep( override def configureContainers(hadoopConfigSpec: HadoopConfigSpec): HadoopConfigSpec = { val hadoopConf = SparkHadoopUtil.get.newConfiguration(submissionSparkConf) - if (hadoopUGI.isSecurityEnabled) logDebug("Hadoop not configured with Kerberos") + if (!hadoopUGI.isSecurityEnabled) logDebug("Hadoop not configured with Kerberos") val maybeJobUserUGI = for { principal <- maybePrincipal From 04aa26f0fdafc3648d82e16d82d23ab3977e29d9 Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Fri, 3 Nov 2017 21:52:12 -0400 Subject: [PATCH 06/14] handle most PR comments --- docs/running-on-kubernetes.md | 2 +- .../spark/deploy/k8s/HadoopConfBootstrap.scala | 15 ++++++++------- .../deploy/k8s/KerberosTokenConfBootstrap.scala | 10 +++++----- .../org/apache/spark/deploy/k8s/config.scala | 8 +------- .../org/apache/spark/deploy/k8s/constants.scala | 5 +++++ .../DriverConfigurationStepsOrchestrator.scala | 7 ++++++- .../hadoopsteps/HadoopConfMounterStep.scala | 5 +++-- .../HadoopKerberosKeytabResolverStep.scala | 6 ++++-- .../hadoopsteps/HadoopStepsOrchestrator.scala | 4 +++- 9 files changed, 36 insertions(+), 26 deletions(-) diff --git a/docs/running-on-kubernetes.md b/docs/running-on-kubernetes.md index 13efe88eb34cb..1b070973afe7e 100644 --- a/docs/running-on-kubernetes.md +++ b/docs/running-on-kubernetes.md @@ -780,7 +780,7 @@ from the other deployment modes. See the [configuration page](configuration.html - spark.kubernetes.kerberos.rewewer.principal + spark.kubernetes.kerberos.renewer.principal (none) Assuming you have set spark.kubernetes.kerberos.enabled to be true. This will let you specify diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/HadoopConfBootstrap.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/HadoopConfBootstrap.scala index a49bfa7032429..3177d32a92e01 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/HadoopConfBootstrap.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/HadoopConfBootstrap.scala @@ -47,12 +47,13 @@ private[spark] class HadoopConfBootstrapImpl( override def bootstrapMainContainerAndVolumes( originalPodWithMainContainer: PodWithMainContainer) : PodWithMainContainer = { - logInfo("HADOOP_CONF_DIR defined. Mounting HDFS specific .xml files") - val keyPaths = hadoopConfigFiles.map(file => + logInfo("HADOOP_CONF_DIR defined. Mounting Hadoop specific .xml files") + val keyPaths = hadoopConfigFiles.map{ file => + val fileStringPath = file.toPath.getFileName.toString new KeyToPathBuilder() - .withKey(file.toPath.getFileName.toString) - .withPath(file.toPath.getFileName.toString) - .build()) + .withKey(fileStringPath) + .withPath(fileStringPath) + .build() } val hadoopSupportedPod = new PodBuilder(originalPodWithMainContainer.pod) .editSpec() .addNewVolume() @@ -64,7 +65,7 @@ private[spark] class HadoopConfBootstrapImpl( .endVolume() .endSpec() .build() - val mainContainerWithMountedHadoopConf = new ContainerBuilder( + val hadoopSupportedContainer = new ContainerBuilder( originalPodWithMainContainer.mainContainer) .addNewVolumeMount() .withName(HADOOP_FILE_VOLUME) @@ -81,6 +82,6 @@ private[spark] class HadoopConfBootstrapImpl( .build() originalPodWithMainContainer.copy( pod = hadoopSupportedPod, - mainContainer = mainContainerWithMountedHadoopConf) + mainContainer = hadoopSupportedContainer) } } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KerberosTokenConfBootstrap.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KerberosTokenConfBootstrap.scala index 9b8f83dc3f891..5002673a3d962 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KerberosTokenConfBootstrap.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KerberosTokenConfBootstrap.scala @@ -39,12 +39,11 @@ private[spark] class KerberosTokenConfBootstrapImpl( secretItemKey: String, userName: String) extends KerberosTokenConfBootstrap with Logging{ - override def bootstrapMainContainerAndVolumes( originalPodWithMainContainer: PodWithMainContainer) : PodWithMainContainer = { logInfo("Mounting HDFS DT from Secret for Secure HDFS") - val dtMountedPod = new PodBuilder(originalPodWithMainContainer.pod) + val secretMountedPod = new PodBuilder(originalPodWithMainContainer.pod) .editOrNewSpec() .addNewVolume() .withName(SPARK_APP_HADOOP_SECRET_VOLUME_NAME) @@ -54,7 +53,8 @@ private[spark] class KerberosTokenConfBootstrapImpl( .endVolume() .endSpec() .build() - val mainContainerWithMountedKerberos = new ContainerBuilder( + // TODO: ENV_HADOOP_TOKEN_FILE_LOCATION should point to the latest token data item key. + val secretMountedContainer = new ContainerBuilder( originalPodWithMainContainer.mainContainer) .addNewVolumeMount() .withName(SPARK_APP_HADOOP_SECRET_VOLUME_NAME) @@ -70,7 +70,7 @@ private[spark] class KerberosTokenConfBootstrapImpl( .endEnv() .build() originalPodWithMainContainer.copy( - pod = dtMountedPod, - mainContainer = mainContainerWithMountedKerberos) + pod = secretMountedPod, + mainContainer = secretMountedContainer) } } 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 739d205c0070e..cfbd242c0a62c 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 @@ -157,12 +157,6 @@ package object config extends Logging { .stringConf .createOptional - private[spark] val KUBERNETES_SHUFFLE_DIR = - ConfigBuilder("spark.kubernetes.shuffle.dir") - .doc("Path to the shared shuffle directories.") - .stringConf - .createOptional - private[spark] val KUBERNETES_SHUFFLE_APISERVER_URI = ConfigBuilder("spark.kubernetes.shuffle.apiServer.url") .doc("URL to the Kubernetes API server that the shuffle service will monitor for Spark pods.") @@ -523,7 +517,7 @@ package object config extends Logging { .createOptional private[spark] val KUBERNETES_KERBEROS_RENEWER_PRINCIPAL = - ConfigBuilder("spark.kubernetes.kerberos.rewnewer.principal") + ConfigBuilder("spark.kubernetes.kerberos.renewer.principal") .doc("Specify the principal" + " you wish to renew and retrieve your Kerberos values with") .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 df1b8ee4a2026..9dfc73039a704 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 @@ -120,6 +120,11 @@ package object constants { private[spark] val SPARK_HADOOP_PREFIX = "spark.hadoop." private[spark] val HADOOP_SECURITY_AUTHENTICATION = SPARK_HADOOP_PREFIX + "hadoop.security.authentication" + + // Kerberos Token-Refresh Server + private[spark] val KERBEROS_REFRESH_LABEL_KEY = "refresh-hadoop-tokens" + private[spark] val KERBEROS_REFRESH_LABEL_VALUE = "yes" + // Bootstrapping dependencies via a secret private[spark] val MOUNTED_SMALL_FILES_SECRET_MOUNT_PATH = "/etc/spark-submitted-files" diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestrator.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestrator.scala index 61a4fd8842683..810af15212056 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestrator.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestrator.scala @@ -120,7 +120,12 @@ private[spark] class DriverConfigurationStepsOrchestrator( val hadoopConfigSteps = hadoopConfDir.map { conf => val hadoopStepsOrchestrator = - new HadoopStepsOrchestrator(namespace, hadoopConfigMapName, submissionSparkConf, conf) + new HadoopStepsOrchestrator( + kubernetesResourceNamePrefix, + namespace, + hadoopConfigMapName, + submissionSparkConf, + conf) val hadoopConfSteps = hadoopStepsOrchestrator.getHadoopSteps() Some(new HadoopConfigBootstrapStep(hadoopConfSteps, hadoopConfigMapName))} .getOrElse(Option.empty[DriverConfigurationStep]) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopConfMounterStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopConfMounterStep.scala index 0f00eca4abf75..37a41d71ba616 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopConfMounterStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopConfMounterStep.scala @@ -18,7 +18,8 @@ package org.apache.spark.deploy.k8s.submit.submitsteps.hadoopsteps import java.io.File -import org.apache.commons.io.FileUtils.readFileToString +import com.google.common.base.Charsets +import com.google.common.io.Files import org.apache.spark.deploy.k8s.{HadoopConfBootstrap, PodWithMainContainer} import org.apache.spark.deploy.k8s.constants._ @@ -48,7 +49,7 @@ private[spark] class HadoopConfMounterStep( driverContainer = bootstrappedPodAndMainContainer.mainContainer, configMapProperties = hadoopConfigurationFiles.map(file => - (file.toPath.getFileName.toString, readFileToString(file))).toMap, + (file.toPath.getFileName.toString, Files.toString(file, Charsets.UTF_8))).toMap, additionalDriverSparkConf = hadoopConfigSpec.additionalDriverSparkConf ++ Map(HADOOP_CONF_DIR_LOC -> hadoopConfDir) ) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopKerberosKeytabResolverStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopKerberosKeytabResolverStep.scala index 90cc4e04b8f26..d0b3b172f508f 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopKerberosKeytabResolverStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopKerberosKeytabResolverStep.scala @@ -46,6 +46,7 @@ import org.apache.spark.internal.Logging * DriverSpec. */ private[spark] class HadoopKerberosKeytabResolverStep( + kubernetesResourceNamePrefix: String, submissionSparkConf: SparkConf, maybePrincipal: Option[String], maybeKeytab: Option[File], @@ -94,12 +95,13 @@ private[spark] class HadoopKerberosKeytabResolverStep( hadoopUGI.getTokenRenewalInterval(tokens, hadoopConf).getOrElse(Long.MaxValue) val currentTime: Long = hadoopUGI.getCurrentTime val initialTokenDataKeyName = s"$KERBEROS_SECRET_LABEL_PREFIX-$currentTime-$renewalInterval" - val uniqueSecretName = s"$HADOOP_KERBEROS_SECRET_NAME.$currentTime" + val uniqueSecretName = + s"$kubernetesResourceNamePrefix-$HADOOP_KERBEROS_SECRET_NAME.$currentTime" val secretDT = new SecretBuilder() .withNewMetadata() .withName(uniqueSecretName) - .withLabels(Map("refresh-hadoop-tokens" -> "yes").asJava) + .withLabels(Map(KERBEROS_REFRESH_LABEL_KEY -> KERBEROS_REFRESH_LABEL_VALUE).asJava) .endMetadata() .addToData(initialTokenDataKeyName, Base64.encodeBase64String(data)) .build() diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopStepsOrchestrator.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopStepsOrchestrator.scala index 8f052662ba600..0bc24026f34ae 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopStepsOrchestrator.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopStepsOrchestrator.scala @@ -27,10 +27,11 @@ import org.apache.spark.internal.Logging * Returns the complete ordered list of steps required to configure the hadoop configurations. */ private[spark] class HadoopStepsOrchestrator( + kubernetesResourceNamePrefix: String, namespace: String, hadoopConfigMapName: String, submissionSparkConf: SparkConf, - hadoopConfDir: String) extends Logging{ + hadoopConfDir: String) extends Logging { private val isKerberosEnabled = submissionSparkConf.get(KUBERNETES_KERBEROS_SUPPORT) private val maybePrincipal = submissionSparkConf.get(KUBERNETES_KERBEROS_PRINCIPAL) private val maybeKeytab = submissionSparkConf.get(KUBERNETES_KERBEROS_KEYTAB) @@ -81,6 +82,7 @@ private[spark] class HadoopStepsOrchestrator( existingSecretName, maybeExistingSecretItemKey.get))).getOrElse(Some( new HadoopKerberosKeytabResolverStep( + kubernetesResourceNamePrefix, submissionSparkConf, maybePrincipal, maybeKeytab, From 765455da297ee50dd8ebad5e16806ae782fa548f Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Sun, 5 Nov 2017 21:42:44 -0500 Subject: [PATCH 07/14] third round of PR comments --- .../org/apache/spark/deploy/k8s/HadoopConfBootstrap.scala | 2 +- .../scala/org/apache/spark/deploy/k8s/HadoopUGIUtil.scala | 4 +++- .../apache/spark/deploy/k8s/KerberosTokenConfBootstrap.scala | 2 +- .../k8s/submit/submitsteps/HadoopConfigBootstrapStep.scala | 4 ++-- 4 files changed, 7 insertions(+), 5 deletions(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/HadoopConfBootstrap.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/HadoopConfBootstrap.scala index 3177d32a92e01..5ca8e31d86f71 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/HadoopConfBootstrap.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/HadoopConfBootstrap.scala @@ -42,7 +42,7 @@ private[spark] trait HadoopConfBootstrap { private[spark] class HadoopConfBootstrapImpl( hadoopConfConfigMapName: String, hadoopConfigFiles: Seq[File], - hadoopUGI: HadoopUGIUtil) extends HadoopConfBootstrap with Logging{ + hadoopUGI: HadoopUGIUtil) extends HadoopConfBootstrap with Logging { override def bootstrapMainContainerAndVolumes( originalPodWithMainContainer: PodWithMainContainer) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/HadoopUGIUtil.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/HadoopUGIUtil.scala index c5e2c8fcb2f27..30473c25cd025 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/HadoopUGIUtil.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/HadoopUGIUtil.scala @@ -74,7 +74,9 @@ private[spark] class HadoopUGIUtil{ @deprecated("Moved to core in 2.3", "2.3") def deserialize(tokenBytes: Array[Byte]): Credentials = { val creds = new Credentials() - creds.readTokenStorageStream(new DataInputStream(new ByteArrayInputStream(tokenBytes))) + val dataStream = new DataInputStream(new ByteArrayInputStream(tokenBytes)) + creds.readTokenStorageStream(dataStream) + dataStream.close() creds } } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KerberosTokenConfBootstrap.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KerberosTokenConfBootstrap.scala index 5002673a3d962..6414a912725ee 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KerberosTokenConfBootstrap.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KerberosTokenConfBootstrap.scala @@ -37,7 +37,7 @@ private[spark] trait KerberosTokenConfBootstrap { private[spark] class KerberosTokenConfBootstrapImpl( secretName: String, secretItemKey: String, - userName: String) extends KerberosTokenConfBootstrap with Logging{ + userName: String) extends KerberosTokenConfBootstrap with Logging { override def bootstrapMainContainerAndVolumes( originalPodWithMainContainer: PodWithMainContainer) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/HadoopConfigBootstrapStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/HadoopConfigBootstrapStep.scala index 66c513be985b7..b20b637414cc2 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/HadoopConfigBootstrapStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/HadoopConfigBootstrapStep.scala @@ -53,13 +53,13 @@ private[spark] class HadoopConfigBootstrapStep( .endMetadata() .addToData(currentHadoopSpec.configMapProperties.asJava) .build() - val executorSparkConf = driverSpec.driverSparkConf.clone() + val driverSparkConfWithExecutorSetup = driverSpec.driverSparkConf.clone() .set(HADOOP_CONFIG_MAP_SPARK_CONF_NAME, hadoopConfigMapName) .setAll(currentHadoopSpec.additionalDriverSparkConf) driverSpec.copy( driverPod = currentHadoopSpec.driverPod, driverContainer = currentHadoopSpec.driverContainer, - driverSparkConf = executorSparkConf, + driverSparkConf = driverSparkConfWithExecutorSetup, otherKubernetesResources = driverSpec.otherKubernetesResources ++ Seq(configMap) ++ currentHadoopSpec.dtSecret.toSeq From 488b37e51db2107b556e8bccabf7359f0584f1ae Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Sat, 25 Nov 2017 00:58:19 -0500 Subject: [PATCH 08/14] initial round of comments and initial unit tests for deploy --- .../deploy/k8s/HadoopConfBootstrap.scala | 5 +- .../spark/deploy/k8s/HadoopUGIUtil.scala | 46 +++-- .../k8s/KerberosTokenConfBootstrap.scala | 76 ++++---- .../spark/deploy/k8s/submit/Client.scala | 18 +- .../HadoopKerberosKeytabResolverStep.scala | 22 ++- .../hadoopsteps/HadoopStepsOrchestrator.scala | 18 +- .../cluster/k8s/ExecutorPodFactory.scala | 2 + .../k8s/KubernetesClusterManager.scala | 34 ++-- .../spark/deploy/k8s/submit/ClientSuite.scala | 8 +- .../k8s/submit/HadoopConfBootstrapSuite.scala | 90 +++++++++ .../KerberosTokenConfBootstrapSuite.scala | 60 ++++++ .../HadoopConfigBootstrapStepSuite.scala | 90 +++++++++ .../HadoopConfMounterStepSuite.scala | 102 ++++++++++ ...adoopKerberosKeytabResolverStepSuite.scala | 175 ++++++++++++++++++ ...adoopKerberosSecretResolverStepSuite.scala | 63 +++++++ .../HadoopStepsOrchestratorSuite.scala | 90 +++++++++ .../k8s/integrationtest/KubernetesSuite.scala | 5 +- 17 files changed, 800 insertions(+), 104 deletions(-) create mode 100644 resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/HadoopConfBootstrapSuite.scala create mode 100644 resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/KerberosTokenConfBootstrapSuite.scala create mode 100644 resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/HadoopConfigBootstrapStepSuite.scala create mode 100644 resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopConfMounterStepSuite.scala create mode 100644 resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopKerberosKeytabResolverStepSuite.scala create mode 100644 resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopKerberosSecretResolverStepSuite.scala create mode 100644 resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopStepsOrchestratorSuite.scala diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/HadoopConfBootstrap.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/HadoopConfBootstrap.scala index 5ca8e31d86f71..910e7fce05179 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/HadoopConfBootstrap.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/HadoopConfBootstrap.scala @@ -44,8 +44,7 @@ private[spark] class HadoopConfBootstrapImpl( hadoopConfigFiles: Seq[File], hadoopUGI: HadoopUGIUtil) extends HadoopConfBootstrap with Logging { - override def bootstrapMainContainerAndVolumes( - originalPodWithMainContainer: PodWithMainContainer) + override def bootstrapMainContainerAndVolumes(originalPodWithMainContainer: PodWithMainContainer) : PodWithMainContainer = { logInfo("HADOOP_CONF_DIR defined. Mounting Hadoop specific .xml files") val keyPaths = hadoopConfigFiles.map{ file => @@ -77,7 +76,7 @@ private[spark] class HadoopConfBootstrapImpl( .endEnv() .addNewEnv() .withName(ENV_SPARK_USER) - .withValue(hadoopUGI.getShortName) + .withValue(hadoopUGI.getShortUserName) .endEnv() .build() originalPodWithMainContainer.copy( diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/HadoopUGIUtil.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/HadoopUGIUtil.scala index 30473c25cd025..3c2b6f420725b 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/HadoopUGIUtil.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/HadoopUGIUtil.scala @@ -26,12 +26,29 @@ import org.apache.hadoop.security.{Credentials, UserGroupInformation} import org.apache.hadoop.security.token.{Token, TokenIdentifier} import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenIdentifier +import org.apache.spark.util.{Clock, SystemClock, Utils} -// Function of this class is merely for mocking reasons -private[spark] class HadoopUGIUtil{ - def getCurrentUser: UserGroupInformation = UserGroupInformation.getCurrentUser +private[spark] trait HadoopUGIUtil { + def getCurrentUser: UserGroupInformation + def getShortUserName: String + def isSecurityEnabled: Boolean + def loginUserFromKeytabAndReturnUGI(principal: String, keytab: String) : + UserGroupInformation + def dfsAddDelegationToken(hadoopConf: Configuration, renewer: String, creds: Credentials) : + Iterable[Token[_ <: TokenIdentifier]] + def getCurrentTime: Long + def getTokenRenewalInterval( + renewedTokens: Iterable[Token[_ <: TokenIdentifier]], + hadoopConf: Configuration) : Option[Long] + def serialize(creds: Credentials): Array[Byte] + def deserialize(tokenBytes: Array[Byte]): Credentials +} - def getShortName: String = getCurrentUser.getShortUserName +private[spark] class HadoopUGIUtilImpl extends HadoopUGIUtil { + + private val clock: Clock = new SystemClock() + def getCurrentUser: UserGroupInformation = UserGroupInformation.getCurrentUser + def getShortUserName : String = getCurrentUser.getShortUserName def isSecurityEnabled: Boolean = UserGroupInformation.isSecurityEnabled @@ -42,7 +59,7 @@ private[spark] class HadoopUGIUtil{ : Iterable[Token[_ <: TokenIdentifier]] = FileSystem.get(hadoopConf).addDelegationTokens(renewer, creds) - def getCurrentTime: Long = System.currentTimeMillis() + def getCurrentTime: Long = clock.getTimeMillis() // Functions that should be in Core with Rebase to 2.3 @deprecated("Moved to core in 2.3", "2.3") @@ -64,19 +81,22 @@ private[spark] class HadoopUGIUtil{ @deprecated("Moved to core in 2.3", "2.3") def serialize(creds: Credentials): Array[Byte] = { - val byteStream = new ByteArrayOutputStream - val dataStream = new DataOutputStream(byteStream) - creds.writeTokenStorageToStream(dataStream) - dataStream.close() - byteStream.toByteArray + Utils.tryWithResource(new ByteArrayOutputStream()) { byteStream => + Utils.tryWithResource(new DataOutputStream(byteStream)) { dataStream => + creds.writeTokenStorageToStream(dataStream) + } + byteStream.toByteArray + } } @deprecated("Moved to core in 2.3", "2.3") def deserialize(tokenBytes: Array[Byte]): Credentials = { val creds = new Credentials() - val dataStream = new DataInputStream(new ByteArrayInputStream(tokenBytes)) - creds.readTokenStorageStream(dataStream) - dataStream.close() + Utils.tryWithResource(new ByteArrayInputStream(tokenBytes)) { byteStream => + Utils.tryWithResource(new DataInputStream(byteStream)) { dataStream => + creds.readTokenStorageStream(dataStream) + } + } creds } } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KerberosTokenConfBootstrap.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KerberosTokenConfBootstrap.scala index 6414a912725ee..300483795ce47 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KerberosTokenConfBootstrap.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KerberosTokenConfBootstrap.scala @@ -29,48 +29,46 @@ import org.apache.spark.internal.Logging private[spark] trait KerberosTokenConfBootstrap { // Bootstraps a main container with the Secret mounted as volumes and an ENV variable // pointing to the mounted file containing the DT for Secure HDFS interaction - def bootstrapMainContainerAndVolumes( - originalPodWithMainContainer: PodWithMainContainer) - : PodWithMainContainer + def bootstrapMainContainerAndVolumes(originalPodWithMainContainer: PodWithMainContainer) + : PodWithMainContainer } private[spark] class KerberosTokenConfBootstrapImpl( - secretName: String, - secretItemKey: String, - userName: String) extends KerberosTokenConfBootstrap with Logging { + secretName: String, + secretItemKey: String, + userName: String) extends KerberosTokenConfBootstrap with Logging { - override def bootstrapMainContainerAndVolumes( - originalPodWithMainContainer: PodWithMainContainer) - : PodWithMainContainer = { - logInfo("Mounting HDFS DT from Secret for Secure HDFS") - val secretMountedPod = new PodBuilder(originalPodWithMainContainer.pod) - .editOrNewSpec() - .addNewVolume() + override def bootstrapMainContainerAndVolumes( + originalPodWithMainContainer: PodWithMainContainer) : PodWithMainContainer = { + logInfo("Mounting HDFS DT from Secret for Secure HDFS") + val secretMountedPod = new PodBuilder(originalPodWithMainContainer.pod) + .editOrNewSpec() + .addNewVolume() + .withName(SPARK_APP_HADOOP_SECRET_VOLUME_NAME) + .withNewSecret() + .withSecretName(secretName) + .endSecret() + .endVolume() + .endSpec() + .build() + // TODO: ENV_HADOOP_TOKEN_FILE_LOCATION should point to the latest token data item key. + val secretMountedContainer = new ContainerBuilder( + originalPodWithMainContainer.mainContainer) + .addNewVolumeMount() .withName(SPARK_APP_HADOOP_SECRET_VOLUME_NAME) - .withNewSecret() - .withSecretName(secretName) - .endSecret() - .endVolume() - .endSpec() - .build() - // TODO: ENV_HADOOP_TOKEN_FILE_LOCATION should point to the latest token data item key. - val secretMountedContainer = new ContainerBuilder( - originalPodWithMainContainer.mainContainer) - .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() - originalPodWithMainContainer.copy( - pod = secretMountedPod, - mainContainer = secretMountedContainer) - } + .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() + originalPodWithMainContainer.copy( + pod = secretMountedPod, + mainContainer = secretMountedContainer) + } } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/Client.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/Client.scala index a2b26b237ce01..35f78ddfa2c17 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/Client.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/Client.scala @@ -35,7 +35,8 @@ private[spark] case class ClientArguments( mainAppResource: MainAppResource, otherPyFiles: Seq[String], mainClass: String, - driverArgs: Array[String]) + driverArgs: Array[String], + hadoopConfDir: Option[String]) private[spark] object ClientArguments { def fromCommandLineArgs(args: Array[String]): ClientArguments = { @@ -67,7 +68,8 @@ private[spark] object ClientArguments { mainAppResource.get, otherPyFiles, mainClass.get, - driverArgs.toArray) + driverArgs.toArray, + sys.env.get("HADOOP_CONF_DIR")) } } @@ -82,6 +84,8 @@ private[spark] class Client( private val driverJavaOptions = submissionSparkConf.get( org.apache.spark.internal.config.DRIVER_JAVA_OPTIONS) private val isKerberosEnabled = submissionSparkConf.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 private val maybeSimpleAuthentication = if (isKerberosEnabled) Some(s"-D$HADOOP_SECURITY_AUTHENTICATION=simple") else None @@ -160,8 +164,7 @@ private[spark] class Client( private[spark] object Client { def run(sparkConf: SparkConf, - clientArguments: ClientArguments, - hadoopConfDir: Option[String]): Unit = { + clientArguments: ClientArguments): Unit = { val namespace = sparkConf.get(KUBERNETES_NAMESPACE) val kubernetesAppId = s"spark-${UUID.randomUUID().toString.replaceAll("-", "")}" val launchTime = System.currentTimeMillis() @@ -180,7 +183,7 @@ private[spark] object Client { clientArguments.mainClass, clientArguments.driverArgs, clientArguments.otherPyFiles, - hadoopConfDir, + clientArguments.hadoopConfDir, sparkConf) Utils.tryWithResource(SparkKubernetesClientFactory.createKubernetesClient( master, @@ -208,9 +211,6 @@ private[spark] object Client { def main(args: Array[String]): Unit = { val parsedArguments = ClientArguments.fromCommandLineArgs(args) val sparkConf = new SparkConf() - // hadoopConfDir is passed into Client#run() to allow for us to - // test this env variable within the integration test environment - val hadoopConfDir = sys.env.get("HADOOP_CONF_DIR") - run(sparkConf, parsedArguments, hadoopConfDir) + run(sparkConf, parsedArguments) } } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopKerberosKeytabResolverStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopKerberosKeytabResolverStep.scala index d0b3b172f508f..08d812cef9a90 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopKerberosKeytabResolverStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopKerberosKeytabResolverStep.scala @@ -28,6 +28,7 @@ import org.apache.hadoop.security.Credentials import org.apache.hadoop.security.token.{Token, TokenIdentifier} import org.apache.spark.SparkConf +import org.apache.spark.SparkException import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.deploy.k8s.{HadoopUGIUtil, KerberosTokenConfBootstrapImpl, PodWithMainContainer} import org.apache.spark.deploy.k8s.constants._ @@ -46,12 +47,13 @@ import org.apache.spark.internal.Logging * DriverSpec. */ private[spark] class HadoopKerberosKeytabResolverStep( - kubernetesResourceNamePrefix: String, - submissionSparkConf: SparkConf, - maybePrincipal: Option[String], - maybeKeytab: Option[File], - maybeRenewerPrincipal: Option[String], - hadoopUGI: HadoopUGIUtil) extends HadoopConfigurationStep with Logging { + kubernetesResourceNamePrefix: String, + submissionSparkConf: SparkConf, + maybePrincipal: Option[String], + maybeKeytab: Option[File], + maybeRenewerPrincipal: Option[String], + hadoopUGI: HadoopUGIUtil) extends HadoopConfigurationStep with Logging { + private var originalCredentials: Credentials = _ private var dfs : FileSystem = _ private var renewer: String = _ @@ -60,7 +62,8 @@ private[spark] class HadoopKerberosKeytabResolverStep( override def configureContainers(hadoopConfigSpec: HadoopConfigSpec): HadoopConfigSpec = { val hadoopConf = SparkHadoopUtil.get.newConfiguration(submissionSparkConf) - if (!hadoopUGI.isSecurityEnabled) logDebug("Hadoop not configured with Kerberos") + if (!hadoopUGI.isSecurityEnabled) { + throw new SparkException("Hadoop not configured with Kerberos") } val maybeJobUserUGI = for { principal <- maybePrincipal @@ -89,11 +92,12 @@ private[spark] class HadoopKerberosKeytabResolverStep( tokens = credentials.getAllTokens.asScala null }}) - if (tokens.isEmpty) logDebug("Did not obtain any Delegation Tokens") + // TODO: Figure out how to MOCK this properly so exception can be thrown + // if (tokens.isEmpty) throw new SparkException(s"${credentials.getAllTokens.asScala}") val data = hadoopUGI.serialize(credentials) val renewalInterval = hadoopUGI.getTokenRenewalInterval(tokens, hadoopConf).getOrElse(Long.MaxValue) - val currentTime: Long = hadoopUGI.getCurrentTime + val currentTime = hadoopUGI.getCurrentTime val initialTokenDataKeyName = s"$KERBEROS_SECRET_LABEL_PREFIX-$currentTime-$renewalInterval" val uniqueSecretName = s"$kubernetesResourceNamePrefix-$HADOOP_KERBEROS_SECRET_NAME.$currentTime" diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopStepsOrchestrator.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopStepsOrchestrator.scala index 0bc24026f34ae..941865f8e7dec 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopStepsOrchestrator.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopStepsOrchestrator.scala @@ -19,7 +19,7 @@ package org.apache.spark.deploy.k8s.submit.submitsteps.hadoopsteps import java.io.File import org.apache.spark.SparkConf -import org.apache.spark.deploy.k8s.{HadoopConfBootstrapImpl, HadoopUGIUtil, OptionRequirements} +import org.apache.spark.deploy.k8s.{HadoopConfBootstrapImpl, HadoopUGIUtilImpl, OptionRequirements} import org.apache.spark.deploy.k8s.config._ import org.apache.spark.internal.Logging @@ -27,11 +27,12 @@ import org.apache.spark.internal.Logging * Returns the complete ordered list of steps required to configure the hadoop configurations. */ private[spark] class HadoopStepsOrchestrator( - kubernetesResourceNamePrefix: String, - namespace: String, - hadoopConfigMapName: String, - submissionSparkConf: SparkConf, - hadoopConfDir: String) extends Logging { + kubernetesResourceNamePrefix: String, + namespace: String, + hadoopConfigMapName: String, + submissionSparkConf: SparkConf, + hadoopConfDir: String) extends Logging { + private val isKerberosEnabled = submissionSparkConf.get(KUBERNETES_KERBEROS_SUPPORT) private val maybePrincipal = submissionSparkConf.get(KUBERNETES_KERBEROS_PRINCIPAL) private val maybeKeytab = submissionSparkConf.get(KUBERNETES_KERBEROS_KEYTAB) @@ -42,7 +43,7 @@ private[spark] class HadoopStepsOrchestrator( private val maybeRenewerPrincipal = submissionSparkConf.get(KUBERNETES_KERBEROS_RENEWER_PRINCIPAL) private val hadoopConfigurationFiles = getHadoopConfFiles(hadoopConfDir) - private val hadoopUGI = new HadoopUGIUtil + private val hadoopUGI = new HadoopUGIUtilImpl logInfo(s"Hadoop Conf directory: $hadoopConfDir") require(maybeKeytab.forall( _ => isKerberosEnabled ), @@ -95,10 +96,9 @@ private[spark] class HadoopStepsOrchestrator( } private def getHadoopConfFiles(path: String) : Seq[File] = { - def isFile(file: File) = if (file.isFile) Some(file) else None val dir = new File(path) if (dir.isDirectory) { - dir.listFiles.flatMap { file => isFile(file) }.toSeq + dir.listFiles.flatMap { file => Some(file).filter(_.isFile) }.toSeq } else { Seq.empty[File] } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala index 16ed1eead7167..edff83e8c8190 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala @@ -58,6 +58,8 @@ private[spark] class ExecutorPodFactoryImpl( private val executorJarsDownloadDir = sparkConf.get(INIT_CONTAINER_JARS_DOWNLOAD_LOCATION) private val isKerberosEnabled = 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 private val maybeSimpleAuthentication = if (isKerberosEnabled) Some(s"-D$HADOOP_SECURITY_AUTHENTICATION=simple") else None private val executorLabels = ConfigurationUtils.parsePrefixedKeyValuePairs( diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala index d624564924cd6..b98ed375fa744 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala @@ -21,7 +21,7 @@ import java.io.File import io.fabric8.kubernetes.client.Config import org.apache.spark.SparkContext -import org.apache.spark.deploy.k8s.{ConfigurationUtils, HadoopConfBootstrapImpl, HadoopUGIUtil, InitContainerResourceStagingServerSecretPluginImpl, KerberosTokenConfBootstrapImpl, SparkKubernetesClientFactory, SparkPodInitContainerBootstrapImpl} +import org.apache.spark.deploy.k8s.{ConfigurationUtils, HadoopConfBootstrapImpl, HadoopUGIUtilImpl, InitContainerResourceStagingServerSecretPluginImpl, KerberosTokenConfBootstrapImpl, SparkKubernetesClientFactory, SparkPodInitContainerBootstrapImpl} import org.apache.spark.deploy.k8s.config._ import org.apache.spark.deploy.k8s.constants._ import org.apache.spark.deploy.k8s.submit.{MountSecretsBootstrapImpl, MountSmallFilesBootstrapImpl} @@ -85,27 +85,26 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit configMapKey) } - val hadoopBootStrap = for { - hadoopConfigMap <- maybeHadoopConfigMap - } yield { - val hadoopUtil = new HadoopUGIUtil + val hadoopBootStrap = maybeHadoopConfigMap.map{ hadoopConfigMap => + val hadoopUtil = new HadoopUGIUtilImpl val hadoopConfigurations = maybeHadoopConfDir.map( conf_dir => getHadoopConfFiles(conf_dir)).getOrElse(Array.empty[File]) new HadoopConfBootstrapImpl( hadoopConfigMap, hadoopConfigurations, - hadoopUtil - ) - } - val kerberosBootstrap = for { - secretName <- maybeDTSecretName - secretItemKey <- maybeDTDataItem - } yield { - new KerberosTokenConfBootstrapImpl( - secretName, - secretItemKey, - Utils.getCurrentUserName) + hadoopUtil) } + val kerberosBootstrap = + maybeHadoopConfigMap.flatMap { _ => + for { + secretName <- maybeDTSecretName + secretItemKey <- maybeDTDataItem + } yield { + new KerberosTokenConfBootstrapImpl( + secretName, + secretItemKey, + Utils.getCurrentUserName() ) } + } val mountSmallFilesBootstrap = for { secretName <- maybeSubmittedFilesSecret secretMountPath <- maybeSubmittedFilesSecretMountPath @@ -184,10 +183,9 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit scheduler.asInstanceOf[TaskSchedulerImpl].initialize(backend) } private def getHadoopConfFiles(path: String) : Array[File] = { - def isFile(file: File) = if (file.isFile) Some(file) else None val dir = new File(path) if (dir.isDirectory) { - dir.listFiles.flatMap { file => isFile(file) } + dir.listFiles.flatMap { file => Some(file).filter(_.isFile) } } else { Array.empty[File] } 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 041f51e912002..8a7cc54e7672a 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 @@ -29,6 +29,7 @@ import org.scalatest.mock.MockitoSugar._ import scala.collection.JavaConverters._ import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.deploy.k8s.config._ import org.apache.spark.deploy.k8s.constants._ import org.apache.spark.deploy.k8s.submit.submitsteps.{DriverConfigurationStep, KubernetesDriverSpec} @@ -136,6 +137,7 @@ class ClientSuite extends SparkFunSuite with BeforeAndAfter { .set( org.apache.spark.internal.config.DRIVER_JAVA_OPTIONS, "-XX:+HeapDumpOnOutOfMemoryError -XX:+PrintGCDetails") + .set(KUBERNETES_KERBEROS_SUPPORT, true) val submissionClient = new Client( submissionSteps, sparkConf, @@ -150,14 +152,16 @@ class ClientSuite extends SparkFunSuite with BeforeAndAfter { val driverJvmOptsEnvs = driverContainer.getEnv.asScala.filter { env => env.getName.startsWith(ENV_JAVA_OPT_PREFIX) }.sortBy(_.getName) - assert(driverJvmOptsEnvs.size === 4) + assert(driverJvmOptsEnvs.size === 6) val expectedJvmOptsValues = Seq( + "-Dspark.kubernetes.kerberos.enabled=true", "-Dspark.logConf=true", s"-D${SecondTestConfigurationStep.sparkConfKey}=" + s"${SecondTestConfigurationStep.sparkConfValue}", s"-XX:+HeapDumpOnOutOfMemoryError", - s"-XX:+PrintGCDetails") + s"-XX:+PrintGCDetails", + "-Dspark.hadoop.hadoop.security.authentication=simple") driverJvmOptsEnvs.zip(expectedJvmOptsValues).zipWithIndex.foreach { case ((resolvedEnv, expectedJvmOpt), index) => assert(resolvedEnv.getName === s"$ENV_JAVA_OPT_PREFIX$index") diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/HadoopConfBootstrapSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/HadoopConfBootstrapSuite.scala new file mode 100644 index 0000000000000..8ace2fe143f39 --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/HadoopConfBootstrapSuite.scala @@ -0,0 +1,90 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.k8s.submit + +import java.io.File +import java.util.UUID + +import scala.collection.JavaConverters._ + +import com.google.common.io.Files +import io.fabric8.kubernetes.api.model._ +import org.mockito.{Mock, MockitoAnnotations} +import org.mockito.Mockito.when +import org.scalatest.BeforeAndAfter + +import org.apache.spark.SparkFunSuite +import org.apache.spark.deploy.k8s.{HadoopConfBootstrapImpl, HadoopUGIUtilImpl, PodWithMainContainer} +import org.apache.spark.deploy.k8s.constants._ +import org.apache.spark.util.Utils + +private[spark] class HadoopConfBootstrapSuite extends SparkFunSuite with BeforeAndAfter{ + private val CONFIG_MAP_NAME = "config-map" + private val TEMP_HADOOP_FILE = createTempFile("core-site.xml") + private val HADOOP_FILES = Seq(TEMP_HADOOP_FILE) + private val SPARK_USER_VALUE = "sparkUser" + + @Mock + private var hadoopUtil: HadoopUGIUtilImpl = _ + + before { + MockitoAnnotations.initMocks(this) + when(hadoopUtil.getShortUserName).thenReturn(SPARK_USER_VALUE) + } + + test("Test of bootstrapping hadoop_conf_dir files") { + val hadoopConfStep = new HadoopConfBootstrapImpl( + CONFIG_MAP_NAME, + HADOOP_FILES, + hadoopUtil) + val expectedKeyPaths = Seq( + new KeyToPathBuilder() + .withKey(TEMP_HADOOP_FILE.toPath.getFileName.toString) + .withPath(TEMP_HADOOP_FILE.toPath.getFileName.toString) + .build()) + val expectedPod = new PodBuilder() + .editOrNewSpec() + .addNewVolume() + .withName(HADOOP_FILE_VOLUME) + .withNewConfigMap() + .withName(CONFIG_MAP_NAME) + .withItems(expectedKeyPaths.asJava) + .endConfigMap() + .endVolume() + .endSpec() + .build() + + val podWithMain = PodWithMainContainer( + new PodBuilder().withNewSpec().endSpec().build(), + new Container()) + val returnedPodContainer = hadoopConfStep.bootstrapMainContainerAndVolumes(podWithMain) + assert(expectedPod === returnedPodContainer.pod) + assert(returnedPodContainer.mainContainer.getVolumeMounts.asScala.map(vm => + (vm.getName, vm.getMountPath)).head === (HADOOP_FILE_VOLUME, HADOOP_CONF_DIR_PATH)) + assert(returnedPodContainer.mainContainer.getEnv.asScala.head === + new EnvVarBuilder().withName(ENV_HADOOP_CONF_DIR).withValue(HADOOP_CONF_DIR_PATH).build()) + assert(returnedPodContainer.mainContainer.getEnv.asScala(1) === + new EnvVarBuilder().withName(ENV_SPARK_USER).withValue(SPARK_USER_VALUE).build()) + } + + private def createTempFile(contents: String): File = { + val dir = Utils.createTempDir() + val file = new File(dir, s"${UUID.randomUUID().toString}") + Files.write(contents.getBytes, file) + file + } +} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/KerberosTokenConfBootstrapSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/KerberosTokenConfBootstrapSuite.scala new file mode 100644 index 0000000000000..dae097a22d3fb --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/KerberosTokenConfBootstrapSuite.scala @@ -0,0 +1,60 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.k8s.submit + +import scala.collection.JavaConverters._ + +import io.fabric8.kubernetes.api.model._ + +import org.apache.spark.SparkFunSuite +import org.apache.spark.deploy.k8s.{KerberosTokenConfBootstrapImpl, PodWithMainContainer} +import org.apache.spark.deploy.k8s.constants._ + + +private[spark] class KerberosTokenConfBootstrapSuite extends SparkFunSuite { + private val SECRET_NAME = "dtSecret" + private val SECRET_LABEL = "dtLabel" + private val TEST_SPARK_USER = "hdfs" + + test("Test of bootstrapping kerberos secrets and env") { + val kerberosConfStep = new KerberosTokenConfBootstrapImpl( + SECRET_NAME, + SECRET_LABEL, + TEST_SPARK_USER) + val expectedPod = new PodBuilder() + .editOrNewSpec() + .addNewVolume() + .withName(SPARK_APP_HADOOP_SECRET_VOLUME_NAME) + .withNewSecret() + .withSecretName(SECRET_NAME) + .endSecret() + .endVolume() + .endSpec() + .build() + val podWithMain = PodWithMainContainer( + new PodBuilder().withNewSpec().endSpec().build(), + new Container()) + val returnedPodContainer = kerberosConfStep.bootstrapMainContainerAndVolumes(podWithMain) + assert(expectedPod === returnedPodContainer.pod) + assert(returnedPodContainer.mainContainer.getVolumeMounts.asScala.map(vm => + (vm.getName, vm.getMountPath)).head === + (SPARK_APP_HADOOP_SECRET_VOLUME_NAME, SPARK_APP_HADOOP_CREDENTIALS_BASE_DIR)) + assert(returnedPodContainer.mainContainer.getEnv.asScala.head.getName === + ENV_HADOOP_TOKEN_FILE_LOCATION) + assert(returnedPodContainer.mainContainer.getEnv.asScala(1).getName === ENV_SPARK_USER) + } +} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/HadoopConfigBootstrapStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/HadoopConfigBootstrapStepSuite.scala new file mode 100644 index 0000000000000..4ae9fcae81edf --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/HadoopConfigBootstrapStepSuite.scala @@ -0,0 +1,90 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.k8s.submit.submitsteps + +import scala.collection.JavaConverters._ + +import io.fabric8.kubernetes.api.model._ +import org.mockito.{Mock, MockitoAnnotations} +import org.mockito.Matchers.any +import org.mockito.Mockito.when +import org.scalatest.BeforeAndAfter + +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.deploy.k8s.constants._ +import org.apache.spark.deploy.k8s.submit.submitsteps.hadoopsteps.{HadoopConfigSpec, HadoopConfigurationStep} + + +private[spark] class HadoopConfigBootstrapStepSuite extends SparkFunSuite with BeforeAndAfter{ + private val CONFIG_MAP_NAME = "config-map" + private val POD_LABEL = Map("bootstrap" -> "true") + private val DRIVER_CONTAINER_NAME = "driver-container" + private val EXPECTED_SECRET = new SecretBuilder() + .withNewMetadata() + .withName(HADOOP_KERBEROS_SECRET_NAME) + .endMetadata() + .addToData("data", "secretdata") + .build() + + @Mock + private var hadoopConfigStep : HadoopConfigurationStep = _ + + before { + MockitoAnnotations.initMocks(this) + when(hadoopConfigStep.configureContainers(any[HadoopConfigSpec])).thenReturn( + HadoopConfigSpec( + configMapProperties = Map("data" -> "dataBytesToString"), + driverPod = new PodBuilder() + .withNewMetadata() + .addToLabels("bootstrap", "true") + .endMetadata() + .withNewSpec().endSpec() + .build(), + driverContainer = new ContainerBuilder().withName(DRIVER_CONTAINER_NAME).build(), + additionalDriverSparkConf = Map("sparkConf" -> "confValue"), + dtSecret = + Some(EXPECTED_SECRET), + dtSecretName = HADOOP_KERBEROS_SECRET_NAME, + dtSecretItemKey = "")) + } + + test("Test modification of driverSpec with Hadoop Steps") { + val hadoopConfStep = new HadoopConfigBootstrapStep( + Seq(hadoopConfigStep), + CONFIG_MAP_NAME) + val expectedDriverSparkConf = new SparkConf(true) + .set(HADOOP_CONFIG_MAP_SPARK_CONF_NAME, CONFIG_MAP_NAME) + .set("sparkConf", "confValue") + val expectedConfigMap = new ConfigMapBuilder() + .withNewMetadata() + .withName(CONFIG_MAP_NAME) + .endMetadata() + .addToData(Map("data" -> "dataBytesToString").asJava) + .build() + val expectedResources = Seq(expectedConfigMap, EXPECTED_SECRET) + val driverSpec = KubernetesDriverSpec( + driverPod = new Pod(), + driverContainer = new Container(), + driverSparkConf = new SparkConf(true), + otherKubernetesResources = Seq.empty[HasMetadata]) + val returnContainerSpec = hadoopConfStep.configureDriver(driverSpec) + assert(expectedDriverSparkConf.getAll === returnContainerSpec.driverSparkConf.getAll) + assert(returnContainerSpec.driverContainer.getName == DRIVER_CONTAINER_NAME) + assert(returnContainerSpec.driverPod.getMetadata.getLabels.asScala === POD_LABEL) + assert(returnContainerSpec.otherKubernetesResources === expectedResources) + } +} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopConfMounterStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopConfMounterStepSuite.scala new file mode 100644 index 0000000000000..f6b7a2af7152c --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopConfMounterStepSuite.scala @@ -0,0 +1,102 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.k8s.submit.submitsteps.hadoopsteps + +import java.io.File +import java.util.UUID + +import scala.collection.JavaConverters._ + +import com.google.common.io.Files +import io.fabric8.kubernetes.api.model._ +import org.apache.commons.io.FileUtils.readFileToString +import org.mockito.{Mock, MockitoAnnotations} +import org.mockito.Matchers.any +import org.mockito.Mockito.when +import org.mockito.invocation.InvocationOnMock +import org.mockito.stubbing.Answer +import org.scalatest.BeforeAndAfter + +import org.apache.spark.SparkFunSuite +import org.apache.spark.deploy.k8s.{HadoopConfBootstrap, PodWithMainContainer} +import org.apache.spark.deploy.k8s.constants.HADOOP_CONF_DIR_LOC +import org.apache.spark.util.Utils + + +private[spark] class HadoopConfMounterStepSuite extends SparkFunSuite with BeforeAndAfter{ + private val CONFIG_MAP_NAME = "config-map" + private val HADOOP_CONF_DIR_VAL = "/etc/hadoop" + private val POD_LABEL = Map("bootstrap" -> "true") + private val DRIVER_CONTAINER_NAME = "driver-container" + private val TEMP_HADOOP_FILE = createTempFile("core-site.xml") + private val HADOOP_FILES = Seq(TEMP_HADOOP_FILE) + + @Mock + private var hadoopConfBootstrap : HadoopConfBootstrap = _ + + before { + MockitoAnnotations.initMocks(this) + when(hadoopConfBootstrap.bootstrapMainContainerAndVolumes( + any[PodWithMainContainer])).thenAnswer(new Answer[PodWithMainContainer] { + override def answer(invocation: InvocationOnMock) : PodWithMainContainer = { + val pod = invocation.getArgumentAt(0, classOf[PodWithMainContainer]) + pod.copy( + pod = + new PodBuilder(pod.pod) + .withNewMetadata() + .addToLabels("bootstrap", "true") + .endMetadata() + .withNewSpec().endSpec() + .build(), + mainContainer = + new ContainerBuilder() + .withName(DRIVER_CONTAINER_NAME).build() + )}}) + } + + test("Test of mounting hadoop_conf_dir files into HadoopConfigSpec") { + val hadoopConfStep = new HadoopConfMounterStep( + CONFIG_MAP_NAME, + HADOOP_FILES, + hadoopConfBootstrap, + HADOOP_CONF_DIR_VAL) + val expectedDriverSparkConf = Map(HADOOP_CONF_DIR_LOC -> HADOOP_CONF_DIR_VAL) + val expectedConfigMap = Map( + TEMP_HADOOP_FILE.toPath.getFileName.toString -> + readFileToString(TEMP_HADOOP_FILE) + ) + val hadoopConfSpec = HadoopConfigSpec( + Map.empty[String, String], + new Pod(), + new Container(), + Map.empty[String, String], + None, + "", + "") + val returnContainerSpec = hadoopConfStep.configureContainers(hadoopConfSpec) + assert(expectedDriverSparkConf === returnContainerSpec.additionalDriverSparkConf) + assert(returnContainerSpec.driverContainer.getName == DRIVER_CONTAINER_NAME) + assert(returnContainerSpec.driverPod.getMetadata.getLabels.asScala === POD_LABEL) + assert(returnContainerSpec.configMapProperties === expectedConfigMap) + } + private def createTempFile(contents: String): File = { + val dir = Utils.createTempDir() + val file = new File(dir, s"${UUID.randomUUID().toString}") + Files.write(contents.getBytes, file) + file + } +} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopKerberosKeytabResolverStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopKerberosKeytabResolverStepSuite.scala new file mode 100644 index 0000000000000..fa1070e840319 --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopKerberosKeytabResolverStepSuite.scala @@ -0,0 +1,175 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.k8s.submit.submitsteps.hadoopsteps + +import java.io.File +import java.util.UUID + +import scala.collection.JavaConverters._ + +import com.google.common.io.Files +import io.fabric8.kubernetes.api.model._ +import org.apache.commons.codec.binary.Base64 +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.security.{Credentials, UserGroupInformation} +import org.apache.hadoop.security.token.{Token, TokenIdentifier} +import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenIdentifier +import org.mockito.{Mock, MockitoAnnotations} +import org.mockito.Matchers.any +import org.mockito.Mockito.when +import org.mockito.invocation.InvocationOnMock +import org.mockito.stubbing.Answer +import org.scalatest.BeforeAndAfter + +import org.apache.spark.{SparkConf, SparkException, SparkFunSuite} +import org.apache.spark.deploy.k8s.HadoopUGIUtilImpl +import org.apache.spark.deploy.k8s.constants._ +import org.apache.spark.util.Utils + + + +private[spark] class HadoopKerberosKeytabResolverStepSuite + extends SparkFunSuite with BeforeAndAfter{ + private val POD_LABEL = Map("bootstrap" -> "true") + private val DRIVER_CONTAINER_NAME = "driver-container" + private val TEMP_KEYTAB_FILE = createTempFile("keytab") + private val KERB_PRINCIPAL = "user@k8s.com" + private val SPARK_USER_VALUE = "sparkUser" + private val TEST_TOKEN_VALUE = "data" + private def getByteArray(input: String) = input.toCharArray.map(_.toByte) + private val TEST_DATA = getByteArray(TEST_TOKEN_VALUE) + private val OUTPUT_TEST_DATA = Base64.encodeBase64String(TEST_DATA) + private val INTERVAL = 500L + private val CURR_TIME = System.currentTimeMillis() + private val KUBE_TEST_NAME = "spark-testing" + private val DATA_KEY_NAME = + s"$KERBEROS_SECRET_LABEL_PREFIX-$CURR_TIME-$INTERVAL" + private val SECRET_NAME = + s"$KUBE_TEST_NAME-$HADOOP_KERBEROS_SECRET_NAME.$CURR_TIME" + + private val hadoopUGI = new HadoopUGIUtilImpl + + @Mock + private var hadoopUtil: HadoopUGIUtilImpl = _ + + @Mock + private var ugi: UserGroupInformation = _ + + @Mock + private var creds: Credentials = _ + + @Mock + private var token: Token[AbstractDelegationTokenIdentifier] = _ + + @Mock + private var identifier: AbstractDelegationTokenIdentifier = _ + + before { + MockitoAnnotations.initMocks(this) + when(hadoopUtil.loginUserFromKeytabAndReturnUGI(any[String], any[String])) + .thenAnswer(new Answer[UserGroupInformation] { + override def answer(invocation: InvocationOnMock): UserGroupInformation = { + hadoopUGI.getCurrentUser + } + }) + when(hadoopUtil.getCurrentUser).thenReturn(ugi) + when(hadoopUtil.getShortUserName).thenReturn(SPARK_USER_VALUE) + when(hadoopUtil.dfsAddDelegationToken(any(), any(), any())).thenReturn(null) + when(ugi.getCredentials).thenReturn(creds) + val tokens = List[Token[_ <: TokenIdentifier]](token).asJavaCollection + when(creds.getAllTokens).thenReturn(tokens) + when(hadoopUtil.serialize(any[Credentials])) + .thenReturn(TEST_DATA) + when(token.decodeIdentifier()).thenReturn(identifier) + when(hadoopUtil.getCurrentTime).thenReturn(CURR_TIME) + when(hadoopUtil.getTokenRenewalInterval(any[Iterable[Token[_ <: TokenIdentifier]]], + any[Configuration])).thenReturn(Some(INTERVAL)) + } + + test("Testing Error Catching for Security Enabling") { + when(hadoopUtil.isSecurityEnabled).thenReturn(false) + val keytabStep = new HadoopKerberosKeytabResolverStep( + KUBE_TEST_NAME, + new SparkConf(), + Some(KERB_PRINCIPAL), + Some(TEMP_KEYTAB_FILE), + None, + hadoopUtil) + val hadoopConfSpec = HadoopConfigSpec( + Map.empty[String, String], + new PodBuilder() + .withNewMetadata() + .addToLabels("bootstrap", "true") + .endMetadata() + .withNewSpec().endSpec() + .build(), + new ContainerBuilder().withName(DRIVER_CONTAINER_NAME).build(), + Map.empty[String, String], + None, + "", + "") + withClue("Security was not enabled as true for Kerberos conf") { + intercept[SparkException]{keytabStep.configureContainers(hadoopConfSpec)} + } + } + test("Testing keytab login") { + when(hadoopUtil.isSecurityEnabled).thenReturn(true) + val keytabStep = new HadoopKerberosKeytabResolverStep( + KUBE_TEST_NAME, + new SparkConf(), + Some(KERB_PRINCIPAL), + Some(TEMP_KEYTAB_FILE), + None, + hadoopUtil) + val hadoopConfSpec = HadoopConfigSpec( + Map.empty[String, String], + new PodBuilder() + .withNewMetadata() + .addToLabels("bootstrap", "true") + .endMetadata() + .withNewSpec().endSpec() + .build(), + new ContainerBuilder().withName(DRIVER_CONTAINER_NAME).build(), + Map.empty[String, String], + None, + "", + "") + val returnContainerSpec = keytabStep.configureContainers(hadoopConfSpec) + assert(returnContainerSpec.additionalDriverSparkConf(HADOOP_KERBEROS_CONF_ITEM_KEY) + .contains(KERBEROS_SECRET_LABEL_PREFIX)) + assert(returnContainerSpec.additionalDriverSparkConf === + Map(HADOOP_KERBEROS_CONF_ITEM_KEY -> DATA_KEY_NAME, + HADOOP_KERBEROS_CONF_SECRET -> SECRET_NAME)) + assert(returnContainerSpec.driverContainer.getName == DRIVER_CONTAINER_NAME) + assert(returnContainerSpec.driverPod.getMetadata.getLabels.asScala === POD_LABEL) + assert(returnContainerSpec.dtSecretItemKey === DATA_KEY_NAME) + assert(returnContainerSpec.dtSecret.get.getData.asScala === Map( + DATA_KEY_NAME -> OUTPUT_TEST_DATA)) + assert(returnContainerSpec.dtSecretName === SECRET_NAME) + assert(returnContainerSpec.dtSecret.get.getMetadata.getLabels.asScala === + Map("refresh-hadoop-tokens" -> "yes")) + assert(returnContainerSpec.dtSecret.nonEmpty) + assert(returnContainerSpec.dtSecret.get.getMetadata.getName === SECRET_NAME) + } + + private def createTempFile(contents: String): File = { + val dir = Utils.createTempDir() + val file = new File(dir, s"${UUID.randomUUID().toString}") + Files.write(contents.getBytes, file) + file + } +} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopKerberosSecretResolverStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopKerberosSecretResolverStepSuite.scala new file mode 100644 index 0000000000000..b1074d1933afa --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopKerberosSecretResolverStepSuite.scala @@ -0,0 +1,63 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.k8s.submit.submitsteps.hadoopsteps + +import scala.collection.JavaConverters._ + +import io.fabric8.kubernetes.api.model._ + +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.deploy.k8s.constants._ + +private[spark] class HadoopKerberosSecretResolverStepSuite extends SparkFunSuite { + private val CONFIG_MAP_NAME = "config-map" + private val HADOOP_CONF_DIR_VAL = "/etc/hadoop" + private val POD_LABEL = Map("bootstrap" -> "true") + private val DRIVER_CONTAINER_NAME = "driver-container" + private val TOKEN_SECRET_NAME = "secretName" + private val TOKEN_SECRET_DATA_ITEM_KEY = "secretItemKey" + + test("Testing kerberos with Secret") { + val keytabStep = new HadoopKerberosSecretResolverStep( + new SparkConf(), + TOKEN_SECRET_NAME, + TOKEN_SECRET_DATA_ITEM_KEY) + val expectedDriverSparkConf = Map( + HADOOP_KERBEROS_CONF_ITEM_KEY -> TOKEN_SECRET_DATA_ITEM_KEY, + HADOOP_KERBEROS_CONF_SECRET -> TOKEN_SECRET_NAME) + val hadoopConfSpec = HadoopConfigSpec( + Map.empty[String, String], + new PodBuilder() + .withNewMetadata() + .addToLabels("bootstrap", "true") + .endMetadata() + .withNewSpec().endSpec() + .build(), + new ContainerBuilder().withName(DRIVER_CONTAINER_NAME).build(), + Map.empty[String, String], + None, + "", + "") + val returnContainerSpec = keytabStep.configureContainers(hadoopConfSpec) + assert(expectedDriverSparkConf === returnContainerSpec.additionalDriverSparkConf) + assert(returnContainerSpec.driverContainer.getName == DRIVER_CONTAINER_NAME) + assert(returnContainerSpec.driverPod.getMetadata.getLabels.asScala === POD_LABEL) + assert(returnContainerSpec.dtSecret === None) + assert(returnContainerSpec.dtSecretItemKey === TOKEN_SECRET_DATA_ITEM_KEY) + assert(returnContainerSpec.dtSecretName === TOKEN_SECRET_NAME) + } +} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopStepsOrchestratorSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopStepsOrchestratorSuite.scala new file mode 100644 index 0000000000000..fa21c2633750d --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopStepsOrchestratorSuite.scala @@ -0,0 +1,90 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.k8s.submit.submitsteps.hadoopsteps + +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.deploy.k8s.config._ + +private[spark] class HadoopStepsOrchestratorSuite extends SparkFunSuite { + private val NAMESPACE = "testNamespace" + private val HADOOP_CONFIG_MAP = "hadoop-config-map" + private val HADOOP_CONF_DIR_VAL = "/etc/hadoop/conf" + private val KUBE_PREFIX = "spark-test" + + test("Testing without Kerberos") { + val sparkTestConf = new SparkConf(true) + .set(KUBERNETES_KERBEROS_SUPPORT, false) + val hadoopOrchestrator = new HadoopStepsOrchestrator( + KUBE_PREFIX, + NAMESPACE, + HADOOP_CONFIG_MAP, + sparkTestConf, + HADOOP_CONF_DIR_VAL) + val steps = hadoopOrchestrator.getHadoopSteps() + assert(steps.length === 1) + assert(steps.head.isInstanceOf[HadoopConfMounterStep]) + } + + test("Testing with Keytab Kerberos Login") { + val sparkTestConf = new SparkConf(true) + .set(KUBERNETES_KERBEROS_SUPPORT, true) + .set(KUBERNETES_KERBEROS_KEYTAB, "keytab.file") + .set(KUBERNETES_KERBEROS_PRINCIPAL, "user@kerberos") + val hadoopOrchestrator = new HadoopStepsOrchestrator( + KUBE_PREFIX, + NAMESPACE, + HADOOP_CONFIG_MAP, + sparkTestConf, + HADOOP_CONF_DIR_VAL) + val steps = hadoopOrchestrator.getHadoopSteps() + assert(steps.length === 2) + assert(steps.head.isInstanceOf[HadoopConfMounterStep]) + assert(steps(1).isInstanceOf[HadoopKerberosKeytabResolverStep]) + } + + test("Testing with kinit Kerberos Login") { + val sparkTestConf = new SparkConf(true) + .set(KUBERNETES_KERBEROS_SUPPORT, true) + val hadoopOrchestrator = new HadoopStepsOrchestrator( + KUBE_PREFIX, + NAMESPACE, + HADOOP_CONFIG_MAP, + sparkTestConf, + HADOOP_CONF_DIR_VAL) + val steps = hadoopOrchestrator.getHadoopSteps() + assert(steps.length === 2) + assert(steps.head.isInstanceOf[HadoopConfMounterStep]) + assert(steps(1).isInstanceOf[HadoopKerberosKeytabResolverStep]) + } + + test("Testing with Secret stored Kerberos") { + val sparkTestConf = new SparkConf(true) + .set(KUBERNETES_KERBEROS_SUPPORT, true) + .set(KUBERNETES_KERBEROS_DT_SECRET_NAME, "dtSecret") + .set(KUBERNETES_KERBEROS_DT_SECRET_ITEM_KEY, "dtItemKey") + val hadoopOrchestrator = new HadoopStepsOrchestrator( + KUBE_PREFIX, + NAMESPACE, + HADOOP_CONFIG_MAP, + sparkTestConf, + HADOOP_CONF_DIR_VAL) + val steps = hadoopOrchestrator.getHadoopSteps() + assert(steps.length === 2) + assert(steps.head.isInstanceOf[HadoopConfMounterStep]) + assert(steps(1).isInstanceOf[HadoopKerberosSecretResolverStep]) + } +} diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala index f761d547eb513..50ee46c93b592 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala @@ -366,8 +366,9 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { mainAppResource = appResource, mainClass = mainClass, driverArgs = appArgs, - otherPyFiles = otherPyFiles) - Client.run(sparkConf, clientArguments, None) + otherPyFiles = otherPyFiles, + hadoopConfDir = None) + Client.run(sparkConf, clientArguments) val driverPod = kubernetesTestComponents.kubernetesClient .pods() .withLabel("spark-app-locator", APP_LOCATOR_LABEL) From 37feb22b506c32a206b9d939024a78d99b2308e9 Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Sun, 26 Nov 2017 16:10:32 -0500 Subject: [PATCH 09/14] handled most of the comments and added test cases for pods --- .../deploy/k8s/HadoopConfBootstrap.scala | 4 - .../k8s/HadoopConfSparkUserBootstrap.scala | 46 +++++ .../hadoopsteps/HadoopConfSparkUserStep.scala | 36 ++++ .../hadoopsteps/HadoopStepsOrchestrator.scala | 3 +- .../cluster/k8s/ExecutorPodFactory.scala | 17 +- .../k8s/KubernetesClusterManager.scala | 14 +- .../k8s/submit/HadoopConfBootstrapSuite.scala | 2 - .../HadoopConfSparkUserBootstrapSuite.scala | 53 +++++ .../HadoopConfMounterStepSuite.scala | 3 +- .../HadoopConfSparkUserStepSuite.scala | 74 +++++++ .../HadoopStepsOrchestratorSuite.scala | 3 +- .../cluster/k8s/ExecutorPodFactorySuite.scala | 181 +++++++++++++++++- 12 files changed, 415 insertions(+), 21 deletions(-) create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/HadoopConfSparkUserBootstrap.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopConfSparkUserStep.scala create mode 100644 resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/HadoopConfSparkUserBootstrapSuite.scala create mode 100644 resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopConfSparkUserStepSuite.scala diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/HadoopConfBootstrap.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/HadoopConfBootstrap.scala index 910e7fce05179..23d1b5eb0541f 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/HadoopConfBootstrap.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/HadoopConfBootstrap.scala @@ -74,10 +74,6 @@ private[spark] class HadoopConfBootstrapImpl( .withName(ENV_HADOOP_CONF_DIR) .withValue(HADOOP_CONF_DIR_PATH) .endEnv() - .addNewEnv() - .withName(ENV_SPARK_USER) - .withValue(hadoopUGI.getShortUserName) - .endEnv() .build() originalPodWithMainContainer.copy( pod = hadoopSupportedPod, diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/HadoopConfSparkUserBootstrap.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/HadoopConfSparkUserBootstrap.scala new file mode 100644 index 0000000000000..6ddabbbf1e0c8 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/HadoopConfSparkUserBootstrap.scala @@ -0,0 +1,46 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.k8s + +import io.fabric8.kubernetes.api.model.ContainerBuilder + +import org.apache.spark.deploy.k8s.constants._ + +// This trait 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] trait HadoopConfSparkUserBootstrap { + def bootstrapMainContainerAndVolumes(originalPodWithMainContainer: PodWithMainContainer) + : PodWithMainContainer +} + +private[spark] class HadoopConfSparkUserBootstrapImpl(hadoopUGIUtil: HadoopUGIUtil) + extends HadoopConfSparkUserBootstrap { + + override def bootstrapMainContainerAndVolumes(originalPodWithMainContainer: PodWithMainContainer) + : PodWithMainContainer = { + val envModifiedContainer = new ContainerBuilder( + originalPodWithMainContainer.mainContainer) + .addNewEnv() + .withName(ENV_SPARK_USER) + .withValue(hadoopUGIUtil.getShortUserName) + .endEnv() + .build() + originalPodWithMainContainer.copy( + pod = originalPodWithMainContainer.pod, + mainContainer = envModifiedContainer) + } +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopConfSparkUserStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopConfSparkUserStep.scala new file mode 100644 index 0000000000000..43562dcc9f340 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopConfSparkUserStep.scala @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.k8s.submit.submitsteps.hadoopsteps + +import org.apache.spark.deploy.k8s.{HadoopConfSparkUserBootstrap, PodWithMainContainer} + +private[spark] class HadoopConfSparkUserStep(hadoopUserBootstrapConf: HadoopConfSparkUserBootstrap) + extends HadoopConfigurationStep { + + override def configureContainers(hadoopConfigSpec: HadoopConfigSpec): HadoopConfigSpec = { + val bootstrappedPodAndMainContainer = + hadoopUserBootstrapConf.bootstrapMainContainerAndVolumes( + PodWithMainContainer( + hadoopConfigSpec.driverPod, + hadoopConfigSpec.driverContainer)) + hadoopConfigSpec.copy( + driverPod = bootstrappedPodAndMainContainer.pod, + driverContainer = bootstrappedPodAndMainContainer.mainContainer, + configMapProperties = hadoopConfigSpec.configMapProperties, + additionalDriverSparkConf = hadoopConfigSpec.additionalDriverSparkConf) + } +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopStepsOrchestrator.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopStepsOrchestrator.scala index 941865f8e7dec..06de3fc6c74b9 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopStepsOrchestrator.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopStepsOrchestrator.scala @@ -20,6 +20,7 @@ import java.io.File import org.apache.spark.SparkConf import org.apache.spark.deploy.k8s.{HadoopConfBootstrapImpl, HadoopUGIUtilImpl, OptionRequirements} +import org.apache.spark.deploy.k8s.HadoopConfSparkUserBootstrapImpl import org.apache.spark.deploy.k8s.config._ import org.apache.spark.internal.Logging @@ -90,7 +91,7 @@ private[spark] class HadoopStepsOrchestrator( maybeRenewerPrincipal, hadoopUGI))) } else { - Option.empty[HadoopConfigurationStep] + Some(new HadoopConfSparkUserStep(new HadoopConfSparkUserBootstrapImpl(hadoopUGI))) } Seq(hadoopConfMounterStep) ++ maybeKerberosStep.toSeq } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala index edff83e8c8190..248efd4d53e57 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala @@ -21,7 +21,7 @@ import scala.collection.JavaConverters._ import io.fabric8.kubernetes.api.model.{ContainerBuilder, ContainerPortBuilder, EnvVar, EnvVarBuilder, EnvVarSourceBuilder, Pod, PodBuilder, QuantityBuilder, VolumeBuilder, VolumeMountBuilder} import org.apache.spark.{SparkConf, SparkException} -import org.apache.spark.deploy.k8s.{ConfigurationUtils, HadoopConfBootstrap, InitContainerResourceStagingServerSecretPlugin, KerberosTokenConfBootstrap, PodWithDetachedInitContainer, PodWithMainContainer, SparkPodInitContainerBootstrap} +import org.apache.spark.deploy.k8s.{ConfigurationUtils, HadoopConfBootstrap, HadoopConfSparkUserBootstrap, InitContainerResourceStagingServerSecretPlugin, KerberosTokenConfBootstrap, PodWithDetachedInitContainer, PodWithMainContainer, SparkPodInitContainerBootstrap} import org.apache.spark.deploy.k8s.config._ import org.apache.spark.deploy.k8s.constants._ import org.apache.spark.deploy.k8s.submit.{InitContainerUtil, MountSecretsBootstrap, MountSmallFilesBootstrap} @@ -48,7 +48,8 @@ private[spark] class ExecutorPodFactoryImpl( executorMountInitContainerSecretPlugin: Option[InitContainerResourceStagingServerSecretPlugin], executorLocalDirVolumeProvider: ExecutorLocalDirVolumeProvider, hadoopBootStrap: Option[HadoopConfBootstrap], - kerberosBootstrap: Option[KerberosTokenConfBootstrap]) + kerberosBootstrap: Option[KerberosTokenConfBootstrap], + hadoopUserBootstrap: Option[HadoopConfSparkUserBootstrap]) extends ExecutorPodFactory { import ExecutorPodFactoryImpl._ @@ -282,9 +283,17 @@ private[spark] class ExecutorPodFactoryImpl( PodWithMainContainer(executorHadoopConfPod, executorHadoopConfContainer)) (podWithMainContainer.pod, podWithMainContainer.mainContainer) }.getOrElse((executorHadoopConfPod, executorHadoopConfContainer)) - new PodBuilder(executorKerberosPod) + + val (executorSparkUserPod, executorSparkUserContainer) = + hadoopUserBootstrap.map { bootstrap => + val podWithMainContainer = bootstrap.bootstrapMainContainerAndVolumes( + PodWithMainContainer(executorKerberosPod, executorKerberosContainer)) + (podWithMainContainer.pod, podWithMainContainer.mainContainer) + }.getOrElse((executorKerberosPod, executorKerberosContainer)) + + new PodBuilder(executorSparkUserPod) .editSpec() - .addToContainers(executorKerberosContainer) + .addToContainers(executorSparkUserContainer) .endSpec() .build() } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala index b98ed375fa744..59365947c57bd 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala @@ -21,7 +21,7 @@ import java.io.File import io.fabric8.kubernetes.client.Config import org.apache.spark.SparkContext -import org.apache.spark.deploy.k8s.{ConfigurationUtils, HadoopConfBootstrapImpl, HadoopUGIUtilImpl, InitContainerResourceStagingServerSecretPluginImpl, KerberosTokenConfBootstrapImpl, SparkKubernetesClientFactory, SparkPodInitContainerBootstrapImpl} +import org.apache.spark.deploy.k8s.{ConfigurationUtils, HadoopConfBootstrapImpl, HadoopConfSparkUserBootstrapImpl, HadoopUGIUtilImpl, InitContainerResourceStagingServerSecretPluginImpl, KerberosTokenConfBootstrapImpl, SparkKubernetesClientFactory, SparkPodInitContainerBootstrapImpl} import org.apache.spark.deploy.k8s.config._ import org.apache.spark.deploy.k8s.constants._ import org.apache.spark.deploy.k8s.submit.{MountSecretsBootstrapImpl, MountSmallFilesBootstrapImpl} @@ -84,9 +84,8 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit configMap, configMapKey) } - + val hadoopUtil = new HadoopUGIUtilImpl val hadoopBootStrap = maybeHadoopConfigMap.map{ hadoopConfigMap => - val hadoopUtil = new HadoopUGIUtilImpl val hadoopConfigurations = maybeHadoopConfDir.map( conf_dir => getHadoopConfFiles(conf_dir)).getOrElse(Array.empty[File]) new HadoopConfBootstrapImpl( @@ -105,6 +104,12 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit secretItemKey, Utils.getCurrentUserName() ) } } + val hadoopUserBootstrap = + if (hadoopBootStrap.isDefined && kerberosBootstrap.isEmpty) { + Some(new HadoopConfSparkUserBootstrapImpl(hadoopUtil)) + } else { + None + } val mountSmallFilesBootstrap = for { secretName <- maybeSubmittedFilesSecret secretMountPath <- maybeSubmittedFilesSecretMountPath @@ -164,7 +169,8 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit executorInitContainerSecretVolumePlugin, executorLocalDirVolumeProvider, hadoopBootStrap, - kerberosBootstrap) + kerberosBootstrap, + hadoopUserBootstrap) val allocatorExecutor = ThreadUtils .newDaemonSingleThreadScheduledExecutor("kubernetes-pod-allocator") val requestExecutorsService = ThreadUtils.newDaemonCachedThreadPool( diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/HadoopConfBootstrapSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/HadoopConfBootstrapSuite.scala index 8ace2fe143f39..03ada4090a56a 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/HadoopConfBootstrapSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/HadoopConfBootstrapSuite.scala @@ -77,8 +77,6 @@ private[spark] class HadoopConfBootstrapSuite extends SparkFunSuite with BeforeA (vm.getName, vm.getMountPath)).head === (HADOOP_FILE_VOLUME, HADOOP_CONF_DIR_PATH)) assert(returnedPodContainer.mainContainer.getEnv.asScala.head === new EnvVarBuilder().withName(ENV_HADOOP_CONF_DIR).withValue(HADOOP_CONF_DIR_PATH).build()) - assert(returnedPodContainer.mainContainer.getEnv.asScala(1) === - new EnvVarBuilder().withName(ENV_SPARK_USER).withValue(SPARK_USER_VALUE).build()) } private def createTempFile(contents: String): File = { diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/HadoopConfSparkUserBootstrapSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/HadoopConfSparkUserBootstrapSuite.scala new file mode 100644 index 0000000000000..3c9b25208d5eb --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/HadoopConfSparkUserBootstrapSuite.scala @@ -0,0 +1,53 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.k8s.submit + +import scala.collection.JavaConverters._ + +import io.fabric8.kubernetes.api.model._ +import org.mockito.{Mock, MockitoAnnotations} +import org.mockito.Mockito.when +import org.scalatest.BeforeAndAfter + +import org.apache.spark.SparkFunSuite +import org.apache.spark.deploy.k8s.{HadoopUGIUtilImpl, PodWithMainContainer} +import org.apache.spark.deploy.k8s.HadoopConfSparkUserBootstrapImpl +import org.apache.spark.deploy.k8s.constants._ + +private[spark] class HadoopConfSparkUserBootstrapSuite extends SparkFunSuite with BeforeAndAfter{ + private val SPARK_USER_VALUE = "sparkUser" + + @Mock + private var hadoopUtil: HadoopUGIUtilImpl = _ + + before { + MockitoAnnotations.initMocks(this) + when(hadoopUtil.getShortUserName).thenReturn(SPARK_USER_VALUE) + } + + test("Test of bootstrapping hadoop_conf_dir files") { + val hadoopConfStep = new HadoopConfSparkUserBootstrapImpl(hadoopUtil) + val emptyPod = new PodBuilder().withNewSpec().endSpec().build() + val podWithMain = PodWithMainContainer( + emptyPod, + new Container()) + val returnedPodContainer = hadoopConfStep.bootstrapMainContainerAndVolumes(podWithMain) + assert(emptyPod === returnedPodContainer.pod) + assert(returnedPodContainer.mainContainer.getEnv.asScala.head === + new EnvVarBuilder().withName(ENV_SPARK_USER).withValue(SPARK_USER_VALUE).build()) + } +} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopConfMounterStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopConfMounterStepSuite.scala index f6b7a2af7152c..a98bfd8640b96 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopConfMounterStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopConfMounterStepSuite.scala @@ -77,8 +77,7 @@ private[spark] class HadoopConfMounterStepSuite extends SparkFunSuite with Befor val expectedDriverSparkConf = Map(HADOOP_CONF_DIR_LOC -> HADOOP_CONF_DIR_VAL) val expectedConfigMap = Map( TEMP_HADOOP_FILE.toPath.getFileName.toString -> - readFileToString(TEMP_HADOOP_FILE) - ) + readFileToString(TEMP_HADOOP_FILE)) val hadoopConfSpec = HadoopConfigSpec( Map.empty[String, String], new Pod(), diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopConfSparkUserStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopConfSparkUserStepSuite.scala new file mode 100644 index 0000000000000..91b5c10376e73 --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopConfSparkUserStepSuite.scala @@ -0,0 +1,74 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.k8s.submit.submitsteps.hadoopsteps + +import scala.collection.JavaConverters._ + +import io.fabric8.kubernetes.api.model._ +import org.mockito.{Mock, MockitoAnnotations} +import org.mockito.Matchers.any +import org.mockito.Mockito.when +import org.mockito.invocation.InvocationOnMock +import org.mockito.stubbing.Answer +import org.scalatest.BeforeAndAfter + +import org.apache.spark.SparkFunSuite +import org.apache.spark.deploy.k8s.{HadoopConfSparkUserBootstrap, PodWithMainContainer} + + +private[spark] class HadoopConfSparkUserStepSuite extends SparkFunSuite with BeforeAndAfter{ + private val POD_LABEL = Map("bootstrap" -> "true") + private val DRIVER_CONTAINER_NAME = "driver-container" + + @Mock + private var hadoopConfSparkUserBootstrap : HadoopConfSparkUserBootstrap = _ + + before { + MockitoAnnotations.initMocks(this) + when(hadoopConfSparkUserBootstrap.bootstrapMainContainerAndVolumes( + any[PodWithMainContainer])).thenAnswer(new Answer[PodWithMainContainer] { + override def answer(invocation: InvocationOnMock) : PodWithMainContainer = { + val pod = invocation.getArgumentAt(0, classOf[PodWithMainContainer]) + pod.copy( + pod = + new PodBuilder(pod.pod) + .withNewMetadata() + .addToLabels("bootstrap", "true") + .endMetadata() + .withNewSpec().endSpec() + .build(), + mainContainer = + new ContainerBuilder() + .withName(DRIVER_CONTAINER_NAME).build() + )}}) + } + + test("Test of mounting hadoop_conf_dir files into HadoopConfigSpec") { + val hadoopSparkUserStep = new HadoopConfSparkUserStep(hadoopConfSparkUserBootstrap) + val hadoopConfSpec = HadoopConfigSpec( + Map.empty[String, String], + new Pod(), + new Container(), + Map.empty[String, String], + None, + "", + "") + val returnContainerSpec = hadoopSparkUserStep.configureContainers(hadoopConfSpec) + assert(returnContainerSpec.driverContainer.getName == DRIVER_CONTAINER_NAME) + assert(returnContainerSpec.driverPod.getMetadata.getLabels.asScala === POD_LABEL) + } +} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopStepsOrchestratorSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopStepsOrchestratorSuite.scala index fa21c2633750d..b7701b12c5b0c 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopStepsOrchestratorSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopStepsOrchestratorSuite.scala @@ -35,8 +35,9 @@ private[spark] class HadoopStepsOrchestratorSuite extends SparkFunSuite { sparkTestConf, HADOOP_CONF_DIR_VAL) val steps = hadoopOrchestrator.getHadoopSteps() - assert(steps.length === 1) + assert(steps.length === 2) assert(steps.head.isInstanceOf[HadoopConfMounterStep]) + assert(steps(1).isInstanceOf[HadoopConfSparkUserStep]) } test("Testing with Keytab Kerberos Login") { diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactorySuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactorySuite.scala index b77cd2b655594..1488a0c81418d 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactorySuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactorySuite.scala @@ -16,19 +16,27 @@ */ package org.apache.spark.scheduler.cluster.k8s +import java.io.File +import java.util.UUID + +import scala.collection.JavaConverters._ + +import com.google.common.base.Charsets +import com.google.common.io.Files import io.fabric8.kubernetes.api.model.{Pod, PodBuilder, VolumeBuilder, VolumeMountBuilder} +import io.fabric8.kubernetes.api.model.KeyToPathBuilder import io.fabric8.kubernetes.client.KubernetesClient import org.mockito.{AdditionalAnswers, Mock, Mockito, MockitoAnnotations} import org.mockito.Matchers.any import org.mockito.Mockito._ import org.scalatest.{BeforeAndAfter, BeforeAndAfterEach} -import scala.collection.JavaConverters._ import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.deploy.k8s.{PodWithDetachedInitContainer, SparkPodInitContainerBootstrap} +import org.apache.spark.deploy.k8s.{HadoopConfBootstrapImpl, HadoopConfSparkUserBootstrapImpl, HadoopUGIUtilImpl, KerberosTokenConfBootstrapImpl, PodWithDetachedInitContainer, SparkPodInitContainerBootstrap} import org.apache.spark.deploy.k8s.config._ import org.apache.spark.deploy.k8s.constants._ import org.apache.spark.deploy.k8s.submit.{MountSecretsBootstrapImpl, MountSmallFilesBootstrap, MountSmallFilesBootstrapImpl} +import org.apache.spark.util.Utils class ExecutorPodFactorySuite extends SparkFunSuite with BeforeAndAfter with BeforeAndAfterEach { private val driverPodName: String = "driver-pod" @@ -56,6 +64,9 @@ class ExecutorPodFactorySuite extends SparkFunSuite with BeforeAndAfter with Bef @Mock private var executorLocalDirVolumeProvider: ExecutorLocalDirVolumeProvider = _ + @Mock + private var hadoopUGI: HadoopUGIUtilImpl = _ + before { MockitoAnnotations.initMocks(this) baseConf = new SparkConf() @@ -79,6 +90,7 @@ class ExecutorPodFactorySuite extends SparkFunSuite with BeforeAndAfter with Bef None, executorLocalDirVolumeProvider, None, + None, None) val executor = factory.createExecutorPod( "1", "dummy", "dummy", Seq[(String, String)](), driverPod, Map[String, Int]()) @@ -121,6 +133,7 @@ class ExecutorPodFactorySuite extends SparkFunSuite with BeforeAndAfter with Bef None, executorLocalDirVolumeProvider, None, + None, None) val executor = factory.createExecutorPod( "1", "dummy", "dummy", Seq[(String, String)](), driverPod, Map[String, Int]()) @@ -144,6 +157,7 @@ class ExecutorPodFactorySuite extends SparkFunSuite with BeforeAndAfter with Bef None, executorLocalDirVolumeProvider, None, + None, None) val executor = factory.createExecutorPod( "1", "dummy", "dummy", Seq[(String, String)](), driverPod, Map[String, Int]()) @@ -180,6 +194,7 @@ class ExecutorPodFactorySuite extends SparkFunSuite with BeforeAndAfter with Bef None, executorLocalDirVolumeProvider, None, + None, None) val executor = factory.createExecutorPod( "1", "dummy", "dummy", Seq[(String, String)](), driverPod, Map[String, Int]()) @@ -212,6 +227,7 @@ class ExecutorPodFactorySuite extends SparkFunSuite with BeforeAndAfter with Bef None, executorLocalDirVolumeProvider, None, + None, None) val executor = factory.createExecutorPod( "1", "dummy", "dummy", Seq[(String, String)](), driverPod, Map[String, Int]()) @@ -224,7 +240,6 @@ class ExecutorPodFactorySuite extends SparkFunSuite with BeforeAndAfter with Bef test("Small-files add a secret & secret volume mount to the container") { val conf = baseConf.clone() - val smallFiles = new MountSmallFilesBootstrapImpl("secret1", "/var/secret1") val factory = new ExecutorPodFactoryImpl( conf, @@ -235,6 +250,7 @@ class ExecutorPodFactorySuite extends SparkFunSuite with BeforeAndAfter with Bef None, executorLocalDirVolumeProvider, None, + None, None) val executor = factory.createExecutorPod( "1", "dummy", "dummy", Seq[(String, String)](), driverPod, Map[String, Int]()) @@ -270,6 +286,7 @@ class ExecutorPodFactorySuite extends SparkFunSuite with BeforeAndAfter with Bef None, executorLocalDirVolumeProvider, None, + None, None) val executor = factory.createExecutorPod( "1", "dummy", "dummy", Seq[(String, String)]("qux" -> "quux"), driverPod, Map[String, Int]()) @@ -284,6 +301,129 @@ class ExecutorPodFactorySuite extends SparkFunSuite with BeforeAndAfter with Bef checkOwnerReferences(executor, driverPodUid) } + test("check that hadoop bootstrap mounts files w/o SPARK_USER") { + when(hadoopUGI.getShortUserName).thenReturn("test-user") + val conf = baseConf.clone() + val configName = "hadoop-test" + val hadoopFile = createTempFile + val hadoopFiles = Seq(hadoopFile) + val hadoopBootsrap = new HadoopConfBootstrapImpl( + hadoopConfConfigMapName = configName, + hadoopConfigFiles = hadoopFiles, + hadoopUGI = hadoopUGI) + + val factory = new ExecutorPodFactoryImpl( + conf, + nodeAffinityExecutorPodModifier, + None, + None, + None, + None, + executorLocalDirVolumeProvider, + Some(hadoopBootsrap), + None, + None) + val executor = factory.createExecutorPod( + "1", "dummy", "dummy", Seq[(String, String)]("qux" -> "quux"), driverPod, Map[String, Int]()) + + checkEnv(executor, + Map(ENV_HADOOP_CONF_DIR -> HADOOP_CONF_DIR_PATH, + "qux" -> "quux")) + checkOwnerReferences(executor, driverPodUid) + checkConfigMapVolumes(executor, + HADOOP_FILE_VOLUME, + configName, + hadoopFile.toPath.getFileName.toString) + checkVolumeMounts(executor, HADOOP_FILE_VOLUME, HADOOP_CONF_DIR_PATH) + } + + test("check that hadoop bootstrap mounts files w/ SPARK_USER") { + when(hadoopUGI.getShortUserName).thenReturn("test-user") + val conf = baseConf.clone() + val configName = "hadoop-test" + val hadoopFile = createTempFile + val hadoopFiles = Seq(hadoopFile) + val hadoopBootstrap = new HadoopConfBootstrapImpl( + hadoopConfConfigMapName = configName, + hadoopConfigFiles = hadoopFiles, + hadoopUGI = hadoopUGI) + val hadoopUserBootstrap = new HadoopConfSparkUserBootstrapImpl(hadoopUGI) + + val factory = new ExecutorPodFactoryImpl( + conf, + nodeAffinityExecutorPodModifier, + None, + None, + None, + None, + executorLocalDirVolumeProvider, + Some(hadoopBootstrap), + None, + Some(hadoopUserBootstrap)) + val executor = factory.createExecutorPod( + "1", "dummy", "dummy", Seq[(String, String)]("qux" -> "quux"), driverPod, Map[String, Int]()) + + checkEnv(executor, + Map(ENV_SPARK_USER -> "test-user", + ENV_HADOOP_CONF_DIR -> HADOOP_CONF_DIR_PATH, + "qux" -> "quux")) + checkOwnerReferences(executor, driverPodUid) + checkConfigMapVolumes(executor, + HADOOP_FILE_VOLUME, + configName, + hadoopFile.toPath.getFileName.toString) + checkVolumeMounts(executor, HADOOP_FILE_VOLUME, HADOOP_CONF_DIR_PATH) + } + + test("check that hadoop and kerberos bootstrap function properly") { + when(hadoopUGI.getShortUserName).thenReturn("test-user") + val conf = baseConf.clone() + val configName = "hadoop-test" + val hadoopFile = createTempFile + val hadoopFiles = Seq(hadoopFile) + val hadoopBootstrap = new HadoopConfBootstrapImpl( + hadoopConfConfigMapName = configName, + hadoopConfigFiles = hadoopFiles, + hadoopUGI = hadoopUGI) + val secretName = "secret-test" + val secretItemKey = "item-test" + val userName = "sparkUser" + val kerberosBootstrap = new KerberosTokenConfBootstrapImpl( + secretName = secretName, + secretItemKey = secretItemKey, + userName = userName) + val factory = new ExecutorPodFactoryImpl( + conf, + nodeAffinityExecutorPodModifier, + None, + None, + None, + None, + executorLocalDirVolumeProvider, + Some(hadoopBootstrap), + Some(kerberosBootstrap), + None) + val executor = factory.createExecutorPod( + "1", "dummy", "dummy", Seq[(String, String)]("qux" -> "quux"), driverPod, Map[String, Int]()) + + checkEnv(executor, + Map(ENV_SPARK_USER -> userName, + ENV_HADOOP_CONF_DIR -> HADOOP_CONF_DIR_PATH, + ENV_HADOOP_TOKEN_FILE_LOCATION -> + s"$SPARK_APP_HADOOP_CREDENTIALS_BASE_DIR/$secretItemKey", + "qux" -> "quux")) + checkOwnerReferences(executor, driverPodUid) + checkConfigMapVolumes(executor, + HADOOP_FILE_VOLUME, + configName, + hadoopFile.toPath.getFileName.toString) + checkSecretVolumes(executor, SPARK_APP_HADOOP_SECRET_VOLUME_NAME, secretName) + checkVolumeMounts(executor, HADOOP_FILE_VOLUME, HADOOP_CONF_DIR_PATH) + checkVolumeMounts(executor, + SPARK_APP_HADOOP_SECRET_VOLUME_NAME, + SPARK_APP_HADOOP_CREDENTIALS_BASE_DIR) + } + // There is always exactly one controller reference, and it points to the driver pod. private def checkOwnerReferences(executor: Pod, driverPodUid: String): Unit = { assert(executor.getMetadata.getOwnerReferences.size() === 1) @@ -310,4 +450,39 @@ class ExecutorPodFactorySuite extends SparkFunSuite with BeforeAndAfter with Bef }.toMap assert(defaultEnvs === mapEnvs) } + + private def checkVolumeMounts(executor: Pod, name: String, mountPath: String) : Unit = { + assert(executor.getSpec.getContainers.size() === 1) + val volumeMount = executor.getSpec.getContainers + .get(0).getVolumeMounts.asScala.find(_.getName == name) + assert(volumeMount.nonEmpty) + assert(volumeMount.get.getMountPath == mountPath) + } + + private def checkConfigMapVolumes(executor: Pod, + volName: String, + configMapName: String, + content: String) : Unit = { + val volume = executor.getSpec.getVolumes.asScala.find(_.getName == volName) + assert(volume.nonEmpty) + assert(volume.get.getConfigMap.getName == configMapName) + assert(volume.get.getConfigMap.getItems.asScala.find(_.getKey == content).get == + new KeyToPathBuilder() + .withKey(content) + .withPath(content).build() ) + } + + private def checkSecretVolumes(executor: Pod, volName: String, secretName: String) : Unit = { + val volume = executor.getSpec.getVolumes.asScala.find(_.getName == volName) + assert(volume.nonEmpty) + assert(volume.get.getSecret.getSecretName == secretName) + } + + // Creates temp files for the purpose of testing file mounting + private def createTempFile: File = { + val dir = Utils.createTempDir() + val file = new File(dir, s"${UUID.randomUUID().toString}") + Files.write(UUID.randomUUID().toString, file, Charsets.UTF_8) + file + } } From 4e440273adfb7a0aab0159c658685d4c68a9e7fd Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Wed, 29 Nov 2017 11:03:25 -0500 Subject: [PATCH 10/14] resolve conflicts --- .../k8s/submit/DriverConfigurationStepsOrchestratorSuite.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestratorSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestratorSuite.scala index 64ab0b10f49ff..10de399f5f583 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestratorSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestratorSuite.scala @@ -131,7 +131,8 @@ private[spark] class DriverConfigurationStepsOrchestratorSuite extends SparkFunS classOf[RStep]) } - test("Only local files without a resource staging server.") { + + test("Only submitter local files without a resource staging server.") { val sparkConf = new SparkConf(false).set("spark.files", "/var/spark/file1.txt") val mainAppResource = JavaMainAppResource("local:///var/apps/jars/main.jar") val orchestrator = new DriverConfigurationStepsOrchestrator( From 64b0af74ec99de4702aa6b2c06c3b48493d0f9f3 Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Wed, 29 Nov 2017 12:33:25 -0500 Subject: [PATCH 11/14] merge conflicts --- .../k8s/submit/DriverConfigurationStepsOrchestratorSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestratorSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestratorSuite.scala index 45f900f189c95..605a9aa2def29 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestratorSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestratorSuite.scala @@ -131,7 +131,6 @@ private[spark] class DriverConfigurationStepsOrchestratorSuite extends SparkFunS classOf[RStep]) } - test("Only submitter local files without a resource staging server.") { val sparkConf = new SparkConf(false).set("spark.files", "/var/spark/file1.txt") val mainAppResource = JavaMainAppResource("local:///var/apps/jars/main.jar") @@ -169,6 +168,7 @@ private[spark] class DriverConfigurationStepsOrchestratorSuite extends SparkFunS MAIN_CLASS, APP_ARGS, Seq.empty[String], + None, sparkConf) validateStepTypes( orchestrator, From ba2bafcfdee79301ae3afc73c2ce5475528c6378 Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Wed, 29 Nov 2017 12:41:32 -0500 Subject: [PATCH 12/14] adding thread sleeping for RSS issues as a test --- .../spark/deploy/k8s/integrationtest/KubernetesSuite.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala index 50ee46c93b592..e56c5b90952d1 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala @@ -58,6 +58,7 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { override def afterAll(): Unit = { testBackend.cleanUp() + Thread.sleep(5000) } before { From 0c99503b95a5453dc14d47387dd5034f1c0175bb Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Wed, 29 Nov 2017 16:43:32 -0500 Subject: [PATCH 13/14] resolving comments and unit testing --- .../deploy/k8s/HadoopConfBootstrap.scala | 5 +- .../spark/deploy/k8s/HadoopUGIUtil.scala | 17 +- .../HadoopKerberosKeytabResolverStep.scala | 25 +- ...adoopKerberosKeytabResolverStepSuite.scala | 219 ++++++++++++++++-- .../k8s/integrationtest/KubernetesSuite.scala | 1 - 5 files changed, 225 insertions(+), 42 deletions(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/HadoopConfBootstrap.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/HadoopConfBootstrap.scala index 23d1b5eb0541f..62dd15ba9a819 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/HadoopConfBootstrap.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/HadoopConfBootstrap.scala @@ -34,9 +34,8 @@ private[spark] trait HadoopConfBootstrap { * Bootstraps a main container with the ConfigMaps containing Hadoop config files * mounted as volumes and an ENV variable pointing to the mounted file. */ - def bootstrapMainContainerAndVolumes( - originalPodWithMainContainer: PodWithMainContainer) - : PodWithMainContainer + def bootstrapMainContainerAndVolumes(originalPodWithMainContainer: PodWithMainContainer) + : PodWithMainContainer } private[spark] class HadoopConfBootstrapImpl( diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/HadoopUGIUtil.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/HadoopUGIUtil.scala index 3c2b6f420725b..4dcfe6642d919 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/HadoopUGIUtil.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/HadoopUGIUtil.scala @@ -31,11 +31,14 @@ import org.apache.spark.util.{Clock, SystemClock, Utils} private[spark] trait HadoopUGIUtil { def getCurrentUser: UserGroupInformation def getShortUserName: String + def getFileSystem(hadoopConf: Configuration): FileSystem def isSecurityEnabled: Boolean def loginUserFromKeytabAndReturnUGI(principal: String, keytab: String) : UserGroupInformation - def dfsAddDelegationToken(hadoopConf: Configuration, renewer: String, creds: Credentials) : - Iterable[Token[_ <: TokenIdentifier]] + def dfsAddDelegationToken(fileSystem: FileSystem, + hadoopConf: Configuration, + renewer: String, + creds: Credentials) : Iterable[Token[_ <: TokenIdentifier]] def getCurrentTime: Long def getTokenRenewalInterval( renewedTokens: Iterable[Token[_ <: TokenIdentifier]], @@ -49,15 +52,17 @@ private[spark] class HadoopUGIUtilImpl extends HadoopUGIUtil { 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 dfsAddDelegationToken(hadoopConf: Configuration, renewer: String, creds: Credentials) - : Iterable[Token[_ <: TokenIdentifier]] = - FileSystem.get(hadoopConf).addDelegationTokens(renewer, creds) + def dfsAddDelegationToken(fileSystem: FileSystem, + hadoopConf: Configuration, + renewer: String, + creds: Credentials) : Iterable[Token[_ <: TokenIdentifier]] = + fileSystem.addDelegationTokens(renewer, creds) def getCurrentTime: Long = clock.getTimeMillis() diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopKerberosKeytabResolverStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopKerberosKeytabResolverStep.scala index 08d812cef9a90..4c3184e7f550b 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopKerberosKeytabResolverStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopKerberosKeytabResolverStep.scala @@ -23,7 +23,6 @@ import scala.collection.JavaConverters._ import io.fabric8.kubernetes.api.model.SecretBuilder import org.apache.commons.codec.binary.Base64 -import org.apache.hadoop.fs.FileSystem import org.apache.hadoop.security.Credentials import org.apache.hadoop.security.token.{Token, TokenIdentifier} @@ -54,11 +53,7 @@ private[spark] class HadoopKerberosKeytabResolverStep( maybeRenewerPrincipal: Option[String], hadoopUGI: HadoopUGIUtil) extends HadoopConfigurationStep with Logging { - private var originalCredentials: Credentials = _ - private var dfs : FileSystem = _ - private var renewer: String = _ private var credentials: Credentials = _ - private var tokens: Iterable[Token[_ <: TokenIdentifier]] = _ override def configureContainers(hadoopConfigSpec: HadoopConfigSpec): HadoopConfigSpec = { val hadoopConf = SparkHadoopUtil.get.newConfiguration(submissionSparkConf) @@ -81,19 +76,21 @@ private[spark] class HadoopKerberosKeytabResolverStep( // In the case that keytab is not specified we will read from Local Ticket Cache val jobUserUGI = maybeJobUserUGI.getOrElse(hadoopUGI.getCurrentUser) // It is necessary to run as jobUserUGI because logged in user != Current User - jobUserUGI.doAs(new PrivilegedExceptionAction[Void] { - override def run(): Void = { - originalCredentials = jobUserUGI.getCredentials + val tokens = jobUserUGI.doAs( + new PrivilegedExceptionAction[Iterable[Token[_ <: TokenIdentifier]]] { + override def run(): Iterable[Token[_ <: TokenIdentifier]] = { + val originalCredentials = jobUserUGI.getCredentials // TODO: This is not necessary with [Spark-20328] since we would be using // Spark core providers to handle delegation token renewal - renewer = maybeRenewerPrincipal.getOrElse(jobUserUGI.getShortUserName) + val renewerPrincipal = maybeRenewerPrincipal.getOrElse(jobUserUGI.getShortUserName) credentials = new Credentials(originalCredentials) - hadoopUGI.dfsAddDelegationToken(hadoopConf, renewer, credentials) - tokens = credentials.getAllTokens.asScala - null + hadoopUGI.dfsAddDelegationToken(hadoopUGI.getFileSystem(hadoopConf), + hadoopConf, + renewerPrincipal, + credentials) + credentials.getAllTokens.asScala }}) - // TODO: Figure out how to MOCK this properly so exception can be thrown - // if (tokens.isEmpty) throw new SparkException(s"${credentials.getAllTokens.asScala}") + if (tokens.isEmpty) throw new SparkException(s"Did not obtain any delegation tokens") val data = hadoopUGI.serialize(credentials) val renewalInterval = hadoopUGI.getTokenRenewalInterval(tokens, hadoopConf).getOrElse(Long.MaxValue) diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopKerberosKeytabResolverStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopKerberosKeytabResolverStepSuite.scala index fa1070e840319..1a022f7f115b7 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopKerberosKeytabResolverStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopKerberosKeytabResolverStepSuite.scala @@ -17,6 +17,7 @@ package org.apache.spark.deploy.k8s.submit.submitsteps.hadoopsteps import java.io.File +import java.security.PrivilegedExceptionAction import java.util.UUID import scala.collection.JavaConverters._ @@ -25,12 +26,15 @@ import com.google.common.io.Files import io.fabric8.kubernetes.api.model._ import org.apache.commons.codec.binary.Base64 import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.FileSystem +import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier +import org.apache.hadoop.io.Text import org.apache.hadoop.security.{Credentials, UserGroupInformation} import org.apache.hadoop.security.token.{Token, TokenIdentifier} import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenIdentifier -import org.mockito.{Mock, MockitoAnnotations} -import org.mockito.Matchers.any -import org.mockito.Mockito.when +import org.mockito.{ArgumentCaptor, Mock, MockitoAnnotations} +import org.mockito.Matchers.{any, eq => mockitoEq} +import org.mockito.Mockito.{verify, when} import org.mockito.invocation.InvocationOnMock import org.mockito.stubbing.Answer import org.scalatest.BeforeAndAfter @@ -38,12 +42,12 @@ import org.scalatest.BeforeAndAfter import org.apache.spark.{SparkConf, SparkException, SparkFunSuite} import org.apache.spark.deploy.k8s.HadoopUGIUtilImpl import org.apache.spark.deploy.k8s.constants._ -import org.apache.spark.util.Utils - +import org.apache.spark.util.{Clock, SystemClock, Utils} private[spark] class HadoopKerberosKeytabResolverStepSuite extends SparkFunSuite with BeforeAndAfter{ + private val clock: Clock = new SystemClock() private val POD_LABEL = Map("bootstrap" -> "true") private val DRIVER_CONTAINER_NAME = "driver-container" private val TEMP_KEYTAB_FILE = createTempFile("keytab") @@ -53,8 +57,12 @@ private[spark] class HadoopKerberosKeytabResolverStepSuite private def getByteArray(input: String) = input.toCharArray.map(_.toByte) private val TEST_DATA = getByteArray(TEST_TOKEN_VALUE) private val OUTPUT_TEST_DATA = Base64.encodeBase64String(TEST_DATA) + private val TEST_TOKEN_SERVICE = new Text("hdfsService") + private val TEST_TOKEN = + new Token[DelegationTokenIdentifier](TEST_DATA, TEST_DATA, + DelegationTokenIdentifier.HDFS_DELEGATION_KIND, TEST_TOKEN_SERVICE) private val INTERVAL = 500L - private val CURR_TIME = System.currentTimeMillis() + private val CURR_TIME = clock.getTimeMillis() private val KUBE_TEST_NAME = "spark-testing" private val DATA_KEY_NAME = s"$KERBEROS_SECRET_LABEL_PREFIX-$CURR_TIME-$INTERVAL" @@ -64,13 +72,13 @@ private[spark] class HadoopKerberosKeytabResolverStepSuite private val hadoopUGI = new HadoopUGIUtilImpl @Mock - private var hadoopUtil: HadoopUGIUtilImpl = _ + private var fileSystem: FileSystem = _ @Mock - private var ugi: UserGroupInformation = _ + private var hadoopUtil: HadoopUGIUtilImpl = _ @Mock - private var creds: Credentials = _ + private var ugi: UserGroupInformation = _ @Mock private var token: Token[AbstractDelegationTokenIdentifier] = _ @@ -83,20 +91,19 @@ private[spark] class HadoopKerberosKeytabResolverStepSuite when(hadoopUtil.loginUserFromKeytabAndReturnUGI(any[String], any[String])) .thenAnswer(new Answer[UserGroupInformation] { override def answer(invocation: InvocationOnMock): UserGroupInformation = { - hadoopUGI.getCurrentUser + ugi } }) when(hadoopUtil.getCurrentUser).thenReturn(ugi) + when(ugi.getShortUserName).thenReturn(SPARK_USER_VALUE) when(hadoopUtil.getShortUserName).thenReturn(SPARK_USER_VALUE) - when(hadoopUtil.dfsAddDelegationToken(any(), any(), any())).thenReturn(null) - when(ugi.getCredentials).thenReturn(creds) - val tokens = List[Token[_ <: TokenIdentifier]](token).asJavaCollection - when(creds.getAllTokens).thenReturn(tokens) - when(hadoopUtil.serialize(any[Credentials])) + when(hadoopUtil.getFileSystem(any[Configuration])).thenReturn(fileSystem) + val tokens = Iterable[Token[_ <: TokenIdentifier]](token) + when(hadoopUtil.serialize(any())) .thenReturn(TEST_DATA) when(token.decodeIdentifier()).thenReturn(identifier) when(hadoopUtil.getCurrentTime).thenReturn(CURR_TIME) - when(hadoopUtil.getTokenRenewalInterval(any[Iterable[Token[_ <: TokenIdentifier]]], + when(hadoopUtil.getTokenRenewalInterval(mockitoEq(tokens), any[Configuration])).thenReturn(Some(INTERVAL)) } @@ -126,8 +133,46 @@ private[spark] class HadoopKerberosKeytabResolverStepSuite intercept[SparkException]{keytabStep.configureContainers(hadoopConfSpec)} } } - test("Testing keytab login") { + + test("Testing Error Catching for No Token catching") { + when(hadoopUtil.isSecurityEnabled).thenReturn(false) + when(ugi.doAs(any(classOf[PrivilegedExceptionAction[Iterable[Token[_ <: TokenIdentifier]]]]))) + .thenReturn(Iterable[Token[_ <: TokenIdentifier]]()) + val keytabStep = new HadoopKerberosKeytabResolverStep( + KUBE_TEST_NAME, + new SparkConf(), + Some(KERB_PRINCIPAL), + Some(TEMP_KEYTAB_FILE), + None, + hadoopUtil) + val hadoopConfSpec = HadoopConfigSpec( + Map.empty[String, String], + new PodBuilder() + .withNewMetadata() + .addToLabels("bootstrap", "true") + .endMetadata() + .withNewSpec().endSpec() + .build(), + new ContainerBuilder().withName(DRIVER_CONTAINER_NAME).build(), + Map.empty[String, String], + None, + "", + "") + withClue("Error Catching for No Token Catching") { + intercept[SparkException]{keytabStep.configureContainers(hadoopConfSpec)} + } + } + + test("Testing keytab login with Principal and Keytab") { when(hadoopUtil.isSecurityEnabled).thenReturn(true) + when(ugi.doAs(any(classOf[PrivilegedExceptionAction[Iterable[Token[_ <: TokenIdentifier]]]]))) + .thenReturn(Iterable[Token[_ <: TokenIdentifier]](token)) + val creds = new Credentials() + when(ugi.getCredentials).thenReturn(creds) + val actionCaptor: ArgumentCaptor[ + PrivilegedExceptionAction[Iterable[Token[_ <: TokenIdentifier]]]] = + ArgumentCaptor.forClass( + classOf[PrivilegedExceptionAction[Iterable[Token[_ <: TokenIdentifier]]]]) val keytabStep = new HadoopKerberosKeytabResolverStep( KUBE_TEST_NAME, new SparkConf(), @@ -149,6 +194,144 @@ private[spark] class HadoopKerberosKeytabResolverStepSuite "", "") val returnContainerSpec = keytabStep.configureContainers(hadoopConfSpec) + verify(ugi).doAs(actionCaptor.capture()) + val action = actionCaptor.getValue + when(hadoopUtil.dfsAddDelegationToken(mockitoEq(fileSystem), + any[Configuration], + mockitoEq(SPARK_USER_VALUE), + any())).thenAnswer(new Answer[Iterable[Token[_ <: TokenIdentifier]]] { + override def answer(invocation: InvocationOnMock) + : Iterable[Token[_ <: TokenIdentifier]] = { + creds.addToken(TEST_TOKEN_SERVICE, TEST_TOKEN) + Iterable[Token[_ <: TokenIdentifier]](TEST_TOKEN) + } + }) + // TODO: ACTION.run() is still not calling the above function + // assert(action.run() == Iterable[Token[_ <: TokenIdentifier]](TEST_TOKEN)) + assert(returnContainerSpec.additionalDriverSparkConf(HADOOP_KERBEROS_CONF_ITEM_KEY) + .contains(KERBEROS_SECRET_LABEL_PREFIX)) + assert(returnContainerSpec.additionalDriverSparkConf === + Map(HADOOP_KERBEROS_CONF_ITEM_KEY -> DATA_KEY_NAME, + HADOOP_KERBEROS_CONF_SECRET -> SECRET_NAME)) + assert(returnContainerSpec.driverContainer.getName == DRIVER_CONTAINER_NAME) + assert(returnContainerSpec.driverPod.getMetadata.getLabels.asScala === POD_LABEL) + assert(returnContainerSpec.dtSecretItemKey === DATA_KEY_NAME) + assert(returnContainerSpec.dtSecret.get.getData.asScala === Map( + DATA_KEY_NAME -> OUTPUT_TEST_DATA)) + assert(returnContainerSpec.dtSecretName === SECRET_NAME) + assert(returnContainerSpec.dtSecret.get.getMetadata.getLabels.asScala === + Map(KERBEROS_REFRESH_LABEL_KEY -> KERBEROS_REFRESH_LABEL_VALUE)) + assert(returnContainerSpec.dtSecret.nonEmpty) + assert(returnContainerSpec.dtSecret.get.getMetadata.getName === SECRET_NAME) + } + + test("Testing keytab login w/o Principal and Keytab") { + when(hadoopUtil.isSecurityEnabled).thenReturn(true) + when(ugi.doAs(any(classOf[PrivilegedExceptionAction[Iterable[Token[_ <: TokenIdentifier]]]]))) + .thenReturn(Iterable[Token[_ <: TokenIdentifier]](token)) + val creds = new Credentials() + when(ugi.getCredentials).thenReturn(creds) + val actionCaptor: ArgumentCaptor[ + PrivilegedExceptionAction[Iterable[Token[_ <: TokenIdentifier]]]] = + ArgumentCaptor.forClass( + classOf[PrivilegedExceptionAction[Iterable[Token[_ <: TokenIdentifier]]]]) + val keytabStep = new HadoopKerberosKeytabResolverStep( + KUBE_TEST_NAME, + new SparkConf(), + None, + None, + None, + hadoopUtil) + val hadoopConfSpec = HadoopConfigSpec( + Map.empty[String, String], + new PodBuilder() + .withNewMetadata() + .addToLabels("bootstrap", "true") + .endMetadata() + .withNewSpec().endSpec() + .build(), + new ContainerBuilder().withName(DRIVER_CONTAINER_NAME).build(), + Map.empty[String, String], + None, + "", + "") + val returnContainerSpec = keytabStep.configureContainers(hadoopConfSpec) + verify(ugi).doAs(actionCaptor.capture()) + val action = actionCaptor.getValue + when(hadoopUtil.dfsAddDelegationToken(mockitoEq(fileSystem), + any[Configuration], + mockitoEq(SPARK_USER_VALUE), + any())).thenAnswer(new Answer[Iterable[Token[_ <: TokenIdentifier]]] { + override def answer(invocation: InvocationOnMock) + : Iterable[Token[_ <: TokenIdentifier]] = { + creds.addToken(TEST_TOKEN_SERVICE, TEST_TOKEN) + Iterable[Token[_ <: TokenIdentifier]](TEST_TOKEN) + } + }) + // TODO: ACTION.run() is still not calling the above function + // assert(action.run() == Iterable[Token[_ <: TokenIdentifier]](TEST_TOKEN)) + assert(returnContainerSpec.additionalDriverSparkConf(HADOOP_KERBEROS_CONF_ITEM_KEY) + .contains(KERBEROS_SECRET_LABEL_PREFIX)) + assert(returnContainerSpec.additionalDriverSparkConf === + Map(HADOOP_KERBEROS_CONF_ITEM_KEY -> DATA_KEY_NAME, + HADOOP_KERBEROS_CONF_SECRET -> SECRET_NAME)) + assert(returnContainerSpec.driverContainer.getName == DRIVER_CONTAINER_NAME) + assert(returnContainerSpec.driverPod.getMetadata.getLabels.asScala === POD_LABEL) + assert(returnContainerSpec.dtSecretItemKey === DATA_KEY_NAME) + assert(returnContainerSpec.dtSecret.get.getData.asScala === Map( + DATA_KEY_NAME -> OUTPUT_TEST_DATA)) + assert(returnContainerSpec.dtSecretName === SECRET_NAME) + assert(returnContainerSpec.dtSecret.get.getMetadata.getLabels.asScala === + Map(KERBEROS_REFRESH_LABEL_KEY -> KERBEROS_REFRESH_LABEL_VALUE)) + assert(returnContainerSpec.dtSecret.nonEmpty) + assert(returnContainerSpec.dtSecret.get.getMetadata.getName === SECRET_NAME) + } + + test("Testing keytab login with Principal, Keytab, and Renewer Principle") { + when(hadoopUtil.isSecurityEnabled).thenReturn(true) + when(ugi.doAs(any(classOf[PrivilegedExceptionAction[Iterable[Token[_ <: TokenIdentifier]]]]))) + .thenReturn(Iterable[Token[_ <: TokenIdentifier]](token)) + val creds = new Credentials() + when(ugi.getCredentials).thenReturn(creds) + val actionCaptor: ArgumentCaptor[ + PrivilegedExceptionAction[Iterable[Token[_ <: TokenIdentifier]]]] = + ArgumentCaptor.forClass( + classOf[PrivilegedExceptionAction[Iterable[Token[_ <: TokenIdentifier]]]]) + val keytabStep = new HadoopKerberosKeytabResolverStep( + KUBE_TEST_NAME, + new SparkConf(), + Some(KERB_PRINCIPAL), + Some(TEMP_KEYTAB_FILE), + Some("SHORT_NAME"), + hadoopUtil) + val hadoopConfSpec = HadoopConfigSpec( + Map.empty[String, String], + new PodBuilder() + .withNewMetadata() + .addToLabels("bootstrap", "true") + .endMetadata() + .withNewSpec().endSpec() + .build(), + new ContainerBuilder().withName(DRIVER_CONTAINER_NAME).build(), + Map.empty[String, String], + None, + "", + "") + val returnContainerSpec = keytabStep.configureContainers(hadoopConfSpec) + verify(ugi).doAs(actionCaptor.capture()) + val action = actionCaptor.getValue + when(hadoopUtil.dfsAddDelegationToken(mockitoEq(fileSystem), + any[Configuration], + mockitoEq("SHORT_NAME"), + any())).thenAnswer(new Answer[Iterable[Token[_ <: TokenIdentifier]]] { + override def answer(invocation: InvocationOnMock) + : Iterable[Token[_ <: TokenIdentifier]] = { + creds.addToken(TEST_TOKEN_SERVICE, TEST_TOKEN) + Iterable[Token[_ <: TokenIdentifier]](TEST_TOKEN) + } + }) + // TODO: ACTION.run() is still not calling the above function + // assert(action.run() == Iterable[Token[_ <: TokenIdentifier]](TEST_TOKEN)) assert(returnContainerSpec.additionalDriverSparkConf(HADOOP_KERBEROS_CONF_ITEM_KEY) .contains(KERBEROS_SECRET_LABEL_PREFIX)) assert(returnContainerSpec.additionalDriverSparkConf === @@ -161,7 +344,7 @@ private[spark] class HadoopKerberosKeytabResolverStepSuite DATA_KEY_NAME -> OUTPUT_TEST_DATA)) assert(returnContainerSpec.dtSecretName === SECRET_NAME) assert(returnContainerSpec.dtSecret.get.getMetadata.getLabels.asScala === - Map("refresh-hadoop-tokens" -> "yes")) + Map(KERBEROS_REFRESH_LABEL_KEY -> KERBEROS_REFRESH_LABEL_VALUE)) assert(returnContainerSpec.dtSecret.nonEmpty) assert(returnContainerSpec.dtSecret.get.getMetadata.getName === SECRET_NAME) } diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala index e56c5b90952d1..50ee46c93b592 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala @@ -58,7 +58,6 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { override def afterAll(): Unit = { testBackend.cleanUp() - Thread.sleep(5000) } before { From a9d074b30a0459972916c00df1a7fcfb132d48b3 Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Sat, 2 Dec 2017 21:12:57 -0500 Subject: [PATCH 14/14] regarding comments on PR --- .../deploy/k8s/HadoopConfBootstrap.scala | 5 +++-- .../k8s/KerberosTokenConfBootstrap.scala | 2 +- .../org/apache/spark/deploy/k8s/config.scala | 20 +++++++++---------- .../apache/spark/deploy/k8s/constants.scala | 6 +++--- .../spark/deploy/k8s/submit/Client.scala | 5 ++--- .../HadoopKerberosKeytabResolverStep.scala | 4 +++- .../k8s/KubernetesClusterManager.scala | 9 ++++++++- .../spark/deploy/k8s/submit/ClientSuite.scala | 2 +- ...rConfigurationStepsOrchestratorSuite.scala | 1 + .../HadoopConfSparkUserBootstrapSuite.scala | 2 +- .../HadoopConfMounterStepSuite.scala | 1 + .../HadoopConfSparkUserStepSuite.scala | 2 +- ...adoopKerberosKeytabResolverStepSuite.scala | 4 ++-- 13 files changed, 37 insertions(+), 26 deletions(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/HadoopConfBootstrap.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/HadoopConfBootstrap.scala index 62dd15ba9a819..ad6c805c20b2e 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/HadoopConfBootstrap.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/HadoopConfBootstrap.scala @@ -45,8 +45,8 @@ private[spark] class HadoopConfBootstrapImpl( override def bootstrapMainContainerAndVolumes(originalPodWithMainContainer: PodWithMainContainer) : PodWithMainContainer = { - logInfo("HADOOP_CONF_DIR defined. Mounting Hadoop specific .xml files") - val keyPaths = hadoopConfigFiles.map{ file => + logInfo("HADOOP_CONF_DIR defined. Mounting Hadoop specific files") + val keyPaths = hadoopConfigFiles.map { file => val fileStringPath = file.toPath.getFileName.toString new KeyToPathBuilder() .withKey(fileStringPath) @@ -74,6 +74,7 @@ private[spark] class HadoopConfBootstrapImpl( .withValue(HADOOP_CONF_DIR_PATH) .endEnv() .build() + originalPodWithMainContainer.copy( pod = hadoopSupportedPod, mainContainer = hadoopSupportedContainer) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KerberosTokenConfBootstrap.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KerberosTokenConfBootstrap.scala index 300483795ce47..c873f72aac956 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KerberosTokenConfBootstrap.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KerberosTokenConfBootstrap.scala @@ -40,7 +40,7 @@ private[spark] class KerberosTokenConfBootstrapImpl( override def bootstrapMainContainerAndVolumes( originalPodWithMainContainer: PodWithMainContainer) : PodWithMainContainer = { - logInfo("Mounting HDFS DT from Secret for Secure HDFS") + logInfo(s"Mounting HDFS DT from Secret $secretName for Secure HDFS") val secretMountedPod = new PodBuilder(originalPodWithMainContainer.pod) .editOrNewSpec() .addNewVolume() 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 cfbd242c0a62c..e395fed810a3d 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 @@ -504,38 +504,38 @@ package object config extends Logging { private[spark] 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 private[spark] 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 private[spark] 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 private[spark] 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") + "your existing delegation token is stored. This removes the need " + + "for the job user to provide any keytab for launching a job") .stringConf .createOptional private[spark] 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") + "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/constants.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/constants.scala index 9dfc73039a704..0a0253eef626d 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 @@ -110,11 +110,11 @@ package object constants { // Kerberos Configuration private[spark] val HADOOP_KERBEROS_SECRET_NAME = - "spark.kubernetes.kerberos.dt" + "spark.kubernetes.kerberos.delegationTokenSecretName" private[spark] val HADOOP_KERBEROS_CONF_SECRET = - "spark.kubernetes.kerberos.secretname" + "spark.kubernetes.kerberos.keyTabSecretName" private[spark] val HADOOP_KERBEROS_CONF_ITEM_KEY = - "spark.kubernetes.kerberos.itemkeyname" + "spark.kubernetes.kerberos.keyTabSecretKey" private[spark] val KERBEROS_SECRET_LABEL_PREFIX = "hadoop-tokens" private[spark] val SPARK_HADOOP_PREFIX = "spark.hadoop." diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/Client.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/Client.scala index 35f78ddfa2c17..fadd13dbe0b62 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/Client.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/Client.scala @@ -69,7 +69,7 @@ private[spark] object ClientArguments { otherPyFiles, mainClass.get, driverArgs.toArray, - sys.env.get("HADOOP_CONF_DIR")) + sys.env.get(ENV_HADOOP_CONF_DIR)) } } @@ -163,8 +163,7 @@ private[spark] class Client( } private[spark] object Client { - def run(sparkConf: SparkConf, - clientArguments: ClientArguments): Unit = { + def run(sparkConf: SparkConf, clientArguments: ClientArguments): Unit = { val namespace = sparkConf.get(KUBERNETES_NAMESPACE) val kubernetesAppId = s"spark-${UUID.randomUUID().toString.replaceAll("-", "")}" val launchTime = System.currentTimeMillis() diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopKerberosKeytabResolverStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopKerberosKeytabResolverStep.scala index 4c3184e7f550b..1a213ae3aa163 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopKerberosKeytabResolverStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopKerberosKeytabResolverStep.scala @@ -58,7 +58,8 @@ private[spark] class HadoopKerberosKeytabResolverStep( override def configureContainers(hadoopConfigSpec: HadoopConfigSpec): HadoopConfigSpec = { val hadoopConf = SparkHadoopUtil.get.newConfiguration(submissionSparkConf) if (!hadoopUGI.isSecurityEnabled) { - throw new SparkException("Hadoop not configured with Kerberos") } + throw new SparkException("Hadoop not configured with Kerberos") + } val maybeJobUserUGI = for { principal <- maybePrincipal @@ -90,6 +91,7 @@ private[spark] class HadoopKerberosKeytabResolverStep( credentials) credentials.getAllTokens.asScala }}) + if (tokens.isEmpty) throw new SparkException(s"Did not obtain any delegation tokens") val data = hadoopUGI.serialize(credentials) val renewalInterval = diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala index 59365947c57bd..eb19c7c69aa7b 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala @@ -84,6 +84,7 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit configMap, configMapKey) } + val hadoopUtil = new HadoopUGIUtilImpl val hadoopBootStrap = maybeHadoopConfigMap.map{ hadoopConfigMap => val hadoopConfigurations = maybeHadoopConfDir.map( @@ -93,6 +94,7 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit hadoopConfigurations, hadoopUtil) } + val kerberosBootstrap = maybeHadoopConfigMap.flatMap { _ => for { @@ -104,12 +106,14 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit secretItemKey, Utils.getCurrentUserName() ) } } + val hadoopUserBootstrap = if (hadoopBootStrap.isDefined && kerberosBootstrap.isEmpty) { Some(new HadoopConfSparkUserBootstrapImpl(hadoopUtil)) } else { None } + val mountSmallFilesBootstrap = for { secretName <- maybeSubmittedFilesSecret secretMountPath <- maybeSubmittedFilesSecretMountPath @@ -129,6 +133,7 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit logWarning("The executor's init-container config map was not specified. Executors will" + " therefore not attempt to fetch remote or submitted dependencies.") } + if (maybeInitContainerConfigMapKey.isEmpty) { logWarning("The executor's init-container config map key was not specified. Executors will" + " therefore not attempt to fetch remote or submitted dependencies.") @@ -136,8 +141,9 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit if (maybeHadoopConfigMap.isEmpty) { logWarning("The executor's hadoop config map key was not specified. Executors will" + - " therefore not attempt to fetch hadoop configuration files.") + " therefore not attempt to mount hadoop configuration files.") } + val kubernetesClient = SparkKubernetesClientFactory.createKubernetesClient( KUBERNETES_MASTER_INTERNAL_URL, Some(sparkConf.get(KUBERNETES_NAMESPACE)), @@ -188,6 +194,7 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit override def initialize(scheduler: TaskScheduler, backend: SchedulerBackend): Unit = { scheduler.asInstanceOf[TaskSchedulerImpl].initialize(backend) } + private def getHadoopConfFiles(path: String) : Array[File] = { val dir = new File(path) if (dir.isDirectory) { 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 8a7cc54e7672a..846335889ee54 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 @@ -161,7 +161,7 @@ class ClientSuite extends SparkFunSuite with BeforeAndAfter { s"${SecondTestConfigurationStep.sparkConfValue}", s"-XX:+HeapDumpOnOutOfMemoryError", s"-XX:+PrintGCDetails", - "-Dspark.hadoop.hadoop.security.authentication=simple") + s"-D$HADOOP_SECURITY_AUTHENTICATION=simple") driverJvmOptsEnvs.zip(expectedJvmOptsValues).zipWithIndex.foreach { case ((resolvedEnv, expectedJvmOpt), index) => assert(resolvedEnv.getName === s"$ENV_JAVA_OPT_PREFIX$index") diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestratorSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestratorSuite.scala index 605a9aa2def29..39ab330786943 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestratorSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestratorSuite.scala @@ -205,6 +205,7 @@ private[spark] class DriverConfigurationStepsOrchestratorSuite extends SparkFunS classOf[LocalDirectoryMountConfigurationStep], classOf[MountSecretsStep]) } + test("Submission steps with hdfs interaction and HADOOP_CONF_DIR defined") { val sparkConf = new SparkConf(false) val mainAppResource = JavaMainAppResource("local:///var/apps/jars/main.jar") diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/HadoopConfSparkUserBootstrapSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/HadoopConfSparkUserBootstrapSuite.scala index 3c9b25208d5eb..f252799377c05 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/HadoopConfSparkUserBootstrapSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/HadoopConfSparkUserBootstrapSuite.scala @@ -39,7 +39,7 @@ private[spark] class HadoopConfSparkUserBootstrapSuite extends SparkFunSuite wit when(hadoopUtil.getShortUserName).thenReturn(SPARK_USER_VALUE) } - test("Test of bootstrapping hadoop_conf_dir files") { + test("Test of bootstrapping ENV_VARs for SPARK_USER") { val hadoopConfStep = new HadoopConfSparkUserBootstrapImpl(hadoopUtil) val emptyPod = new PodBuilder().withNewSpec().endSpec().build() val podWithMain = PodWithMainContainer( diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopConfMounterStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopConfMounterStepSuite.scala index a98bfd8640b96..18bb3b631cf28 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopConfMounterStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopConfMounterStepSuite.scala @@ -92,6 +92,7 @@ private[spark] class HadoopConfMounterStepSuite extends SparkFunSuite with Befor assert(returnContainerSpec.driverPod.getMetadata.getLabels.asScala === POD_LABEL) assert(returnContainerSpec.configMapProperties === expectedConfigMap) } + private def createTempFile(contents: String): File = { val dir = Utils.createTempDir() val file = new File(dir, s"${UUID.randomUUID().toString}") diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopConfSparkUserStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopConfSparkUserStepSuite.scala index 91b5c10376e73..82085fb979309 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopConfSparkUserStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopConfSparkUserStepSuite.scala @@ -57,7 +57,7 @@ private[spark] class HadoopConfSparkUserStepSuite extends SparkFunSuite with Bef )}}) } - test("Test of mounting hadoop_conf_dir files into HadoopConfigSpec") { + test("Test of calling the SPARK_USER bootstrap to modify the HadoopConfSpec") { val hadoopSparkUserStep = new HadoopConfSparkUserStep(hadoopConfSparkUserBootstrap) val hadoopConfSpec = HadoopConfigSpec( Map.empty[String, String], diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopKerberosKeytabResolverStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopKerberosKeytabResolverStepSuite.scala index 1a022f7f115b7..9dfa799727c34 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopKerberosKeytabResolverStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopKerberosKeytabResolverStepSuite.scala @@ -107,7 +107,7 @@ private[spark] class HadoopKerberosKeytabResolverStepSuite any[Configuration])).thenReturn(Some(INTERVAL)) } - test("Testing Error Catching for Security Enabling") { + test("Testing error catching for security enabling") { when(hadoopUtil.isSecurityEnabled).thenReturn(false) val keytabStep = new HadoopKerberosKeytabResolverStep( KUBE_TEST_NAME, @@ -134,7 +134,7 @@ private[spark] class HadoopKerberosKeytabResolverStepSuite } } - test("Testing Error Catching for No Token catching") { + test("Testing error catching for no token catching") { when(hadoopUtil.isSecurityEnabled).thenReturn(false) when(ugi.doAs(any(classOf[PrivilegedExceptionAction[Iterable[Token[_ <: TokenIdentifier]]]]))) .thenReturn(Iterable[Token[_ <: TokenIdentifier]]())