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) +}