From 0f3501a0217e450727d3b871db455fd3f70f2840 Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Fri, 14 Jul 2017 18:37:49 -0700 Subject: [PATCH 01/15] Initial architecture design for HDFS support --- .../kubernetes/HadoopConfBootstrap.scala | 78 +++++++++++++++++++ .../kubernetes/PodWithMainContainer.scala | 23 ++++++ .../spark/deploy/kubernetes/config.scala | 6 ++ .../spark/deploy/kubernetes/constants.scala | 3 + ...DriverConfigurationStepsOrchestrator.scala | 32 +++++++- .../HadoopConfigBootstrapStep.scala | 39 ++++++++++ .../hadoopsteps/HadoopConfMounterStep.scala | 49 ++++++++++++ .../hadoopsteps/HadoopConfigSpec.scala | 33 ++++++++ .../hadoopsteps/HadoopConfigurationStep.scala | 25 ++++++ .../hadoopsteps/HadoopStepsOrchestrator.scala | 53 +++++++++++++ .../kubernetes/KubernetesClusterManager.scala | 7 +- 11 files changed, 343 insertions(+), 5 deletions(-) create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/HadoopConfBootstrap.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/PodWithMainContainer.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/HadoopConfigBootstrapStep.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopConfMounterStep.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopConfigSpec.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopConfigurationStep.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopStepsOrchestrator.scala 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..bc83baa527a27 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/HadoopConfBootstrap.scala @@ -0,0 +1,78 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.kubernetes + +import java.io.File + +import org.apache.spark.deploy.kubernetes.constants._ +import io.fabric8.kubernetes.api.model.{ConfigMapBuilder, ContainerBuilder, KeyToPathBuilder, PodBuilder} + +import collection.JavaConverters._ + + +/** + * This is separated out from the HadoopConf steps API because this component can be reused to + * set up the hadoop-conf 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: Array[File]) extends HadoopConfBootstrap { + + override def bootstrapMainContainerAndVolumes( + originalPodWithMainContainer: PodWithMainContainer) + : PodWithMainContainer = { + val fileContents = hadoopConfigFiles.map(file => (file.getPath, file.toString)).toMap + val keyPaths = hadoopConfigFiles.map(file => + new KeyToPathBuilder().withKey(file.getPath).withPath(file.getAbsolutePath).build()) + val hadoopSupportedPod = new PodBuilder(originalPodWithMainContainer.pod) + .editSpec() + .addNewVolume() + .withName(HADOOP_FILE_VOLUME) + .withNewConfigMap() + .withName(hadoopConfConfigMapName) + .addAllToItems(keyPaths.toList.asJavaCollection) + .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(s"$HADOOP_FILE_DIR/$HADOOP_FILE_VOLUME") + .endEnv() + .build() + PodWithMainContainer( + hadoopSupportedPod, + mainContainerWithMountedHadoopConf + ) + } +} \ No newline at end of file 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..4f182c250fcf4 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/PodWithMainContainer.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 + +import io.fabric8.kubernetes.api.model.{Container, Pod} + +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..7ebcf7253ebb3 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 @@ -498,6 +498,12 @@ package object config extends Logging { .createOptional 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] def resolveK8sMaster(rawMasterString: String): String = { if (!rawMasterString.startsWith("k8s://")) { 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..0dd2940faf66d 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 @@ -86,6 +86,9 @@ package object constants { private[spark] val INIT_CONTAINER_PROPERTIES_FILE_VOLUME = "spark-init-properties" private[spark] val INIT_CONTAINER_PROPERTIES_FILE_DIR = "/etc/spark-init" private[spark] val INIT_CONTAINER_PROPERTIES_FILE_NAME = "spark-init.properties" + 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 INIT_CONTAINER_PROPERTIES_FILE_PATH = s"$INIT_CONTAINER_PROPERTIES_FILE_DIR/$INIT_CONTAINER_PROPERTIES_FILE_NAME" private[spark] val DEFAULT_SHUFFLE_MOUNT_NAME = "shuffle" 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..4b88cd2a3fb00 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 @@ -16,11 +16,14 @@ */ package org.apache.spark.deploy.kubernetes.submit +import java.io.File + 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.hadoopsteps.HadoopStepsOrchestrator +import org.apache.spark.deploy.kubernetes.submit.submitsteps._ import org.apache.spark.deploy.kubernetes.submit.submitsteps.initcontainer.InitContainerConfigurationStepsOrchestrator import org.apache.spark.launcher.SparkLauncher import org.apache.spark.util.Utils @@ -51,6 +54,7 @@ private[spark] class DriverConfigurationStepsOrchestrator( private val filesDownloadPath = submissionSparkConf.get(INIT_CONTAINER_FILES_DOWNLOAD_LOCATION) private val dockerImagePullPolicy = submissionSparkConf.get(DOCKER_IMAGE_PULL_POLICY) private val initContainerConfigMapName = s"$kubernetesResourceNamePrefix-init-config" + private val hadoopConfigMapName = s"$kubernetesResourceNamePrefix-hadoop-config" def getAllConfigurationSteps(): Seq[DriverConfigurationStep] = { val additionalMainAppJar = mainAppResource match { @@ -94,6 +98,22 @@ private[spark] class DriverConfigurationStepsOrchestrator( submissionSparkConf) val kubernetesCredentialsStep = new DriverKubernetesCredentialsStep( submissionSparkConf, kubernetesResourceNamePrefix) + val hadoopConfigurations = + sys.env.get("HADOOP_CONF_DIR").map{ conf => getHadoopConfFiles(conf)} + .getOrElse(Array.empty[File]) + val hadoopConfigSteps = + if (hadoopConfigurations.isEmpty) { + Option.empty[DriverConfigurationStep] + } else { + val hadoopStepsOrchestrator = new HadoopStepsOrchestrator( + namespace, + kubernetesResourceNamePrefix, + submissionSparkConf, + hadoopConfigurations) + val hadoopConfSteps = + hadoopStepsOrchestrator.getHadoopSteps() + Some(new HadoopConfigBootstrapStep(hadoopConfSteps)) + } val pythonStep = mainAppResource match { case PythonMainAppResource(mainPyResource) => Option(new PythonStep(mainPyResource, additionalPythonFiles, filesDownloadPath)) @@ -133,6 +153,16 @@ private[spark] class DriverConfigurationStepsOrchestrator( kubernetesCredentialsStep, dependencyResolutionStep) ++ initContainerBootstrapStep.toSeq ++ + hadoopConfigSteps.toSeq ++ pythonStep.toSeq } + 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/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..685fdd188b497 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/HadoopConfigBootstrapStep.scala @@ -0,0 +1,39 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.kubernetes.submit.submitsteps + +import org.apache.spark.deploy.kubernetes.submit.submitsteps.hadoopsteps.{HadoopConfigSpec, HadoopConfigurationStep} + + /** + * Configures the driverSpec that bootstraps dependencies into the driver pod. + */ +private[spark] class HadoopConfigBootstrapStep( + hadoopConfigurationSteps: Seq[HadoopConfigurationStep]) + extends DriverConfigurationStep { + + override def configureDriver(driverSpec: KubernetesDriverSpec): KubernetesDriverSpec = { + var currentHadoopSpec = HadoopConfigSpec( + driverPod = driverSpec.driverPod, + driverContainer = driverSpec.driverContainer) + for (nextStep <- hadoopConfigurationSteps) { + currentHadoopSpec = nextStep.configureContainers(currentHadoopSpec) + } + driverSpec.copy( + driverPod = currentHadoopSpec.driverPod, + driverContainer = currentHadoopSpec.driverContainer) + } +} 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..6410221fc48e0 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopConfMounterStep.scala @@ -0,0 +1,49 @@ +/* + * 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 io.fabric8.kubernetes.api.model._ +import org.apache.spark.deploy.kubernetes.{HadoopConfBootstrap, PodWithMainContainer} +import org.apache.spark.deploy.kubernetes.config._ +import org.apache.spark.deploy.kubernetes.constants._ +import org.apache.spark.deploy.kubernetes.submit.KubernetesFileUtils +import org.apache.spark.deploy.kubernetes.submit.submitsteps.{DriverKubernetesCredentialsStep, KubernetesDriverSpec} +import scala.collection.JavaConverters._ + + /** + * Step that configures the ConfigMap + Volumes for the driver + */ +private[spark] class HadoopConfMounterStep( + hadoopConfigMapName: String, + hadoopConfBootstrapConf: HadoopConfBootstrap) + 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 + ) + } +} \ No newline at end of file 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..aaa49d9b45496 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopConfigSpec.scala @@ -0,0 +1,33 @@ +/* + * 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, HasMetadata, Pod} + + /** + * 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) + */ +private[spark] case class HadoopConfigSpec( +// additionalDriverSparkConf: Map[String, String], + driverPod: Pod, + driverContainer: Container) 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..2b5aca1aadddc --- /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 + */ +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/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..b6ecc5dce8a82 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopStepsOrchestrator.scala @@ -0,0 +1,53 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.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, + kubernetesResourceNamePrefix: String, + submissionSparkConf: SparkConf, + hadoopConfigurationFiles: Array[File]) { + private val hadoopConfigMapName = s"$kubernetesResourceNamePrefix-hadoop-config" + private val maybeKerberosSupport = submissionSparkConf.get(KUBERNETES_KERBEROS_SUPPORT) + + def getHadoopSteps(): Seq[HadoopConfigurationStep] = { + val hadoopConfBootstrapImpl = new HadoopConfBootstrapImpl( + hadoopConfigMapName, + hadoopConfigurationFiles) + val hadoopConfMounterStep = new HadoopConfMounterStep( + hadoopConfigMapName, + hadoopConfBootstrapImpl) + val maybeHadoopKerberosMountingStep = + if (maybeKerberosSupport) { + // TODO: Implement mounting secrets + Option.empty[HadoopConfigurationStep] + } else { + Option.empty[HadoopConfigurationStep] + } + Seq(hadoopConfMounterStep) ++ maybeHadoopKerberosMountingStep.toSeq + } +} 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..fe3109d9f14ed 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 @@ -19,9 +19,8 @@ package org.apache.spark.scheduler.cluster.kubernetes 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.{HadoopConfBootstrapImpl, InitContainerResourceStagingServerSecretPluginImpl, SparkKubernetesClientFactory, SparkPodInitContainerBootstrapImpl} import org.apache.spark.deploy.kubernetes.config._ import org.apache.spark.deploy.kubernetes.constants._ import org.apache.spark.internal.Logging @@ -59,7 +58,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 { @@ -90,7 +89,7 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit new KubernetesClusterSchedulerBackend( sc.taskScheduler.asInstanceOf[TaskSchedulerImpl], sc, - bootStrap, + initBootStrap, executorInitContainerSecretVolumePlugin, kubernetesClient) } From b12a43913ec807103390bc2a5064f432a298a759 Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Fri, 14 Jul 2017 19:30:38 -0700 Subject: [PATCH 02/15] Minor styling --- .../kubernetes/HadoopConfBootstrap.scala | 23 +++++++++---------- ...DriverConfigurationStepsOrchestrator.scala | 2 +- .../hadoopsteps/HadoopConfMounterStep.scala | 10 +------- .../kubernetes/KubernetesClusterManager.scala | 6 ++--- 4 files changed, 16 insertions(+), 25 deletions(-) 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 index bc83baa527a27..18daff73b82b2 100644 --- 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 @@ -18,21 +18,19 @@ package org.apache.spark.deploy.kubernetes import java.io.File -import org.apache.spark.deploy.kubernetes.constants._ -import io.fabric8.kubernetes.api.model.{ConfigMapBuilder, ContainerBuilder, KeyToPathBuilder, PodBuilder} - -import collection.JavaConverters._ +import io.fabric8.kubernetes.api.model.{ContainerBuilder, KeyToPathBuilder, PodBuilder} +import org.apache.spark.deploy.kubernetes.constants._ /** - * This is separated out from the HadoopConf steps API because this component can be reused to - * set up the hadoop-conf for executors as well. - */ + * This is separated out from the HadoopConf steps API because this component can be reused to + * set up the hadoop-conf 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. - */ + /** + * Bootstraps a main container with the ConfigMaps mounted as volumes and an ENV variable + * pointing to the mounted file. + */ def bootstrapMainContainerAndVolumes( originalPodWithMainContainer: PodWithMainContainer) : PodWithMainContainer @@ -45,6 +43,7 @@ private[spark] class HadoopConfBootstrapImpl( override def bootstrapMainContainerAndVolumes( originalPodWithMainContainer: PodWithMainContainer) : PodWithMainContainer = { + import collection.JavaConverters._ val fileContents = hadoopConfigFiles.map(file => (file.getPath, file.toString)).toMap val keyPaths = hadoopConfigFiles.map(file => new KeyToPathBuilder().withKey(file.getPath).withPath(file.getAbsolutePath).build()) @@ -75,4 +74,4 @@ private[spark] class HadoopConfBootstrapImpl( mainContainerWithMountedHadoopConf ) } -} \ No newline at end of file +} 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 4b88cd2a3fb00..dbaf45849b69c 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 @@ -22,8 +22,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.hadoopsteps.HadoopStepsOrchestrator 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 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 index 6410221fc48e0..7f65147a4f2b4 100644 --- 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 @@ -16,15 +16,7 @@ */ package org.apache.spark.deploy.kubernetes.submit.submitsteps.hadoopsteps -import java.io.File - -import io.fabric8.kubernetes.api.model._ import org.apache.spark.deploy.kubernetes.{HadoopConfBootstrap, PodWithMainContainer} -import org.apache.spark.deploy.kubernetes.config._ -import org.apache.spark.deploy.kubernetes.constants._ -import org.apache.spark.deploy.kubernetes.submit.KubernetesFileUtils -import org.apache.spark.deploy.kubernetes.submit.submitsteps.{DriverKubernetesCredentialsStep, KubernetesDriverSpec} -import scala.collection.JavaConverters._ /** * Step that configures the ConfigMap + Volumes for the driver @@ -46,4 +38,4 @@ private[spark] class HadoopConfMounterStep( driverContainer = bootstrappedPodAndMainContainer.mainContainer ) } -} \ No newline at end of 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 fe3109d9f14ed..ca837919e5ff4 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 @@ -19,15 +19,15 @@ package org.apache.spark.scheduler.cluster.kubernetes import java.io.File import io.fabric8.kubernetes.client.Config -import org.apache.spark.SparkContext -import org.apache.spark.deploy.kubernetes.{HadoopConfBootstrapImpl, InitContainerResourceStagingServerSecretPluginImpl, SparkKubernetesClientFactory, SparkPodInitContainerBootstrapImpl} + +import org.apache.spark.deploy.kubernetes.{InitContainerResourceStagingServerSecretPluginImpl, SparkKubernetesClientFactory, SparkPodInitContainerBootstrapImpl} 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 = { From 2d8797f9475f0ac127657f6c91509a008ee8ae23 Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Mon, 17 Jul 2017 17:14:01 -0700 Subject: [PATCH 03/15] Added proper logic for mounting ConfigMaps --- .../kubernetes/HadoopConfBootstrap.scala | 8 +++--- .../spark/deploy/kubernetes/constants.scala | 2 ++ ...DriverConfigurationStepsOrchestrator.scala | 2 +- .../HadoopConfigBootstrapStep.scala | 26 +++++++++++++++-- .../hadoopsteps/HadoopConfMounterStep.scala | 8 +++++- .../hadoopsteps/HadoopConfigSpec.scala | 5 +++- .../hadoopsteps/HadoopStepsOrchestrator.scala | 1 + .../kubernetes/KubernetesClusterManager.scala | 28 ++++++++++++++++++- .../KubernetesClusterSchedulerBackend.scala | 15 ++++++++-- .../integrationtest/KubernetesSuite.scala | 6 ++++ 10 files changed, 87 insertions(+), 14 deletions(-) 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 index 18daff73b82b2..8a6679f80cb8b 100644 --- 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 @@ -43,17 +43,17 @@ private[spark] class HadoopConfBootstrapImpl( override def bootstrapMainContainerAndVolumes( originalPodWithMainContainer: PodWithMainContainer) : PodWithMainContainer = { - import collection.JavaConverters._ - val fileContents = hadoopConfigFiles.map(file => (file.getPath, file.toString)).toMap + import scala.collection.JavaConverters._ val keyPaths = hadoopConfigFiles.map(file => - new KeyToPathBuilder().withKey(file.getPath).withPath(file.getAbsolutePath).build()) + 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) - .addAllToItems(keyPaths.toList.asJavaCollection) + .withItems(keyPaths.asJava) .endConfigMap() .endVolume() .endSpec() 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 0dd2940faf66d..1e61e2f57a683 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 @@ -89,6 +89,8 @@ package object constants { 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_CONFIG_MAP_SPARK_CONF_NAME = + "spark.kubernetes.hadoop.executor.hadoopconfigmapname" private[spark] val INIT_CONTAINER_PROPERTIES_FILE_PATH = s"$INIT_CONTAINER_PROPERTIES_FILE_DIR/$INIT_CONTAINER_PROPERTIES_FILE_NAME" private[spark] val DEFAULT_SHUFFLE_MOUNT_NAME = "shuffle" 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 dbaf45849b69c..a4b0cb7760f0c 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 @@ -112,7 +112,7 @@ private[spark] class DriverConfigurationStepsOrchestrator( hadoopConfigurations) val hadoopConfSteps = hadoopStepsOrchestrator.getHadoopSteps() - Some(new HadoopConfigBootstrapStep(hadoopConfSteps)) + Some(new HadoopConfigBootstrapStep(hadoopConfSteps, kubernetesResourceNamePrefix)) } val pythonStep = mainAppResource match { case PythonMainAppResource(mainPyResource) => 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 index 685fdd188b497..6091dc5b36c98 100644 --- 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 @@ -16,24 +16,44 @@ */ package org.apache.spark.deploy.kubernetes.submit.submitsteps +import java.io.StringWriter +import java.util.Properties + +import io.fabric8.kubernetes.api.model.{ConfigMap, ConfigMapBuilder, HasMetadata} +import org.apache.spark.deploy.kubernetes.constants._ import org.apache.spark.deploy.kubernetes.submit.submitsteps.hadoopsteps.{HadoopConfigSpec, HadoopConfigurationStep} /** * Configures the driverSpec that bootstraps dependencies into the driver pod. */ private[spark] class HadoopConfigBootstrapStep( - hadoopConfigurationSteps: Seq[HadoopConfigurationStep]) + hadoopConfigurationSteps: Seq[HadoopConfigurationStep], kubernetesResourceNamePrefix: String) extends DriverConfigurationStep { + private val hadoopConfigMapName = s"$kubernetesResourceNamePrefix-hadoop-config" override def configureDriver(driverSpec: KubernetesDriverSpec): KubernetesDriverSpec = { + import scala.collection.JavaConverters._ var currentHadoopSpec = HadoopConfigSpec( driverPod = driverSpec.driverPod, - driverContainer = driverSpec.driverContainer) + driverContainer = driverSpec.driverContainer, + configMapProperties = Map.empty[String, String]) 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) driverSpec.copy( driverPod = currentHadoopSpec.driverPod, - driverContainer = currentHadoopSpec.driverContainer) + driverContainer = currentHadoopSpec.driverContainer, + driverSparkConf = executorSparkConf, + otherKubernetesResources = Seq(configMap) + ) } } 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 index 7f65147a4f2b4..e9035561d7ed6 100644 --- 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 @@ -16,6 +16,8 @@ */ package org.apache.spark.deploy.kubernetes.submit.submitsteps.hadoopsteps +import java.io.File + import org.apache.spark.deploy.kubernetes.{HadoopConfBootstrap, PodWithMainContainer} /** @@ -23,6 +25,7 @@ import org.apache.spark.deploy.kubernetes.{HadoopConfBootstrap, PodWithMainConta */ private[spark] class HadoopConfMounterStep( hadoopConfigMapName: String, + hadoopConfigurationFiles: Array[File], hadoopConfBootstrapConf: HadoopConfBootstrap) extends HadoopConfigurationStep { @@ -35,7 +38,10 @@ private[spark] class HadoopConfMounterStep( )) hadoopConfigSpec.copy( driverPod = bootstrappedPodAndMainContainer.pod, - driverContainer = bootstrappedPodAndMainContainer.mainContainer + driverContainer = bootstrappedPodAndMainContainer.mainContainer, + configMapProperties = + hadoopConfigurationFiles.map(file => + (file.toPath.getFileName.toString, file.toString)).toMap ) } } 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 index aaa49d9b45496..e31f6c5901a55 100644 --- 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 @@ -26,8 +26,11 @@ import io.fabric8.kubernetes.api.model.{Container, HasMetadata, Pod} * - 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) */ private[spark] case class HadoopConfigSpec( // additionalDriverSparkConf: Map[String, String], driverPod: Pod, - driverContainer: Container) + driverContainer: Container, + configMapProperties: Map[String, String]) 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 index b6ecc5dce8a82..1d3b64931aefa 100644 --- 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 @@ -40,6 +40,7 @@ private[spark] class HadoopStepsOrchestrator( hadoopConfigurationFiles) val hadoopConfMounterStep = new HadoopConfMounterStep( hadoopConfigMapName, + hadoopConfigurationFiles, hadoopConfBootstrapImpl) val maybeHadoopKerberosMountingStep = if (maybeKerberosSupport) { 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 ca837919e5ff4..ff482e9ae3b8f 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,7 +20,7 @@ import java.io.File import io.fabric8.kubernetes.client.Config -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 @@ -41,6 +41,7 @@ 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 maybeExecutorInitContainerSecretName = sparkConf.get(EXECUTOR_INIT_CONTAINER_SECRET) @@ -71,6 +72,17 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit configMap, configMapKey) } + val hadoopBootStrap = for { + hadoopConfigMap <- maybeHadoopConfigMap + } yield { + val hadoopConfigurations = + sys.env.get("HADOOP_CONF_DIR").map{ conf => getHadoopConfFiles(conf)} + .getOrElse(Array.empty[File]) + new HadoopConfBootstrapImpl( + hadoopConfigMap, + hadoopConfigurations + ) + } 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.") @@ -79,6 +91,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,6 +106,7 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit sc.taskScheduler.asInstanceOf[TaskSchedulerImpl], sc, initBootStrap, + hadoopBootStrap, executorInitContainerSecretVolumePlugin, kubernetesClient) } @@ -97,4 +114,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..9ff26822e46e2 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,7 @@ private[spark] class KubernetesClusterSchedulerBackend( scheduler: TaskSchedulerImpl, val sc: SparkContext, executorInitContainerBootstrap: Option[SparkPodInitContainerBootstrap], + executorHadoopBootStrap: Option[HadoopConfBootstrap], executorMountInitContainerSecretPlugin: Option[InitContainerResourceStagingServerSecretPlugin], kubernetesClient: KubernetesClient) extends CoarseGrainedSchedulerBackend(scheduler, sc.env.rpcEnv) { @@ -428,6 +429,7 @@ private[spark] class KubernetesClusterSchedulerBackend( * @return A tuple of the new executor name and the Pod data structure. */ private def allocateNewExecutorPod(nodeToLocalTaskCount: Map[String, Int]): (String, Pod) = { + import scala.collection.JavaConverters._ val executorId = EXECUTOR_ID_COUNTER.incrementAndGet().toString val name = s"$executorPodNamePrefix-exec-$executorId" @@ -582,9 +584,16 @@ 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 resolvedExecutorPod = new PodBuilder(executorHadoopConfPod) .editSpec() - .addToContainers(initBootstrappedExecutorContainer) + .addToContainers(executorHadoopConfContainer) .endSpec() .build() try { 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..360a83bb9e1fe 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 @@ -72,6 +72,12 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { kubernetesTestComponents.deleteNamespace() } + test("Include HADOOP_CONF for HDFS based jobs ") { + assume(testBackend.name == MINIKUBE_TEST_BACKEND) + + runSparkPiAndVerifyCompletion(CONTAINER_LOCAL_MAIN_APP_RESOURCE) + } + test("Run PySpark Job on file from SUBMITTER with --py-files") { assume(testBackend.name == MINIKUBE_TEST_BACKEND) From d0f492f46100741cb2685bbabe168b35734a52db Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Mon, 17 Jul 2017 17:32:54 -0700 Subject: [PATCH 04/15] styling --- .../submit/DriverConfigurationStepsOrchestrator.scala | 4 ++-- .../submit/submitsteps/HadoopConfigBootstrapStep.scala | 5 +++-- .../submitsteps/hadoopsteps/HadoopStepsOrchestrator.scala | 3 +-- 3 files changed, 6 insertions(+), 6 deletions(-) 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 a4b0cb7760f0c..5470c8e0ba977 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 @@ -107,12 +107,12 @@ private[spark] class DriverConfigurationStepsOrchestrator( } else { val hadoopStepsOrchestrator = new HadoopStepsOrchestrator( namespace, - kubernetesResourceNamePrefix, + hadoopConfigMapName, submissionSparkConf, hadoopConfigurations) val hadoopConfSteps = hadoopStepsOrchestrator.getHadoopSteps() - Some(new HadoopConfigBootstrapStep(hadoopConfSteps, kubernetesResourceNamePrefix)) + Some(new HadoopConfigBootstrapStep(hadoopConfSteps, hadoopConfigMapName)) } val pythonStep = mainAppResource match { case PythonMainAppResource(mainPyResource) => 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 index 6091dc5b36c98..755105f41f467 100644 --- 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 @@ -20,6 +20,7 @@ import java.io.StringWriter import java.util.Properties import io.fabric8.kubernetes.api.model.{ConfigMap, ConfigMapBuilder, HasMetadata} + import org.apache.spark.deploy.kubernetes.constants._ import org.apache.spark.deploy.kubernetes.submit.submitsteps.hadoopsteps.{HadoopConfigSpec, HadoopConfigurationStep} @@ -27,9 +28,9 @@ import org.apache.spark.deploy.kubernetes.submit.submitsteps.hadoopsteps.{Hadoop * Configures the driverSpec that bootstraps dependencies into the driver pod. */ private[spark] class HadoopConfigBootstrapStep( - hadoopConfigurationSteps: Seq[HadoopConfigurationStep], kubernetesResourceNamePrefix: String) + hadoopConfigurationSteps: Seq[HadoopConfigurationStep], + hadoopConfigMapName: String ) extends DriverConfigurationStep { - private val hadoopConfigMapName = s"$kubernetesResourceNamePrefix-hadoop-config" override def configureDriver(driverSpec: KubernetesDriverSpec): KubernetesDriverSpec = { import scala.collection.JavaConverters._ 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 index 1d3b64931aefa..55722c75b5926 100644 --- 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 @@ -28,10 +28,9 @@ import org.apache.spark.deploy.kubernetes.config._ */ private[spark] class HadoopStepsOrchestrator( namespace: String, - kubernetesResourceNamePrefix: String, + hadoopConfigMapName: String, submissionSparkConf: SparkConf, hadoopConfigurationFiles: Array[File]) { - private val hadoopConfigMapName = s"$kubernetesResourceNamePrefix-hadoop-config" private val maybeKerberosSupport = submissionSparkConf.get(KUBERNETES_KERBEROS_SUPPORT) def getHadoopSteps(): Seq[HadoopConfigurationStep] = { From a41366258a5d06325a053b1eaf545e5e87473b89 Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Mon, 17 Jul 2017 18:25:39 -0700 Subject: [PATCH 05/15] modified otherKubernetesResource logic --- .../submit/submitsteps/HadoopConfigBootstrapStep.scala | 4 +++- .../submitsteps/hadoopsteps/HadoopConfMounterStep.scala | 3 ++- 2 files changed, 5 insertions(+), 2 deletions(-) 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 index 755105f41f467..17772aa0b0514 100644 --- 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 @@ -54,7 +54,9 @@ private[spark] class HadoopConfigBootstrapStep( driverPod = currentHadoopSpec.driverPod, driverContainer = currentHadoopSpec.driverContainer, driverSparkConf = executorSparkConf, - otherKubernetesResources = Seq(configMap) + otherKubernetesResources = + driverSpec.otherKubernetesResources ++ + Seq(configMap) ) } } 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 index e9035561d7ed6..73374a18062a9 100644 --- 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 @@ -17,6 +17,7 @@ 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} @@ -41,7 +42,7 @@ private[spark] class HadoopConfMounterStep( driverContainer = bootstrappedPodAndMainContainer.mainContainer, configMapProperties = hadoopConfigurationFiles.map(file => - (file.toPath.getFileName.toString, file.toString)).toMap + (file.toPath.getFileName.toString, readFileToString(file))).toMap ) } } From 72e3f31062cd496c355e870cb1797926e9bf6f35 Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Mon, 17 Jul 2017 19:12:03 -0700 Subject: [PATCH 06/15] fixed Integration tests and modified HADOOP_CONF_DIR variable to be FILE_DIR for Volume mount --- .../kubernetes/HadoopConfBootstrap.scala | 7 +- .../integrationtest/KubernetesSuite.scala | 2 +- .../test-data/hadoop-conf-files/core-site.xml | 34 +++++ .../test-data/hadoop-conf-files/hdfs-site.xml | 132 ++++++++++++++++++ 4 files changed, 172 insertions(+), 3 deletions(-) create mode 100644 resource-managers/kubernetes/integration-tests/test-data/hadoop-conf-files/core-site.xml create mode 100644 resource-managers/kubernetes/integration-tests/test-data/hadoop-conf-files/hdfs-site.xml 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 index 8a6679f80cb8b..a8ab857ff145d 100644 --- 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 @@ -21,6 +21,8 @@ import java.io.File 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 @@ -38,12 +40,13 @@ private[spark] trait HadoopConfBootstrap { private[spark] class HadoopConfBootstrapImpl( hadoopConfConfigMapName: String, - hadoopConfigFiles: Array[File]) extends HadoopConfBootstrap { + hadoopConfigFiles: Array[File]) extends HadoopConfBootstrap with Logging{ override def bootstrapMainContainerAndVolumes( originalPodWithMainContainer: PodWithMainContainer) : PodWithMainContainer = { import scala.collection.JavaConverters._ + 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 @@ -66,7 +69,7 @@ private[spark] class HadoopConfBootstrapImpl( .endVolumeMount() .addNewEnv() .withName(HADOOP_CONF_DIR) - .withValue(s"$HADOOP_FILE_DIR/$HADOOP_FILE_VOLUME") + .withValue(HADOOP_FILE_DIR) .endEnv() .build() PodWithMainContainer( 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 360a83bb9e1fe..cc3396c97f61d 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 @@ -74,7 +74,7 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { test("Include HADOOP_CONF for HDFS based jobs ") { assume(testBackend.name == MINIKUBE_TEST_BACKEND) - + sparkConf.setJars(Seq(CONTAINER_LOCAL_HELPER_JAR_PATH)) runSparkPiAndVerifyCompletion(CONTAINER_LOCAL_MAIN_APP_RESOURCE) } diff --git a/resource-managers/kubernetes/integration-tests/test-data/hadoop-conf-files/core-site.xml b/resource-managers/kubernetes/integration-tests/test-data/hadoop-conf-files/core-site.xml new file mode 100644 index 0000000000000..f9e27564b7e9c --- /dev/null +++ b/resource-managers/kubernetes/integration-tests/test-data/hadoop-conf-files/core-site.xml @@ -0,0 +1,34 @@ + + + + + + + + + hadoop.security.authentication + kerberos + + + + hadoop.security.authorization + true + + + + fs.defaultFS + hdfs://nn.default.svc.cluster.local:9000 + + \ No newline at end of file diff --git a/resource-managers/kubernetes/integration-tests/test-data/hadoop-conf-files/hdfs-site.xml b/resource-managers/kubernetes/integration-tests/test-data/hadoop-conf-files/hdfs-site.xml new file mode 100644 index 0000000000000..1b2de25958c65 --- /dev/null +++ b/resource-managers/kubernetes/integration-tests/test-data/hadoop-conf-files/hdfs-site.xml @@ -0,0 +1,132 @@ + + + + + + + + + + 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.default.svc.cluster.local@CLUSTER.LOCAL + + + dfs.namenode.kerberos.internal.spnego.principal + HTTP/nn.default.svc.cluster.local@CLUSTER.LOCAL + + + + + + dfs.namenode.delegation.token.max-lifetime + 18000000 + + + dfs.namenode.delegation.token.renew-interval + 1800000 + + + + + + + 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.default.svc.cluster.local@CLUSTER.LOCAL + + + dfs.namenode.kerberos.internal.spnego.principal + HTTP/nn.default.svc.cluster.local@CLUSTER.LOCAL + + + + + dfs.namenode.datanode.registration.ip-hostname-check + false + + + + + dfs.datanode.data.dir.perm + 700 + + + dfs.datanode.keytab.file + /var/keytabs/hdfs.keytab + + + dfs.datanode.kerberos.principal + hdfs/dn1.default.svc.cluster.local@CLUSTER.LOCAL + + + + + dfs.webhdfs.enabled + true + + + dfs.web.authentication.kerberos.principal + HTTP/dn1.default.svc.cluster.local@CLUSTER.LOCAL + + + dfs.web.authentication.kerberos.keytab + /var/keytabs/hdfs.keytab + + + From 4971219cc5a181f69d8553bb351f145a0ae46d24 Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Mon, 17 Jul 2017 19:53:31 -0700 Subject: [PATCH 07/15] setting HADOOP_CONF_DIR env variables --- .../integrationtest/KubernetesSuite.scala | 24 ++++++++++++++++--- 1 file changed, 21 insertions(+), 3 deletions(-) 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 cc3396c97f61d..f4aa53c6f4f11 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 @@ -28,15 +28,15 @@ 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.util.{RedirectThread, Utils} private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { import KubernetesSuite._ @@ -74,13 +74,31 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { test("Include HADOOP_CONF for HDFS based jobs ") { assume(testBackend.name == MINIKUBE_TEST_BACKEND) + // Ensuring that HADOOP_CONF_DIR env variable is set + val builder = new ProcessBuilder( + Seq("/bin/bash", "-c", "export HADOOP_CONF_DIR=" + + "test-data/hadoop-conf-files && exec").asJava) + builder.redirectErrorStream(true) // Ugly but needed for stdout and stderr to synchronize + val process = builder.start() + new RedirectThread(process.getInputStream, System.out, "redirect output").start() + val exitCode = process.waitFor() + if (exitCode != 0) { + logInfo(s"exitCode: $exitCode") + } sparkConf.setJars(Seq(CONTAINER_LOCAL_HELPER_JAR_PATH)) runSparkPiAndVerifyCompletion(CONTAINER_LOCAL_MAIN_APP_RESOURCE) } test("Run PySpark Job on file from SUBMITTER with --py-files") { assume(testBackend.name == MINIKUBE_TEST_BACKEND) - + // Ensuring that HADOOP_CONF_DIR env variable is unset + val builder = new ProcessBuilder( + Seq("/bin/bash", "-c", "export HADOOP_CONF_DIR=" + + " && exec").asJava) + builder.redirectErrorStream(true) // Ugly but needed for stdout and stderr to synchronize + val process = builder.start() + new RedirectThread(process.getInputStream, System.out, "redirect output").start() + val exitCode = process.waitFor() launchStagingServer(SSLOptions(), None) sparkConf .set(DRIVER_DOCKER_IMAGE, From 4911ea8e18bd5f308ff2456d061a97ce769e96ab Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Tue, 18 Jul 2017 14:26:04 -0700 Subject: [PATCH 08/15] Included integration tests for Stage 1 --- .../spark/deploy/kubernetes/constants.scala | 10 ++-- .../deploy/kubernetes/submit/Client.scala | 8 ++- ...DriverConfigurationStepsOrchestrator.scala | 9 ++-- .../HadoopConfigBootstrapStep.scala | 4 +- .../hadoopsteps/HadoopConfMounterStep.scala | 10 +++- .../hadoopsteps/HadoopConfigSpec.scala | 2 +- .../hadoopsteps/HadoopStepsOrchestrator.scala | 6 ++- .../kubernetes/KubernetesClusterManager.scala | 6 +-- ...rConfigurationStepsOrchestratorSuite.scala | 3 ++ .../integrationtest/KubernetesSuite.scala | 49 +++++++++---------- 10 files changed, 61 insertions(+), 46 deletions(-) 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 1e61e2f57a683..6e1a66c0fedb0 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 @@ -86,15 +86,17 @@ package object constants { private[spark] val INIT_CONTAINER_PROPERTIES_FILE_VOLUME = "spark-init-properties" private[spark] val INIT_CONTAINER_PROPERTIES_FILE_DIR = "/etc/spark-init" private[spark] val INIT_CONTAINER_PROPERTIES_FILE_NAME = "spark-init.properties" + private[spark] val INIT_CONTAINER_PROPERTIES_FILE_PATH = + s"$INIT_CONTAINER_PROPERTIES_FILE_DIR/$INIT_CONTAINER_PROPERTIES_FILE_NAME" + private[spark] val DEFAULT_SHUFFLE_MOUNT_NAME = "shuffle" + private[spark] val INIT_CONTAINER_SECRET_VOLUME_NAME = "spark-init-secret" + 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" - private[spark] val INIT_CONTAINER_PROPERTIES_FILE_PATH = - s"$INIT_CONTAINER_PROPERTIES_FILE_DIR/$INIT_CONTAINER_PROPERTIES_FILE_NAME" - private[spark] val DEFAULT_SHUFFLE_MOUNT_NAME = "shuffle" - private[spark] val INIT_CONTAINER_SECRET_VOLUME_NAME = "spark-init-secret" // Miscellaneous private[spark] val ANNOTATION_EXECUTOR_NODE_AFFINITY = "scheduler.alpha.kubernetes.io/affinity" 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..c4fd6e413b0d4 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,7 @@ private[spark] object Client { def main(args: Array[String]): Unit = { val parsedArguments = ClientArguments.fromCommandLineArgs(args) val sparkConf = new SparkConf() - run(sparkConf, parsedArguments) + 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 5470c8e0ba977..41b7885b633bf 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 @@ -40,6 +40,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 @@ -98,9 +99,8 @@ private[spark] class DriverConfigurationStepsOrchestrator( submissionSparkConf) val kubernetesCredentialsStep = new DriverKubernetesCredentialsStep( submissionSparkConf, kubernetesResourceNamePrefix) - val hadoopConfigurations = - sys.env.get("HADOOP_CONF_DIR").map{ conf => getHadoopConfFiles(conf)} - .getOrElse(Array.empty[File]) + val hadoopConfigurations = hadoopConfDir.map(conf => getHadoopConfFiles(conf)) + .getOrElse(Array.empty[File]) val hadoopConfigSteps = if (hadoopConfigurations.isEmpty) { Option.empty[DriverConfigurationStep] @@ -109,7 +109,8 @@ private[spark] class DriverConfigurationStepsOrchestrator( namespace, hadoopConfigMapName, submissionSparkConf, - hadoopConfigurations) + hadoopConfigurations, + hadoopConfDir) val hadoopConfSteps = hadoopStepsOrchestrator.getHadoopSteps() Some(new HadoopConfigBootstrapStep(hadoopConfSteps, hadoopConfigMapName)) 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 index 17772aa0b0514..dd84c50c86253 100644 --- 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 @@ -37,7 +37,8 @@ private[spark] class HadoopConfigBootstrapStep( var currentHadoopSpec = HadoopConfigSpec( driverPod = driverSpec.driverPod, driverContainer = driverSpec.driverContainer, - configMapProperties = Map.empty[String, String]) + configMapProperties = Map.empty[String, String], + additionalDriverSparkConf = Map.empty[String, String]) for (nextStep <- hadoopConfigurationSteps) { currentHadoopSpec = nextStep.configureContainers(currentHadoopSpec) } @@ -50,6 +51,7 @@ private[spark] class HadoopConfigBootstrapStep( .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, 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 index 73374a18062a9..02e8dd7803ab3 100644 --- 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 @@ -17,9 +17,11 @@ 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._ /** * Step that configures the ConfigMap + Volumes for the driver @@ -27,7 +29,8 @@ import org.apache.spark.deploy.kubernetes.{HadoopConfBootstrap, PodWithMainConta private[spark] class HadoopConfMounterStep( hadoopConfigMapName: String, hadoopConfigurationFiles: Array[File], - hadoopConfBootstrapConf: HadoopConfBootstrap) + hadoopConfBootstrapConf: HadoopConfBootstrap, + hadoopConfDir: Option[String]) extends HadoopConfigurationStep { override def configureContainers(hadoopConfigSpec: HadoopConfigSpec): HadoopConfigSpec = { @@ -42,7 +45,10 @@ private[spark] class HadoopConfMounterStep( driverContainer = bootstrappedPodAndMainContainer.mainContainer, configMapProperties = hadoopConfigurationFiles.map(file => - (file.toPath.getFileName.toString, readFileToString(file))).toMap + (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 index e31f6c5901a55..872593f849ff1 100644 --- 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 @@ -30,7 +30,7 @@ import io.fabric8.kubernetes.api.model.{Container, HasMetadata, Pod} * pairs of (path, data) */ private[spark] case class HadoopConfigSpec( -// additionalDriverSparkConf: Map[String, String], + additionalDriverSparkConf: Map[String, String], driverPod: Pod, driverContainer: Container, configMapProperties: Map[String, String]) 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 index 55722c75b5926..8acbab922a06e 100644 --- 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 @@ -30,7 +30,8 @@ private[spark] class HadoopStepsOrchestrator( namespace: String, hadoopConfigMapName: String, submissionSparkConf: SparkConf, - hadoopConfigurationFiles: Array[File]) { + hadoopConfigurationFiles: Array[File], + hadoopConfDir: Option[String]) { private val maybeKerberosSupport = submissionSparkConf.get(KUBERNETES_KERBEROS_SUPPORT) def getHadoopSteps(): Seq[HadoopConfigurationStep] = { @@ -40,7 +41,8 @@ private[spark] class HadoopStepsOrchestrator( val hadoopConfMounterStep = new HadoopConfMounterStep( hadoopConfigMapName, hadoopConfigurationFiles, - hadoopConfBootstrapImpl) + hadoopConfBootstrapImpl, + hadoopConfDir) val maybeHadoopKerberosMountingStep = if (maybeKerberosSupport) { // TODO: Implement mounting secrets 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 ff482e9ae3b8f..6fb76affe5c86 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 @@ -42,6 +42,7 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit 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 maybeExecutorInitContainerSecretName = sparkConf.get(EXECUTOR_INIT_CONTAINER_SECRET) @@ -75,9 +76,8 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit val hadoopBootStrap = for { hadoopConfigMap <- maybeHadoopConfigMap } yield { - val hadoopConfigurations = - sys.env.get("HADOOP_CONF_DIR").map{ conf => getHadoopConfFiles(conf)} - .getOrElse(Array.empty[File]) + val hadoopConfigurations = maybeHadoopConfDir.map( + conf_dir => getHadoopConfFiles(conf_dir)).getOrElse(Array.empty[File]) new HadoopConfBootstrapImpl( hadoopConfigMap, hadoopConfigurations 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/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 f4aa53c6f4f11..810ed5e30d6c2 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 @@ -74,31 +74,19 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { test("Include HADOOP_CONF for HDFS based jobs ") { assume(testBackend.name == MINIKUBE_TEST_BACKEND) - // Ensuring that HADOOP_CONF_DIR env variable is set - val builder = new ProcessBuilder( - Seq("/bin/bash", "-c", "export HADOOP_CONF_DIR=" + - "test-data/hadoop-conf-files && exec").asJava) - builder.redirectErrorStream(true) // Ugly but needed for stdout and stderr to synchronize - val process = builder.start() - new RedirectThread(process.getInputStream, System.out, "redirect output").start() - val exitCode = process.waitFor() - if (exitCode != 0) { - logInfo(s"exitCode: $exitCode") - } + // 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) + runSparkApplicationAndVerifyCompletion( + JavaMainAppResource(CONTAINER_LOCAL_MAIN_APP_RESOURCE), + SPARK_PI_MAIN_CLASS, + Seq("HADOOP_CONF_DIR defined. Mounting HDFS specific .xml files", "Pi is roughly 3"), + Array("5"), + Seq.empty[String], + Some("test-data/hadoop-conf-files")) } test("Run PySpark Job on file from SUBMITTER with --py-files") { assume(testBackend.name == MINIKUBE_TEST_BACKEND) - // Ensuring that HADOOP_CONF_DIR env variable is unset - val builder = new ProcessBuilder( - Seq("/bin/bash", "-c", "export HADOOP_CONF_DIR=" + - " && exec").asJava) - builder.redirectErrorStream(true) // Ugly but needed for stdout and stderr to synchronize - val process = builder.start() - new RedirectThread(process.getInputStream, System.out, "redirect output").start() - val exitCode = process.waitFor() launchStagingServer(SSLOptions(), None) sparkConf .set(DRIVER_DOCKER_IMAGE, @@ -183,7 +171,8 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { GROUP_BY_MAIN_CLASS, Seq("The Result is"), Array.empty[String], - Seq.empty[String]) + Seq.empty[String], + None) } test("Use remote resources without the resource staging server.") { @@ -247,7 +236,8 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { 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]) + Seq.empty[String], + None) } test("Use a very long application name.") { @@ -277,9 +267,12 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { 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( @@ -289,7 +282,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( @@ -297,13 +291,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) From c41c38b0fde3f6a994b46209f08c2a14061e9c1e Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Tue, 18 Jul 2017 19:09:16 -0700 Subject: [PATCH 09/15] Initial Kerberos support --- .../spark/deploy/kubernetes/config.scala | 14 +++++ .../HadoopKerberosMounterStep.scala | 60 +++++++++++++++++++ 2 files changed, 74 insertions(+) create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopKerberosMounterStep.scala 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 7ebcf7253ebb3..121f3f347af38 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 @@ -505,6 +505,20 @@ package object config extends Logging { .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] 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/submit/submitsteps/hadoopsteps/HadoopKerberosMounterStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopKerberosMounterStep.scala new file mode 100644 index 0000000000000..66c652232501b --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopKerberosMounterStep.scala @@ -0,0 +1,60 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.kubernetes.submit.submitsteps.hadoopsteps + +import java.io.File + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.security.{Credentials, UserGroupInformation} + +import org.apache.spark.SparkConf +import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.deploy.kubernetes.config._ +import org.apache.spark.internal.Logging + +// import org.apache.spark.deploy.security.HadoopDelegationTokenManager + +private[spark] case class DelegationToken( + principle: String, + bytes: Array[Byte], + renewal: Long) + + /** + * This class is responsible for Hadoop DT renewal + * TODO: THIS IS BLOCKED BY SPARK 2.2 REBASE + */ +private[spark] class HadoopKerberosMounterStep( + submissionSparkConf: SparkConf) + extends HadoopConfigurationStep with Logging { + + private val maybePrincipal = submissionSparkConf.get(KUBERNETES_KERBEROS_PRINCIPAL) + private val maybeKeytab = submissionSparkConf.get(KUBERNETES_KERBEROS_KEYTAB).map( + k => new File(k)) + + override def configureContainers(hadoopConfigSpec: HadoopConfigSpec): HadoopConfigSpec = { + val hadoopConf = SparkHadoopUtil.get.newConfiguration(submissionSparkConf) + if (!UserGroupInformation.isSecurityEnabled) logError("Hadoop not configuration with Kerberos") + for { + principal <- maybePrincipal + keytab <- maybeKeytab + } yield { + submissionSparkConf.set("spark.yarn.principal", principal) + submissionSparkConf.set("spark.yarn.keytab", keytab.toURI.toString) + } + hadoopConfigSpec + } + } From bff5a4214b2bc68d922569bc2cbec8b1959d9231 Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Fri, 21 Jul 2017 15:44:25 -0700 Subject: [PATCH 10/15] initial Stage 2 architecture using deprecated 2.1 methods --- .../org/apache/spark/deploy/SparkSubmit.scala | 2 +- .../kubernetes/HadoopConfBootstrap.scala | 18 +-- .../kubernetes/KerberosConfBootstrap.scala | 42 ++++++ .../spark/deploy/kubernetes/config.scala | 11 +- .../spark/deploy/kubernetes/constants.scala | 8 ++ .../deploy/kubernetes/submit/Client.scala | 2 +- ...DriverConfigurationStepsOrchestrator.scala | 16 +-- .../HadoopConfigBootstrapStep.scala | 12 +- .../hadoopsteps/HDFSDelegationToken.scala | 19 +++ .../hadoopsteps/HadoopConfMounterStep.scala | 2 +- .../hadoopsteps/HadoopConfigSpec.scala | 6 +- .../HadoopKerberosKeytabResolverStep.scala | 130 ++++++++++++++++++ .../HadoopKerberosMounterStep.scala | 60 -------- .../HadoopKerberosSecretResolverStep.scala | 39 ++++++ .../hadoopsteps/HadoopStepsOrchestrator.scala | 29 +++- .../kubernetes/KubernetesClusterManager.scala | 7 + .../KubernetesClusterSchedulerBackend.scala | 13 +- 17 files changed, 312 insertions(+), 104 deletions(-) create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/KerberosConfBootstrap.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HDFSDelegationToken.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopKerberosKeytabResolverStep.scala delete mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopKerberosMounterStep.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopKerberosSecretResolverStep.scala diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index 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/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 index a8ab857ff145d..733fbeffe45b7 100644 --- 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 @@ -18,6 +18,8 @@ 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._ @@ -40,16 +42,17 @@ private[spark] trait HadoopConfBootstrap { private[spark] class HadoopConfBootstrapImpl( hadoopConfConfigMapName: String, - hadoopConfigFiles: Array[File]) extends HadoopConfBootstrap with Logging{ + hadoopConfigFiles: Seq[File]) extends HadoopConfBootstrap with Logging{ override def bootstrapMainContainerAndVolumes( originalPodWithMainContainer: PodWithMainContainer) : PodWithMainContainer = { - import scala.collection.JavaConverters._ 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 + new KeyToPathBuilder() + .withKey(file.toPath.getFileName.toString) + .withPath(file.toPath.getFileName.toString) + .build()).toList val hadoopSupportedPod = new PodBuilder(originalPodWithMainContainer.pod) .editSpec() .addNewVolume() @@ -72,9 +75,8 @@ private[spark] class HadoopConfBootstrapImpl( .withValue(HADOOP_FILE_DIR) .endEnv() .build() - PodWithMainContainer( - hadoopSupportedPod, - mainContainerWithMountedHadoopConf - ) + 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..e0c1b28f05046 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/KerberosConfBootstrap.scala @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.kubernetes + +import io.fabric8.kubernetes.api.model.ContainerBuilder + +import org.apache.spark.deploy.kubernetes.constants._ + +private[spark] trait KerberosConfBootstrap { + 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/config.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/config.scala index 121f3f347af38..9e397befdba4e 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 @@ -500,8 +500,7 @@ 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") + .doc("Specify whether your job is a job that will require a Delegation Token to access HDFS") .booleanConf .createWithDefault(false) @@ -519,6 +518,14 @@ package object config extends Logging { .stringConf .createOptional + private[spark] val KUBERNETES_KERBEROS_DT_SECRET = + ConfigBuilder("spark.kubernetes.kerberos.tokensecret") + .doc("Specify the label 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] 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 6e1a66c0fedb0..389297a5cec55 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 @@ -97,6 +97,14 @@ package object constants { 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" + 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" 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 c4fd6e413b0d4..7a30bbe50790f 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,7 @@ private[spark] class Client( } private[spark] object Client { - def run(sparkConf: SparkConf, + def run(sparkConf: SparkConf, clientArguments: ClientArguments, hadoopConfDir: Option[String]): Unit = { val namespace = sparkConf.get(KUBERNETES_NAMESPACE) 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 41b7885b633bf..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 @@ -16,8 +16,6 @@ */ package org.apache.spark.deploy.kubernetes.submit -import java.io.File - import org.apache.spark.SparkConf import org.apache.spark.deploy.kubernetes.ConfigurationUtils import org.apache.spark.deploy.kubernetes.config._ @@ -99,17 +97,14 @@ private[spark] class DriverConfigurationStepsOrchestrator( submissionSparkConf) val kubernetesCredentialsStep = new DriverKubernetesCredentialsStep( submissionSparkConf, kubernetesResourceNamePrefix) - val hadoopConfigurations = hadoopConfDir.map(conf => getHadoopConfFiles(conf)) - .getOrElse(Array.empty[File]) val hadoopConfigSteps = - if (hadoopConfigurations.isEmpty) { + if (hadoopConfDir.isEmpty) { Option.empty[DriverConfigurationStep] } else { val hadoopStepsOrchestrator = new HadoopStepsOrchestrator( namespace, hadoopConfigMapName, submissionSparkConf, - hadoopConfigurations, hadoopConfDir) val hadoopConfSteps = hadoopStepsOrchestrator.getHadoopSteps() @@ -157,13 +152,4 @@ private[spark] class DriverConfigurationStepsOrchestrator( hadoopConfigSteps.toSeq ++ pythonStep.toSeq } - 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/deploy/kubernetes/submit/submitsteps/HadoopConfigBootstrapStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/HadoopConfigBootstrapStep.scala index dd84c50c86253..f6dedf827a2d8 100644 --- 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 @@ -16,14 +16,14 @@ */ package org.apache.spark.deploy.kubernetes.submit.submitsteps -import java.io.StringWriter -import java.util.Properties +import scala.collection.JavaConverters._ -import io.fabric8.kubernetes.api.model.{ConfigMap, ConfigMapBuilder, HasMetadata} +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} + /** * Configures the driverSpec that bootstraps dependencies into the driver pod. */ @@ -33,12 +33,12 @@ private[spark] class HadoopConfigBootstrapStep( extends DriverConfigurationStep { override def configureDriver(driverSpec: KubernetesDriverSpec): KubernetesDriverSpec = { - import scala.collection.JavaConverters._ var currentHadoopSpec = HadoopConfigSpec( driverPod = driverSpec.driverPod, driverContainer = driverSpec.driverContainer, configMapProperties = Map.empty[String, String], - additionalDriverSparkConf = Map.empty[String, String]) + additionalDriverSparkConf = Map.empty[String, String], + dtSecret = None) for (nextStep <- hadoopConfigurationSteps) { currentHadoopSpec = nextStep.configureContainers(currentHadoopSpec) } @@ -58,7 +58,7 @@ private[spark] class HadoopConfigBootstrapStep( driverSparkConf = executorSparkConf, otherKubernetesResources = driverSpec.otherKubernetesResources ++ - Seq(configMap) + 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..308bea183d141 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HDFSDelegationToken.scala @@ -0,0 +1,19 @@ +/* + * 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 + +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 index 02e8dd7803ab3..a1f399033afdb 100644 --- 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 @@ -28,7 +28,7 @@ import org.apache.spark.deploy.kubernetes.constants._ */ private[spark] class HadoopConfMounterStep( hadoopConfigMapName: String, - hadoopConfigurationFiles: Array[File], + hadoopConfigurationFiles: Seq[File], hadoopConfBootstrapConf: HadoopConfBootstrap, hadoopConfDir: Option[String]) extends HadoopConfigurationStep { 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 index 872593f849ff1..7f399311737bc 100644 --- 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 @@ -16,7 +16,7 @@ */ package org.apache.spark.deploy.kubernetes.submit.submitsteps.hadoopsteps -import io.fabric8.kubernetes.api.model.{Container, HasMetadata, Pod} +import io.fabric8.kubernetes.api.model.{Container, Pod, Secret} /** * Represents a given configuration of the hadoop configuration logic, informing the @@ -28,9 +28,11 @@ import io.fabric8.kubernetes.api.model.{Container, HasMetadata, Pod} * 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 re-built */ private[spark] case class HadoopConfigSpec( additionalDriverSparkConf: Map[String, String], driverPod: Pod, driverContainer: Container, - configMapProperties: Map[String, String]) + 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/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..2872791f9c43a --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopKerberosKeytabResolverStep.scala @@ -0,0 +1,130 @@ +/* + * 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 io.fabric8.kubernetes.api.model.SecretBuilder +import org.apache.commons.codec.binary.Base64 +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.security.{Credentials, UserGroupInformation} + +import org.apache.spark.SparkConf +import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.deploy.kubernetes.{KerberosConfBootstrapImpl, PodWithMainContainer} +import org.apache.spark.deploy.kubernetes.constants._ +import org.apache.spark.internal.Logging + + /** + * Step that configures the ConfigMap + Volumes for the driver + */ +private[spark] class HadoopKerberosKeytabResolverStep( + submissionSparkConf: SparkConf, + maybePrincipal: Option[String], + maybeKeytab: Option[File]) extends HadoopConfigurationStep with Logging{ + + override def configureContainers(hadoopConfigSpec: HadoopConfigSpec): HadoopConfigSpec = { + // FIXME: Pass down hadoopConf so you can call sc.hadoopConfiguration + val hadoopConf = SparkHadoopUtil.get.newConfiguration(submissionSparkConf) + 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) + val credentials: Credentials = jobUserUGI.getCredentials + val credentialsManager = newHadoopTokenManager(submissionSparkConf, hadoopConf) + var renewalTime = Long.MaxValue + jobUserUGI.doAs(new PrivilegedExceptionAction[Void] { + override def run(): Void = { + renewalTime = Math.min( + obtainCredentials(credentialsManager, hadoopConf, credentials), + renewalTime) + null + } + }) + if (credentials.getAllTokens.isEmpty) logError("Did not obtain any Delegation Tokens") + val data = serialize(credentials) + val delegationToken = HDFSDelegationToken(data, renewalTime) + val initialTokenLabelName = s"$KERBEROS_SECRET_LABEL_PREFIX-1-$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 SparkHadoopUtil with Rebase to 2.2 + @deprecated("Moved to core in 2.2", "2.2") + private def obtainCredentials(instance: Any, args: AnyRef*): Long = { + val method = Class + .forName("org.apache.spark.deploy.yarn.security.ConfigurableCredentialManager") + .getMethod("obtainCredentials", classOf[Configuration], classOf[Configuration]) + method.setAccessible(true) + method.invoke(instance, args: _*).asInstanceOf[Long] + } + @deprecated("Moved to core in 2.2", "2.2") + // This method will instead be using HadoopDelegationTokenManager from Spark 2.2 + private def newHadoopTokenManager(args: AnyRef*): Any = { + val constructor = Class + .forName("org.apache.spark.deploy.yarn.security.ConfigurableCredentialManager") + .getConstructor(classOf[SparkConf], classOf[Configuration]) + constructor.setAccessible(true) + constructor.newInstance(args: _*) + } + @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/HadoopKerberosMounterStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopKerberosMounterStep.scala deleted file mode 100644 index 66c652232501b..0000000000000 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopKerberosMounterStep.scala +++ /dev/null @@ -1,60 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.spark.deploy.kubernetes.submit.submitsteps.hadoopsteps - -import java.io.File - -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.security.{Credentials, UserGroupInformation} - -import org.apache.spark.SparkConf -import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.deploy.kubernetes.config._ -import org.apache.spark.internal.Logging - -// import org.apache.spark.deploy.security.HadoopDelegationTokenManager - -private[spark] case class DelegationToken( - principle: String, - bytes: Array[Byte], - renewal: Long) - - /** - * This class is responsible for Hadoop DT renewal - * TODO: THIS IS BLOCKED BY SPARK 2.2 REBASE - */ -private[spark] class HadoopKerberosMounterStep( - submissionSparkConf: SparkConf) - extends HadoopConfigurationStep with Logging { - - private val maybePrincipal = submissionSparkConf.get(KUBERNETES_KERBEROS_PRINCIPAL) - private val maybeKeytab = submissionSparkConf.get(KUBERNETES_KERBEROS_KEYTAB).map( - k => new File(k)) - - override def configureContainers(hadoopConfigSpec: HadoopConfigSpec): HadoopConfigSpec = { - val hadoopConf = SparkHadoopUtil.get.newConfiguration(submissionSparkConf) - if (!UserGroupInformation.isSecurityEnabled) logError("Hadoop not configuration with Kerberos") - for { - principal <- maybePrincipal - keytab <- maybeKeytab - } yield { - submissionSparkConf.set("spark.yarn.principal", principal) - submissionSparkConf.set("spark.yarn.keytab", keytab.toURI.toString) - } - hadoopConfigSpec - } - } 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..9406204988403 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopKerberosSecretResolverStep.scala @@ -0,0 +1,39 @@ +/* + * 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} + + /** + * Step that configures the ConfigMap + Volumes for the driver + */ +private[spark] class HadoopKerberosSecretResolverStep( + submissionSparkConf: SparkConf, + 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 index 8acbab922a06e..032ac27939813 100644 --- 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 @@ -30,9 +30,14 @@ private[spark] class HadoopStepsOrchestrator( namespace: String, hadoopConfigMapName: String, submissionSparkConf: SparkConf, - hadoopConfigurationFiles: Array[File], 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) + private val hadoopConfigurationFiles = hadoopConfDir.map(conf => getHadoopConfFiles(conf)) + .getOrElse(Seq.empty[File]) def getHadoopSteps(): Seq[HadoopConfigurationStep] = { val hadoopConfBootstrapImpl = new HadoopConfBootstrapImpl( @@ -43,13 +48,27 @@ private[spark] class HadoopStepsOrchestrator( hadoopConfigurationFiles, hadoopConfBootstrapImpl, hadoopConfDir) - val maybeHadoopKerberosMountingStep = + val maybeKerberosStep = if (maybeKerberosSupport) { - // TODO: Implement mounting secrets - Option.empty[HadoopConfigurationStep] + maybeExistingSecret.map(secretLabel => Some(new HadoopKerberosSecretResolverStep( + submissionSparkConf, + secretLabel))).getOrElse(Some( + new HadoopKerberosKeytabResolverStep( + submissionSparkConf, + maybePrincipal, + maybeKeytab))) } else { Option.empty[HadoopConfigurationStep] } - Seq(hadoopConfMounterStep) ++ maybeHadoopKerberosMountingStep.toSeq + 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 6fb76affe5c86..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 @@ -43,6 +43,7 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit 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) @@ -83,6 +84,11 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit 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.") @@ -107,6 +113,7 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit sc, initBootStrap, hadoopBootStrap, + kerberosBootstrap, executorInitContainerSecretVolumePlugin, kubernetesClient) } 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 9ff26822e46e2..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 @@ -51,6 +51,7 @@ private[spark] class KubernetesClusterSchedulerBackend( val sc: SparkContext, executorInitContainerBootstrap: Option[SparkPodInitContainerBootstrap], executorHadoopBootStrap: Option[HadoopConfBootstrap], + executorKerberosBootStrap: Option[KerberosConfBootstrap], executorMountInitContainerSecretPlugin: Option[InitContainerResourceStagingServerSecretPlugin], kubernetesClient: KubernetesClient) extends CoarseGrainedSchedulerBackend(scheduler, sc.env.rpcEnv) { @@ -429,7 +430,6 @@ private[spark] class KubernetesClusterSchedulerBackend( * @return A tuple of the new executor name and the Pod data structure. */ private def allocateNewExecutorPod(nodeToLocalTaskCount: Map[String, Int]): (String, Pod) = { - import scala.collection.JavaConverters._ val executorId = EXECUTOR_ID_COUNTER.incrementAndGet().toString val name = s"$executorPodNamePrefix-exec-$executorId" @@ -591,9 +591,16 @@ private[spark] class KubernetesClusterSchedulerBackend( ) (podWithMainContainer.pod, podWithMainContainer.mainContainer) }.getOrElse(executorPodWithNodeAffinity, initBootstrappedExecutorContainer) - val resolvedExecutorPod = new PodBuilder(executorHadoopConfPod) + 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(executorHadoopConfContainer) + .addToContainers(executorKerberosContainer) .endSpec() .build() try { From d6d49dc2331a01e1222a75d5b59396a85a35dbb4 Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Wed, 26 Jul 2017 10:40:56 -0700 Subject: [PATCH 11/15] Added current, BROKEN, integration test environment for review --- .../HadoopKerberosKeytabResolverStep.scala | 2 + .../data-populator-deployment.yml | 28 ++ .../kerberos-yml/data-populator-service.yml | 16 + .../kerberos-yml/dn1-deployment.yml | 28 ++ .../kerberos-yml/dn1-service.yml | 16 + .../kerberos-yml/kerberos-deployment.yml | 28 ++ .../kerberos-yml/kerberos-service.yml | 16 + .../kerberos-yml/namenode-hadoop-pv.yml | 13 + .../kerberos-yml/namenode-hadoop.yml | 10 + .../kerberos-yml/nn-deployment.yml | 33 ++ .../kerberos-yml/nn-service.yml | 16 + .../kerberos-yml/server-keytab-pv.yml | 13 + .../kerberos-yml/server-keytab.yml | 10 + .../kubernetes/integration-tests/pom.xml | 1 + .../src/test/resources/core-site.xml | 34 ++ .../src/test/resources/hdfs-site.xml | 132 ++++++ .../src/test/resources/krb5.conf | 25 ++ .../src/test/resources/yarn-site.xml | 26 ++ .../KerberizedHadoopClusterLauncher.scala | 112 +++++ .../integrationtest/KubernetesSuite.scala | 382 ++++++++++-------- .../KubernetesTestComponents.scala | 23 +- .../minikube/MinikubeTestBackend.scala | 4 +- .../docker/SparkDockerImageBuilder.scala | 7 +- .../test-data/hadoop-conf-files/core-site.xml | 2 +- .../test-data/hadoop-conf-files/hdfs-site.xml | 12 +- .../test-data/hadoop-conf-files/yarn-site.xml | 26 ++ 26 files changed, 816 insertions(+), 199 deletions(-) create mode 100644 resource-managers/kubernetes/integration-tests/kerberos-yml/data-populator-deployment.yml create mode 100644 resource-managers/kubernetes/integration-tests/kerberos-yml/data-populator-service.yml create mode 100644 resource-managers/kubernetes/integration-tests/kerberos-yml/dn1-deployment.yml create mode 100644 resource-managers/kubernetes/integration-tests/kerberos-yml/dn1-service.yml create mode 100644 resource-managers/kubernetes/integration-tests/kerberos-yml/kerberos-deployment.yml create mode 100644 resource-managers/kubernetes/integration-tests/kerberos-yml/kerberos-service.yml create mode 100644 resource-managers/kubernetes/integration-tests/kerberos-yml/namenode-hadoop-pv.yml create mode 100644 resource-managers/kubernetes/integration-tests/kerberos-yml/namenode-hadoop.yml create mode 100644 resource-managers/kubernetes/integration-tests/kerberos-yml/nn-deployment.yml create mode 100644 resource-managers/kubernetes/integration-tests/kerberos-yml/nn-service.yml create mode 100644 resource-managers/kubernetes/integration-tests/kerberos-yml/server-keytab-pv.yml create mode 100644 resource-managers/kubernetes/integration-tests/kerberos-yml/server-keytab.yml create mode 100644 resource-managers/kubernetes/integration-tests/src/test/resources/core-site.xml create mode 100644 resource-managers/kubernetes/integration-tests/src/test/resources/hdfs-site.xml create mode 100644 resource-managers/kubernetes/integration-tests/src/test/resources/krb5.conf create mode 100644 resource-managers/kubernetes/integration-tests/src/test/resources/yarn-site.xml create mode 100644 resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KerberizedHadoopClusterLauncher.scala create mode 100644 resource-managers/kubernetes/integration-tests/test-data/hadoop-conf-files/yarn-site.xml 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 index 2872791f9c43a..401fc60f73cad 100644 --- 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 @@ -41,6 +41,7 @@ private[spark] class HadoopKerberosKeytabResolverStep( override def configureContainers(hadoopConfigSpec: HadoopConfigSpec): HadoopConfigSpec = { // FIXME: Pass down hadoopConf so you can call sc.hadoopConfiguration val hadoopConf = SparkHadoopUtil.get.newConfiguration(submissionSparkConf) + logInfo(s"Hadoop Configuration: ${hadoopConf.toString}") if (!UserGroupInformation.isSecurityEnabled) logError("Hadoop not configuration with Kerberos") val maybeJobUserUGI = for { @@ -58,6 +59,7 @@ private[spark] class HadoopKerberosKeytabResolverStep( } // In the case that keytab is not specified we will read from Local Ticket Cache val jobUserUGI = maybeJobUserUGI.getOrElse(UserGroupInformation.getCurrentUser) + logInfo("Primary group name: jobUserUGI.getPrimaryGroupName") val credentials: Credentials = jobUserUGI.getCredentials val credentialsManager = newHadoopTokenManager(submissionSparkConf, hadoopConf) var renewalTime = Long.MaxValue 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..5d6c752f78ea9 --- /dev/null +++ b/resource-managers/kubernetes/integration-tests/kerberos-yml/data-populator-deployment.yml @@ -0,0 +1,28 @@ +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 + spec: + containers: + - command: + - /populate-data.sh + name: data-populator + image: ifilonenko/hadoop-base:latest + imagePullPolicy: IfNotPresent + 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..8d8e02671c2c2 --- /dev/null +++ b/resource-managers/kubernetes/integration-tests/kerberos-yml/data-populator-service.yml @@ -0,0 +1,16 @@ +apiVersion: v1 +kind: Service +metadata: + annotations: + service.alpha.kubernetes.io/tolerate-unready-endpoints: "true" + labels: + kerberosService: data-populator + 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..eb68d5c011881 --- /dev/null +++ b/resource-managers/kubernetes/integration-tests/kerberos-yml/dn1-deployment.yml @@ -0,0 +1,28 @@ +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 + spec: + containers: + - command: + - /start-datanode.sh + name: dn1 + image: ifilonenko/hadoop-base:latest + imagePullPolicy: IfNotPresent + 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..f746661721da6 --- /dev/null +++ b/resource-managers/kubernetes/integration-tests/kerberos-yml/dn1-service.yml @@ -0,0 +1,16 @@ +apiVersion: v1 +kind: Service +metadata: + annotations: + service.alpha.kubernetes.io/tolerate-unready-endpoints: "true" + labels: + kerberosService: dn1 + 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..4268e424e4b28 --- /dev/null +++ b/resource-managers/kubernetes/integration-tests/kerberos-yml/kerberos-deployment.yml @@ -0,0 +1,28 @@ +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 + spec: + containers: + - command: + - /start-kdc.sh + name: kerberos + image: ifilonenko/hadoop-base:latest + imagePullPolicy: IfNotPresent + 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..38746af6a0ad0 --- /dev/null +++ b/resource-managers/kubernetes/integration-tests/kerberos-yml/kerberos-service.yml @@ -0,0 +1,16 @@ +apiVersion: v1 +kind: Service +metadata: + annotations: + service.alpha.kubernetes.io/tolerate-unready-endpoints: "true" + labels: + kerberosService: kerberos + 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/namenode-hadoop-pv.yml b/resource-managers/kubernetes/integration-tests/kerberos-yml/namenode-hadoop-pv.yml new file mode 100644 index 0000000000000..d3d09f9d0699c --- /dev/null +++ b/resource-managers/kubernetes/integration-tests/kerberos-yml/namenode-hadoop-pv.yml @@ -0,0 +1,13 @@ +kind: PersistentVolume +apiVersion: v1 +metadata: + name: nn-hadoop + labels: + type: local +spec: + capacity: + storage: 10Gi + 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..3f7a74726ae01 --- /dev/null +++ b/resource-managers/kubernetes/integration-tests/kerberos-yml/namenode-hadoop.yml @@ -0,0 +1,10 @@ +apiVersion: v1 +kind: PersistentVolumeClaim +metadata: + name: nn-hadoop +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..ca2afa6a4c5c2 --- /dev/null +++ b/resource-managers/kubernetes/integration-tests/kerberos-yml/nn-deployment.yml @@ -0,0 +1,33 @@ +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 + spec: + containers: + - command: + - /start-namenode.sh + name: nn + ports: + - containerPort: 9000 + image: ifilonenko/hadoop-base:latest + imagePullPolicy: IfNotPresent + 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..4f1e44a22ba55 --- /dev/null +++ b/resource-managers/kubernetes/integration-tests/kerberos-yml/nn-service.yml @@ -0,0 +1,16 @@ +apiVersion: v1 +kind: Service +metadata: + annotations: + service.alpha.kubernetes.io/tolerate-unready-endpoints: "true" + labels: + kerberosService: nn + 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..bf1667ebf66bd --- /dev/null +++ b/resource-managers/kubernetes/integration-tests/kerberos-yml/server-keytab-pv.yml @@ -0,0 +1,13 @@ +kind: PersistentVolume +apiVersion: v1 +metadata: + name: server-keytab + labels: + type: local +spec: + capacity: + storage: 10Gi + 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..873f263c42fcd --- /dev/null +++ b/resource-managers/kubernetes/integration-tests/kerberos-yml/server-keytab.yml @@ -0,0 +1,10 @@ +apiVersion: v1 +kind: PersistentVolumeClaim +metadata: + name: server-keytab +spec: + accessModes: + - ReadWriteOnce + resources: + requests: + storage: 100Mi \ 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..c31f75db5783e 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.${NAMESPACE}.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..8424aaba3c08d --- /dev/null +++ b/resource-managers/kubernetes/integration-tests/src/test/resources/hdfs-site.xml @@ -0,0 +1,132 @@ + + + + + + + + + + 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.${NAMESPACE}.svc.cluster.local@CLUSTER.LOCAL + + + dfs.namenode.kerberos.internal.spnego.principal + HTTP/nn.${NAMESPACE}.svc.cluster.local@CLUSTER.LOCAL + + + + + + dfs.namenode.delegation.token.max-lifetime + 18000000 + + + dfs.namenode.delegation.token.renew-interval + 1800000 + + + + + + + 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.${NAMESPACE}.svc.cluster.local@CLUSTER.LOCAL + + + dfs.namenode.kerberos.internal.spnego.principal + HTTP/nn.${NAMESPACE}.svc.cluster.local@CLUSTER.LOCAL + + + + + dfs.namenode.datanode.registration.ip-hostname-check + false + + + + + dfs.datanode.data.dir.perm + 700 + + + dfs.datanode.keytab.file + /var/keytabs/hdfs.keytab + + + dfs.datanode.kerberos.principal + hdfs/dn1.${NAMESPACE}.svc.cluster.local@CLUSTER.LOCAL + + + + + dfs.webhdfs.enabled + true + + + dfs.web.authentication.kerberos.principal + HTTP/dn1.${NAMESPACE}.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..2bb007bba3bbd --- /dev/null +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KerberizedHadoopClusterLauncher.scala @@ -0,0 +1,112 @@ +/* + * 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 io.fabric8.kubernetes.api.model.extensions.{Deployment, DeploymentBuilder} +import io.fabric8.kubernetes.api.model.{ConfigMapBuilder, KeyToPathBuilder, Service} +import io.fabric8.kubernetes.client.KubernetesClient +import org.apache.commons.io.FileUtils.readFileToString +import org.apache.spark.deploy.kubernetes.submit.ContainerNameEqualityPredicate + + /** + * Stuff + */ +private[spark] class KerberizedHadoopClusterLauncher( + kubernetesClient: KubernetesClient, + namespace: String) { + private def yamlLocation(loc: String) = s"kerberos-yml/$loc.yml" + private def loadFromYaml(resource: String) = + kubernetesClient.load(new FileInputStream(new File(yamlLocation(resource)))) +// private val regex = "REPLACE_ME".r +// private val krb5ConfFile = +// regex.replaceAllIn( +// readFileToString(new File("src/test/resources/krb5.conf")), +// namespace) + private val KRB_VOLUME = "krb5-conf" + private val KRB_FILE_DIR = "/etc" + private val KRB_CONFIG_MAP_NAME = "krb-config-map" + private val KRB_CONF_FILE = "krb5.conf" + private val KRB_KEY_PATH = + new KeyToPathBuilder() + .withKey(KRB_CONF_FILE) + .withPath(KRB_CONF_FILE) + .build() + + def launchKerberizedCluster(): Unit = { + val persistantVolumeList = Seq( + "namenode-hadoop", + "namenode-hadoop-pv", + "server-keytab", + "server-keytab-pv") + val deploymentServiceList = Seq( + "kerberos-deployment", + "kerberos-service", + "nn-deployment", + "nn-service", + "dn1-deployment", + "dn1-service", + "data-populator-deployment", + "data-populator-service") + persistantVolumeList.foreach{resource => + loadFromYaml(resource).createOrReplace() + Thread.sleep(10000)} +// val configMap = new ConfigMapBuilder() +// .withNewMetadata() +// .withName(KRB_CONFIG_MAP_NAME) +// .endMetadata() +// .addToData(KRB_CONF_FILE, krb5ConfFile) +// .build() +// kubernetesClient.configMaps().inNamespace(namespace).createOrReplace(configMap) +// Thread.sleep(2000) + deploymentServiceList.foreach{ resource => loadFromYaml(resource).get().get(0) match { + case deployment: Deployment => + val deploymentWithEnv = new DeploymentBuilder(deployment) + .editSpec() + .editTemplate() + .editSpec() + .addNewVolume() + .withName(KRB_VOLUME) + .withNewConfigMap() + .withName(KRB_CONFIG_MAP_NAME) + .withItems(KRB_KEY_PATH) + .endConfigMap() + .endVolume() + .editMatchingContainer(new ContainerNameEqualityPredicate( + deployment.getMetadata.getName)) + .addNewEnv() + .withName("NAMESPACE") + .withValue(namespace) + .endEnv() + .addNewVolumeMount() + .withName(KRB_VOLUME) + .withMountPath(KRB_FILE_DIR) + .endVolumeMount() + .endContainer() + .endSpec() + .endTemplate() + .endSpec() + .build() + kubernetesClient.extensions().deployments().inNamespace(namespace).create(deploymentWithEnv) + Thread.sleep(10000) + case service: Service => + loadFromYaml(resource).createOrReplace() + Thread.sleep(10000)} + } + } +} \ 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 810ed5e30d6c2..48ed5868e2c5c 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,21 @@ */ 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.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.{RedirectThread, Utils} +import org.apache.spark.deploy.kubernetes.submit._ +import org.apache.spark.{SSLOptions, SparkConf, SparkFunSuite} +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 +40,7 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { private var sparkConf: SparkConf = _ private var resourceStagingServerLauncher: ResourceStagingServerLauncher = _ private var staticAssetServerLauncher: StaticAssetServerLauncher = _ + private var kerberizedHadoopClusterLauncher: KerberizedHadoopClusterLauncher = _ override def beforeAll(): Unit = { testBackend.initialize() @@ -54,6 +49,9 @@ 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) } override def afterAll(): Unit = { @@ -69,13 +67,32 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { } after { - kubernetesTestComponents.deleteNamespace() + kubernetesTestComponents.deletePersistentVolumes() + // kubernetesTestComponents.deleteNamespace() } - test("Include HADOOP_CONF for HDFS based jobs ") { +// test("Include HADOOP_CONF for HDFS based jobs") { +// 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)) +// runSparkApplicationAndVerifyCompletion( +// JavaMainAppResource(CONTAINER_LOCAL_MAIN_APP_RESOURCE), +// SPARK_PI_MAIN_CLASS, +// Seq("HADOOP_CONF_DIR defined. Mounting HDFS specific .xml files", "Pi is roughly 3"), +// Array("5"), +// Seq.empty[String], +// Some("src/test/resources")) +// } + + test("Secure HDFS test with HDFS keytab") { assume(testBackend.name == MINIKUBE_TEST_BACKEND) - // Ensuring that HADOOP_CONF_DIR variable is set, could also be one via env HADOOP_CONF_DIR + launchKerberizedCluster() sparkConf.setJars(Seq(CONTAINER_LOCAL_HELPER_JAR_PATH)) + sparkConf.set(KUBERNETES_KERBEROS_SUPPORT, true) + sparkConf.set(KUBERNETES_KERBEROS_KEYTAB, "/tmp/keytabs/hdfs.keytab") + sparkConf.set(KUBERNETES_KERBEROS_PRINCIPAL, + s"hdfs/nn.${kubernetesTestComponents.namespace}.svc.cluster.local@CLUSTER.LOCAL") + System.setProperty("java.security.krb5.conf", "src/test/resources/krb5.conf") runSparkApplicationAndVerifyCompletion( JavaMainAppResource(CONTAINER_LOCAL_MAIN_APP_RESOURCE), SPARK_PI_MAIN_CLASS, @@ -85,167 +102,167 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { Some("test-data/hadoop-conf-files")) } - 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) - } +// 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 = { @@ -263,6 +280,12 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { s"${Minikube.getMinikubeIp}:$resourceStagingServerPort") } + private def launchKerberizedCluster(): Unit = { + assume(testBackend.name == MINIKUBE_TEST_BACKEND) + kerberizedHadoopClusterLauncher.launchKerberizedCluster() + Thread.sleep(60000) + } + private def runSparkPiAndVerifyCompletion(appResource: String): Unit = { runSparkApplicationAndVerifyCompletion( JavaMainAppResource(appResource), @@ -373,8 +396,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(5, Minutes)) + val INTERVAL = PatienceConfiguration.Interval(Span(5, Seconds)) val SPARK_PI_MAIN_CLASS = "org.apache.spark.deploy.kubernetes" + ".integrationtest.jobs.SparkPiWithInfiniteWait" val PYSPARK_PI_MAIN_CLASS = "org.apache.spark.deploy.PythonRunner" @@ -389,5 +412,6 @@ private[spark] object KubernetesSuite { ".integrationtest.jobs.GroupByTest" 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..54e2f62d87c2e 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 @@ -16,18 +16,17 @@ */ 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 + +import scala.collection.JavaConverters._ private[spark] class KubernetesTestComponents(defaultClient: DefaultKubernetesClient) { - val namespace = UUID.randomUUID().toString.replaceAll("-", "") + // val namespace = UUID.randomUUID().toString.replaceAll("-", "") + val namespace = "kerberostest" val kubernetesClient = defaultClient.inNamespace(namespace) val clientConfig = kubernetesClient.getConfiguration @@ -51,6 +50,18 @@ private[spark] class KubernetesTestComponents(defaultClient: DefaultKubernetesCl } } + def deletePersistentVolumes(): Unit = { + kubernetesClient.persistentVolumes().delete() + Eventually.eventually(KubernetesSuite.TIMEOUT, KubernetesSuite.INTERVAL) { + val persistentList = kubernetesClient + .persistentVolumes() + .list() + .getItems() + .asScala + require(!persistentList.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/backend/minikube/MinikubeTestBackend.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/backend/minikube/MinikubeTestBackend.scala index 461264877edc2..265daf40905e4 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/backend/minikube/MinikubeTestBackend.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/backend/minikube/MinikubeTestBackend.scala @@ -17,17 +17,15 @@ package org.apache.spark.deploy.kubernetes.integrationtest.backend.minikube import io.fabric8.kubernetes.client.DefaultKubernetesClient - import org.apache.spark.deploy.kubernetes.integrationtest.backend.IntegrationTestBackend import org.apache.spark.deploy.kubernetes.integrationtest.constants.MINIKUBE_TEST_BACKEND -import org.apache.spark.deploy.kubernetes.integrationtest.docker.SparkDockerImageBuilder private[spark] class MinikubeTestBackend extends IntegrationTestBackend { private var defaultClient: DefaultKubernetesClient = _ override def initialize(): Unit = { Minikube.startMinikube() - new SparkDockerImageBuilder(Minikube.getDockerEnv).buildSparkDockerImages() + // new SparkDockerImageBuilder(Minikube.getDockerEnv).buildSparkDockerImages() defaultClient = Minikube.getKubernetesClient } 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..66f616511bc39 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,14 @@ 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.apache.spark.internal.Logging +import org.apache.spark.util.RedirectThread 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 scala.collection.JavaConverters._ diff --git a/resource-managers/kubernetes/integration-tests/test-data/hadoop-conf-files/core-site.xml b/resource-managers/kubernetes/integration-tests/test-data/hadoop-conf-files/core-site.xml index f9e27564b7e9c..99425ba48d4b0 100644 --- a/resource-managers/kubernetes/integration-tests/test-data/hadoop-conf-files/core-site.xml +++ b/resource-managers/kubernetes/integration-tests/test-data/hadoop-conf-files/core-site.xml @@ -29,6 +29,6 @@ fs.defaultFS - hdfs://nn.default.svc.cluster.local:9000 + hdfs://nn.kerberostest.svc.cluster.local:9000 \ No newline at end of file diff --git a/resource-managers/kubernetes/integration-tests/test-data/hadoop-conf-files/hdfs-site.xml b/resource-managers/kubernetes/integration-tests/test-data/hadoop-conf-files/hdfs-site.xml index 1b2de25958c65..bd87f26be8626 100644 --- a/resource-managers/kubernetes/integration-tests/test-data/hadoop-conf-files/hdfs-site.xml +++ b/resource-managers/kubernetes/integration-tests/test-data/hadoop-conf-files/hdfs-site.xml @@ -40,11 +40,11 @@ dfs.namenode.kerberos.principal - hdfs/nn.default.svc.cluster.local@CLUSTER.LOCAL + hdfs/nn.kerberostest.svc.cluster.local@CLUSTER.LOCAL dfs.namenode.kerberos.internal.spnego.principal - HTTP/nn.default.svc.cluster.local@CLUSTER.LOCAL + HTTP/nn.kerberostest.svc.cluster.local@CLUSTER.LOCAL @@ -88,11 +88,11 @@ dfs.namenode.kerberos.principal - hdfs/nn.default.svc.cluster.local@CLUSTER.LOCAL + hdfs/nn.kerberostest.svc.cluster.local@CLUSTER.LOCAL dfs.namenode.kerberos.internal.spnego.principal - HTTP/nn.default.svc.cluster.local@CLUSTER.LOCAL + HTTP/nn.kerberostest.svc.cluster.local@CLUSTER.LOCAL @@ -112,7 +112,7 @@ dfs.datanode.kerberos.principal - hdfs/dn1.default.svc.cluster.local@CLUSTER.LOCAL + hdfs/dn1.kerberostest.svc.cluster.local@CLUSTER.LOCAL @@ -122,7 +122,7 @@ dfs.web.authentication.kerberos.principal - HTTP/dn1.default.svc.cluster.local@CLUSTER.LOCAL + HTTP/dn1.kerberostest.svc.cluster.local@CLUSTER.LOCAL dfs.web.authentication.kerberos.keytab diff --git a/resource-managers/kubernetes/integration-tests/test-data/hadoop-conf-files/yarn-site.xml b/resource-managers/kubernetes/integration-tests/test-data/hadoop-conf-files/yarn-site.xml new file mode 100644 index 0000000000000..92d9346232c76 --- /dev/null +++ b/resource-managers/kubernetes/integration-tests/test-data/hadoop-conf-files/yarn-site.xml @@ -0,0 +1,26 @@ + + + + + + + + + + + yarn.resourcemanager.principal + yarn/_HOST@CLUSTER.LOCAL + + \ No newline at end of file From 21e2dd69d842254cc377f59d22d0c43b318d73a3 Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Thu, 27 Jul 2017 21:34:45 -0700 Subject: [PATCH 12/15] working hadoop cluster --- .../HadoopKerberosKeytabResolverStep.scala | 2 +- .../src/main/docker/kerberos-test/Dockerfile | 45 ++++++++++ .../integrationtest/jobs/HDFSTest.scala | 46 ++++++++++ .../data-populator-deployment.yml | 1 + .../kerberos-yml/dn1-deployment.yml | 1 + .../kerberos-yml/kerberos-deployment.yml | 1 + .../kerberos-yml/kerberos-test.yml | 23 +++++ .../kerberos-yml/namenode-hadoop-pv.yml | 1 + .../kerberos-yml/nn-deployment.yml | 1 + .../kerberos-yml/server-keytab-pv.yml | 1 + .../kerberos-yml/test-env.sh | 2 + .../kubernetes/integration-tests/pom.xml | 33 +++++++ .../src/test/resources/core-site.xml | 2 +- .../src/test/resources/hdfs-site.xml | 12 +-- .../KerberizedHadoopClusterLauncher.scala | 82 ++++++++++-------- .../KerberosPVWatcherCache.scala | 83 ++++++++++++++++++ .../KerberosPodWatcherCache.scala | 85 +++++++++++++++++++ .../KerberosTestPodLauncher.scala | 80 +++++++++++++++++ .../integrationtest/KubernetesSuite.scala | 55 ++++++++---- .../KubernetesTestComponents.scala | 21 +++-- .../minikube/MinikubeTestBackend.scala | 4 +- .../docker/SparkDockerImageBuilder.scala | 22 +++-- .../test-data/hadoop-conf-files/core-site.xml | 2 +- .../test-data/hadoop-conf-files/hdfs-site.xml | 14 +-- 24 files changed, 535 insertions(+), 84 deletions(-) create mode 100644 resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/kerberos-test/Dockerfile create mode 100644 resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/kubernetes/integrationtest/jobs/HDFSTest.scala create mode 100644 resource-managers/kubernetes/integration-tests/kerberos-yml/kerberos-test.yml create mode 100644 resource-managers/kubernetes/integration-tests/kerberos-yml/test-env.sh create mode 100644 resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KerberosPVWatcherCache.scala create mode 100644 resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KerberosPodWatcherCache.scala create mode 100644 resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KerberosTestPodLauncher.scala 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 index 401fc60f73cad..7e870e6df9a38 100644 --- 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 @@ -59,7 +59,7 @@ private[spark] class HadoopKerberosKeytabResolverStep( } // In the case that keytab is not specified we will read from Local Ticket Cache val jobUserUGI = maybeJobUserUGI.getOrElse(UserGroupInformation.getCurrentUser) - logInfo("Primary group name: jobUserUGI.getPrimaryGroupName") + logInfo(s"Primary group name: ${jobUserUGI.getPrimaryGroupName}") val credentials: Credentials = jobUserUGI.getCredentials val credentialsManager = newHadoopTokenManager(submissionSparkConf, hadoopConf) var renewalTime = Long.MaxValue 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..6d6cb0078fd50 --- /dev/null +++ b/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/kerberos-test/Dockerfile @@ -0,0 +1,45 @@ +# +# 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 . + +COPY examples /opt/spark/examples +RUN apk add --no-cache --update krb5 krb5-libs +COPY hconf/krb5.conf /etc/krb5.conf +COPY test-scripts/test-env.sh / + +CMD /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.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.labels=spark-app-locator=${APP_LOCATOR_LABEL} \ + --files local:///etc/hadoop/core-site.xml,local:///etc/hadoop/hdfs-site.xml,local:///etc/hadoop/yarn-site.xml \ + ${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-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 index 5d6c752f78ea9..000d0c9665d3b 100644 --- a/resource-managers/kubernetes/integration-tests/kerberos-yml/data-populator-deployment.yml +++ b/resource-managers/kubernetes/integration-tests/kerberos-yml/data-populator-deployment.yml @@ -18,6 +18,7 @@ spec: name: data-populator image: ifilonenko/hadoop-base:latest imagePullPolicy: IfNotPresent + runAsNonRoot: false volumeMounts: - mountPath: /var/keytabs name: data-populator-keytab diff --git a/resource-managers/kubernetes/integration-tests/kerberos-yml/dn1-deployment.yml b/resource-managers/kubernetes/integration-tests/kerberos-yml/dn1-deployment.yml index eb68d5c011881..b4f94b40c9f29 100644 --- a/resource-managers/kubernetes/integration-tests/kerberos-yml/dn1-deployment.yml +++ b/resource-managers/kubernetes/integration-tests/kerberos-yml/dn1-deployment.yml @@ -18,6 +18,7 @@ spec: name: dn1 image: ifilonenko/hadoop-base:latest imagePullPolicy: IfNotPresent + runAsNonRoot: false volumeMounts: - mountPath: /var/keytabs name: dn1-keytab diff --git a/resource-managers/kubernetes/integration-tests/kerberos-yml/kerberos-deployment.yml b/resource-managers/kubernetes/integration-tests/kerberos-yml/kerberos-deployment.yml index 4268e424e4b28..9684e3fbc30f9 100644 --- a/resource-managers/kubernetes/integration-tests/kerberos-yml/kerberos-deployment.yml +++ b/resource-managers/kubernetes/integration-tests/kerberos-yml/kerberos-deployment.yml @@ -18,6 +18,7 @@ spec: name: kerberos image: ifilonenko/hadoop-base:latest imagePullPolicy: IfNotPresent + runAsNonRoot: false volumeMounts: - mountPath: /var/keytabs name: kerb-keytab 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..349ec784016e9 --- /dev/null +++ b/resource-managers/kubernetes/integration-tests/kerberos-yml/kerberos-test.yml @@ -0,0 +1,23 @@ +apiVersion: extensions/v1beta1 +kind: Deployment +metadata: + name: kerberos-test +spec: + replicas: 1 + template: + metadata: + labels: + name: kerberos-test + spec: + containers: + - 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 index d3d09f9d0699c..bc9da215d7b49 100644 --- a/resource-managers/kubernetes/integration-tests/kerberos-yml/namenode-hadoop-pv.yml +++ b/resource-managers/kubernetes/integration-tests/kerberos-yml/namenode-hadoop-pv.yml @@ -4,6 +4,7 @@ metadata: name: nn-hadoop labels: type: local + job: kerberostest spec: capacity: storage: 10Gi diff --git a/resource-managers/kubernetes/integration-tests/kerberos-yml/nn-deployment.yml b/resource-managers/kubernetes/integration-tests/kerberos-yml/nn-deployment.yml index ca2afa6a4c5c2..f9f6166ca6459 100644 --- a/resource-managers/kubernetes/integration-tests/kerberos-yml/nn-deployment.yml +++ b/resource-managers/kubernetes/integration-tests/kerberos-yml/nn-deployment.yml @@ -20,6 +20,7 @@ spec: - containerPort: 9000 image: ifilonenko/hadoop-base:latest imagePullPolicy: IfNotPresent + runAsNonRoot: false volumeMounts: - mountPath: /var/keytabs name: nn-keytab 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 index bf1667ebf66bd..01d50f2d9c46e 100644 --- a/resource-managers/kubernetes/integration-tests/kerberos-yml/server-keytab-pv.yml +++ b/resource-managers/kubernetes/integration-tests/kerberos-yml/server-keytab-pv.yml @@ -4,6 +4,7 @@ metadata: name: server-keytab labels: type: local + job: kerberostest spec: capacity: storage: 10Gi 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..13cb553753688 --- /dev/null +++ b/resource-managers/kubernetes/integration-tests/kerberos-yml/test-env.sh @@ -0,0 +1,2 @@ +#!/usr/bin/env bash +/usr/bin/kinit -kt /var/keytabs/hdfs.keytab hdfs/nn.${NAMESPACE}.svc.cluster.local \ 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 c31f75db5783e..ab9f356b2d680 100644 --- a/resource-managers/kubernetes/integration-tests/pom.xml +++ b/resource-managers/kubernetes/integration-tests/pom.xml @@ -240,6 +240,39 @@ + + copy-integration-bash + pre-integration-test + + copy-resources + + + ${project.build.directory}/docker/test-scripts + + + kerberos-yml + + test-env.sh + + + + + + + copy-integration-kerberos-conf + pre-integration-test + + copy-resources + + + ${project.build.directory}/docker/hconf + + + src/test/resources + + + + copy-integration-data pre-integration-test diff --git a/resource-managers/kubernetes/integration-tests/src/test/resources/core-site.xml b/resource-managers/kubernetes/integration-tests/src/test/resources/core-site.xml index 2165d00052a72..2d61ad740385a 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/resources/core-site.xml +++ b/resource-managers/kubernetes/integration-tests/src/test/resources/core-site.xml @@ -29,6 +29,6 @@ fs.defaultFS - hdfs://nn.${NAMESPACE}.svc.cluster.local:9000 + 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 index 8424aaba3c08d..37c1ed6a342da 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/resources/hdfs-site.xml +++ b/resource-managers/kubernetes/integration-tests/src/test/resources/hdfs-site.xml @@ -40,11 +40,11 @@ dfs.namenode.kerberos.principal - hdfs/nn.${NAMESPACE}.svc.cluster.local@CLUSTER.LOCAL + hdfs/nn.REPLACE_ME.svc.cluster.local@CLUSTER.LOCAL dfs.namenode.kerberos.internal.spnego.principal - HTTP/nn.${NAMESPACE}.svc.cluster.local@CLUSTER.LOCAL + HTTP/nn.REPLACE_ME.svc.cluster.local@CLUSTER.LOCAL @@ -88,11 +88,11 @@ dfs.namenode.kerberos.principal - hdfs/nn.${NAMESPACE}.svc.cluster.local@CLUSTER.LOCAL + hdfs/nn.REPLACE_ME.svc.cluster.local@CLUSTER.LOCAL dfs.namenode.kerberos.internal.spnego.principal - HTTP/nn.${NAMESPACE}.svc.cluster.local@CLUSTER.LOCAL + HTTP/nn.REPLACE_ME.svc.cluster.local@CLUSTER.LOCAL @@ -112,7 +112,7 @@ dfs.datanode.kerberos.principal - hdfs/dn1.${NAMESPACE}.svc.cluster.local@CLUSTER.LOCAL + hdfs/dn1.REPLACE_ME.svc.cluster.local@CLUSTER.LOCAL @@ -122,7 +122,7 @@ dfs.web.authentication.kerberos.principal - HTTP/dn1.${NAMESPACE}.svc.cluster.local@CLUSTER.LOCAL + HTTP/dn1.REPLACE_ME.svc.cluster.local@CLUSTER.LOCAL dfs.web.authentication.kerberos.keytab 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 index 2bb007bba3bbd..9324f130c4b86 100644 --- 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 @@ -24,30 +24,30 @@ import io.fabric8.kubernetes.client.KubernetesClient import org.apache.commons.io.FileUtils.readFileToString import org.apache.spark.deploy.kubernetes.submit.ContainerNameEqualityPredicate +import scala.collection.JavaConverters._ /** * Stuff */ private[spark] class KerberizedHadoopClusterLauncher( - kubernetesClient: KubernetesClient, - namespace: String) { - private def yamlLocation(loc: String) = s"kerberos-yml/$loc.yml" - private def loadFromYaml(resource: String) = + kubernetesClient: KubernetesClient, + namespace: String) { + private def yamlLocation(loc: String) = s"kerberos-yml/$loc.yml" + private def loadFromYaml(resource: String) = kubernetesClient.load(new FileInputStream(new File(yamlLocation(resource)))) -// private val regex = "REPLACE_ME".r -// private val krb5ConfFile = -// regex.replaceAllIn( -// readFileToString(new File("src/test/resources/krb5.conf")), -// namespace) - private val KRB_VOLUME = "krb5-conf" - private val KRB_FILE_DIR = "/etc" - private val KRB_CONFIG_MAP_NAME = "krb-config-map" - private val KRB_CONF_FILE = "krb5.conf" - private val KRB_KEY_PATH = - new KeyToPathBuilder() - .withKey(KRB_CONF_FILE) - .withPath(KRB_CONF_FILE) - .build() - + 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 keyPaths = kerberosFiles.map(file => + new KeyToPathBuilder() + .withKey(file) + .withPath(file) + .build()).toList def launchKerberizedCluster(): Unit = { val persistantVolumeList = Seq( "namenode-hadoop", @@ -65,15 +65,15 @@ private[spark] class KerberizedHadoopClusterLauncher( "data-populator-service") persistantVolumeList.foreach{resource => loadFromYaml(resource).createOrReplace() - Thread.sleep(10000)} -// val configMap = new ConfigMapBuilder() -// .withNewMetadata() -// .withName(KRB_CONFIG_MAP_NAME) -// .endMetadata() -// .addToData(KRB_CONF_FILE, krb5ConfFile) -// .build() -// kubernetesClient.configMaps().inNamespace(namespace).createOrReplace(configMap) -// Thread.sleep(2000) + Thread.sleep(20000)} + val configMap = new ConfigMapBuilder() + .withNewMetadata() + .withName(KRB_CONFIG_MAP_NAME) + .endMetadata() + .addToData(kerberosConfTupList.toMap.asJava) + .build() + kubernetesClient.configMaps().inNamespace(namespace).create(configMap) + Thread.sleep(2000) deploymentServiceList.foreach{ resource => loadFromYaml(resource).get().get(0) match { case deployment: Deployment => val deploymentWithEnv = new DeploymentBuilder(deployment) @@ -81,11 +81,11 @@ private[spark] class KerberizedHadoopClusterLauncher( .editTemplate() .editSpec() .addNewVolume() - .withName(KRB_VOLUME) - .withNewConfigMap() - .withName(KRB_CONFIG_MAP_NAME) - .withItems(KRB_KEY_PATH) - .endConfigMap() + .withName(KRB_VOLUME) + .withNewConfigMap() + .withName(KRB_CONFIG_MAP_NAME) + .withItems(keyPaths.asJava) + .endConfigMap() .endVolume() .editMatchingContainer(new ContainerNameEqualityPredicate( deployment.getMetadata.getName)) @@ -93,6 +93,18 @@ private[spark] class KerberizedHadoopClusterLauncher( .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) @@ -104,8 +116,8 @@ private[spark] class KerberizedHadoopClusterLauncher( .build() kubernetesClient.extensions().deployments().inNamespace(namespace).create(deploymentWithEnv) Thread.sleep(10000) - case service: Service => - loadFromYaml(resource).createOrReplace() + case serviceFromResource: Service => + kubernetesClient.services().inNamespace(namespace).create(serviceFromResource) Thread.sleep(10000)} } } diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KerberosPVWatcherCache.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KerberosPVWatcherCache.scala new file mode 100644 index 0000000000000..47963f9f65c36 --- /dev/null +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KerberosPVWatcherCache.scala @@ -0,0 +1,83 @@ +/* + * 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 + +private[spark] class KerberosPVWatcherCache() { +// client: KubernetesClient, +// dsNamespace: String, +// dsLabels: Map[String, String]) extends Logging { +// +// private var shufflePodCache = 2 +// private var watcher: Watch = _ +// +// def start(): Unit = { +// // seed the initial cache. +// val pvs = client.persistentVolumes().withLabels(dsLabels.asJava).list() +// pvs.getItems.asScala.foreach { +// pv => +// if (Readiness.isReady(pv)) { +// pvs.len +// } else { +// logWarning(s"Found unready shuffle pod ${pod.getMetadata.getName} " + +// s"on node ${pod.getSpec.getNodeName}") +// } +// } +// +// watcher = client +// .pods() +// .inNamespace(dsNamespace) +// .withLabels(dsLabels.asJava) +// .watch(new Watcher[Pod] { +// override def eventReceived(action: Watcher.Action, p: Pod): Unit = { +// action match { +// case Action.DELETED | Action.ERROR => +// shufflePodCache.remove(p.getSpec.getNodeName) +// case Action.ADDED | Action.MODIFIED if Readiness.isReady(p) => +// addShufflePodToCache(p) +// } +// } +// override def onClose(e: KubernetesClientException): Unit = {} +// }) +// } +// +// private def addShufflePodToCache(pod: Pod): Unit = { +// if (shufflePodCache.contains(pod.getSpec.getNodeName)) { +// val registeredPodName = shufflePodCache.get(pod.getSpec.getNodeName).get +// logError(s"Ambiguous specification of shuffle service pod. " + +// s"Found multiple matching pods: ${pod.getMetadata.getName}, " + +// s"${registeredPodName} on ${pod.getSpec.getNodeName}") +// +// throw new SparkException(s"Ambiguous specification of shuffle service pod. " + +// s"Found multiple matching pods: ${pod.getMetadata.getName}, " + +// s"${registeredPodName} on ${pod.getSpec.getNodeName}") +// } else { +// shufflePodCache(pod.getSpec.getNodeName) = pod.getStatus.getPodIP +// } +// } +// +// def stop(): Unit = { +// watcher.close() +// } +// +// def getShufflePodForExecutor(executorNode: String): String = { +// shufflePodCache.get(executorNode) +// .getOrElse(throw new SparkException(s"Unable to find shuffle pod on node $executorNode")) +// } + +} + diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KerberosPodWatcherCache.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KerberosPodWatcherCache.scala new file mode 100644 index 0000000000000..bd33ffd6e34ed --- /dev/null +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KerberosPodWatcherCache.scala @@ -0,0 +1,85 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.deploy.kubernetes.integrationtest + +private[spark] class KerberosPodWatcherCache() { + + +// client: KubernetesClient, +// dsNamespace: String, +// dsLabels: Map[String, String]) extends Logging { +// +// private var shufflePodCache = scala.collection.mutable.Map[String, String]() +// private var watcher: Watch = _ +// +// def start(): Unit = { +// // seed the initial cache. +// val pods = client.pods() +// .inNamespace(dsNamespace).withLabels(dsLabels.asJava).list() +// pods.getItems.asScala.foreach { +// pod => +// if (Readiness.isReady(pod)) { +// addShufflePodToCache(pod) +// } else { +// logWarning(s"Found unready shuffle pod ${pod.getMetadata.getName} " + +// s"on node ${pod.getSpec.getNodeName}") +// } +// } +// +// watcher = client +// .pods() +// .inNamespace(dsNamespace) +// .withLabels(dsLabels.asJava) +// .watch(new Watcher[Pod] { +// override def eventReceived(action: Watcher.Action, p: Pod): Unit = { +// action match { +// case Action.DELETED | Action.ERROR => +// shufflePodCache.remove(p.getSpec.getNodeName) +// case Action.ADDED | Action.MODIFIED if Readiness.isReady(p) => +// addShufflePodToCache(p) +// } +// } +// override def onClose(e: KubernetesClientException): Unit = {} +// }) +// } +// +// private def addShufflePodToCache(pod: Pod): Unit = { +// if (shufflePodCache.contains(pod.getSpec.getNodeName)) { +// val registeredPodName = shufflePodCache.get(pod.getSpec.getNodeName).get +// logError(s"Ambiguous specification of shuffle service pod. " + +// s"Found multiple matching pods: ${pod.getMetadata.getName}, " + +// s"${registeredPodName} on ${pod.getSpec.getNodeName}") +// +// throw new SparkException(s"Ambiguous specification of shuffle service pod. " + +// s"Found multiple matching pods: ${pod.getMetadata.getName}, " + +// s"${registeredPodName} on ${pod.getSpec.getNodeName}") +// } else { +// shufflePodCache(pod.getSpec.getNodeName) = pod.getStatus.getPodIP +// } +// } +// +// def stop(): Unit = { +// watcher.close() +// } +// +// def getShufflePodForExecutor(executorNode: String): String = { +// shufflePodCache.get(executorNode) +// .getOrElse(throw new SparkException(s"Unable to find shuffle pod on node $executorNode")) +// } +} + 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..705f1dd3646ac --- /dev/null +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KerberosTestPodLauncher.scala @@ -0,0 +1,80 @@ +/* + * 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 io.fabric8.kubernetes.api.model.extensions.{Deployment, DeploymentBuilder} +import io.fabric8.kubernetes.client.KubernetesClient + +import org.apache.spark.deploy.kubernetes.submit.ContainerNameEqualityPredicate + + /** + * Stuff + */ +private[spark] class KerberosTestPodLauncher( + kubernetesClient: KubernetesClient, + namespace: String) { + + private val yamlLocation = "kerberos-yml/kerberos-test.yml" + 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() + .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("hconf") + .endEnv() + .addNewEnv() + .withName("APP_LOCATOR_LABEL") + .withValue(appLabel) + .endEnv() + .addNewEnv() + .withName("SPARK_PRINT_LAUNCH_COMMAND") + .withValue("true") + .endEnv() + .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 48ed5868e2c5c..97a9ee3824de2 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 @@ -20,12 +20,16 @@ import java.nio.file.Paths import java.util.UUID import io.fabric8.kubernetes.client.internal.readiness.Readiness + +import org.apache.spark.{SparkConf, SSLOptions, 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._ -import org.apache.spark.{SSLOptions, SparkConf, SparkFunSuite} + import org.scalatest.BeforeAndAfter import org.scalatest.concurrent.{Eventually, PatienceConfiguration} import org.scalatest.time.{Minutes, Seconds, Span} @@ -41,6 +45,7 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { private var resourceStagingServerLauncher: ResourceStagingServerLauncher = _ private var staticAssetServerLauncher: StaticAssetServerLauncher = _ private var kerberizedHadoopClusterLauncher: KerberizedHadoopClusterLauncher = _ + private var kerberosTestLauncher: KerberosTestPodLauncher = _ override def beforeAll(): Unit = { testBackend.initialize() @@ -52,6 +57,9 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { kerberizedHadoopClusterLauncher = new KerberizedHadoopClusterLauncher( kubernetesTestComponents.kubernetesClient.inNamespace(kubernetesTestComponents.namespace), kubernetesTestComponents.namespace) + kerberosTestLauncher = new KerberosTestPodLauncher( + kubernetesTestComponents.kubernetesClient.inNamespace(kubernetesTestComponents.namespace), + kubernetesTestComponents.namespace) } override def afterAll(): Unit = { @@ -67,7 +75,7 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { } after { - kubernetesTestComponents.deletePersistentVolumes() + kubernetesTestComponents.deleteKubernetesResources() // kubernetesTestComponents.deleteNamespace() } @@ -87,19 +95,24 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { test("Secure HDFS test with HDFS keytab") { assume(testBackend.name == MINIKUBE_TEST_BACKEND) launchKerberizedCluster() - sparkConf.setJars(Seq(CONTAINER_LOCAL_HELPER_JAR_PATH)) - sparkConf.set(KUBERNETES_KERBEROS_SUPPORT, true) - sparkConf.set(KUBERNETES_KERBEROS_KEYTAB, "/tmp/keytabs/hdfs.keytab") - sparkConf.set(KUBERNETES_KERBEROS_PRINCIPAL, - s"hdfs/nn.${kubernetesTestComponents.namespace}.svc.cluster.local@CLUSTER.LOCAL") - System.setProperty("java.security.krb5.conf", "src/test/resources/krb5.conf") - runSparkApplicationAndVerifyCompletion( - JavaMainAppResource(CONTAINER_LOCAL_MAIN_APP_RESOURCE), - SPARK_PI_MAIN_CLASS, - Seq("HADOOP_CONF_DIR defined. Mounting HDFS specific .xml files", "Pi is roughly 3"), - Array("5"), - Seq.empty[String], - Some("test-data/hadoop-conf-files")) + createKerberosTestPod(CONTAINER_LOCAL_MAIN_APP_RESOURCE, HDFS_TEST_CLASS, APP_LOCATOR_LABEL) + val expectedLogOnCompletion = Seq("Something something something") + Thread.sleep(50000) +// val driverPod = kubernetesTestComponents.kubernetesClient +// .pods() +// .withLabel("spark-app-locator", APP_LOCATOR_LABEL) +// .list() +// .getItems +// .get(0) +// Eventually.eventually(TIMEOUT, INTERVAL) { +// expectedLogOnCompletion.foreach { e => +// assert(kubernetesTestComponents.kubernetesClient +// .pods() +// .withName(driverPod.getMetadata.getName) +// .getLog +// .contains(e), "The application did not complete.") +// } +// } } // test("Run PySpark Job on file from SUBMITTER with --py-files") { @@ -283,7 +296,11 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { private def launchKerberizedCluster(): Unit = { assume(testBackend.name == MINIKUBE_TEST_BACKEND) kerberizedHadoopClusterLauncher.launchKerberizedCluster() - Thread.sleep(60000) + } + + 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 = { @@ -396,8 +413,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(5, Minutes)) - val INTERVAL = PatienceConfiguration.Interval(Span(5, 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" @@ -410,6 +427,8 @@ 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" 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 54e2f62d87c2e..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 @@ -16,17 +16,19 @@ */ package org.apache.spark.deploy.kubernetes.integrationtest +import java.util.UUID + import io.fabric8.kubernetes.client.DefaultKubernetesClient +import scala.collection.JavaConverters._ + import org.apache.spark.SparkConf import org.apache.spark.deploy.kubernetes.config._ -import org.scalatest.concurrent.Eventually -import scala.collection.JavaConverters._ +import org.scalatest.concurrent.Eventually private[spark] class KubernetesTestComponents(defaultClient: DefaultKubernetesClient) { - // val namespace = UUID.randomUUID().toString.replaceAll("-", "") - val namespace = "kerberostest" + val namespace = UUID.randomUUID().toString.replaceAll("-", "") val kubernetesClient = defaultClient.inNamespace(namespace) val clientConfig = kubernetesClient.getConfiguration @@ -50,7 +52,7 @@ private[spark] class KubernetesTestComponents(defaultClient: DefaultKubernetesCl } } - def deletePersistentVolumes(): Unit = { + def deleteKubernetesResources(): Unit = { kubernetesClient.persistentVolumes().delete() Eventually.eventually(KubernetesSuite.TIMEOUT, KubernetesSuite.INTERVAL) { val persistentList = kubernetesClient @@ -60,6 +62,15 @@ private[spark] class KubernetesTestComponents(defaultClient: DefaultKubernetesCl .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 = { diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/backend/minikube/MinikubeTestBackend.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/backend/minikube/MinikubeTestBackend.scala index 265daf40905e4..461264877edc2 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/backend/minikube/MinikubeTestBackend.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/backend/minikube/MinikubeTestBackend.scala @@ -17,15 +17,17 @@ package org.apache.spark.deploy.kubernetes.integrationtest.backend.minikube import io.fabric8.kubernetes.client.DefaultKubernetesClient + import org.apache.spark.deploy.kubernetes.integrationtest.backend.IntegrationTestBackend import org.apache.spark.deploy.kubernetes.integrationtest.constants.MINIKUBE_TEST_BACKEND +import org.apache.spark.deploy.kubernetes.integrationtest.docker.SparkDockerImageBuilder private[spark] class MinikubeTestBackend extends IntegrationTestBackend { private var defaultClient: DefaultKubernetesClient = _ override def initialize(): Unit = { Minikube.startMinikube() - // new SparkDockerImageBuilder(Minikube.getDockerEnv).buildSparkDockerImages() + new SparkDockerImageBuilder(Minikube.getDockerEnv).buildSparkDockerImages() defaultClient = Minikube.getKubernetesClient } 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 66f616511bc39..fabbbb1d68c03 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 @@ -22,7 +22,9 @@ import java.nio.file.Paths import com.spotify.docker.client.{DefaultDockerClient, DockerCertificates, LoggingBuildHandler} import org.apache.http.client.utils.URIBuilder + 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} @@ -46,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", @@ -85,15 +88,16 @@ private[spark] class SparkDockerImageBuilder if (exitCode != 0) { logInfo(s"exitCode: $exitCode") } - buildImage("spark-base", BASE_DOCKER_FILE) - buildImage("spark-driver", DRIVER_DOCKER_FILE) - buildImage("spark-driver-py", DRIVERPY_DOCKER_FILE) - buildImage("spark-executor", EXECUTOR_DOCKER_FILE) - buildImage("spark-executor-py", EXECUTORPY_DOCKER_FILE) - buildImage("spark-shuffle", SHUFFLE_SERVICE_DOCKER_FILE) - 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("spark-base", BASE_DOCKER_FILE) +// buildImage("spark-driver", DRIVER_DOCKER_FILE) +// buildImage("spark-driver-py", DRIVERPY_DOCKER_FILE) +// buildImage("spark-executor", EXECUTOR_DOCKER_FILE) +// buildImage("spark-executor-py", EXECUTORPY_DOCKER_FILE) +// buildImage("spark-shuffle", SHUFFLE_SERVICE_DOCKER_FILE) +// 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/test-data/hadoop-conf-files/core-site.xml b/resource-managers/kubernetes/integration-tests/test-data/hadoop-conf-files/core-site.xml index 99425ba48d4b0..2165d00052a72 100644 --- a/resource-managers/kubernetes/integration-tests/test-data/hadoop-conf-files/core-site.xml +++ b/resource-managers/kubernetes/integration-tests/test-data/hadoop-conf-files/core-site.xml @@ -29,6 +29,6 @@ fs.defaultFS - hdfs://nn.kerberostest.svc.cluster.local:9000 + hdfs://nn.${NAMESPACE}.svc.cluster.local:9000 \ No newline at end of file diff --git a/resource-managers/kubernetes/integration-tests/test-data/hadoop-conf-files/hdfs-site.xml b/resource-managers/kubernetes/integration-tests/test-data/hadoop-conf-files/hdfs-site.xml index bd87f26be8626..8a00d75d7882d 100644 --- a/resource-managers/kubernetes/integration-tests/test-data/hadoop-conf-files/hdfs-site.xml +++ b/resource-managers/kubernetes/integration-tests/test-data/hadoop-conf-files/hdfs-site.xml @@ -40,11 +40,11 @@ dfs.namenode.kerberos.principal - hdfs/nn.kerberostest.svc.cluster.local@CLUSTER.LOCAL + hdfs/nn.${NAMESPACE}.svc.cluster.local@CLUSTER.LOCAL dfs.namenode.kerberos.internal.spnego.principal - HTTP/nn.kerberostest.svc.cluster.local@CLUSTER.LOCAL + HTTP/nn.${NAMESPACE}.svc.cluster.local@CLUSTER.LOCAL @@ -88,11 +88,11 @@ dfs.namenode.kerberos.principal - hdfs/nn.kerberostest.svc.cluster.local@CLUSTER.LOCAL + hdfs/nn.${NAMESPACE}.svc.cluster.local@CLUSTER.LOCAL dfs.namenode.kerberos.internal.spnego.principal - HTTP/nn.kerberostest.svc.cluster.local@CLUSTER.LOCAL + HTTP/nn.${NAMESPACE}.svc.cluster.local@CLUSTER.LOCAL @@ -108,11 +108,11 @@ dfs.datanode.keytab.file - /var/keytabs/hdfs.keytab + /var/keytabs/hdfs.keytab dfs.datanode.kerberos.principal - hdfs/dn1.kerberostest.svc.cluster.local@CLUSTER.LOCAL + hdfs/dn1.${NAMESPACE}.svc.cluster.local@CLUSTER.LOCAL @@ -122,7 +122,7 @@ dfs.web.authentication.kerberos.principal - HTTP/dn1.kerberostest.svc.cluster.local@CLUSTER.LOCAL + HTTP/dn1.${NAMESPACE}.svc.cluster.local@CLUSTER.LOCAL dfs.web.authentication.kerberos.keytab From 6fbdee26eb4aa4530d0197e677a2c7bcc4b3e06b Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Sat, 29 Jul 2017 03:05:39 -0700 Subject: [PATCH 13/15] Using locks and monitors to ensure proper configs for setting up kerberized cluster in integration tests --- .../HadoopKerberosKeytabResolverStep.scala | 75 +++-- .../src/main/docker/kerberos-test/Dockerfile | 24 +- .../data-populator-deployment.yml | 1 + .../kerberos-yml/data-populator-service.yml | 1 + .../kerberos-yml/dn1-deployment.yml | 1 + .../kerberos-yml/dn1-service.yml | 1 + .../kerberos-yml/kerberos-deployment.yml | 1 + .../kerberos-yml/kerberos-service.yml | 1 + .../kerberos-yml/namenode-hadoop-pv.yml | 2 +- .../kerberos-yml/namenode-hadoop.yml | 2 + .../kerberos-yml/nn-deployment.yml | 1 + .../kerberos-yml/nn-service.yml | 1 + .../kerberos-yml/server-keytab-pv.yml | 2 +- .../kerberos-yml/server-keytab.yml | 2 + .../kerberos-yml/test-env.sh | 25 +- .../src/test/resources/hdfs-site.xml | 12 + .../KerberizedHadoopClusterLauncher.scala | 111 +------- .../KerberosCMWatcherCache.scala | 100 +++++++ .../integrationtest/KerberosDeployment.scala | 24 ++ .../KerberosPVWatcherCache.scala | 220 +++++++++++---- .../KerberosPodWatcherCache.scala | 264 +++++++++++++----- .../integrationtest/KerberosStorage.scala | 23 ++ .../KerberosTestPodLauncher.scala | 127 ++++++--- .../integrationtest/KerberosUtils.scala | 126 +++++++++ .../integrationtest/KubernetesSuite.scala | 1 - .../minikube/MinikubeTestBackend.scala | 2 +- .../docker/SparkDockerImageBuilder.scala | 18 +- .../test-data/hadoop-conf-files/core-site.xml | 34 --- .../test-data/hadoop-conf-files/hdfs-site.xml | 132 --------- .../test-data/hadoop-conf-files/yarn-site.xml | 26 -- 30 files changed, 835 insertions(+), 525 deletions(-) create mode 100644 resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KerberosCMWatcherCache.scala create mode 100644 resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KerberosDeployment.scala create mode 100644 resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KerberosStorage.scala create mode 100644 resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KerberosUtils.scala delete mode 100644 resource-managers/kubernetes/integration-tests/test-data/hadoop-conf-files/core-site.xml delete mode 100644 resource-managers/kubernetes/integration-tests/test-data/hadoop-conf-files/hdfs-site.xml delete mode 100644 resource-managers/kubernetes/integration-tests/test-data/hadoop-conf-files/yarn-site.xml 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 index 7e870e6df9a38..8614bf1d6cace 100644 --- 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 @@ -17,12 +17,17 @@ 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.SparkHadoopUtil @@ -30,6 +35,8 @@ import org.apache.spark.deploy.kubernetes.{KerberosConfBootstrapImpl, PodWithMai import org.apache.spark.deploy.kubernetes.constants._ import org.apache.spark.internal.Logging + + /** * Step that configures the ConfigMap + Volumes for the driver */ @@ -59,20 +66,26 @@ private[spark] class HadoopKerberosKeytabResolverStep( } // In the case that keytab is not specified we will read from Local Ticket Cache val jobUserUGI = maybeJobUserUGI.getOrElse(UserGroupInformation.getCurrentUser) - logInfo(s"Primary group name: ${jobUserUGI.getPrimaryGroupName}") - val credentials: Credentials = jobUserUGI.getCredentials - val credentialsManager = newHadoopTokenManager(submissionSparkConf, hadoopConf) - var renewalTime = Long.MaxValue - jobUserUGI.doAs(new PrivilegedExceptionAction[Void] { - override def run(): Void = { - renewalTime = Math.min( - obtainCredentials(credentialsManager, hadoopConf, credentials), - renewalTime) - null - } - }) - if (credentials.getAllTokens.isEmpty) logError("Did not obtain any Delegation Tokens") - val data = serialize(credentials) + logInfo(s"Retrieved Job User UGI: $jobUserUGI") + val originalCredentials: Credentials = jobUserUGI.getCredentials + logInfo(s"Original tokens: ${originalCredentials.toString}") + logInfo(s"All tokens: ${originalCredentials.getAllTokens}") + logInfo(s"All secret keys: ${originalCredentials.getAllSecretKeys}") + val dfs: FileSystem = FileSystem.get(hadoopConf) + // This is not necessary with [Spark-20328] since we would be using + // Spark core providers to handle delegation token renewal + val renewer: String = jobUserUGI.getShortUserName + logInfo(s"Renewer is: $renewer") + val renewedCredentials: Credentials = new Credentials(originalCredentials) + dfs.addDelegationTokens(renewer, renewedCredentials) + val renewedTokens = renewedCredentials.getAllTokens.asScala + logInfo(s"Renewed tokens: ${renewedCredentials.toString}") + logInfo(s"All renewed tokens: ${renewedTokens}") + logInfo(s"All renewed secret keys: ${renewedCredentials.getAllSecretKeys}") + 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 initialTokenLabelName = s"$KERBEROS_SECRET_LABEL_PREFIX-1-$renewalTime" logInfo(s"Storing dt in $initialTokenLabelName") @@ -97,24 +110,24 @@ private[spark] class HadoopKerberosKeytabResolverStep( dtSecret = Some(secretDT)) } - // Functions that should be in SparkHadoopUtil with Rebase to 2.2 + // Functions that should be in Core with Rebase to 2.3 @deprecated("Moved to core in 2.2", "2.2") - private def obtainCredentials(instance: Any, args: AnyRef*): Long = { - val method = Class - .forName("org.apache.spark.deploy.yarn.security.ConfigurableCredentialManager") - .getMethod("obtainCredentials", classOf[Configuration], classOf[Configuration]) - method.setAccessible(true) - method.invoke(instance, args: _*).asInstanceOf[Long] + 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") - // This method will instead be using HadoopDelegationTokenManager from Spark 2.2 - private def newHadoopTokenManager(args: AnyRef*): Any = { - val constructor = Class - .forName("org.apache.spark.deploy.yarn.security.ConfigurableCredentialManager") - .getConstructor(classOf[SparkConf], classOf[Configuration]) - constructor.setAccessible(true) - constructor.newInstance(args: _*) - } + @deprecated("Moved to core in 2.2", "2.2") private def serialize(creds: Credentials): Array[Byte] = { val byteStream = new ByteArrayOutputStream 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 index 6d6cb0078fd50..c4ba43ad511d8 100644 --- 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 @@ -21,25 +21,7 @@ FROM spark-base # 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 . -COPY examples /opt/spark/examples RUN apk add --no-cache --update krb5 krb5-libs -COPY hconf/krb5.conf /etc/krb5.conf -COPY test-scripts/test-env.sh / - -CMD /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.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.labels=spark-app-locator=${APP_LOCATOR_LABEL} \ - --files local:///etc/hadoop/core-site.xml,local:///etc/hadoop/hdfs-site.xml,local:///etc/hadoop/yarn-site.xml \ - ${SUBMIT_RESOURCE} \ - hdfs://nn.${NAMESPACE}.svc.cluster.local:9000/user/ifilonenko/wordcount.txt \ No newline at end of file +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/kerberos-yml/data-populator-deployment.yml b/resource-managers/kubernetes/integration-tests/kerberos-yml/data-populator-deployment.yml index 000d0c9665d3b..87826f9e9fc02 100644 --- a/resource-managers/kubernetes/integration-tests/kerberos-yml/data-populator-deployment.yml +++ b/resource-managers/kubernetes/integration-tests/kerberos-yml/data-populator-deployment.yml @@ -11,6 +11,7 @@ spec: labels: name: hdfs-data-populator kerberosService: data-populator + job: kerberos-test spec: containers: - command: 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 index 8d8e02671c2c2..366ca23657c56 100644 --- a/resource-managers/kubernetes/integration-tests/kerberos-yml/data-populator-service.yml +++ b/resource-managers/kubernetes/integration-tests/kerberos-yml/data-populator-service.yml @@ -5,6 +5,7 @@ metadata: service.alpha.kubernetes.io/tolerate-unready-endpoints: "true" labels: kerberosService: data-populator + job: kerberos-test name: data-populator spec: clusterIP: None diff --git a/resource-managers/kubernetes/integration-tests/kerberos-yml/dn1-deployment.yml b/resource-managers/kubernetes/integration-tests/kerberos-yml/dn1-deployment.yml index b4f94b40c9f29..73873ff860342 100644 --- a/resource-managers/kubernetes/integration-tests/kerberos-yml/dn1-deployment.yml +++ b/resource-managers/kubernetes/integration-tests/kerberos-yml/dn1-deployment.yml @@ -11,6 +11,7 @@ spec: labels: name: hdfs-dn1 kerberosService: dn1 + job: kerberos-test spec: containers: - command: diff --git a/resource-managers/kubernetes/integration-tests/kerberos-yml/dn1-service.yml b/resource-managers/kubernetes/integration-tests/kerberos-yml/dn1-service.yml index f746661721da6..9afe0c1c14e12 100644 --- a/resource-managers/kubernetes/integration-tests/kerberos-yml/dn1-service.yml +++ b/resource-managers/kubernetes/integration-tests/kerberos-yml/dn1-service.yml @@ -5,6 +5,7 @@ metadata: service.alpha.kubernetes.io/tolerate-unready-endpoints: "true" labels: kerberosService: dn1 + job: kerberos-test name: dn1 spec: clusterIP: None diff --git a/resource-managers/kubernetes/integration-tests/kerberos-yml/kerberos-deployment.yml b/resource-managers/kubernetes/integration-tests/kerberos-yml/kerberos-deployment.yml index 9684e3fbc30f9..a2a977ca66a48 100644 --- a/resource-managers/kubernetes/integration-tests/kerberos-yml/kerberos-deployment.yml +++ b/resource-managers/kubernetes/integration-tests/kerberos-yml/kerberos-deployment.yml @@ -11,6 +11,7 @@ spec: labels: name: hdfs-kerberos kerberosService: kerberos + job: kerberos-test spec: containers: - command: diff --git a/resource-managers/kubernetes/integration-tests/kerberos-yml/kerberos-service.yml b/resource-managers/kubernetes/integration-tests/kerberos-yml/kerberos-service.yml index 38746af6a0ad0..0fb6e31cd8d6c 100644 --- a/resource-managers/kubernetes/integration-tests/kerberos-yml/kerberos-service.yml +++ b/resource-managers/kubernetes/integration-tests/kerberos-yml/kerberos-service.yml @@ -5,6 +5,7 @@ metadata: service.alpha.kubernetes.io/tolerate-unready-endpoints: "true" labels: kerberosService: kerberos + job: kerberos-test name: kerberos spec: clusterIP: None 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 index bc9da215d7b49..d813c90030444 100644 --- a/resource-managers/kubernetes/integration-tests/kerberos-yml/namenode-hadoop-pv.yml +++ b/resource-managers/kubernetes/integration-tests/kerberos-yml/namenode-hadoop-pv.yml @@ -7,7 +7,7 @@ metadata: job: kerberostest spec: capacity: - storage: 10Gi + storage: 1Gi accessModes: - ReadWriteOnce hostPath: diff --git a/resource-managers/kubernetes/integration-tests/kerberos-yml/namenode-hadoop.yml b/resource-managers/kubernetes/integration-tests/kerberos-yml/namenode-hadoop.yml index 3f7a74726ae01..3e72046cd1d8f 100644 --- a/resource-managers/kubernetes/integration-tests/kerberos-yml/namenode-hadoop.yml +++ b/resource-managers/kubernetes/integration-tests/kerberos-yml/namenode-hadoop.yml @@ -2,6 +2,8 @@ apiVersion: v1 kind: PersistentVolumeClaim metadata: name: nn-hadoop + labels: + job: kerberostest spec: accessModes: - ReadWriteOnce diff --git a/resource-managers/kubernetes/integration-tests/kerberos-yml/nn-deployment.yml b/resource-managers/kubernetes/integration-tests/kerberos-yml/nn-deployment.yml index f9f6166ca6459..2907f2e356258 100644 --- a/resource-managers/kubernetes/integration-tests/kerberos-yml/nn-deployment.yml +++ b/resource-managers/kubernetes/integration-tests/kerberos-yml/nn-deployment.yml @@ -11,6 +11,7 @@ spec: labels: name: hdfs-nn kerberosService: nn + job: kerberos-test spec: containers: - command: diff --git a/resource-managers/kubernetes/integration-tests/kerberos-yml/nn-service.yml b/resource-managers/kubernetes/integration-tests/kerberos-yml/nn-service.yml index 4f1e44a22ba55..bf85e12a96df4 100644 --- a/resource-managers/kubernetes/integration-tests/kerberos-yml/nn-service.yml +++ b/resource-managers/kubernetes/integration-tests/kerberos-yml/nn-service.yml @@ -5,6 +5,7 @@ metadata: service.alpha.kubernetes.io/tolerate-unready-endpoints: "true" labels: kerberosService: nn + job: kerberos-test name: nn spec: clusterIP: None 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 index 01d50f2d9c46e..50298c5fce13e 100644 --- a/resource-managers/kubernetes/integration-tests/kerberos-yml/server-keytab-pv.yml +++ b/resource-managers/kubernetes/integration-tests/kerberos-yml/server-keytab-pv.yml @@ -7,7 +7,7 @@ metadata: job: kerberostest spec: capacity: - storage: 10Gi + storage: 1Gi accessModes: - ReadWriteOnce hostPath: diff --git a/resource-managers/kubernetes/integration-tests/kerberos-yml/server-keytab.yml b/resource-managers/kubernetes/integration-tests/kerberos-yml/server-keytab.yml index 873f263c42fcd..6c6a1008e1441 100644 --- a/resource-managers/kubernetes/integration-tests/kerberos-yml/server-keytab.yml +++ b/resource-managers/kubernetes/integration-tests/kerberos-yml/server-keytab.yml @@ -2,6 +2,8 @@ apiVersion: v1 kind: PersistentVolumeClaim metadata: name: server-keytab + labels: + job: kerberostest spec: accessModes: - ReadWriteOnce diff --git a/resource-managers/kubernetes/integration-tests/kerberos-yml/test-env.sh b/resource-managers/kubernetes/integration-tests/kerberos-yml/test-env.sh index 13cb553753688..e070655bfa47d 100644 --- a/resource-managers/kubernetes/integration-tests/kerberos-yml/test-env.sh +++ b/resource-managers/kubernetes/integration-tests/kerberos-yml/test-env.sh @@ -1,2 +1,25 @@ #!/usr/bin/env bash -/usr/bin/kinit -kt /var/keytabs/hdfs.keytab hdfs/nn.${NAMESPACE}.svc.cluster.local \ No newline at end of file +sed -i -e 's/#//' -e 's/default_ccache_name/# default_ccache_name/' /etc/krb5.conf +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 +/usr/bin/kinit -kt /var/keytabs/hdfs.keytab hdfs/nn.${NAMESPACE}.svc.cluster.local + +/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.labels=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/src/test/resources/hdfs-site.xml b/resource-managers/kubernetes/integration-tests/src/test/resources/hdfs-site.xml index 37c1ed6a342da..7233af3c4b951 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/resources/hdfs-site.xml +++ b/resource-managers/kubernetes/integration-tests/src/test/resources/hdfs-site.xml @@ -46,6 +46,10 @@ 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 + @@ -106,6 +110,14 @@ 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 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 index 9324f130c4b86..d8fad52414ad5 100644 --- 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 @@ -16,109 +16,26 @@ */ package org.apache.spark.deploy.kubernetes.integrationtest -import java.io.{File, FileInputStream} - -import io.fabric8.kubernetes.api.model.extensions.{Deployment, DeploymentBuilder} -import io.fabric8.kubernetes.api.model.{ConfigMapBuilder, KeyToPathBuilder, Service} import io.fabric8.kubernetes.client.KubernetesClient -import org.apache.commons.io.FileUtils.readFileToString -import org.apache.spark.deploy.kubernetes.submit.ContainerNameEqualityPredicate -import scala.collection.JavaConverters._ /** * Stuff */ private[spark] class KerberizedHadoopClusterLauncher( kubernetesClient: KubernetesClient, namespace: String) { - private def yamlLocation(loc: String) = s"kerberos-yml/$loc.yml" - private def loadFromYaml(resource: String) = - kubernetesClient.load(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 keyPaths = kerberosFiles.map(file => - new KeyToPathBuilder() - .withKey(file) - .withPath(file) - .build()).toList - def launchKerberizedCluster(): Unit = { - val persistantVolumeList = Seq( - "namenode-hadoop", - "namenode-hadoop-pv", - "server-keytab", - "server-keytab-pv") - val deploymentServiceList = Seq( - "kerberos-deployment", - "kerberos-service", - "nn-deployment", - "nn-service", - "dn1-deployment", - "dn1-service", - "data-populator-deployment", - "data-populator-service") - persistantVolumeList.foreach{resource => - loadFromYaml(resource).createOrReplace() - Thread.sleep(20000)} - val configMap = new ConfigMapBuilder() - .withNewMetadata() - .withName(KRB_CONFIG_MAP_NAME) - .endMetadata() - .addToData(kerberosConfTupList.toMap.asJava) - .build() - kubernetesClient.configMaps().inNamespace(namespace).create(configMap) - Thread.sleep(2000) - deploymentServiceList.foreach{ resource => loadFromYaml(resource).get().get(0) match { - case deployment: Deployment => - val deploymentWithEnv = 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.extensions().deployments().inNamespace(namespace).create(deploymentWithEnv) - Thread.sleep(10000) - case serviceFromResource: Service => - kubernetesClient.services().inNamespace(namespace).create(serviceFromResource) - Thread.sleep(10000)} - } - } + private val LABELS = Map("job" -> "kerberostest") + + def launchKerberizedCluster(): Unit = { + val kerberosUtils = new KerberosUtils(kubernetesClient, namespace) + val pvWatcherCache = new KerberosPVWatcherCache(kerberosUtils, LABELS) + pvWatcherCache.start() + pvWatcherCache.stop() + val cmWatcherCache = new KerberosCMWatcherCache(kerberosUtils) + cmWatcherCache.start() + cmWatcherCache.stop() + 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/KerberosCMWatcherCache.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KerberosCMWatcherCache.scala new file mode 100644 index 0000000000000..cb69a093b4ec9 --- /dev/null +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KerberosCMWatcherCache.scala @@ -0,0 +1,100 @@ +/* + * 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.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 + +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/KerberosDeployment.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KerberosDeployment.scala new file mode 100644 index 0000000000000..aa2e6635ff787 --- /dev/null +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/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 + +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/KerberosPVWatcherCache.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KerberosPVWatcherCache.scala index 47963f9f65c36..f6e349a6801b4 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KerberosPVWatcherCache.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KerberosPVWatcherCache.scala @@ -17,67 +17,163 @@ package org.apache.spark.deploy.kubernetes.integrationtest -private[spark] class KerberosPVWatcherCache() { -// client: KubernetesClient, -// dsNamespace: String, -// dsLabels: Map[String, String]) extends Logging { -// -// private var shufflePodCache = 2 -// private var watcher: Watch = _ -// -// def start(): Unit = { -// // seed the initial cache. -// val pvs = client.persistentVolumes().withLabels(dsLabels.asJava).list() -// pvs.getItems.asScala.foreach { -// pv => -// if (Readiness.isReady(pv)) { -// pvs.len -// } else { -// logWarning(s"Found unready shuffle pod ${pod.getMetadata.getName} " + -// s"on node ${pod.getSpec.getNodeName}") -// } -// } -// -// watcher = client -// .pods() -// .inNamespace(dsNamespace) -// .withLabels(dsLabels.asJava) -// .watch(new Watcher[Pod] { -// override def eventReceived(action: Watcher.Action, p: Pod): Unit = { -// action match { -// case Action.DELETED | Action.ERROR => -// shufflePodCache.remove(p.getSpec.getNodeName) -// case Action.ADDED | Action.MODIFIED if Readiness.isReady(p) => -// addShufflePodToCache(p) -// } -// } -// override def onClose(e: KubernetesClientException): Unit = {} -// }) -// } -// -// private def addShufflePodToCache(pod: Pod): Unit = { -// if (shufflePodCache.contains(pod.getSpec.getNodeName)) { -// val registeredPodName = shufflePodCache.get(pod.getSpec.getNodeName).get -// logError(s"Ambiguous specification of shuffle service pod. " + -// s"Found multiple matching pods: ${pod.getMetadata.getName}, " + -// s"${registeredPodName} on ${pod.getSpec.getNodeName}") -// -// throw new SparkException(s"Ambiguous specification of shuffle service pod. " + -// s"Found multiple matching pods: ${pod.getMetadata.getName}, " + -// s"${registeredPodName} on ${pod.getSpec.getNodeName}") -// } else { -// shufflePodCache(pod.getSpec.getNodeName) = pod.getStatus.getPodIP -// } -// } -// -// def stop(): Unit = { -// watcher.close() -// } -// -// def getShufflePodForExecutor(executorNode: String): String = { -// shufflePodCache.get(executorNode) -// .getOrElse(throw new SparkException(s"Unable to find shuffle pod on node $executorNode")) -// } +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 + +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 Volumes") + 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/KerberosPodWatcherCache.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KerberosPodWatcherCache.scala index bd33ffd6e34ed..70da043b12ea7 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KerberosPodWatcherCache.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KerberosPodWatcherCache.scala @@ -17,69 +17,207 @@ package org.apache.spark.deploy.kubernetes.integrationtest -private[spark] class KerberosPodWatcherCache() { +import java.util.concurrent.locks.{Condition, Lock, ReentrantLock} +import scala.collection.JavaConverters._ -// client: KubernetesClient, -// dsNamespace: String, -// dsLabels: Map[String, String]) extends Logging { -// -// private var shufflePodCache = scala.collection.mutable.Map[String, String]() -// private var watcher: Watch = _ -// -// def start(): Unit = { -// // seed the initial cache. -// val pods = client.pods() -// .inNamespace(dsNamespace).withLabels(dsLabels.asJava).list() -// pods.getItems.asScala.foreach { -// pod => -// if (Readiness.isReady(pod)) { -// addShufflePodToCache(pod) -// } else { -// logWarning(s"Found unready shuffle pod ${pod.getMetadata.getName} " + -// s"on node ${pod.getSpec.getNodeName}") -// } -// } -// -// watcher = client -// .pods() -// .inNamespace(dsNamespace) -// .withLabels(dsLabels.asJava) -// .watch(new Watcher[Pod] { -// override def eventReceived(action: Watcher.Action, p: Pod): Unit = { -// action match { -// case Action.DELETED | Action.ERROR => -// shufflePodCache.remove(p.getSpec.getNodeName) -// case Action.ADDED | Action.MODIFIED if Readiness.isReady(p) => -// addShufflePodToCache(p) -// } -// } -// override def onClose(e: KubernetesClientException): Unit = {} -// }) -// } -// -// private def addShufflePodToCache(pod: Pod): Unit = { -// if (shufflePodCache.contains(pod.getSpec.getNodeName)) { -// val registeredPodName = shufflePodCache.get(pod.getSpec.getNodeName).get -// logError(s"Ambiguous specification of shuffle service pod. " + -// s"Found multiple matching pods: ${pod.getMetadata.getName}, " + -// s"${registeredPodName} on ${pod.getSpec.getNodeName}") -// -// throw new SparkException(s"Ambiguous specification of shuffle service pod. " + -// s"Found multiple matching pods: ${pod.getMetadata.getName}, " + -// s"${registeredPodName} on ${pod.getSpec.getNodeName}") -// } else { -// shufflePodCache(pod.getSpec.getNodeName) = pod.getStatus.getPodIP -// } -// } -// -// def stop(): Unit = { -// watcher.close() -// } -// -// def getShufflePodForExecutor(executorNode: String): String = { -// shufflePodCache.get(executorNode) -// .getOrElse(throw new SparkException(s"Unable to find shuffle pod on node $executorNode")) -// } -} +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 + + +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 + 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 (maybeDeploymentAndServiceDone(name)) { + val modifyAndSignal: Runnable = new MSThread(name) + 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 == "kdc") 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 == "kdc") { + 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 (!dpIsUp) dnRunning.await() + dpIsUp = true + logInfo(s"data-populator has signaled") + try { + dpRunning.signalAll() + } finally { + lock.unlock() + } + + } + } + } +} diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KerberosStorage.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KerberosStorage.scala new file mode 100644 index 0000000000000..ad192a5cfb90f --- /dev/null +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/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 + +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/KerberosTestPodLauncher.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KerberosTestPodLauncher.scala index 705f1dd3646ac..ad2192c2c8810 100644 --- 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 @@ -18,63 +18,100 @@ 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 + + /** * Stuff */ private[spark] class KerberosTestPodLauncher( kubernetesClient: KubernetesClient, namespace: String) { - - private val yamlLocation = "kerberos-yml/kerberos-test.yml" - 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() - .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) + 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("CLASS_NAME") - .withValue(className) - .endEnv() - .addNewEnv() - .withName("HADOOP_CONF_DIR") - .withValue("hconf") - .endEnv() - .addNewEnv() - .withName("APP_LOCATOR_LABEL") - .withValue(appLabel) - .endEnv() - .addNewEnv() - .withName("SPARK_PRINT_LAUNCH_COMMAND") - .withValue("true") + .addNewEnv() + .withName("SUBMIT_RESOURCE") + .withValue(resource) .endEnv() - .endContainer() - .endSpec() - .endTemplate() - .endSpec() - .build() - kubernetesClient.extensions().deployments() - .inNamespace(namespace).create(deploymentWithEnv)} + .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/KerberosUtils.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KerberosUtils.scala new file mode 100644 index 0000000000000..6e9736f289647 --- /dev/null +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KerberosUtils.scala @@ -0,0 +1,126 @@ +/* + * 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._ +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 + +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) +} 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 97a9ee3824de2..d85acc20ff6b3 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 @@ -97,7 +97,6 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { launchKerberizedCluster() createKerberosTestPod(CONTAINER_LOCAL_MAIN_APP_RESOURCE, HDFS_TEST_CLASS, APP_LOCATOR_LABEL) val expectedLogOnCompletion = Seq("Something something something") - Thread.sleep(50000) // val driverPod = kubernetesTestComponents.kubernetesClient // .pods() // .withLabel("spark-app-locator", APP_LOCATOR_LABEL) diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/backend/minikube/MinikubeTestBackend.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/backend/minikube/MinikubeTestBackend.scala index 461264877edc2..9d0e09dc1e6e5 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/backend/minikube/MinikubeTestBackend.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/backend/minikube/MinikubeTestBackend.scala @@ -27,7 +27,7 @@ private[spark] class MinikubeTestBackend extends IntegrationTestBackend { override def initialize(): Unit = { Minikube.startMinikube() - new SparkDockerImageBuilder(Minikube.getDockerEnv).buildSparkDockerImages() + // new SparkDockerImageBuilder(Minikube.getDockerEnv).buildSparkDockerImages() defaultClient = Minikube.getKubernetesClient } 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 fabbbb1d68c03..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 @@ -88,15 +88,15 @@ private[spark] class SparkDockerImageBuilder if (exitCode != 0) { logInfo(s"exitCode: $exitCode") } -// buildImage("spark-base", BASE_DOCKER_FILE) -// buildImage("spark-driver", DRIVER_DOCKER_FILE) -// buildImage("spark-driver-py", DRIVERPY_DOCKER_FILE) -// buildImage("spark-executor", EXECUTOR_DOCKER_FILE) -// buildImage("spark-executor-py", EXECUTORPY_DOCKER_FILE) -// buildImage("spark-shuffle", SHUFFLE_SERVICE_DOCKER_FILE) -// 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("spark-base", BASE_DOCKER_FILE) + buildImage("spark-driver", DRIVER_DOCKER_FILE) + buildImage("spark-driver-py", DRIVERPY_DOCKER_FILE) + buildImage("spark-executor", EXECUTOR_DOCKER_FILE) + buildImage("spark-executor-py", EXECUTORPY_DOCKER_FILE) + buildImage("spark-shuffle", SHUFFLE_SERVICE_DOCKER_FILE) + 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) } diff --git a/resource-managers/kubernetes/integration-tests/test-data/hadoop-conf-files/core-site.xml b/resource-managers/kubernetes/integration-tests/test-data/hadoop-conf-files/core-site.xml deleted file mode 100644 index 2165d00052a72..0000000000000 --- a/resource-managers/kubernetes/integration-tests/test-data/hadoop-conf-files/core-site.xml +++ /dev/null @@ -1,34 +0,0 @@ - - - - - - - - - hadoop.security.authentication - kerberos - - - - hadoop.security.authorization - true - - - - fs.defaultFS - hdfs://nn.${NAMESPACE}.svc.cluster.local:9000 - - \ No newline at end of file diff --git a/resource-managers/kubernetes/integration-tests/test-data/hadoop-conf-files/hdfs-site.xml b/resource-managers/kubernetes/integration-tests/test-data/hadoop-conf-files/hdfs-site.xml deleted file mode 100644 index 8a00d75d7882d..0000000000000 --- a/resource-managers/kubernetes/integration-tests/test-data/hadoop-conf-files/hdfs-site.xml +++ /dev/null @@ -1,132 +0,0 @@ - - - - - - - - - - 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.${NAMESPACE}.svc.cluster.local@CLUSTER.LOCAL - - - dfs.namenode.kerberos.internal.spnego.principal - HTTP/nn.${NAMESPACE}.svc.cluster.local@CLUSTER.LOCAL - - - - - - dfs.namenode.delegation.token.max-lifetime - 18000000 - - - dfs.namenode.delegation.token.renew-interval - 1800000 - - - - - - - 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.${NAMESPACE}.svc.cluster.local@CLUSTER.LOCAL - - - dfs.namenode.kerberos.internal.spnego.principal - HTTP/nn.${NAMESPACE}.svc.cluster.local@CLUSTER.LOCAL - - - - - dfs.namenode.datanode.registration.ip-hostname-check - false - - - - - dfs.datanode.data.dir.perm - 700 - - - dfs.datanode.keytab.file - /var/keytabs/hdfs.keytab - - - dfs.datanode.kerberos.principal - hdfs/dn1.${NAMESPACE}.svc.cluster.local@CLUSTER.LOCAL - - - - - dfs.webhdfs.enabled - true - - - dfs.web.authentication.kerberos.principal - HTTP/dn1.${NAMESPACE}.svc.cluster.local@CLUSTER.LOCAL - - - dfs.web.authentication.kerberos.keytab - /var/keytabs/hdfs.keytab - - - diff --git a/resource-managers/kubernetes/integration-tests/test-data/hadoop-conf-files/yarn-site.xml b/resource-managers/kubernetes/integration-tests/test-data/hadoop-conf-files/yarn-site.xml deleted file mode 100644 index 92d9346232c76..0000000000000 --- a/resource-managers/kubernetes/integration-tests/test-data/hadoop-conf-files/yarn-site.xml +++ /dev/null @@ -1,26 +0,0 @@ - - - - - - - - - - - yarn.resourcemanager.principal - yarn/_HOST@CLUSTER.LOCAL - - \ No newline at end of file From 1ac9a194cef5f1f029d63acf474a5e88ff47dfa0 Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Mon, 31 Jul 2017 16:24:40 -0700 Subject: [PATCH 14/15] working Stage 2 --- .../HadoopKerberosKeytabResolverStep.scala | 51 ++++++---- .../data-populator-deployment.yml | 2 +- .../kerberos-yml/data-populator-service.yml | 2 +- .../kerberos-yml/dn1-deployment.yml | 2 +- .../kerberos-yml/dn1-service.yml | 2 +- .../kerberos-yml/kerberos-deployment.yml | 2 +- .../kerberos-yml/kerberos-service.yml | 2 +- .../kerberos-yml/kerberos-test.yml | 4 +- .../kerberos-yml/nn-deployment.yml | 2 +- .../kerberos-yml/nn-service.yml | 2 +- .../kerberos-yml/test-env.sh | 8 +- .../src/test/resources/hdfs-site.xml | 4 +- .../KerberizedHadoopClusterLauncher.scala | 2 + .../KerberosTestPodLauncher.scala | 2 - .../integrationtest/KubernetesSuite.scala | 43 +++++---- .../minikube/MinikubeTestBackend.scala | 2 +- .../KerberosCMWatcherCache.scala | 2 +- .../{ => kerberos}/KerberosDeployment.scala | 2 +- .../kerberos/KerberosDriverWatcherCache.scala | 95 +++++++++++++++++++ .../KerberosPVWatcherCache.scala | 5 +- .../KerberosPodWatcherCache.scala | 63 +++++++----- .../{ => kerberos}/KerberosStorage.scala | 2 +- .../{ => kerberos}/KerberosUtils.scala | 3 +- 23 files changed, 215 insertions(+), 89 deletions(-) rename resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/{ => kerberos}/KerberosCMWatcherCache.scala (98%) rename resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/{ => kerberos}/KerberosDeployment.scala (93%) create mode 100644 resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/kerberos/KerberosDriverWatcherCache.scala rename resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/{ => kerberos}/KerberosPVWatcherCache.scala (98%) rename resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/{ => kerberos}/KerberosPodWatcherCache.scala (85%) rename resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/{ => kerberos}/KerberosStorage.scala (93%) rename resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/{ => kerberos}/KerberosUtils.scala (98%) 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 index 8614bf1d6cace..69f1090228da4 100644 --- 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 @@ -17,6 +17,7 @@ package org.apache.spark.deploy.kubernetes.submit.submitsteps.hadoopsteps import java.io._ +import java.security.PrivilegedExceptionAction import scala.collection.JavaConverters._ import scala.util.Try @@ -30,6 +31,7 @@ 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.SparkHadoopUtil import org.apache.spark.deploy.kubernetes.{KerberosConfBootstrapImpl, PodWithMainContainer} import org.apache.spark.deploy.kubernetes.constants._ @@ -44,9 +46,12 @@ private[spark] class HadoopKerberosKeytabResolverStep( submissionSparkConf: SparkConf, maybePrincipal: Option[String], maybeKeytab: Option[File]) extends HadoopConfigurationStep with Logging{ - - override def configureContainers(hadoopConfigSpec: HadoopConfigSpec): HadoopConfigSpec = { - // FIXME: Pass down hadoopConf so you can call sc.hadoopConfiguration + 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") @@ -66,26 +71,30 @@ private[spark] class HadoopKerberosKeytabResolverStep( } // In the case that keytab is not specified we will read from Local Ticket Cache val jobUserUGI = maybeJobUserUGI.getOrElse(UserGroupInformation.getCurrentUser) - logInfo(s"Retrieved Job User UGI: $jobUserUGI") - val originalCredentials: Credentials = jobUserUGI.getCredentials - logInfo(s"Original tokens: ${originalCredentials.toString}") - logInfo(s"All tokens: ${originalCredentials.getAllTokens}") - logInfo(s"All secret keys: ${originalCredentials.getAllSecretKeys}") - val dfs: FileSystem = FileSystem.get(hadoopConf) - // This is not necessary with [Spark-20328] since we would be using - // Spark core providers to handle delegation token renewal - val renewer: String = jobUserUGI.getShortUserName - logInfo(s"Renewer is: $renewer") - val renewedCredentials: Credentials = new Credentials(originalCredentials) - dfs.addDelegationTokens(renewer, renewedCredentials) - val renewedTokens = renewedCredentials.getAllTokens.asScala - logInfo(s"Renewed tokens: ${renewedCredentials.toString}") - logInfo(s"All renewed tokens: ${renewedTokens}") - logInfo(s"All renewed secret keys: ${renewedCredentials.getAllSecretKeys}") + // 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 renewalTime = getTokenRenewalInterval(renewedTokens, hadoopConf).getOrElse(Long.MaxValue) val delegationToken = HDFSDelegationToken(data, renewalTime) val initialTokenLabelName = s"$KERBEROS_SECRET_LABEL_PREFIX-1-$renewalTime" logInfo(s"Storing dt in $initialTokenLabelName") 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 index 87826f9e9fc02..b87f8ceafe6cf 100644 --- a/resource-managers/kubernetes/integration-tests/kerberos-yml/data-populator-deployment.yml +++ b/resource-managers/kubernetes/integration-tests/kerberos-yml/data-populator-deployment.yml @@ -11,7 +11,7 @@ spec: labels: name: hdfs-data-populator kerberosService: data-populator - job: kerberos-test + job: kerberostest spec: containers: - command: 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 index 366ca23657c56..45b4bfb4bfeb5 100644 --- a/resource-managers/kubernetes/integration-tests/kerberos-yml/data-populator-service.yml +++ b/resource-managers/kubernetes/integration-tests/kerberos-yml/data-populator-service.yml @@ -5,7 +5,7 @@ metadata: service.alpha.kubernetes.io/tolerate-unready-endpoints: "true" labels: kerberosService: data-populator - job: kerberos-test + job: kerberostest name: data-populator spec: clusterIP: None diff --git a/resource-managers/kubernetes/integration-tests/kerberos-yml/dn1-deployment.yml b/resource-managers/kubernetes/integration-tests/kerberos-yml/dn1-deployment.yml index 73873ff860342..9f028d31e4424 100644 --- a/resource-managers/kubernetes/integration-tests/kerberos-yml/dn1-deployment.yml +++ b/resource-managers/kubernetes/integration-tests/kerberos-yml/dn1-deployment.yml @@ -11,7 +11,7 @@ spec: labels: name: hdfs-dn1 kerberosService: dn1 - job: kerberos-test + job: kerberostest spec: containers: - command: diff --git a/resource-managers/kubernetes/integration-tests/kerberos-yml/dn1-service.yml b/resource-managers/kubernetes/integration-tests/kerberos-yml/dn1-service.yml index 9afe0c1c14e12..ee258c87942f8 100644 --- a/resource-managers/kubernetes/integration-tests/kerberos-yml/dn1-service.yml +++ b/resource-managers/kubernetes/integration-tests/kerberos-yml/dn1-service.yml @@ -5,7 +5,7 @@ metadata: service.alpha.kubernetes.io/tolerate-unready-endpoints: "true" labels: kerberosService: dn1 - job: kerberos-test + job: kerberostest name: dn1 spec: clusterIP: None diff --git a/resource-managers/kubernetes/integration-tests/kerberos-yml/kerberos-deployment.yml b/resource-managers/kubernetes/integration-tests/kerberos-yml/kerberos-deployment.yml index a2a977ca66a48..6037a3c50b11e 100644 --- a/resource-managers/kubernetes/integration-tests/kerberos-yml/kerberos-deployment.yml +++ b/resource-managers/kubernetes/integration-tests/kerberos-yml/kerberos-deployment.yml @@ -11,7 +11,7 @@ spec: labels: name: hdfs-kerberos kerberosService: kerberos - job: kerberos-test + job: kerberostest spec: containers: - command: diff --git a/resource-managers/kubernetes/integration-tests/kerberos-yml/kerberos-service.yml b/resource-managers/kubernetes/integration-tests/kerberos-yml/kerberos-service.yml index 0fb6e31cd8d6c..da7b994f6e2af 100644 --- a/resource-managers/kubernetes/integration-tests/kerberos-yml/kerberos-service.yml +++ b/resource-managers/kubernetes/integration-tests/kerberos-yml/kerberos-service.yml @@ -5,7 +5,7 @@ metadata: service.alpha.kubernetes.io/tolerate-unready-endpoints: "true" labels: kerberosService: kerberos - job: kerberos-test + job: kerberostest name: kerberos spec: clusterIP: None diff --git a/resource-managers/kubernetes/integration-tests/kerberos-yml/kerberos-test.yml b/resource-managers/kubernetes/integration-tests/kerberos-yml/kerberos-test.yml index 349ec784016e9..9115b45cc0135 100644 --- a/resource-managers/kubernetes/integration-tests/kerberos-yml/kerberos-test.yml +++ b/resource-managers/kubernetes/integration-tests/kerberos-yml/kerberos-test.yml @@ -10,7 +10,9 @@ spec: name: kerberos-test spec: containers: - - name: kerberos-test + - command: ["/bin/bash"] + args: ["/opt/spark/test-env.sh"] + name: kerberos-test image: kerberos-test:latest imagePullPolicy: IfNotPresent volumeMounts: diff --git a/resource-managers/kubernetes/integration-tests/kerberos-yml/nn-deployment.yml b/resource-managers/kubernetes/integration-tests/kerberos-yml/nn-deployment.yml index 2907f2e356258..d2f473aa66c1a 100644 --- a/resource-managers/kubernetes/integration-tests/kerberos-yml/nn-deployment.yml +++ b/resource-managers/kubernetes/integration-tests/kerberos-yml/nn-deployment.yml @@ -11,7 +11,7 @@ spec: labels: name: hdfs-nn kerberosService: nn - job: kerberos-test + job: kerberostest spec: containers: - command: diff --git a/resource-managers/kubernetes/integration-tests/kerberos-yml/nn-service.yml b/resource-managers/kubernetes/integration-tests/kerberos-yml/nn-service.yml index bf85e12a96df4..649302150aa39 100644 --- a/resource-managers/kubernetes/integration-tests/kerberos-yml/nn-service.yml +++ b/resource-managers/kubernetes/integration-tests/kerberos-yml/nn-service.yml @@ -5,7 +5,7 @@ metadata: service.alpha.kubernetes.io/tolerate-unready-endpoints: "true" labels: kerberosService: nn - job: kerberos-test + job: kerberostest name: nn spec: clusterIP: None diff --git a/resource-managers/kubernetes/integration-tests/kerberos-yml/test-env.sh b/resource-managers/kubernetes/integration-tests/kerberos-yml/test-env.sh index e070655bfa47d..fca6fd01b7052 100644 --- a/resource-managers/kubernetes/integration-tests/kerberos-yml/test-env.sh +++ b/resource-managers/kubernetes/integration-tests/kerberos-yml/test-env.sh @@ -1,11 +1,13 @@ #!/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 -/usr/bin/kinit -kt /var/keytabs/hdfs.keytab hdfs/nn.${NAMESPACE}.svc.cluster.local - +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} \ @@ -20,6 +22,6 @@ mkdir -p /etc/krb5.conf.d --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.labels=spark-app-locator=${APP_LOCATOR_LABEL} \ + --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/src/test/resources/hdfs-site.xml b/resource-managers/kubernetes/integration-tests/src/test/resources/hdfs-site.xml index 7233af3c4b951..bf77244d22567 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/resources/hdfs-site.xml +++ b/resource-managers/kubernetes/integration-tests/src/test/resources/hdfs-site.xml @@ -55,11 +55,11 @@ dfs.namenode.delegation.token.max-lifetime - 18000000 + 3600000 dfs.namenode.delegation.token.renew-interval - 1800000 + 3600000 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 index d8fad52414ad5..9c1be3e9a521e 100644 --- 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 @@ -18,6 +18,8 @@ package org.apache.spark.deploy.kubernetes.integrationtest import io.fabric8.kubernetes.client.KubernetesClient +import org.apache.spark.deploy.kubernetes.integrationtest.kerberos._ + /** * Stuff */ 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 index ad2192c2c8810..cad74f50c9860 100644 --- 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 @@ -26,8 +26,6 @@ import io.fabric8.kubernetes.client.KubernetesClient import org.apache.spark.deploy.kubernetes.submit.ContainerNameEqualityPredicate - - /** * Stuff */ 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 d85acc20ff6b3..02b1402b8c8ca 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 @@ -20,16 +20,13 @@ import java.nio.file.Paths import java.util.UUID import io.fabric8.kubernetes.client.internal.readiness.Readiness - -import org.apache.spark.{SparkConf, SSLOptions, SparkFunSuite} - +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.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} @@ -96,22 +93,28 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { assume(testBackend.name == MINIKUBE_TEST_BACKEND) 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 = kubernetesTestComponents.kubernetesClient -// .pods() -// .withLabel("spark-app-locator", APP_LOCATOR_LABEL) -// .list() -// .getItems -// .get(0) -// Eventually.eventually(TIMEOUT, INTERVAL) { -// expectedLogOnCompletion.foreach { e => -// assert(kubernetesTestComponents.kubernetesClient -// .pods() -// .withName(driverPod.getMetadata.getName) -// .getLog -// .contains(e), "The application did not complete.") -// } -// } + val driverPod = kubernetesClient + .pods() + .withLabel("spark-app-locator", APP_LOCATOR_LABEL) + .list() + .getItems + .get(0) + Eventually.eventually(TIMEOUT, INTERVAL) { + expectedLogOnCompletion.foreach { e => + assert(kubernetesClient + .pods() + .withName(driverPod.getMetadata.getName) + .getLog + .contains(e), "The application did not complete.") + } + } } // test("Run PySpark Job on file from SUBMITTER with --py-files") { diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/backend/minikube/MinikubeTestBackend.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/backend/minikube/MinikubeTestBackend.scala index 9d0e09dc1e6e5..461264877edc2 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/backend/minikube/MinikubeTestBackend.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/backend/minikube/MinikubeTestBackend.scala @@ -27,7 +27,7 @@ private[spark] class MinikubeTestBackend extends IntegrationTestBackend { override def initialize(): Unit = { Minikube.startMinikube() - // new SparkDockerImageBuilder(Minikube.getDockerEnv).buildSparkDockerImages() + new SparkDockerImageBuilder(Minikube.getDockerEnv).buildSparkDockerImages() defaultClient = Minikube.getKubernetesClient } diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KerberosCMWatcherCache.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/kerberos/KerberosCMWatcherCache.scala similarity index 98% rename from resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KerberosCMWatcherCache.scala rename to resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/kerberos/KerberosCMWatcherCache.scala index cb69a093b4ec9..7382ebf3a45c2 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KerberosCMWatcherCache.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/kerberos/KerberosCMWatcherCache.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.integrationtest +package org.apache.spark.deploy.kubernetes.integrationtest.kerberos import java.util.concurrent.locks.{Condition, Lock, ReentrantLock} diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KerberosDeployment.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/kerberos/KerberosDeployment.scala similarity index 93% rename from resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KerberosDeployment.scala rename to resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/kerberos/KerberosDeployment.scala index aa2e6635ff787..1e1db004fbb7b 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KerberosDeployment.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/kerberos/KerberosDeployment.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.integrationtest +package org.apache.spark.deploy.kubernetes.integrationtest.kerberos import io.fabric8.kubernetes.api.model.Service import io.fabric8.kubernetes.api.model.extensions.Deployment 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..b921291c3fec0 --- /dev/null +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/kerberos/KerberosDriverWatcherCache.scala @@ -0,0 +1,95 @@ +/* + * 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 + +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/KerberosPVWatcherCache.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/kerberos/KerberosPVWatcherCache.scala similarity index 98% rename from resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KerberosPVWatcherCache.scala rename to resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/kerberos/KerberosPVWatcherCache.scala index f6e349a6801b4..2f2d0aa154043 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KerberosPVWatcherCache.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/kerberos/KerberosPVWatcherCache.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.integrationtest +package org.apache.spark.deploy.kubernetes.integrationtest.kerberos import java.util.concurrent.locks.{Condition, Lock, ReentrantLock} @@ -27,6 +27,7 @@ import io.fabric8.kubernetes.client.Watcher.Action import org.apache.spark.internal.Logging + private[spark] class KerberosPVWatcherCache( kerberosUtils: KerberosUtils, labels: Map[String, String]) extends Logging { @@ -91,7 +92,7 @@ private[spark] class KerberosPVWatcherCache( .withLabels(labels.asJava) .watch(new Watcher[PersistentVolumeClaim] { override def onClose(cause: KubernetesClientException): Unit = - logInfo("Ending the watch of Persistent Volumes") + logInfo("Ending the watch of Persistent Volume Claims") override def eventReceived( action: Watcher.Action, resource: PersistentVolumeClaim): Unit = { diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KerberosPodWatcherCache.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/kerberos/KerberosPodWatcherCache.scala similarity index 85% rename from resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KerberosPodWatcherCache.scala rename to resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/kerberos/KerberosPodWatcherCache.scala index 70da043b12ea7..6f6664a1674a9 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KerberosPodWatcherCache.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/kerberos/KerberosPodWatcherCache.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.integrationtest +package org.apache.spark.deploy.kubernetes.integrationtest.kerberos import java.util.concurrent.locks.{Condition, Lock, ReentrantLock} @@ -36,7 +36,7 @@ private[spark] class KerberosPodWatcherCache( private var podWatcher: Watch = _ private var serviceWatcher: Watch = _ private var podCache = - scala.collection.mutable.Map[String, String]() + scala.collection.mutable.Map[String, String]() private var serviceCache = scala.collection.mutable.Map[String, String]() private var lock: Lock = new ReentrantLock() @@ -52,22 +52,24 @@ private[spark] class KerberosPodWatcherCache( 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() - } - } - }) + 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") @@ -79,16 +81,17 @@ private[spark] class KerberosPodWatcherCache( 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(name) + podCache.remove(keyName) case Action.ADDED | Action.MODIFIED => val phase = resource.getStatus.getPhase logInfo(s"$name is as $phase") - podCache(name) = phase - if (maybeDeploymentAndServiceDone(name)) { - val modifyAndSignal: Runnable = new MSThread(name) + podCache(keyName) = phase + if (maybeDeploymentAndServiceDone(keyName)) { + val modifyAndSignal: Runnable = new MSThread(keyName) new Thread(modifyAndSignal).start() }}}}) }}) @@ -123,6 +126,7 @@ private[spark] class KerberosPodWatcherCache( deploy(kerberosUtils.getKDC) } }}) + def start(): Unit = { blockingThread.start() podWatcherThread.start() @@ -131,6 +135,7 @@ private[spark] class KerberosPodWatcherCache( podWatcherThread.join() serviceWatcherThread.join() } + def stop(): Unit = { podWatcher.close() serviceWatcher.close() @@ -141,7 +146,7 @@ private[spark] class KerberosPodWatcherCache( serviceCache.get(name).contains(name) if (!finished) { logInfo(s"$name is not up with a service") - if (name == "kdc") kdcIsUp = false + if (name == "kerberos") kdcIsUp = false else if (name == "nn") nnIsUp = false else if (name == "dn1") dnIsUp = false else if (name == "data-populator") dpIsUp = false @@ -160,7 +165,7 @@ private[spark] class KerberosPodWatcherCache( override def run(): Unit = { logInfo(s"$name Node and Service is up") lock.lock() - if (name == "kdc") { + if (name == "kerberos") { kdcIsUp = true logInfo(s"kdc has signaled") try { @@ -208,7 +213,7 @@ private[spark] class KerberosPodWatcherCache( else if (name == "data-populator") { while (!kdcIsUp) kdcRunning.await() while (!nnIsUp) nnRunning.await() - while (!dpIsUp) dnRunning.await() + while (!dnIsUp) dnRunning.await() dpIsUp = true logInfo(s"data-populator has signaled") try { @@ -216,8 +221,16 @@ private[spark] class KerberosPodWatcherCache( } 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/KerberosStorage.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/kerberos/KerberosStorage.scala similarity index 93% rename from resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KerberosStorage.scala rename to resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/kerberos/KerberosStorage.scala index ad192a5cfb90f..16284fd49c270 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KerberosStorage.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/kerberos/KerberosStorage.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.integrationtest +package org.apache.spark.deploy.kubernetes.integrationtest.kerberos import io.fabric8.kubernetes.api.model.{PersistentVolume, PersistentVolumeClaim} diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KerberosUtils.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/kerberos/KerberosUtils.scala similarity index 98% rename from resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KerberosUtils.scala rename to resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/kerberos/KerberosUtils.scala index 6e9736f289647..4eb91b4d4fa12 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KerberosUtils.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/kerberos/KerberosUtils.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.integrationtest +package org.apache.spark.deploy.kubernetes.integrationtest.kerberos import java.io.{File, FileInputStream} @@ -27,6 +27,7 @@ import org.apache.commons.io.FileUtils.readFileToString import org.apache.spark.deploy.kubernetes.submit.ContainerNameEqualityPredicate + private[spark] class KerberosUtils( kubernetesClient: KubernetesClient, namespace: String) { From 32ff0de9486d2ab5795d34a1ddfe1412e1a94b87 Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Tue, 1 Aug 2017 16:19:54 -0700 Subject: [PATCH 15/15] documentation --- docs/running-on-kubernetes.md | 47 +++++++++++++++++++ .../kubernetes/HadoopConfBootstrap.scala | 2 +- .../kubernetes/KerberosConfBootstrap.scala | 10 +++- .../kubernetes/PodWithMainContainer.scala | 5 ++ .../spark/deploy/kubernetes/config.scala | 15 ++++-- .../spark/deploy/kubernetes/constants.scala | 3 ++ .../deploy/kubernetes/submit/Client.scala | 2 + .../HadoopConfigBootstrapStep.scala | 6 ++- .../hadoopsteps/HDFSDelegationToken.scala | 4 ++ .../hadoopsteps/HadoopConfMounterStep.scala | 6 ++- .../hadoopsteps/HadoopConfigSpec.scala | 2 +- .../hadoopsteps/HadoopConfigurationStep.scala | 2 +- .../HadoopKerberosKeytabResolverStep.scala | 19 +++++--- .../HadoopKerberosSecretResolverStep.scala | 6 ++- .../hadoopsteps/HadoopStepsOrchestrator.scala | 7 ++- .../KerberizedHadoopClusterLauncher.scala | 11 ++++- .../KerberosTestPodLauncher.scala | 5 +- .../integrationtest/KubernetesSuite.scala | 26 +++++----- .../kerberos/KerberosCMWatcherCache.scala | 4 ++ .../kerberos/KerberosDriverWatcherCache.scala | 4 ++ .../kerberos/KerberosPVWatcherCache.scala | 6 ++- .../kerberos/KerberosPodWatcherCache.scala | 6 ++- .../kerberos/KerberosUtils.scala | 4 +- 23 files changed, 164 insertions(+), 38 deletions(-) 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 index 733fbeffe45b7..54b59d235860a 100644 --- 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 @@ -28,7 +28,7 @@ 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-conf for executors as well. + * set up the Hadoop Configuration for executors as well. */ private[spark] trait HadoopConfBootstrap { /** 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 index e0c1b28f05046..2aa772ae7b04f 100644 --- 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 @@ -19,12 +19,18 @@ 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( 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 index 4f182c250fcf4..664eb41bd68d5 100644 --- 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 @@ -18,6 +18,11 @@ 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 9e397befdba4e..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 @@ -498,6 +498,7 @@ package object config extends Logging { .createOptional 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") @@ -518,14 +519,22 @@ package object config extends Logging { .stringConf .createOptional - private[spark] val KUBERNETES_KERBEROS_DT_SECRET = - ConfigBuilder("spark.kubernetes.kerberos.tokensecret") - .doc("Specify the label of the secret where " + + 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 389297a5cec55..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,12 +91,15 @@ 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 = 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 7a30bbe50790f..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 @@ -198,6 +198,8 @@ private[spark] object Client { def main(args: Array[String]): Unit = { val parsedArguments = ClientArguments.fromCommandLineArgs(args) val sparkConf = new SparkConf() + // hadoopConfDir is passed into Client#run() to allow for us to + // test this env variable within the integration test environment val hadoopConfDir = sys.env.get("HADOOP_CONF_DIR") run(sparkConf, parsedArguments, hadoopConfDir) } 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 index f6dedf827a2d8..894e73e4286c7 100644 --- 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 @@ -23,9 +23,11 @@ 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} - /** - * Configures the driverSpec that bootstraps dependencies into the driver pod. + * 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], 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 index 308bea183d141..4f6f0953c5340 100644 --- 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 @@ -16,4 +16,8 @@ */ 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 index a1f399033afdb..e3b7f674d0225 100644 --- 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 @@ -24,7 +24,11 @@ import org.apache.spark.deploy.kubernetes.{HadoopConfBootstrap, PodWithMainConta import org.apache.spark.deploy.kubernetes.constants._ /** - * Step that configures the ConfigMap + Volumes for the driver + * 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, 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 index 7f399311737bc..b69b18c11470f 100644 --- 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 @@ -28,7 +28,7 @@ import io.fabric8.kubernetes.api.model.{Container, Pod, Secret} * 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 re-built + * - The secret containing a DT, either previously specified or built on the fly */ private[spark] case class HadoopConfigSpec( additionalDriverSparkConf: Map[String, String], diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopConfigurationStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopConfigurationStep.scala index 2b5aca1aadddc..54f46b9a17fc9 100644 --- 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 @@ -17,7 +17,7 @@ package org.apache.spark.deploy.kubernetes.submit.submitsteps.hadoopsteps /** - * Represents a step in preparing the driver + * Represents a step in preparing the driver with Hadoop Configuration logic. */ private[spark] trait HadoopConfigurationStep { 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 index 69f1090228da4..7aec765e00f17 100644 --- 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 @@ -31,16 +31,22 @@ 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.SparkHadoopUtil 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 - - /** - * Step that configures the ConfigMap + Volumes for the driver + * 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, @@ -96,7 +102,8 @@ private[spark] class HadoopKerberosKeytabResolverStep( val data = serialize(renewedCredentials) val renewalTime = getTokenRenewalInterval(renewedTokens, hadoopConf).getOrElse(Long.MaxValue) val delegationToken = HDFSDelegationToken(data, renewalTime) - val initialTokenLabelName = s"$KERBEROS_SECRET_LABEL_PREFIX-1-$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() 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 index 9406204988403..4613436fe414c 100644 --- 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 @@ -20,10 +20,14 @@ import org.apache.spark.SparkConf import org.apache.spark.deploy.kubernetes.{KerberosConfBootstrapImpl, PodWithMainContainer} /** - * Step that configures the ConfigMap + Volumes for the driver + * 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 = { 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 index 032ac27939813..b43b99f9b1a06 100644 --- 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 @@ -35,7 +35,9 @@ private[spark] class HadoopStepsOrchestrator( 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) + 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]) @@ -52,7 +54,8 @@ private[spark] class HadoopStepsOrchestrator( if (maybeKerberosSupport) { maybeExistingSecret.map(secretLabel => Some(new HadoopKerberosSecretResolverStep( submissionSparkConf, - secretLabel))).getOrElse(Some( + secretLabel, + maybeExistingSecretLabel))).getOrElse(Some( new HadoopKerberosKeytabResolverStep( submissionSparkConf, maybePrincipal, 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 index 9c1be3e9a521e..ce8f456d3e11d 100644 --- 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 @@ -21,7 +21,11 @@ import io.fabric8.kubernetes.client.KubernetesClient import org.apache.spark.deploy.kubernetes.integrationtest.kerberos._ /** - * Stuff + * 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, @@ -29,13 +33,18 @@ private[spark] class KerberizedHadoopClusterLauncher( 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() 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 index cad74f50c9860..532fc221878ff 100644 --- 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 @@ -27,7 +27,10 @@ import io.fabric8.kubernetes.client.KubernetesClient import org.apache.spark.deploy.kubernetes.submit.ContainerNameEqualityPredicate /** - * Stuff + * 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, 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 02b1402b8c8ca..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 @@ -73,21 +73,21 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { after { kubernetesTestComponents.deleteKubernetesResources() - // kubernetesTestComponents.deleteNamespace() + kubernetesTestComponents.deleteNamespace() } -// test("Include HADOOP_CONF for HDFS based jobs") { -// 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)) -// runSparkApplicationAndVerifyCompletion( -// JavaMainAppResource(CONTAINER_LOCAL_MAIN_APP_RESOURCE), -// SPARK_PI_MAIN_CLASS, -// Seq("HADOOP_CONF_DIR defined. Mounting HDFS specific .xml files", "Pi is roughly 3"), -// Array("5"), -// Seq.empty[String], -// Some("src/test/resources")) -// } + test("Include HADOOP_CONF for HDFS based jobs") { + 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)) + runSparkApplicationAndVerifyCompletion( + JavaMainAppResource(CONTAINER_LOCAL_MAIN_APP_RESOURCE), + SPARK_PI_MAIN_CLASS, + Seq("HADOOP_CONF_DIR defined. Mounting HDFS specific .xml files", "Pi is roughly 3"), + Array("5"), + Seq.empty[String], + Some("src/test/resources")) + } test("Secure HDFS test with HDFS keytab") { assume(testBackend.name == MINIKUBE_TEST_BACKEND) 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 index 7382ebf3a45c2..59968534c8312 100644 --- 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 @@ -27,6 +27,10 @@ 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 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 index b921291c3fec0..b7a2176194e2c 100644 --- 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 @@ -27,6 +27,10 @@ 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 { 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 index 2f2d0aa154043..fff02c2d06ef7 100644 --- 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 @@ -27,7 +27,11 @@ 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 { 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 index 6f6664a1674a9..ee7aeeaa9c7dc 100644 --- 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 @@ -27,7 +27,11 @@ 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 { 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 index 4eb91b4d4fa12..9bb06d88ef608 100644 --- 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 @@ -27,7 +27,9 @@ 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) {