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 1305aeb8c1faf..f1bd59da9ec69 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 5e23801e15b10..7803d1b7692c3 100644
--- a/docs/running-on-kubernetes.md
+++ b/docs/running-on-kubernetes.md
@@ -768,6 +768,53 @@ from the other deployment modes. See the [configuration page](configuration.html
myIdentifier
. Multiple node selector keys can be added by setting multiple configurations with this prefix.
+
+ spark.kubernetes.kerberos |
+ false |
+
+ Specify whether your job is a job that will require a Delegation Token to access HDFS. By default, we
+ will assume that you will not require secure HDFS access.
+ |
+
+
+ spark.kubernetes.kerberos.keytab |
+ (none) |
+
+ Assuming you have set spark.kubernetes.kerberos to be true. This will let you specify
+ the location of your Kerberos keytab to be used in order to access Secure HDFS. This is optional as you
+ may login by running kinit -kt before running the spark-submit, and the submission client
+ will look within your local TGT cache to resolve this.
+ |
+
+
+ spark.kubernetes.kerberos.principal |
+ (none) |
+
+ Assuming you have set spark.kubernetes.kerberos to be true. This will let you specify
+ your Kerberos principal that you wish to use to access Secure HDFS. This is optional as you
+ may login by running kinit -kt before running the spark-submit, and the submission client
+ will look within your local TGT cache to resolve this.
+ |
+
+
+ spark.kubernetes.kerberos.tokensecret.name |
+ (none) |
+
+ Assuming you have set spark.kubernetes.kerberos to be true. This will let you specify
+ the name of the secret where your existing delegation token data is stored. You must also specify the
+ label spark.kubernetes.kerberos.tokensecret.name where your data is stored on the secret.
+ |
+
+
+ spark.kubernetes.kerberos.tokensecret.label |
+ spark.kubernetes.kerberos.dt.label |
+
+ Assuming you have set spark.kubernetes.kerberos to be true. This will let you specify
+ the label 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.dt.label should you not include it. But
+ you should always include this if you are proposing a pre-existing secret contain the delegation token data.
+ |
+
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..54b59d235860a
--- /dev/null
+++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/HadoopConfBootstrap.scala
@@ -0,0 +1,82 @@
+/*
+ * 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 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]) 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_FILE_DIR)
+ .endVolumeMount()
+ .addNewEnv()
+ .withName(HADOOP_CONF_DIR)
+ .withValue(HADOOP_FILE_DIR)
+ .endEnv()
+ .build()
+ originalPodWithMainContainer.copy(
+ pod = hadoopSupportedPod,
+ mainContainer = mainContainerWithMountedHadoopConf)
+ }
+}
diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/KerberosConfBootstrap.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/KerberosConfBootstrap.scala
new file mode 100644
index 0000000000000..2aa772ae7b04f
--- /dev/null
+++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/KerberosConfBootstrap.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
+
+import io.fabric8.kubernetes.api.model.ContainerBuilder
+
+import org.apache.spark.deploy.kubernetes.constants._
+ /**
+ * This is separated out from hadoopsteps because this component can be reused to
+ * set up the Kerberos logic for executors as well.
+ */
+private[spark] trait KerberosConfBootstrap {
+ /**
+ * Bootstraps a main container with an ENV variable
+ * pointing to the data storing the DT in the secret
+ */
+ def bootstrapMainContainerAndVolumes(originalPodWithMainContainer: PodWithMainContainer)
+ : PodWithMainContainer
+}
+private[spark] class KerberosConfBootstrapImpl(
+ delegationTokenLabelName: String) extends KerberosConfBootstrap{
+ override def bootstrapMainContainerAndVolumes(
+ originalPodWithMainContainer: PodWithMainContainer)
+ : PodWithMainContainer = {
+ val mainContainerWithMountedHadoopConf = new ContainerBuilder(
+ originalPodWithMainContainer.mainContainer)
+ .addNewEnv()
+ .withName(ENV_KERBEROS_SECRET_LABEL)
+ .withValue(delegationTokenLabelName)
+ .endEnv()
+ .build()
+ originalPodWithMainContainer.copy(mainContainer = mainContainerWithMountedHadoopConf)
+ }
+}
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 c6772c1cb5ae4..4f74dfa9c6c1d 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
@@ -499,6 +499,42 @@ 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")
+ .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_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_LABEL =
+ ConfigBuilder("spark.kubernetes.kerberos.tokensecret.label")
+ .doc("Specify the label 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
+ .createWithDefault("spark.kubernetes.kerberos.dt.label")
+
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 92f051b2ac298..be31ae65c6d58 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
@@ -91,6 +91,24 @@ 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_FILE_DIR = "/etc/hadoop"
+ private[spark] val 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 KERBEROS_SPARK_CONF_NAME =
+ "spark.kubernetes.kerberos.secretlabelname"
+ private[spark] val KERBEROS_SECRET_LABEL_PREFIX =
+ "hadoop-tokens"
+ private[spark] val ENV_KERBEROS_SECRET_LABEL =
+ "KERBEROS_SECRET_LABEL"
+
// Miscellaneous
private[spark] val ANNOTATION_EXECUTOR_NODE_AFFINITY = "scheduler.alpha.kubernetes.io/affinity"
private[spark] val DRIVER_CONTAINER_NAME = "spark-kubernetes-driver"
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 2fa9b416330e5..1595f145c05b9 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
@@ -149,7 +149,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()
@@ -168,6 +170,7 @@ private[spark] object Client {
clientArguments.mainClass,
clientArguments.driverArgs,
clientArguments.otherPyFiles,
+ hadoopConfDir,
sparkConf)
Utils.tryWithResource(SparkKubernetesClientFactory.createKubernetesClient(
master,
@@ -195,6 +198,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 82abe55ac6989..096518432e2de 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
@@ -20,7 +20,8 @@ import org.apache.spark.SparkConf
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.{BaseDriverConfigurationStep, DependencyResolutionStep, DriverConfigurationStep, DriverKubernetesCredentialsStep, InitContainerBootstrapStep, PythonStep}
+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,6 +97,19 @@ private[spark] class DriverConfigurationStepsOrchestrator(
submissionSparkConf)
val kubernetesCredentialsStep = new DriverKubernetesCredentialsStep(
submissionSparkConf, kubernetesResourceNamePrefix)
+ val hadoopConfigSteps =
+ if (hadoopConfDir.isEmpty) {
+ Option.empty[DriverConfigurationStep]
+ } else {
+ val hadoopStepsOrchestrator = new HadoopStepsOrchestrator(
+ namespace,
+ hadoopConfigMapName,
+ submissionSparkConf,
+ hadoopConfDir)
+ val hadoopConfSteps =
+ hadoopStepsOrchestrator.getHadoopSteps()
+ Some(new HadoopConfigBootstrapStep(hadoopConfSteps, hadoopConfigMapName))
+ }
val pythonStep = mainAppResource match {
case PythonMainAppResource(mainPyResource) =>
Option(new PythonStep(mainPyResource, additionalPythonFiles, filesDownloadPath))
@@ -133,6 +149,7 @@ private[spark] class DriverConfigurationStepsOrchestrator(
kubernetesCredentialsStep,
dependencyResolutionStep) ++
initContainerBootstrapStep.toSeq ++
+ hadoopConfigSteps.toSeq ++
pythonStep.toSeq
}
}
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..894e73e4286c7
--- /dev/null
+++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/HadoopConfigBootstrapStep.scala
@@ -0,0 +1,66 @@
+/*
+ * 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)
+ 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/HDFSDelegationToken.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HDFSDelegationToken.scala
new file mode 100644
index 0000000000000..4f6f0953c5340
--- /dev/null
+++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HDFSDelegationToken.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.submit.submitsteps.hadoopsteps
+
+ /**
+ * This case class contain the information that is important to be stored for
+ * delegation token logic
+ */
+private[spark] case class HDFSDelegationToken(bytes: Array[Byte], renewal: Long)
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..e3b7f674d0225
--- /dev/null
+++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopConfMounterStep.scala
@@ -0,0 +1,58 @@
+/*
+ * 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: Option[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 ++
+ hadoopConfDir.map(conf_dir => Map(HADOOP_CONF_DIR_LOC -> conf_dir)).getOrElse(
+ Map.empty[String, String])
+ )
+ }
+}
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..b69b18c11470f
--- /dev/null
+++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopConfigSpec.scala
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.deploy.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
+ */
+private[spark] case class HadoopConfigSpec(
+ additionalDriverSparkConf: Map[String, String],
+ driverPod: Pod,
+ driverContainer: Container,
+ configMapProperties: Map[String, String],
+ dtSecret: Option[Secret])
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..7aec765e00f17
--- /dev/null
+++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopKerberosKeytabResolverStep.scala
@@ -0,0 +1,161 @@
+/*
+ * 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 scala.util.Try
+
+import io.fabric8.kubernetes.api.model.SecretBuilder
+import org.apache.commons.codec.binary.Base64
+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
+
+import org.apache.spark.SparkConf
+import org.apache.spark.deploy.kubernetes.{KerberosConfBootstrapImpl, PodWithMainContainer}
+import org.apache.spark.deploy.kubernetes.constants._
+import org.apache.spark.deploy.SparkHadoopUtil
+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.logingUserFromKeytabAndReturnUGI 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]) extends HadoopConfigurationStep with Logging{
+ private var originalCredentials: Credentials = _
+ private var dfs : FileSystem = _
+ private var renewer: String = _
+ private var renewedCredentials: Credentials = _
+ private var renewedTokens: Iterable[Token[_ <: TokenIdentifier]] = _
+ override def configureContainers(hadoopConfigSpec: HadoopConfigSpec): HadoopConfigSpec = {
+ val hadoopConf = SparkHadoopUtil.get.newConfiguration(submissionSparkConf)
+ logInfo(s"Hadoop Configuration: ${hadoopConf.toString}")
+ if (!UserGroupInformation.isSecurityEnabled) logError("Hadoop not configuration 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)
+ logInfo("Logged into KDC with keytab using Job User UGI")
+ UserGroupInformation.loginUserFromKeytabAndReturnUGI(
+ principal,
+ keytab.toURI.toString)
+ }
+ // In the case that keytab is not specified we will read from Local Ticket Cache
+ val jobUserUGI = maybeJobUserUGI.getOrElse(UserGroupInformation.getCurrentUser)
+ // It is necessary to run as jobUserUGI because logged in user != Current User
+ jobUserUGI.doAs(new PrivilegedExceptionAction[Void] {
+ override def run(): Void = {
+ logInfo(s"Retrieved Job User UGI: $jobUserUGI")
+ originalCredentials = jobUserUGI.getCredentials
+ logInfo(s"Original tokens: ${originalCredentials.toString}")
+ logInfo(s"All tokens: ${originalCredentials.getAllTokens}")
+ logInfo(s"All secret keys: ${originalCredentials.getAllSecretKeys}")
+ dfs = FileSystem.get(hadoopConf)
+ // This is not necessary with [Spark-20328] since we would be using
+ // Spark core providers to handle delegation token renewal
+ renewer = jobUserUGI.getShortUserName
+ logInfo(s"Renewer is: $renewer")
+ renewedCredentials = new Credentials(originalCredentials)
+ dfs.addDelegationTokens(renewer, renewedCredentials)
+ renewedTokens = renewedCredentials.getAllTokens.asScala
+ logInfo(s"Renewed tokens: ${renewedCredentials.toString}")
+ logInfo(s"All renewed tokens: ${renewedTokens.mkString(",")}")
+ logInfo(s"All renewed secret keys: ${renewedCredentials.getAllSecretKeys}")
+ null
+ }})
+ if (renewedTokens.isEmpty) logError("Did not obtain any Delegation Tokens")
+ val data = serialize(renewedCredentials)
+ val renewalTime = getTokenRenewalInterval(renewedTokens, hadoopConf).getOrElse(Long.MaxValue)
+ val delegationToken = HDFSDelegationToken(data, renewalTime)
+ val currentTime: Long = System.currentTimeMillis()
+ val initialTokenLabelName = s"$KERBEROS_SECRET_LABEL_PREFIX-$currentTime-$renewalTime"
+ logInfo(s"Storing dt in $initialTokenLabelName")
+ val secretDT =
+ new SecretBuilder()
+ .withNewMetadata()
+ .withName(HADOOP_KERBEROS_SECRET_NAME)
+ .endMetadata()
+ .addToData(initialTokenLabelName, Base64.encodeBase64String(delegationToken.bytes))
+ .build()
+ val bootstrapKerberos = new KerberosConfBootstrapImpl(initialTokenLabelName)
+ val withKerberosEnvPod = bootstrapKerberos.bootstrapMainContainerAndVolumes(
+ PodWithMainContainer(
+ hadoopConfigSpec.driverPod,
+ hadoopConfigSpec.driverContainer))
+ hadoopConfigSpec.copy(
+ additionalDriverSparkConf =
+ hadoopConfigSpec.additionalDriverSparkConf ++ Map(
+ KERBEROS_SPARK_CONF_NAME -> initialTokenLabelName),
+ driverPod = withKerberosEnvPod.pod,
+ driverContainer = withKerberosEnvPod.mainContainer,
+ dtSecret = Some(secretDT))
+ }
+
+ // Functions that should be in Core with Rebase to 2.3
+ @deprecated("Moved to core in 2.2", "2.2")
+ private 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
+ logInfo(s"Renewal interval is $interval for token ${token.getKind.toString}")
+ interval
+ }.toOption}
+ if (renewIntervals.isEmpty) None else Some(renewIntervals.min)
+ }
+
+ @deprecated("Moved to core in 2.2", "2.2")
+ private 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")
+ private 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/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..4613436fe414c
--- /dev/null
+++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopKerberosSecretResolverStep.scala
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.deploy.kubernetes.submit.submitsteps.hadoopsteps
+
+import org.apache.spark.SparkConf
+import org.apache.spark.deploy.kubernetes.{KerberosConfBootstrapImpl, PodWithMainContainer}
+
+ /**
+ * 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 label corresponding to the
+ * data where the delegation token is stored.
+ */
+private[spark] class HadoopKerberosSecretResolverStep(
+ submissionSparkConf: SparkConf,
+ tokenSecretName: String,
+ tokenLabelName: String) extends HadoopConfigurationStep {
+
+ override def configureContainers(hadoopConfigSpec: HadoopConfigSpec): HadoopConfigSpec = {
+ val bootstrapKerberos = new KerberosConfBootstrapImpl(tokenLabelName)
+ val withKerberosEnvPod = bootstrapKerberos.bootstrapMainContainerAndVolumes(
+ PodWithMainContainer(
+ hadoopConfigSpec.driverPod,
+ hadoopConfigSpec.driverContainer))
+ hadoopConfigSpec.copy(
+ driverPod = withKerberosEnvPod.pod,
+ driverContainer = withKerberosEnvPod.mainContainer)
+ }
+}
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..b43b99f9b1a06
--- /dev/null
+++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopStepsOrchestrator.scala
@@ -0,0 +1,77 @@
+/*
+ * 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
+import org.apache.spark.deploy.kubernetes.config._
+
+
+ /**
+ * Returns the complete ordered list of steps required to configure the hadoop configurations.
+ */
+private[spark] class HadoopStepsOrchestrator(
+ namespace: String,
+ hadoopConfigMapName: String,
+ submissionSparkConf: SparkConf,
+ hadoopConfDir: Option[String]) {
+ private val maybeKerberosSupport = 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 maybeExistingSecretLabel =
+ submissionSparkConf.get(KUBERNETES_KERBEROS_DT_SECRET_LABEL)
+ private val hadoopConfigurationFiles = hadoopConfDir.map(conf => getHadoopConfFiles(conf))
+ .getOrElse(Seq.empty[File])
+
+ def getHadoopSteps(): Seq[HadoopConfigurationStep] = {
+ val hadoopConfBootstrapImpl = new HadoopConfBootstrapImpl(
+ hadoopConfigMapName,
+ hadoopConfigurationFiles)
+ val hadoopConfMounterStep = new HadoopConfMounterStep(
+ hadoopConfigMapName,
+ hadoopConfigurationFiles,
+ hadoopConfBootstrapImpl,
+ hadoopConfDir)
+ val maybeKerberosStep =
+ if (maybeKerberosSupport) {
+ maybeExistingSecret.map(secretLabel => Some(new HadoopKerberosSecretResolverStep(
+ submissionSparkConf,
+ secretLabel,
+ maybeExistingSecretLabel))).getOrElse(Some(
+ new HadoopKerberosKeytabResolverStep(
+ submissionSparkConf,
+ maybePrincipal,
+ maybeKeytab)))
+ } 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 fa0ecca3b4ee6..ff348cf06b431 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,15 +20,14 @@ 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.internal.Logging
import org.apache.spark.scheduler.{ExternalClusterManager, SchedulerBackend, TaskScheduler, TaskSchedulerImpl}
private[spark] class KubernetesClusterManager extends ExternalClusterManager with Logging {
-
+ import org.apache.spark.SparkContext
override def canCreate(masterURL: String): Boolean = masterURL.startsWith("k8s")
override def createTaskScheduler(sc: SparkContext, masterURL: String): TaskScheduler = {
@@ -42,6 +41,9 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit
val sparkConf = sc.getConf
val maybeConfigMap = sparkConf.get(EXECUTOR_INIT_CONTAINER_CONFIG_MAP)
val maybeConfigMapKey = sparkConf.get(EXECUTOR_INIT_CONTAINER_CONFIG_MAP_KEY)
+ val maybeHadoopConfigMap = sparkConf.getOption(HADOOP_CONFIG_MAP_SPARK_CONF_NAME)
+ val maybeHadoopConfDir = sparkConf.getOption(HADOOP_CONF_DIR_LOC)
+ val maybeDTSecret = sparkConf.getOption(KERBEROS_SPARK_CONF_NAME)
val maybeExecutorInitContainerSecretName =
sparkConf.get(EXECUTOR_INIT_CONTAINER_SECRET)
@@ -59,7 +61,7 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit
// name. Note that we generally expect both to have been set from spark-submit V2, but for
// testing developers may simply run the driver JVM locally, but the config map won't be set
// then.
- val bootStrap = for {
+ val initBootStrap = for {
configMap <- maybeConfigMap
configMapKey <- maybeConfigMapKey
} yield {
@@ -72,6 +74,21 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit
configMap,
configMapKey)
}
+ val hadoopBootStrap = for {
+ hadoopConfigMap <- maybeHadoopConfigMap
+ } yield {
+ val hadoopConfigurations = maybeHadoopConfDir.map(
+ conf_dir => getHadoopConfFiles(conf_dir)).getOrElse(Array.empty[File])
+ new HadoopConfBootstrapImpl(
+ hadoopConfigMap,
+ hadoopConfigurations
+ )
+ }
+ val kerberosBootstrap = for {
+ dTSecret <- maybeDTSecret
+ } yield {
+ new KerberosConfBootstrapImpl(dTSecret)
+ }
if (maybeConfigMap.isEmpty) {
logWarning("The executor's init-container config map was not specified. Executors will" +
" therefore not attempt to fetch remote or submitted dependencies.")
@@ -80,6 +97,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)),
@@ -90,7 +111,9 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit
new KubernetesClusterSchedulerBackend(
sc.taskScheduler.asInstanceOf[TaskSchedulerImpl],
sc,
- bootStrap,
+ initBootStrap,
+ hadoopBootStrap,
+ kerberosBootstrap,
executorInitContainerSecretVolumePlugin,
kubernetesClient)
}
@@ -98,4 +121,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 a50a9c8bb9c3b..a374c4805aa23 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
@@ -34,7 +34,7 @@ import io.fabric8.kubernetes.client.Watcher.Action
import org.apache.commons.io.FilenameUtils
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.InitContainerUtil
@@ -50,6 +50,8 @@ private[spark] class KubernetesClusterSchedulerBackend(
scheduler: TaskSchedulerImpl,
val sc: SparkContext,
executorInitContainerBootstrap: Option[SparkPodInitContainerBootstrap],
+ executorHadoopBootStrap: Option[HadoopConfBootstrap],
+ executorKerberosBootStrap: Option[KerberosConfBootstrap],
executorMountInitContainerSecretPlugin: Option[InitContainerResourceStagingServerSecretPlugin],
kubernetesClient: KubernetesClient)
extends CoarseGrainedSchedulerBackend(scheduler, sc.env.rpcEnv) {
@@ -582,9 +584,23 @@ private[spark] class KubernetesClusterSchedulerBackend(
val executorPodWithNodeAffinity = addNodeAffinityAnnotationIfUseful(
executorPodWithInitContainer, nodeToLocalTaskCount)
- val resolvedExecutorPod = new PodBuilder(executorPodWithNodeAffinity)
+ 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(initBootstrappedExecutorContainer)
+ .addToContainers(executorKerberosContainer)
.endSpec()
.build()
try {
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 e4f221ad99cc5..babb65bf743b1 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
@@ -42,6 +42,7 @@ private[spark] class DriverConfigurationStepsOrchestratorSuite extends SparkFunS
MAIN_CLASS,
APP_ARGS,
ADDITIONAL_PYTHON_FILES,
+ None,
sparkConf)
val steps = orchestrator.getAllConfigurationSteps()
assert(steps.size === 3)
@@ -63,6 +64,7 @@ private[spark] class DriverConfigurationStepsOrchestratorSuite extends SparkFunS
MAIN_CLASS,
APP_ARGS,
ADDITIONAL_PYTHON_FILES,
+ None,
sparkConf)
val steps = orchestrator.getAllConfigurationSteps()
assert(steps.size === 4)
@@ -84,6 +86,7 @@ private[spark] class DriverConfigurationStepsOrchestratorSuite extends SparkFunS
MAIN_CLASS,
APP_ARGS,
ADDITIONAL_PYTHON_FILES,
+ None,
sparkConf)
val steps = orchestrator.getAllConfigurationSteps()
assert(steps.size === 4)
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..3ed77980d8502
--- /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,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.deploy.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")
+ }
+ // 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/namenode-hadoop-pv.yml b/resource-managers/kubernetes/integration-tests/kerberos-yml/namenode-hadoop-pv.yml
new file mode 100644
index 0000000000000..d813c90030444
--- /dev/null
+++ b/resource-managers/kubernetes/integration-tests/kerberos-yml/namenode-hadoop-pv.yml
@@ -0,0 +1,14 @@
+kind: PersistentVolume
+apiVersion: v1
+metadata:
+ name: nn-hadoop
+ labels:
+ type: local
+ job: kerberostest
+spec:
+ capacity:
+ storage: 1Gi
+ accessModes:
+ - ReadWriteOnce
+ hostPath:
+ path: "/tmp/nn"
\ No newline at end of file
diff --git a/resource-managers/kubernetes/integration-tests/kerberos-yml/namenode-hadoop.yml b/resource-managers/kubernetes/integration-tests/kerberos-yml/namenode-hadoop.yml
new file mode 100644
index 0000000000000..3e72046cd1d8f
--- /dev/null
+++ b/resource-managers/kubernetes/integration-tests/kerberos-yml/namenode-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-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-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-pv.yml b/resource-managers/kubernetes/integration-tests/kerberos-yml/server-keytab-pv.yml
new file mode 100644
index 0000000000000..50298c5fce13e
--- /dev/null
+++ b/resource-managers/kubernetes/integration-tests/kerberos-yml/server-keytab-pv.yml
@@ -0,0 +1,14 @@
+kind: PersistentVolume
+apiVersion: v1
+metadata:
+ name: server-keytab
+ labels:
+ type: local
+ job: kerberostest
+spec:
+ capacity:
+ storage: 1Gi
+ accessModes:
+ - ReadWriteOnce
+ hostPath:
+ path: "/tmp/keytab"
\ No newline at end of file
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..fca6fd01b7052
--- /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.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.hadoop.fs.defaultFS=hdfs://nn.${NAMESPACE}.svc.cluster.local:9000 \
+ --conf spark.kubernetes.kerberos=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 0da1e38d8c211..ab9f356b2d680 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
+
+
+
+
+
+ hadoop.security.authentication
+ kerberos
+
+
+
+ hadoop.security.authorization
+ true
+
+
+
+ fs.defaultFS
+ hdfs://nn.REPLACE_ME.svc.cluster.local:9000
+
+
\ No newline at end of file
diff --git a/resource-managers/kubernetes/integration-tests/src/test/resources/hdfs-site.xml b/resource-managers/kubernetes/integration-tests/src/test/resources/hdfs-site.xml
new file mode 100644
index 0000000000000..bf77244d22567
--- /dev/null
+++ b/resource-managers/kubernetes/integration-tests/src/test/resources/hdfs-site.xml
@@ -0,0 +1,144 @@
+
+
+
+
+
+
+
+
+
+ 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.datanode.keytab.file
+ /var/keytabs/hdfs.keytab
+
+
+ dfs.datanode.kerberos.principal
+ hdfs/dn1.REPLACE_ME.svc.cluster.local@CLUSTER.LOCAL
+
+
+
+
+ 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/src/test/resources/krb5.conf b/resource-managers/kubernetes/integration-tests/src/test/resources/krb5.conf
new file mode 100644
index 0000000000000..5c189a09be6c4
--- /dev/null
+++ b/resource-managers/kubernetes/integration-tests/src/test/resources/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 = KEYRING:persistent:%{uid}
+
+[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/src/test/resources/yarn-site.xml b/resource-managers/kubernetes/integration-tests/src/test/resources/yarn-site.xml
new file mode 100644
index 0000000000000..92d9346232c76
--- /dev/null
+++ b/resource-managers/kubernetes/integration-tests/src/test/resources/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/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KerberizedHadoopClusterLauncher.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KerberizedHadoopClusterLauncher.scala
new file mode 100644
index 0000000000000..ce8f456d3e11d
--- /dev/null
+++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KerberizedHadoopClusterLauncher.scala
@@ -0,0 +1,52 @@
+/*
+ * 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 io.fabric8.kubernetes.client.KubernetesClient
+
+import org.apache.spark.deploy.kubernetes.integrationtest.kerberos._
+
+ /**
+ * This class is responsible for launching a psuedo-distributed, single noded,
+ * kerberized, Hadoop cluster to test secure HDFS interaction. Because each node:
+ * kdc, data node, and name node rely on Persistent Volumes and Config Maps to be set,
+ * and a particular order in pod-launching, this class leverages Watchers and thread locks
+ * to ensure that order is always preserved and the cluster is the same for every run.
+ */
+private[spark] class KerberizedHadoopClusterLauncher(
+ kubernetesClient: KubernetesClient,
+ namespace: String) {
+ private val LABELS = Map("job" -> "kerberostest")
+
+ def launchKerberizedCluster(): Unit = {
+ // These Utils allow for each step in this launch process to re-use
+ // common functionality for setting up hadoop nodes.
+ val kerberosUtils = new KerberosUtils(kubernetesClient, namespace)
+ // Launches persistent volumes and its claims for sharing keytabs across pods
+ val pvWatcherCache = new KerberosPVWatcherCache(kerberosUtils, LABELS)
+ pvWatcherCache.start()
+ pvWatcherCache.stop()
+ // Launches config map for the files in HADOOP_CONF_DIR
+ val cmWatcherCache = new KerberosCMWatcherCache(kerberosUtils)
+ cmWatcherCache.start()
+ cmWatcherCache.stop()
+ // Launches the Hadoop cluster pods: KDC --> NN --> DN1 --> Data-Populator
+ val podWatcherCache = new KerberosPodWatcherCache(kerberosUtils, LABELS)
+ podWatcherCache.start()
+ podWatcherCache.stop()
+ }
+}
\ No newline at end of file
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..532fc221878ff
--- /dev/null
+++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KerberosTestPodLauncher.scala
@@ -0,0 +1,118 @@
+/*
+ * 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 yamlLocation = "kerberos-yml/kerberos-test.yml"
+ 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): 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)}
+ }
+}
\ No newline at end of file
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 c6cd6a74c88d1..3c01f17501285 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
@@ -16,27 +16,22 @@
*/
package org.apache.spark.deploy.kubernetes.integrationtest
-import java.io.File
import java.nio.file.Paths
import java.util.UUID
-import com.google.common.base.Charsets
-import com.google.common.io.Files
import io.fabric8.kubernetes.client.internal.readiness.Readiness
-import org.scalatest.BeforeAndAfter
-import org.scalatest.concurrent.{Eventually, PatienceConfiguration}
-import org.scalatest.time.{Minutes, Seconds, Span}
-import scala.collection.JavaConverters._
-
-import org.apache.spark.{SparkConf, SparkFunSuite, SSLOptions}
-import org.apache.spark.deploy.kubernetes.SSLUtils
+import org.apache.spark.{SSLOptions, SparkConf, SparkFunSuite}
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.submit.{Client, ClientArguments, JavaMainAppResource, KeyAndCertPem, MainAppResource, PythonMainAppResource}
-import org.apache.spark.launcher.SparkLauncher
-import org.apache.spark.util.Utils
+import org.apache.spark.deploy.kubernetes.integrationtest.kerberos.KerberosDriverWatcherCache
+import org.apache.spark.deploy.kubernetes.submit._
+import org.scalatest.BeforeAndAfter
+import org.scalatest.concurrent.{Eventually, PatienceConfiguration}
+import org.scalatest.time.{Minutes, Seconds, Span}
+
+import scala.collection.JavaConverters._
private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter {
import KubernetesSuite._
@@ -46,6 +41,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 +51,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,169 +72,212 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter {
}
after {
+ kubernetesTestComponents.deleteKubernetesResources()
kubernetesTestComponents.deleteNamespace()
}
- 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,
- System.getProperty("spark.docker.test.driverImage", "spark-driver-py:latest"))
- .set(EXECUTOR_DOCKER_IMAGE,
- System.getProperty("spark.docker.test.executorImage", "spark-executor-py:latest"))
-
- runPySparkPiAndVerifyCompletion(
- PYSPARK_PI_SUBMITTER_LOCAL_FILE_LOCATION,
- Seq(PYSPARK_SORT_CONTAINER_LOCAL_FILE_LOCATION)
- )
- }
-
- test("Run PySpark Job on file from CONTAINER with spark.jar defined") {
- assume(testBackend.name == MINIKUBE_TEST_BACKEND)
-
- sparkConf.setJars(Seq(CONTAINER_LOCAL_HELPER_JAR_PATH))
- sparkConf
- .set(DRIVER_DOCKER_IMAGE,
- System.getProperty("spark.docker.test.driverImage", "spark-driver-py:latest"))
- .set(EXECUTOR_DOCKER_IMAGE,
- System.getProperty("spark.docker.test.executorImage", "spark-executor-py:latest"))
-
- runPySparkPiAndVerifyCompletion(PYSPARK_PI_CONTAINER_LOCAL_FILE_LOCATION, Seq.empty[String])
- }
-
- test("Simple submission test with the resource staging server.") {
- assume(testBackend.name == MINIKUBE_TEST_BACKEND)
-
- launchStagingServer(SSLOptions(), None)
- runSparkPiAndVerifyCompletion(SUBMITTER_LOCAL_MAIN_APP_RESOURCE)
- }
-
- test("Enable SSL on the resource staging server") {
+ test("Include HADOOP_CONF for HDFS based jobs") {
assume(testBackend.name == MINIKUBE_TEST_BACKEND)
-
- val keyStoreAndTrustStore = SSLUtils.generateKeyStoreTrustStorePair(
- ipAddress = Minikube.getMinikubeIp,
- keyStorePassword = "keyStore",
- keyPassword = "key",
- trustStorePassword = "trustStore")
- sparkConf.set(RESOURCE_STAGING_SERVER_SSL_ENABLED, true)
- .set("spark.ssl.kubernetes.resourceStagingServer.keyStore",
- keyStoreAndTrustStore.keyStore.getAbsolutePath)
- .set("spark.ssl.kubernetes.resourceStagingServer.trustStore",
- keyStoreAndTrustStore.trustStore.getAbsolutePath)
- .set("spark.ssl.kubernetes.resourceStagingServer.keyStorePassword", "keyStore")
- .set("spark.ssl.kubernetes.resourceStagingServer.keyPassword", "key")
- .set("spark.ssl.kubernetes.resourceStagingServer.trustStorePassword", "trustStore")
- launchStagingServer(SSLOptions(
- enabled = true,
- keyStore = Some(keyStoreAndTrustStore.keyStore),
- trustStore = Some(keyStoreAndTrustStore.trustStore),
- keyStorePassword = Some("keyStore"),
- keyPassword = Some("key"),
- trustStorePassword = Some("trustStore")),
- None)
- runSparkPiAndVerifyCompletion(SUBMITTER_LOCAL_MAIN_APP_RESOURCE)
- }
-
- test("Use container-local resources without the resource staging server") {
- assume(testBackend.name == MINIKUBE_TEST_BACKEND)
-
+ // Ensuring that HADOOP_CONF_DIR variable is set, could also be one via env HADOOP_CONF_DIR
sparkConf.setJars(Seq(CONTAINER_LOCAL_HELPER_JAR_PATH))
- runSparkPiAndVerifyCompletion(CONTAINER_LOCAL_MAIN_APP_RESOURCE)
- }
-
- test("Dynamic executor scaling basic test") {
- assume(testBackend.name == MINIKUBE_TEST_BACKEND)
-
- launchStagingServer(SSLOptions(), None)
- createShuffleServiceDaemonSet()
-
- sparkConf.setJars(Seq(CONTAINER_LOCAL_HELPER_JAR_PATH))
- sparkConf.set("spark.dynamicAllocation.enabled", "true")
- sparkConf.set("spark.shuffle.service.enabled", "true")
- sparkConf.set("spark.kubernetes.shuffle.labels", "app=spark-shuffle-service")
- sparkConf.set("spark.kubernetes.shuffle.namespace", kubernetesTestComponents.namespace)
- sparkConf.set("spark.app.name", "group-by-test")
runSparkApplicationAndVerifyCompletion(
- JavaMainAppResource(SUBMITTER_LOCAL_MAIN_APP_RESOURCE),
- GROUP_BY_MAIN_CLASS,
- Seq("The Result is"),
- Array.empty[String],
- Seq.empty[String])
- }
-
- test("Use remote resources without the resource staging server.") {
- assume(testBackend.name == MINIKUBE_TEST_BACKEND)
- val assetServerUri = staticAssetServerLauncher.launchStaticAssetServer()
- sparkConf.setJars(Seq(
- s"$assetServerUri/${EXAMPLES_JAR_FILE.getName}",
- s"$assetServerUri/${HELPER_JAR_FILE.getName}"
- ))
- runSparkPiAndVerifyCompletion(SparkLauncher.NO_RESOURCE)
- }
-
- test("Mix remote resources with submitted ones.") {
- assume(testBackend.name == MINIKUBE_TEST_BACKEND)
- launchStagingServer(SSLOptions(), None)
- val assetServerUri = staticAssetServerLauncher.launchStaticAssetServer()
- sparkConf.setJars(Seq(
- SUBMITTER_LOCAL_MAIN_APP_RESOURCE, s"$assetServerUri/${HELPER_JAR_FILE.getName}"
- ))
- runSparkPiAndVerifyCompletion(SparkLauncher.NO_RESOURCE)
- }
-
- test("Use key and certificate PEM files for TLS.") {
- assume(testBackend.name == MINIKUBE_TEST_BACKEND)
- val keyAndCertificate = SSLUtils.generateKeyCertPemPair(Minikube.getMinikubeIp)
- launchStagingServer(
- SSLOptions(enabled = true),
- Some(keyAndCertificate))
- sparkConf.set(RESOURCE_STAGING_SERVER_SSL_ENABLED, true)
- .set(
- RESOURCE_STAGING_SERVER_CLIENT_CERT_PEM.key, keyAndCertificate.certPem.getAbsolutePath)
- runSparkPiAndVerifyCompletion(SUBMITTER_LOCAL_MAIN_APP_RESOURCE)
- }
-
- test("Use client key and client cert file when requesting executors") {
- assume(testBackend.name == MINIKUBE_TEST_BACKEND)
- sparkConf.setJars(Seq(
- CONTAINER_LOCAL_MAIN_APP_RESOURCE,
- CONTAINER_LOCAL_HELPER_JAR_PATH))
- sparkConf.set(
- s"$APISERVER_AUTH_DRIVER_CONF_PREFIX.$CLIENT_KEY_FILE_CONF_SUFFIX",
- kubernetesTestComponents.clientConfig.getClientKeyFile)
- sparkConf.set(
- s"$APISERVER_AUTH_DRIVER_CONF_PREFIX.$CLIENT_CERT_FILE_CONF_SUFFIX",
- kubernetesTestComponents.clientConfig.getClientCertFile)
- sparkConf.set(
- s"$APISERVER_AUTH_DRIVER_CONF_PREFIX.$CA_CERT_FILE_CONF_SUFFIX",
- kubernetesTestComponents.clientConfig.getCaCertFile)
- runSparkPiAndVerifyCompletion(SparkLauncher.NO_RESOURCE)
+ 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("src/test/resources"))
}
- test("Added files should be placed in the driver's working directory.") {
+ test("Secure HDFS test with HDFS keytab") {
assume(testBackend.name == MINIKUBE_TEST_BACKEND)
- val testExistenceFileTempDir = Utils.createTempDir(namePrefix = "test-existence-file-temp-dir")
- val testExistenceFile = new File(testExistenceFileTempDir, "input.txt")
- Files.write(TEST_EXISTENCE_FILE_CONTENTS, testExistenceFile, Charsets.UTF_8)
- launchStagingServer(SSLOptions(), None)
- sparkConf.set("spark.files", testExistenceFile.getAbsolutePath)
- runSparkApplicationAndVerifyCompletion(
- JavaMainAppResource(SUBMITTER_LOCAL_MAIN_APP_RESOURCE),
- FILE_EXISTENCE_MAIN_CLASS,
- Seq(s"File found at /opt/spark/${testExistenceFile.getName} with correct contents."),
- Array(testExistenceFile.getName, TEST_EXISTENCE_FILE_CONTENTS),
- Seq.empty[String])
+ launchKerberizedCluster()
+ createKerberosTestPod(CONTAINER_LOCAL_MAIN_APP_RESOURCE, HDFS_TEST_CLASS, APP_LOCATOR_LABEL)
+ val kubernetesClient = kubernetesTestComponents.kubernetesClient
+ val driverWatcherCache = new KerberosDriverWatcherCache(
+ kubernetesClient,
+ Map("spark-app-locator" -> APP_LOCATOR_LABEL))
+ driverWatcherCache.start()
+ driverWatcherCache.stop()
+ val expectedLogOnCompletion = Seq("Something something something")
+ 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("Use a very long application name.") {
- assume(testBackend.name == MINIKUBE_TEST_BACKEND)
-
- sparkConf.setJars(Seq(CONTAINER_LOCAL_HELPER_JAR_PATH)).setAppName("long" * 40)
- runSparkPiAndVerifyCompletion(CONTAINER_LOCAL_MAIN_APP_RESOURCE)
- }
+// 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,
+// System.getProperty("spark.docker.test.driverImage", "spark-driver-py:latest"))
+// .set(EXECUTOR_DOCKER_IMAGE,
+// System.getProperty("spark.docker.test.executorImage", "spark-executor-py:latest"))
+//
+// runPySparkPiAndVerifyCompletion(
+// PYSPARK_PI_SUBMITTER_LOCAL_FILE_LOCATION,
+// Seq(PYSPARK_SORT_CONTAINER_LOCAL_FILE_LOCATION)
+// )
+// }
+//
+// test("Run PySpark Job on file from CONTAINER with spark.jar defined") {
+// assume(testBackend.name == MINIKUBE_TEST_BACKEND)
+//
+// sparkConf.setJars(Seq(CONTAINER_LOCAL_HELPER_JAR_PATH))
+// sparkConf
+// .set(DRIVER_DOCKER_IMAGE,
+// System.getProperty("spark.docker.test.driverImage", "spark-driver-py:latest"))
+// .set(EXECUTOR_DOCKER_IMAGE,
+// System.getProperty("spark.docker.test.executorImage", "spark-executor-py:latest"))
+//
+// runPySparkPiAndVerifyCompletion(PYSPARK_PI_CONTAINER_LOCAL_FILE_LOCATION, Seq.empty[String])
+// }
+//
+// test("Simple submission test with the resource staging server.") {
+// assume(testBackend.name == MINIKUBE_TEST_BACKEND)
+//
+// launchStagingServer(SSLOptions(), None)
+// runSparkPiAndVerifyCompletion(SUBMITTER_LOCAL_MAIN_APP_RESOURCE)
+// }
+//
+// test("Enable SSL on the resource staging server") {
+// assume(testBackend.name == MINIKUBE_TEST_BACKEND)
+//
+// val keyStoreAndTrustStore = SSLUtils.generateKeyStoreTrustStorePair(
+// ipAddress = Minikube.getMinikubeIp,
+// keyStorePassword = "keyStore",
+// keyPassword = "key",
+// trustStorePassword = "trustStore")
+// sparkConf.set(RESOURCE_STAGING_SERVER_SSL_ENABLED, true)
+// .set("spark.ssl.kubernetes.resourceStagingServer.keyStore",
+// keyStoreAndTrustStore.keyStore.getAbsolutePath)
+// .set("spark.ssl.kubernetes.resourceStagingServer.trustStore",
+// keyStoreAndTrustStore.trustStore.getAbsolutePath)
+// .set("spark.ssl.kubernetes.resourceStagingServer.keyStorePassword", "keyStore")
+// .set("spark.ssl.kubernetes.resourceStagingServer.keyPassword", "key")
+// .set("spark.ssl.kubernetes.resourceStagingServer.trustStorePassword", "trustStore")
+// launchStagingServer(SSLOptions(
+// enabled = true,
+// keyStore = Some(keyStoreAndTrustStore.keyStore),
+// trustStore = Some(keyStoreAndTrustStore.trustStore),
+// keyStorePassword = Some("keyStore"),
+// keyPassword = Some("key"),
+// trustStorePassword = Some("trustStore")),
+// None)
+// runSparkPiAndVerifyCompletion(SUBMITTER_LOCAL_MAIN_APP_RESOURCE)
+// }
+//
+// test("Use container-local resources without the resource staging server") {
+// assume(testBackend.name == MINIKUBE_TEST_BACKEND)
+//
+// sparkConf.setJars(Seq(CONTAINER_LOCAL_HELPER_JAR_PATH))
+// runSparkPiAndVerifyCompletion(CONTAINER_LOCAL_MAIN_APP_RESOURCE)
+// }
+//
+// test("Dynamic executor scaling basic test") {
+// assume(testBackend.name == MINIKUBE_TEST_BACKEND)
+//
+// launchStagingServer(SSLOptions(), None)
+// createShuffleServiceDaemonSet()
+//
+// sparkConf.setJars(Seq(CONTAINER_LOCAL_HELPER_JAR_PATH))
+// sparkConf.set("spark.dynamicAllocation.enabled", "true")
+// sparkConf.set("spark.shuffle.service.enabled", "true")
+// sparkConf.set("spark.kubernetes.shuffle.labels", "app=spark-shuffle-service")
+// sparkConf.set("spark.kubernetes.shuffle.namespace", kubernetesTestComponents.namespace)
+// sparkConf.set("spark.app.name", "group-by-test")
+// runSparkApplicationAndVerifyCompletion(
+// JavaMainAppResource(SUBMITTER_LOCAL_MAIN_APP_RESOURCE),
+// GROUP_BY_MAIN_CLASS,
+// Seq("The Result is"),
+// Array.empty[String],
+// Seq.empty[String],
+// None)
+// }
+//
+// test("Use remote resources without the resource staging server.") {
+// assume(testBackend.name == MINIKUBE_TEST_BACKEND)
+// val assetServerUri = staticAssetServerLauncher.launchStaticAssetServer()
+// sparkConf.setJars(Seq(
+// s"$assetServerUri/${EXAMPLES_JAR_FILE.getName}",
+// s"$assetServerUri/${HELPER_JAR_FILE.getName}"
+// ))
+// runSparkPiAndVerifyCompletion(SparkLauncher.NO_RESOURCE)
+// }
+//
+// test("Mix remote resources with submitted ones.") {
+// assume(testBackend.name == MINIKUBE_TEST_BACKEND)
+// launchStagingServer(SSLOptions(), None)
+// val assetServerUri = staticAssetServerLauncher.launchStaticAssetServer()
+// sparkConf.setJars(Seq(
+// SUBMITTER_LOCAL_MAIN_APP_RESOURCE, s"$assetServerUri/${HELPER_JAR_FILE.getName}"
+// ))
+// runSparkPiAndVerifyCompletion(SparkLauncher.NO_RESOURCE)
+// }
+//
+// test("Use key and certificate PEM files for TLS.") {
+// assume(testBackend.name == MINIKUBE_TEST_BACKEND)
+// val keyAndCertificate = SSLUtils.generateKeyCertPemPair(Minikube.getMinikubeIp)
+// launchStagingServer(
+// SSLOptions(enabled = true),
+// Some(keyAndCertificate))
+// sparkConf.set(RESOURCE_STAGING_SERVER_SSL_ENABLED, true)
+// .set(
+// RESOURCE_STAGING_SERVER_CLIENT_CERT_PEM.key, keyAndCertificate.certPem.getAbsolutePath)
+// runSparkPiAndVerifyCompletion(SUBMITTER_LOCAL_MAIN_APP_RESOURCE)
+// }
+//
+// test("Use client key and client cert file when requesting executors") {
+// assume(testBackend.name == MINIKUBE_TEST_BACKEND)
+// sparkConf.setJars(Seq(
+// CONTAINER_LOCAL_MAIN_APP_RESOURCE,
+// CONTAINER_LOCAL_HELPER_JAR_PATH))
+// sparkConf.set(
+// s"$APISERVER_AUTH_DRIVER_CONF_PREFIX.$CLIENT_KEY_FILE_CONF_SUFFIX",
+// kubernetesTestComponents.clientConfig.getClientKeyFile)
+// sparkConf.set(
+// s"$APISERVER_AUTH_DRIVER_CONF_PREFIX.$CLIENT_CERT_FILE_CONF_SUFFIX",
+// kubernetesTestComponents.clientConfig.getClientCertFile)
+// sparkConf.set(
+// s"$APISERVER_AUTH_DRIVER_CONF_PREFIX.$CA_CERT_FILE_CONF_SUFFIX",
+// kubernetesTestComponents.clientConfig.getCaCertFile)
+// runSparkPiAndVerifyCompletion(SparkLauncher.NO_RESOURCE)
+// }
+//
+// test("Added files should be placed in the driver's working directory.") {
+// assume(testBackend.name == MINIKUBE_TEST_BACKEND)
+// val testExistenceFileTempDir = Utils.createTempDir(namePrefix = "test-existence-file-temp-dir")
+// val testExistenceFile = new File(testExistenceFileTempDir, "input.txt")
+// Files.write(TEST_EXISTENCE_FILE_CONTENTS, testExistenceFile, Charsets.UTF_8)
+// launchStagingServer(SSLOptions(), None)
+// sparkConf.set("spark.files", testExistenceFile.getAbsolutePath)
+// runSparkApplicationAndVerifyCompletion(
+// JavaMainAppResource(SUBMITTER_LOCAL_MAIN_APP_RESOURCE),
+// FILE_EXISTENCE_MAIN_CLASS,
+// Seq(s"File found at /opt/spark/${testExistenceFile.getName} with correct contents."),
+// Array(testExistenceFile.getName, TEST_EXISTENCE_FILE_CONTENTS),
+// Seq.empty[String],
+// None)
+// }
+//
+// test("Use a very long application name.") {
+// assume(testBackend.name == MINIKUBE_TEST_BACKEND)
+//
+// sparkConf.setJars(Seq(CONTAINER_LOCAL_HELPER_JAR_PATH)).setAppName("long" * 40)
+// runSparkPiAndVerifyCompletion(CONTAINER_LOCAL_MAIN_APP_RESOURCE)
+// }
private def launchStagingServer(
resourceStagingServerSslOptions: SSLOptions, keyAndCertPem: Option[KeyAndCertPem]): Unit = {
@@ -249,13 +295,26 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter {
s"${Minikube.getMinikubeIp}:$resourceStagingServerPort")
}
+ private def launchKerberizedCluster(): Unit = {
+ assume(testBackend.name == MINIKUBE_TEST_BACKEND)
+ kerberizedHadoopClusterLauncher.launchKerberizedCluster()
+ }
+
+ private def createKerberosTestPod(resource: String, className: String, appLabel: String): Unit = {
+ assume(testBackend.name == MINIKUBE_TEST_BACKEND)
+ kerberosTestLauncher.startKerberosTest(resource, className, appLabel)
+ }
+
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(
@@ -265,7 +324,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(
@@ -273,13 +333,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)
@@ -354,8 +415,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(20, Minutes))
+ val INTERVAL = PatienceConfiguration.Interval(Span(20, Seconds))
val SPARK_PI_MAIN_CLASS = "org.apache.spark.deploy.kubernetes" +
".integrationtest.jobs.SparkPiWithInfiniteWait"
val PYSPARK_PI_MAIN_CLASS = "org.apache.spark.deploy.PythonRunner"
@@ -368,7 +429,10 @@ 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 TEST_EXISTENCE_FILE_CONTENTS = "contents"
+
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..a0693f2f46dd4 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
@@ -19,12 +19,13 @@ package org.apache.spark.deploy.kubernetes.integrationtest
import java.util.UUID
import io.fabric8.kubernetes.client.DefaultKubernetesClient
-import org.scalatest.concurrent.Eventually
import scala.collection.JavaConverters._
import org.apache.spark.SparkConf
import org.apache.spark.deploy.kubernetes.config._
+import org.scalatest.concurrent.Eventually
+
private[spark] class KubernetesTestComponents(defaultClient: DefaultKubernetesClient) {
val namespace = UUID.randomUUID().toString.replaceAll("-", "")
@@ -51,6 +52,27 @@ private[spark] class KubernetesTestComponents(defaultClient: DefaultKubernetesCl
}
}
+ def deleteKubernetesResources(): Unit = {
+ kubernetesClient.persistentVolumes().delete()
+ Eventually.eventually(KubernetesSuite.TIMEOUT, KubernetesSuite.INTERVAL) {
+ val persistentList = kubernetesClient
+ .persistentVolumes()
+ .list()
+ .getItems()
+ .asScala
+ require(!persistentList.exists(_.getMetadata.getNamespace == namespace))
+ }
+ kubernetesClient.configMaps().delete()
+ Eventually.eventually(KubernetesSuite.TIMEOUT, KubernetesSuite.INTERVAL) {
+ val configMapsList = kubernetesClient
+ .configMaps()
+ .list()
+ .getItems()
+ .asScala
+ require(!configMapsList.exists(_.getMetadata.getNamespace == namespace))
+ }
+ }
+
def newSparkConf(): SparkConf = {
new SparkConf(true)
.setMaster(s"k8s://${kubernetesClient.getMasterUrl}")
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..84f38e482f037 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
@@ -20,15 +20,16 @@ import java.io.File
import java.net.URI
import java.nio.file.Paths
-import scala.collection.JavaConverters._
-
import com.spotify.docker.client.{DefaultDockerClient, DockerCertificates, LoggingBuildHandler}
import org.apache.http.client.utils.URIBuilder
-import org.scalatest.concurrent.{Eventually, PatienceConfiguration}
-import org.scalatest.time.{Minutes, Seconds, Span}
import org.apache.spark.internal.Logging
+
import org.apache.spark.util.RedirectThread
+import org.scalatest.concurrent.{Eventually, PatienceConfiguration}
+import org.scalatest.time.{Minutes, Seconds, Span}
+
+import scala.collection.JavaConverters._
@@ -47,6 +48,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 +97,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 = {
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..1e1db004fbb7b
--- /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)
\ No newline at end of file
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..fff02c2d06ef7
--- /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")
+ 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..ee7aeeaa9c7dc
--- /dev/null
+++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/kerberos/KerberosPodWatcherCache.scala
@@ -0,0 +1,240 @@
+/*
+ * 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 if 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 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")
+ 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(): Unit = {
+ podWatcher.close()
+ serviceWatcher.close()
+ }
+
+ 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()
+ 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"
+ }
+ }
+}
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..16284fd49c270
--- /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)
\ No newline at end of file
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..9bb06d88ef608
--- /dev/null
+++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/kerberos/KerberosUtils.scala
@@ -0,0 +1,129 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.deploy.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 def locationResolver(loc: String) = s"src/test/resources/$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))}
+ 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.map(file =>
+ new KeyToPathBuilder()
+ .withKey(file)
+ .withPath(file)
+ .build()).toList
+ private val pvNN = Seq("namenode-hadoop", "namenode-hadoop-pv")
+ private val pvKT = Seq("server-keytab", "server-keytab-pv")
+ private def buildKerberosPV(seqPair: Seq[String]) = {
+ KerberosStorage(
+ kubernetesClient.load(loadFromYaml(seqPair.head))
+ .get().get(0).asInstanceOf[PersistentVolumeClaim],
+ kubernetesClient.load(loadFromYaml(seqPair(1)))
+ .get().get(0).asInstanceOf[PersistentVolume])
+ }
+ def getNNStorage: KerberosStorage = buildKerberosPV(pvNN)
+ def getKTStorage: KerberosStorage = buildKerberosPV(pvKT)
+ def getLabels: Map[String, String] = PV_LABELS
+ def getPVNN: Seq[String] = pvNN
+ 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_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)
+}