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 ed46adcbe9dfb..c04938fdf26da 100644
--- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
@@ -575,7 +575,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 5346e54e62aec..f66fa7935b15c 100644
--- a/docs/running-on-kubernetes.md
+++ b/docs/running-on-kubernetes.md
@@ -783,6 +783,61 @@ from the other deployment modes. See the [configuration page](configuration.html
+ 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) |
diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/HadoopConfBootstrap.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/HadoopConfBootstrap.scala
new file mode 100644
index 0000000000000..17efe19b8f206
--- /dev/null
+++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/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.kubernetes
+
+import java.io.File
+
+import scala.collection.JavaConverters._
+
+import io.fabric8.kubernetes.api.model.{ContainerBuilder, KeyToPathBuilder, PodBuilder}
+
+import org.apache.spark.deploy.kubernetes.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()).toList
+ 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/kubernetes/HadoopUGIUtil.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/HadoopUGIUtil.scala
new file mode 100644
index 0000000000000..8a004c0e5414f
--- /dev/null
+++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/HadoopUGIUtil.scala
@@ -0,0 +1,78 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.deploy.kubernetes
+
+import 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}
+ if (renewIntervals.isEmpty) None else Some(renewIntervals.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/kubernetes/KerberosTokenConfBootstrap.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/KerberosTokenConfBootstrap.scala
new file mode 100644
index 0000000000000..6225142fa4b52
--- /dev/null
+++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/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.kubernetes
+
+import io.fabric8.kubernetes.api.model.{ContainerBuilder, PodBuilder}
+
+import org.apache.spark.deploy.kubernetes.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)
+ }
+}
diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/PodWithMainContainer.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/PodWithMainContainer.scala
new file mode 100644
index 0000000000000..664eb41bd68d5
--- /dev/null
+++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/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.kubernetes
+
+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/kubernetes/config.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/config.scala
index 5bd61328f3a0b..442bdb01b979b 100644
--- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/config.scala
+++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/config.scala
@@ -542,6 +542,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/kubernetes/constants.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/constants.scala
index 3faf387aadfc0..dfb4e0838113f 100644
--- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/constants.scala
+++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/constants.scala
@@ -45,9 +45,6 @@ package object constants {
// 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_TOKEN_FILE_SECRET_NAME = "hadoop-token-file"
- private[spark] val SPARK_APP_HADOOP_TOKEN_FILE_PATH =
- s"$SPARK_APP_HADOOP_CREDENTIALS_BASE_DIR/$SPARK_APP_HADOOP_TOKEN_FILE_SECRET_NAME"
private[spark] val SPARK_APP_HADOOP_SECRET_VOLUME_NAME = "hadoop-secret"
// Default and fixed ports
@@ -79,6 +76,7 @@ package object constants {
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"
@@ -101,6 +99,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/kubernetes/submit/Client.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/Client.scala
index 1901f1198a84f..15f6c9d7ce0d8 100644
--- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/Client.scala
+++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/Client.scala
@@ -79,6 +79,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
@@ -99,7 +102,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")
@@ -153,7 +157,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()
@@ -172,6 +178,7 @@ private[spark] object Client {
clientArguments.mainClass,
clientArguments.driverArgs,
clientArguments.otherPyFiles,
+ hadoopConfDir,
sparkConf)
Utils.tryWithResource(SparkKubernetesClientFactory.createKubernetesClient(
master,
@@ -199,6 +206,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/kubernetes/submit/DriverConfigurationStepsOrchestrator.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/DriverConfigurationStepsOrchestrator.scala
index 1e9f5863a0d96..bc0e29ec6980d 100644
--- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/DriverConfigurationStepsOrchestrator.scala
+++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/DriverConfigurationStepsOrchestrator.scala
@@ -21,6 +21,7 @@ import org.apache.spark.deploy.kubernetes.ConfigurationUtils
import org.apache.spark.deploy.kubernetes.config._
import org.apache.spark.deploy.kubernetes.constants._
import org.apache.spark.deploy.kubernetes.submit.submitsteps._
+import org.apache.spark.deploy.kubernetes.submit.submitsteps.hadoopsteps.HadoopStepsOrchestrator
import org.apache.spark.deploy.kubernetes.submit.submitsteps.initcontainer.InitContainerConfigurationStepsOrchestrator
import org.apache.spark.launcher.SparkLauncher
import org.apache.spark.util.Utils
@@ -37,6 +38,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
@@ -51,6 +53,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 {
@@ -94,7 +97,13 @@ private[spark] class DriverConfigurationStepsOrchestrator(
submissionSparkConf)
val kubernetesCredentialsStep = new DriverKubernetesCredentialsStep(
submissionSparkConf, kubernetesResourceNamePrefix)
- val hadoopCredentialsStep = new DriverHadoopCredentialsStep(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 pythonStep = mainAppResource match {
case PythonMainAppResource(mainPyResource) =>
Option(new PythonStep(mainPyResource, additionalPythonFiles, filesDownloadPath))
@@ -162,9 +171,9 @@ private[spark] class DriverConfigurationStepsOrchestrator(
Seq(
initialSubmissionStep,
kubernetesCredentialsStep,
- hadoopCredentialsStep,
dependencyResolutionStep) ++
submittedDependenciesBootstrapSteps ++
+ hadoopConfigSteps.toSeq ++
pythonStep.toSeq
}
diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/HadoopSecretUtil.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/HadoopSecretUtil.scala
deleted file mode 100644
index b7a9e9e3fd6cd..0000000000000
--- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/HadoopSecretUtil.scala
+++ /dev/null
@@ -1,55 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.spark.deploy.kubernetes.submit
-
-import io.fabric8.kubernetes.api.model.{Container, ContainerBuilder, Pod, PodBuilder}
-
-import org.apache.spark.deploy.kubernetes.constants._
-
-object HadoopSecretUtil {
-
- def configurePod(secretNameOption: Option[String], pod: Pod) : Pod = {
- secretNameOption.map { secret =>
- new PodBuilder(pod)
- .editOrNewSpec()
- .addNewVolume()
- .withName(SPARK_APP_HADOOP_SECRET_VOLUME_NAME)
- .withNewSecret()
- .withSecretName(secret)
- .endSecret()
- .endVolume()
- .endSpec()
- .build()
- }.getOrElse(pod)
- }
-
- def configureContainer(secretNameOption: Option[String],
- containerSpec: Container) : Container = {
- secretNameOption.map { secret =>
- new ContainerBuilder(containerSpec)
- .addNewVolumeMount()
- .withName(SPARK_APP_HADOOP_SECRET_VOLUME_NAME)
- .withMountPath(SPARK_APP_HADOOP_CREDENTIALS_BASE_DIR)
- .endVolumeMount()
- .addNewEnv()
- .withName(ENV_HADOOP_TOKEN_FILE_LOCATION)
- .withValue(SPARK_APP_HADOOP_TOKEN_FILE_PATH)
- .endEnv()
- .build()
- }.getOrElse(containerSpec)
- }
-}
diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DriverHadoopCredentialsStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DriverHadoopCredentialsStep.scala
deleted file mode 100644
index 88f2e7b3836c8..0000000000000
--- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DriverHadoopCredentialsStep.scala
+++ /dev/null
@@ -1,38 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.spark.deploy.kubernetes.submit.submitsteps
-
-import org.apache.spark.SparkConf
-import org.apache.spark.deploy.kubernetes.config._
-import org.apache.spark.deploy.kubernetes.submit.HadoopSecretUtil
-
-private[spark] class DriverHadoopCredentialsStep(submissionSparkConf: SparkConf)
- extends DriverConfigurationStep {
-
- private val maybeMountedHadoopSecret = submissionSparkConf.get(MOUNTED_HADOOP_SECRET_CONF)
-
- override def configureDriver(driverSpec: KubernetesDriverSpec): KubernetesDriverSpec = {
- val podWithMountedHadoopToken = HadoopSecretUtil.configurePod(maybeMountedHadoopSecret,
- driverSpec.driverPod)
- val containerWithMountedHadoopToken = HadoopSecretUtil.configureContainer(
- maybeMountedHadoopSecret,
- driverSpec.driverContainer)
- driverSpec.copy(
- driverPod = podWithMountedHadoopToken,
- driverContainer = containerWithMountedHadoopToken)
- }
-}
diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/HadoopConfigBootstrapStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/HadoopConfigBootstrapStep.scala
new file mode 100644
index 0000000000000..a9c66aed7075e
--- /dev/null
+++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/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.kubernetes.submit.submitsteps
+
+import scala.collection.JavaConverters._
+
+import io.fabric8.kubernetes.api.model.ConfigMapBuilder
+
+import org.apache.spark.deploy.kubernetes.constants._
+import org.apache.spark.deploy.kubernetes.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/kubernetes/submit/submitsteps/hadoopsteps/HadoopConfMounterStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopConfMounterStep.scala
new file mode 100644
index 0000000000000..53b979df326c2
--- /dev/null
+++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopConfMounterStep.scala
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.deploy.kubernetes.submit.submitsteps.hadoopsteps
+
+import java.io.File
+
+import org.apache.commons.io.FileUtils.readFileToString
+
+import org.apache.spark.deploy.kubernetes.{HadoopConfBootstrap, PodWithMainContainer}
+import org.apache.spark.deploy.kubernetes.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/kubernetes/submit/submitsteps/hadoopsteps/HadoopConfigSpec.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopConfigSpec.scala
new file mode 100644
index 0000000000000..51e9e506bcc22
--- /dev/null
+++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/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.kubernetes.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/kubernetes/submit/submitsteps/hadoopsteps/HadoopConfigurationStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopConfigurationStep.scala
new file mode 100644
index 0000000000000..54f46b9a17fc9
--- /dev/null
+++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/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.kubernetes.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/kubernetes/submit/submitsteps/hadoopsteps/HadoopKerberosKeytabResolverStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopKerberosKeytabResolverStep.scala
new file mode 100644
index 0000000000000..4e0a0cfd8fe9b
--- /dev/null
+++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopKerberosKeytabResolverStep.scala
@@ -0,0 +1,134 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.deploy.kubernetes.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.kubernetes.{HadoopUGIUtil, KerberosTokenConfBootstrapImpl, PodWithMainContainer}
+import org.apache.spark.deploy.kubernetes.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)
+ logDebug(s"Hadoop Configuration: ${hadoopConf.toString}")
+ 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 = {
+ logDebug(s"Retrieved Job User UGI: $jobUserUGI")
+ originalCredentials = jobUserUGI.getCredentials
+ logDebug(s"Original tokens: ${originalCredentials.toString}")
+ logDebug(s"All tokens: ${originalCredentials.getAllTokens}")
+ logDebug(s"All secret keys: ${originalCredentials.getAllSecretKeys}")
+ // 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)
+ logDebug(s"Renewer is: $renewer")
+ credentials = new Credentials(originalCredentials)
+ hadoopUGI.dfsAddDelegationToken(hadoopConf, renewer, credentials)
+ tokens = credentials.getAllTokens.asScala
+ logDebug(s"Tokens: ${credentials.toString}")
+ logDebug(s"All tokens: ${tokens.mkString(",")}")
+ logDebug(s"All secret keys: ${credentials.getAllSecretKeys}")
+ 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/kubernetes/submit/submitsteps/hadoopsteps/HadoopKerberosSecretResolverStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopKerberosSecretResolverStep.scala
new file mode 100644
index 0000000000000..3eeec1374ef96
--- /dev/null
+++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/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.kubernetes.submit.submitsteps.hadoopsteps
+
+import org.apache.hadoop.security.UserGroupInformation
+
+import org.apache.spark.SparkConf
+import org.apache.spark.deploy.kubernetes.{KerberosTokenConfBootstrapImpl, PodWithMainContainer}
+import org.apache.spark.deploy.kubernetes.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/kubernetes/submit/submitsteps/hadoopsteps/HadoopStepsOrchestrator.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopStepsOrchestrator.scala
new file mode 100644
index 0000000000000..93d184fedbcf1
--- /dev/null
+++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/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.kubernetes.submit.submitsteps.hadoopsteps
+
+import java.io.File
+
+import org.apache.spark.SparkConf
+import org.apache.spark.deploy.kubernetes.{HadoopConfBootstrapImpl, HadoopUGIUtil, OptionRequirements}
+import org.apache.spark.deploy.kubernetes.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]
+ }
+ }
+}
diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterManager.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterManager.scala
index b89e81bcb0be9..ff02c08ee2dce 100644
--- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterManager.scala
+++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterManager.scala
@@ -20,16 +20,16 @@ import java.io.File
import io.fabric8.kubernetes.client.Config
-import org.apache.spark.SparkContext
-import org.apache.spark.deploy.kubernetes.{InitContainerResourceStagingServerSecretPluginImpl, SparkKubernetesClientFactory, SparkPodInitContainerBootstrapImpl}
+import org.apache.spark.deploy.kubernetes._
import org.apache.spark.deploy.kubernetes.config._
import org.apache.spark.deploy.kubernetes.constants._
import org.apache.spark.deploy.kubernetes.submit.MountSmallFilesBootstrapImpl
import org.apache.spark.internal.Logging
import org.apache.spark.scheduler.{ExternalClusterManager, SchedulerBackend, TaskScheduler, TaskSchedulerImpl}
+import org.apache.spark.util.Utils
+import org.apache.spark.SparkContext
private[spark] class KubernetesClusterManager extends ExternalClusterManager with Logging {
-
override def canCreate(masterURL: String): Boolean = masterURL.startsWith("k8s")
override def createTaskScheduler(sc: SparkContext, masterURL: String): TaskScheduler = {
@@ -41,6 +41,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)
@@ -75,6 +79,27 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit
configMap,
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
@@ -89,6 +114,10 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit
logWarning("The executor's init-container config map key was not specified. Executors will" +
" 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)),
@@ -100,6 +129,8 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit
sc.taskScheduler.asInstanceOf[TaskSchedulerImpl],
sc,
executorInitContainerbootStrap,
+ hadoopBootStrap,
+ kerberosBootstrap,
executorInitContainerSecretVolumePlugin,
mountSmallFilesBootstrap,
kubernetesClient)
@@ -108,4 +139,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]
+ }
+ }
}
diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala
index 415c1bda7b632..3738e1b8d3991 100644
--- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala
+++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala
@@ -33,10 +33,10 @@ import scala.collection.JavaConverters._
import scala.concurrent.{ExecutionContext, Future}
import org.apache.spark.{SparkContext, SparkEnv, SparkException}
-import org.apache.spark.deploy.kubernetes.{ConfigurationUtils, InitContainerResourceStagingServerSecretPlugin, PodWithDetachedInitContainer, SparkPodInitContainerBootstrap}
+import org.apache.spark.deploy.kubernetes._
import org.apache.spark.deploy.kubernetes.config._
import org.apache.spark.deploy.kubernetes.constants._
-import org.apache.spark.deploy.kubernetes.submit.{HadoopSecretUtil, InitContainerUtil, MountSmallFilesBootstrap}
+import org.apache.spark.deploy.kubernetes.submit.{InitContainerUtil, MountSmallFilesBootstrap}
import org.apache.spark.network.netty.SparkTransportConf
import org.apache.spark.network.shuffle.kubernetes.KubernetesExternalShuffleClient
import org.apache.spark.rpc.{RpcAddress, RpcCallContext, RpcEndpointAddress, RpcEnv}
@@ -49,6 +49,8 @@ private[spark] class KubernetesClusterSchedulerBackend(
scheduler: TaskSchedulerImpl,
val sc: SparkContext,
executorInitContainerBootstrap: Option[SparkPodInitContainerBootstrap],
+ executorHadoopBootStrap: Option[HadoopConfBootstrap],
+ executorKerberosBootStrap: Option[KerberosTokenConfBootstrap],
executorMountInitContainerSecretPlugin: Option[InitContainerResourceStagingServerSecretPlugin],
mountSmallFilesBootstrap: Option[MountSmallFilesBootstrap],
kubernetesClient: KubernetesClient)
@@ -73,7 +75,9 @@ private[spark] class KubernetesClusterSchedulerBackend(
private val executorExtraClasspath = conf.get(
org.apache.spark.internal.config.EXECUTOR_CLASS_PATH)
private val executorJarsDownloadDir = conf.get(INIT_CONTAINER_JARS_DOWNLOAD_LOCATION)
-
+ private val isKerberosEnabled = conf.get(KUBERNETES_KERBEROS_SUPPORT)
+ private val maybeSimpleAuthentication =
+ if (isKerberosEnabled) Some(s"-D$HADOOP_SECURITY_AUTHENTICATION=simple") else None
private val executorLabels = ConfigurationUtils.combinePrefixedKeyValuePairsWithDeprecatedConf(
conf,
KUBERNETES_EXECUTOR_LABEL_PREFIX,
@@ -130,8 +134,6 @@ private[spark] class KubernetesClusterSchedulerBackend(
private implicit val requestExecutorContext = ExecutionContext.fromExecutorService(
ThreadUtils.newDaemonCachedThreadPool("kubernetes-executor-requests"))
- private val maybeMountedHadoopSecret = conf.get(MOUNTED_HADOOP_SECRET_CONF)
-
private val driverPod = try {
kubernetesClient.pods().inNamespace(kubernetesNamespace).
withName(kubernetesDriverPodName).get()
@@ -457,15 +459,19 @@ private[spark] class KubernetesClusterSchedulerBackend(
.withValue(cp)
.build()
}
- val executorExtraJavaOptionsEnv = conf
- .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 = (
+ conf.get(org.apache.spark.internal.config.EXECUTOR_JAVA_OPTIONS)
+ ++ maybeSimpleAuthentication).mkString(" ") match {
+ case "" => None
+ case str => Some(str)
+ }
+ 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),
@@ -597,14 +603,23 @@ private[spark] class KubernetesClusterSchedulerBackend(
val executorPodWithNodeAffinity = addNodeAffinityAnnotationIfUseful(
executorPodWithInitContainer, nodeToLocalTaskCount)
- val executorPodWithMountedHadoopToken = HadoopSecretUtil.configurePod(maybeMountedHadoopSecret,
- executorPodWithNodeAffinity)
- val containerWithMountedHadoopToken = HadoopSecretUtil.configureContainer(
- maybeMountedHadoopSecret, initBootstrappedExecutorContainer)
-
- val resolvedExecutorPod = new PodBuilder(executorPodWithMountedHadoopToken)
+ val (executorHadoopConfPod, executorHadoopConfContainer) =
+ executorHadoopBootStrap.map { bootstrap =>
+ val podWithMainContainer = bootstrap.bootstrapMainContainerAndVolumes(
+ PodWithMainContainer(executorPodWithNodeAffinity, initBootstrappedExecutorContainer)
+ )
+ (podWithMainContainer.pod, podWithMainContainer.mainContainer)
+ }.getOrElse(executorPodWithNodeAffinity, initBootstrappedExecutorContainer)
+
+ val (executorKerberosPod, executorKerberosContainer) =
+ executorKerberosBootStrap.map { bootstrap =>
+ val podWithMainContainer = bootstrap.bootstrapMainContainerAndVolumes(
+ PodWithMainContainer(executorHadoopConfPod, executorHadoopConfContainer))
+ (podWithMainContainer.pod, podWithMainContainer.mainContainer)
+ }.getOrElse((executorHadoopConfPod, executorHadoopConfContainer))
+ val resolvedExecutorPod = new PodBuilder(executorKerberosPod)
.editSpec()
- .addToContainers(containerWithMountedHadoopToken)
+ .addToContainers(executorKerberosContainer)
.endSpec()
.build()
try {
diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/InitContainerResourceStagingServerSecretPluginSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/InitContainerResourceStagingServerSecretPluginSuite.scala
index 597bcdb416fc0..51e695fc08a36 100644
--- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/InitContainerResourceStagingServerSecretPluginSuite.scala
+++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/InitContainerResourceStagingServerSecretPluginSuite.scala
@@ -23,7 +23,8 @@ import scala.collection.JavaConverters._
import org.apache.spark.deploy.kubernetes.constants._
import org.apache.spark.SparkFunSuite
-class InitContainerResourceStagingServerSecretPluginSuite extends SparkFunSuite with BeforeAndAfter{
+class InitContainerResourceStagingServerSecretPluginSuite
+ extends SparkFunSuite with BeforeAndAfter{
private val INIT_CONTAINER_SECRET_NAME = "init-secret"
private val INIT_CONTAINER_SECRET_MOUNT = "/tmp/secret"
diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/ClientSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/ClientSuite.scala
index 0100dce454a3f..35fd5c1f206fa 100644
--- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/ClientSuite.scala
+++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/ClientSuite.scala
@@ -16,23 +16,25 @@
*/
package org.apache.spark.deploy.kubernetes.submit
+import scala.collection.JavaConverters._
+
import com.google.common.collect.Iterables
-import io.fabric8.kubernetes.api.model.{ContainerBuilder, DoneablePod, HasMetadata, Pod, PodBuilder, PodList, Secret, SecretBuilder}
+import io.fabric8.kubernetes.api.model._
import io.fabric8.kubernetes.client.{KubernetesClient, Watch}
-import io.fabric8.kubernetes.client.dsl.{MixedOperation, NamespaceListVisitFromServerGetDeleteRecreateWaitApplicable, NamespaceVisitFromServerGetWatchDeleteRecreateWaitApplicable, PodResource, Resource}
+import io.fabric8.kubernetes.client.dsl.{MixedOperation, NamespaceListVisitFromServerGetDeleteRecreateWaitApplicable, PodResource}
import org.mockito.{ArgumentCaptor, Mock, MockitoAnnotations}
-import org.mockito.Mockito.{doReturn, verify, when}
import org.mockito.invocation.InvocationOnMock
+import org.mockito.Mockito.{doReturn, verify, when}
import org.mockito.stubbing.Answer
import org.scalatest.BeforeAndAfter
import org.scalatest.mock.MockitoSugar._
-import scala.collection.JavaConverters._
import org.apache.spark.{SparkConf, SparkFunSuite}
+import org.apache.spark.deploy.kubernetes.config._
import org.apache.spark.deploy.kubernetes.constants._
import org.apache.spark.deploy.kubernetes.submit.submitsteps.{DriverConfigurationStep, KubernetesDriverSpec}
-class ClientSuite extends SparkFunSuite with BeforeAndAfter {
+private[spark] class ClientSuite extends SparkFunSuite with BeforeAndAfter {
private val DRIVER_POD_UID = "pod-id"
private val DRIVER_POD_API_VERSION = "v1"
@@ -136,6 +138,10 @@ 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 +156,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/kubernetes/submit/DriverConfigurationStepsOrchestratorSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/DriverConfigurationStepsOrchestratorSuite.scala
index c168e7b5407ba..b87fa48291a8c 100644
--- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/DriverConfigurationStepsOrchestratorSuite.scala
+++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/DriverConfigurationStepsOrchestratorSuite.scala
@@ -18,7 +18,7 @@ package org.apache.spark.deploy.kubernetes.submit
import org.apache.spark.{SparkConf, SparkFunSuite}
import org.apache.spark.deploy.kubernetes.config._
-import org.apache.spark.deploy.kubernetes.submit.submitsteps.{BaseDriverConfigurationStep, DependencyResolutionStep, DriverConfigurationStep, DriverKubernetesCredentialsStep, InitContainerBootstrapStep, MountSmallLocalFilesStep, PythonStep}
+import org.apache.spark.deploy.kubernetes.submit.submitsteps.{BaseDriverConfigurationStep, DependencyResolutionStep, DriverConfigurationStep, DriverKubernetesCredentialsStep, HadoopConfigBootstrapStep, InitContainerBootstrapStep, MountSmallLocalFilesStep, PythonStep}
private[spark] class DriverConfigurationStepsOrchestratorSuite extends SparkFunSuite {
@@ -43,6 +43,7 @@ private[spark] class DriverConfigurationStepsOrchestratorSuite extends SparkFunS
MAIN_CLASS,
APP_ARGS,
ADDITIONAL_PYTHON_FILES,
+ None,
sparkConf)
validateStepTypes(
orchestrator,
@@ -65,6 +66,7 @@ private[spark] class DriverConfigurationStepsOrchestratorSuite extends SparkFunS
MAIN_CLASS,
APP_ARGS,
ADDITIONAL_PYTHON_FILES,
+ None,
sparkConf)
validateStepTypes(
orchestrator,
@@ -86,6 +88,7 @@ private[spark] class DriverConfigurationStepsOrchestratorSuite extends SparkFunS
MAIN_CLASS,
APP_ARGS,
ADDITIONAL_PYTHON_FILES,
+ None,
sparkConf)
validateStepTypes(
orchestrator,
@@ -107,6 +110,7 @@ private[spark] class DriverConfigurationStepsOrchestratorSuite extends SparkFunS
MAIN_CLASS,
APP_ARGS,
ADDITIONAL_PYTHON_FILES,
+ None,
sparkConf)
validateStepTypes(
orchestrator,
@@ -116,11 +120,36 @@ private[spark] class DriverConfigurationStepsOrchestratorSuite extends SparkFunS
classOf[MountSmallLocalFilesStep])
}
+ 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[DriverKubernetesCredentialsStep],
+ classOf[DependencyResolutionStep],
+ classOf[HadoopConfigBootstrapStep])
+ }
+
private def validateStepTypes(
- orchestrator: DriverConfigurationStepsOrchestrator,
- types: Class[_ <: DriverConfigurationStep]*): Unit = {
+ orchestrator: DriverConfigurationStepsOrchestrator,
+ types: Class[_ <: DriverConfigurationStep]*): Unit = {
val steps = orchestrator.getAllConfigurationSteps()
assert(steps.size === types.size)
assert(steps.map(_.getClass) === types)
}
+
}
diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/HadoopConfBootstrapSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/HadoopConfBootstrapSuite.scala
new file mode 100644
index 0000000000000..64426f0deb15e
--- /dev/null
+++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/HadoopConfBootstrapSuite.scala
@@ -0,0 +1,89 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.deploy.kubernetes.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.kubernetes.{HadoopConfBootstrapImpl, HadoopUGIUtil, PodWithMainContainer}
+import org.apache.spark.deploy.kubernetes.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: HadoopUGIUtil = _
+
+ before {
+ MockitoAnnotations.initMocks(this)
+ when(hadoopUtil.getShortName).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/kubernetes/submit/KerberosTokenConfBootstrapSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/KerberosTokenConfBootstrapSuite.scala
new file mode 100644
index 0000000000000..95d7159736213
--- /dev/null
+++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/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.kubernetes.submit
+
+import scala.collection.JavaConverters._
+
+import io.fabric8.kubernetes.api.model._
+
+import org.apache.spark.SparkFunSuite
+import org.apache.spark.deploy.kubernetes.{KerberosTokenConfBootstrapImpl, PodWithMainContainer}
+import org.apache.spark.deploy.kubernetes.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/kubernetes/submit/submitsteps/HadoopConfigBootstrapStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/HadoopConfigBootstrapStepSuite.scala
new file mode 100644
index 0000000000000..074a02c339d95
--- /dev/null
+++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/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.kubernetes.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.kubernetes.constants._
+import org.apache.spark.deploy.kubernetes.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", "secretata")
+ .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/kubernetes/submit/submitsteps/initContainerBootstrapStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/InitContainerBootstrapStepSuite.scala
similarity index 99%
rename from resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initContainerBootstrapStepSuite.scala
rename to resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/InitContainerBootstrapStepSuite.scala
index b11b487111496..1c9a1ac50229e 100644
--- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initContainerBootstrapStepSuite.scala
+++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/InitContainerBootstrapStepSuite.scala
@@ -31,7 +31,7 @@ import org.apache.spark.deploy.kubernetes.constants._
import org.apache.spark.deploy.kubernetes.submit.submitsteps.initcontainer.{InitContainerConfigurationStep, InitContainerSpec}
import org.apache.spark.util.Utils
-private[spark] class initContainerBootstrapStepSuite extends SparkFunSuite {
+class InitContainerBootstrapStepSuite extends SparkFunSuite {
private val OBJECT_MAPPER = new ObjectMapper().registerModule(DefaultScalaModule)
private val CONFIG_MAP_NAME = "spark-init-config-map"
diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopConfMounterStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopConfMounterStepSuite.scala
new file mode 100644
index 0000000000000..876675d5a4d96
--- /dev/null
+++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/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.kubernetes.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.kubernetes.{HadoopConfBootstrap, PodWithMainContainer}
+import org.apache.spark.deploy.kubernetes.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/kubernetes/submit/submitsteps/hadoopsteps/HadoopKerberosKeytabResolverStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopKerberosKeytabResolverStepSuite.scala
new file mode 100644
index 0000000000000..997556d1508a8
--- /dev/null
+++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopKerberosKeytabResolverStepSuite.scala
@@ -0,0 +1,146 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.deploy.kubernetes.submit.submitsteps.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, SparkFunSuite}
+import org.apache.spark.deploy.kubernetes.HadoopUGIUtil
+import org.apache.spark.deploy.kubernetes.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 DATA_KEY_NAME =
+ s"$KERBEROS_SECRET_LABEL_PREFIX-$CURR_TIME-$INTERVAL"
+ private val SECRET_NAME = s"$HADOOP_KERBEROS_SECRET_NAME.$CURR_TIME"
+
+ private val hadoopUGI = new HadoopUGIUtil()
+
+ @Mock
+ private var hadoopUtil: HadoopUGIUtil = _
+
+ @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.getShortName).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 keytab login") {
+ when(hadoopUtil.isSecurityEnabled).thenReturn(true)
+ val keytabStep = new HadoopKerberosKeytabResolverStep(
+ 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/kubernetes/submit/submitsteps/hadoopsteps/HadoopKerberosSecretResolverStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopKerberosSecretResolverStepSuite.scala
new file mode 100644
index 0000000000000..bb0fa237a2d5e
--- /dev/null
+++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/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.kubernetes.submit.submitsteps.hadoopsteps
+
+import scala.collection.JavaConverters._
+
+import io.fabric8.kubernetes.api.model._
+
+import org.apache.spark.{SparkConf, SparkFunSuite}
+import org.apache.spark.deploy.kubernetes.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/kubernetes/submit/submitsteps/hadoopsteps/HadoopStepsOrchestratorSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopStepsOrchestratorSuite.scala
new file mode 100644
index 0000000000000..1f946666f53ce
--- /dev/null
+++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopStepsOrchestratorSuite.scala
@@ -0,0 +1,85 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.deploy.kubernetes.submit.submitsteps.hadoopsteps
+
+import org.apache.spark.{SparkConf, SparkFunSuite}
+import org.apache.spark.deploy.kubernetes.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"
+
+ test("Testing without Kerberos") {
+ val sparkTestConf = new SparkConf(true)
+ .set(KUBERNETES_KERBEROS_SUPPORT, false)
+ val hadoopOrchestrator = new HadoopStepsOrchestrator(
+ 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(
+ 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(
+ 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(
+ 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/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/BaseInitContainerConfigurationStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/BaseInitContainerConfigurationStepSuite.scala
index fe1af4bc5be2a..79545a55f33b2 100644
--- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/BaseInitContainerConfigurationStepSuite.scala
+++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/BaseInitContainerConfigurationStepSuite.scala
@@ -29,7 +29,8 @@ import org.apache.spark.SparkFunSuite
import org.apache.spark.deploy.kubernetes.{PodWithDetachedInitContainer, SparkPodInitContainerBootstrap}
import org.apache.spark.deploy.kubernetes.config._
-class BaseInitContainerConfigurationStepSuite extends SparkFunSuite with BeforeAndAfter{
+private[spark] class BaseInitContainerConfigurationStepSuite
+ extends SparkFunSuite with BeforeAndAfter{
private val SPARK_JARS = Seq(
"hdfs://localhost:9000/app/jars/jar1.jar", "file:///app/jars/jar2.jar")
private val SPARK_FILES = Seq(
diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/SubmittedResourcesInitContainerStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/SubmittedResourcesInitContainerStepSuite.scala
index 2edaba93fe07f..e9c828fdb397c 100644
--- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/SubmittedResourcesInitContainerStepSuite.scala
+++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/SubmittedResourcesInitContainerStepSuite.scala
@@ -37,7 +37,8 @@ import org.apache.spark.deploy.kubernetes.constants._
import org.apache.spark.deploy.kubernetes.submit.{SubmittedDependencyUploader, SubmittedResourceIdAndSecret}
import org.apache.spark.util.Utils
-class SubmittedResourcesInitContainerStepSuite extends SparkFunSuite with BeforeAndAfter {
+private[spark] class SubmittedResourcesInitContainerStepSuite
+ extends SparkFunSuite with BeforeAndAfter {
private val RESOURCE_SECRET_NAME = "secret"
private val JARS_RESOURCE_ID = "jarsID"
private val JARS_SECRET = "jarsSecret"
diff --git a/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/driver-py/Dockerfile b/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/driver-py/Dockerfile
index 7b1effa911f19..27339d72cfd38 100644
--- a/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/driver-py/Dockerfile
+++ b/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/driver-py/Dockerfile
@@ -44,6 +44,7 @@ CMD SPARK_CLASSPATH="${SPARK_HOME}/jars/*" && \
if ! [ -z ${SPARK_MOUNTED_CLASSPATH+x} ]; then SPARK_CLASSPATH="$SPARK_MOUNTED_CLASSPATH:$SPARK_CLASSPATH"; fi && \
if ! [ -z ${SPARK_SUBMIT_EXTRA_CLASSPATH+x} ]; then SPARK_CLASSPATH="$SPARK_SUBMIT_EXTRA_CLASSPATH:$SPARK_CLASSPATH"; fi && \
if ! [ -z ${SPARK_EXTRA_CLASSPATH+x} ]; then SPARK_CLASSPATH="$SPARK_EXTRA_CLASSPATH:$SPARK_CLASSPATH"; fi && \
+ if ! [ -z ${HADOOP_CONF_DIR+x} ]; then SPARK_CLASSPATH="$HADOOP_CONF_DIR:$SPARK_CLASSPATH"; fi && \
if ! [ -z ${SPARK_MOUNTED_FILES_DIR+x} ]; then cp -R "$SPARK_MOUNTED_FILES_DIR/." .; fi && \
if ! [ -z ${SPARK_MOUNTED_FILES_FROM_SECRET_DIR+x} ]; then cp -R "$SPARK_MOUNTED_FILES_FROM_SECRET_DIR/." .; fi && \
${JAVA_HOME}/bin/java "${SPARK_DRIVER_JAVA_OPTS[@]}" -cp $SPARK_CLASSPATH -Xms$SPARK_DRIVER_MEMORY -Xmx$SPARK_DRIVER_MEMORY $SPARK_DRIVER_CLASS $PYSPARK_PRIMARY $PYSPARK_FILES $SPARK_DRIVER_ARGS
diff --git a/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/driver/Dockerfile b/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/driver/Dockerfile
index 26d1d805fde2b..3e99ef2809dcb 100644
--- a/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/driver/Dockerfile
+++ b/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/driver/Dockerfile
@@ -29,6 +29,7 @@ CMD SPARK_CLASSPATH="${SPARK_HOME}/jars/*" && \
if ! [ -z ${SPARK_MOUNTED_CLASSPATH+x} ]; then SPARK_CLASSPATH="$SPARK_MOUNTED_CLASSPATH:$SPARK_CLASSPATH"; fi && \
if ! [ -z ${SPARK_SUBMIT_EXTRA_CLASSPATH+x} ]; then SPARK_CLASSPATH="$SPARK_SUBMIT_EXTRA_CLASSPATH:$SPARK_CLASSPATH"; fi && \
if ! [ -z ${SPARK_EXTRA_CLASSPATH+x} ]; then SPARK_CLASSPATH="$SPARK_EXTRA_CLASSPATH:$SPARK_CLASSPATH"; fi && \
+ if ! [ -z ${HADOOP_CONF_DIR+x} ]; then SPARK_CLASSPATH="$HADOOP_CONF_DIR:$SPARK_CLASSPATH"; fi && \
if ! [ -z ${SPARK_MOUNTED_FILES_DIR+x} ]; then cp -R "$SPARK_MOUNTED_FILES_DIR/." .; fi && \
if ! [ -z ${SPARK_MOUNTED_FILES_FROM_SECRET_DIR} ]; then cp -R "$SPARK_MOUNTED_FILES_FROM_SECRET_DIR/." .; fi && \
${JAVA_HOME}/bin/java "${SPARK_DRIVER_JAVA_OPTS[@]}" -cp $SPARK_CLASSPATH -Xms$SPARK_DRIVER_MEMORY -Xmx$SPARK_DRIVER_MEMORY $SPARK_DRIVER_CLASS $SPARK_DRIVER_ARGS
diff --git a/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/executor-py/Dockerfile b/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/executor-py/Dockerfile
index a8bb5b362ab52..4e1b2ed91a1b5 100644
--- a/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/executor-py/Dockerfile
+++ b/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/executor-py/Dockerfile
@@ -43,6 +43,7 @@ CMD SPARK_CLASSPATH="${SPARK_HOME}/jars/*" && \
readarray -t SPARK_EXECUTOR_JAVA_OPTS < /tmp/java_opts.txt && \
if ! [ -z ${SPARK_MOUNTED_CLASSPATH}+x} ]; then SPARK_CLASSPATH="$SPARK_MOUNTED_CLASSPATH:$SPARK_CLASSPATH"; fi && \
if ! [ -z ${SPARK_EXECUTOR_EXTRA_CLASSPATH+x} ]; then SPARK_CLASSPATH="$SPARK_EXECUTOR_EXTRA_CLASSPATH:$SPARK_CLASSPATH"; fi && \
+ if ! [ -z ${HADOOP_CONF_DIR+x} ]; then SPARK_CLASSPATH="$HADOOP_CONF_DIR:$SPARK_CLASSPATH"; fi && \
if ! [ -z ${SPARK_MOUNTED_FILES_DIR+x} ]; then cp -R "$SPARK_MOUNTED_FILES_DIR/." .; fi && \
if ! [ -z ${SPARK_MOUNTED_FILES_FROM_SECRET_DIR+x} ]; then cp -R "$SPARK_MOUNTED_FILES_FROM_SECRET_DIR/." .; fi && \
${JAVA_HOME}/bin/java "${SPARK_EXECUTOR_JAVA_OPTS[@]}" -Dspark.executor.port=$SPARK_EXECUTOR_PORT -Xms$SPARK_EXECUTOR_MEMORY -Xmx$SPARK_EXECUTOR_MEMORY -cp $SPARK_CLASSPATH org.apache.spark.executor.CoarseGrainedExecutorBackend --driver-url $SPARK_DRIVER_URL --executor-id $SPARK_EXECUTOR_ID --cores $SPARK_EXECUTOR_CORES --app-id $SPARK_APPLICATION_ID --hostname $SPARK_EXECUTOR_POD_IP
diff --git a/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/executor/Dockerfile b/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/executor/Dockerfile
index ab9f67e95a8e5..28e7b8ec3b44f 100644
--- a/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/executor/Dockerfile
+++ b/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/executor/Dockerfile
@@ -29,6 +29,7 @@ CMD SPARK_CLASSPATH="${SPARK_HOME}/jars/*" && \
if ! [ -z ${SPARK_MOUNTED_CLASSPATH}+x} ]; then SPARK_CLASSPATH="$SPARK_MOUNTED_CLASSPATH:$SPARK_CLASSPATH"; fi && \
if ! [ -z ${SPARK_EXECUTOR_EXTRA_CLASSPATH+x} ]; then SPARK_CLASSPATH="$SPARK_EXECUTOR_EXTRA_CLASSPATH:$SPARK_CLASSPATH"; fi && \
if ! [ -z ${SPARK_EXTRA_CLASSPATH+x} ]; then SPARK_CLASSPATH="$SPARK_EXTRA_CLASSPATH:$SPARK_CLASSPATH"; fi && \
+ if ! [ -z ${HADOOP_CONF_DIR+x} ]; then SPARK_CLASSPATH="$HADOOP_CONF_DIR:$SPARK_CLASSPATH"; fi && \
if ! [ -z ${SPARK_MOUNTED_FILES_DIR+x} ]; then cp -R "$SPARK_MOUNTED_FILES_DIR/." .; fi && \
if ! [ -z ${SPARK_MOUNTED_FILES_FROM_SECRET_DIR+x} ]; then cp -R "$SPARK_MOUNTED_FILES_FROM_SECRET_DIR/." .; fi && \
${JAVA_HOME}/bin/java "${SPARK_EXECUTOR_JAVA_OPTS[@]}" -Dspark.executor.port=$SPARK_EXECUTOR_PORT -Xms$SPARK_EXECUTOR_MEMORY -Xmx$SPARK_EXECUTOR_MEMORY -cp $SPARK_CLASSPATH org.apache.spark.executor.CoarseGrainedExecutorBackend --driver-url $SPARK_DRIVER_URL --executor-id $SPARK_EXECUTOR_ID --cores $SPARK_EXECUTOR_CORES --app-id $SPARK_APPLICATION_ID --hostname $SPARK_EXECUTOR_POD_IP
diff --git a/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/kerberos-test/Dockerfile b/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/kerberos-test/Dockerfile
new file mode 100644
index 0000000000000..c4ba43ad511d8
--- /dev/null
+++ b/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/kerberos-test/Dockerfile
@@ -0,0 +1,27 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements. See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+FROM spark-base
+
+# If this docker file is being used in the context of building your images from a Spark distribution, the docker build
+# command should be invoked from the top level directory of the Spark distribution. E.g.:
+# docker build -t kerberos-test:latest -f dockerfiles/kerberos-test/Dockerfile .
+
+RUN apk add --no-cache --update krb5 krb5-libs
+COPY examples /opt/spark/examples
+COPY test-scripts/test-env.sh /opt/spark/
+COPY hconf /opt/spark/hconf
\ No newline at end of file
diff --git a/resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/kubernetes/integrationtest/jobs/HDFSTest.scala b/resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/kubernetes/integrationtest/jobs/HDFSTest.scala
new file mode 100644
index 0000000000000..463dd25d14a1f
--- /dev/null
+++ b/resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/kubernetes/integrationtest/jobs/HDFSTest.scala
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.deploy.kubernetes.integrationtest.jobs
+
+import org.apache.spark.sql.SparkSession
+
+private[spark] object HDFSTest{
+
+ def main(args: Array[String]): Unit = {
+ // scalastyle:off println
+ if (args.length < 1) {
+ System.err.println("Usage: HdfsTest ")
+ System.exit(1)
+ }
+ // scalastyle:on println
+ val spark = SparkSession
+ .builder
+ .appName("HdfsTest")
+ .getOrCreate()
+ val file = spark.read.text(args(0)).rdd
+ val mapped = file.map(s => s.length).cache()
+ // scalastyle:off println
+ for (iter <- 1 to 10) {
+ val start = System.currentTimeMillis()
+ for (x <- mapped) { x + 2 }
+ val end = System.currentTimeMillis()
+ println("Iteration " + iter + " took " + (end-start) + " ms")
+ }
+ println(s"File contents: ${file.map(s => s.toString).collect().mkString(",")}")
+ println(s"Returned length(s) of: ${file.map(s => s.length).collect().mkString(",")}")
+ // scalastyle:on println
+ spark.stop()
+ }
+}
diff --git a/resource-managers/kubernetes/integration-tests/kerberos-yml/data-populator-deployment.yml b/resource-managers/kubernetes/integration-tests/kerberos-yml/data-populator-deployment.yml
new file mode 100644
index 0000000000000..b87f8ceafe6cf
--- /dev/null
+++ b/resource-managers/kubernetes/integration-tests/kerberos-yml/data-populator-deployment.yml
@@ -0,0 +1,30 @@
+apiVersion: extensions/v1beta1
+kind: Deployment
+metadata:
+ name: data-populator
+spec:
+ replicas: 1
+ template:
+ metadata:
+ annotations:
+ pod.beta.kubernetes.io/hostname: data-populator
+ labels:
+ name: hdfs-data-populator
+ kerberosService: data-populator
+ job: kerberostest
+ spec:
+ containers:
+ - command:
+ - /populate-data.sh
+ name: data-populator
+ image: ifilonenko/hadoop-base:latest
+ imagePullPolicy: IfNotPresent
+ runAsNonRoot: false
+ volumeMounts:
+ - mountPath: /var/keytabs
+ name: data-populator-keytab
+ restartPolicy: Always
+ volumes:
+ - name: data-populator-keytab
+ persistentVolumeClaim:
+ claimName: server-keytab
\ No newline at end of file
diff --git a/resource-managers/kubernetes/integration-tests/kerberos-yml/data-populator-service.yml b/resource-managers/kubernetes/integration-tests/kerberos-yml/data-populator-service.yml
new file mode 100644
index 0000000000000..45b4bfb4bfeb5
--- /dev/null
+++ b/resource-managers/kubernetes/integration-tests/kerberos-yml/data-populator-service.yml
@@ -0,0 +1,17 @@
+apiVersion: v1
+kind: Service
+metadata:
+ annotations:
+ service.alpha.kubernetes.io/tolerate-unready-endpoints: "true"
+ labels:
+ kerberosService: data-populator
+ job: kerberostest
+ name: data-populator
+spec:
+ clusterIP: None
+ ports:
+ - protocol: TCP
+ port: 55555
+ targetPort: 0
+ selector:
+ kerberosService: data-populator
\ No newline at end of file
diff --git a/resource-managers/kubernetes/integration-tests/kerberos-yml/dn1-deployment.yml b/resource-managers/kubernetes/integration-tests/kerberos-yml/dn1-deployment.yml
new file mode 100644
index 0000000000000..9f028d31e4424
--- /dev/null
+++ b/resource-managers/kubernetes/integration-tests/kerberos-yml/dn1-deployment.yml
@@ -0,0 +1,30 @@
+apiVersion: extensions/v1beta1
+kind: Deployment
+metadata:
+ name: dn1
+spec:
+ replicas: 1
+ template:
+ metadata:
+ annotations:
+ pod.beta.kubernetes.io/hostname: dn1
+ labels:
+ name: hdfs-dn1
+ kerberosService: dn1
+ job: kerberostest
+ spec:
+ containers:
+ - command:
+ - /start-datanode.sh
+ name: dn1
+ image: ifilonenko/hadoop-base:latest
+ imagePullPolicy: IfNotPresent
+ runAsNonRoot: false
+ volumeMounts:
+ - mountPath: /var/keytabs
+ name: dn1-keytab
+ restartPolicy: Always
+ volumes:
+ - name: dn1-keytab
+ persistentVolumeClaim:
+ claimName: server-keytab
\ No newline at end of file
diff --git a/resource-managers/kubernetes/integration-tests/kerberos-yml/dn1-service.yml b/resource-managers/kubernetes/integration-tests/kerberos-yml/dn1-service.yml
new file mode 100644
index 0000000000000..ee258c87942f8
--- /dev/null
+++ b/resource-managers/kubernetes/integration-tests/kerberos-yml/dn1-service.yml
@@ -0,0 +1,17 @@
+apiVersion: v1
+kind: Service
+metadata:
+ annotations:
+ service.alpha.kubernetes.io/tolerate-unready-endpoints: "true"
+ labels:
+ kerberosService: dn1
+ job: kerberostest
+ name: dn1
+spec:
+ clusterIP: None
+ ports:
+ - protocol: TCP
+ port: 55555
+ targetPort: 0
+ selector:
+ kerberosService: dn1
\ No newline at end of file
diff --git a/resource-managers/kubernetes/integration-tests/kerberos-yml/kerberos-deployment.yml b/resource-managers/kubernetes/integration-tests/kerberos-yml/kerberos-deployment.yml
new file mode 100644
index 0000000000000..6037a3c50b11e
--- /dev/null
+++ b/resource-managers/kubernetes/integration-tests/kerberos-yml/kerberos-deployment.yml
@@ -0,0 +1,30 @@
+apiVersion: extensions/v1beta1
+kind: Deployment
+metadata:
+ name: kerberos
+spec:
+ replicas: 1
+ template:
+ metadata:
+ annotations:
+ pod.beta.kubernetes.io/hostname: kerberos
+ labels:
+ name: hdfs-kerberos
+ kerberosService: kerberos
+ job: kerberostest
+ spec:
+ containers:
+ - command:
+ - /start-kdc.sh
+ name: kerberos
+ image: ifilonenko/hadoop-base:latest
+ imagePullPolicy: IfNotPresent
+ runAsNonRoot: false
+ volumeMounts:
+ - mountPath: /var/keytabs
+ name: kerb-keytab
+ restartPolicy: Always
+ volumes:
+ - name: kerb-keytab
+ persistentVolumeClaim:
+ claimName: server-keytab
\ No newline at end of file
diff --git a/resource-managers/kubernetes/integration-tests/kerberos-yml/kerberos-service.yml b/resource-managers/kubernetes/integration-tests/kerberos-yml/kerberos-service.yml
new file mode 100644
index 0000000000000..da7b994f6e2af
--- /dev/null
+++ b/resource-managers/kubernetes/integration-tests/kerberos-yml/kerberos-service.yml
@@ -0,0 +1,17 @@
+apiVersion: v1
+kind: Service
+metadata:
+ annotations:
+ service.alpha.kubernetes.io/tolerate-unready-endpoints: "true"
+ labels:
+ kerberosService: kerberos
+ job: kerberostest
+ name: kerberos
+spec:
+ clusterIP: None
+ ports:
+ - protocol: TCP
+ port: 55555
+ targetPort: 0
+ selector:
+ kerberosService: kerberos
diff --git a/resource-managers/kubernetes/integration-tests/kerberos-yml/kerberos-test.yml b/resource-managers/kubernetes/integration-tests/kerberos-yml/kerberos-test.yml
new file mode 100644
index 0000000000000..9115b45cc0135
--- /dev/null
+++ b/resource-managers/kubernetes/integration-tests/kerberos-yml/kerberos-test.yml
@@ -0,0 +1,25 @@
+apiVersion: extensions/v1beta1
+kind: Deployment
+metadata:
+ name: kerberos-test
+spec:
+ replicas: 1
+ template:
+ metadata:
+ labels:
+ name: kerberos-test
+ spec:
+ containers:
+ - command: ["/bin/bash"]
+ args: ["/opt/spark/test-env.sh"]
+ name: kerberos-test
+ image: kerberos-test:latest
+ imagePullPolicy: IfNotPresent
+ volumeMounts:
+ - mountPath: /var/keytabs
+ name: kerberos-test-keytab
+ restartPolicy: Always
+ volumes:
+ - name: kerberos-test-keytab
+ persistentVolumeClaim:
+ claimName: server-keytab
\ No newline at end of file
diff --git a/resource-managers/kubernetes/integration-tests/kerberos-yml/kerberos-test2.yml b/resource-managers/kubernetes/integration-tests/kerberos-yml/kerberos-test2.yml
new file mode 100644
index 0000000000000..3aaa442bbe3d8
--- /dev/null
+++ b/resource-managers/kubernetes/integration-tests/kerberos-yml/kerberos-test2.yml
@@ -0,0 +1,25 @@
+apiVersion: extensions/v1beta1
+kind: Deployment
+metadata:
+ name: kerberos-test
+spec:
+ replicas: 1
+ template:
+ metadata:
+ labels:
+ name: kerberos-test
+ spec:
+ containers:
+ - command: ["/bin/bash"]
+ args: ["/opt/spark/test-env2.sh"]
+ name: kerberos-test
+ image: kerberos-test:latest
+ imagePullPolicy: IfNotPresent
+ volumeMounts:
+ - mountPath: /var/keytabs
+ name: kerberos-test-keytab
+ restartPolicy: Always
+ volumes:
+ - name: kerberos-test-keytab
+ persistentVolumeClaim:
+ claimName: server-keytab
\ No newline at end of file
diff --git a/resource-managers/kubernetes/integration-tests/kerberos-yml/nn-deployment.yml b/resource-managers/kubernetes/integration-tests/kerberos-yml/nn-deployment.yml
new file mode 100644
index 0000000000000..d2f473aa66c1a
--- /dev/null
+++ b/resource-managers/kubernetes/integration-tests/kerberos-yml/nn-deployment.yml
@@ -0,0 +1,35 @@
+apiVersion: extensions/v1beta1
+kind: Deployment
+metadata:
+ name: nn
+spec:
+ replicas: 1
+ template:
+ metadata:
+ annotations:
+ pod.beta.kubernetes.io/hostname: nn
+ labels:
+ name: hdfs-nn
+ kerberosService: nn
+ job: kerberostest
+ spec:
+ containers:
+ - command:
+ - /start-namenode.sh
+ name: nn
+ ports:
+ - containerPort: 9000
+ image: ifilonenko/hadoop-base:latest
+ imagePullPolicy: IfNotPresent
+ runAsNonRoot: false
+ volumeMounts:
+ - mountPath: /var/keytabs
+ name: nn-keytab
+ restartPolicy: Always
+ volumes:
+ - name: nn-keytab
+ persistentVolumeClaim:
+ claimName: server-keytab
+ - name: nn-hadoop
+ persistentVolumeClaim:
+ claimName: nn-hadoop
diff --git a/resource-managers/kubernetes/integration-tests/kerberos-yml/nn-hadoop.yml b/resource-managers/kubernetes/integration-tests/kerberos-yml/nn-hadoop.yml
new file mode 100644
index 0000000000000..3e72046cd1d8f
--- /dev/null
+++ b/resource-managers/kubernetes/integration-tests/kerberos-yml/nn-hadoop.yml
@@ -0,0 +1,12 @@
+apiVersion: v1
+kind: PersistentVolumeClaim
+metadata:
+ name: nn-hadoop
+ labels:
+ job: kerberostest
+spec:
+ accessModes:
+ - ReadWriteOnce
+ resources:
+ requests:
+ storage: 100Mi
\ No newline at end of file
diff --git a/resource-managers/kubernetes/integration-tests/kerberos-yml/nn-service.yml b/resource-managers/kubernetes/integration-tests/kerberos-yml/nn-service.yml
new file mode 100644
index 0000000000000..649302150aa39
--- /dev/null
+++ b/resource-managers/kubernetes/integration-tests/kerberos-yml/nn-service.yml
@@ -0,0 +1,17 @@
+apiVersion: v1
+kind: Service
+metadata:
+ annotations:
+ service.alpha.kubernetes.io/tolerate-unready-endpoints: "true"
+ labels:
+ kerberosService: nn
+ job: kerberostest
+ name: nn
+spec:
+ clusterIP: None
+ ports:
+ - protocol: TCP
+ port: 9000
+ targetPort: 9000
+ selector:
+ kerberosService: nn
diff --git a/resource-managers/kubernetes/integration-tests/kerberos-yml/server-keytab.yml b/resource-managers/kubernetes/integration-tests/kerberos-yml/server-keytab.yml
new file mode 100644
index 0000000000000..6c6a1008e1441
--- /dev/null
+++ b/resource-managers/kubernetes/integration-tests/kerberos-yml/server-keytab.yml
@@ -0,0 +1,12 @@
+apiVersion: v1
+kind: PersistentVolumeClaim
+metadata:
+ name: server-keytab
+ labels:
+ job: kerberostest
+spec:
+ accessModes:
+ - ReadWriteOnce
+ resources:
+ requests:
+ storage: 100Mi
\ No newline at end of file
diff --git a/resource-managers/kubernetes/integration-tests/kerberos-yml/test-env.sh b/resource-managers/kubernetes/integration-tests/kerberos-yml/test-env.sh
new file mode 100644
index 0000000000000..056f0c5936e98
--- /dev/null
+++ b/resource-managers/kubernetes/integration-tests/kerberos-yml/test-env.sh
@@ -0,0 +1,27 @@
+#!/usr/bin/env bash
+sed -i -e 's/#//' -e 's/default_ccache_name/# default_ccache_name/' /etc/krb5.conf
+export HADOOP_OPTS="-Djava.net.preferIPv4Stack=true -Dsun.security.krb5.debug=true"
+export HADOOP_JAAS_DEBUG=true
+export HADOOP_ROOT_LOGGER=DEBUG,console
+cp ${TMP_KRB_LOC} /etc/krb5.conf
+cp ${TMP_CORE_LOC} /opt/spark/hconf/core-site.xml
+cp ${TMP_HDFS_LOC} /opt/spark/hconf/hdfs-site.xml
+mkdir -p /etc/krb5.conf.d
+until /usr/bin/kinit -kt /var/keytabs/hdfs.keytab hdfs/nn.${NAMESPACE}.svc.cluster.local; do sleep 15; done
+/opt/spark/bin/spark-submit \
+ --deploy-mode cluster \
+ --class ${CLASS_NAME} \
+ --master k8s://${MASTER_URL} \
+ --kubernetes-namespace ${NAMESPACE} \
+ --conf spark.executor.instances=1 \
+ --conf spark.app.name=spark-hdfs \
+ --conf spark.driver.extraClassPath=/opt/spark/hconf/core-site.xml:/opt/spark/hconf/hdfs-site.xml:/opt/spark/hconf/yarn-site.xml:/etc/krb5.conf \
+ --conf spark.kubernetes.driver.docker.image=spark-driver:latest \
+ --conf spark.kubernetes.executor.docker.image=spark-executor:latest \
+ --conf spark.kubernetes.initcontainer.docker.image=spark-init:latest \
+ --conf spark.kubernetes.kerberos.enabled=true \
+ --conf spark.kubernetes.kerberos.keytab=/var/keytabs/hdfs.keytab \
+ --conf spark.kubernetes.kerberos.principal=hdfs/nn.${NAMESPACE}.svc.cluster.local@CLUSTER.LOCAL \
+ --conf spark.kubernetes.driver.label.spark-app-locator=${APP_LOCATOR_LABEL} \
+ ${SUBMIT_RESOURCE} \
+ hdfs://nn.${NAMESPACE}.svc.cluster.local:9000/user/ifilonenko/wordcount.txt
\ No newline at end of file
diff --git a/resource-managers/kubernetes/integration-tests/pom.xml b/resource-managers/kubernetes/integration-tests/pom.xml
index fc28026de1e43..51eb884784a3c 100644
--- a/resource-managers/kubernetes/integration-tests/pom.xml
+++ b/resource-managers/kubernetes/integration-tests/pom.xml
@@ -92,6 +92,7 @@
com.spotify
docker-client
+ 8.8.2
test
NN --> DN1 --> Data-Populator
+ val podWatcherCache = new KerberosPodWatcherCache(kerberosUtils, LABELS)
+ podWatcherCache.start()
+ val dpNode = podWatcherCache.stop()
+ while (!podWatcherCache.hasInLogs(dpNode, "")) {
+ logInfo("Waiting for data-populator to be formatted")
+ Thread.sleep(500)
+ }
+ }
+}
diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KerberosTestPodLauncher.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KerberosTestPodLauncher.scala
new file mode 100644
index 0000000000000..53d372faace41
--- /dev/null
+++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KerberosTestPodLauncher.scala
@@ -0,0 +1,121 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.deploy.kubernetes.integrationtest
+
+import java.io.{File, FileInputStream}
+
+import scala.collection.JavaConverters._
+
+import io.fabric8.kubernetes.api.model.KeyToPathBuilder
+import io.fabric8.kubernetes.api.model.extensions.{Deployment, DeploymentBuilder}
+import io.fabric8.kubernetes.client.KubernetesClient
+
+import org.apache.spark.deploy.kubernetes.submit.ContainerNameEqualityPredicate
+
+ /**
+ * This class is responsible for launching a pod that runs spark-submit to simulate
+ * the necessary global environmental variables and files expected for a Kerberos task.
+ * In this test we specify HADOOP_CONF_DIR and ensure that for any arbitrary namespace
+ * the krb5.conf, core-site.xml, and hdfs-site.xml are resolved accordingly.
+ */
+private[spark] class KerberosTestPodLauncher(
+ kubernetesClient: KubernetesClient,
+ namespace: String) {
+ private val kerberosFiles = Seq("krb5.conf", "core-site.xml", "hdfs-site.xml")
+ private val KRB_VOLUME = "krb5-conf"
+ private val KRB_FILE_DIR = "/tmp"
+ private val KRB_CONFIG_MAP_NAME = "krb-config-map"
+ private val HADOOP_CONF_DIR_PATH = "/opt/spark/hconf"
+ private val keyPaths = kerberosFiles.map(file =>
+ new KeyToPathBuilder()
+ .withKey(file)
+ .withPath(file)
+ .build()).toList
+ def startKerberosTest(
+ resource: String,
+ className: String,
+ appLabel: String,
+ yamlLocation: String): Unit = {
+ kubernetesClient.load(new FileInputStream(new File(yamlLocation)))
+ .get().get(0) match {
+ case deployment: Deployment =>
+ val deploymentWithEnv: Deployment = new DeploymentBuilder(deployment)
+ .editSpec()
+ .editTemplate()
+ .editSpec()
+ .addNewVolume()
+ .withName(KRB_VOLUME)
+ .withNewConfigMap()
+ .withName(KRB_CONFIG_MAP_NAME)
+ .withItems(keyPaths.asJava)
+ .endConfigMap()
+ .endVolume()
+ .editMatchingContainer(new ContainerNameEqualityPredicate(
+ deployment.getMetadata.getName))
+ .addNewEnv()
+ .withName("NAMESPACE")
+ .withValue(namespace)
+ .endEnv()
+ .addNewEnv()
+ .withName("MASTER_URL")
+ .withValue(kubernetesClient.getMasterUrl.toString)
+ .endEnv()
+ .addNewEnv()
+ .withName("SUBMIT_RESOURCE")
+ .withValue(resource)
+ .endEnv()
+ .addNewEnv()
+ .withName("CLASS_NAME")
+ .withValue(className)
+ .endEnv()
+ .addNewEnv()
+ .withName("HADOOP_CONF_DIR")
+ .withValue(HADOOP_CONF_DIR_PATH)
+ .endEnv()
+ .addNewEnv()
+ .withName("APP_LOCATOR_LABEL")
+ .withValue(appLabel)
+ .endEnv()
+ .addNewEnv()
+ .withName("SPARK_PRINT_LAUNCH_COMMAND")
+ .withValue("true")
+ .endEnv()
+ .addNewEnv()
+ .withName("TMP_KRB_LOC")
+ .withValue(s"$KRB_FILE_DIR/${kerberosFiles.head}")
+ .endEnv()
+ .addNewEnv()
+ .withName("TMP_CORE_LOC")
+ .withValue(s"$KRB_FILE_DIR/${kerberosFiles(1)}")
+ .endEnv()
+ .addNewEnv()
+ .withName("TMP_HDFS_LOC")
+ .withValue(s"$KRB_FILE_DIR/${kerberosFiles(2)}")
+ .endEnv()
+ .addNewVolumeMount()
+ .withName(KRB_VOLUME)
+ .withMountPath(KRB_FILE_DIR)
+ .endVolumeMount()
+ .endContainer()
+ .endSpec()
+ .endTemplate()
+ .endSpec()
+ .build()
+ kubernetesClient.extensions().deployments()
+ .inNamespace(namespace).create(deploymentWithEnv)}
+ }
+}
diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KubernetesSuite.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KubernetesSuite.scala
index e204d0173aff8..17d48b8b508dd 100644
--- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KubernetesSuite.scala
+++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KubernetesSuite.scala
@@ -34,6 +34,7 @@ import org.apache.spark.deploy.kubernetes.config._
import org.apache.spark.deploy.kubernetes.integrationtest.backend.IntegrationTestBackendFactory
import org.apache.spark.deploy.kubernetes.integrationtest.backend.minikube.Minikube
import org.apache.spark.deploy.kubernetes.integrationtest.constants.MINIKUBE_TEST_BACKEND
+import org.apache.spark.deploy.kubernetes.integrationtest.kerberos.KerberosDriverWatcherCache
import org.apache.spark.deploy.kubernetes.submit.{Client, ClientArguments, JavaMainAppResource, KeyAndCertPem, MainAppResource, PythonMainAppResource}
import org.apache.spark.launcher.SparkLauncher
import org.apache.spark.util.Utils
@@ -46,6 +47,8 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter {
private var sparkConf: SparkConf = _
private var resourceStagingServerLauncher: ResourceStagingServerLauncher = _
private var staticAssetServerLauncher: StaticAssetServerLauncher = _
+ private var kerberizedHadoopClusterLauncher: KerberizedHadoopClusterLauncher = _
+ private var kerberosTestLauncher: KerberosTestPodLauncher = _
override def beforeAll(): Unit = {
testBackend.initialize()
@@ -54,6 +57,12 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter {
kubernetesTestComponents.kubernetesClient.inNamespace(kubernetesTestComponents.namespace))
staticAssetServerLauncher = new StaticAssetServerLauncher(
kubernetesTestComponents.kubernetesClient.inNamespace(kubernetesTestComponents.namespace))
+ kerberizedHadoopClusterLauncher = new KerberizedHadoopClusterLauncher(
+ kubernetesTestComponents.kubernetesClient.inNamespace(kubernetesTestComponents.namespace),
+ kubernetesTestComponents.namespace)
+ kerberosTestLauncher = new KerberosTestPodLauncher(
+ kubernetesTestComponents.kubernetesClient.inNamespace(kubernetesTestComponents.namespace),
+ kubernetesTestComponents.namespace)
}
override def afterAll(): Unit = {
@@ -69,12 +78,59 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter {
}
after {
+ kubernetesTestComponents.deleteKubernetesPVs()
kubernetesTestComponents.deleteNamespace()
}
- test("Run PySpark Job on file from SUBMITTER with --py-files") {
+ test("Include HADOOP_CONF for HDFS based jobs") {
assume(testBackend.name == MINIKUBE_TEST_BACKEND)
+ // Ensuring that HADOOP_CONF_DIR variable is set
+ sparkConf.setJars(Seq(CONTAINER_LOCAL_HELPER_JAR_PATH))
+ runSparkApplicationAndVerifyCompletion(
+ JavaMainAppResource(CONTAINER_LOCAL_MAIN_APP_RESOURCE),
+ SPARK_PI_MAIN_CLASS,
+ Seq("HADOOP_CONF_DIR defined. Mounting HDFS specific .xml files", "Pi is roughly 3"),
+ Array("5"),
+ Seq.empty[String],
+ Some("test-data/simple-hadoop-conf"))
+ }
+ test("Secure HDFS test with HDFS keytab") {
+ assume(testBackend.name == MINIKUBE_TEST_BACKEND)
+ launchKerberizedCluster()
+ createKerberosTestPod(
+ CONTAINER_LOCAL_MAIN_APP_RESOURCE,
+ HDFS_TEST_CLASS,
+ APP_LOCATOR_LABEL,
+ "kerberos-yml/kerberos-test.yml")
+ val kubernetesClient = kubernetesTestComponents.kubernetesClient
+ val driverWatcherCache = new KerberosDriverWatcherCache(
+ kubernetesClient,
+ Map("spark-app-locator" -> APP_LOCATOR_LABEL))
+ driverWatcherCache.start()
+ driverWatcherCache.stop()
+ val expectedLogOnCompletion = Seq(
+ "Returned length(s) of: 1",
+ "File contents: [This is an awesome word count file]")
+ val driverPod = kubernetesClient
+ .pods()
+ .withLabel("spark-app-locator", APP_LOCATOR_LABEL)
+ .list()
+ .getItems
+ .get(0)
+ Eventually.eventually(TIMEOUT, INTERVAL) {
+ expectedLogOnCompletion.foreach { e =>
+ assert(kubernetesClient
+ .pods()
+ .withName(driverPod.getMetadata.getName)
+ .getLog
+ .contains(e), "The application did not complete.")
+ }
+ }
+ }
+
+ test("Run PySpark Job on file from SUBMITTER with --py-files") {
+ assume(testBackend.name == MINIKUBE_TEST_BACKEND)
launchStagingServer(SSLOptions(), None)
sparkConf
.set(DRIVER_DOCKER_IMAGE,
@@ -103,7 +159,6 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter {
test("Simple submission test with the resource staging server.") {
assume(testBackend.name == MINIKUBE_TEST_BACKEND)
-
launchStagingServer(SSLOptions(), None)
runSparkPiAndVerifyCompletion(SUBMITTER_LOCAL_MAIN_APP_RESOURCE)
}
@@ -159,7 +214,8 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter {
GROUP_BY_MAIN_CLASS,
Seq("The Result is"),
Array.empty[String],
- Seq.empty[String])
+ Seq.empty[String],
+ None)
}
test("Use remote resources without the resource staging server.") {
@@ -226,7 +282,8 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter {
s"File found on the executors at the relative path ${testExistenceFile.getName} with" +
s" the correct contents."),
Array(testExistenceFile.getName, TEST_EXISTENCE_FILE_CONTENTS),
- Seq.empty[String])
+ Seq.empty[String],
+ None)
}
test("Setting JVM options on the driver and executors with spaces.") {
@@ -256,7 +313,8 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter {
JAVA_OPTIONS_MAIN_CLASS,
Seq(s"All expected JVM options were present on the driver and executors."),
Array(driverJvmOptionsFile.getName, executorJvmOptionsFile.getName),
- Seq.empty[String])
+ Seq.empty[String],
+ None)
}
test("Submit small local files without the resource staging server.") {
@@ -274,7 +332,8 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter {
s"File found on the executors at the relative path ${testExistenceFile.getName} with" +
s" the correct contents."),
Array(testExistenceFile.getName, TEST_EXISTENCE_FILE_CONTENTS),
- Seq.empty[String])
+ Seq.empty[String],
+ None)
}
test("Use a very long application name.") {
@@ -297,16 +356,33 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter {
}
sparkConf.set(RESOURCE_STAGING_SERVER_URI,
s"$resourceStagingServerUriScheme://" +
- s"${Minikube.getMinikubeIp}:$resourceStagingServerPort")
+ s"${Minikube.getMinikubeIp}:$resourceStagingServerPort").set(KERBEROS_CONF, "simple")
+ }
+
+ private def launchKerberizedCluster(): Unit = {
+ assume(testBackend.name == MINIKUBE_TEST_BACKEND)
+ kerberizedHadoopClusterLauncher.launchKerberizedCluster()
+ }
+
+ private def createKerberosTestPod(
+ resource: String,
+ className: String,
+ appLabel: String,
+ yamlLocation: String): Unit = {
+ assume(testBackend.name == MINIKUBE_TEST_BACKEND)
+ kerberosTestLauncher.startKerberosTest(resource, className, appLabel, yamlLocation)
}
private def runSparkPiAndVerifyCompletion(appResource: String): Unit = {
runSparkApplicationAndVerifyCompletion(
JavaMainAppResource(appResource),
SPARK_PI_MAIN_CLASS,
- Seq("Pi is roughly 3"),
+ Seq(
+ "hadoop config map key was not specified",
+ "Pi is roughly 3"),
Array.empty[String],
- Seq.empty[String])
+ Seq.empty[String],
+ None)
}
private def runPySparkPiAndVerifyCompletion(
@@ -316,7 +392,8 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter {
PYSPARK_PI_MAIN_CLASS,
Seq("Submitting 5 missing tasks from ResultStage", "Pi is roughly 3"),
Array("5"),
- otherPyFiles)
+ otherPyFiles,
+ None)
}
private def runSparkApplicationAndVerifyCompletion(
@@ -324,13 +401,14 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter {
mainClass: String,
expectedLogOnCompletion: Seq[String],
appArgs: Array[String],
- otherPyFiles: Seq[String]): Unit = {
+ otherPyFiles: Seq[String],
+ hadoopConfDir: Option[String]): Unit = {
val clientArguments = ClientArguments(
mainAppResource = appResource,
mainClass = mainClass,
driverArgs = appArgs,
otherPyFiles = otherPyFiles)
- Client.run(sparkConf, clientArguments)
+ Client.run(sparkConf, clientArguments, hadoopConfDir)
val driverPod = kubernetesTestComponents.kubernetesClient
.pods()
.withLabel("spark-app-locator", APP_LOCATOR_LABEL)
@@ -419,8 +497,8 @@ private[spark] object KubernetesSuite {
s"integration-tests-jars/${EXAMPLES_JAR_FILE.getName}"
val CONTAINER_LOCAL_HELPER_JAR_PATH = s"local:///opt/spark/examples/" +
s"integration-tests-jars/${HELPER_JAR_FILE.getName}"
- val TIMEOUT = PatienceConfiguration.Timeout(Span(2, Minutes))
- val INTERVAL = PatienceConfiguration.Interval(Span(2, Seconds))
+ val TIMEOUT = PatienceConfiguration.Timeout(Span(10, Minutes))
+ val INTERVAL = PatienceConfiguration.Interval(Span(10, Seconds))
val SPARK_PI_MAIN_CLASS = "org.apache.spark.deploy.kubernetes" +
".integrationtest.jobs.SparkPiWithInfiniteWait"
val PYSPARK_PI_MAIN_CLASS = "org.apache.spark.deploy.PythonRunner"
@@ -433,9 +511,13 @@ private[spark] object KubernetesSuite {
".integrationtest.jobs.FileExistenceTest"
val GROUP_BY_MAIN_CLASS = "org.apache.spark.deploy.kubernetes" +
".integrationtest.jobs.GroupByTest"
+ val HDFS_TEST_CLASS = "org.apache.spark.deploy.kubernetes" +
+ ".integrationtest.jobs.HDFSTest"
val JAVA_OPTIONS_MAIN_CLASS = "org.apache.spark.deploy.kubernetes" +
".integrationtest.jobs.JavaOptionsTest"
val TEST_EXISTENCE_FILE_CONTENTS = "contents"
+ val KERBEROS_CONF = "spark.hadoop.hadoop.security.authentication"
+
case object ShuffleNotReadyException extends Exception
}
diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KubernetesTestComponents.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KubernetesTestComponents.scala
index 0ca1f482269db..c04bb0c525d86 100644
--- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KubernetesTestComponents.scala
+++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KubernetesTestComponents.scala
@@ -38,7 +38,21 @@ private[spark] class KubernetesTestComponents(defaultClient: DefaultKubernetesCl
.endMetadata()
.done()
}
-
+ def deleteKubernetesPVs(): Unit = {
+ // Temporary hack until client library for fabric8 is updated to get around
+ // the NPE that comes about when I do .list().getItems().asScala
+ try {
+ val pvList = kubernetesClient.persistentVolumes().list().getItems().asScala
+ if (pvList.nonEmpty) {
+ kubernetesClient.persistentVolumes().delete()
+ Eventually.eventually(KubernetesSuite.TIMEOUT, KubernetesSuite.INTERVAL) {
+ require(!pvList.exists(_.getMetadata.getNamespace == namespace))
+ }
+ }
+ } catch {
+ case ex: java.lang.NullPointerException =>
+ }
+ }
def deleteNamespace(): Unit = {
defaultClient.namespaces.withName(namespace).delete()
Eventually.eventually(KubernetesSuite.TIMEOUT, KubernetesSuite.INTERVAL) {
diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/docker/SparkDockerImageBuilder.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/docker/SparkDockerImageBuilder.scala
index e240fcf953f8c..a210aab4b78fe 100644
--- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/docker/SparkDockerImageBuilder.scala
+++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/docker/SparkDockerImageBuilder.scala
@@ -31,7 +31,6 @@ import org.apache.spark.internal.Logging
import org.apache.spark.util.RedirectThread
-
private[spark] class SparkDockerImageBuilder
(private val dockerEnv: Map[String, String]) extends Logging{
@@ -47,6 +46,7 @@ private[spark] class SparkDockerImageBuilder
private val STAGING_SERVER_DOCKER_FILE = "dockerfiles/resource-staging-server/Dockerfile"
private val STATIC_ASSET_SERVER_DOCKER_FILE =
"dockerfiles/integration-test-asset-server/Dockerfile"
+ private val KERBEROS_DOCKER_FILE = "dockerfiles/kerberos-test/Dockerfile"
private val TIMEOUT = PatienceConfiguration.Timeout(Span(2, Minutes))
private val INTERVAL = PatienceConfiguration.Interval(Span(2, Seconds))
private val dockerHost = dockerEnv.getOrElse("DOCKER_HOST",
@@ -95,6 +95,7 @@ private[spark] class SparkDockerImageBuilder
buildImage("spark-resource-staging-server", STAGING_SERVER_DOCKER_FILE)
buildImage("spark-init", INIT_CONTAINER_DOCKER_FILE)
buildImage("spark-integration-test-asset-server", STATIC_ASSET_SERVER_DOCKER_FILE)
+ buildImage("kerberos-test", KERBEROS_DOCKER_FILE)
}
private def buildImage(name: String, dockerFile: String): Unit = {
@@ -103,5 +104,6 @@ private[spark] class SparkDockerImageBuilder
name,
dockerFile,
new LoggingBuildHandler())
+ logInfo(s"Built docker image for $name")
}
}
diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/kerberos/KerberosCMWatcherCache.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/kerberos/KerberosCMWatcherCache.scala
new file mode 100644
index 0000000000000..59968534c8312
--- /dev/null
+++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/kerberos/KerberosCMWatcherCache.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.kubernetes.integrationtest.kerberos
+
+import java.util.concurrent.locks.{Condition, Lock, ReentrantLock}
+
+import scala.collection.JavaConverters._
+
+import io.fabric8.kubernetes.api.model.ConfigMap
+import io.fabric8.kubernetes.client.{KubernetesClientException, Watch, Watcher}
+import io.fabric8.kubernetes.client.Watcher.Action
+
+import org.apache.spark.internal.Logging
+
+ /**
+ * This class is responsible for ensuring that no logic progresses in the cluster launcher
+ * until a configmap with the HADOOP_CONF_DIR specifications has been created.
+ */
+private[spark] class KerberosCMWatcherCache(kerberosUtils: KerberosUtils) extends Logging {
+ private val kubernetesClient = kerberosUtils.getClient
+ private val namespace = kerberosUtils.getNamespace
+ private val requiredFiles = Seq("core-site.xml", "hdfs-site.xml", "krb5.conf")
+ private var watcher: Watch = _
+ private var cmCache = scala.collection.mutable.Map[String, Map[String, String]]()
+ private var lock: Lock = new ReentrantLock()
+ private var cmCreated: Condition = lock.newCondition()
+ private val configMap = kerberosUtils.getConfigMap
+ private val configMapName = configMap.getMetadata.getName
+ private val blockingThread = new Thread(new Runnable {
+ override def run(): Unit = {
+ logInfo("Beginning of ConfigMap lock")
+ lock.lock()
+ try {
+ while (!created()) cmCreated.await()
+ } finally {
+ logInfo("Ending the ConfigMap lock")
+ lock.unlock()
+ stop()
+ }
+ }})
+
+ private val watcherThread = new Thread(new Runnable {
+ override def run(): Unit = {
+ logInfo("Beginning the watch of the Kerberos Config Map")
+ watcher = kubernetesClient
+ .configMaps()
+ .withName(configMapName)
+ .watch(new Watcher[ConfigMap] {
+ override def onClose(cause: KubernetesClientException): Unit =
+ logInfo("Ending the watch of Kerberos Config Map")
+ override def eventReceived(action: Watcher.Action, resource: ConfigMap): Unit = {
+ val name = resource.getMetadata.getName
+ action match {
+ case Action.DELETED | Action.ERROR =>
+ logInfo(s"$name either deleted or error")
+ cmCache.remove(name)
+ case Action.ADDED | Action.MODIFIED =>
+ val data = resource.getData.asScala.toMap
+ logInfo(s"$name includes ${data.keys.mkString(",")}")
+ cmCache(name) = data
+ if (created()) {
+ lock.lock()
+ try {
+ cmCreated.signalAll()
+ } finally {
+ lock.unlock()
+ }
+ }
+ }}}
+ )
+ logInfo("Launching the Config Map")
+ kerberosUtils.getClient.configMaps().inNamespace(namespace).createOrReplace(configMap)
+ }})
+
+ def start(): Unit = {
+ blockingThread.start()
+ watcherThread.start()
+ blockingThread.join()
+ watcherThread.join()}
+
+ def stop(): Unit = {
+ watcher.close()
+ }
+
+ def created(): Boolean = {
+ cmCache.get(configMapName).exists{ data =>
+ requiredFiles.forall(data.keys.toSeq.contains)}
+ }
+}
diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/kerberos/KerberosDeployment.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/kerberos/KerberosDeployment.scala
new file mode 100644
index 0000000000000..a60011153f5d5
--- /dev/null
+++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/kerberos/KerberosDeployment.scala
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.deploy.kubernetes.integrationtest.kerberos
+
+import io.fabric8.kubernetes.api.model.Service
+import io.fabric8.kubernetes.api.model.extensions.Deployment
+
+private[spark] case class KerberosDeployment(
+ podDeployment: Deployment,
+ service: Service)
diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/kerberos/KerberosDriverWatcherCache.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/kerberos/KerberosDriverWatcherCache.scala
new file mode 100644
index 0000000000000..b7a2176194e2c
--- /dev/null
+++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/kerberos/KerberosDriverWatcherCache.scala
@@ -0,0 +1,99 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.deploy.kubernetes.integrationtest.kerberos
+
+import java.util.concurrent.locks.{Condition, Lock, ReentrantLock}
+
+import scala.collection.JavaConverters._
+
+import io.fabric8.kubernetes.api.model.Pod
+import io.fabric8.kubernetes.client.{KubernetesClient, KubernetesClientException, Watch, Watcher}
+import io.fabric8.kubernetes.client.Watcher.Action
+
+import org.apache.spark.internal.Logging
+
+ /**
+ * This class is responsible for ensuring that the driver-pod launched by the KerberosTestPod
+ * is running before trying to grab its logs for the sake of monitoring success of completition.
+ */
+private[spark] class KerberosDriverWatcherCache(
+ kubernetesClient: KubernetesClient,
+ labels: Map[String, String]) extends Logging {
+ private var podWatcher: Watch = _
+ private var podCache =
+ scala.collection.mutable.Map[String, String]()
+ private var lock: Lock = new ReentrantLock()
+ private var driverRunning: Condition = lock.newCondition()
+ private var driverIsUp: Boolean = false
+ private val blockingThread = new Thread(new Runnable {
+ override def run(): Unit = {
+ logInfo("Beginning of Driver lock")
+ lock.lock()
+ try {
+ while (!driverIsUp) driverRunning.await()
+ } finally {
+ logInfo("Ending the Driver lock")
+ lock.unlock()
+ stop()
+ }
+ }
+ })
+
+ private val podWatcherThread = new Thread(new Runnable {
+ override def run(): Unit = {
+ logInfo("Beginning the watch of Driver pod")
+ podWatcher = kubernetesClient
+ .pods()
+ .withLabels(labels.asJava)
+ .watch(new Watcher[Pod] {
+ override def onClose(cause: KubernetesClientException): Unit =
+ logInfo("Ending the watch of Driver pod")
+ override def eventReceived(action: Watcher.Action, resource: Pod): Unit = {
+ val name = resource.getMetadata.getName
+ action match {
+ case Action.DELETED | Action.ERROR =>
+ logInfo(s"$name either deleted or error")
+ podCache.remove(name)
+ case Action.ADDED | Action.MODIFIED =>
+ val phase = resource.getStatus.getPhase
+ logInfo(s"$name is as $phase")
+ podCache(name) = phase
+ if (maybeDriverDone(name)) {
+ lock.lock()
+ try {
+ driverIsUp = true
+ driverRunning.signalAll()
+ } finally {
+ lock.unlock()
+ }
+ }}}})
+ }})
+
+ def start(): Unit = {
+ blockingThread.start()
+ podWatcherThread.start()
+ blockingThread.join()
+ podWatcherThread.join()
+ }
+
+ def stop(): Unit = {
+ podWatcher.close()
+ }
+
+ private def maybeDriverDone(name: String): Boolean = podCache.get(name).contains("Running")
+}
diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/kerberos/KerberosPVWatcherCache.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/kerberos/KerberosPVWatcherCache.scala
new file mode 100644
index 0000000000000..b58aa2e0e532e
--- /dev/null
+++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/kerberos/KerberosPVWatcherCache.scala
@@ -0,0 +1,184 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.deploy.kubernetes.integrationtest.kerberos
+
+import java.util.concurrent.locks.{Condition, Lock, ReentrantLock}
+
+import scala.collection.JavaConverters._
+
+import io.fabric8.kubernetes.api.model.{PersistentVolume, PersistentVolumeClaim}
+import io.fabric8.kubernetes.client.{KubernetesClientException, Watch, Watcher}
+import io.fabric8.kubernetes.client.Watcher.Action
+
+import org.apache.spark.internal.Logging
+
+ /**
+ * This class is responsible for ensuring that the persistent volume claims are bounded
+ * to the correct persistent volume and that they are both created before launching the
+ * pods which expect to use them.
+ */
+private[spark] class KerberosPVWatcherCache(
+ kerberosUtils: KerberosUtils,
+ labels: Map[String, String]) extends Logging {
+ private val kubernetesClient = kerberosUtils.getClient
+ private val namespace = kerberosUtils.getNamespace
+ private var pvWatcher: Watch = _
+ private var pvcWatcher: Watch = _
+ private var pvCache =
+ scala.collection.mutable.Map[String, String]()
+ private var pvcCache =
+ scala.collection.mutable.Map[String, String]()
+ private var lock: Lock = new ReentrantLock()
+ private var nnBounded: Condition = lock.newCondition()
+ private var ktBounded: Condition = lock.newCondition()
+ private var nnIsUp: Boolean = false
+ private var ktIsUp: Boolean = false
+ private var nnSpawned: Boolean = false
+ private var ktSpawned: Boolean = false
+ private val blockingThread = new Thread(new Runnable {
+ override def run(): Unit = {
+ logInfo("Beginning of Persistent Storage Lock")
+ lock.lock()
+ try {
+ while (!nnIsUp) nnBounded.await()
+ while (!ktIsUp) ktBounded.await()
+ } finally {
+ logInfo("Ending the Persistent Storage lock")
+ lock.unlock()
+ stop()
+ }
+ }
+ })
+ private val pvWatcherThread = new Thread(new Runnable {
+ override def run(): Unit = {
+ logInfo("Beginning the watch of Persistent Volumes")
+ pvWatcher = kubernetesClient
+ .persistentVolumes()
+ .withLabels(labels.asJava)
+ .watch(new Watcher[PersistentVolume] {
+ override def onClose(cause: KubernetesClientException): Unit =
+ logInfo("Ending the watch of Persistent Volumes", cause)
+ override def eventReceived(action: Watcher.Action, resource: PersistentVolume): Unit = {
+ val name = resource.getMetadata.getName
+ action match {
+ case Action.DELETED | Action.ERROR =>
+ logInfo(s"$name either deleted or error")
+ pvCache.remove(name)
+ case Action.ADDED | Action.MODIFIED =>
+ val phase = resource.getStatus.getPhase
+ logInfo(s"$name is at stage: $phase")
+ pvCache(name) = phase
+ if (maybeDeploymentAndServiceDone(name)) {
+ val modifyAndSignal: Runnable = new MSThread(name)
+ new Thread(modifyAndSignal).start()
+ }}}})
+ }})
+ private val pvcWatcherThread = new Thread(new Runnable {
+ override def run(): Unit = {
+ logInfo("Beginning the watch of Persistent Volume Claims")
+ pvcWatcher = kubernetesClient
+ .persistentVolumeClaims()
+ .withLabels(labels.asJava)
+ .watch(new Watcher[PersistentVolumeClaim] {
+ override def onClose(cause: KubernetesClientException): Unit =
+ logInfo("Ending the watch of Persistent Volume Claims")
+ override def eventReceived(
+ action: Watcher.Action,
+ resource: PersistentVolumeClaim): Unit = {
+ val name = resource.getMetadata.getName
+ action match {
+ case Action.DELETED | Action.ERROR =>
+ logInfo(s"$name either deleted or error")
+ pvcCache.remove(name)
+ case Action.ADDED | Action.MODIFIED =>
+ val volumeName = resource.getSpec.getVolumeName
+ logInfo(s"$name claims itself to $volumeName")
+ pvcCache(name) = volumeName
+ if (maybeDeploymentAndServiceDone(name)) {
+ val modifyAndSignal: Runnable = new MSThread(name)
+ new Thread(modifyAndSignal).start()
+ }}}})
+ logInfo("Launching the Persistent Storage")
+ if (!nnSpawned) {
+ logInfo("Launching the NN Hadoop PV+PVC")
+ nnSpawned = true
+ deploy(kerberosUtils.getNNStorage)
+ }
+ }})
+
+ def start(): Unit = {
+ blockingThread.start()
+ pvWatcherThread.start()
+ pvcWatcherThread.start()
+ blockingThread.join()
+ pvWatcherThread.join()
+ pvcWatcherThread.join()
+ }
+ def stop(): Unit = {
+ pvWatcher.close()
+ pvcWatcher.close()
+ }
+
+ private def maybeDeploymentAndServiceDone(name: String): Boolean = {
+ val finished = pvCache.get(name).contains("Bound") &&
+ pvcCache.get(name).contains(name)
+ if (!finished) {
+ logInfo(s"$name is not bounded")
+ if (name == "nn-hadoop") nnIsUp = false
+ else if (name == "server-keytab") ktIsUp = false
+ }
+ finished
+ }
+
+ private def deploy(kbs: KerberosStorage) : Unit = {
+ kubernetesClient
+ .persistentVolumeClaims().inNamespace(namespace).create(kbs.persistentVolumeClaim)
+ kubernetesClient
+ .persistentVolumes().create(kbs.persistentVolume)
+ }
+
+ private class MSThread(name: String) extends Runnable {
+ override def run(): Unit = {
+ logInfo(s"$name PV and PVC are bounded")
+ lock.lock()
+ if (name == "nn-hadoop") {
+ nnIsUp = true
+ logInfo(s"nn-hadoop is bounded")
+ try {
+ nnBounded.signalAll()
+ } finally {
+ lock.unlock()
+ }
+ if (!ktSpawned) {
+ logInfo("Launching the KT Hadoop PV+PVC")
+ ktSpawned = true
+ deploy(kerberosUtils.getKTStorage)
+ }
+ }
+ else if (name == "server-keytab") {
+ while (!nnIsUp) ktBounded.await()
+ ktIsUp = true
+ logInfo(s"server-keytab is bounded")
+ try {
+ ktBounded.signalAll()
+ } finally {
+ lock.unlock()
+ }
+ }}
+ }
+}
diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/kerberos/KerberosPodWatcherCache.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/kerberos/KerberosPodWatcherCache.scala
new file mode 100644
index 0000000000000..60cc9735212fa
--- /dev/null
+++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/kerberos/KerberosPodWatcherCache.scala
@@ -0,0 +1,256 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.deploy.kubernetes.integrationtest.kerberos
+
+import java.util.concurrent.locks.{Condition, Lock, ReentrantLock}
+
+import scala.collection.JavaConverters._
+
+import io.fabric8.kubernetes.api.model.{Pod, Service}
+import io.fabric8.kubernetes.client.{KubernetesClientException, Watch, Watcher}
+import io.fabric8.kubernetes.client.Watcher.Action
+
+import org.apache.spark.internal.Logging
+
+ /**
+ * This class is used to ensure that the Hadoop cluster that is launched is executed
+ * in this order: KDC --> NN --> DN --> Data-Populator and that each one of these nodes
+ * is running before launching the Kerberos test.
+ */
+private[spark] class KerberosPodWatcherCache(
+ kerberosUtils: KerberosUtils,
+ labels: Map[String, String]) extends Logging {
+ private val kubernetesClient = kerberosUtils.getClient
+ private val namespace = kerberosUtils.getNamespace
+ private var podWatcher: Watch = _
+ private var serviceWatcher: Watch = _
+ private var podCache =
+ scala.collection.mutable.Map[String, String]()
+ private var serviceCache =
+ scala.collection.mutable.Map[String, String]()
+ private var lock: Lock = new ReentrantLock()
+ private var kdcRunning: Condition = lock.newCondition()
+ private var nnRunning: Condition = lock.newCondition()
+ private var dnRunning: Condition = lock.newCondition()
+ private var dpRunning: Condition = lock.newCondition()
+ private var kdcIsUp: Boolean = false
+ private var nnIsUp: Boolean = false
+ private var dnIsUp: Boolean = false
+ private var dpIsUp: Boolean = false
+ private var kdcSpawned: Boolean = false
+ private var nnSpawned: Boolean = false
+ private var dnSpawned: Boolean = false
+ private var dpSpawned: Boolean = false
+ private var dnName: String = _
+ private var dpName: String = _
+
+ private val blockingThread = new Thread(new Runnable {
+ override def run(): Unit = {
+ logInfo("Beginning of Cluster lock")
+ lock.lock()
+ try {
+ while (!kdcIsUp) kdcRunning.await()
+ while (!nnIsUp) nnRunning.await()
+ while (!dnIsUp) dnRunning.await()
+ while (!dpIsUp) dpRunning.await()
+ } finally {
+ logInfo("Ending the Cluster lock")
+ lock.unlock()
+ stop()
+ }
+ }
+ })
+
+ private val podWatcherThread = new Thread(new Runnable {
+ override def run(): Unit = {
+ logInfo("Beginning the watch of Pods")
+ podWatcher = kubernetesClient
+ .pods()
+ .withLabels(labels.asJava)
+ .watch(new Watcher[Pod] {
+ override def onClose(cause: KubernetesClientException): Unit =
+ logInfo("Ending the watch of Pods")
+ override def eventReceived(action: Watcher.Action, resource: Pod): Unit = {
+ val name = resource.getMetadata.getName
+ val keyName = podNameParse(name)
+ action match {
+ case Action.DELETED | Action.ERROR =>
+ logInfo(s"$name either deleted or error")
+ podCache.remove(keyName)
+ case Action.ADDED | Action.MODIFIED =>
+ val phase = resource.getStatus.getPhase
+ logInfo(s"$name is as $phase")
+ if (name.startsWith("dn1")) { dnName = name }
+ if (name.startsWith("data-populator")) { dpName = name }
+ podCache(keyName) = phase
+ if (maybeDeploymentAndServiceDone(keyName)) {
+ val modifyAndSignal: Runnable = new MSThread(keyName)
+ new Thread(modifyAndSignal).start()
+ }}}})
+ }})
+
+ private val serviceWatcherThread = new Thread(new Runnable {
+ override def run(): Unit = {
+ logInfo("Beginning the watch of Services")
+ serviceWatcher = kubernetesClient
+ .services()
+ .withLabels(labels.asJava)
+ .watch(new Watcher[Service] {
+ override def onClose(cause: KubernetesClientException): Unit =
+ logInfo("Ending the watch of Services")
+ override def eventReceived(action: Watcher.Action, resource: Service): Unit = {
+ val name = resource.getMetadata.getName
+ action match {
+ case Action.DELETED | Action.ERROR =>
+ logInfo(s"$name either deleted or error")
+ serviceCache.remove(name)
+ case Action.ADDED | Action.MODIFIED =>
+ val bound = resource.getSpec.getSelector.get("kerberosService")
+ logInfo(s"$name is bounded to $bound")
+ serviceCache(name) = bound
+ if (maybeDeploymentAndServiceDone(name)) {
+ val modifyAndSignal: Runnable = new MSThread(name)
+ new Thread(modifyAndSignal).start()
+ }}}})
+ logInfo("Launching the Cluster")
+ if (!kdcSpawned) {
+ logInfo("Launching the KDC Node")
+ kdcSpawned = true
+ deploy(kerberosUtils.getKDC)
+ }
+ }})
+
+ def start(): Unit = {
+ blockingThread.start()
+ podWatcherThread.start()
+ serviceWatcherThread.start()
+ blockingThread.join()
+ podWatcherThread.join()
+ serviceWatcherThread.join()
+ }
+
+ def stop(): String = {
+ podWatcher.close()
+ serviceWatcher.close()
+ dpName
+ }
+
+ private def maybeDeploymentAndServiceDone(name: String): Boolean = {
+ val finished = podCache.get(name).contains("Running") &&
+ serviceCache.get(name).contains(name)
+ if (!finished) {
+ logInfo(s"$name is not up with a service")
+ if (name == "kerberos") kdcIsUp = false
+ else if (name == "nn") nnIsUp = false
+ else if (name == "dn1") dnIsUp = false
+ else if (name == "data-populator") dpIsUp = false
+ }
+ finished
+ }
+
+ private def deploy(kdc: KerberosDeployment) : Unit = {
+ kubernetesClient
+ .extensions().deployments().inNamespace(namespace).create(kdc.podDeployment)
+ kubernetesClient
+ .services().inNamespace(namespace).create(kdc.service)
+ }
+
+ private class MSThread(name: String) extends Runnable {
+ override def run(): Unit = {
+ logInfo(s"$name Node and Service is up")
+ lock.lock()
+ if (name == "kerberos") {
+ kdcIsUp = true
+ logInfo(s"kdc has signaled")
+ try {
+ kdcRunning.signalAll()
+ } finally {
+ lock.unlock()
+ }
+ if (!nnSpawned) {
+ logInfo("Launching the NN Node")
+ nnSpawned = true
+ deploy(kerberosUtils.getNN)
+ }
+ }
+ else if (name == "nn") {
+ while (!kdcIsUp) kdcRunning.await()
+ nnIsUp = true
+ logInfo(s"nn has signaled")
+ try {
+ nnRunning.signalAll()
+ } finally {
+ lock.unlock()
+ }
+ if (!dnSpawned) {
+ logInfo("Launching the DN Node")
+ dnSpawned = true
+ deploy(kerberosUtils.getDN)
+ }
+ }
+ else if (name == "dn1") {
+ while (!kdcIsUp) kdcRunning.await()
+ while (!nnIsUp) nnRunning.await()
+ dnIsUp = true
+ logInfo(s"dn1 has signaled")
+ try {
+ dnRunning.signalAll()
+ } finally {
+ lock.unlock()
+ }
+ if (!dpSpawned) {
+ logInfo("Launching the DP Node")
+ dpSpawned = true
+ deploy(kerberosUtils.getDP)
+ }
+ }
+ else if (name == "data-populator") {
+ while (!kdcIsUp) kdcRunning.await()
+ while (!nnIsUp) nnRunning.await()
+ while (!dnIsUp) dnRunning.await()
+ while (!hasInLogs(dnName, "Got finalize command for block pool")) {
+ logInfo("Waiting on DN to be formatted")
+ Thread.sleep(500)
+ }
+ dpIsUp = true
+ logInfo(s"data-populator has signaled")
+ try {
+ dpRunning.signalAll()
+ } finally {
+ lock.unlock()
+ }
+ }
+ }
+ }
+
+ private def podNameParse(name: String) : String = {
+ name match {
+ case _ if name.startsWith("kerberos") => "kerberos"
+ case _ if name.startsWith("nn") => "nn"
+ case _ if name.startsWith("dn1") => "dn1"
+ case _ if name.startsWith("data-populator") => "data-populator"
+ }
+ }
+
+ def hasInLogs(name: String, expectation: String): Boolean = {
+ kubernetesClient
+ .pods()
+ .withName(name)
+ .getLog().contains(expectation)
+ }
+}
diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/kerberos/KerberosStorage.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/kerberos/KerberosStorage.scala
new file mode 100644
index 0000000000000..28fe77c274ea1
--- /dev/null
+++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/kerberos/KerberosStorage.scala
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.deploy.kubernetes.integrationtest.kerberos
+
+import io.fabric8.kubernetes.api.model.{PersistentVolume, PersistentVolumeClaim}
+
+private[spark] case class KerberosStorage(
+ persistentVolumeClaim: PersistentVolumeClaim,
+ persistentVolume: PersistentVolume)
diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/kerberos/KerberosUtils.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/kerberos/KerberosUtils.scala
new file mode 100644
index 0000000000000..145683fdc5eb2
--- /dev/null
+++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/kerberos/KerberosUtils.scala
@@ -0,0 +1,154 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.deploy.kubernetes.integrationtest.kerberos
+
+import java.io.{File, FileInputStream}
+
+import scala.collection.JavaConverters._
+
+import io.fabric8.kubernetes.api.model._
+import io.fabric8.kubernetes.api.model.extensions.{Deployment, DeploymentBuilder}
+import io.fabric8.kubernetes.client.KubernetesClient
+import org.apache.commons.io.FileUtils.readFileToString
+
+import org.apache.spark.deploy.kubernetes.submit.ContainerNameEqualityPredicate
+
+ /**
+ * This class is responsible for handling all Utils and Constants necessary for testing
+ */
+private[spark] class KerberosUtils(
+ kubernetesClient: KubernetesClient,
+ namespace: String) {
+ def getClient: KubernetesClient = kubernetesClient
+ def getNamespace: String = namespace
+ def yamlLocation(loc: String): String = s"kerberos-yml/$loc.yml"
+ def loadFromYaml(resource: String): FileInputStream =
+ new FileInputStream(new File(yamlLocation(resource)))
+ private val regex = "REPLACE_ME".r
+ private val regexDP = "# default_ccache_name = MEMORY".r
+ private val defaultCacheDP = "default_ccache_name = KRBCONF"
+ private def locationResolver(loc: String) = s"test-data/hadoop-conf/$loc"
+ private val kerberosFiles = Seq("krb5.conf", "core-site.xml", "hdfs-site.xml")
+ private val kerberosConfTupList =
+ kerberosFiles.map { file =>
+ (file, regex.replaceAllIn(readFileToString(new File(locationResolver(file))), namespace))} ++
+ Seq(("krb5-dp.conf", regexDP.replaceAllIn(regex.replaceAllIn(readFileToString(
+ new File(locationResolver("krb5.conf"))), namespace), defaultCacheDP)))
+ private val KRB_VOLUME = "krb5-conf"
+ private val KRB_FILE_DIR = "/tmp"
+ private val KRB_CONFIG_MAP_NAME = "krb-config-map"
+ private val PV_LABELS = Map("job" -> "kerberostest")
+ private val keyPaths: Seq[KeyToPath] = (kerberosFiles ++ Seq("krb5-dp.conf"))
+ .map(file =>
+ new KeyToPathBuilder()
+ .withKey(file)
+ .withPath(file)
+ .build()).toList
+ private def createPVTemplate(name: String, pathType: String) : PersistentVolume =
+ new PersistentVolumeBuilder()
+ .withNewMetadata()
+ .withName(name)
+ .withLabels(Map(
+ "type" -> "local",
+ "job" -> "kerberostest").asJava)
+ .endMetadata()
+ .withNewSpec()
+ .withCapacity(Map("storage" -> new Quantity("1Gi")).asJava)
+ .withAccessModes("ReadWriteOnce")
+ .withHostPath(
+ new HostPathVolumeSource(s"/tmp/$namespace/$pathType"))
+ .endSpec()
+ .build()
+ private val pvNN = "nn-hadoop"
+ private val pvKT = "server-keytab"
+ private val persistentVolumeMap: Map[String, PersistentVolume] = Map(
+ pvNN -> createPVTemplate(pvNN, "nn"),
+ pvKT -> createPVTemplate(pvKT, "keytab"))
+ private def buildKerberosPV(pvType: String) = {
+ KerberosStorage(
+ kubernetesClient.load(loadFromYaml(pvType))
+ .get().get(0).asInstanceOf[PersistentVolumeClaim],
+ persistentVolumeMap(pvType))
+ }
+ def getNNStorage: KerberosStorage = buildKerberosPV(pvNN)
+ def getKTStorage: KerberosStorage = buildKerberosPV(pvKT)
+ def getLabels: Map[String, String] = PV_LABELS
+ def getKeyPaths: Seq[KeyToPath] = keyPaths
+ def getConfigMap: ConfigMap = new ConfigMapBuilder()
+ .withNewMetadata()
+ .withName(KRB_CONFIG_MAP_NAME)
+ .endMetadata()
+ .addToData(kerberosConfTupList.toMap.asJava)
+ .build()
+ private val kdcNode = Seq("kerberos-deployment", "kerberos-service")
+ private val nnNode = Seq("nn-deployment", "nn-service")
+ private val dnNode = Seq("dn1-deployment", "dn1-service")
+ private val dataPopulator = Seq("data-populator-deployment", "data-populator-service")
+ private def buildKerberosDeployment(seqPair: Seq[String]) = {
+ val deployment =
+ kubernetesClient.load(loadFromYaml(seqPair.head)).get().get(0).asInstanceOf[Deployment]
+ KerberosDeployment(
+ new DeploymentBuilder(deployment)
+ .editSpec()
+ .editTemplate()
+ .editSpec()
+ .addNewVolume()
+ .withName(KRB_VOLUME)
+ .withNewConfigMap()
+ .withName(KRB_CONFIG_MAP_NAME)
+ .withItems(keyPaths.asJava)
+ .endConfigMap()
+ .endVolume()
+ .editMatchingContainer(new ContainerNameEqualityPredicate(
+ deployment.getMetadata.getName))
+ .addNewEnv()
+ .withName("NAMESPACE")
+ .withValue(namespace)
+ .endEnv()
+ .addNewEnv()
+ .withName("TMP_KRB_LOC")
+ .withValue(s"$KRB_FILE_DIR/${kerberosFiles.head}")
+ .endEnv()
+ .addNewEnv()
+ .withName("TMP_KRB_DP_LOC")
+ .withValue(s"$KRB_FILE_DIR/krb5-dp.conf")
+ .endEnv()
+ .addNewEnv()
+ .withName("TMP_CORE_LOC")
+ .withValue(s"$KRB_FILE_DIR/${kerberosFiles(1)}")
+ .endEnv()
+ .addNewEnv()
+ .withName("TMP_HDFS_LOC")
+ .withValue(s"$KRB_FILE_DIR/${kerberosFiles(2)}")
+ .endEnv()
+ .addNewVolumeMount()
+ .withName(KRB_VOLUME)
+ .withMountPath(KRB_FILE_DIR)
+ .endVolumeMount()
+ .endContainer()
+ .endSpec()
+ .endTemplate()
+ .endSpec()
+ .build(),
+ kubernetesClient.load(loadFromYaml(seqPair(1))).get().get(0).asInstanceOf[Service]
+ )
+ }
+ def getKDC: KerberosDeployment = buildKerberosDeployment(kdcNode)
+ def getNN: KerberosDeployment = buildKerberosDeployment(nnNode)
+ def getDN: KerberosDeployment = buildKerberosDeployment(dnNode)
+ def getDP: KerberosDeployment = buildKerberosDeployment(dataPopulator)
+}
diff --git a/resource-managers/kubernetes/integration-tests/test-data/hadoop-conf/core-site.xml b/resource-managers/kubernetes/integration-tests/test-data/hadoop-conf/core-site.xml
new file mode 100644
index 0000000000000..9e168812052ea
--- /dev/null
+++ b/resource-managers/kubernetes/integration-tests/test-data/hadoop-conf/core-site.xml
@@ -0,0 +1,38 @@
+
+
+
+
+
+
+
+
+ hadoop.security.authentication
+ kerberos
+
+
+
+ hadoop.security.authorization
+ true
+
+
+
+ fs.defaultFS
+ hdfs://nn.REPLACE_ME.svc.cluster.local:9000
+
+
+ hadoop.rpc.protection
+ authentication
+
+
\ No newline at end of file
diff --git a/resource-managers/kubernetes/integration-tests/test-data/hadoop-conf/hdfs-site.xml b/resource-managers/kubernetes/integration-tests/test-data/hadoop-conf/hdfs-site.xml
new file mode 100644
index 0000000000000..66dc969c46b63
--- /dev/null
+++ b/resource-managers/kubernetes/integration-tests/test-data/hadoop-conf/hdfs-site.xml
@@ -0,0 +1,157 @@
+
+
+
+
+
+
+
+
+
+ dfs.replication
+ 1
+
+
+
+
+ dfs.permissions
+ true
+
+
+ dfs.block.access.token.enable
+ true
+
+
+
+
+ dfs.namenode.keytab.file
+ /var/keytabs/hdfs.keytab
+
+
+ dfs.namenode.kerberos.principal
+ hdfs/nn.REPLACE_ME.svc.cluster.local@CLUSTER.LOCAL
+
+
+ dfs.namenode.kerberos.internal.spnego.principal
+ HTTP/nn.REPLACE_ME.svc.cluster.local@CLUSTER.LOCAL
+
+
+ dfs.namenode.rpc-address
+ nn.REPLACE_ME.svc.cluster.local:9000
+
+
+
+
+
+ dfs.namenode.delegation.token.max-lifetime
+ 3600000
+
+
+ dfs.namenode.delegation.token.renew-interval
+ 3600000
+
+
+
+
+
+
+ dfs.data.transfer.protection
+ integrity
+
+
+ dfs.datanode.address
+ 0.0.0.0:10019
+
+
+
+ dfs.datanode.http.address
+ 0.0.0.0:10022
+
+
+
+ dfs.http.policy
+ HTTPS_ONLY
+
+
+
+
+ dfs.namenode.keytab.file
+ /var/keytabs/hdfs.keytab
+
+
+ dfs.namenode.kerberos.principal
+ hdfs/nn.REPLACE_ME.svc.cluster.local@CLUSTER.LOCAL
+
+
+ dfs.namenode.kerberos.internal.spnego.principal
+ HTTP/nn.REPLACE_ME.svc.cluster.local@CLUSTER.LOCAL
+
+
+
+
+ dfs.namenode.datanode.registration.ip-hostname-check
+ false
+
+
+ dfs.datanode.data.dir.perm
+ 700
+
+
+ dfs.namenode.name.dir
+ file:///hadoop/etc/data
+
+
+ dfs.datanode.name.dir
+ file:///hadoop/etc/data
+
+
+ dfs.data.dir
+ file:///hadoop/etc/data
+
+
+ dfs.datanode.keytab.file
+ /var/keytabs/hdfs.keytab
+
+
+ dfs.datanode.kerberos.principal
+ hdfs/dn1.REPLACE_ME.svc.cluster.local@CLUSTER.LOCAL
+
+
+ dfs.encrypt.data.transfer
+ true
+
+
+ dfs.encrypt.data.transfer.cipher.suites
+ AES/CTR/NoPadding
+
+
+ dfs.encrypt.data.transfer.cipher.key.bitlength
+ 256
+
+
+
+
+ dfs.webhdfs.enabled
+ true
+
+
+ dfs.web.authentication.kerberos.principal
+ HTTP/dn1.REPLACE_ME.svc.cluster.local@CLUSTER.LOCAL
+
+
+ dfs.web.authentication.kerberos.keytab
+ /var/keytabs/hdfs.keytab
+
+
+
diff --git a/resource-managers/kubernetes/integration-tests/test-data/hadoop-conf/krb5.conf b/resource-managers/kubernetes/integration-tests/test-data/hadoop-conf/krb5.conf
new file mode 100644
index 0000000000000..144f77d8995df
--- /dev/null
+++ b/resource-managers/kubernetes/integration-tests/test-data/hadoop-conf/krb5.conf
@@ -0,0 +1,25 @@
+includedir /etc/krb5.conf.d/
+
+[logging]
+default = FILE:/var/log/krb5libs.log
+kdc = FILE:/var/log/krb5kdc.log
+admin_server = FILE:/var/log/kadmind.log
+
+[libdefaults]
+dns_lookup_realm = false
+ticket_lifetime = 24h
+renew_lifetime = 7d
+forwardable = true
+rdns = false
+default_realm = CLUSTER.LOCAL
+# default_ccache_name = MEMORY
+
+[realms]
+CLUSTER.LOCAL = {
+ kdc = kerberos.REPLACE_ME.svc.cluster.local
+ admin_server = kerberos.REPLACE_ME.svc.cluster.local
+}
+
+[domain_realm]
+.cluster.local = CLUSTER.LOCAL
+cluster.local = CLUSTER.LOCAL
diff --git a/resource-managers/kubernetes/integration-tests/test-data/hadoop-conf/yarn-site.xml b/resource-managers/kubernetes/integration-tests/test-data/hadoop-conf/yarn-site.xml
new file mode 100644
index 0000000000000..92d9346232c76
--- /dev/null
+++ b/resource-managers/kubernetes/integration-tests/test-data/hadoop-conf/yarn-site.xml
@@ -0,0 +1,26 @@
+
+
+
+
+
+
+
+
+
+
+ yarn.resourcemanager.principal
+ yarn/_HOST@CLUSTER.LOCAL
+
+
\ No newline at end of file
diff --git a/resource-managers/kubernetes/integration-tests/test-data/simple-hadoop-conf/core-site.xml b/resource-managers/kubernetes/integration-tests/test-data/simple-hadoop-conf/core-site.xml
new file mode 100644
index 0000000000000..67975bd180c26
--- /dev/null
+++ b/resource-managers/kubernetes/integration-tests/test-data/simple-hadoop-conf/core-site.xml
@@ -0,0 +1,24 @@
+
+
+
+
+
+
+
+
+ fs.defaultFS
+ hdfs://nn.REPLACE_ME.svc.cluster.local:9000
+
+
\ No newline at end of file
diff --git a/resource-managers/kubernetes/integration-tests/test-data/simple-hadoop-conf/hdfs-site.xml b/resource-managers/kubernetes/integration-tests/test-data/simple-hadoop-conf/hdfs-site.xml
new file mode 100644
index 0000000000000..76fc9c68fa375
--- /dev/null
+++ b/resource-managers/kubernetes/integration-tests/test-data/simple-hadoop-conf/hdfs-site.xml
@@ -0,0 +1,24 @@
+
+
+
+
+
+
+
+
+ dfs.replication
+ 1
+
+
|