From ea9e516148e83696fa82a80c648f0dba47cdca16 Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Fri, 14 Jul 2017 18:37:49 -0700 Subject: [PATCH 01/37] 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 | 30 +++++++ .../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, 342 insertions(+), 4 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 e746410772f62..abe87cc70d975 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 @@ -510,6 +510,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 f39a3fc2f4bb5..e1a7067e41bf9 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 @@ -94,6 +94,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 06f1547e6a5b1..79269a0d34cb1 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,10 +16,13 @@ */ 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.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 @@ -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 { @@ -95,6 +99,22 @@ private[spark] class DriverConfigurationStepsOrchestrator( val kubernetesCredentialsStep = new DriverKubernetesCredentialsStep( submissionSparkConf, kubernetesResourceNamePrefix) val hadoopCredentialsStep = new DriverHadoopCredentialsStep(submissionSparkConf) + 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)) @@ -135,6 +155,16 @@ private[spark] class DriverConfigurationStepsOrchestrator( hadoopCredentialsStep, 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 47ea3072c09c4763859018f7946922258c01e3c2 Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Fri, 14 Jul 2017 19:30:38 -0700 Subject: [PATCH 02/37] 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 79269a0d34cb1..757854121a248 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 60a19caa46a93f6853a8e2c5d190d90a3f612d21 Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Mon, 17 Jul 2017 17:14:01 -0700 Subject: [PATCH 03/37] 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 | 18 ++++++++++-- .../integrationtest/KubernetesSuite.scala | 6 ++++ 10 files changed, 91 insertions(+), 13 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 e1a7067e41bf9..44243c9c9924e 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,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 757854121a248..ed4226f77e964 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 @@ -113,7 +113,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 f6aecc402c0fe..4a5ad52c3291e 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.{HadoopSecretUtil, 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) { @@ -430,6 +431,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" @@ -584,6 +586,7 @@ private[spark] class KubernetesClusterSchedulerBackend( val executorPodWithNodeAffinity = addNodeAffinityAnnotationIfUseful( executorPodWithInitContainer, nodeToLocalTaskCount) + // CHANGES val executorPodWithMountedHadoopToken = HadoopSecretUtil.configurePod(maybeMountedHadoopSecret, executorPodWithNodeAffinity) val containerWithMountedHadoopToken = HadoopSecretUtil.configureContainer( @@ -592,10 +595,21 @@ private[spark] class KubernetesClusterSchedulerBackend( val resolvedExecutorPod = new PodBuilder(executorPodWithMountedHadoopToken) .editSpec() .addToContainers(containerWithMountedHadoopToken) + // CHANGES + val (executorHadoopConfPod, executorHadoopConfContainer) = + executorHadoopBootStrap.map { bootstrap => + val podWithMainContainer = bootstrap.bootstrapMainContainerAndVolumes( + PodWithMainContainer(executorPodWithNodeAffinity, initBootstrappedExecutorContainer) + ) + (podWithMainContainer.pod, podWithMainContainer.mainContainer) + }.getOrElse(executorPodWithNodeAffinity, initBootstrappedExecutorContainer) + val resolvedExecutorPod2 = new PodBuilder(executorHadoopConfPod) + .editSpec() + .addToContainers(executorHadoopConfContainer) .endSpec() .build() try { - (executorId, kubernetesClient.pods.create(resolvedExecutorPod)) + (executorId, kubernetesClient.pods.create(resolvedExecutorPod2)) } catch { case throwable: Throwable => logError("Failed to allocate executor pod.", throwable) 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 1d0175a8ecaafbf10a223f5bab900251435d7aab Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Mon, 17 Jul 2017 17:32:54 -0700 Subject: [PATCH 04/37] 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 ed4226f77e964..76062c71b6ec3 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 @@ -108,12 +108,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 163193a5c1cec83b2e1046487848ea2d7d611200 Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Mon, 17 Jul 2017 18:25:39 -0700 Subject: [PATCH 05/37] 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 8381fa616c9caba61d64639f14f9a587e16a1d71 Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Mon, 17 Jul 2017 19:12:03 -0700 Subject: [PATCH 06/37] 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 d4b1a68f5e8fcb8fa937ace691f130c58667875e Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Mon, 17 Jul 2017 19:53:31 -0700 Subject: [PATCH 07/37] 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 0bba0923c2315bf532c93ddd3ff52abba7385db6 Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Tue, 18 Jul 2017 14:26:04 -0700 Subject: [PATCH 08/37] Included integration tests for Stage 1 --- .../spark/deploy/kubernetes/constants.scala | 10 ++-- .../deploy/kubernetes/submit/Client.scala | 8 ++- ...DriverConfigurationStepsOrchestrator.scala | 10 +++- .../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, 64 insertions(+), 44 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 44243c9c9924e..8bc1446db7518 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 @@ -94,15 +94,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 76062c71b6ec3..92bcbadb8e6e9 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,10 +99,14 @@ private[spark] class DriverConfigurationStepsOrchestrator( submissionSparkConf) val kubernetesCredentialsStep = new DriverKubernetesCredentialsStep( submissionSparkConf, kubernetesResourceNamePrefix) + // CHANGES val hadoopCredentialsStep = new DriverHadoopCredentialsStep(submissionSparkConf) - val hadoopConfigurations = + val hadoopConfigurations2 = sys.env.get("HADOOP_CONF_DIR").map{ conf => getHadoopConfFiles(conf)} .getOrElse(Array.empty[File]) + // CHANGES + val hadoopConfigurations = hadoopConfDir.map(conf => getHadoopConfFiles(conf)) + .getOrElse(Array.empty[File]) val hadoopConfigSteps = if (hadoopConfigurations.isEmpty) { Option.empty[DriverConfigurationStep] @@ -110,7 +115,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 06df96225eeffd527ce141f9445db3ccadad0361 Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Tue, 18 Jul 2017 19:09:16 -0700 Subject: [PATCH 09/37] 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 abe87cc70d975..d9307e666e413 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 @@ -517,6 +517,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 d7f54ddcda3e2fe9138c6bf43bbca11b4f86a39c Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Fri, 21 Jul 2017 15:44:25 -0700 Subject: [PATCH 10/37] 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 | 12 +- .../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 | 14 +- 17 files changed, 313 insertions(+), 100 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 d9307e666e413..408f4e7dd39e7 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 @@ -512,8 +512,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) @@ -531,6 +530,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 8bc1446db7518..5a3ab74137b3f 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 @@ -105,6 +105,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 92bcbadb8e6e9..714a5ec89d53d 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 @@ -108,14 +108,13 @@ private[spark] class DriverConfigurationStepsOrchestrator( 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() @@ -164,13 +163,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 4a5ad52c3291e..5a14ea160d2c7 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) { @@ -431,7 +432,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" @@ -603,9 +603,17 @@ private[spark] class KubernetesClusterSchedulerBackend( ) (podWithMainContainer.pod, podWithMainContainer.mainContainer) }.getOrElse(executorPodWithNodeAffinity, initBootstrappedExecutorContainer) - val resolvedExecutorPod2 = new PodBuilder(executorHadoopConfPod) + 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 resolvedExecutorPod2 = new PodBuilder(executorKerberosPod) .editSpec() - .addToContainers(executorHadoopConfContainer) + .addToContainers(executorKerberosContainer) .endSpec() .build() try { From d3c5a0363dd94c1df2cafce3184625bb97cfe903 Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Wed, 26 Jul 2017 10:40:56 -0700 Subject: [PATCH 11/37] 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 d7441ba50686c9b525fc6676c7c9442b69c9e01e Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Thu, 27 Jul 2017 21:34:45 -0700 Subject: [PATCH 12/37] 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 04eed68a8ce5cba2b525616d00feb1e2f4c5a9ed Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Sat, 29 Jul 2017 03:05:39 -0700 Subject: [PATCH 13/37] 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 62354eb2c3fb6d6d6c1711da5c7426007ea8740a Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Mon, 31 Jul 2017 16:24:40 -0700 Subject: [PATCH 14/37] 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 514ac19b6bcbf25ef76be6a2d0cb694b6d9a6bd5 Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Tue, 1 Aug 2017 16:19:54 -0700 Subject: [PATCH 15/37] 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 408f4e7dd39e7..0262357c2e5ff 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 @@ -510,6 +510,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") @@ -530,14 +531,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 5a3ab74137b3f..92e8cc5eb49e6 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 @@ -99,12 +99,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) { From 3fbf88c679dc33e29b4e3c25803995bbc3555196 Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Tue, 1 Aug 2017 20:47:13 -0700 Subject: [PATCH 16/37] Integration Stages 1,2 and 3 --- .../kubernetes/KerberosConfBootstrap.scala | 48 ------------- .../KerberosTokenConfBootstrap.scala | 72 +++++++++++++++++++ .../spark/deploy/kubernetes/constants.scala | 11 ++- ...DriverConfigurationStepsOrchestrator.scala | 11 --- .../kubernetes/submit/HadoopSecretUtil.scala | 55 -------------- .../DriverHadoopCredentialsStep.scala | 38 ---------- .../HadoopConfigBootstrapStep.scala | 4 +- .../hadoopsteps/HDFSDelegationToken.scala | 23 ------ .../hadoopsteps/HadoopConfigSpec.scala | 6 +- .../HadoopKerberosKeytabResolverStep.scala | 19 ++--- .../HadoopKerberosSecretResolverStep.scala | 16 ++++- .../kubernetes/KubernetesClusterManager.scala | 10 ++- .../KubernetesClusterSchedulerBackend.scala | 22 ++---- .../src/test/resources/hdfs-site.xml | 6 +- .../integrationtest/KubernetesSuite.scala | 26 +++---- .../docker/SparkDockerImageBuilder.scala | 27 ++++--- .../kerberos/KerberosPodWatcherCache.scala | 14 ++++ 17 files changed, 165 insertions(+), 243 deletions(-) delete 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/KerberosTokenConfBootstrap.scala delete mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/HadoopSecretUtil.scala delete mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DriverHadoopCredentialsStep.scala delete mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HDFSDelegationToken.scala 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 deleted file mode 100644 index 2aa772ae7b04f..0000000000000 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/KerberosConfBootstrap.scala +++ /dev/null @@ -1,48 +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 - -import io.fabric8.kubernetes.api.model.ContainerBuilder - -import org.apache.spark.deploy.kubernetes.constants._ - /** - * This is separated out from hadoopsteps because this component can be reused to - * set up the Kerberos logic for executors as well. - */ -private[spark] trait KerberosConfBootstrap { - /** - * Bootstraps a main container with an ENV variable - * pointing to the data storing the DT in the secret - */ - def bootstrapMainContainerAndVolumes(originalPodWithMainContainer: PodWithMainContainer) - : PodWithMainContainer -} -private[spark] class KerberosConfBootstrapImpl( - delegationTokenLabelName: String) extends KerberosConfBootstrap{ - override def bootstrapMainContainerAndVolumes( - originalPodWithMainContainer: PodWithMainContainer) - : PodWithMainContainer = { - val mainContainerWithMountedHadoopConf = new ContainerBuilder( - originalPodWithMainContainer.mainContainer) - .addNewEnv() - .withName(ENV_KERBEROS_SECRET_LABEL) - .withValue(delegationTokenLabelName) - .endEnv() - .build() - originalPodWithMainContainer.copy(mainContainer = mainContainerWithMountedHadoopConf) - } -} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/KerberosTokenConfBootstrap.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/KerberosTokenConfBootstrap.scala new file mode 100644 index 0000000000000..cb25eb9c0a1e1 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/KerberosTokenConfBootstrap.scala @@ -0,0 +1,72 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.kubernetes + +import io.fabric8.kubernetes.api.model.{ContainerBuilder, PodBuilder} + +import org.apache.spark.deploy.kubernetes.constants._ +import org.apache.spark.internal.Logging + + + /** + * This is separated out from the HadoopConf steps API because this component can be reused to + * mounted the DT secret for executors as well. + */ +private[spark] trait KerberosTokenBootstrapConf { + /** + * Bootstraps a main container with the Secret mounted as volumes and an ENV variable + * pointing to the mounted file containing the DT for Secure HDFS interaction + */ + def bootstrapMainContainerAndVolumes( + originalPodWithMainContainer: PodWithMainContainer) + : PodWithMainContainer +} + +private[spark] class KerberosTokenConfBootstrapImpl( + secretName: String, + secretLabel: String) extends KerberosTokenBootstrapConf with Logging{ + + override def bootstrapMainContainerAndVolumes( + originalPodWithMainContainer: PodWithMainContainer) + : PodWithMainContainer = { + logInfo("Mounting HDFS DT from Secret for Secure HDFS") + val dtMountedPod = new PodBuilder(originalPodWithMainContainer.pod) + .editOrNewSpec() + .addNewVolume() + .withName(SPARK_APP_HADOOP_SECRET_VOLUME_NAME) + .withNewSecret() + .withSecretName(secretName) + .endSecret() + .endVolume() + .endSpec() + .build() + val mainContainerWithMountedKerberos = new ContainerBuilder( + originalPodWithMainContainer.mainContainer) + .addNewVolumeMount() + .withName(SPARK_APP_HADOOP_SECRET_VOLUME_NAME) + .withMountPath(SPARK_APP_HADOOP_CREDENTIALS_BASE_DIR) + .endVolumeMount() + .addNewEnv() + .withName(ENV_HADOOP_TOKEN_FILE_LOCATION) + .withValue(s"$SPARK_APP_HADOOP_CREDENTIALS_BASE_DIR/$secretLabel") + .endEnv() + .build() + originalPodWithMainContainer.copy( + pod = dtMountedPod, + mainContainer = mainContainerWithMountedKerberos) + } +} 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 92e8cc5eb49e6..25a023074fdc7 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/constants.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/constants.scala @@ -45,9 +45,6 @@ package object constants { // Hadoop credentials secrets for the Spark app. private[spark] val SPARK_APP_HADOOP_CREDENTIALS_BASE_DIR = "/mnt/secrets/hadoop-credentials" - private[spark] val SPARK_APP_HADOOP_TOKEN_FILE_SECRET_NAME = "hadoop-token-file" - private[spark] val SPARK_APP_HADOOP_TOKEN_FILE_PATH = - s"$SPARK_APP_HADOOP_CREDENTIALS_BASE_DIR/$SPARK_APP_HADOOP_TOKEN_FILE_SECRET_NAME" private[spark] val SPARK_APP_HADOOP_SECRET_VOLUME_NAME = "hadoop-secret" // Default and fixed ports @@ -110,12 +107,12 @@ package object constants { // Kerberos Configuration private[spark] val HADOOP_KERBEROS_SECRET_NAME = "spark.kubernetes.kerberos.dt" - private[spark] val KERBEROS_SPARK_CONF_NAME = - "spark.kubernetes.kerberos.secretlabelname" + private[spark] val HADOOP_KERBEROS_CONF_SECRET = + "spark.kubernetes.kerberos.secretname" + private[spark] val HADOOP_KERBEROS_CONF_LABEL = + "spark.kubernetes.kerberos.labelname" 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/DriverConfigurationStepsOrchestrator.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/DriverConfigurationStepsOrchestrator.scala index 714a5ec89d53d..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,14 +97,6 @@ private[spark] class DriverConfigurationStepsOrchestrator( submissionSparkConf) val kubernetesCredentialsStep = new DriverKubernetesCredentialsStep( submissionSparkConf, kubernetesResourceNamePrefix) - // CHANGES - val hadoopCredentialsStep = new DriverHadoopCredentialsStep(submissionSparkConf) - val hadoopConfigurations2 = - sys.env.get("HADOOP_CONF_DIR").map{ conf => getHadoopConfFiles(conf)} - .getOrElse(Array.empty[File]) - // CHANGES - val hadoopConfigurations = hadoopConfDir.map(conf => getHadoopConfFiles(conf)) - .getOrElse(Array.empty[File]) val hadoopConfigSteps = if (hadoopConfDir.isEmpty) { Option.empty[DriverConfigurationStep] @@ -157,7 +147,6 @@ private[spark] class DriverConfigurationStepsOrchestrator( Seq( initialSubmissionStep, kubernetesCredentialsStep, - hadoopCredentialsStep, dependencyResolutionStep) ++ initContainerBootstrapStep.toSeq ++ hadoopConfigSteps.toSeq ++ diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/HadoopSecretUtil.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/HadoopSecretUtil.scala deleted file mode 100644 index b7a9e9e3fd6cd..0000000000000 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/HadoopSecretUtil.scala +++ /dev/null @@ -1,55 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.spark.deploy.kubernetes.submit - -import io.fabric8.kubernetes.api.model.{Container, ContainerBuilder, Pod, PodBuilder} - -import org.apache.spark.deploy.kubernetes.constants._ - -object HadoopSecretUtil { - - def configurePod(secretNameOption: Option[String], pod: Pod) : Pod = { - secretNameOption.map { secret => - new PodBuilder(pod) - .editOrNewSpec() - .addNewVolume() - .withName(SPARK_APP_HADOOP_SECRET_VOLUME_NAME) - .withNewSecret() - .withSecretName(secret) - .endSecret() - .endVolume() - .endSpec() - .build() - }.getOrElse(pod) - } - - def configureContainer(secretNameOption: Option[String], - containerSpec: Container) : Container = { - secretNameOption.map { secret => - new ContainerBuilder(containerSpec) - .addNewVolumeMount() - .withName(SPARK_APP_HADOOP_SECRET_VOLUME_NAME) - .withMountPath(SPARK_APP_HADOOP_CREDENTIALS_BASE_DIR) - .endVolumeMount() - .addNewEnv() - .withName(ENV_HADOOP_TOKEN_FILE_LOCATION) - .withValue(SPARK_APP_HADOOP_TOKEN_FILE_PATH) - .endEnv() - .build() - }.getOrElse(containerSpec) - } -} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DriverHadoopCredentialsStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DriverHadoopCredentialsStep.scala deleted file mode 100644 index 88f2e7b3836c8..0000000000000 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DriverHadoopCredentialsStep.scala +++ /dev/null @@ -1,38 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.spark.deploy.kubernetes.submit.submitsteps - -import org.apache.spark.SparkConf -import org.apache.spark.deploy.kubernetes.config._ -import org.apache.spark.deploy.kubernetes.submit.HadoopSecretUtil - -private[spark] class DriverHadoopCredentialsStep(submissionSparkConf: SparkConf) - extends DriverConfigurationStep { - - private val maybeMountedHadoopSecret = submissionSparkConf.get(MOUNTED_HADOOP_SECRET_CONF) - - override def configureDriver(driverSpec: KubernetesDriverSpec): KubernetesDriverSpec = { - val podWithMountedHadoopToken = HadoopSecretUtil.configurePod(maybeMountedHadoopSecret, - driverSpec.driverPod) - val containerWithMountedHadoopToken = HadoopSecretUtil.configureContainer( - maybeMountedHadoopSecret, - driverSpec.driverContainer) - driverSpec.copy( - driverPod = podWithMountedHadoopToken, - driverContainer = containerWithMountedHadoopToken) - } -} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/HadoopConfigBootstrapStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/HadoopConfigBootstrapStep.scala index 894e73e4286c7..315dad9c5a591 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 @@ -40,7 +40,9 @@ private[spark] class HadoopConfigBootstrapStep( driverContainer = driverSpec.driverContainer, configMapProperties = Map.empty[String, String], additionalDriverSparkConf = Map.empty[String, String], - dtSecret = None) + dtSecret = None, + dtSecretName = HADOOP_KERBEROS_SECRET_NAME, + dtSecretLabel = "") for (nextStep <- hadoopConfigurationSteps) { currentHadoopSpec = nextStep.configureContainers(currentHadoopSpec) } 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 deleted file mode 100644 index 4f6f0953c5340..0000000000000 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HDFSDelegationToken.scala +++ /dev/null @@ -1,23 +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 - - /** - * 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/HadoopConfigSpec.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopConfigSpec.scala index b69b18c11470f..1ccce0f9ce66a 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 @@ -29,10 +29,14 @@ import io.fabric8.kubernetes.api.model.{Container, Pod, Secret} * - The properties that will be stored into the config map which have (key, value) * pairs of (path, data) * - The secret containing a DT, either previously specified or built on the fly + * - The name of the secret where the DT will be stored + * - The label on the secret which correlates with where the current DT data is stored */ private[spark] case class HadoopConfigSpec( additionalDriverSparkConf: Map[String, String], driverPod: Pod, driverContainer: Container, configMapProperties: Map[String, String], - dtSecret: Option[Secret]) + dtSecret: Option[Secret], + dtSecretName: String, + dtSecretLabel: String) 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 7aec765e00f17..2f57ffe18f063 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,9 +31,9 @@ import org.apache.hadoop.security.token.{Token, TokenIdentifier} import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenIdentifier import org.apache.spark.SparkConf -import org.apache.spark.deploy.kubernetes.{KerberosConfBootstrapImpl, PodWithMainContainer} -import org.apache.spark.deploy.kubernetes.constants._ import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.deploy.kubernetes.{KerberosTokenConfBootstrapImpl, PodWithMainContainer} +import org.apache.spark.deploy.kubernetes.constants._ import org.apache.spark.internal.Logging /** @@ -101,18 +101,18 @@ private[spark] class HadoopKerberosKeytabResolverStep( if (renewedTokens.isEmpty) logError("Did not obtain any Delegation Tokens") val data = serialize(renewedCredentials) val renewalTime = getTokenRenewalInterval(renewedTokens, hadoopConf).getOrElse(Long.MaxValue) - val delegationToken = HDFSDelegationToken(data, renewalTime) val currentTime: Long = System.currentTimeMillis() val initialTokenLabelName = s"$KERBEROS_SECRET_LABEL_PREFIX-$currentTime-$renewalTime" - logInfo(s"Storing dt in $initialTokenLabelName") val secretDT = new SecretBuilder() .withNewMetadata() .withName(HADOOP_KERBEROS_SECRET_NAME) .endMetadata() - .addToData(initialTokenLabelName, Base64.encodeBase64String(delegationToken.bytes)) + .addToData(initialTokenLabelName, Base64.encodeBase64String(data)) .build() - val bootstrapKerberos = new KerberosConfBootstrapImpl(initialTokenLabelName) + val bootstrapKerberos = new KerberosTokenConfBootstrapImpl( + HADOOP_KERBEROS_SECRET_NAME, + initialTokenLabelName) val withKerberosEnvPod = bootstrapKerberos.bootstrapMainContainerAndVolumes( PodWithMainContainer( hadoopConfigSpec.driverPod, @@ -120,10 +120,13 @@ private[spark] class HadoopKerberosKeytabResolverStep( hadoopConfigSpec.copy( additionalDriverSparkConf = hadoopConfigSpec.additionalDriverSparkConf ++ Map( - KERBEROS_SPARK_CONF_NAME -> initialTokenLabelName), + HADOOP_KERBEROS_CONF_LABEL -> initialTokenLabelName, + HADOOP_KERBEROS_CONF_SECRET -> HADOOP_KERBEROS_SECRET_NAME), driverPod = withKerberosEnvPod.pod, driverContainer = withKerberosEnvPod.mainContainer, - dtSecret = Some(secretDT)) + dtSecret = Some(secretDT), + dtSecretName = HADOOP_KERBEROS_SECRET_NAME, + dtSecretLabel = initialTokenLabelName) } // Functions that should be in Core with Rebase to 2.3 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 4613436fe414c..1bd8dd84bcb2b 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 @@ -17,7 +17,8 @@ package org.apache.spark.deploy.kubernetes.submit.submitsteps.hadoopsteps import org.apache.spark.SparkConf -import org.apache.spark.deploy.kubernetes.{KerberosConfBootstrapImpl, PodWithMainContainer} +import org.apache.spark.deploy.kubernetes.{KerberosTokenConfBootstrapImpl, PodWithMainContainer} +import org.apache.spark.deploy.kubernetes.constants._ /** * This step assumes that you have already done all the heavy lifting in retrieving a @@ -31,13 +32,22 @@ private[spark] class HadoopKerberosSecretResolverStep( tokenLabelName: String) extends HadoopConfigurationStep { override def configureContainers(hadoopConfigSpec: HadoopConfigSpec): HadoopConfigSpec = { - val bootstrapKerberos = new KerberosConfBootstrapImpl(tokenLabelName) + val bootstrapKerberos = new KerberosTokenConfBootstrapImpl( + tokenSecretName, + tokenLabelName) val withKerberosEnvPod = bootstrapKerberos.bootstrapMainContainerAndVolumes( PodWithMainContainer( hadoopConfigSpec.driverPod, hadoopConfigSpec.driverContainer)) hadoopConfigSpec.copy( driverPod = withKerberosEnvPod.pod, - driverContainer = withKerberosEnvPod.mainContainer) + driverContainer = withKerberosEnvPod.mainContainer, + additionalDriverSparkConf = + hadoopConfigSpec.additionalDriverSparkConf ++ Map( + HADOOP_KERBEROS_CONF_LABEL -> tokenLabelName, + HADOOP_KERBEROS_CONF_SECRET -> tokenSecretName), + dtSecret = None, + dtSecretName = tokenSecretName, + dtSecretLabel = tokenLabelName) } } 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 ff348cf06b431..5cef4d5de984d 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,7 +43,8 @@ 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 maybeDTSecretName = sparkConf.getOption(HADOOP_KERBEROS_CONF_SECRET) + val maybeDTLabelName = sparkConf.getOption(HADOOP_KERBEROS_CONF_LABEL) val maybeExecutorInitContainerSecretName = sparkConf.get(EXECUTOR_INIT_CONTAINER_SECRET) @@ -85,9 +86,12 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit ) } val kerberosBootstrap = for { - dTSecret <- maybeDTSecret + secretName <- maybeDTSecretName + secretLabel <- maybeDTLabelName } yield { - new KerberosConfBootstrapImpl(dTSecret) + new KerberosTokenConfBootstrapImpl( + secretName, + secretLabel) } if (maybeConfigMap.isEmpty) { logWarning("The executor's init-container config map was not specified. Executors will" + 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 5a14ea160d2c7..d890f9a4e994e 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 @@ -37,7 +37,7 @@ import org.apache.spark.{SparkContext, SparkEnv, SparkException} import org.apache.spark.deploy.kubernetes._ import org.apache.spark.deploy.kubernetes.config._ import org.apache.spark.deploy.kubernetes.constants._ -import org.apache.spark.deploy.kubernetes.submit.{HadoopSecretUtil, InitContainerUtil} +import org.apache.spark.deploy.kubernetes.submit.InitContainerUtil import org.apache.spark.network.netty.SparkTransportConf import org.apache.spark.network.shuffle.kubernetes.KubernetesExternalShuffleClient import org.apache.spark.rpc.{RpcAddress, RpcCallContext, RpcEndpointAddress, RpcEnv} @@ -51,7 +51,7 @@ private[spark] class KubernetesClusterSchedulerBackend( val sc: SparkContext, executorInitContainerBootstrap: Option[SparkPodInitContainerBootstrap], executorHadoopBootStrap: Option[HadoopConfBootstrap], - executorKerberosBootStrap: Option[KerberosConfBootstrap], + executorKerberosBootStrap: Option[KerberosTokenBootstrapConf], executorMountInitContainerSecretPlugin: Option[InitContainerResourceStagingServerSecretPlugin], kubernetesClient: KubernetesClient) extends CoarseGrainedSchedulerBackend(scheduler, sc.env.rpcEnv) { @@ -586,16 +586,6 @@ private[spark] class KubernetesClusterSchedulerBackend( val executorPodWithNodeAffinity = addNodeAffinityAnnotationIfUseful( executorPodWithInitContainer, nodeToLocalTaskCount) - // CHANGES - val executorPodWithMountedHadoopToken = HadoopSecretUtil.configurePod(maybeMountedHadoopSecret, - executorPodWithNodeAffinity) - val containerWithMountedHadoopToken = HadoopSecretUtil.configureContainer( - maybeMountedHadoopSecret, initBootstrappedExecutorContainer) - - val resolvedExecutorPod = new PodBuilder(executorPodWithMountedHadoopToken) - .editSpec() - .addToContainers(containerWithMountedHadoopToken) - // CHANGES val (executorHadoopConfPod, executorHadoopConfContainer) = executorHadoopBootStrap.map { bootstrap => val podWithMainContainer = bootstrap.bootstrapMainContainerAndVolumes( @@ -603,21 +593,19 @@ 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(executorHadoopConfPod, executorHadoopConfContainer)) (podWithMainContainer.pod, podWithMainContainer.mainContainer) }.getOrElse((executorHadoopConfPod, executorHadoopConfContainer)) - val resolvedExecutorPod2 = new PodBuilder(executorKerberosPod) + val resolvedExecutorPod = new PodBuilder(executorKerberosPod) .editSpec() .addToContainers(executorKerberosContainer) .endSpec() .build() try { - (executorId, kubernetesClient.pods.create(resolvedExecutorPod2)) + (executorId, kubernetesClient.pods.create(resolvedExecutorPod)) } catch { case throwable: Throwable => logError("Failed to allocate executor pod.", throwable) 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 bf77244d22567..8995b16ec4d71 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 @@ -67,7 +67,7 @@ dfs.data.transfer.protection - integrity + authentication @@ -118,6 +118,10 @@ dfs.datanode.name.dir file:///hadoop/etc/data + + dfs.data.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/KubernetesSuite.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KubernetesSuite.scala index 3c01f17501285..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 @@ -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/docker/SparkDockerImageBuilder.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/docker/SparkDockerImageBuilder.scala index 84f38e482f037..d596368c7155a 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,19 +20,18 @@ 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 scala.collection.JavaConverters._ - - - private[spark] class SparkDockerImageBuilder (private val dockerEnv: Map[String, String]) extends Logging{ @@ -88,16 +87,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("kerberos-test", KERBEROS_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/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 ee7aeeaa9c7dc..f12b6a2bb5a33 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 @@ -56,6 +56,7 @@ private[spark] class KerberosPodWatcherCache( private var nnSpawned: Boolean = false private var dnSpawned: Boolean = false private var dpSpawned: Boolean = false + private var dnName: String = _ private val blockingThread = new Thread(new Runnable { override def run(): Unit = { @@ -93,6 +94,7 @@ private[spark] class KerberosPodWatcherCache( case Action.ADDED | Action.MODIFIED => val phase = resource.getStatus.getPhase logInfo(s"$name is as $phase") + if (name.startsWith("dn1")) { dnName = name } podCache(keyName) = phase if (maybeDeploymentAndServiceDone(keyName)) { val modifyAndSignal: Runnable = new MSThread(keyName) @@ -218,8 +220,14 @@ private[spark] class KerberosPodWatcherCache( while (!kdcIsUp) kdcRunning.await() while (!nnIsUp) nnRunning.await() while (!dnIsUp) dnRunning.await() + while (!hasInLogs(dnName, "Computing capacity for map BlockMap")) { + logInfo("Waiting on DN to be formatted") + Thread.sleep(500) + } + Thread.sleep(2000) dpIsUp = true logInfo(s"data-populator has signaled") + try { dpRunning.signalAll() } finally { @@ -237,4 +245,10 @@ private[spark] class KerberosPodWatcherCache( case _ if name.startsWith("data-populator") => "data-populator" } } + private def hasInLogs(name: String, expectation: String): Boolean = { + kubernetesClient + .pods() + .withName(name) + .getLog().contains(expectation) + } } From b321436bbf129a12bbe9b9132973a3542b540263 Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Wed, 2 Aug 2017 15:06:31 -0700 Subject: [PATCH 17/37] further testing work --- .../KerberosTokenConfBootstrap.scala | 6 +- .../integrationtest/jobs/HDFSTest.scala | 1 + .../src/test/resources/hdfs-site.xml | 2 - .../src/test/resources/krb5.conf | 2 +- .../integrationtest/KubernetesSuite.scala | 354 +++++++++--------- .../docker/SparkDockerImageBuilder.scala | 20 +- .../kerberos/KerberosPodWatcherCache.scala | 9 +- .../kerberos/KerberosUtils.scala | 13 +- 8 files changed, 206 insertions(+), 201 deletions(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/KerberosTokenConfBootstrap.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/KerberosTokenConfBootstrap.scala index cb25eb9c0a1e1..891be079d8401 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/KerberosTokenConfBootstrap.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/KerberosTokenConfBootstrap.scala @@ -27,10 +27,8 @@ import org.apache.spark.internal.Logging * mounted the DT secret for executors as well. */ private[spark] trait KerberosTokenBootstrapConf { - /** - * Bootstraps a main container with the Secret mounted as volumes and an ENV variable - * pointing to the mounted file containing the DT for Secure HDFS interaction - */ + // Bootstraps a main container with the Secret mounted as volumes and an ENV variable + // pointing to the mounted file containing the DT for Secure HDFS interaction def bootstrapMainContainerAndVolumes( originalPodWithMainContainer: PodWithMainContainer) : PodWithMainContainer 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 index 3ed77980d8502..63edec60f35ce 100644 --- 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 @@ -40,6 +40,7 @@ private[spark] object HDFSTest{ val end = System.currentTimeMillis() println("Iteration " + iter + " took " + (end-start) + " ms") } + println(s"Returned length(s) of: ${file.map(s => s.length).collect().mkString(",")}") // scalastyle:on println spark.stop() } diff --git a/resource-managers/kubernetes/integration-tests/src/test/resources/hdfs-site.xml b/resource-managers/kubernetes/integration-tests/src/test/resources/hdfs-site.xml index 8995b16ec4d71..1dea0f4662fe1 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 @@ -104,8 +104,6 @@ dfs.namenode.datanode.registration.ip-hostname-check false - - dfs.datanode.data.dir.perm 700 diff --git a/resource-managers/kubernetes/integration-tests/src/test/resources/krb5.conf b/resource-managers/kubernetes/integration-tests/src/test/resources/krb5.conf index 5c189a09be6c4..144f77d8995df 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/resources/krb5.conf +++ b/resource-managers/kubernetes/integration-tests/src/test/resources/krb5.conf @@ -12,7 +12,7 @@ renew_lifetime = 7d forwardable = true rdns = false default_realm = CLUSTER.LOCAL -# default_ccache_name = KEYRING:persistent:%{uid} +# default_ccache_name = MEMORY [realms] CLUSTER.LOCAL = { 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..4478def82cda4 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) @@ -99,7 +99,7 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { Map("spark-app-locator" -> APP_LOCATOR_LABEL)) driverWatcherCache.start() driverWatcherCache.stop() - val expectedLogOnCompletion = Seq("Something something something") + val expectedLogOnCompletion = Seq("Returned length(s) of:") val driverPod = kubernetesClient .pods() .withLabel("spark-app-locator", APP_LOCATOR_LABEL) @@ -117,167 +117,167 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { } } -// 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 = { @@ -415,8 +415,8 @@ private[spark] object KubernetesSuite { s"integration-tests-jars/${EXAMPLES_JAR_FILE.getName}" val CONTAINER_LOCAL_HELPER_JAR_PATH = s"local:///opt/spark/examples/" + s"integration-tests-jars/${HELPER_JAR_FILE.getName}" - val TIMEOUT = PatienceConfiguration.Timeout(Span(20, Minutes)) - val INTERVAL = PatienceConfiguration.Interval(Span(20, Seconds)) + val TIMEOUT = PatienceConfiguration.Timeout(Span(15, Minutes)) + val INTERVAL = PatienceConfiguration.Interval(Span(15, Seconds)) val SPARK_PI_MAIN_CLASS = "org.apache.spark.deploy.kubernetes" + ".integrationtest.jobs.SparkPiWithInfiniteWait" val PYSPARK_PI_MAIN_CLASS = "org.apache.spark.deploy.PythonRunner" 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 d596368c7155a..ce1f4d45ec4a6 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 @@ -87,16 +87,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("kerberos-test", KERBEROS_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/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 f12b6a2bb5a33..e212dac4777a1 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 @@ -28,9 +28,9 @@ 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 + * This class is used to ensure that the Hadoop cluster that is launched is executed * in this order: KDC --> NN --> DN --> Data-Populator and that each one of these nodes - * is running before launching the kerberos test. + * is running before launching the Kerberos test. */ private[spark] class KerberosPodWatcherCache( kerberosUtils: KerberosUtils, @@ -220,14 +220,12 @@ private[spark] class KerberosPodWatcherCache( while (!kdcIsUp) kdcRunning.await() while (!nnIsUp) nnRunning.await() while (!dnIsUp) dnRunning.await() - while (!hasInLogs(dnName, "Computing capacity for map BlockMap")) { + while (!hasInLogs(dnName, "Got finalize command for block pool")) { logInfo("Waiting on DN to be formatted") Thread.sleep(500) } - Thread.sleep(2000) dpIsUp = true logInfo(s"data-populator has signaled") - try { dpRunning.signalAll() } finally { @@ -245,6 +243,7 @@ private[spark] class KerberosPodWatcherCache( case _ if name.startsWith("data-populator") => "data-populator" } } + private def hasInLogs(name: String, expectation: String): Boolean = { kubernetesClient .pods() 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 9bb06d88ef608..b436256cf8199 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 @@ -39,16 +39,21 @@ private[spark] class KerberosUtils( def loadFromYaml(resource: String): FileInputStream = new FileInputStream(new File(yamlLocation(resource))) private val regex = "REPLACE_ME".r + private val regexDP = "# default_ccache_name = MEMORY".r + private val defaultCacheDP = "default_ccache_name = KRBCONF" private def locationResolver(loc: String) = s"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))} + (file, regex.replaceAllIn(readFileToString(new File(locationResolver(file))), namespace))} ++ + Seq(("krb5-dp.conf", regexDP.replaceAllIn(regex.replaceAllIn(readFileToString( + new File(locationResolver("krb5.conf"))), namespace), defaultCacheDP))) private val KRB_VOLUME = "krb5-conf" private val KRB_FILE_DIR = "/tmp" private val KRB_CONFIG_MAP_NAME = "krb-config-map" private val PV_LABELS = Map("job" -> "kerberostest") - private val keyPaths: Seq[KeyToPath] = kerberosFiles.map(file => + private val keyPaths: Seq[KeyToPath] = (kerberosFiles ++ Seq("krb5-dp.conf")) + .map(file => new KeyToPathBuilder() .withKey(file) .withPath(file) @@ -102,6 +107,10 @@ private[spark] class KerberosUtils( .withName("TMP_KRB_LOC") .withValue(s"$KRB_FILE_DIR/${kerberosFiles.head}") .endEnv() + .addNewEnv() + .withName("TMP_KRB_DP_LOC") + .withValue(s"$KRB_FILE_DIR/krb5-dp.conf") + .endEnv() .addNewEnv() .withName("TMP_CORE_LOC") .withValue(s"$KRB_FILE_DIR/${kerberosFiles(1)}") From b6912d255bf8385db01651e36d8188b5dc801c4c Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Wed, 2 Aug 2017 16:19:18 -0700 Subject: [PATCH 18/37] fixing imports --- .../integrationtest/KubernetesSuite.scala | 20 ++++++++++++------- 1 file changed, 13 insertions(+), 7 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 4478def82cda4..a5051842744ac 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,22 +16,28 @@ */ 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.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._ + +import org.apache.spark.{SparkConf, SparkFunSuite, SSLOptions} +import org.apache.spark.deploy.kubernetes.SSLUtils 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} - -import scala.collection.JavaConverters._ +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 private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { import KubernetesSuite._ From c6b11f86839ab754a9014960eaa4b909b98e6c88 Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Thu, 3 Aug 2017 15:17:07 -0700 Subject: [PATCH 19/37] Stage 3 Integration tests pass --- .../KubernetesClusterSchedulerBackend.scala | 2 - .../integrationtest/jobs/HDFSTest.scala | 1 + .../kerberos-yml/test-env.sh | 1 + .../src/test/resources/core-site.xml | 4 ++ .../src/test/resources/hdfs-site.xml | 15 +++++++- .../integrationtest/KubernetesSuite.scala | 6 ++- .../KubernetesTestComponents.scala | 37 ++++++++----------- .../docker/SparkDockerImageBuilder.scala | 20 +++++----- 8 files changed, 48 insertions(+), 38 deletions(-) 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 d890f9a4e994e..085886dff51c0 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 @@ -132,8 +132,6 @@ private[spark] class KubernetesClusterSchedulerBackend( private implicit val requestExecutorContext = ExecutionContext.fromExecutorService( ThreadUtils.newDaemonCachedThreadPool("kubernetes-executor-requests")) - private val maybeMountedHadoopSecret = conf.get(MOUNTED_HADOOP_SECRET_CONF) - private val driverPod = try { kubernetesClient.pods().inNamespace(kubernetesNamespace). withName(kubernetesDriverPodName).get() 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 index 63edec60f35ce..463dd25d14a1f 100644 --- 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 @@ -40,6 +40,7 @@ private[spark] object HDFSTest{ val end = System.currentTimeMillis() println("Iteration " + iter + " took " + (end-start) + " ms") } + println(s"File contents: ${file.map(s => s.toString).collect().mkString(",")}") println(s"Returned length(s) of: ${file.map(s => s.length).collect().mkString(",")}") // scalastyle:on println spark.stop() diff --git a/resource-managers/kubernetes/integration-tests/kerberos-yml/test-env.sh b/resource-managers/kubernetes/integration-tests/kerberos-yml/test-env.sh index fca6fd01b7052..079f2e37936b4 100644 --- a/resource-managers/kubernetes/integration-tests/kerberos-yml/test-env.sh +++ b/resource-managers/kubernetes/integration-tests/kerberos-yml/test-env.sh @@ -19,6 +19,7 @@ until /usr/bin/kinit -kt /var/keytabs/hdfs.keytab hdfs/nn.${NAMESPACE}.svc.clust --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.hadoop.dfs.data.transfer.protection=authentication \ --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 \ 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 2d61ad740385a..9e168812052ea 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 @@ -31,4 +31,8 @@ fs.defaultFS hdfs://nn.REPLACE_ME.svc.cluster.local:9000 + + hadoop.rpc.protection + authentication + \ No newline at end of file diff --git a/resource-managers/kubernetes/integration-tests/src/test/resources/hdfs-site.xml b/resource-managers/kubernetes/integration-tests/src/test/resources/hdfs-site.xml index 1dea0f4662fe1..66dc969c46b63 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 @@ -67,9 +67,8 @@ dfs.data.transfer.protection - authentication + integrity - dfs.datanode.address 0.0.0.0:10019 @@ -128,6 +127,18 @@ dfs.datanode.kerberos.principal hdfs/dn1.REPLACE_ME.svc.cluster.local@CLUSTER.LOCAL + + dfs.encrypt.data.transfer + true + + + dfs.encrypt.data.transfer.cipher.suites + AES/CTR/NoPadding + + + dfs.encrypt.data.transfer.cipher.key.bitlength + 256 + 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 a5051842744ac..e509c457c114d 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 @@ -78,7 +78,7 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { } after { - kubernetesTestComponents.deleteKubernetesResources() + kubernetesTestComponents.deleteKubernetesPVs() kubernetesTestComponents.deleteNamespace() } @@ -105,7 +105,9 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { Map("spark-app-locator" -> APP_LOCATOR_LABEL)) driverWatcherCache.start() driverWatcherCache.stop() - val expectedLogOnCompletion = Seq("Returned length(s) of:") + val expectedLogOnCompletion = Seq( + "Returned length(s) of: 1", + "File contents: [This is an awesome word count file]") val driverPod = kubernetesClient .pods() .withLabel("spark-app-locator", APP_LOCATOR_LABEL) 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 a0693f2f46dd4..9e245039cd1ee 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 @@ -39,7 +39,21 @@ private[spark] class KubernetesTestComponents(defaultClient: DefaultKubernetesCl .endMetadata() .done() } - + def deleteKubernetesPVs(): Unit = { + // Temporary hack until client library for fabric8 is updated to get around + // the NPE that comes about when I do .list().getItems().asScala + try { + val pvList = kubernetesClient.persistentVolumes().list().getItems().asScala + if (pvList.nonEmpty) { + kubernetesClient.persistentVolumes().delete() + Eventually.eventually(KubernetesSuite.TIMEOUT, KubernetesSuite.INTERVAL) { + require(!pvList.exists(_.getMetadata.getNamespace == namespace)) + } + } + } catch { + case ex: java.lang.NullPointerException => + } + } def deleteNamespace(): Unit = { defaultClient.namespaces.withName(namespace).delete() Eventually.eventually(KubernetesSuite.TIMEOUT, KubernetesSuite.INTERVAL) { @@ -52,27 +66,6 @@ private[spark] class KubernetesTestComponents(defaultClient: DefaultKubernetesCl } } - def deleteKubernetesResources(): Unit = { - kubernetesClient.persistentVolumes().delete() - Eventually.eventually(KubernetesSuite.TIMEOUT, KubernetesSuite.INTERVAL) { - val persistentList = kubernetesClient - .persistentVolumes() - .list() - .getItems() - .asScala - require(!persistentList.exists(_.getMetadata.getNamespace == namespace)) - } - kubernetesClient.configMaps().delete() - Eventually.eventually(KubernetesSuite.TIMEOUT, KubernetesSuite.INTERVAL) { - val configMapsList = kubernetesClient - .configMaps() - .list() - .getItems() - .asScala - require(!configMapsList.exists(_.getMetadata.getNamespace == namespace)) - } - } - def newSparkConf(): SparkConf = { new SparkConf(true) .setMaster(s"k8s://${kubernetesClient.getMasterUrl}") diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/docker/SparkDockerImageBuilder.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/docker/SparkDockerImageBuilder.scala index ce1f4d45ec4a6..d596368c7155a 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 @@ -87,16 +87,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("kerberos-test", KERBEROS_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 = { From 1e71ca78604dddaea1b75b5f957b06750747960c Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Fri, 4 Aug 2017 08:42:07 -0700 Subject: [PATCH 20/37] uncommented SparkDockerBuilder --- .../docker/SparkDockerImageBuilder.scala | 20 +++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) 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 d596368c7155a..ce1f4d45ec4a6 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 @@ -87,16 +87,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("kerberos-test", KERBEROS_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 = { From 350c8ed28bc28c5a7d5f63bf0fc9bb31402a0403 Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Fri, 4 Aug 2017 11:02:23 -0700 Subject: [PATCH 21/37] testing fix --- resource-managers/kubernetes/core/pom.xml | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/resource-managers/kubernetes/core/pom.xml b/resource-managers/kubernetes/core/pom.xml index 3feecb09230bc..5079c1ad865dc 100644 --- a/resource-managers/kubernetes/core/pom.xml +++ b/resource-managers/kubernetes/core/pom.xml @@ -100,6 +100,12 @@ com.fasterxml.jackson.jaxrs jackson-jaxrs-json-provider + + + javax.ws.rs + jsr311-api + + javax.ws.rs From 5e4051c22a0a75990748186ae15e7af71d11614c Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Tue, 8 Aug 2017 18:25:23 -0400 Subject: [PATCH 22/37] handled comments and increased test hardening --- docs/running-on-kubernetes.md | 16 +++--- .../kubernetes/HadoopConfBootstrap.scala | 10 ++-- .../spark/deploy/kubernetes/config.scala | 2 +- .../spark/deploy/kubernetes/constants.scala | 4 +- ...DriverConfigurationStepsOrchestrator.scala | 18 +++---- .../HadoopConfigBootstrapStep.scala | 6 +-- .../hadoopsteps/HadoopConfMounterStep.scala | 5 +- .../HadoopKerberosKeytabResolverStep.scala | 36 ++++++------- .../hadoopsteps/HadoopStepsOrchestrator.scala | 34 +++++++++--- ...tainerConfigurationStepsOrchestrator.scala | 1 + .../KubernetesClusterSchedulerBackend.scala | 1 + ...rConfigurationStepsOrchestratorSuite.scala | 53 ++++++++++++++----- ... => InitContainerBootstrapStepSuite.scala} | 2 +- .../src/main/assembly/docker-assembly.xml | 1 + .../kerberos-yml/test-env.sh | 2 - .../KerberizedHadoopClusterLauncher.scala | 9 +++- .../kerberos/KerberosPodWatcherCache.scala | 7 ++- 17 files changed, 127 insertions(+), 80 deletions(-) rename resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/{initContainerBootstrapStepSuite.scala => InitContainerBootstrapStepSuite.scala} (99%) diff --git a/docs/running-on-kubernetes.md b/docs/running-on-kubernetes.md index 7803d1b7692c3..2ac8647d575f5 100644 --- a/docs/running-on-kubernetes.md +++ b/docs/running-on-kubernetes.md @@ -769,10 +769,10 @@ from the other deployment modes. See the [configuration page](configuration.html - spark.kubernetes.kerberos + spark.kubernetes.kerberos.enabled false - Specify whether your job is a job that will require a Delegation Token to access HDFS. By default, we + Specify whether your job is a job that will require a Kerberos Authorization to access HDFS. By default, we will assume that you will not require secure HDFS access. @@ -780,9 +780,9 @@ from the other deployment modes. See the [configuration page](configuration.html spark.kubernetes.kerberos.keytab (none) - Assuming you have set spark.kubernetes.kerberos to be true. This will let you specify + Assuming you have set spark.kubernetes.kerberos.enabled to be true. This will let you specify the location of your Kerberos keytab to be used in order to access Secure HDFS. This is optional as you - may login by running kinit -kt before running the spark-submit, and the submission client + may login by running kinit before running the spark-submit, and the submission client will look within your local TGT cache to resolve this. @@ -790,9 +790,9 @@ from the other deployment modes. See the [configuration page](configuration.html spark.kubernetes.kerberos.principal (none) - Assuming you have set spark.kubernetes.kerberos to be true. This will let you specify + Assuming you have set spark.kubernetes.kerberos.enabled to be true. This will let you specify your Kerberos principal that you wish to use to access Secure HDFS. This is optional as you - may login by running kinit -kt before running the spark-submit, and the submission client + may login by running kinit before running the spark-submit, and the submission client will look within your local TGT cache to resolve this. @@ -800,7 +800,7 @@ from the other deployment modes. See the [configuration page](configuration.html spark.kubernetes.kerberos.tokensecret.name (none) - Assuming you have set spark.kubernetes.kerberos to be true. This will let you specify + Assuming you have set spark.kubernetes.kerberos.enabled to be true. This will let you specify the name of the secret where your existing delegation token data is stored. You must also specify the label spark.kubernetes.kerberos.tokensecret.name where your data is stored on the secret. @@ -809,7 +809,7 @@ from the other deployment modes. See the [configuration page](configuration.html 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 + Assuming you have set spark.kubernetes.kerberos.enabled 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 54b59d235860a..f96be25a13319 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 @@ -32,8 +32,8 @@ import org.apache.spark.internal.Logging */ 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 containing Hadoop config files + * mounted as volumes and an ENV variable pointing to the mounted file. */ def bootstrapMainContainerAndVolumes( originalPodWithMainContainer: PodWithMainContainer) @@ -68,11 +68,11 @@ private[spark] class HadoopConfBootstrapImpl( originalPodWithMainContainer.mainContainer) .addNewVolumeMount() .withName(HADOOP_FILE_VOLUME) - .withMountPath(HADOOP_FILE_DIR) + .withMountPath(HADOOP_CONF_DIR_PATH) .endVolumeMount() .addNewEnv() - .withName(HADOOP_CONF_DIR) - .withValue(HADOOP_FILE_DIR) + .withName(ENV_HADOOP_CONF_DIR) + .withValue(HADOOP_CONF_DIR_PATH) .endEnv() .build() originalPodWithMainContainer.copy( 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 0262357c2e5ff..efbf931838708 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 @@ -545,7 +545,7 @@ package object config extends Logging { " 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") + .createOptional 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 25a023074fdc7..8b5e4091a43c2 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 @@ -98,8 +98,8 @@ package object constants { // 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_PATH = "/etc/hadoop/conf" + private[spark] val ENV_HADOOP_CONF_DIR = "HADOOP_CONF_DIR" private[spark] val HADOOP_CONF_DIR_LOC = "spark.kubernetes.hadoop.conf.dir" private[spark] val HADOOP_CONFIG_MAP_SPARK_CONF_NAME = "spark.kubernetes.hadoop.executor.hadoopconfigmapname" 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 096518432e2de..a75cef97e488a 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 @@ -98,18 +98,12 @@ private[spark] class DriverConfigurationStepsOrchestrator( val kubernetesCredentialsStep = new DriverKubernetesCredentialsStep( submissionSparkConf, kubernetesResourceNamePrefix) val hadoopConfigSteps = - if (hadoopConfDir.isEmpty) { - Option.empty[DriverConfigurationStep] - } else { - val hadoopStepsOrchestrator = new HadoopStepsOrchestrator( - namespace, - hadoopConfigMapName, - submissionSparkConf, - hadoopConfDir) - val hadoopConfSteps = - hadoopStepsOrchestrator.getHadoopSteps() - Some(new HadoopConfigBootstrapStep(hadoopConfSteps, hadoopConfigMapName)) - } + hadoopConfDir.map { conf => + val hadoopStepsOrchestrator = + new HadoopStepsOrchestrator(namespace, hadoopConfigMapName, submissionSparkConf, conf) + val hadoopConfSteps = hadoopStepsOrchestrator.getHadoopSteps() + Some(new HadoopConfigBootstrapStep(hadoopConfSteps, hadoopConfigMapName))} + .getOrElse(Option.empty[DriverConfigurationStep]) val pythonStep = mainAppResource match { case PythonMainAppResource(mainPyResource) => Option(new PythonStep(mainPyResource, additionalPythonFiles, filesDownloadPath)) 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 315dad9c5a591..a2e11a327b955 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 @@ -49,9 +49,9 @@ private[spark] class HadoopConfigBootstrapStep( val configMap = new ConfigMapBuilder() .withNewMetadata() - .withName(hadoopConfigMapName) - .endMetadata() - .addToData(currentHadoopSpec.configMapProperties.asJava) + .withName(hadoopConfigMapName) + .endMetadata() + .addToData(currentHadoopSpec.configMapProperties.asJava) .build() val executorSparkConf = driverSpec.driverSparkConf.clone() .set(HADOOP_CONFIG_MAP_SPARK_CONF_NAME, hadoopConfigMapName) 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 e3b7f674d0225..53b979df326c2 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 @@ -34,7 +34,7 @@ private[spark] class HadoopConfMounterStep( hadoopConfigMapName: String, hadoopConfigurationFiles: Seq[File], hadoopConfBootstrapConf: HadoopConfBootstrap, - hadoopConfDir: Option[String]) + hadoopConfDir: String) extends HadoopConfigurationStep { override def configureContainers(hadoopConfigSpec: HadoopConfigSpec): HadoopConfigSpec = { @@ -51,8 +51,7 @@ private[spark] class HadoopConfMounterStep( hadoopConfigurationFiles.map(file => (file.toPath.getFileName.toString, readFileToString(file))).toMap, additionalDriverSparkConf = hadoopConfigSpec.additionalDriverSparkConf ++ - hadoopConfDir.map(conf_dir => Map(HADOOP_CONF_DIR_LOC -> conf_dir)).getOrElse( - Map.empty[String, String]) + Map(HADOOP_CONF_DIR_LOC -> hadoopConfDir) ) } } 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 2f57ffe18f063..624bed1a93019 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 @@ -55,11 +55,11 @@ private[spark] class HadoopKerberosKeytabResolverStep( private var originalCredentials: Credentials = _ private var dfs : FileSystem = _ private var renewer: String = _ - private var renewedCredentials: Credentials = _ - private var renewedTokens: Iterable[Token[_ <: TokenIdentifier]] = _ + private var credentials: Credentials = _ + private var tokens: Iterable[Token[_ <: TokenIdentifier]] = _ override def configureContainers(hadoopConfigSpec: HadoopConfigSpec): HadoopConfigSpec = { val hadoopConf = SparkHadoopUtil.get.newConfiguration(submissionSparkConf) - logInfo(s"Hadoop Configuration: ${hadoopConf.toString}") + logDebug(s"Hadoop Configuration: ${hadoopConf.toString}") if (!UserGroupInformation.isSecurityEnabled) logError("Hadoop not configuration with Kerberos") val maybeJobUserUGI = for { @@ -70,7 +70,7 @@ private[spark] class HadoopKerberosKeytabResolverStep( // 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") + logDebug("Logged into KDC with keytab using Job User UGI") UserGroupInformation.loginUserFromKeytabAndReturnUGI( principal, keytab.toURI.toString) @@ -80,27 +80,27 @@ private[spark] class HadoopKerberosKeytabResolverStep( // 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") + logDebug(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}") + logDebug(s"Original tokens: ${originalCredentials.toString}") + logDebug(s"All tokens: ${originalCredentials.getAllTokens}") + logDebug(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}") + logDebug(s"Renewer is: $renewer") + credentials = new Credentials(originalCredentials) + dfs.addDelegationTokens(renewer, credentials) + tokens = credentials.getAllTokens.asScala + logDebug(s"Tokens: ${credentials.toString}") + logDebug(s"All tokens: ${tokens.mkString(",")}") + logDebug(s"All secret keys: ${credentials.getAllSecretKeys}") null }}) - if (renewedTokens.isEmpty) logError("Did not obtain any Delegation Tokens") - val data = serialize(renewedCredentials) - val renewalTime = getTokenRenewalInterval(renewedTokens, hadoopConf).getOrElse(Long.MaxValue) + if (tokens.isEmpty) logError("Did not obtain any Delegation Tokens") + val data = serialize(credentials) + val renewalTime = getTokenRenewalInterval(tokens, hadoopConf).getOrElse(Long.MaxValue) val currentTime: Long = System.currentTimeMillis() val initialTokenLabelName = s"$KERBEROS_SECRET_LABEL_PREFIX-$currentTime-$renewalTime" val secretDT = 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 b43b99f9b1a06..50b451ea6db7c 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 @@ -19,7 +19,7 @@ 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.{HadoopConfBootstrapImpl, OptionRequirements} import org.apache.spark.deploy.kubernetes.config._ @@ -30,16 +30,35 @@ private[spark] class HadoopStepsOrchestrator( namespace: String, hadoopConfigMapName: String, submissionSparkConf: SparkConf, - hadoopConfDir: Option[String]) { - private val maybeKerberosSupport = submissionSparkConf.get(KUBERNETES_KERBEROS_SUPPORT) + hadoopConfDir: String) { + private val isKerberosEnabled = submissionSparkConf.get(KUBERNETES_KERBEROS_SUPPORT) private val maybePrincipal = submissionSparkConf.get(KUBERNETES_KERBEROS_PRINCIPAL) private val maybeKeytab = submissionSparkConf.get(KUBERNETES_KERBEROS_KEYTAB) .map(k => new File(k)) private val maybeExistingSecret = submissionSparkConf.get(KUBERNETES_KERBEROS_DT_SECRET_NAME) private val maybeExistingSecretLabel = submissionSparkConf.get(KUBERNETES_KERBEROS_DT_SECRET_LABEL) - private val hadoopConfigurationFiles = hadoopConfDir.map(conf => getHadoopConfFiles(conf)) - .getOrElse(Seq.empty[File]) + private val hadoopConfigurationFiles = getHadoopConfFiles(hadoopConfDir) + + require(maybeKeytab.forall( _ => isKerberosEnabled ), + "You must enable Kerberos support if you are specifying a Kerberos Keytab") + + require(maybeExistingSecret.forall( _ => isKerberosEnabled ), + "You must enable Kerberos support if you are specifying a Kerberos Secret") + + OptionRequirements.requireBothOrNeitherDefined( + maybeKeytab, + maybePrincipal, + "If a Kerberos keytab is specified you must also specify a Kerberos principal", + "If a Kerberos principal is specified you must also specify a Kerberos keytab") + + OptionRequirements.requireBothOrNeitherDefined( + maybeExistingSecret, + maybeExistingSecretLabel, + "If a secret storing a Kerberos Delegation Token is specified you must also" + + " specify the label where the data is stored", + "If a secret label where the data of the Kerberos Delegation Token is specified" + + " you must also specify the name of the secret") def getHadoopSteps(): Seq[HadoopConfigurationStep] = { val hadoopConfBootstrapImpl = new HadoopConfBootstrapImpl( @@ -51,11 +70,11 @@ private[spark] class HadoopStepsOrchestrator( hadoopConfBootstrapImpl, hadoopConfDir) val maybeKerberosStep = - if (maybeKerberosSupport) { + if (isKerberosEnabled) { maybeExistingSecret.map(secretLabel => Some(new HadoopKerberosSecretResolverStep( submissionSparkConf, secretLabel, - maybeExistingSecretLabel))).getOrElse(Some( + maybeExistingSecretLabel.get))).getOrElse(Some( new HadoopKerberosKeytabResolverStep( submissionSparkConf, maybePrincipal, @@ -65,6 +84,7 @@ private[spark] class HadoopStepsOrchestrator( } 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) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/InitContainerConfigurationStepsOrchestrator.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/InitContainerConfigurationStepsOrchestrator.scala index e4ea5235af18f..8d69d377376b8 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/InitContainerConfigurationStepsOrchestrator.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/InitContainerConfigurationStepsOrchestrator.scala @@ -62,6 +62,7 @@ private[spark] class InitContainerConfigurationStepsOrchestrator( submissionSparkConf.get(RESOURCE_STAGING_SERVER_INTERNAL_SSL_ENABLED) .orElse(submissionSparkConf.get(RESOURCE_STAGING_SERVER_SSL_ENABLED)) .getOrElse(false) + OptionRequirements.requireNandDefined( maybeResourceStagingServerInternalClientCert, maybeResourceStagingServerInternalTrustStore, 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 085886dff51c0..43b8cb0f2b3d7 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 @@ -591,6 +591,7 @@ private[spark] class KubernetesClusterSchedulerBackend( ) (podWithMainContainer.pod, podWithMainContainer.mainContainer) }.getOrElse(executorPodWithNodeAffinity, initBootstrappedExecutorContainer) + val (executorKerberosPod, executorKerberosContainer) = executorKerberosBootStrap.map { bootstrap => val podWithMainContainer = bootstrap.bootstrapMainContainerAndVolumes( 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 babb65bf743b1..3b34c407267cb 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 @@ -17,7 +17,7 @@ package org.apache.spark.deploy.kubernetes.submit import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.deploy.kubernetes.submit.submitsteps.{BaseDriverConfigurationStep, DependencyResolutionStep, DriverKubernetesCredentialsStep, InitContainerBootstrapStep, PythonStep} +import org.apache.spark.deploy.kubernetes.submit.submitsteps._ private[spark] class DriverConfigurationStepsOrchestratorSuite extends SparkFunSuite { @@ -46,7 +46,7 @@ private[spark] class DriverConfigurationStepsOrchestratorSuite extends SparkFunS sparkConf) val steps = orchestrator.getAllConfigurationSteps() assert(steps.size === 3) - assert(steps(0).isInstanceOf[BaseDriverConfigurationStep]) + assert(steps.head.isInstanceOf[BaseDriverConfigurationStep]) assert(steps(1).isInstanceOf[DriverKubernetesCredentialsStep]) assert(steps(2).isInstanceOf[DependencyResolutionStep]) } @@ -68,7 +68,7 @@ private[spark] class DriverConfigurationStepsOrchestratorSuite extends SparkFunS sparkConf) val steps = orchestrator.getAllConfigurationSteps() assert(steps.size === 4) - assert(steps(0).isInstanceOf[BaseDriverConfigurationStep]) + assert(steps.head.isInstanceOf[BaseDriverConfigurationStep]) assert(steps(1).isInstanceOf[DriverKubernetesCredentialsStep]) assert(steps(2).isInstanceOf[DependencyResolutionStep]) assert(steps(3).isInstanceOf[InitContainerBootstrapStep]) @@ -78,21 +78,46 @@ private[spark] class DriverConfigurationStepsOrchestratorSuite extends SparkFunS val sparkConf = new SparkConf(false) val mainAppResource = PythonMainAppResource("local:///var/apps/python/main.py") val orchestrator = new DriverConfigurationStepsOrchestrator( - NAMESPACE, - APP_ID, - LAUNCH_TIME, - mainAppResource, - APP_NAME, - MAIN_CLASS, - APP_ARGS, - ADDITIONAL_PYTHON_FILES, - None, - sparkConf) + NAMESPACE, + APP_ID, + LAUNCH_TIME, + mainAppResource, + APP_NAME, + MAIN_CLASS, + APP_ARGS, + ADDITIONAL_PYTHON_FILES, + None, + sparkConf) val steps = orchestrator.getAllConfigurationSteps() assert(steps.size === 4) - assert(steps(0).isInstanceOf[BaseDriverConfigurationStep]) + assert(steps.head.isInstanceOf[BaseDriverConfigurationStep]) assert(steps(1).isInstanceOf[DriverKubernetesCredentialsStep]) assert(steps(2).isInstanceOf[DependencyResolutionStep]) assert(steps(3).isInstanceOf[PythonStep]) } + + test("Submission steps with hdfs interaction and HADOOP_CONF_DIR defined") { + val sparkConf = new SparkConf(false) + val mainAppResource = JavaMainAppResource("hdfs:///var/apps/jars/main.jar") + val hadoopConf = Some("/etc/hadoop/conf") + val orchestrator = new DriverConfigurationStepsOrchestrator( + NAMESPACE, + APP_ID, + LAUNCH_TIME, + mainAppResource, + APP_NAME, + MAIN_CLASS, + APP_ARGS, + ADDITIONAL_PYTHON_FILES, + hadoopConf, + sparkConf) + val steps = orchestrator.getAllConfigurationSteps() + assert(steps.size === 5) + assert(steps.head.isInstanceOf[BaseDriverConfigurationStep]) + assert(steps(1).isInstanceOf[DriverKubernetesCredentialsStep]) + assert(steps(2).isInstanceOf[DependencyResolutionStep]) + assert(steps(3).isInstanceOf[InitContainerBootstrapStep]) + assert(steps(4).isInstanceOf[HadoopConfigBootstrapStep]) + } + } diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initContainerBootstrapStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/InitContainerBootstrapStepSuite.scala similarity index 99% rename from resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initContainerBootstrapStepSuite.scala rename to resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/InitContainerBootstrapStepSuite.scala index b11b487111496..8946f88c931ca 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initContainerBootstrapStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/InitContainerBootstrapStepSuite.scala @@ -31,7 +31,7 @@ import org.apache.spark.deploy.kubernetes.constants._ import org.apache.spark.deploy.kubernetes.submit.submitsteps.initcontainer.{InitContainerConfigurationStep, InitContainerSpec} import org.apache.spark.util.Utils -private[spark] class initContainerBootstrapStepSuite extends SparkFunSuite { +private[spark] class InitContainerBootstrapStepSuite extends SparkFunSuite { private val OBJECT_MAPPER = new ObjectMapper().registerModule(DefaultScalaModule) private val CONFIG_MAP_NAME = "spark-init-config-map" diff --git a/resource-managers/kubernetes/docker-minimal-bundle/src/main/assembly/docker-assembly.xml b/resource-managers/kubernetes/docker-minimal-bundle/src/main/assembly/docker-assembly.xml index 2b48d366256fe..e6de2c11a0ecf 100644 --- a/resource-managers/kubernetes/docker-minimal-bundle/src/main/assembly/docker-assembly.xml +++ b/resource-managers/kubernetes/docker-minimal-bundle/src/main/assembly/docker-assembly.xml @@ -69,6 +69,7 @@ org.apache.spark:spark-assembly_${scala.binary.version}:pom org.spark-project.spark:unused + com.sun.jersey:* 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 079f2e37936b4..1248d57bf25a8 100644 --- a/resource-managers/kubernetes/integration-tests/kerberos-yml/test-env.sh +++ b/resource-managers/kubernetes/integration-tests/kerberos-yml/test-env.sh @@ -18,8 +18,6 @@ until /usr/bin/kinit -kt /var/keytabs/hdfs.keytab hdfs/nn.${NAMESPACE}.svc.clust --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.hadoop.dfs.data.transfer.protection=authentication \ --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 \ 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 ce8f456d3e11d..f053cf624ceb6 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 @@ -19,6 +19,7 @@ package org.apache.spark.deploy.kubernetes.integrationtest import io.fabric8.kubernetes.client.KubernetesClient import org.apache.spark.deploy.kubernetes.integrationtest.kerberos._ +import org.apache.spark.internal.Logging /** * This class is responsible for launching a psuedo-distributed, single noded, @@ -29,7 +30,7 @@ import org.apache.spark.deploy.kubernetes.integrationtest.kerberos._ */ private[spark] class KerberizedHadoopClusterLauncher( kubernetesClient: KubernetesClient, - namespace: String) { + namespace: String) extends Logging { private val LABELS = Map("job" -> "kerberostest") def launchKerberizedCluster(): Unit = { @@ -47,6 +48,10 @@ private[spark] class KerberizedHadoopClusterLauncher( // Launches the Hadoop cluster pods: KDC --> NN --> DN1 --> Data-Populator val podWatcherCache = new KerberosPodWatcherCache(kerberosUtils, LABELS) podWatcherCache.start() - podWatcherCache.stop() + val dpNode = podWatcherCache.stop() + while (!podWatcherCache.hasInLogs(dpNode, "")) { + logInfo("Waiting for data-populator to be formatted") + Thread.sleep(500) + } } } \ No newline at end of file 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 e212dac4777a1..60cc9735212fa 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 @@ -57,6 +57,7 @@ private[spark] class KerberosPodWatcherCache( private var dnSpawned: Boolean = false private var dpSpawned: Boolean = false private var dnName: String = _ + private var dpName: String = _ private val blockingThread = new Thread(new Runnable { override def run(): Unit = { @@ -95,6 +96,7 @@ private[spark] class KerberosPodWatcherCache( val phase = resource.getStatus.getPhase logInfo(s"$name is as $phase") if (name.startsWith("dn1")) { dnName = name } + if (name.startsWith("data-populator")) { dpName = name } podCache(keyName) = phase if (maybeDeploymentAndServiceDone(keyName)) { val modifyAndSignal: Runnable = new MSThread(keyName) @@ -142,9 +144,10 @@ private[spark] class KerberosPodWatcherCache( serviceWatcherThread.join() } - def stop(): Unit = { + def stop(): String = { podWatcher.close() serviceWatcher.close() + dpName } private def maybeDeploymentAndServiceDone(name: String): Boolean = { @@ -244,7 +247,7 @@ private[spark] class KerberosPodWatcherCache( } } - private def hasInLogs(name: String, expectation: String): Boolean = { + def hasInLogs(name: String, expectation: String): Boolean = { kubernetesClient .pods() .withName(name) From 8338fdbac15c2af881c7c3745d72c0ffe556120b Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Tue, 8 Aug 2017 22:11:41 -0400 Subject: [PATCH 23/37] Solve failing integration test problem and lower TIMEOUT time --- .../integrationtest/KubernetesSuite.scala | 21 ++++++++++++------- 1 file changed, 14 insertions(+), 7 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 e509c457c114d..e747ed0428b0f 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 @@ -133,6 +133,7 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { System.getProperty("spark.docker.test.driverImage", "spark-driver-py:latest")) .set(EXECUTOR_DOCKER_IMAGE, System.getProperty("spark.docker.test.executorImage", "spark-executor-py:latest")) + .set(KUBERNETES_KERBEROS_SUPPORT, false) runPySparkPiAndVerifyCompletion( PYSPARK_PI_SUBMITTER_LOCAL_FILE_LOCATION, @@ -149,13 +150,14 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { System.getProperty("spark.docker.test.driverImage", "spark-driver-py:latest")) .set(EXECUTOR_DOCKER_IMAGE, System.getProperty("spark.docker.test.executorImage", "spark-executor-py:latest")) + .set(KUBERNETES_KERBEROS_SUPPORT, false) 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) - + sparkConf.set(KUBERNETES_KERBEROS_SUPPORT, false) launchStagingServer(SSLOptions(), None) runSparkPiAndVerifyCompletion(SUBMITTER_LOCAL_MAIN_APP_RESOURCE) } @@ -176,6 +178,7 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { .set("spark.ssl.kubernetes.resourceStagingServer.keyStorePassword", "keyStore") .set("spark.ssl.kubernetes.resourceStagingServer.keyPassword", "key") .set("spark.ssl.kubernetes.resourceStagingServer.trustStorePassword", "trustStore") + .set(KUBERNETES_KERBEROS_SUPPORT, false) launchStagingServer(SSLOptions( enabled = true, keyStore = Some(keyStoreAndTrustStore.keyStore), @@ -190,7 +193,7 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { test("Use container-local resources without the resource staging server") { assume(testBackend.name == MINIKUBE_TEST_BACKEND) - sparkConf.setJars(Seq(CONTAINER_LOCAL_HELPER_JAR_PATH)) + sparkConf.setJars(Seq(CONTAINER_LOCAL_HELPER_JAR_PATH)).set(KUBERNETES_KERBEROS_SUPPORT, false) runSparkPiAndVerifyCompletion(CONTAINER_LOCAL_MAIN_APP_RESOURCE) } @@ -206,6 +209,7 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { 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") + sparkConf.set(KUBERNETES_KERBEROS_SUPPORT, false) runSparkApplicationAndVerifyCompletion( JavaMainAppResource(SUBMITTER_LOCAL_MAIN_APP_RESOURCE), GROUP_BY_MAIN_CLASS, @@ -221,7 +225,7 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { sparkConf.setJars(Seq( s"$assetServerUri/${EXAMPLES_JAR_FILE.getName}", s"$assetServerUri/${HELPER_JAR_FILE.getName}" - )) + )).set(KUBERNETES_KERBEROS_SUPPORT, false) runSparkPiAndVerifyCompletion(SparkLauncher.NO_RESOURCE) } @@ -231,7 +235,7 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { val assetServerUri = staticAssetServerLauncher.launchStaticAssetServer() sparkConf.setJars(Seq( SUBMITTER_LOCAL_MAIN_APP_RESOURCE, s"$assetServerUri/${HELPER_JAR_FILE.getName}" - )) + )).set(KUBERNETES_KERBEROS_SUPPORT, false) runSparkPiAndVerifyCompletion(SparkLauncher.NO_RESOURCE) } @@ -244,6 +248,7 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { sparkConf.set(RESOURCE_STAGING_SERVER_SSL_ENABLED, true) .set( RESOURCE_STAGING_SERVER_CLIENT_CERT_PEM.key, keyAndCertificate.certPem.getAbsolutePath) + .set(KUBERNETES_KERBEROS_SUPPORT, false) runSparkPiAndVerifyCompletion(SUBMITTER_LOCAL_MAIN_APP_RESOURCE) } @@ -261,6 +266,7 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { sparkConf.set( s"$APISERVER_AUTH_DRIVER_CONF_PREFIX.$CA_CERT_FILE_CONF_SUFFIX", kubernetesTestComponents.clientConfig.getCaCertFile) + sparkConf.set(KUBERNETES_KERBEROS_SUPPORT, false) runSparkPiAndVerifyCompletion(SparkLauncher.NO_RESOURCE) } @@ -270,7 +276,7 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { 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) + sparkConf.set("spark.files", testExistenceFile.getAbsolutePath).set(KUBERNETES_KERBEROS_SUPPORT, false) runSparkApplicationAndVerifyCompletion( JavaMainAppResource(SUBMITTER_LOCAL_MAIN_APP_RESOURCE), FILE_EXISTENCE_MAIN_CLASS, @@ -284,6 +290,7 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { assume(testBackend.name == MINIKUBE_TEST_BACKEND) sparkConf.setJars(Seq(CONTAINER_LOCAL_HELPER_JAR_PATH)).setAppName("long" * 40) + .set(KUBERNETES_KERBEROS_SUPPORT, false) runSparkPiAndVerifyCompletion(CONTAINER_LOCAL_MAIN_APP_RESOURCE) } @@ -423,8 +430,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(15, Minutes)) - val INTERVAL = PatienceConfiguration.Interval(Span(15, Seconds)) + val TIMEOUT = PatienceConfiguration.Timeout(Span(10, Minutes)) + val INTERVAL = PatienceConfiguration.Interval(Span(10, Seconds)) val SPARK_PI_MAIN_CLASS = "org.apache.spark.deploy.kubernetes" + ".integrationtest.jobs.SparkPiWithInfiniteWait" val PYSPARK_PI_MAIN_CLASS = "org.apache.spark.deploy.PythonRunner" From d6d094532484583da2e582e667ea46957f05aa90 Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Wed, 9 Aug 2017 00:28:09 -0400 Subject: [PATCH 24/37] modify security.authoization --- .../integrationtest/KubernetesSuite.scala | 21 ++++++++++--------- 1 file changed, 11 insertions(+), 10 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 e747ed0428b0f..9baf9e125e12f 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 @@ -133,7 +133,7 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { System.getProperty("spark.docker.test.driverImage", "spark-driver-py:latest")) .set(EXECUTOR_DOCKER_IMAGE, System.getProperty("spark.docker.test.executorImage", "spark-executor-py:latest")) - .set(KUBERNETES_KERBEROS_SUPPORT, false) + .set(KERBEROS_CONF, "simple") runPySparkPiAndVerifyCompletion( PYSPARK_PI_SUBMITTER_LOCAL_FILE_LOCATION, @@ -150,7 +150,7 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { System.getProperty("spark.docker.test.driverImage", "spark-driver-py:latest")) .set(EXECUTOR_DOCKER_IMAGE, System.getProperty("spark.docker.test.executorImage", "spark-executor-py:latest")) - .set(KUBERNETES_KERBEROS_SUPPORT, false) + .set(KERBEROS_CONF, "simple") runPySparkPiAndVerifyCompletion(PYSPARK_PI_CONTAINER_LOCAL_FILE_LOCATION, Seq.empty[String]) } @@ -178,7 +178,7 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { .set("spark.ssl.kubernetes.resourceStagingServer.keyStorePassword", "keyStore") .set("spark.ssl.kubernetes.resourceStagingServer.keyPassword", "key") .set("spark.ssl.kubernetes.resourceStagingServer.trustStorePassword", "trustStore") - .set(KUBERNETES_KERBEROS_SUPPORT, false) + .set(KERBEROS_CONF, "simple") launchStagingServer(SSLOptions( enabled = true, keyStore = Some(keyStoreAndTrustStore.keyStore), @@ -209,7 +209,7 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { 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") - sparkConf.set(KUBERNETES_KERBEROS_SUPPORT, false) + sparkConf.set(KERBEROS_CONF, "simple") runSparkApplicationAndVerifyCompletion( JavaMainAppResource(SUBMITTER_LOCAL_MAIN_APP_RESOURCE), GROUP_BY_MAIN_CLASS, @@ -225,7 +225,7 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { sparkConf.setJars(Seq( s"$assetServerUri/${EXAMPLES_JAR_FILE.getName}", s"$assetServerUri/${HELPER_JAR_FILE.getName}" - )).set(KUBERNETES_KERBEROS_SUPPORT, false) + )).set(KERBEROS_CONF, "simple") runSparkPiAndVerifyCompletion(SparkLauncher.NO_RESOURCE) } @@ -235,7 +235,7 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { val assetServerUri = staticAssetServerLauncher.launchStaticAssetServer() sparkConf.setJars(Seq( SUBMITTER_LOCAL_MAIN_APP_RESOURCE, s"$assetServerUri/${HELPER_JAR_FILE.getName}" - )).set(KUBERNETES_KERBEROS_SUPPORT, false) + )).set(KERBEROS_CONF, "simple") runSparkPiAndVerifyCompletion(SparkLauncher.NO_RESOURCE) } @@ -248,7 +248,7 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { sparkConf.set(RESOURCE_STAGING_SERVER_SSL_ENABLED, true) .set( RESOURCE_STAGING_SERVER_CLIENT_CERT_PEM.key, keyAndCertificate.certPem.getAbsolutePath) - .set(KUBERNETES_KERBEROS_SUPPORT, false) + .set(KERBEROS_CONF, "simple") runSparkPiAndVerifyCompletion(SUBMITTER_LOCAL_MAIN_APP_RESOURCE) } @@ -266,7 +266,7 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { sparkConf.set( s"$APISERVER_AUTH_DRIVER_CONF_PREFIX.$CA_CERT_FILE_CONF_SUFFIX", kubernetesTestComponents.clientConfig.getCaCertFile) - sparkConf.set(KUBERNETES_KERBEROS_SUPPORT, false) + sparkConf.set(KERBEROS_CONF, "simple") runSparkPiAndVerifyCompletion(SparkLauncher.NO_RESOURCE) } @@ -276,7 +276,7 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { 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).set(KUBERNETES_KERBEROS_SUPPORT, false) + sparkConf.set("spark.files", testExistenceFile.getAbsolutePath).set(KERBEROS_CONF, "simple") runSparkApplicationAndVerifyCompletion( JavaMainAppResource(SUBMITTER_LOCAL_MAIN_APP_RESOURCE), FILE_EXISTENCE_MAIN_CLASS, @@ -290,7 +290,7 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { assume(testBackend.name == MINIKUBE_TEST_BACKEND) sparkConf.setJars(Seq(CONTAINER_LOCAL_HELPER_JAR_PATH)).setAppName("long" * 40) - .set(KUBERNETES_KERBEROS_SUPPORT, false) + .set(KERBEROS_CONF, "simple") runSparkPiAndVerifyCompletion(CONTAINER_LOCAL_MAIN_APP_RESOURCE) } @@ -447,6 +447,7 @@ private[spark] object KubernetesSuite { val HDFS_TEST_CLASS = "org.apache.spark.deploy.kubernetes" + ".integrationtest.jobs.HDFSTest" val TEST_EXISTENCE_FILE_CONTENTS = "contents" + val KERBEROS_CONF = "spark.hadoop.hadoop.security.authentication" case object ShuffleNotReadyException extends Exception From e3f14e141321ffe5892e14b6dc5bed84d5f6b206 Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Wed, 9 Aug 2017 12:37:53 -0400 Subject: [PATCH 25/37] Modifying HADOOP_CONF flags --- .../integrationtest/KubernetesSuite.scala | 16 +++++----------- 1 file changed, 5 insertions(+), 11 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 9baf9e125e12f..4df5ef83806aa 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 @@ -157,7 +157,6 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { test("Simple submission test with the resource staging server.") { assume(testBackend.name == MINIKUBE_TEST_BACKEND) - sparkConf.set(KUBERNETES_KERBEROS_SUPPORT, false) launchStagingServer(SSLOptions(), None) runSparkPiAndVerifyCompletion(SUBMITTER_LOCAL_MAIN_APP_RESOURCE) } @@ -178,7 +177,6 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { .set("spark.ssl.kubernetes.resourceStagingServer.keyStorePassword", "keyStore") .set("spark.ssl.kubernetes.resourceStagingServer.keyPassword", "key") .set("spark.ssl.kubernetes.resourceStagingServer.trustStorePassword", "trustStore") - .set(KERBEROS_CONF, "simple") launchStagingServer(SSLOptions( enabled = true, keyStore = Some(keyStoreAndTrustStore.keyStore), @@ -193,7 +191,7 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { test("Use container-local resources without the resource staging server") { assume(testBackend.name == MINIKUBE_TEST_BACKEND) - sparkConf.setJars(Seq(CONTAINER_LOCAL_HELPER_JAR_PATH)).set(KUBERNETES_KERBEROS_SUPPORT, false) + sparkConf.setJars(Seq(CONTAINER_LOCAL_HELPER_JAR_PATH)) runSparkPiAndVerifyCompletion(CONTAINER_LOCAL_MAIN_APP_RESOURCE) } @@ -209,7 +207,6 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { 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") - sparkConf.set(KERBEROS_CONF, "simple") runSparkApplicationAndVerifyCompletion( JavaMainAppResource(SUBMITTER_LOCAL_MAIN_APP_RESOURCE), GROUP_BY_MAIN_CLASS, @@ -225,7 +222,7 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { sparkConf.setJars(Seq( s"$assetServerUri/${EXAMPLES_JAR_FILE.getName}", s"$assetServerUri/${HELPER_JAR_FILE.getName}" - )).set(KERBEROS_CONF, "simple") + )) runSparkPiAndVerifyCompletion(SparkLauncher.NO_RESOURCE) } @@ -235,7 +232,7 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { val assetServerUri = staticAssetServerLauncher.launchStaticAssetServer() sparkConf.setJars(Seq( SUBMITTER_LOCAL_MAIN_APP_RESOURCE, s"$assetServerUri/${HELPER_JAR_FILE.getName}" - )).set(KERBEROS_CONF, "simple") + )) runSparkPiAndVerifyCompletion(SparkLauncher.NO_RESOURCE) } @@ -248,7 +245,6 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { sparkConf.set(RESOURCE_STAGING_SERVER_SSL_ENABLED, true) .set( RESOURCE_STAGING_SERVER_CLIENT_CERT_PEM.key, keyAndCertificate.certPem.getAbsolutePath) - .set(KERBEROS_CONF, "simple") runSparkPiAndVerifyCompletion(SUBMITTER_LOCAL_MAIN_APP_RESOURCE) } @@ -266,7 +262,6 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { sparkConf.set( s"$APISERVER_AUTH_DRIVER_CONF_PREFIX.$CA_CERT_FILE_CONF_SUFFIX", kubernetesTestComponents.clientConfig.getCaCertFile) - sparkConf.set(KERBEROS_CONF, "simple") runSparkPiAndVerifyCompletion(SparkLauncher.NO_RESOURCE) } @@ -276,7 +271,7 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { 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).set(KERBEROS_CONF, "simple") + sparkConf.set("spark.files", testExistenceFile.getAbsolutePath) runSparkApplicationAndVerifyCompletion( JavaMainAppResource(SUBMITTER_LOCAL_MAIN_APP_RESOURCE), FILE_EXISTENCE_MAIN_CLASS, @@ -290,7 +285,6 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { assume(testBackend.name == MINIKUBE_TEST_BACKEND) sparkConf.setJars(Seq(CONTAINER_LOCAL_HELPER_JAR_PATH)).setAppName("long" * 40) - .set(KERBEROS_CONF, "simple") runSparkPiAndVerifyCompletion(CONTAINER_LOCAL_MAIN_APP_RESOURCE) } @@ -307,7 +301,7 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { } sparkConf.set(RESOURCE_STAGING_SERVER_URI, s"$resourceStagingServerUriScheme://" + - s"${Minikube.getMinikubeIp}:$resourceStagingServerPort") + s"${Minikube.getMinikubeIp}:$resourceStagingServerPort").set(KERBEROS_CONF, "simple") } private def launchKerberizedCluster(): Unit = { From 61a7414868a01933482d739ef493497faeb41c02 Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Tue, 15 Aug 2017 00:32:47 -0400 Subject: [PATCH 26/37] Refactored tests and included modifications to pass all tests regardless of environment --- .../scala/org/apache/spark/SparkConf.scala | 2 +- .../kubernetes/HadoopConfBootstrap.scala | 6 ++++- .../KerberosTokenConfBootstrap.scala | 8 ++++++- .../spark/deploy/kubernetes/constants.scala | 5 ++++ .../deploy/kubernetes/submit/Client.scala | 5 +++- .../HadoopKerberosKeytabResolverStep.scala | 3 ++- .../HadoopKerberosSecretResolverStep.scala | 5 +++- .../hadoopsteps/HadoopStepsOrchestrator.scala | 20 +++++++++------- .../kubernetes/KubernetesClusterManager.scala | 4 +++- .../KubernetesClusterSchedulerBackend.scala | 13 +++++++++- .../src/main/docker/driver-py/Dockerfile | 1 + .../src/main/docker/driver/Dockerfile | 1 + .../src/main/docker/executor-py/Dockerfile | 3 ++- .../src/main/docker/executor/Dockerfile | 3 ++- .../kerberos-yml/test-env.sh | 1 + .../kubernetes/integration-tests/pom.xml | 2 +- .../integrationtest/KubernetesSuite.scala | 2 +- .../kerberos/KerberosUtils.scala | 2 +- .../hadoop-conf}/core-site.xml | 0 .../hadoop-conf}/hdfs-site.xml | 0 .../hadoop-conf}/krb5.conf | 0 .../hadoop-conf}/yarn-site.xml | 0 .../simple-hadoop-conf/core-site.xml | 24 +++++++++++++++++++ .../simple-hadoop-conf/hdfs-site.xml | 24 +++++++++++++++++++ 24 files changed, 112 insertions(+), 22 deletions(-) rename resource-managers/kubernetes/integration-tests/{src/test/resources => test-data/hadoop-conf}/core-site.xml (100%) rename resource-managers/kubernetes/integration-tests/{src/test/resources => test-data/hadoop-conf}/hdfs-site.xml (100%) rename resource-managers/kubernetes/integration-tests/{src/test/resources => test-data/hadoop-conf}/krb5.conf (100%) rename resource-managers/kubernetes/integration-tests/{src/test/resources => test-data/hadoop-conf}/yarn-site.xml (100%) create mode 100644 resource-managers/kubernetes/integration-tests/test-data/simple-hadoop-conf/core-site.xml create mode 100644 resource-managers/kubernetes/integration-tests/test-data/simple-hadoop-conf/hdfs-site.xml diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index 2a2ce0504dbbf..53a6ba4f715cc 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -473,7 +473,7 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging with Seria // Validate spark.executor.extraJavaOptions getOption(executorOptsKey).foreach { javaOpts => - if (javaOpts.contains("-Dspark")) { + if (javaOpts.contains("-Dspark") && !javaOpts.contains("-Dspark.hadoop.")) { val msg = s"$executorOptsKey is not allowed to set Spark options (was '$javaOpts'). " + "Set them directly on a SparkConf or in a properties file when using ./bin/spark-submit." throw new Exception(msg) 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 f96be25a13319..add1a2d09ed74 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,11 +21,11 @@ import java.io.File import scala.collection.JavaConverters._ import io.fabric8.kubernetes.api.model.{ContainerBuilder, KeyToPathBuilder, PodBuilder} +import org.apache.hadoop.security.UserGroupInformation import org.apache.spark.deploy.kubernetes.constants._ import org.apache.spark.internal.Logging - /** * This is separated out from the HadoopConf steps API because this component can be reused to * set up the Hadoop Configuration for executors as well. @@ -74,6 +74,10 @@ private[spark] class HadoopConfBootstrapImpl( .withName(ENV_HADOOP_CONF_DIR) .withValue(HADOOP_CONF_DIR_PATH) .endEnv() + .addNewEnv() + .withName(ENV_SPARK_USER) + .withValue(UserGroupInformation.getCurrentUser.getShortUserName) + .endEnv() .build() originalPodWithMainContainer.copy( pod = hadoopSupportedPod, diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/KerberosTokenConfBootstrap.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/KerberosTokenConfBootstrap.scala index 891be079d8401..0709b3d270fed 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/KerberosTokenConfBootstrap.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/KerberosTokenConfBootstrap.scala @@ -36,7 +36,9 @@ private[spark] trait KerberosTokenBootstrapConf { private[spark] class KerberosTokenConfBootstrapImpl( secretName: String, - secretLabel: String) extends KerberosTokenBootstrapConf with Logging{ + secretLabel: String, + userName: String) extends KerberosTokenBootstrapConf with Logging{ + override def bootstrapMainContainerAndVolumes( originalPodWithMainContainer: PodWithMainContainer) @@ -62,6 +64,10 @@ private[spark] class KerberosTokenConfBootstrapImpl( .withName(ENV_HADOOP_TOKEN_FILE_LOCATION) .withValue(s"$SPARK_APP_HADOOP_CREDENTIALS_BASE_DIR/$secretLabel") .endEnv() + .addNewEnv() + .withName(ENV_SPARK_USER) + .withValue(userName) + .endEnv() .build() originalPodWithMainContainer.copy( pod = dtMountedPod, 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 8b5e4091a43c2..8f2200368893b 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 @@ -60,6 +60,7 @@ package object constants { private[spark] val ENV_DRIVER_URL = "SPARK_DRIVER_URL" private[spark] val ENV_EXECUTOR_CORES = "SPARK_EXECUTOR_CORES" private[spark] val ENV_EXECUTOR_MEMORY = "SPARK_EXECUTOR_MEMORY" + private[spark] val ENV_EXECUTOR_JAVA_OPTS = "SPARK_EXECUTOR_JAVA_OPTS" private[spark] val ENV_APPLICATION_ID = "SPARK_APPLICATION_ID" private[spark] val ENV_EXECUTOR_ID = "SPARK_EXECUTOR_ID" private[spark] val ENV_EXECUTOR_POD_IP = "SPARK_EXECUTOR_POD_IP" @@ -74,6 +75,7 @@ package object constants { private[spark] val ENV_PYSPARK_FILES = "PYSPARK_FILES" private[spark] val ENV_PYSPARK_PRIMARY = "PYSPARK_PRIMARY" private[spark] val ENV_HADOOP_TOKEN_FILE_LOCATION = "HADOOP_TOKEN_FILE_LOCATION" + private[spark] val ENV_SPARK_USER = "SPARK_USER" // Bootstrapping dependencies with the init-container private[spark] val INIT_CONTAINER_ANNOTATION = "pod.beta.kubernetes.io/init-containers" @@ -113,6 +115,9 @@ package object constants { "spark.kubernetes.kerberos.labelname" private[spark] val KERBEROS_SECRET_LABEL_PREFIX = "hadoop-tokens" + private[spark] val SPARK_HADOOP_PREFIX = "spark.hadoop." + private[spark] val HADOOP_SECURITY_AUTHENTICATION = + SPARK_HADOOP_PREFIX + "hadoop.security.authentication" // 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 1595f145c05b9..3aa7868ef4b3c 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 @@ -78,6 +78,9 @@ private[spark] class Client( private val driverJavaOptions = submissionSparkConf.get( org.apache.spark.internal.config.DRIVER_JAVA_OPTIONS) + private val isKerberosEnabled = submissionSparkConf.get(KUBERNETES_KERBEROS_SUPPORT) + private val maybeSimpleAuthentication = + if (isKerberosEnabled) s" -D$HADOOP_SECURITY_AUTHENTICATION=simple" else "" /** * Run command that initalizes a DriverSpec that will be updated after each @@ -98,7 +101,7 @@ private[spark] class Client( .getAll .map { case (confKey, confValue) => s"-D$confKey=$confValue" - }.mkString(" ") + driverJavaOptions.map(" " + _).getOrElse("") + }.mkString(" ") + driverJavaOptions.map(" " + _).getOrElse("") + maybeSimpleAuthentication val resolvedDriverContainer = new ContainerBuilder(currentDriverSpec.driverContainer) .addNewEnv() .withName(ENV_DRIVER_JAVA_OPTS) 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 624bed1a93019..f6702fe651027 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 @@ -112,7 +112,8 @@ private[spark] class HadoopKerberosKeytabResolverStep( .build() val bootstrapKerberos = new KerberosTokenConfBootstrapImpl( HADOOP_KERBEROS_SECRET_NAME, - initialTokenLabelName) + initialTokenLabelName, + jobUserUGI.getShortUserName) val withKerberosEnvPod = bootstrapKerberos.bootstrapMainContainerAndVolumes( PodWithMainContainer( hadoopConfigSpec.driverPod, 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 1bd8dd84bcb2b..957be6dfd4634 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 @@ -16,6 +16,8 @@ */ package org.apache.spark.deploy.kubernetes.submit.submitsteps.hadoopsteps +import org.apache.hadoop.security.UserGroupInformation + import org.apache.spark.SparkConf import org.apache.spark.deploy.kubernetes.{KerberosTokenConfBootstrapImpl, PodWithMainContainer} import org.apache.spark.deploy.kubernetes.constants._ @@ -34,7 +36,8 @@ private[spark] class HadoopKerberosSecretResolverStep( override def configureContainers(hadoopConfigSpec: HadoopConfigSpec): HadoopConfigSpec = { val bootstrapKerberos = new KerberosTokenConfBootstrapImpl( tokenSecretName, - tokenLabelName) + tokenLabelName, + UserGroupInformation.getCurrentUser.getShortUserName) val withKerberosEnvPod = bootstrapKerberos.bootstrapMainContainerAndVolumes( PodWithMainContainer( hadoopConfigSpec.driverPod, 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 50b451ea6db7c..2edaab217fa7a 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 @@ -21,6 +21,7 @@ import java.io.File import org.apache.spark.SparkConf import org.apache.spark.deploy.kubernetes.{HadoopConfBootstrapImpl, OptionRequirements} import org.apache.spark.deploy.kubernetes.config._ +import org.apache.spark.internal.Logging /** @@ -30,15 +31,16 @@ private[spark] class HadoopStepsOrchestrator( namespace: String, hadoopConfigMapName: String, submissionSparkConf: SparkConf, - hadoopConfDir: String) { - private val isKerberosEnabled = submissionSparkConf.get(KUBERNETES_KERBEROS_SUPPORT) - private val maybePrincipal = submissionSparkConf.get(KUBERNETES_KERBEROS_PRINCIPAL) - private val maybeKeytab = submissionSparkConf.get(KUBERNETES_KERBEROS_KEYTAB) - .map(k => new File(k)) - private val maybeExistingSecret = submissionSparkConf.get(KUBERNETES_KERBEROS_DT_SECRET_NAME) - private val maybeExistingSecretLabel = - submissionSparkConf.get(KUBERNETES_KERBEROS_DT_SECRET_LABEL) - private val hadoopConfigurationFiles = getHadoopConfFiles(hadoopConfDir) + hadoopConfDir: String) extends Logging{ + private val isKerberosEnabled = submissionSparkConf.get(KUBERNETES_KERBEROS_SUPPORT) + private val maybePrincipal = submissionSparkConf.get(KUBERNETES_KERBEROS_PRINCIPAL) + private val maybeKeytab = submissionSparkConf.get(KUBERNETES_KERBEROS_KEYTAB) + .map(k => new File(k)) + private val maybeExistingSecret = submissionSparkConf.get(KUBERNETES_KERBEROS_DT_SECRET_NAME) + private val maybeExistingSecretLabel = + submissionSparkConf.get(KUBERNETES_KERBEROS_DT_SECRET_LABEL) + private val hadoopConfigurationFiles = getHadoopConfFiles(hadoopConfDir) + logInfo(s"Hadoop Conf directory: $hadoopConfDir") require(maybeKeytab.forall( _ => isKerberosEnabled ), "You must enable Kerberos support if you are specifying a Kerberos Keytab") 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 5cef4d5de984d..e29368651e2a1 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 @@ -25,6 +25,7 @@ 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} +import org.apache.spark.util.Utils private[spark] class KubernetesClusterManager extends ExternalClusterManager with Logging { import org.apache.spark.SparkContext @@ -91,7 +92,8 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit } yield { new KerberosTokenConfBootstrapImpl( secretName, - secretLabel) + secretLabel, + Utils.getCurrentUserName) } if (maybeConfigMap.isEmpty) { logWarning("The executor's init-container config map was not specified. Executors will" + 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 43b8cb0f2b3d7..8b7f5708f4751 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 @@ -72,10 +72,14 @@ private[spark] class KubernetesClusterSchedulerBackend( private val executorsToRemove = Collections.newSetFromMap[String]( new ConcurrentHashMap[String, java.lang.Boolean]()).asScala + private val executorExtraJavaOpts = conf.get( + org.apache.spark.internal.config.EXECUTOR_JAVA_OPTIONS) private val executorExtraClasspath = conf.get( org.apache.spark.internal.config.EXECUTOR_CLASS_PATH) private val executorJarsDownloadDir = conf.get(INIT_CONTAINER_JARS_DOWNLOAD_LOCATION) - + private val isKerberosEnabled = conf.get(KUBERNETES_KERBEROS_SUPPORT) + private val maybeSimpleAuthentication = + if (isKerberosEnabled) s" -D$HADOOP_SECURITY_AUTHENTICATION=simple" else "" private val executorLabels = ConfigurationUtils.combinePrefixedKeyValuePairsWithDeprecatedConf( conf, KUBERNETES_EXECUTOR_LABEL_PREFIX, @@ -451,6 +455,12 @@ private[spark] class KubernetesClusterSchedulerBackend( val executorCpuQuantity = new QuantityBuilder(false) .withAmount(executorCores.toString) .build() + val executorJavaOpts = executorExtraJavaOpts.getOrElse("") + maybeSimpleAuthentication + val executorJavaOptsEnv = if (executorJavaOpts.nonEmpty) { + Some(new EnvVarBuilder() + .withName(ENV_EXECUTOR_JAVA_OPTS) + .withValue(executorJavaOpts) + .build()) } else None val executorExtraClasspathEnv = executorExtraClasspath.map { cp => new EnvVarBuilder() .withName(ENV_EXECUTOR_EXTRA_CLASSPATH) @@ -499,6 +509,7 @@ private[spark] class KubernetesClusterSchedulerBackend( .endResources() .addAllToEnv(requiredEnv.asJava) .addToEnv(executorExtraClasspathEnv.toSeq: _*) + .addToEnv(executorJavaOptsEnv.toSeq: _*) .withPorts(requiredPorts.asJava) .build() diff --git a/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/driver-py/Dockerfile b/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/driver-py/Dockerfile index 6dcc7511c0dd9..8454b7465564a 100644 --- a/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/driver-py/Dockerfile +++ b/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/driver-py/Dockerfile @@ -42,6 +42,7 @@ CMD SPARK_CLASSPATH="${SPARK_HOME}/jars/*" && \ if ! [ -z ${SPARK_MOUNTED_CLASSPATH+x} ]; then SPARK_CLASSPATH="$SPARK_MOUNTED_CLASSPATH:$SPARK_CLASSPATH"; fi && \ if ! [ -z ${SPARK_SUBMIT_EXTRA_CLASSPATH+x} ]; then SPARK_CLASSPATH="$SPARK_SUBMIT_EXTRA_CLASSPATH:$SPARK_CLASSPATH"; fi && \ if ! [ -z ${SPARK_EXTRA_CLASSPATH+x} ]; then SPARK_CLASSPATH="$SPARK_EXTRA_CLASSPATH:$SPARK_CLASSPATH"; fi && \ + if ! [ -z ${HADOOP_CONF_DIR+x} ]; then SPARK_CLASSPATH="$HADOOP_CONF_DIR:$SPARK_CLASSPATH"; fi && \ if ! [ -z ${SPARK_MOUNTED_FILES_DIR} ]; then cp -R "$SPARK_MOUNTED_FILES_DIR/." .; fi && \ exec /sbin/tini -- ${JAVA_HOME}/bin/java $SPARK_DRIVER_JAVA_OPTS -cp $SPARK_CLASSPATH \ -Xms$SPARK_DRIVER_MEMORY -Xmx$SPARK_DRIVER_MEMORY \ diff --git a/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/driver/Dockerfile b/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/driver/Dockerfile index 6bbff8ef64a0f..2d88bd977d993 100644 --- a/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/driver/Dockerfile +++ b/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/driver/Dockerfile @@ -27,5 +27,6 @@ CMD SPARK_CLASSPATH="${SPARK_HOME}/jars/*" && \ if ! [ -z ${SPARK_MOUNTED_CLASSPATH+x} ]; then SPARK_CLASSPATH="$SPARK_MOUNTED_CLASSPATH:$SPARK_CLASSPATH"; fi && \ if ! [ -z ${SPARK_SUBMIT_EXTRA_CLASSPATH+x} ]; then SPARK_CLASSPATH="$SPARK_SUBMIT_EXTRA_CLASSPATH:$SPARK_CLASSPATH"; fi && \ if ! [ -z ${SPARK_EXTRA_CLASSPATH+x} ]; then SPARK_CLASSPATH="$SPARK_EXTRA_CLASSPATH:$SPARK_CLASSPATH"; fi && \ + if ! [ -z ${HADOOP_CONF_DIR+x} ]; then SPARK_CLASSPATH="$HADOOP_CONF_DIR:$SPARK_CLASSPATH"; fi && \ if ! [ -z ${SPARK_MOUNTED_FILES_DIR} ]; then cp -R "$SPARK_MOUNTED_FILES_DIR/." .; fi && \ exec /sbin/tini -- ${JAVA_HOME}/bin/java $SPARK_DRIVER_JAVA_OPTS -cp $SPARK_CLASSPATH -Xms$SPARK_DRIVER_MEMORY -Xmx$SPARK_DRIVER_MEMORY $SPARK_DRIVER_CLASS $SPARK_DRIVER_ARGS diff --git a/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/executor-py/Dockerfile b/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/executor-py/Dockerfile index 7a65a4f879376..8cadd408744f2 100644 --- a/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/executor-py/Dockerfile +++ b/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/executor-py/Dockerfile @@ -42,5 +42,6 @@ ENV PYTHONPATH ${SPARK_HOME}/python/:${SPARK_HOME}/python/lib/py4j-0.10.4-src.zi CMD SPARK_CLASSPATH="${SPARK_HOME}/jars/*" && \ if ! [ -z ${SPARK_MOUNTED_CLASSPATH}+x} ]; then SPARK_CLASSPATH="$SPARK_MOUNTED_CLASSPATH:$SPARK_CLASSPATH"; fi && \ if ! [ -z ${SPARK_EXECUTOR_EXTRA_CLASSPATH+x} ]; then SPARK_CLASSPATH="$SPARK_EXECUTOR_EXTRA_CLASSPATH:$SPARK_CLASSPATH"; fi && \ + if ! [ -z ${HADOOP_CONF_DIR+x} ]; then SPARK_CLASSPATH="$HADOOP_CONF_DIR:$SPARK_CLASSPATH"; fi && \ if ! [ -z ${SPARK_MOUNTED_FILES_DIR} ]; then cp -R "$SPARK_MOUNTED_FILES_DIR/." .; fi && \ - exec /sbin/tini -- ${JAVA_HOME}/bin/java -Dspark.executor.port=$SPARK_EXECUTOR_PORT -Xms$SPARK_EXECUTOR_MEMORY -Xmx$SPARK_EXECUTOR_MEMORY -cp $SPARK_CLASSPATH org.apache.spark.executor.CoarseGrainedExecutorBackend --driver-url $SPARK_DRIVER_URL --executor-id $SPARK_EXECUTOR_ID --cores $SPARK_EXECUTOR_CORES --app-id $SPARK_APPLICATION_ID --hostname $SPARK_EXECUTOR_POD_IP \ No newline at end of file + exec /sbin/tini -- ${JAVA_HOME}/bin/java $SPARK_EXECUTOR_JAVA_OPTS -Dspark.executor.port=$SPARK_EXECUTOR_PORT -Xms$SPARK_EXECUTOR_MEMORY -Xmx$SPARK_EXECUTOR_MEMORY -cp $SPARK_CLASSPATH org.apache.spark.executor.CoarseGrainedExecutorBackend --driver-url $SPARK_DRIVER_URL --executor-id $SPARK_EXECUTOR_ID --cores $SPARK_EXECUTOR_CORES --app-id $SPARK_APPLICATION_ID --hostname $SPARK_EXECUTOR_POD_IP \ No newline at end of file diff --git a/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/executor/Dockerfile b/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/executor/Dockerfile index b3b0acc3b64b8..3ff61e605c349 100644 --- a/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/executor/Dockerfile +++ b/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/executor/Dockerfile @@ -28,5 +28,6 @@ CMD SPARK_CLASSPATH="${SPARK_HOME}/jars/*" && \ if ! [ -z ${SPARK_MOUNTED_CLASSPATH}+x} ]; then SPARK_CLASSPATH="$SPARK_MOUNTED_CLASSPATH:$SPARK_CLASSPATH"; fi && \ if ! [ -z ${SPARK_EXECUTOR_EXTRA_CLASSPATH+x} ]; then SPARK_CLASSPATH="$SPARK_EXECUTOR_EXTRA_CLASSPATH:$SPARK_CLASSPATH"; fi && \ if ! [ -z ${SPARK_EXTRA_CLASSPATH+x} ]; then SPARK_CLASSPATH="$SPARK_EXTRA_CLASSPATH:$SPARK_CLASSPATH"; fi && \ + if ! [ -z ${HADOOP_CONF_DIR+x} ]; then SPARK_CLASSPATH="$HADOOP_CONF_DIR:$SPARK_CLASSPATH"; fi && \ if ! [ -z ${SPARK_MOUNTED_FILES_DIR} ]; then cp -R "$SPARK_MOUNTED_FILES_DIR/." .; fi && \ - exec /sbin/tini -- ${JAVA_HOME}/bin/java -Dspark.executor.port=$SPARK_EXECUTOR_PORT -Xms$SPARK_EXECUTOR_MEMORY -Xmx$SPARK_EXECUTOR_MEMORY -cp $SPARK_CLASSPATH org.apache.spark.executor.CoarseGrainedExecutorBackend --driver-url $SPARK_DRIVER_URL --executor-id $SPARK_EXECUTOR_ID --cores $SPARK_EXECUTOR_CORES --app-id $SPARK_APPLICATION_ID --hostname $SPARK_EXECUTOR_POD_IP + exec /sbin/tini -- ${JAVA_HOME}/bin/java $SPARK_EXECUTOR_JAVA_OPTS -Dspark.executor.port=$SPARK_EXECUTOR_PORT -Xms$SPARK_EXECUTOR_MEMORY -Xmx$SPARK_EXECUTOR_MEMORY -cp $SPARK_CLASSPATH org.apache.spark.executor.CoarseGrainedExecutorBackend --driver-url $SPARK_DRIVER_URL --executor-id $SPARK_EXECUTOR_ID --cores $SPARK_EXECUTOR_CORES --app-id $SPARK_APPLICATION_ID --hostname $SPARK_EXECUTOR_POD_IP diff --git a/resource-managers/kubernetes/integration-tests/kerberos-yml/test-env.sh b/resource-managers/kubernetes/integration-tests/kerberos-yml/test-env.sh index 1248d57bf25a8..2b1d0c4727b25 100644 --- a/resource-managers/kubernetes/integration-tests/kerberos-yml/test-env.sh +++ b/resource-managers/kubernetes/integration-tests/kerberos-yml/test-env.sh @@ -15,6 +15,7 @@ until /usr/bin/kinit -kt /var/keytabs/hdfs.keytab hdfs/nn.${NAMESPACE}.svc.clust --kubernetes-namespace ${NAMESPACE} \ --conf spark.executor.instances=1 \ --conf spark.app.name=spark-hdfs \ + --conf spark.driver.extraClassPath=/opt/spark/hconf/core-site.xml:/opt/spark/hconf/hdfs-site.xml:/opt/spark/hconf/yarn-site.xml:/etc/krb5.conf \ --conf spark.kubernetes.driver.docker.image=spark-driver:latest \ --conf spark.kubernetes.executor.docker.image=spark-executor:latest \ --conf spark.kubernetes.initcontainer.docker.image=spark-init:latest \ diff --git a/resource-managers/kubernetes/integration-tests/pom.xml b/resource-managers/kubernetes/integration-tests/pom.xml index ab9f356b2d680..1d837b4db14ed 100644 --- a/resource-managers/kubernetes/integration-tests/pom.xml +++ b/resource-managers/kubernetes/integration-tests/pom.xml @@ -268,7 +268,7 @@ ${project.build.directory}/docker/hconf - src/test/resources + test-data/hadoop-conf 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 4df5ef83806aa..d1547a0d58bf3 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 @@ -92,7 +92,7 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { Seq("HADOOP_CONF_DIR defined. Mounting HDFS specific .xml files", "Pi is roughly 3"), Array("5"), Seq.empty[String], - Some("src/test/resources")) + Some("test-data/simple-hadoop-conf")) } test("Secure HDFS test with HDFS keytab") { 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 b436256cf8199..86dcaa4203032 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 @@ -41,7 +41,7 @@ private[spark] class KerberosUtils( private val regex = "REPLACE_ME".r private val regexDP = "# default_ccache_name = MEMORY".r private val defaultCacheDP = "default_ccache_name = KRBCONF" - private def locationResolver(loc: String) = s"src/test/resources/$loc" + private def locationResolver(loc: String) = s"test-data/hadoop-conf/$loc" private val kerberosFiles = Seq("krb5.conf", "core-site.xml", "hdfs-site.xml") private val kerberosConfTupList = kerberosFiles.map { file => diff --git a/resource-managers/kubernetes/integration-tests/src/test/resources/core-site.xml b/resource-managers/kubernetes/integration-tests/test-data/hadoop-conf/core-site.xml similarity index 100% rename from resource-managers/kubernetes/integration-tests/src/test/resources/core-site.xml rename to resource-managers/kubernetes/integration-tests/test-data/hadoop-conf/core-site.xml diff --git a/resource-managers/kubernetes/integration-tests/src/test/resources/hdfs-site.xml b/resource-managers/kubernetes/integration-tests/test-data/hadoop-conf/hdfs-site.xml similarity index 100% rename from resource-managers/kubernetes/integration-tests/src/test/resources/hdfs-site.xml rename to resource-managers/kubernetes/integration-tests/test-data/hadoop-conf/hdfs-site.xml diff --git a/resource-managers/kubernetes/integration-tests/src/test/resources/krb5.conf b/resource-managers/kubernetes/integration-tests/test-data/hadoop-conf/krb5.conf similarity index 100% rename from resource-managers/kubernetes/integration-tests/src/test/resources/krb5.conf rename to resource-managers/kubernetes/integration-tests/test-data/hadoop-conf/krb5.conf diff --git a/resource-managers/kubernetes/integration-tests/src/test/resources/yarn-site.xml b/resource-managers/kubernetes/integration-tests/test-data/hadoop-conf/yarn-site.xml similarity index 100% rename from resource-managers/kubernetes/integration-tests/src/test/resources/yarn-site.xml rename to resource-managers/kubernetes/integration-tests/test-data/hadoop-conf/yarn-site.xml diff --git a/resource-managers/kubernetes/integration-tests/test-data/simple-hadoop-conf/core-site.xml b/resource-managers/kubernetes/integration-tests/test-data/simple-hadoop-conf/core-site.xml new file mode 100644 index 0000000000000..67975bd180c26 --- /dev/null +++ b/resource-managers/kubernetes/integration-tests/test-data/simple-hadoop-conf/core-site.xml @@ -0,0 +1,24 @@ + + + + + + + + + fs.defaultFS + hdfs://nn.REPLACE_ME.svc.cluster.local:9000 + + \ No newline at end of file diff --git a/resource-managers/kubernetes/integration-tests/test-data/simple-hadoop-conf/hdfs-site.xml b/resource-managers/kubernetes/integration-tests/test-data/simple-hadoop-conf/hdfs-site.xml new file mode 100644 index 0000000000000..76fc9c68fa375 --- /dev/null +++ b/resource-managers/kubernetes/integration-tests/test-data/simple-hadoop-conf/hdfs-site.xml @@ -0,0 +1,24 @@ + + + + + + + + + dfs.replication + 1 + + From 7a0b4e4abf4648533f1b5ba6ad074bcbba351e95 Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Wed, 16 Aug 2017 10:15:20 -0400 Subject: [PATCH 27/37] Adding unit test and one more integration test --- .../spark/deploy/kubernetes/config.scala | 2 +- .../HadoopKerberosKeytabResolverStep.scala | 2 +- .../hadoopsteps/HadoopStepsOrchestrator.scala | 1 - ...sourceStagingServerSecretPluginSuite.scala | 2 +- .../kubernetes/submit/ClientSuite.scala | 16 +-- .../submit/submitsteps/PythonStepSuite.scala | 2 +- .../HadoopConfMounterStepSuite.scala | 102 ++++++++++++++++++ ...adoopKerberosKeytabResolverStepSuite.scala | 77 +++++++++++++ ...adoopKerberosSecretResolverStepSuite.scala | 63 +++++++++++ .../HadoopStepsOrchestratorSuite.scala | 72 +++++++++++++ ...eInitContainerConfigurationStepSuite.scala | 3 +- ...ittedResourcesInitContainerStepSuite.scala | 3 +- .../src/main/docker/kerberos-test/Dockerfile | 1 + .../kerberos-yml/kerberos-test2.yml | 25 +++++ .../kerberos-yml/test-env.sh | 2 +- .../kerberos-yml/test-env2.sh | 25 +++++ .../kubernetes/integration-tests/pom.xml | 1 + .../KerberosTestPodLauncher.scala | 7 +- .../integrationtest/KubernetesSuite.scala | 47 +++++++- 19 files changed, 432 insertions(+), 21 deletions(-) create mode 100644 resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopConfMounterStepSuite.scala create mode 100644 resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopKerberosKeytabResolverStepSuite.scala create mode 100644 resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopKerberosSecretResolverStepSuite.scala create mode 100644 resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopStepsOrchestratorSuite.scala create mode 100644 resource-managers/kubernetes/integration-tests/kerberos-yml/kerberos-test2.yml create mode 100644 resource-managers/kubernetes/integration-tests/kerberos-yml/test-env2.sh 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 efbf931838708..49f993c2236ac 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 @@ -512,7 +512,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") + ConfigBuilder("spark.kubernetes.kerberos.enabled") .doc("Specify whether your job is a job that will require a Delegation Token to access HDFS") .booleanConf .createWithDefault(false) 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 f6702fe651027..127c052e208a4 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,7 +41,7 @@ import org.apache.spark.internal.Logging * 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 + * you can run UGI.loginUserFromKeytabAndReturnUGI and by running .doAs run * as the logged into user instead of the current user. With the Job User principal * you then retrieve the delegation token from the NameNode and store values in * DelegationToken. Lastly, the class puts the data into a secret. All this is 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 2edaab217fa7a..8bc2906965598 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 @@ -23,7 +23,6 @@ import org.apache.spark.deploy.kubernetes.{HadoopConfBootstrapImpl, OptionRequir import org.apache.spark.deploy.kubernetes.config._ import org.apache.spark.internal.Logging - /** * Returns the complete ordered list of steps required to configure the hadoop configurations. */ diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/InitContainerResourceStagingServerSecretPluginSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/InitContainerResourceStagingServerSecretPluginSuite.scala index 597bcdb416fc0..14c97adee87ed 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/InitContainerResourceStagingServerSecretPluginSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/InitContainerResourceStagingServerSecretPluginSuite.scala @@ -23,7 +23,7 @@ import scala.collection.JavaConverters._ import org.apache.spark.deploy.kubernetes.constants._ import org.apache.spark.SparkFunSuite -class InitContainerResourceStagingServerSecretPluginSuite extends SparkFunSuite with BeforeAndAfter{ +private[spark] class InitContainerResourceStagingServerSecretPluginSuite extends SparkFunSuite with BeforeAndAfter{ private val INIT_CONTAINER_SECRET_NAME = "init-secret" private val INIT_CONTAINER_SECRET_MOUNT = "/tmp/secret" diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/ClientSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/ClientSuite.scala index 965ee75c248b8..29f25ffc7d795 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/ClientSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/ClientSuite.scala @@ -17,22 +17,22 @@ package org.apache.spark.deploy.kubernetes.submit import com.google.common.collect.Iterables -import io.fabric8.kubernetes.api.model.{ContainerBuilder, DoneablePod, HasMetadata, Pod, PodBuilder, PodList, Secret, SecretBuilder} +import io.fabric8.kubernetes.api.model._ +import io.fabric8.kubernetes.client.dsl.{MixedOperation, NamespaceListVisitFromServerGetDeleteRecreateWaitApplicable, PodResource} import io.fabric8.kubernetes.client.{KubernetesClient, Watch} -import io.fabric8.kubernetes.client.dsl.{MixedOperation, NamespaceListVisitFromServerGetDeleteRecreateWaitApplicable, NamespaceVisitFromServerGetWatchDeleteRecreateWaitApplicable, PodResource, Resource} -import org.mockito.{ArgumentCaptor, Mock, MockitoAnnotations} +import org.apache.spark.deploy.kubernetes.constants._ +import org.apache.spark.deploy.kubernetes.submit.submitsteps.{DriverConfigurationStep, KubernetesDriverSpec} +import org.apache.spark.{SparkConf, SparkFunSuite} import org.mockito.Mockito.{doReturn, verify, when} import org.mockito.invocation.InvocationOnMock import org.mockito.stubbing.Answer +import org.mockito.{ArgumentCaptor, Mock, MockitoAnnotations} import org.scalatest.BeforeAndAfter import org.scalatest.mock.MockitoSugar._ -import scala.collection.JavaConverters._ -import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.deploy.kubernetes.constants._ -import org.apache.spark.deploy.kubernetes.submit.submitsteps.{DriverConfigurationStep, KubernetesDriverSpec} +import scala.collection.JavaConverters._ -class ClientSuite extends SparkFunSuite with BeforeAndAfter { +private[spark] class ClientSuite extends SparkFunSuite with BeforeAndAfter { private val DRIVER_POD_UID = "pod-id" private val DRIVER_POD_API_VERSION = "v1" diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/PythonStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/PythonStepSuite.scala index ce0dcee6acc46..02277bd86bf39 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/PythonStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/PythonStepSuite.scala @@ -22,7 +22,7 @@ import scala.collection.JavaConverters._ import org.apache.spark.{SparkConf, SparkFunSuite} -class PythonStepSuite extends SparkFunSuite with BeforeAndAfter { +private[spark] class PythonStepSuite extends SparkFunSuite with BeforeAndAfter { private val FILE_DOWNLOAD_PATH = "/var/data/spark-files" private val PYSPARK_FILES = Seq( "hdfs://localhost:9000/app/files/file1.py", diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopConfMounterStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopConfMounterStepSuite.scala new file mode 100644 index 0000000000000..876675d5a4d96 --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopConfMounterStepSuite.scala @@ -0,0 +1,102 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.kubernetes.submit.submitsteps.hadoopsteps + +import java.io.File +import java.util.UUID + +import scala.collection.JavaConverters._ + +import com.google.common.io.Files +import io.fabric8.kubernetes.api.model._ +import org.apache.commons.io.FileUtils.readFileToString +import org.mockito.{Mock, MockitoAnnotations} +import org.mockito.Matchers.any +import org.mockito.Mockito.when +import org.mockito.invocation.InvocationOnMock +import org.mockito.stubbing.Answer +import org.scalatest.BeforeAndAfter + +import org.apache.spark.SparkFunSuite +import org.apache.spark.deploy.kubernetes.{HadoopConfBootstrap, PodWithMainContainer} +import org.apache.spark.deploy.kubernetes.constants.HADOOP_CONF_DIR_LOC +import org.apache.spark.util.Utils + + +private[spark] class HadoopConfMounterStepSuite extends SparkFunSuite with BeforeAndAfter{ + private val CONFIG_MAP_NAME = "config-map" + private val HADOOP_CONF_DIR_VAL = "/etc/hadoop" + private val POD_LABEL = Map("bootstrap" -> "true") + private val DRIVER_CONTAINER_NAME = "driver-container" + private val TEMP_HADOOP_FILE = createTempFile("core-site.xml") + private val HADOOP_FILES = Seq(TEMP_HADOOP_FILE) + + @Mock + private var hadoopConfBootstrap : HadoopConfBootstrap = _ + + before { + MockitoAnnotations.initMocks(this) + when(hadoopConfBootstrap.bootstrapMainContainerAndVolumes( + any[PodWithMainContainer])).thenAnswer(new Answer[PodWithMainContainer] { + override def answer(invocation: InvocationOnMock) : PodWithMainContainer = { + val pod = invocation.getArgumentAt(0, classOf[PodWithMainContainer]) + pod.copy( + pod = + new PodBuilder(pod.pod) + .withNewMetadata() + .addToLabels("bootstrap", "true") + .endMetadata() + .withNewSpec().endSpec() + .build(), + mainContainer = + new ContainerBuilder() + .withName(DRIVER_CONTAINER_NAME).build() + )}}) + } + + test("Test of mounting hadoop_conf_dir files into HadoopConfigSpec") { + val hadoopConfStep = new HadoopConfMounterStep( + CONFIG_MAP_NAME, + HADOOP_FILES, + hadoopConfBootstrap, + HADOOP_CONF_DIR_VAL) + val expectedDriverSparkConf = Map(HADOOP_CONF_DIR_LOC -> HADOOP_CONF_DIR_VAL) + val expectedConfigMap = Map( + TEMP_HADOOP_FILE.toPath.getFileName.toString -> + readFileToString(TEMP_HADOOP_FILE) + ) + val hadoopConfSpec = HadoopConfigSpec( + Map.empty[String, String], + new Pod(), + new Container(), + Map.empty[String, String], + None, + "", + "") + val returnContainerSpec = hadoopConfStep.configureContainers(hadoopConfSpec) + assert(expectedDriverSparkConf === returnContainerSpec.additionalDriverSparkConf) + assert(returnContainerSpec.driverContainer.getName == DRIVER_CONTAINER_NAME) + assert(returnContainerSpec.driverPod.getMetadata.getLabels.asScala === POD_LABEL) + assert(returnContainerSpec.configMapProperties === expectedConfigMap) + } + private def createTempFile(contents: String): File = { + val dir = Utils.createTempDir() + val file = new File(dir, s"${UUID.randomUUID().toString}") + Files.write(contents.getBytes, file) + file + } +} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopKerberosKeytabResolverStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopKerberosKeytabResolverStepSuite.scala new file mode 100644 index 0000000000000..f0496c588157a --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopKerberosKeytabResolverStepSuite.scala @@ -0,0 +1,77 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.kubernetes.submit.submitsteps.hadoopsteps + +import java.io.File +import java.util.UUID + +import scala.collection.JavaConverters._ + +import com.google.common.io.Files +import io.fabric8.kubernetes.api.model._ + +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.deploy.kubernetes.constants._ +import org.apache.spark.util.Utils + + + +private[spark] class HadoopKerberosKeytabResolverStepSuite extends SparkFunSuite { + private val POD_LABEL = Map("bootstrap" -> "true") + private val DRIVER_CONTAINER_NAME = "driver-container" + private val TEMP_KEYTAB_FILE = createTempFile("keytab") + private val KERB_PRINCIPAL = "user@k8s.com" + + // TODO: Require mocking of UGI methods + test("Testing keytab login") { + val keytabStep = new HadoopKerberosKeytabResolverStep( + new SparkConf(), + Some(KERB_PRINCIPAL), + Some(TEMP_KEYTAB_FILE)) + val hadoopConfSpec = HadoopConfigSpec( + Map.empty[String, String], + new PodBuilder() + .withNewMetadata() + .addToLabels("bootstrap", "true") + .endMetadata() + .withNewSpec().endSpec() + .build(), + new ContainerBuilder().withName(DRIVER_CONTAINER_NAME).build(), + Map.empty[String, String], + None, + "", + "") + val returnContainerSpec = keytabStep.configureContainers(hadoopConfSpec) + assert(returnContainerSpec.additionalDriverSparkConf(HADOOP_KERBEROS_CONF_LABEL) + .contains(KERBEROS_SECRET_LABEL_PREFIX)) + assert(returnContainerSpec.additionalDriverSparkConf(HADOOP_KERBEROS_CONF_SECRET) === + HADOOP_KERBEROS_SECRET_NAME) + assert(returnContainerSpec.driverContainer.getName == DRIVER_CONTAINER_NAME) + assert(returnContainerSpec.driverPod.getMetadata.getLabels.asScala === POD_LABEL) + assert(returnContainerSpec.dtSecretLabel.contains(KERBEROS_SECRET_LABEL_PREFIX)) + assert(returnContainerSpec.dtSecretName === HADOOP_KERBEROS_SECRET_NAME) + assert(returnContainerSpec.dtSecret.nonEmpty) + assert(returnContainerSpec.dtSecret.get.getMetadata.getName === HADOOP_KERBEROS_SECRET_NAME) + } + + private def createTempFile(contents: String): File = { + val dir = Utils.createTempDir() + val file = new File(dir, s"${UUID.randomUUID().toString}") + Files.write(contents.getBytes, file) + file + } +} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopKerberosSecretResolverStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopKerberosSecretResolverStepSuite.scala new file mode 100644 index 0000000000000..b4305aee1e049 --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopKerberosSecretResolverStepSuite.scala @@ -0,0 +1,63 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.kubernetes.submit.submitsteps.hadoopsteps + +import scala.collection.JavaConverters._ + +import io.fabric8.kubernetes.api.model._ + +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.deploy.kubernetes.constants._ + +private[spark] class HadoopKerberosSecretResolverStepSuite extends SparkFunSuite { + private val CONFIG_MAP_NAME = "config-map" + private val HADOOP_CONF_DIR_VAL = "/etc/hadoop" + private val POD_LABEL = Map("bootstrap" -> "true") + private val DRIVER_CONTAINER_NAME = "driver-container" + private val TOKEN_SECRET_NAME = "secretName" + private val TOKEN_SECRET_LABEL = "secretLabel" + + test("Testing kerberos with Secret") { + val keytabStep = new HadoopKerberosSecretResolverStep( + new SparkConf(), + TOKEN_SECRET_NAME, + TOKEN_SECRET_LABEL) + val expectedDriverSparkConf = Map( + HADOOP_KERBEROS_CONF_LABEL -> TOKEN_SECRET_LABEL, + HADOOP_KERBEROS_CONF_SECRET -> TOKEN_SECRET_NAME) + val hadoopConfSpec = HadoopConfigSpec( + Map.empty[String, String], + new PodBuilder() + .withNewMetadata() + .addToLabels("bootstrap", "true") + .endMetadata() + .withNewSpec().endSpec() + .build(), + new ContainerBuilder().withName(DRIVER_CONTAINER_NAME).build(), + Map.empty[String, String], + None, + "", + "") + val returnContainerSpec = keytabStep.configureContainers(hadoopConfSpec) + assert(expectedDriverSparkConf === returnContainerSpec.additionalDriverSparkConf) + assert(returnContainerSpec.driverContainer.getName == DRIVER_CONTAINER_NAME) + assert(returnContainerSpec.driverPod.getMetadata.getLabels.asScala === POD_LABEL) + assert(returnContainerSpec.dtSecret === None) + assert(returnContainerSpec.dtSecretLabel === TOKEN_SECRET_LABEL) + assert(returnContainerSpec.dtSecretName === TOKEN_SECRET_NAME) + } +} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopStepsOrchestratorSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopStepsOrchestratorSuite.scala new file mode 100644 index 0000000000000..1f8039ddeb2b0 --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopStepsOrchestratorSuite.scala @@ -0,0 +1,72 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.kubernetes.submit.submitsteps.hadoopsteps + +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.deploy.kubernetes.config._ + +private[spark] class HadoopStepsOrchestratorSuite extends SparkFunSuite { + private val NAMESPACE = "testNamespace" + private val HADOOP_CONFIG_MAP = "hadoop-config-map" + private val HADOOP_CONF_DIR_VAL = "/etc/hadoop/conf" + + test("Testing without Kerberos") { + val sparkTestConf = new SparkConf(true) + .set(KUBERNETES_KERBEROS_SUPPORT, false) + val hadoopOrchestrator = new HadoopStepsOrchestrator( + NAMESPACE, + HADOOP_CONFIG_MAP, + sparkTestConf, + HADOOP_CONF_DIR_VAL) + val steps = hadoopOrchestrator.getHadoopSteps() + assert(steps.length === 1) + assert(steps.head.isInstanceOf[HadoopConfMounterStep]) + assert(true) + } + +// test("Testing with Keytab Kerberos Login") { +// val sparkTestConf2 = new SparkConf(true) +// .set(KUBERNETES_KERBEROS_SUPPORT, true) +// .set(KUBERNETES_KERBEROS_KEYTAB, "keytab.file") +// .set(KUBERNETES_KERBEROS_PRINCIPAL, "user@kerberos") +// val hadoopOrchestrator = new HadoopStepsOrchestrator( +// NAMESPACE, +// HADOOP_CONFIG_MAP, +// sparkTestConf2, +// HADOOP_CONF_DIR_VAL) +// val steps = hadoopOrchestrator.getHadoopSteps() +// assert(steps.length === 2) +// assert(steps.head.isInstanceOf[HadoopConfMounterStep]) +// assert(steps(1).isInstanceOf[HadoopKerberosKeytabResolverStep]) +// } + +// test("Testing with Keytab Kerberos Login") { +// val sparkTestConf3 = new SparkConf(true) +// .set(KUBERNETES_KERBEROS_SUPPORT, true) +// .set(KUBERNETES_KERBEROS_DT_SECRET_NAME, "dtSecret") +// .set(KUBERNETES_KERBEROS_DT_SECRET_LABEL, "dtLabel") +// val hadoopOrchestrator = new HadoopStepsOrchestrator( +// NAMESPACE, +// HADOOP_CONFIG_MAP, +// sparkTestConf3, +// HADOOP_CONF_DIR_VAL) +// val steps = hadoopOrchestrator.getHadoopSteps() +// assert(steps.length === 2) +// assert(steps.head.isInstanceOf[HadoopConfMounterStep]) +// assert(steps(1).isInstanceOf[HadoopKerberosSecretResolverStep]) +// } +} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/BaseInitContainerConfigurationStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/BaseInitContainerConfigurationStepSuite.scala index fe1af4bc5be2a..79545a55f33b2 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/BaseInitContainerConfigurationStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/BaseInitContainerConfigurationStepSuite.scala @@ -29,7 +29,8 @@ import org.apache.spark.SparkFunSuite import org.apache.spark.deploy.kubernetes.{PodWithDetachedInitContainer, SparkPodInitContainerBootstrap} import org.apache.spark.deploy.kubernetes.config._ -class BaseInitContainerConfigurationStepSuite extends SparkFunSuite with BeforeAndAfter{ +private[spark] class BaseInitContainerConfigurationStepSuite + extends SparkFunSuite with BeforeAndAfter{ private val SPARK_JARS = Seq( "hdfs://localhost:9000/app/jars/jar1.jar", "file:///app/jars/jar2.jar") private val SPARK_FILES = Seq( diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/SubmittedResourcesInitContainerStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/SubmittedResourcesInitContainerStepSuite.scala index 2edaba93fe07f..e9c828fdb397c 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/SubmittedResourcesInitContainerStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/SubmittedResourcesInitContainerStepSuite.scala @@ -37,7 +37,8 @@ import org.apache.spark.deploy.kubernetes.constants._ import org.apache.spark.deploy.kubernetes.submit.{SubmittedDependencyUploader, SubmittedResourceIdAndSecret} import org.apache.spark.util.Utils -class SubmittedResourcesInitContainerStepSuite extends SparkFunSuite with BeforeAndAfter { +private[spark] class SubmittedResourcesInitContainerStepSuite + extends SparkFunSuite with BeforeAndAfter { private val RESOURCE_SECRET_NAME = "secret" private val JARS_RESOURCE_ID = "jarsID" private val JARS_SECRET = "jarsSecret" diff --git a/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/kerberos-test/Dockerfile b/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/kerberos-test/Dockerfile index c4ba43ad511d8..7f97c10dba8ec 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 @@ -24,4 +24,5 @@ FROM spark-base RUN apk add --no-cache --update krb5 krb5-libs COPY examples /opt/spark/examples COPY test-scripts/test-env.sh /opt/spark/ +COPY test-scripts/test-env2.sh /opt/spark/ COPY hconf /opt/spark/hconf \ No newline at end of file diff --git a/resource-managers/kubernetes/integration-tests/kerberos-yml/kerberos-test2.yml b/resource-managers/kubernetes/integration-tests/kerberos-yml/kerberos-test2.yml new file mode 100644 index 0000000000000..3aaa442bbe3d8 --- /dev/null +++ b/resource-managers/kubernetes/integration-tests/kerberos-yml/kerberos-test2.yml @@ -0,0 +1,25 @@ +apiVersion: extensions/v1beta1 +kind: Deployment +metadata: + name: kerberos-test +spec: + replicas: 1 + template: + metadata: + labels: + name: kerberos-test + spec: + containers: + - command: ["/bin/bash"] + args: ["/opt/spark/test-env2.sh"] + name: kerberos-test + image: kerberos-test:latest + imagePullPolicy: IfNotPresent + volumeMounts: + - mountPath: /var/keytabs + name: kerberos-test-keytab + restartPolicy: Always + volumes: + - name: kerberos-test-keytab + persistentVolumeClaim: + claimName: server-keytab \ No newline at end of file diff --git a/resource-managers/kubernetes/integration-tests/kerberos-yml/test-env.sh b/resource-managers/kubernetes/integration-tests/kerberos-yml/test-env.sh index 2b1d0c4727b25..056f0c5936e98 100644 --- a/resource-managers/kubernetes/integration-tests/kerberos-yml/test-env.sh +++ b/resource-managers/kubernetes/integration-tests/kerberos-yml/test-env.sh @@ -19,7 +19,7 @@ until /usr/bin/kinit -kt /var/keytabs/hdfs.keytab hdfs/nn.${NAMESPACE}.svc.clust --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.enabled=true \ --conf spark.kubernetes.kerberos.keytab=/var/keytabs/hdfs.keytab \ --conf spark.kubernetes.kerberos.principal=hdfs/nn.${NAMESPACE}.svc.cluster.local@CLUSTER.LOCAL \ --conf spark.kubernetes.driver.label.spark-app-locator=${APP_LOCATOR_LABEL} \ diff --git a/resource-managers/kubernetes/integration-tests/kerberos-yml/test-env2.sh b/resource-managers/kubernetes/integration-tests/kerberos-yml/test-env2.sh new file mode 100644 index 0000000000000..0aeb6f131bab3 --- /dev/null +++ b/resource-managers/kubernetes/integration-tests/kerberos-yml/test-env2.sh @@ -0,0 +1,25 @@ +#!/usr/bin/env bash +sed -i -e 's/#//' -e 's/default_ccache_name/# default_ccache_name/' /etc/krb5.conf +export HADOOP_OPTS="-Djava.net.preferIPv4Stack=true -Dsun.security.krb5.debug=true" +export HADOOP_JAAS_DEBUG=true +export HADOOP_ROOT_LOGGER=DEBUG,console +cp ${TMP_KRB_LOC} /etc/krb5.conf +cp ${TMP_CORE_LOC} /opt/spark/hconf/core-site.xml +cp ${TMP_HDFS_LOC} /opt/spark/hconf/hdfs-site.xml +mkdir -p /etc/krb5.conf.d +until /usr/bin/kinit -kt /var/keytabs/hdfs.keytab hdfs/nn.${NAMESPACE}.svc.cluster.local; do sleep 15; done +/opt/spark/bin/spark-submit \ + --deploy-mode cluster \ + --class ${CLASS_NAME} \ + --master k8s://${MASTER_URL} \ + --kubernetes-namespace ${NAMESPACE} \ + --conf spark.executor.instances=1 \ + --conf spark.app.name=spark-hdfs \ + --conf spark.driver.extraClassPath=/opt/spark/hconf/core-site.xml:/opt/spark/hconf/hdfs-site.xml:/opt/spark/hconf/yarn-site.xml:/etc/krb5.conf \ + --conf spark.kubernetes.driver.docker.image=spark-driver:latest \ + --conf spark.kubernetes.executor.docker.image=spark-executor:latest \ + --conf spark.kubernetes.initcontainer.docker.image=spark-init:latest \ + --conf spark.kubernetes.kerberos.enabled=true \ + --conf spark.kubernetes.driver.label.spark-app-locator=${APP_LOCATOR_LABEL} \ + ${SUBMIT_RESOURCE} \ + hdfs://nn.${NAMESPACE}.svc.cluster.local:9000/user/ifilonenko/wordcount.txt \ No newline at end of file diff --git a/resource-managers/kubernetes/integration-tests/pom.xml b/resource-managers/kubernetes/integration-tests/pom.xml index 1d837b4db14ed..fde201779aa46 100644 --- a/resource-managers/kubernetes/integration-tests/pom.xml +++ b/resource-managers/kubernetes/integration-tests/pom.xml @@ -253,6 +253,7 @@ kerberos-yml test-env.sh + test-env2.sh 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 532fc221878ff..be5f5c94721a2 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 @@ -35,7 +35,6 @@ import org.apache.spark.deploy.kubernetes.submit.ContainerNameEqualityPredicate private[spark] class KerberosTestPodLauncher( kubernetesClient: KubernetesClient, namespace: String) { - private val yamlLocation = "kerberos-yml/kerberos-test.yml" private val kerberosFiles = Seq("krb5.conf", "core-site.xml", "hdfs-site.xml") private val KRB_VOLUME = "krb5-conf" private val KRB_FILE_DIR = "/tmp" @@ -46,7 +45,11 @@ private[spark] class KerberosTestPodLauncher( .withKey(file) .withPath(file) .build()).toList - def startKerberosTest(resource: String, className: String, appLabel: String): Unit = { + def startKerberosTest( + resource: String, + className: String, + appLabel: String, + yamlLocation: String): Unit = { kubernetesClient.load(new FileInputStream(new File(yamlLocation))) .get().get(0) match { case deployment: Deployment => 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 d1547a0d58bf3..c76466d4884f2 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 @@ -98,7 +98,11 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { test("Secure HDFS test with HDFS keytab") { assume(testBackend.name == MINIKUBE_TEST_BACKEND) launchKerberizedCluster() - createKerberosTestPod(CONTAINER_LOCAL_MAIN_APP_RESOURCE, HDFS_TEST_CLASS, APP_LOCATOR_LABEL) + createKerberosTestPod( + CONTAINER_LOCAL_MAIN_APP_RESOURCE, + HDFS_TEST_CLASS, + APP_LOCATOR_LABEL, + "kerberos-yml/kerberos-test.yml") val kubernetesClient = kubernetesTestComponents.kubernetesClient val driverWatcherCache = new KerberosDriverWatcherCache( kubernetesClient, @@ -124,6 +128,39 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { } } } +// test("Secure HDFS test with kinit") { +// assume(testBackend.name == MINIKUBE_TEST_BACKEND) +// launchKerberizedCluster() +// createKerberosTestPod( +// CONTAINER_LOCAL_MAIN_APP_RESOURCE, +// HDFS_TEST_CLASS, +// APP_LOCATOR_LABEL, +// "kerberos-yml/kerberos-test2.yml") +// val kubernetesClient = kubernetesTestComponents.kubernetesClient +// val driverWatcherCache = new KerberosDriverWatcherCache( +// kubernetesClient, +// Map("spark-app-locator" -> APP_LOCATOR_LABEL)) +// driverWatcherCache.start() +// driverWatcherCache.stop() +// val expectedLogOnCompletion = Seq( +// "Returned length(s) of: 1", +// "File contents: [This is an awesome word count file]") +// val driverPod = kubernetesClient +// .pods() +// .withLabel("spark-app-locator", APP_LOCATOR_LABEL) +// .list() +// .getItems +// .get(0) +// Eventually.eventually(TIMEOUT, INTERVAL) { +// expectedLogOnCompletion.foreach { e => +// assert(kubernetesClient +// .pods() +// .withName(driverPod.getMetadata.getName) +// .getLog +// .contains(e), "The application did not complete.") +// } +// } +// } test("Run PySpark Job on file from SUBMITTER with --py-files") { assume(testBackend.name == MINIKUBE_TEST_BACKEND) @@ -309,9 +346,13 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { kerberizedHadoopClusterLauncher.launchKerberizedCluster() } - private def createKerberosTestPod(resource: String, className: String, appLabel: String): Unit = { + private def createKerberosTestPod( + resource: String, + className: String, + appLabel: String, + yamlLocation: String): Unit = { assume(testBackend.name == MINIKUBE_TEST_BACKEND) - kerberosTestLauncher.startKerberosTest(resource, className, appLabel) + kerberosTestLauncher.startKerberosTest(resource, className, appLabel, yamlLocation) } private def runSparkPiAndVerifyCompletion(appResource: String): Unit = { From 8dacb19c6749b013ac67f3f905160a9dec981918 Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Wed, 16 Aug 2017 13:43:30 -0400 Subject: [PATCH 28/37] completed unit tests w/o UGI mocking --- .../submit/HadoopConfBootstrapSuite.scala | 75 ++++++++++++++++ .../KerberosTokenConfBootstrapSuite.scala | 63 +++++++++++++ .../HadoopConfigBootstrapStepSuite.scala | 90 +++++++++++++++++++ .../HadoopStepsOrchestratorSuite.scala | 74 ++++++++------- 4 files changed, 272 insertions(+), 30 deletions(-) create mode 100644 resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/HadoopConfBootstrapSuite.scala create mode 100644 resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/KerberosTokenConfBootstrapSuite.scala create mode 100644 resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/HadoopConfigBootstrapStepSuite.scala diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/HadoopConfBootstrapSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/HadoopConfBootstrapSuite.scala new file mode 100644 index 0000000000000..219d45413431a --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/HadoopConfBootstrapSuite.scala @@ -0,0 +1,75 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.kubernetes.submit + +import java.io.File +import java.util.UUID + +import scala.collection.JavaConverters._ + +import com.google.common.io.Files +import io.fabric8.kubernetes.api.model._ + +import org.apache.spark.SparkFunSuite +import org.apache.spark.deploy.kubernetes.{HadoopConfBootstrapImpl, PodWithMainContainer} +import org.apache.spark.deploy.kubernetes.constants._ +import org.apache.spark.util.Utils + + +private[spark] class HadoopConfBootstrapSuite extends SparkFunSuite { + private val CONFIG_MAP_NAME = "config-map" + private val TEMP_HADOOP_FILE = createTempFile("core-site.xml") + private val HADOOP_FILES = Seq(TEMP_HADOOP_FILE) + + test("Test of bootstrapping hadoop_conf_dir files") { + val hadoopConfStep = new HadoopConfBootstrapImpl( + CONFIG_MAP_NAME, + HADOOP_FILES) + val expectedKeyPaths = Seq( + new KeyToPathBuilder() + .withKey(TEMP_HADOOP_FILE.toPath.getFileName.toString) + .withPath(TEMP_HADOOP_FILE.toPath.getFileName.toString) + .build()) + val expectedPod = new PodBuilder() + .editOrNewSpec() + .addNewVolume() + .withName(HADOOP_FILE_VOLUME) + .withNewConfigMap() + .withName(CONFIG_MAP_NAME) + .withItems(expectedKeyPaths.asJava) + .endConfigMap() + .endVolume() + .endSpec() + .build() + val podWithMain = PodWithMainContainer( + new PodBuilder().withNewSpec().endSpec().build(), + new Container()) + val returnedPodContainer = hadoopConfStep.bootstrapMainContainerAndVolumes(podWithMain) + assert(expectedPod === returnedPodContainer.pod) + assert(returnedPodContainer.mainContainer.getVolumeMounts.asScala.map(vm => + (vm.getName, vm.getMountPath)).head === (HADOOP_FILE_VOLUME, HADOOP_CONF_DIR_PATH)) + assert(returnedPodContainer.mainContainer.getEnv.asScala.head === + new EnvVarBuilder().withName(ENV_HADOOP_CONF_DIR).withValue(HADOOP_CONF_DIR_PATH).build()) + assert(returnedPodContainer.mainContainer.getEnv.asScala(1).getName === ENV_SPARK_USER) + } + private def createTempFile(contents: String): File = { + val dir = Utils.createTempDir() + val file = new File(dir, s"${UUID.randomUUID().toString}") + Files.write(contents.getBytes, file) + file + } +} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/KerberosTokenConfBootstrapSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/KerberosTokenConfBootstrapSuite.scala new file mode 100644 index 0000000000000..56d98a4378424 --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/KerberosTokenConfBootstrapSuite.scala @@ -0,0 +1,63 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.kubernetes.submit + +import java.io.File +import java.util.UUID + +import scala.collection.JavaConverters._ +import com.google.common.io.Files +import io.fabric8.kubernetes.api.model._ +import org.apache.spark.SparkFunSuite +import org.apache.spark.deploy.kubernetes.{HadoopConfBootstrapImpl, KerberosTokenConfBootstrapImpl, PodWithMainContainer} +import org.apache.spark.deploy.kubernetes.constants._ +import org.apache.spark.util.Utils + + +private[spark] class KerberosTokenConfBootstrapSuite extends SparkFunSuite { + private val SECRET_NAME = "dtSecret" + private val SECRET_LABEL = "dtLabel" + private val TEST_SPARK_USER = "hdfs" + + test("Test of bootstrapping kerberos secrets and env") { + val kerberosConfStep = new KerberosTokenConfBootstrapImpl( + SECRET_NAME, + SECRET_LABEL, + TEST_SPARK_USER) + val expectedPod = new PodBuilder() + .editOrNewSpec() + .addNewVolume() + .withName(SPARK_APP_HADOOP_SECRET_VOLUME_NAME) + .withNewSecret() + .withSecretName(SECRET_NAME) + .endSecret() + .endVolume() + .endSpec() + .build() + val podWithMain = PodWithMainContainer( + new PodBuilder().withNewSpec().endSpec().build(), + new Container()) + val returnedPodContainer = kerberosConfStep.bootstrapMainContainerAndVolumes(podWithMain) + assert(expectedPod === returnedPodContainer.pod) + assert(returnedPodContainer.mainContainer.getVolumeMounts.asScala.map(vm => + (vm.getName, vm.getMountPath)).head === + (SPARK_APP_HADOOP_SECRET_VOLUME_NAME, SPARK_APP_HADOOP_CREDENTIALS_BASE_DIR)) + assert(returnedPodContainer.mainContainer.getEnv.asScala.head.getName === + ENV_HADOOP_TOKEN_FILE_LOCATION) + assert(returnedPodContainer.mainContainer.getEnv.asScala(1).getName === ENV_SPARK_USER) + } +} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/HadoopConfigBootstrapStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/HadoopConfigBootstrapStepSuite.scala new file mode 100644 index 0000000000000..a7374611ea9cb --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/HadoopConfigBootstrapStepSuite.scala @@ -0,0 +1,90 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.kubernetes.submit.submitsteps + +import scala.collection.JavaConverters._ + +import io.fabric8.kubernetes.api.model._ +import org.mockito.{Mock, MockitoAnnotations} +import org.mockito.Matchers.any +import org.mockito.Mockito.when +import org.scalatest.BeforeAndAfter + +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.deploy.kubernetes.constants._ +import org.apache.spark.deploy.kubernetes.submit.submitsteps.hadoopsteps.{HadoopConfigSpec, HadoopConfigurationStep} + + +private[spark] class HadoopConfigBootstrapStepSuite extends SparkFunSuite with BeforeAndAfter{ + private val CONFIG_MAP_NAME = "config-map" + private val POD_LABEL = Map("bootstrap" -> "true") + private val DRIVER_CONTAINER_NAME = "driver-container" + private val EXPECTED_SECRET = new SecretBuilder() + .withNewMetadata() + .withName(HADOOP_KERBEROS_SECRET_NAME) + .endMetadata() + .addToData("data", "secretata") + .build() + + @Mock + private var hadoopConfigStep : HadoopConfigurationStep = _ + + before { + MockitoAnnotations.initMocks(this) + when(hadoopConfigStep.configureContainers(any[HadoopConfigSpec])).thenReturn( + HadoopConfigSpec( + configMapProperties = Map("data" -> "dataBytesToString"), + driverPod = new PodBuilder() + .withNewMetadata() + .addToLabels("bootstrap", "true") + .endMetadata() + .withNewSpec().endSpec() + .build(), + driverContainer = new ContainerBuilder().withName(DRIVER_CONTAINER_NAME).build(), + additionalDriverSparkConf = Map("sparkConf" -> "confValue"), + dtSecret = + Some(EXPECTED_SECRET), + dtSecretName = HADOOP_KERBEROS_SECRET_NAME, + dtSecretLabel = "")) + } + + test("Test modification of driverSpec with Hadoop Steps") { + val hadoopConfStep = new HadoopConfigBootstrapStep( + Seq(hadoopConfigStep), + CONFIG_MAP_NAME) + val expectedDriverSparkConf = new SparkConf(true) + .set(HADOOP_CONFIG_MAP_SPARK_CONF_NAME, CONFIG_MAP_NAME) + .set("sparkConf", "confValue") + val expectedConfigMap = new ConfigMapBuilder() + .withNewMetadata() + .withName(CONFIG_MAP_NAME) + .endMetadata() + .addToData(Map("data" -> "dataBytesToString").asJava) + .build() + val expectedResources = Seq(expectedConfigMap, EXPECTED_SECRET) + val driverSpec = KubernetesDriverSpec( + driverPod = new Pod(), + driverContainer = new Container(), + driverSparkConf = new SparkConf(true), + otherKubernetesResources = Seq.empty[HasMetadata]) + val returnContainerSpec = hadoopConfStep.configureDriver(driverSpec) + assert(expectedDriverSparkConf.getAll === returnContainerSpec.driverSparkConf.getAll) + assert(returnContainerSpec.driverContainer.getName == DRIVER_CONTAINER_NAME) + assert(returnContainerSpec.driverPod.getMetadata.getLabels.asScala === POD_LABEL) + assert(returnContainerSpec.otherKubernetesResources === expectedResources) + } +} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopStepsOrchestratorSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopStepsOrchestratorSuite.scala index 1f8039ddeb2b0..cc0c5a86a1f51 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopStepsOrchestratorSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopStepsOrchestratorSuite.scala @@ -38,35 +38,49 @@ private[spark] class HadoopStepsOrchestratorSuite extends SparkFunSuite { assert(true) } -// test("Testing with Keytab Kerberos Login") { -// val sparkTestConf2 = new SparkConf(true) -// .set(KUBERNETES_KERBEROS_SUPPORT, true) -// .set(KUBERNETES_KERBEROS_KEYTAB, "keytab.file") -// .set(KUBERNETES_KERBEROS_PRINCIPAL, "user@kerberos") -// val hadoopOrchestrator = new HadoopStepsOrchestrator( -// NAMESPACE, -// HADOOP_CONFIG_MAP, -// sparkTestConf2, -// HADOOP_CONF_DIR_VAL) -// val steps = hadoopOrchestrator.getHadoopSteps() -// assert(steps.length === 2) -// assert(steps.head.isInstanceOf[HadoopConfMounterStep]) -// assert(steps(1).isInstanceOf[HadoopKerberosKeytabResolverStep]) -// } + test("Testing with Keytab Kerberos Login") { + val sparkTestConf = new SparkConf(true) + .set(KUBERNETES_KERBEROS_SUPPORT, true) + .set(KUBERNETES_KERBEROS_KEYTAB, "keytab.file") + .set(KUBERNETES_KERBEROS_PRINCIPAL, "user@kerberos") + val hadoopOrchestrator = new HadoopStepsOrchestrator( + NAMESPACE, + HADOOP_CONFIG_MAP, + sparkTestConf, + HADOOP_CONF_DIR_VAL) + val steps = hadoopOrchestrator.getHadoopSteps() + assert(steps.length === 2) + assert(steps.head.isInstanceOf[HadoopConfMounterStep]) + assert(steps(1).isInstanceOf[HadoopKerberosKeytabResolverStep]) + } + + test("Testing with kinit Kerberos Login") { + val sparkTestConf = new SparkConf(true) + .set(KUBERNETES_KERBEROS_SUPPORT, true) + val hadoopOrchestrator = new HadoopStepsOrchestrator( + NAMESPACE, + HADOOP_CONFIG_MAP, + sparkTestConf, + HADOOP_CONF_DIR_VAL) + val steps = hadoopOrchestrator.getHadoopSteps() + assert(steps.length === 2) + assert(steps.head.isInstanceOf[HadoopConfMounterStep]) + assert(steps(1).isInstanceOf[HadoopKerberosKeytabResolverStep]) + } -// test("Testing with Keytab Kerberos Login") { -// val sparkTestConf3 = new SparkConf(true) -// .set(KUBERNETES_KERBEROS_SUPPORT, true) -// .set(KUBERNETES_KERBEROS_DT_SECRET_NAME, "dtSecret") -// .set(KUBERNETES_KERBEROS_DT_SECRET_LABEL, "dtLabel") -// val hadoopOrchestrator = new HadoopStepsOrchestrator( -// NAMESPACE, -// HADOOP_CONFIG_MAP, -// sparkTestConf3, -// HADOOP_CONF_DIR_VAL) -// val steps = hadoopOrchestrator.getHadoopSteps() -// assert(steps.length === 2) -// assert(steps.head.isInstanceOf[HadoopConfMounterStep]) -// assert(steps(1).isInstanceOf[HadoopKerberosSecretResolverStep]) -// } + test("Testing with Secret stored Kerberos") { + val sparkTestConf = new SparkConf(true) + .set(KUBERNETES_KERBEROS_SUPPORT, true) + .set(KUBERNETES_KERBEROS_DT_SECRET_NAME, "dtSecret") + .set(KUBERNETES_KERBEROS_DT_SECRET_LABEL, "dtLabel") + val hadoopOrchestrator = new HadoopStepsOrchestrator( + NAMESPACE, + HADOOP_CONFIG_MAP, + sparkTestConf, + HADOOP_CONF_DIR_VAL) + val steps = hadoopOrchestrator.getHadoopSteps() + assert(steps.length === 2) + assert(steps.head.isInstanceOf[HadoopConfMounterStep]) + assert(steps(1).isInstanceOf[HadoopKerberosSecretResolverStep]) + } } From d9b7b50bd24ea14d4dff5dd1e2db493b019fc1d2 Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Fri, 18 Aug 2017 09:54:59 -0400 Subject: [PATCH 29/37] cleanup and various small fixes --- .../KerberosTokenConfBootstrapSuite.scala | 9 ++--- .../src/main/docker/kerberos-test/Dockerfile | 1 - .../kerberos-yml/namenode-hadoop-pv.yml | 14 -------- .../{namenode-hadoop.yml => nn-hadoop.yml} | 0 .../kerberos-yml/server-keytab-pv.yml | 14 -------- .../kerberos-yml/test-env2.sh | 25 -------------- .../kubernetes/integration-tests/pom.xml | 1 - .../integrationtest/KubernetesSuite.scala | 33 ------------------- .../docker/SparkDockerImageBuilder.scala | 20 +++++------ .../kerberos/KerberosUtils.scala | 30 +++++++++++++---- 10 files changed, 36 insertions(+), 111 deletions(-) delete mode 100644 resource-managers/kubernetes/integration-tests/kerberos-yml/namenode-hadoop-pv.yml rename resource-managers/kubernetes/integration-tests/kerberos-yml/{namenode-hadoop.yml => nn-hadoop.yml} (100%) delete mode 100644 resource-managers/kubernetes/integration-tests/kerberos-yml/server-keytab-pv.yml delete mode 100644 resource-managers/kubernetes/integration-tests/kerberos-yml/test-env2.sh diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/KerberosTokenConfBootstrapSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/KerberosTokenConfBootstrapSuite.scala index 56d98a4378424..95d7159736213 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/KerberosTokenConfBootstrapSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/KerberosTokenConfBootstrapSuite.scala @@ -16,16 +16,13 @@ */ package org.apache.spark.deploy.kubernetes.submit -import java.io.File -import java.util.UUID - import scala.collection.JavaConverters._ -import com.google.common.io.Files + import io.fabric8.kubernetes.api.model._ + import org.apache.spark.SparkFunSuite -import org.apache.spark.deploy.kubernetes.{HadoopConfBootstrapImpl, KerberosTokenConfBootstrapImpl, PodWithMainContainer} +import org.apache.spark.deploy.kubernetes.{KerberosTokenConfBootstrapImpl, PodWithMainContainer} import org.apache.spark.deploy.kubernetes.constants._ -import org.apache.spark.util.Utils private[spark] class KerberosTokenConfBootstrapSuite extends SparkFunSuite { 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 7f97c10dba8ec..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 @@ -24,5 +24,4 @@ FROM spark-base RUN apk add --no-cache --update krb5 krb5-libs COPY examples /opt/spark/examples COPY test-scripts/test-env.sh /opt/spark/ -COPY test-scripts/test-env2.sh /opt/spark/ COPY hconf /opt/spark/hconf \ 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 deleted file mode 100644 index d813c90030444..0000000000000 --- a/resource-managers/kubernetes/integration-tests/kerberos-yml/namenode-hadoop-pv.yml +++ /dev/null @@ -1,14 +0,0 @@ -kind: PersistentVolume -apiVersion: v1 -metadata: - name: nn-hadoop - labels: - type: local - job: kerberostest -spec: - capacity: - storage: 1Gi - accessModes: - - ReadWriteOnce - hostPath: - path: "/tmp/nn" \ No newline at end of file diff --git a/resource-managers/kubernetes/integration-tests/kerberos-yml/namenode-hadoop.yml b/resource-managers/kubernetes/integration-tests/kerberos-yml/nn-hadoop.yml similarity index 100% rename from resource-managers/kubernetes/integration-tests/kerberos-yml/namenode-hadoop.yml rename to resource-managers/kubernetes/integration-tests/kerberos-yml/nn-hadoop.yml 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 deleted file mode 100644 index 50298c5fce13e..0000000000000 --- a/resource-managers/kubernetes/integration-tests/kerberos-yml/server-keytab-pv.yml +++ /dev/null @@ -1,14 +0,0 @@ -kind: PersistentVolume -apiVersion: v1 -metadata: - name: server-keytab - labels: - type: local - job: kerberostest -spec: - capacity: - storage: 1Gi - accessModes: - - ReadWriteOnce - hostPath: - path: "/tmp/keytab" \ No newline at end of file diff --git a/resource-managers/kubernetes/integration-tests/kerberos-yml/test-env2.sh b/resource-managers/kubernetes/integration-tests/kerberos-yml/test-env2.sh deleted file mode 100644 index 0aeb6f131bab3..0000000000000 --- a/resource-managers/kubernetes/integration-tests/kerberos-yml/test-env2.sh +++ /dev/null @@ -1,25 +0,0 @@ -#!/usr/bin/env bash -sed -i -e 's/#//' -e 's/default_ccache_name/# default_ccache_name/' /etc/krb5.conf -export HADOOP_OPTS="-Djava.net.preferIPv4Stack=true -Dsun.security.krb5.debug=true" -export HADOOP_JAAS_DEBUG=true -export HADOOP_ROOT_LOGGER=DEBUG,console -cp ${TMP_KRB_LOC} /etc/krb5.conf -cp ${TMP_CORE_LOC} /opt/spark/hconf/core-site.xml -cp ${TMP_HDFS_LOC} /opt/spark/hconf/hdfs-site.xml -mkdir -p /etc/krb5.conf.d -until /usr/bin/kinit -kt /var/keytabs/hdfs.keytab hdfs/nn.${NAMESPACE}.svc.cluster.local; do sleep 15; done -/opt/spark/bin/spark-submit \ - --deploy-mode cluster \ - --class ${CLASS_NAME} \ - --master k8s://${MASTER_URL} \ - --kubernetes-namespace ${NAMESPACE} \ - --conf spark.executor.instances=1 \ - --conf spark.app.name=spark-hdfs \ - --conf spark.driver.extraClassPath=/opt/spark/hconf/core-site.xml:/opt/spark/hconf/hdfs-site.xml:/opt/spark/hconf/yarn-site.xml:/etc/krb5.conf \ - --conf spark.kubernetes.driver.docker.image=spark-driver:latest \ - --conf spark.kubernetes.executor.docker.image=spark-executor:latest \ - --conf spark.kubernetes.initcontainer.docker.image=spark-init:latest \ - --conf spark.kubernetes.kerberos.enabled=true \ - --conf spark.kubernetes.driver.label.spark-app-locator=${APP_LOCATOR_LABEL} \ - ${SUBMIT_RESOURCE} \ - hdfs://nn.${NAMESPACE}.svc.cluster.local:9000/user/ifilonenko/wordcount.txt \ No newline at end of file diff --git a/resource-managers/kubernetes/integration-tests/pom.xml b/resource-managers/kubernetes/integration-tests/pom.xml index fde201779aa46..1d837b4db14ed 100644 --- a/resource-managers/kubernetes/integration-tests/pom.xml +++ b/resource-managers/kubernetes/integration-tests/pom.xml @@ -253,7 +253,6 @@ kerberos-yml test-env.sh - test-env2.sh 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 c76466d4884f2..0d9ee14ecbb29 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 @@ -128,39 +128,6 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { } } } -// test("Secure HDFS test with kinit") { -// assume(testBackend.name == MINIKUBE_TEST_BACKEND) -// launchKerberizedCluster() -// createKerberosTestPod( -// CONTAINER_LOCAL_MAIN_APP_RESOURCE, -// HDFS_TEST_CLASS, -// APP_LOCATOR_LABEL, -// "kerberos-yml/kerberos-test2.yml") -// val kubernetesClient = kubernetesTestComponents.kubernetesClient -// val driverWatcherCache = new KerberosDriverWatcherCache( -// kubernetesClient, -// Map("spark-app-locator" -> APP_LOCATOR_LABEL)) -// driverWatcherCache.start() -// driverWatcherCache.stop() -// val expectedLogOnCompletion = Seq( -// "Returned length(s) of: 1", -// "File contents: [This is an awesome word count file]") -// val driverPod = kubernetesClient -// .pods() -// .withLabel("spark-app-locator", APP_LOCATOR_LABEL) -// .list() -// .getItems -// .get(0) -// Eventually.eventually(TIMEOUT, INTERVAL) { -// expectedLogOnCompletion.foreach { e => -// assert(kubernetesClient -// .pods() -// .withName(driverPod.getMetadata.getName) -// .getLog -// .contains(e), "The application did not complete.") -// } -// } -// } test("Run PySpark Job on file from SUBMITTER with --py-files") { assume(testBackend.name == MINIKUBE_TEST_BACKEND) 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 ce1f4d45ec4a6..fdd32d2830449 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 @@ -87,16 +87,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("kerberos-test", KERBEROS_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/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 86dcaa4203032..145683fdc5eb2 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 @@ -58,19 +58,35 @@ private[spark] class KerberosUtils( .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]) = { + private def createPVTemplate(name: String, pathType: String) : PersistentVolume = + new PersistentVolumeBuilder() + .withNewMetadata() + .withName(name) + .withLabels(Map( + "type" -> "local", + "job" -> "kerberostest").asJava) + .endMetadata() + .withNewSpec() + .withCapacity(Map("storage" -> new Quantity("1Gi")).asJava) + .withAccessModes("ReadWriteOnce") + .withHostPath( + new HostPathVolumeSource(s"/tmp/$namespace/$pathType")) + .endSpec() + .build() + private val pvNN = "nn-hadoop" + private val pvKT = "server-keytab" + private val persistentVolumeMap: Map[String, PersistentVolume] = Map( + pvNN -> createPVTemplate(pvNN, "nn"), + pvKT -> createPVTemplate(pvKT, "keytab")) + private def buildKerberosPV(pvType: String) = { KerberosStorage( - kubernetesClient.load(loadFromYaml(seqPair.head)) + kubernetesClient.load(loadFromYaml(pvType)) .get().get(0).asInstanceOf[PersistentVolumeClaim], - kubernetesClient.load(loadFromYaml(seqPair(1))) - .get().get(0).asInstanceOf[PersistentVolume]) + persistentVolumeMap(pvType)) } 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() From d53a50ff3c7b648060d7e5229f8cb3c732284f48 Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Tue, 22 Aug 2017 09:20:13 -0400 Subject: [PATCH 30/37] added back sparkdockerbuilder images --- .../docker/SparkDockerImageBuilder.scala | 20 +++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) 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 fdd32d2830449..ce1f4d45ec4a6 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 @@ -87,16 +87,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("kerberos-test", KERBEROS_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 = { From ffe78919ea0e9d861468a4225d2c4e6218e17d99 Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Thu, 31 Aug 2017 12:19:41 -0400 Subject: [PATCH 31/37] address initial comments and scalastyle issues --- .../InitContainerConfigurationStepsOrchestrator.scala | 1 - .../cluster/kubernetes/KubernetesClusterManager.scala | 2 +- ...InitContainerResourceStagingServerSecretPluginSuite.scala | 3 ++- .../apache/spark/deploy/kubernetes/submit/ClientSuite.scala | 1 - .../submit/submitsteps/InitContainerBootstrapStepSuite.scala | 2 +- .../kubernetes/submit/submitsteps/PythonStepSuite.scala | 2 +- .../integrationtest/KerberizedHadoopClusterLauncher.scala | 2 +- .../kubernetes/integrationtest/KerberosTestPodLauncher.scala | 2 +- .../integrationtest/KubernetesTestComponents.scala | 3 +-- .../integrationtest/docker/SparkDockerImageBuilder.scala | 5 ++--- .../integrationtest/kerberos/KerberosDeployment.scala | 2 +- .../integrationtest/kerberos/KerberosStorage.scala | 2 +- 12 files changed, 12 insertions(+), 15 deletions(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/InitContainerConfigurationStepsOrchestrator.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/InitContainerConfigurationStepsOrchestrator.scala index 8d69d377376b8..e4ea5235af18f 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/InitContainerConfigurationStepsOrchestrator.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/InitContainerConfigurationStepsOrchestrator.scala @@ -62,7 +62,6 @@ private[spark] class InitContainerConfigurationStepsOrchestrator( submissionSparkConf.get(RESOURCE_STAGING_SERVER_INTERNAL_SSL_ENABLED) .orElse(submissionSparkConf.get(RESOURCE_STAGING_SERVER_SSL_ENABLED)) .getOrElse(false) - OptionRequirements.requireNandDefined( maybeResourceStagingServerInternalClientCert, maybeResourceStagingServerInternalTrustStore, 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 0ff8f3493a32b..5a590dd0bff54 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 @@ -27,9 +27,9 @@ import org.apache.spark.deploy.kubernetes.submit.MountSmallFilesBootstrapImpl import org.apache.spark.internal.Logging import org.apache.spark.scheduler.{ExternalClusterManager, SchedulerBackend, TaskScheduler, TaskSchedulerImpl} import org.apache.spark.util.Utils +import org.apache.spark.SparkContext private[spark] class KubernetesClusterManager extends ExternalClusterManager with Logging { - import org.apache.spark.SparkContext override def canCreate(masterURL: String): Boolean = masterURL.startsWith("k8s") override def createTaskScheduler(sc: SparkContext, masterURL: String): TaskScheduler = { diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/InitContainerResourceStagingServerSecretPluginSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/InitContainerResourceStagingServerSecretPluginSuite.scala index 14c97adee87ed..51e695fc08a36 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/InitContainerResourceStagingServerSecretPluginSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/InitContainerResourceStagingServerSecretPluginSuite.scala @@ -23,7 +23,8 @@ import scala.collection.JavaConverters._ import org.apache.spark.deploy.kubernetes.constants._ import org.apache.spark.SparkFunSuite -private[spark] class InitContainerResourceStagingServerSecretPluginSuite extends SparkFunSuite with BeforeAndAfter{ +class InitContainerResourceStagingServerSecretPluginSuite + extends SparkFunSuite with BeforeAndAfter{ private val INIT_CONTAINER_SECRET_NAME = "init-secret" private val INIT_CONTAINER_SECRET_MOUNT = "/tmp/secret" diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/ClientSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/ClientSuite.scala index 948d2b5abfdf2..35fd5c1f206fa 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/ClientSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/ClientSuite.scala @@ -156,7 +156,6 @@ private[spark] class ClientSuite extends SparkFunSuite with BeforeAndAfter { val driverJvmOptsEnvs = driverContainer.getEnv.asScala.filter { env => env.getName.startsWith(ENV_JAVA_OPT_PREFIX) }.sortBy(_.getName) - logInfo(s"driverJVM Options $driverJvmOptsEnvs") assert(driverJvmOptsEnvs.size === 6) val expectedJvmOptsValues = Seq( diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/InitContainerBootstrapStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/InitContainerBootstrapStepSuite.scala index 8946f88c931ca..1c9a1ac50229e 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/InitContainerBootstrapStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/InitContainerBootstrapStepSuite.scala @@ -31,7 +31,7 @@ import org.apache.spark.deploy.kubernetes.constants._ import org.apache.spark.deploy.kubernetes.submit.submitsteps.initcontainer.{InitContainerConfigurationStep, InitContainerSpec} import org.apache.spark.util.Utils -private[spark] class InitContainerBootstrapStepSuite extends SparkFunSuite { +class InitContainerBootstrapStepSuite extends SparkFunSuite { private val OBJECT_MAPPER = new ObjectMapper().registerModule(DefaultScalaModule) private val CONFIG_MAP_NAME = "spark-init-config-map" diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/PythonStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/PythonStepSuite.scala index 02277bd86bf39..ce0dcee6acc46 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/PythonStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/PythonStepSuite.scala @@ -22,7 +22,7 @@ import scala.collection.JavaConverters._ import org.apache.spark.{SparkConf, SparkFunSuite} -private[spark] class PythonStepSuite extends SparkFunSuite with BeforeAndAfter { +class PythonStepSuite extends SparkFunSuite with BeforeAndAfter { private val FILE_DOWNLOAD_PATH = "/var/data/spark-files" private val PYSPARK_FILES = Seq( "hdfs://localhost:9000/app/files/file1.py", 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 f053cf624ceb6..687c125eb583b 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 @@ -54,4 +54,4 @@ private[spark] class KerberizedHadoopClusterLauncher( Thread.sleep(500) } } -} \ 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 be5f5c94721a2..53d372faace41 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 @@ -118,4 +118,4 @@ private[spark] class KerberosTestPodLauncher( 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/KubernetesTestComponents.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KubernetesTestComponents.scala index 9e245039cd1ee..c04bb0c525d86 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KubernetesTestComponents.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KubernetesTestComponents.scala @@ -19,13 +19,12 @@ package org.apache.spark.deploy.kubernetes.integrationtest import java.util.UUID import io.fabric8.kubernetes.client.DefaultKubernetesClient +import org.scalatest.concurrent.Eventually import scala.collection.JavaConverters._ import org.apache.spark.SparkConf import org.apache.spark.deploy.kubernetes.config._ -import org.scalatest.concurrent.Eventually - private[spark] class KubernetesTestComponents(defaultClient: DefaultKubernetesClient) { val namespace = UUID.randomUUID().toString.replaceAll("-", "") 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 ce1f4d45ec4a6..8b7aad95d8ca7 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 @@ -24,13 +24,12 @@ import scala.collection.JavaConverters._ import com.spotify.docker.client.{DefaultDockerClient, DockerCertificates, LoggingBuildHandler} import org.apache.http.client.utils.URIBuilder +import org.scalatest.concurrent.{Eventually, PatienceConfiguration} +import org.scalatest.time.{Minutes, Seconds, Span} import org.apache.spark.internal.Logging - import org.apache.spark.util.RedirectThread -import org.scalatest.concurrent.{Eventually, PatienceConfiguration} -import org.scalatest.time.{Minutes, Seconds, Span} private[spark] class SparkDockerImageBuilder (private val dockerEnv: Map[String, String]) extends Logging{ diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/kerberos/KerberosDeployment.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/kerberos/KerberosDeployment.scala index 1e1db004fbb7b..a60011153f5d5 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/kerberos/KerberosDeployment.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/kerberos/KerberosDeployment.scala @@ -21,4 +21,4 @@ import io.fabric8.kubernetes.api.model.extensions.Deployment private[spark] case class KerberosDeployment( podDeployment: Deployment, - service: Service) \ No newline at end of file + service: Service) diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/kerberos/KerberosStorage.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/kerberos/KerberosStorage.scala index 16284fd49c270..28fe77c274ea1 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/kerberos/KerberosStorage.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/kerberos/KerberosStorage.scala @@ -20,4 +20,4 @@ import io.fabric8.kubernetes.api.model.{PersistentVolume, PersistentVolumeClaim} private[spark] case class KerberosStorage( persistentVolumeClaim: PersistentVolumeClaim, - persistentVolume: PersistentVolume) \ No newline at end of file + persistentVolume: PersistentVolume) From 6efa37906ccce9992b7faf27622a3ecb1ba14cd3 Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Thu, 31 Aug 2017 12:51:50 -0400 Subject: [PATCH 32/37] addresses comments from PR --- docs/running-on-kubernetes.md | 6 +++--- .../kubernetes/KerberosTokenConfBootstrap.scala | 4 ++-- .../apache/spark/deploy/kubernetes/config.scala | 6 +++--- .../apache/spark/deploy/kubernetes/constants.scala | 4 ++-- .../submitsteps/HadoopConfigBootstrapStep.scala | 2 +- .../submitsteps/hadoopsteps/HadoopConfigSpec.scala | 4 ++-- .../HadoopKerberosKeytabResolverStep.scala | 11 ++++++----- .../HadoopKerberosSecretResolverStep.scala | 12 ++++++------ .../hadoopsteps/HadoopStepsOrchestrator.scala | 14 +++++++------- .../kubernetes/KubernetesClusterManager.scala | 6 +++--- .../HadoopConfigBootstrapStepSuite.scala | 2 +- .../HadoopKerberosKeytabResolverStepSuite.scala | 6 ++++-- .../HadoopKerberosSecretResolverStepSuite.scala | 8 ++++---- .../hadoopsteps/HadoopStepsOrchestratorSuite.scala | 3 +-- 14 files changed, 45 insertions(+), 43 deletions(-) diff --git a/docs/running-on-kubernetes.md b/docs/running-on-kubernetes.md index faac8d175b396..9182560fbd227 100644 --- a/docs/running-on-kubernetes.md +++ b/docs/running-on-kubernetes.md @@ -786,7 +786,7 @@ from the other deployment modes. See the [configuration page](configuration.html spark.kubernetes.kerberos.enabled false - Specify whether your job is a job that will require a Kerberos Authorization to access HDFS. By default, we + Specify whether your job requires a Kerberos Authentication to access HDFS. By default, we will assume that you will not require secure HDFS access. @@ -820,12 +820,12 @@ from the other deployment modes. See the [configuration page](configuration.html - spark.kubernetes.kerberos.tokensecret.label + spark.kubernetes.kerberos.tokensecret.itemkey spark.kubernetes.kerberos.dt.label Assuming you have set spark.kubernetes.kerberos.enabled to be true. This will let you specify the 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 + We have a default value of spark.kubernetes.kerberos.tokensecret.itemkey should you not include it. But you should always include this if you are proposing a pre-existing secret contain the delegation token data. spark.executorEnv.[EnvironmentVariableName] (none) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/KerberosTokenConfBootstrap.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/KerberosTokenConfBootstrap.scala index 0709b3d270fed..f8f42b5cd0346 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/KerberosTokenConfBootstrap.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/KerberosTokenConfBootstrap.scala @@ -36,7 +36,7 @@ private[spark] trait KerberosTokenBootstrapConf { private[spark] class KerberosTokenConfBootstrapImpl( secretName: String, - secretLabel: String, + secretItemKey: String, userName: String) extends KerberosTokenBootstrapConf with Logging{ @@ -62,7 +62,7 @@ private[spark] class KerberosTokenConfBootstrapImpl( .endVolumeMount() .addNewEnv() .withName(ENV_HADOOP_TOKEN_FILE_LOCATION) - .withValue(s"$SPARK_APP_HADOOP_CREDENTIALS_BASE_DIR/$secretLabel") + .withValue(s"$SPARK_APP_HADOOP_CREDENTIALS_BASE_DIR/$secretItemKey") .endEnv() .addNewEnv() .withName(ENV_SPARK_USER) 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 6d6332487f240..3f983149445cc 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 @@ -570,9 +570,9 @@ package object config extends Logging { .stringConf .createOptional - private[spark] val KUBERNETES_KERBEROS_DT_SECRET_LABEL = - ConfigBuilder("spark.kubernetes.kerberos.tokensecret.label") - .doc("Specify the label of the data where " + + private[spark] val KUBERNETES_KERBEROS_DT_SECRET_ITEM_KEY = + ConfigBuilder("spark.kubernetes.kerberos.tokensecret.itemkey") + .doc("Specify the item key of the data where " + " your existing delegation token is stored. This removes the need" + " for the job user to provide any keytab for launching a job") .stringConf 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 95e441df017a3..af44c8cb7c697 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 @@ -113,8 +113,8 @@ package object constants { "spark.kubernetes.kerberos.dt" private[spark] val HADOOP_KERBEROS_CONF_SECRET = "spark.kubernetes.kerberos.secretname" - private[spark] val HADOOP_KERBEROS_CONF_LABEL = - "spark.kubernetes.kerberos.labelname" + private[spark] val HADOOP_KERBEROS_CONF_ITEM_KEY = + "spark.kubernetes.kerberos.itemkeyname" private[spark] val KERBEROS_SECRET_LABEL_PREFIX = "hadoop-tokens" private[spark] val SPARK_HADOOP_PREFIX = "spark.hadoop." 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 a2e11a327b955..a9c66aed7075e 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 @@ -42,7 +42,7 @@ private[spark] class HadoopConfigBootstrapStep( additionalDriverSparkConf = Map.empty[String, String], dtSecret = None, dtSecretName = HADOOP_KERBEROS_SECRET_NAME, - dtSecretLabel = "") + dtSecretItemKey = "") for (nextStep <- hadoopConfigurationSteps) { currentHadoopSpec = nextStep.configureContainers(currentHadoopSpec) } 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 1ccce0f9ce66a..51e9e506bcc22 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, Pod, Secret} * pairs of (path, data) * - The secret containing a DT, either previously specified or built on the fly * - The name of the secret where the DT will be stored - * - The label on the secret which correlates with where the current DT data is stored + * - The data item-key on the secret which correlates with where the current DT data is stored */ private[spark] case class HadoopConfigSpec( additionalDriverSparkConf: Map[String, String], @@ -39,4 +39,4 @@ private[spark] case class HadoopConfigSpec( configMapProperties: Map[String, String], dtSecret: Option[Secret], dtSecretName: String, - dtSecretLabel: String) + dtSecretItemKey: String) 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 127c052e208a4..de2ed5c185af1 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 @@ -102,17 +102,18 @@ private[spark] class HadoopKerberosKeytabResolverStep( val data = serialize(credentials) val renewalTime = getTokenRenewalInterval(tokens, hadoopConf).getOrElse(Long.MaxValue) val currentTime: Long = System.currentTimeMillis() - val initialTokenLabelName = s"$KERBEROS_SECRET_LABEL_PREFIX-$currentTime-$renewalTime" + val initialTokenDataKeyName = s"$KERBEROS_SECRET_LABEL_PREFIX-$currentTime-$renewalTime" val secretDT = new SecretBuilder() .withNewMetadata() .withName(HADOOP_KERBEROS_SECRET_NAME) + .withLabels(Map("refresh-hadoop-tokens" -> "yes").asJava) .endMetadata() - .addToData(initialTokenLabelName, Base64.encodeBase64String(data)) + .addToData(initialTokenDataKeyName, Base64.encodeBase64String(data)) .build() val bootstrapKerberos = new KerberosTokenConfBootstrapImpl( HADOOP_KERBEROS_SECRET_NAME, - initialTokenLabelName, + initialTokenDataKeyName, jobUserUGI.getShortUserName) val withKerberosEnvPod = bootstrapKerberos.bootstrapMainContainerAndVolumes( PodWithMainContainer( @@ -121,13 +122,13 @@ private[spark] class HadoopKerberosKeytabResolverStep( hadoopConfigSpec.copy( additionalDriverSparkConf = hadoopConfigSpec.additionalDriverSparkConf ++ Map( - HADOOP_KERBEROS_CONF_LABEL -> initialTokenLabelName, + HADOOP_KERBEROS_CONF_ITEM_KEY -> initialTokenDataKeyName, HADOOP_KERBEROS_CONF_SECRET -> HADOOP_KERBEROS_SECRET_NAME), driverPod = withKerberosEnvPod.pod, driverContainer = withKerberosEnvPod.mainContainer, dtSecret = Some(secretDT), dtSecretName = HADOOP_KERBEROS_SECRET_NAME, - dtSecretLabel = initialTokenLabelName) + dtSecretItemKey = initialTokenDataKeyName) } // Functions that should be in Core with Rebase to 2.3 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 957be6dfd4634..3eeec1374ef96 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 @@ -25,18 +25,18 @@ import org.apache.spark.deploy.kubernetes.constants._ /** * This step assumes that you have already done all the heavy lifting in retrieving a * delegation token and storing the following data in a secret before running this job. - * This step requires that you just specify the secret name and label corresponding to the - * data where the delegation token is stored. + * This step requires that you just specify the secret name and data item-key corresponding + * to the data where the delegation token is stored. */ private[spark] class HadoopKerberosSecretResolverStep( submissionSparkConf: SparkConf, tokenSecretName: String, - tokenLabelName: String) extends HadoopConfigurationStep { + tokenItemKeyName: String) extends HadoopConfigurationStep { override def configureContainers(hadoopConfigSpec: HadoopConfigSpec): HadoopConfigSpec = { val bootstrapKerberos = new KerberosTokenConfBootstrapImpl( tokenSecretName, - tokenLabelName, + tokenItemKeyName, UserGroupInformation.getCurrentUser.getShortUserName) val withKerberosEnvPod = bootstrapKerberos.bootstrapMainContainerAndVolumes( PodWithMainContainer( @@ -47,10 +47,10 @@ private[spark] class HadoopKerberosSecretResolverStep( driverContainer = withKerberosEnvPod.mainContainer, additionalDriverSparkConf = hadoopConfigSpec.additionalDriverSparkConf ++ Map( - HADOOP_KERBEROS_CONF_LABEL -> tokenLabelName, + HADOOP_KERBEROS_CONF_ITEM_KEY -> tokenItemKeyName, HADOOP_KERBEROS_CONF_SECRET -> tokenSecretName), dtSecret = None, dtSecretName = tokenSecretName, - dtSecretLabel = tokenLabelName) + dtSecretItemKey = tokenItemKeyName) } } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopStepsOrchestrator.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopStepsOrchestrator.scala index 8bc2906965598..16bb0745c1e3e 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 @@ -36,8 +36,8 @@ private[spark] class HadoopStepsOrchestrator( private val maybeKeytab = submissionSparkConf.get(KUBERNETES_KERBEROS_KEYTAB) .map(k => new File(k)) private val maybeExistingSecret = submissionSparkConf.get(KUBERNETES_KERBEROS_DT_SECRET_NAME) - private val maybeExistingSecretLabel = - submissionSparkConf.get(KUBERNETES_KERBEROS_DT_SECRET_LABEL) + private val maybeExistingSecretItemKey = + submissionSparkConf.get(KUBERNETES_KERBEROS_DT_SECRET_ITEM_KEY) private val hadoopConfigurationFiles = getHadoopConfFiles(hadoopConfDir) logInfo(s"Hadoop Conf directory: $hadoopConfDir") @@ -55,10 +55,10 @@ private[spark] class HadoopStepsOrchestrator( OptionRequirements.requireBothOrNeitherDefined( maybeExistingSecret, - maybeExistingSecretLabel, + maybeExistingSecretItemKey, "If a secret storing a Kerberos Delegation Token is specified you must also" + " specify the label where the data is stored", - "If a secret label where the data of the Kerberos Delegation Token is specified" + + "If a secret data item-key where the data of the Kerberos Delegation Token is specified" + " you must also specify the name of the secret") def getHadoopSteps(): Seq[HadoopConfigurationStep] = { @@ -72,10 +72,10 @@ private[spark] class HadoopStepsOrchestrator( hadoopConfDir) val maybeKerberosStep = if (isKerberosEnabled) { - maybeExistingSecret.map(secretLabel => Some(new HadoopKerberosSecretResolverStep( + maybeExistingSecret.map(secretItemKey => Some(new HadoopKerberosSecretResolverStep( submissionSparkConf, - secretLabel, - maybeExistingSecretLabel.get))).getOrElse(Some( + secretItemKey, + maybeExistingSecretItemKey.get))).getOrElse(Some( new HadoopKerberosKeytabResolverStep( submissionSparkConf, maybePrincipal, 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 5a590dd0bff54..a640f973adbf6 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 @@ -44,7 +44,7 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit val maybeHadoopConfigMap = sparkConf.getOption(HADOOP_CONFIG_MAP_SPARK_CONF_NAME) val maybeHadoopConfDir = sparkConf.getOption(HADOOP_CONF_DIR_LOC) val maybeDTSecretName = sparkConf.getOption(HADOOP_KERBEROS_CONF_SECRET) - val maybeDTLabelName = sparkConf.getOption(HADOOP_KERBEROS_CONF_LABEL) + val maybeDTLabelName = sparkConf.getOption(HADOOP_KERBEROS_CONF_ITEM_KEY) val maybeInitContainerConfigMap = sparkConf.get(EXECUTOR_INIT_CONTAINER_CONFIG_MAP) val maybeInitContainerConfigMapKey = sparkConf.get(EXECUTOR_INIT_CONTAINER_CONFIG_MAP_KEY) val maybeSubmittedFilesSecret = sparkConf.get(EXECUTOR_SUBMITTED_SMALL_FILES_SECRET) @@ -91,11 +91,11 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit } val kerberosBootstrap = for { secretName <- maybeDTSecretName - secretLabel <- maybeDTLabelName + secretItemKey <- maybeDTLabelName } yield { new KerberosTokenConfBootstrapImpl( secretName, - secretLabel, + secretItemKey, Utils.getCurrentUserName) } val mountSmallFilesBootstrap = for { diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/HadoopConfigBootstrapStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/HadoopConfigBootstrapStepSuite.scala index a7374611ea9cb..074a02c339d95 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/HadoopConfigBootstrapStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/HadoopConfigBootstrapStepSuite.scala @@ -59,7 +59,7 @@ private[spark] class HadoopConfigBootstrapStepSuite extends SparkFunSuite with B dtSecret = Some(EXPECTED_SECRET), dtSecretName = HADOOP_KERBEROS_SECRET_NAME, - dtSecretLabel = "")) + dtSecretItemKey = "")) } test("Test modification of driverSpec with Hadoop Steps") { diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopKerberosKeytabResolverStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopKerberosKeytabResolverStepSuite.scala index f0496c588157a..d540cedac29e1 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopKerberosKeytabResolverStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopKerberosKeytabResolverStepSuite.scala @@ -56,14 +56,16 @@ private[spark] class HadoopKerberosKeytabResolverStepSuite extends SparkFunSuite "", "") val returnContainerSpec = keytabStep.configureContainers(hadoopConfSpec) - assert(returnContainerSpec.additionalDriverSparkConf(HADOOP_KERBEROS_CONF_LABEL) + assert(returnContainerSpec.additionalDriverSparkConf(HADOOP_KERBEROS_CONF_ITEM_KEY) .contains(KERBEROS_SECRET_LABEL_PREFIX)) assert(returnContainerSpec.additionalDriverSparkConf(HADOOP_KERBEROS_CONF_SECRET) === HADOOP_KERBEROS_SECRET_NAME) assert(returnContainerSpec.driverContainer.getName == DRIVER_CONTAINER_NAME) assert(returnContainerSpec.driverPod.getMetadata.getLabels.asScala === POD_LABEL) - assert(returnContainerSpec.dtSecretLabel.contains(KERBEROS_SECRET_LABEL_PREFIX)) + assert(returnContainerSpec.dtSecretItemKey.contains(KERBEROS_SECRET_LABEL_PREFIX)) assert(returnContainerSpec.dtSecretName === HADOOP_KERBEROS_SECRET_NAME) + assert(returnContainerSpec.dtSecret.get.getMetadata.getLabels.asScala === + Map("refresh-hadoop-tokens" -> "yes")) assert(returnContainerSpec.dtSecret.nonEmpty) assert(returnContainerSpec.dtSecret.get.getMetadata.getName === HADOOP_KERBEROS_SECRET_NAME) } diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopKerberosSecretResolverStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopKerberosSecretResolverStepSuite.scala index b4305aee1e049..bb0fa237a2d5e 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopKerberosSecretResolverStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopKerberosSecretResolverStepSuite.scala @@ -29,15 +29,15 @@ private[spark] class HadoopKerberosSecretResolverStepSuite extends SparkFunSuite private val POD_LABEL = Map("bootstrap" -> "true") private val DRIVER_CONTAINER_NAME = "driver-container" private val TOKEN_SECRET_NAME = "secretName" - private val TOKEN_SECRET_LABEL = "secretLabel" + private val TOKEN_SECRET_DATA_ITEM_KEY = "secretItemKey" test("Testing kerberos with Secret") { val keytabStep = new HadoopKerberosSecretResolverStep( new SparkConf(), TOKEN_SECRET_NAME, - TOKEN_SECRET_LABEL) + TOKEN_SECRET_DATA_ITEM_KEY) val expectedDriverSparkConf = Map( - HADOOP_KERBEROS_CONF_LABEL -> TOKEN_SECRET_LABEL, + HADOOP_KERBEROS_CONF_ITEM_KEY -> TOKEN_SECRET_DATA_ITEM_KEY, HADOOP_KERBEROS_CONF_SECRET -> TOKEN_SECRET_NAME) val hadoopConfSpec = HadoopConfigSpec( Map.empty[String, String], @@ -57,7 +57,7 @@ private[spark] class HadoopKerberosSecretResolverStepSuite extends SparkFunSuite assert(returnContainerSpec.driverContainer.getName == DRIVER_CONTAINER_NAME) assert(returnContainerSpec.driverPod.getMetadata.getLabels.asScala === POD_LABEL) assert(returnContainerSpec.dtSecret === None) - assert(returnContainerSpec.dtSecretLabel === TOKEN_SECRET_LABEL) + assert(returnContainerSpec.dtSecretItemKey === TOKEN_SECRET_DATA_ITEM_KEY) assert(returnContainerSpec.dtSecretName === TOKEN_SECRET_NAME) } } diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopStepsOrchestratorSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopStepsOrchestratorSuite.scala index cc0c5a86a1f51..1f946666f53ce 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopStepsOrchestratorSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopStepsOrchestratorSuite.scala @@ -35,7 +35,6 @@ private[spark] class HadoopStepsOrchestratorSuite extends SparkFunSuite { val steps = hadoopOrchestrator.getHadoopSteps() assert(steps.length === 1) assert(steps.head.isInstanceOf[HadoopConfMounterStep]) - assert(true) } test("Testing with Keytab Kerberos Login") { @@ -72,7 +71,7 @@ private[spark] class HadoopStepsOrchestratorSuite extends SparkFunSuite { val sparkTestConf = new SparkConf(true) .set(KUBERNETES_KERBEROS_SUPPORT, true) .set(KUBERNETES_KERBEROS_DT_SECRET_NAME, "dtSecret") - .set(KUBERNETES_KERBEROS_DT_SECRET_LABEL, "dtLabel") + .set(KUBERNETES_KERBEROS_DT_SECRET_ITEM_KEY, "dtItemKey") val hadoopOrchestrator = new HadoopStepsOrchestrator( NAMESPACE, HADOOP_CONFIG_MAP, From 6052a1334cfac93912a2478673cd8235358ab0ef Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Thu, 31 Aug 2017 17:17:34 -0400 Subject: [PATCH 33/37] mocking hadoopUGI --- docs/running-on-kubernetes.md | 2 +- .../kubernetes/HadoopConfBootstrap.scala | 6 +- .../deploy/kubernetes/HadoopUGIUtil.scala | 27 +++++++++ .../HadoopKerberosKeytabResolverStep.scala | 29 ++++++---- .../hadoopsteps/HadoopStepsOrchestrator.scala | 9 ++- .../kubernetes/KubernetesClusterManager.scala | 4 +- .../submit/HadoopConfBootstrapSuite.scala | 24 ++++++-- ...adoopKerberosKeytabResolverStepSuite.scala | 58 +++++++++++++++++-- 8 files changed, 129 insertions(+), 30 deletions(-) create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/HadoopUGIUtil.scala diff --git a/docs/running-on-kubernetes.md b/docs/running-on-kubernetes.md index 9182560fbd227..4f3b5b993294c 100644 --- a/docs/running-on-kubernetes.md +++ b/docs/running-on-kubernetes.md @@ -816,7 +816,7 @@ from the other deployment modes. See the [configuration page](configuration.html Assuming you have set spark.kubernetes.kerberos.enabled to be true. This will let you specify the name of the secret where your existing delegation token data is stored. You must also specify the - label spark.kubernetes.kerberos.tokensecret.name where your data is stored on the secret. + item key spark.kubernetes.kerberos.tokensecret.itemkey where your data is stored on the secret. 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 add1a2d09ed74..17efe19b8f206 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,7 +21,6 @@ import java.io.File import scala.collection.JavaConverters._ import io.fabric8.kubernetes.api.model.{ContainerBuilder, KeyToPathBuilder, PodBuilder} -import org.apache.hadoop.security.UserGroupInformation import org.apache.spark.deploy.kubernetes.constants._ import org.apache.spark.internal.Logging @@ -42,7 +41,8 @@ private[spark] trait HadoopConfBootstrap { private[spark] class HadoopConfBootstrapImpl( hadoopConfConfigMapName: String, - hadoopConfigFiles: Seq[File]) extends HadoopConfBootstrap with Logging{ + hadoopConfigFiles: Seq[File], + hadoopUGI: HadoopUGIUtil) extends HadoopConfBootstrap with Logging{ override def bootstrapMainContainerAndVolumes( originalPodWithMainContainer: PodWithMainContainer) @@ -76,7 +76,7 @@ private[spark] class HadoopConfBootstrapImpl( .endEnv() .addNewEnv() .withName(ENV_SPARK_USER) - .withValue(UserGroupInformation.getCurrentUser.getShortUserName) + .withValue(hadoopUGI.getShortName) .endEnv() .build() originalPodWithMainContainer.copy( diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/HadoopUGIUtil.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/HadoopUGIUtil.scala new file mode 100644 index 0000000000000..03178b13f68ef --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/HadoopUGIUtil.scala @@ -0,0 +1,27 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.kubernetes + +import org.apache.hadoop.security.UserGroupInformation + +private[spark] class HadoopUGIUtil{ + def getCurrentUser: UserGroupInformation = UserGroupInformation.getCurrentUser + def getShortName: String = getCurrentUser.getShortUserName + def isSecurityEnabled: Boolean = UserGroupInformation.isSecurityEnabled + def loginUserFromKeytabAndReturnUGI(principal: String, keytab: String): UserGroupInformation = + UserGroupInformation.loginUserFromKeytabAndReturnUGI(principal, keytab) +} 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 de2ed5c185af1..7b098e226f4de 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 @@ -26,13 +26,13 @@ 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.Credentials 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.{KerberosTokenConfBootstrapImpl, PodWithMainContainer} +import org.apache.spark.deploy.kubernetes.{HadoopUGIUtil, KerberosTokenConfBootstrapImpl, PodWithMainContainer} import org.apache.spark.deploy.kubernetes.constants._ import org.apache.spark.internal.Logging @@ -51,16 +51,18 @@ import org.apache.spark.internal.Logging private[spark] class HadoopKerberosKeytabResolverStep( submissionSparkConf: SparkConf, maybePrincipal: Option[String], - maybeKeytab: Option[File]) extends HadoopConfigurationStep with Logging{ - private var originalCredentials: Credentials = _ - private var dfs : FileSystem = _ - private var renewer: String = _ - private var credentials: Credentials = _ - private var tokens: Iterable[Token[_ <: TokenIdentifier]] = _ - override def configureContainers(hadoopConfigSpec: HadoopConfigSpec): HadoopConfigSpec = { + maybeKeytab: Option[File], + hadoopUGI: HadoopUGIUtil) extends HadoopConfigurationStep with Logging{ + private var originalCredentials: Credentials = _ + private var dfs : FileSystem = _ + private var renewer: String = _ + private var credentials: Credentials = _ + private var tokens: Iterable[Token[_ <: TokenIdentifier]] = _ + + override def configureContainers(hadoopConfigSpec: HadoopConfigSpec): HadoopConfigSpec = { val hadoopConf = SparkHadoopUtil.get.newConfiguration(submissionSparkConf) logDebug(s"Hadoop Configuration: ${hadoopConf.toString}") - if (!UserGroupInformation.isSecurityEnabled) logError("Hadoop not configuration with Kerberos") + if (hadoopUGI.isSecurityEnabled) logError("Hadoop not configuration with Kerberos") val maybeJobUserUGI = for { principal <- maybePrincipal @@ -71,12 +73,12 @@ private[spark] class HadoopKerberosKeytabResolverStep( submissionSparkConf.set("spark.yarn.principal", principal) submissionSparkConf.set("spark.yarn.keytab", keytab.toURI.toString) logDebug("Logged into KDC with keytab using Job User UGI") - UserGroupInformation.loginUserFromKeytabAndReturnUGI( + hadoopUGI.loginUserFromKeytabAndReturnUGI( principal, keytab.toURI.toString) } // In the case that keytab is not specified we will read from Local Ticket Cache - val jobUserUGI = maybeJobUserUGI.getOrElse(UserGroupInformation.getCurrentUser) + val jobUserUGI = maybeJobUserUGI.getOrElse(hadoopUGI.getCurrentUser) // It is necessary to run as jobUserUGI because logged in user != Current User jobUserUGI.doAs(new PrivilegedExceptionAction[Void] { override def run(): Void = { @@ -92,12 +94,15 @@ private[spark] class HadoopKerberosKeytabResolverStep( logDebug(s"Renewer is: $renewer") credentials = new Credentials(originalCredentials) dfs.addDelegationTokens(renewer, credentials) + // This is difficult to Mock and will require refactoring tokens = credentials.getAllTokens.asScala logDebug(s"Tokens: ${credentials.toString}") logDebug(s"All tokens: ${tokens.mkString(",")}") logDebug(s"All secret keys: ${credentials.getAllSecretKeys}") null }}) + credentials.getAllTokens.asScala.isEmpty + tokens.isEmpty if (tokens.isEmpty) logError("Did not obtain any Delegation Tokens") val data = serialize(credentials) val renewalTime = getTokenRenewalInterval(tokens, hadoopConf).getOrElse(Long.MaxValue) 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 16bb0745c1e3e..a1c3bc141a271 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 @@ -19,7 +19,7 @@ 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, OptionRequirements} +import org.apache.spark.deploy.kubernetes.{HadoopConfBootstrapImpl, HadoopUGIUtil, OptionRequirements} import org.apache.spark.deploy.kubernetes.config._ import org.apache.spark.internal.Logging @@ -39,6 +39,7 @@ private[spark] class HadoopStepsOrchestrator( private val maybeExistingSecretItemKey = submissionSparkConf.get(KUBERNETES_KERBEROS_DT_SECRET_ITEM_KEY) private val hadoopConfigurationFiles = getHadoopConfFiles(hadoopConfDir) + private val hadoopUGI = new HadoopUGIUtil logInfo(s"Hadoop Conf directory: $hadoopConfDir") require(maybeKeytab.forall( _ => isKerberosEnabled ), @@ -64,7 +65,8 @@ private[spark] class HadoopStepsOrchestrator( def getHadoopSteps(): Seq[HadoopConfigurationStep] = { val hadoopConfBootstrapImpl = new HadoopConfBootstrapImpl( hadoopConfigMapName, - hadoopConfigurationFiles) + hadoopConfigurationFiles, + hadoopUGI) val hadoopConfMounterStep = new HadoopConfMounterStep( hadoopConfigMapName, hadoopConfigurationFiles, @@ -79,7 +81,8 @@ private[spark] class HadoopStepsOrchestrator( new HadoopKerberosKeytabResolverStep( submissionSparkConf, maybePrincipal, - maybeKeytab))) + maybeKeytab, + hadoopUGI))) } else { Option.empty[HadoopConfigurationStep] } 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 a640f973adbf6..90bc9d1adaa96 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 @@ -82,11 +82,13 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit val hadoopBootStrap = for { hadoopConfigMap <- maybeHadoopConfigMap } yield { + val hadoopUtil = new HadoopUGIUtil val hadoopConfigurations = maybeHadoopConfDir.map( conf_dir => getHadoopConfFiles(conf_dir)).getOrElse(Array.empty[File]) new HadoopConfBootstrapImpl( hadoopConfigMap, - hadoopConfigurations + hadoopConfigurations, + hadoopUtil ) } val kerberosBootstrap = for { diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/HadoopConfBootstrapSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/HadoopConfBootstrapSuite.scala index 219d45413431a..64426f0deb15e 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/HadoopConfBootstrapSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/HadoopConfBootstrapSuite.scala @@ -23,22 +23,34 @@ import scala.collection.JavaConverters._ import com.google.common.io.Files import io.fabric8.kubernetes.api.model._ +import org.mockito.{Mock, MockitoAnnotations} +import org.mockito.Mockito.when +import org.scalatest.BeforeAndAfter import org.apache.spark.SparkFunSuite -import org.apache.spark.deploy.kubernetes.{HadoopConfBootstrapImpl, PodWithMainContainer} +import org.apache.spark.deploy.kubernetes.{HadoopConfBootstrapImpl, HadoopUGIUtil, PodWithMainContainer} import org.apache.spark.deploy.kubernetes.constants._ import org.apache.spark.util.Utils - -private[spark] class HadoopConfBootstrapSuite extends SparkFunSuite { +private[spark] class HadoopConfBootstrapSuite extends SparkFunSuite with BeforeAndAfter{ private val CONFIG_MAP_NAME = "config-map" private val TEMP_HADOOP_FILE = createTempFile("core-site.xml") private val HADOOP_FILES = Seq(TEMP_HADOOP_FILE) + private val SPARK_USER_VALUE = "sparkUser" + + @Mock + private var hadoopUtil: HadoopUGIUtil = _ + + before { + MockitoAnnotations.initMocks(this) + when(hadoopUtil.getShortName).thenReturn(SPARK_USER_VALUE) + } test("Test of bootstrapping hadoop_conf_dir files") { val hadoopConfStep = new HadoopConfBootstrapImpl( CONFIG_MAP_NAME, - HADOOP_FILES) + HADOOP_FILES, + hadoopUtil) val expectedKeyPaths = Seq( new KeyToPathBuilder() .withKey(TEMP_HADOOP_FILE.toPath.getFileName.toString) @@ -55,6 +67,7 @@ private[spark] class HadoopConfBootstrapSuite extends SparkFunSuite { .endVolume() .endSpec() .build() + val podWithMain = PodWithMainContainer( new PodBuilder().withNewSpec().endSpec().build(), new Container()) @@ -64,7 +77,8 @@ private[spark] class HadoopConfBootstrapSuite extends SparkFunSuite { (vm.getName, vm.getMountPath)).head === (HADOOP_FILE_VOLUME, HADOOP_CONF_DIR_PATH)) assert(returnedPodContainer.mainContainer.getEnv.asScala.head === new EnvVarBuilder().withName(ENV_HADOOP_CONF_DIR).withValue(HADOOP_CONF_DIR_PATH).build()) - assert(returnedPodContainer.mainContainer.getEnv.asScala(1).getName === ENV_SPARK_USER) + assert(returnedPodContainer.mainContainer.getEnv.asScala(1) === + new EnvVarBuilder().withName(ENV_SPARK_USER).withValue(SPARK_USER_VALUE).build()) } private def createTempFile(contents: String): File = { val dir = Utils.createTempDir() diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopKerberosKeytabResolverStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopKerberosKeytabResolverStepSuite.scala index d540cedac29e1..8b9e26e257f92 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopKerberosKeytabResolverStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopKerberosKeytabResolverStepSuite.scala @@ -23,25 +23,73 @@ import scala.collection.JavaConverters._ import com.google.common.io.Files import io.fabric8.kubernetes.api.model._ +import org.apache.hadoop.fs.FileSystem +import org.apache.hadoop.io.Text +import org.apache.hadoop.security.Credentials +import org.apache.hadoop.security.UserGroupInformation +import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenIdentifier +import org.apache.hadoop.security.token.Token +import org.mockito.{Mock, MockitoAnnotations} +import org.mockito.Matchers.any +import org.mockito.Mockito.when +import org.mockito.invocation.InvocationOnMock +import org.mockito.stubbing.Answer +import org.scalatest.BeforeAndAfter import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.deploy.kubernetes.HadoopUGIUtil import org.apache.spark.deploy.kubernetes.constants._ import org.apache.spark.util.Utils - - -private[spark] class HadoopKerberosKeytabResolverStepSuite extends SparkFunSuite { +private[spark] class HadoopKerberosKeytabResolverStepSuite + extends SparkFunSuite with BeforeAndAfter{ private val POD_LABEL = Map("bootstrap" -> "true") private val DRIVER_CONTAINER_NAME = "driver-container" private val TEMP_KEYTAB_FILE = createTempFile("keytab") private val KERB_PRINCIPAL = "user@k8s.com" + private val SPARK_USER_VALUE = "sparkUser" + private var oldCredentials = new Credentials() + private val TEST_IDENTIFIER = "identifier" + private val TEST_PASSWORD = "password" + private val TEST_TOKEN_VALUE = "data" + private def getByteArray(input: String) = input.toCharArray.map(_.toByte) + private def getStringFromArray(input: Array[Byte]) = new String(input) + private val TEST_TOKEN = new Token[AbstractDelegationTokenIdentifier]( + getByteArray(TEST_IDENTIFIER), + getByteArray(TEST_PASSWORD), + new Text("kind"), + new Text("service")) + oldCredentials.addToken(new Text("testToken"), TEST_TOKEN) + private val dfs = FileSystem.get(SparkHadoopUtil.get.newConfiguration(new SparkConf())) + private val hadoopUGI = new HadoopUGIUtil() + + @Mock + private var hadoopUtil: HadoopUGIUtil = _ + + @Mock + private var ugi: UserGroupInformation = _ + + before { + MockitoAnnotations.initMocks(this) + when(hadoopUtil.loginUserFromKeytabAndReturnUGI(any[String], any[String])) + .thenAnswer(new Answer[UserGroupInformation] { + override def answer(invocation: InvocationOnMock): UserGroupInformation = { + hadoopUGI.getCurrentUser + } + }) + when(hadoopUtil.getCurrentUser).thenReturn(ugi) + when(hadoopUtil.getShortName).thenReturn(SPARK_USER_VALUE) + when(ugi.getCredentials).thenReturn(oldCredentials) + } - // TODO: Require mocking of UGI methods test("Testing keytab login") { + when(hadoopUtil.isSecurityEnabled).thenReturn(true) val keytabStep = new HadoopKerberosKeytabResolverStep( new SparkConf(), Some(KERB_PRINCIPAL), - Some(TEMP_KEYTAB_FILE)) + Some(TEMP_KEYTAB_FILE), + hadoopUtil) val hadoopConfSpec = HadoopConfigSpec( Map.empty[String, String], new PodBuilder() From f9ca47da811c2f198d4ace6c54649d36518d8c98 Mon Sep 17 00:00:00 2001 From: Kimoon Kim Date: Fri, 1 Sep 2017 13:15:00 -0700 Subject: [PATCH 34/37] Fix executor env to include simple authn --- .../spark/deploy/kubernetes/constants.scala | 1 - .../KubernetesClusterSchedulerBackend.scala | 28 ++++++------------- .../docker/SparkDockerImageBuilder.scala | 1 + 3 files changed, 10 insertions(+), 20 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 af44c8cb7c697..dfb4e0838113f 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/constants.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/constants.scala @@ -60,7 +60,6 @@ package object constants { private[spark] val ENV_DRIVER_URL = "SPARK_DRIVER_URL" private[spark] val ENV_EXECUTOR_CORES = "SPARK_EXECUTOR_CORES" private[spark] val ENV_EXECUTOR_MEMORY = "SPARK_EXECUTOR_MEMORY" - private[spark] val ENV_EXECUTOR_JAVA_OPTS = "SPARK_EXECUTOR_JAVA_OPTS" private[spark] val ENV_APPLICATION_ID = "SPARK_APPLICATION_ID" private[spark] val ENV_EXECUTOR_ID = "SPARK_EXECUTOR_ID" private[spark] val ENV_EXECUTOR_POD_IP = "SPARK_EXECUTOR_POD_IP" 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 3d933577b5495..35d0f8c8dc8a6 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 @@ -72,14 +72,12 @@ private[spark] class KubernetesClusterSchedulerBackend( private val executorsToRemove = Collections.newSetFromMap[String]( new ConcurrentHashMap[String, java.lang.Boolean]()).asScala - private val executorExtraJavaOpts = conf.get( - org.apache.spark.internal.config.EXECUTOR_JAVA_OPTIONS) private val executorExtraClasspath = conf.get( org.apache.spark.internal.config.EXECUTOR_CLASS_PATH) private val executorJarsDownloadDir = conf.get(INIT_CONTAINER_JARS_DOWNLOAD_LOCATION) private val isKerberosEnabled = conf.get(KUBERNETES_KERBEROS_SUPPORT) private val maybeSimpleAuthentication = - if (isKerberosEnabled) s" -D$HADOOP_SECURITY_AUTHENTICATION=simple" else "" + if (isKerberosEnabled) Some(s"-D$HADOOP_SECURITY_AUTHENTICATION=simple") else None private val executorLabels = ConfigurationUtils.combinePrefixedKeyValuePairsWithDeprecatedConf( conf, KUBERNETES_EXECUTOR_LABEL_PREFIX, @@ -455,12 +453,6 @@ private[spark] class KubernetesClusterSchedulerBackend( val executorCpuQuantity = new QuantityBuilder(false) .withAmount(executorCores.toString) .build() - val executorJavaOpts = executorExtraJavaOpts.getOrElse("") + maybeSimpleAuthentication - val executorJavaOptsEnv = if (executorJavaOpts.nonEmpty) { - Some(new EnvVarBuilder() - .withName(ENV_EXECUTOR_JAVA_OPTS) - .withValue(executorJavaOpts) - .build()) } else None val executorExtraClasspathEnv = executorExtraClasspath.map { cp => new EnvVarBuilder() .withName(ENV_EXECUTOR_EXTRA_CLASSPATH) @@ -468,14 +460,14 @@ private[spark] class KubernetesClusterSchedulerBackend( .build() } val executorExtraJavaOptionsEnv = conf - .get(org.apache.spark.internal.config.EXECUTOR_JAVA_OPTIONS) - .map { opts => - val delimitedOpts = Utils.splitCommandString(opts) - delimitedOpts.zipWithIndex.map { - case (opt, index) => - new EnvVarBuilder().withName(s"$ENV_JAVA_OPT_PREFIX$index").withValue(opt).build() - } - }.getOrElse(Seq.empty[EnvVar]) + .get(org.apache.spark.internal.config.EXECUTOR_JAVA_OPTIONS) + .map { opts => + val delimitedOpts = Utils.splitCommandString(opts) ++ maybeSimpleAuthentication + delimitedOpts.zipWithIndex.map { + case (opt, index) => + new EnvVarBuilder().withName(s"$ENV_JAVA_OPT_PREFIX$index").withValue(opt).build() + } + }.getOrElse(Seq.empty[EnvVar]) val executorEnv = (Seq( (ENV_EXECUTOR_PORT, executorPort.toString), (ENV_DRIVER_URL, driverUrl), @@ -516,8 +508,6 @@ private[spark] class KubernetesClusterSchedulerBackend( .addToLimits("memory", executorMemoryLimitQuantity) .addToRequests("cpu", executorCpuQuantity) .endResources() - .addToEnv(executorExtraClasspathEnv.toSeq: _*) - .addToEnv(executorJavaOptsEnv.toSeq: _*) .addAllToEnv(executorEnv.asJava) .withPorts(requiredPorts.asJava) .build() 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 8b7aad95d8ca7..a210aab4b78fe 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/docker/SparkDockerImageBuilder.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/docker/SparkDockerImageBuilder.scala @@ -104,5 +104,6 @@ private[spark] class SparkDockerImageBuilder name, dockerFile, new LoggingBuildHandler()) + logInfo(s"Built docker image for $name") } } From d2c86493215517c9914e1a143ec67c92346b659a Mon Sep 17 00:00:00 2001 From: Kimoon Kim Date: Fri, 1 Sep 2017 16:17:57 -0700 Subject: [PATCH 35/37] Fix a bug in executor env handling --- .../KubernetesClusterSchedulerBackend.scala | 12 ++++++++---- .../kerberos/KerberosPVWatcherCache.scala | 2 +- 2 files changed, 9 insertions(+), 5 deletions(-) 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 35d0f8c8dc8a6..4f76d5e9c69cf 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 @@ -459,10 +459,14 @@ private[spark] class KubernetesClusterSchedulerBackend( .withValue(cp) .build() } - val executorExtraJavaOptionsEnv = conf - .get(org.apache.spark.internal.config.EXECUTOR_JAVA_OPTIONS) - .map { opts => - val delimitedOpts = Utils.splitCommandString(opts) ++ maybeSimpleAuthentication + val executorExtraJavaOptions = ( + conf.get(org.apache.spark.internal.config.EXECUTOR_JAVA_OPTIONS) + ++ maybeSimpleAuthentication).mkString(" ") match { + case "" => None + case str => Some(str) + } + val executorExtraJavaOptionsEnv = executorExtraJavaOptions.map { opts => + val delimitedOpts = Utils.splitCommandString(opts) delimitedOpts.zipWithIndex.map { case (opt, index) => new EnvVarBuilder().withName(s"$ENV_JAVA_OPT_PREFIX$index").withValue(opt).build() 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 fff02c2d06ef7..b58aa2e0e532e 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 @@ -72,7 +72,7 @@ private[spark] class KerberosPVWatcherCache( .withLabels(labels.asJava) .watch(new Watcher[PersistentVolume] { override def onClose(cause: KubernetesClientException): Unit = - logInfo("Ending the watch of Persistent Volumes") + logInfo("Ending the watch of Persistent Volumes", cause) override def eventReceived(action: Watcher.Action, resource: PersistentVolume): Unit = { val name = resource.getMetadata.getName action match { From b566fa93f687e19e27f5c7c534ad4b83572bf26c Mon Sep 17 00:00:00 2001 From: Kimoon Kim Date: Fri, 1 Sep 2017 16:48:00 -0700 Subject: [PATCH 36/37] Fix a bug in how the driver sets simple authn --- .../org/apache/spark/deploy/kubernetes/submit/Client.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) 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 d6935dec54f2e..15f6c9d7ce0d8 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/Client.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/Client.scala @@ -81,7 +81,7 @@ private[spark] class Client( org.apache.spark.internal.config.DRIVER_JAVA_OPTIONS) private val isKerberosEnabled = submissionSparkConf.get(KUBERNETES_KERBEROS_SUPPORT) private val maybeSimpleAuthentication = - if (isKerberosEnabled) s"-D$HADOOP_SECURITY_AUTHENTICATION=simple" else "" + if (isKerberosEnabled) Some(s"-D$HADOOP_SECURITY_AUTHENTICATION=simple") else None /** * Run command that initalizes a DriverSpec that will be updated after each @@ -102,7 +102,7 @@ private[spark] class Client( .getAll .map { case (confKey, confValue) => s"-D$confKey=$confValue" - } ++ driverJavaOptions.map(Utils.splitCommandString).getOrElse(Seq.empty) :+ + } ++ driverJavaOptions.map(Utils.splitCommandString).getOrElse(Seq.empty) ++ maybeSimpleAuthentication val driverJavaOptsEnvs: Seq[EnvVar] = resolvedDriverJavaOpts.zipWithIndex.map { case (option, index) => new EnvVarBuilder() From 2d48613d676efed5034b8d1de05581241fd9ce40 Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Tue, 19 Sep 2017 17:44:11 -0400 Subject: [PATCH 37/37] handling Pr comments --- docs/running-on-kubernetes.md | 13 +++- resource-managers/kubernetes/core/pom.xml | 6 -- .../deploy/kubernetes/HadoopUGIUtil.scala | 53 +++++++++++++- .../KerberosTokenConfBootstrap.scala | 4 +- .../spark/deploy/kubernetes/config.scala | 7 ++ .../HadoopKerberosKeytabResolverStep.scala | 69 +++++-------------- .../hadoopsteps/HadoopStepsOrchestrator.scala | 7 +- .../kubernetes/KubernetesClusterManager.scala | 4 +- .../KubernetesClusterSchedulerBackend.scala | 2 +- ...adoopKerberosKeytabResolverStepSuite.scala | 65 ++++++++++------- .../integrationtest/KubernetesSuite.scala | 2 - 11 files changed, 139 insertions(+), 93 deletions(-) diff --git a/docs/running-on-kubernetes.md b/docs/running-on-kubernetes.md index 4f3b5b993294c..f66fa7935b15c 100644 --- a/docs/running-on-kubernetes.md +++ b/docs/running-on-kubernetes.md @@ -810,6 +810,15 @@ from the other deployment modes. See the [configuration page](configuration.html will look within your local TGT cache to resolve this. + + spark.kubernetes.kerberos.rewewer.principal + (none) + + Assuming you have set spark.kubernetes.kerberos.enabled to be true. This will let you specify + the principal that you wish to use to handle renewing of Delegation Tokens. This is optional as you + we will set the principal to be the job users principal by default. + + spark.kubernetes.kerberos.tokensecret.name (none) @@ -817,6 +826,8 @@ from the other deployment modes. See the [configuration page](configuration.html Assuming you have set spark.kubernetes.kerberos.enabled to be true. This will let you specify the name of the secret where your existing delegation token data is stored. You must also specify the item key spark.kubernetes.kerberos.tokensecret.itemkey where your data is stored on the secret. + This is optional in the case that you want to use pre-existing secret, otherwise a new secret will be automatically + created. @@ -824,7 +835,7 @@ from the other deployment modes. See the [configuration page](configuration.html spark.kubernetes.kerberos.dt.label Assuming you have set spark.kubernetes.kerberos.enabled 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. + the data item key name within the pre-specified secret where the data of your existing delegation token data is stored. We have a default value of spark.kubernetes.kerberos.tokensecret.itemkey should you not include it. But you should always include this if you are proposing a pre-existing secret contain the delegation token data. spark.executorEnv.[EnvironmentVariableName] diff --git a/resource-managers/kubernetes/core/pom.xml b/resource-managers/kubernetes/core/pom.xml index 6ee9edfb30903..dfbf8adc5bc7b 100644 --- a/resource-managers/kubernetes/core/pom.xml +++ b/resource-managers/kubernetes/core/pom.xml @@ -100,12 +100,6 @@ com.fasterxml.jackson.jaxrs jackson-jaxrs-json-provider - - - javax.ws.rs - jsr311-api - - javax.ws.rs diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/HadoopUGIUtil.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/HadoopUGIUtil.scala index 03178b13f68ef..8a004c0e5414f 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/HadoopUGIUtil.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/HadoopUGIUtil.scala @@ -16,12 +16,63 @@ */ package org.apache.spark.deploy.kubernetes -import org.apache.hadoop.security.UserGroupInformation +import java.io.{ByteArrayInputStream, ByteArrayOutputStream, DataInputStream, DataOutputStream} +import scala.util.Try + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.FileSystem +import org.apache.hadoop.security.{Credentials, UserGroupInformation} +import org.apache.hadoop.security.token.{Token, TokenIdentifier} +import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenIdentifier + + +// Function of this class is merely for mocking reasons private[spark] class HadoopUGIUtil{ def getCurrentUser: UserGroupInformation = UserGroupInformation.getCurrentUser + def getShortName: String = getCurrentUser.getShortUserName + def isSecurityEnabled: Boolean = UserGroupInformation.isSecurityEnabled + def loginUserFromKeytabAndReturnUGI(principal: String, keytab: String): UserGroupInformation = UserGroupInformation.loginUserFromKeytabAndReturnUGI(principal, keytab) + + def dfsAddDelegationToken(hadoopConf: Configuration, renewer: String, creds: Credentials) + : Iterable[Token[_ <: TokenIdentifier]] = + FileSystem.get(hadoopConf).addDelegationTokens(renewer, creds) + + def getCurrentTime: Long = System.currentTimeMillis() + + // Functions that should be in Core with Rebase to 2.3 + @deprecated("Moved to core in 2.2", "2.2") + def getTokenRenewalInterval( + renewedTokens: Iterable[Token[_ <: TokenIdentifier]], + hadoopConf: Configuration): Option[Long] = { + val renewIntervals = renewedTokens.filter { + _.decodeIdentifier().isInstanceOf[AbstractDelegationTokenIdentifier]} + .flatMap { token => + Try { + val newExpiration = token.renew(hadoopConf) + val identifier = token.decodeIdentifier().asInstanceOf[AbstractDelegationTokenIdentifier] + val interval = newExpiration - identifier.getIssueDate + interval + }.toOption} + if (renewIntervals.isEmpty) None else Some(renewIntervals.min) + } + + @deprecated("Moved to core in 2.2", "2.2") + def serialize(creds: Credentials): Array[Byte] = { + val byteStream = new ByteArrayOutputStream + val dataStream = new DataOutputStream(byteStream) + creds.writeTokenStorageToStream(dataStream) + byteStream.toByteArray + } + + @deprecated("Moved to core in 2.2", "2.2") + def deserialize(tokenBytes: Array[Byte]): Credentials = { + val creds = new Credentials() + creds.readTokenStorageStream(new DataInputStream(new ByteArrayInputStream(tokenBytes))) + creds + } } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/KerberosTokenConfBootstrap.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/KerberosTokenConfBootstrap.scala index f8f42b5cd0346..6225142fa4b52 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/KerberosTokenConfBootstrap.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/KerberosTokenConfBootstrap.scala @@ -26,7 +26,7 @@ import org.apache.spark.internal.Logging * This is separated out from the HadoopConf steps API because this component can be reused to * mounted the DT secret for executors as well. */ -private[spark] trait KerberosTokenBootstrapConf { +private[spark] trait KerberosTokenConfBootstrap { // Bootstraps a main container with the Secret mounted as volumes and an ENV variable // pointing to the mounted file containing the DT for Secure HDFS interaction def bootstrapMainContainerAndVolumes( @@ -37,7 +37,7 @@ private[spark] trait KerberosTokenBootstrapConf { private[spark] class KerberosTokenConfBootstrapImpl( secretName: String, secretItemKey: String, - userName: String) extends KerberosTokenBootstrapConf with Logging{ + userName: String) extends KerberosTokenConfBootstrap with Logging{ override def bootstrapMainContainerAndVolumes( 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 3f983149445cc..442bdb01b979b 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/config.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/config.scala @@ -562,6 +562,13 @@ package object config extends Logging { .stringConf .createOptional + private[spark] val KUBERNETES_KERBEROS_RENEWER_PRINCIPAL = + ConfigBuilder("spark.kubernetes.kerberos.rewnewer.principal") + .doc("Specify the principal" + + " you wish to renew and retrieve your Kerberos values with") + .stringConf + .createOptional + private[spark] val KUBERNETES_KERBEROS_DT_SECRET_NAME = ConfigBuilder("spark.kubernetes.kerberos.tokensecret.name") .doc("Specify the name of the secret where " + 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 7b098e226f4de..4e0a0cfd8fe9b 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 @@ -20,15 +20,12 @@ 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 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 @@ -52,6 +49,7 @@ private[spark] class HadoopKerberosKeytabResolverStep( submissionSparkConf: SparkConf, maybePrincipal: Option[String], maybeKeytab: Option[File], + maybeRenewerPrincipal: Option[String], hadoopUGI: HadoopUGIUtil) extends HadoopConfigurationStep with Logging{ private var originalCredentials: Credentials = _ private var dfs : FileSystem = _ @@ -62,7 +60,7 @@ private[spark] class HadoopKerberosKeytabResolverStep( override def configureContainers(hadoopConfigSpec: HadoopConfigSpec): HadoopConfigSpec = { val hadoopConf = SparkHadoopUtil.get.newConfiguration(submissionSparkConf) logDebug(s"Hadoop Configuration: ${hadoopConf.toString}") - if (hadoopUGI.isSecurityEnabled) logError("Hadoop not configuration with Kerberos") + if (hadoopUGI.isSecurityEnabled) logDebug("Hadoop not configured with Kerberos") val maybeJobUserUGI = for { principal <- maybePrincipal @@ -87,37 +85,35 @@ private[spark] class HadoopKerberosKeytabResolverStep( logDebug(s"Original tokens: ${originalCredentials.toString}") logDebug(s"All tokens: ${originalCredentials.getAllTokens}") logDebug(s"All secret keys: ${originalCredentials.getAllSecretKeys}") - dfs = FileSystem.get(hadoopConf) - // This is not necessary with [Spark-20328] since we would be using + // TODO: This is not necessary with [Spark-20328] since we would be using // Spark core providers to handle delegation token renewal - renewer = jobUserUGI.getShortUserName + renewer = maybeRenewerPrincipal.getOrElse(jobUserUGI.getShortUserName) logDebug(s"Renewer is: $renewer") credentials = new Credentials(originalCredentials) - dfs.addDelegationTokens(renewer, credentials) - // This is difficult to Mock and will require refactoring + hadoopUGI.dfsAddDelegationToken(hadoopConf, renewer, credentials) tokens = credentials.getAllTokens.asScala logDebug(s"Tokens: ${credentials.toString}") logDebug(s"All tokens: ${tokens.mkString(",")}") logDebug(s"All secret keys: ${credentials.getAllSecretKeys}") null }}) - credentials.getAllTokens.asScala.isEmpty - tokens.isEmpty - if (tokens.isEmpty) logError("Did not obtain any Delegation Tokens") - val data = serialize(credentials) - val renewalTime = getTokenRenewalInterval(tokens, hadoopConf).getOrElse(Long.MaxValue) - val currentTime: Long = System.currentTimeMillis() - val initialTokenDataKeyName = s"$KERBEROS_SECRET_LABEL_PREFIX-$currentTime-$renewalTime" + if (tokens.isEmpty) logDebug("Did not obtain any Delegation Tokens") + val data = hadoopUGI.serialize(credentials) + val renewalInterval = + hadoopUGI.getTokenRenewalInterval(tokens, hadoopConf).getOrElse(Long.MaxValue) + val currentTime: Long = hadoopUGI.getCurrentTime + val initialTokenDataKeyName = s"$KERBEROS_SECRET_LABEL_PREFIX-$currentTime-$renewalInterval" + val uniqueSecretName = s"$HADOOP_KERBEROS_SECRET_NAME.$currentTime" val secretDT = new SecretBuilder() .withNewMetadata() - .withName(HADOOP_KERBEROS_SECRET_NAME) + .withName(uniqueSecretName) .withLabels(Map("refresh-hadoop-tokens" -> "yes").asJava) .endMetadata() .addToData(initialTokenDataKeyName, Base64.encodeBase64String(data)) .build() val bootstrapKerberos = new KerberosTokenConfBootstrapImpl( - HADOOP_KERBEROS_SECRET_NAME, + uniqueSecretName, initialTokenDataKeyName, jobUserUGI.getShortUserName) val withKerberosEnvPod = bootstrapKerberos.bootstrapMainContainerAndVolumes( @@ -128,44 +124,11 @@ private[spark] class HadoopKerberosKeytabResolverStep( additionalDriverSparkConf = hadoopConfigSpec.additionalDriverSparkConf ++ Map( HADOOP_KERBEROS_CONF_ITEM_KEY -> initialTokenDataKeyName, - HADOOP_KERBEROS_CONF_SECRET -> HADOOP_KERBEROS_SECRET_NAME), + HADOOP_KERBEROS_CONF_SECRET -> uniqueSecretName), driverPod = withKerberosEnvPod.pod, driverContainer = withKerberosEnvPod.mainContainer, dtSecret = Some(secretDT), - dtSecretName = HADOOP_KERBEROS_SECRET_NAME, + dtSecretName = uniqueSecretName, dtSecretItemKey = initialTokenDataKeyName) } - - // Functions that should be in Core with Rebase to 2.3 - @deprecated("Moved to core in 2.2", "2.2") - private def getTokenRenewalInterval( - renewedTokens: Iterable[Token[_ <: TokenIdentifier]], - hadoopConf: Configuration): Option[Long] = { - val renewIntervals = renewedTokens.filter { - _.decodeIdentifier().isInstanceOf[AbstractDelegationTokenIdentifier]} - .flatMap { token => - Try { - val newExpiration = token.renew(hadoopConf) - val identifier = token.decodeIdentifier().asInstanceOf[AbstractDelegationTokenIdentifier] - val interval = newExpiration - identifier.getIssueDate - logInfo(s"Renewal interval is $interval for token ${token.getKind.toString}") - interval - }.toOption} - if (renewIntervals.isEmpty) None else Some(renewIntervals.min) - } - - @deprecated("Moved to core in 2.2", "2.2") - private def serialize(creds: Credentials): Array[Byte] = { - val byteStream = new ByteArrayOutputStream - val dataStream = new DataOutputStream(byteStream) - creds.writeTokenStorageToStream(dataStream) - byteStream.toByteArray - } - - @deprecated("Moved to core in 2.2", "2.2") - private def deserialize(tokenBytes: Array[Byte]): Credentials = { - val creds = new Credentials() - creds.readTokenStorageStream(new DataInputStream(new ByteArrayInputStream(tokenBytes))) - creds - } } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopStepsOrchestrator.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopStepsOrchestrator.scala index a1c3bc141a271..93d184fedbcf1 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 @@ -38,6 +38,8 @@ private[spark] class HadoopStepsOrchestrator( private val maybeExistingSecret = submissionSparkConf.get(KUBERNETES_KERBEROS_DT_SECRET_NAME) private val maybeExistingSecretItemKey = submissionSparkConf.get(KUBERNETES_KERBEROS_DT_SECRET_ITEM_KEY) + private val maybeRenewerPrincipal = + submissionSparkConf.get(KUBERNETES_KERBEROS_RENEWER_PRINCIPAL) private val hadoopConfigurationFiles = getHadoopConfFiles(hadoopConfDir) private val hadoopUGI = new HadoopUGIUtil logInfo(s"Hadoop Conf directory: $hadoopConfDir") @@ -74,14 +76,15 @@ private[spark] class HadoopStepsOrchestrator( hadoopConfDir) val maybeKerberosStep = if (isKerberosEnabled) { - maybeExistingSecret.map(secretItemKey => Some(new HadoopKerberosSecretResolverStep( + maybeExistingSecret.map(existingSecretName => Some(new HadoopKerberosSecretResolverStep( submissionSparkConf, - secretItemKey, + existingSecretName, maybeExistingSecretItemKey.get))).getOrElse(Some( new HadoopKerberosKeytabResolverStep( submissionSparkConf, maybePrincipal, maybeKeytab, + maybeRenewerPrincipal, hadoopUGI))) } else { Option.empty[HadoopConfigurationStep] 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 90bc9d1adaa96..ff02c08ee2dce 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterManager.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterManager.scala @@ -44,7 +44,7 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit val maybeHadoopConfigMap = sparkConf.getOption(HADOOP_CONFIG_MAP_SPARK_CONF_NAME) val maybeHadoopConfDir = sparkConf.getOption(HADOOP_CONF_DIR_LOC) val maybeDTSecretName = sparkConf.getOption(HADOOP_KERBEROS_CONF_SECRET) - val maybeDTLabelName = sparkConf.getOption(HADOOP_KERBEROS_CONF_ITEM_KEY) + val maybeDTDataItem = sparkConf.getOption(HADOOP_KERBEROS_CONF_ITEM_KEY) val maybeInitContainerConfigMap = sparkConf.get(EXECUTOR_INIT_CONTAINER_CONFIG_MAP) val maybeInitContainerConfigMapKey = sparkConf.get(EXECUTOR_INIT_CONTAINER_CONFIG_MAP_KEY) val maybeSubmittedFilesSecret = sparkConf.get(EXECUTOR_SUBMITTED_SMALL_FILES_SECRET) @@ -93,7 +93,7 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit } val kerberosBootstrap = for { secretName <- maybeDTSecretName - secretItemKey <- maybeDTLabelName + secretItemKey <- maybeDTDataItem } yield { new KerberosTokenConfBootstrapImpl( secretName, 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 4f76d5e9c69cf..3738e1b8d3991 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala @@ -50,7 +50,7 @@ private[spark] class KubernetesClusterSchedulerBackend( val sc: SparkContext, executorInitContainerBootstrap: Option[SparkPodInitContainerBootstrap], executorHadoopBootStrap: Option[HadoopConfBootstrap], - executorKerberosBootStrap: Option[KerberosTokenBootstrapConf], + executorKerberosBootStrap: Option[KerberosTokenConfBootstrap], executorMountInitContainerSecretPlugin: Option[InitContainerResourceStagingServerSecretPlugin], mountSmallFilesBootstrap: Option[MountSmallFilesBootstrap], kubernetesClient: KubernetesClient) diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopKerberosKeytabResolverStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopKerberosKeytabResolverStepSuite.scala index 8b9e26e257f92..997556d1508a8 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopKerberosKeytabResolverStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopKerberosKeytabResolverStepSuite.scala @@ -23,12 +23,11 @@ import scala.collection.JavaConverters._ import com.google.common.io.Files import io.fabric8.kubernetes.api.model._ -import org.apache.hadoop.fs.FileSystem -import org.apache.hadoop.io.Text -import org.apache.hadoop.security.Credentials -import org.apache.hadoop.security.UserGroupInformation +import org.apache.commons.codec.binary.Base64 +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.security.{Credentials, UserGroupInformation} +import org.apache.hadoop.security.token.{Token, TokenIdentifier} import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenIdentifier -import org.apache.hadoop.security.token.Token import org.mockito.{Mock, MockitoAnnotations} import org.mockito.Matchers.any import org.mockito.Mockito.when @@ -37,11 +36,12 @@ import org.mockito.stubbing.Answer import org.scalatest.BeforeAndAfter import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.deploy.kubernetes.HadoopUGIUtil import org.apache.spark.deploy.kubernetes.constants._ import org.apache.spark.util.Utils + + private[spark] class HadoopKerberosKeytabResolverStepSuite extends SparkFunSuite with BeforeAndAfter{ private val POD_LABEL = Map("bootstrap" -> "true") @@ -49,19 +49,16 @@ private[spark] class HadoopKerberosKeytabResolverStepSuite private val TEMP_KEYTAB_FILE = createTempFile("keytab") private val KERB_PRINCIPAL = "user@k8s.com" private val SPARK_USER_VALUE = "sparkUser" - private var oldCredentials = new Credentials() - private val TEST_IDENTIFIER = "identifier" - private val TEST_PASSWORD = "password" private val TEST_TOKEN_VALUE = "data" private def getByteArray(input: String) = input.toCharArray.map(_.toByte) - private def getStringFromArray(input: Array[Byte]) = new String(input) - private val TEST_TOKEN = new Token[AbstractDelegationTokenIdentifier]( - getByteArray(TEST_IDENTIFIER), - getByteArray(TEST_PASSWORD), - new Text("kind"), - new Text("service")) - oldCredentials.addToken(new Text("testToken"), TEST_TOKEN) - private val dfs = FileSystem.get(SparkHadoopUtil.get.newConfiguration(new SparkConf())) + private val TEST_DATA = getByteArray(TEST_TOKEN_VALUE) + private val OUTPUT_TEST_DATA = Base64.encodeBase64String(TEST_DATA) + private val INTERVAL = 500L + private val CURR_TIME = System.currentTimeMillis() + private val DATA_KEY_NAME = + s"$KERBEROS_SECRET_LABEL_PREFIX-$CURR_TIME-$INTERVAL" + private val SECRET_NAME = s"$HADOOP_KERBEROS_SECRET_NAME.$CURR_TIME" + private val hadoopUGI = new HadoopUGIUtil() @Mock @@ -70,6 +67,15 @@ private[spark] class HadoopKerberosKeytabResolverStepSuite @Mock private var ugi: UserGroupInformation = _ + @Mock + private var creds: Credentials = _ + + @Mock + private var token: Token[AbstractDelegationTokenIdentifier] = _ + + @Mock + private var identifier: AbstractDelegationTokenIdentifier = _ + before { MockitoAnnotations.initMocks(this) when(hadoopUtil.loginUserFromKeytabAndReturnUGI(any[String], any[String])) @@ -80,7 +86,16 @@ private[spark] class HadoopKerberosKeytabResolverStepSuite }) when(hadoopUtil.getCurrentUser).thenReturn(ugi) when(hadoopUtil.getShortName).thenReturn(SPARK_USER_VALUE) - when(ugi.getCredentials).thenReturn(oldCredentials) + when(hadoopUtil.dfsAddDelegationToken(any(), any(), any())).thenReturn(null) + when(ugi.getCredentials).thenReturn(creds) + val tokens = List[Token[_ <: TokenIdentifier]](token).asJavaCollection + when(creds.getAllTokens).thenReturn(tokens) + when(hadoopUtil.serialize(any[Credentials])) + .thenReturn(TEST_DATA) + when(token.decodeIdentifier()).thenReturn(identifier) + when(hadoopUtil.getCurrentTime).thenReturn(CURR_TIME) + when(hadoopUtil.getTokenRenewalInterval(any[Iterable[Token[_ <: TokenIdentifier]]], + any[Configuration])).thenReturn(Some(INTERVAL)) } test("Testing keytab login") { @@ -89,6 +104,7 @@ private[spark] class HadoopKerberosKeytabResolverStepSuite new SparkConf(), Some(KERB_PRINCIPAL), Some(TEMP_KEYTAB_FILE), + None, hadoopUtil) val hadoopConfSpec = HadoopConfigSpec( Map.empty[String, String], @@ -106,16 +122,19 @@ private[spark] class HadoopKerberosKeytabResolverStepSuite val returnContainerSpec = keytabStep.configureContainers(hadoopConfSpec) assert(returnContainerSpec.additionalDriverSparkConf(HADOOP_KERBEROS_CONF_ITEM_KEY) .contains(KERBEROS_SECRET_LABEL_PREFIX)) - assert(returnContainerSpec.additionalDriverSparkConf(HADOOP_KERBEROS_CONF_SECRET) === - HADOOP_KERBEROS_SECRET_NAME) + assert(returnContainerSpec.additionalDriverSparkConf === + Map(HADOOP_KERBEROS_CONF_ITEM_KEY -> DATA_KEY_NAME, + HADOOP_KERBEROS_CONF_SECRET -> SECRET_NAME)) assert(returnContainerSpec.driverContainer.getName == DRIVER_CONTAINER_NAME) assert(returnContainerSpec.driverPod.getMetadata.getLabels.asScala === POD_LABEL) - assert(returnContainerSpec.dtSecretItemKey.contains(KERBEROS_SECRET_LABEL_PREFIX)) - assert(returnContainerSpec.dtSecretName === HADOOP_KERBEROS_SECRET_NAME) + assert(returnContainerSpec.dtSecretItemKey === DATA_KEY_NAME) + assert(returnContainerSpec.dtSecret.get.getData.asScala === Map( + DATA_KEY_NAME -> OUTPUT_TEST_DATA)) + assert(returnContainerSpec.dtSecretName === SECRET_NAME) assert(returnContainerSpec.dtSecret.get.getMetadata.getLabels.asScala === Map("refresh-hadoop-tokens" -> "yes")) assert(returnContainerSpec.dtSecret.nonEmpty) - assert(returnContainerSpec.dtSecret.get.getMetadata.getName === HADOOP_KERBEROS_SECRET_NAME) + assert(returnContainerSpec.dtSecret.get.getMetadata.getName === SECRET_NAME) } private def createTempFile(contents: String): 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 658b216dfcbba..17d48b8b508dd 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KubernetesSuite.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KubernetesSuite.scala @@ -137,7 +137,6 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { System.getProperty("spark.docker.test.driverImage", "spark-driver-py:latest")) .set(EXECUTOR_DOCKER_IMAGE, System.getProperty("spark.docker.test.executorImage", "spark-executor-py:latest")) - .set(KERBEROS_CONF, "simple") runPySparkPiAndVerifyCompletion( PYSPARK_PI_SUBMITTER_LOCAL_FILE_LOCATION, @@ -154,7 +153,6 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { System.getProperty("spark.docker.test.driverImage", "spark-driver-py:latest")) .set(EXECUTOR_DOCKER_IMAGE, System.getProperty("spark.docker.test.executorImage", "spark-executor-py:latest")) - .set(KERBEROS_CONF, "simple") runPySparkPiAndVerifyCompletion(PYSPARK_PI_CONTAINER_LOCAL_FILE_LOCATION, Seq.empty[String]) }