You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by ifilonenko <gi...@git.apache.org> on 2018/09/04 16:25:12 UTC

[GitHub] spark pull request #21669: [SPARK-23257][K8S][WIP] Kerberos Support for Spar...

Github user ifilonenko commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21669#discussion_r214981406
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadoopsteps/HadoopBootstrapUtil.scala ---
    @@ -0,0 +1,186 @@
    +/*
    + * 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.k8s.features.hadoopsteps
    +
    +import java.io.File
    +
    +import scala.collection.JavaConverters._
    +
    +import com.google.common.base.Charsets
    +import com.google.common.io.Files
    +import io.fabric8.kubernetes.api.model.{ConfigMap, ConfigMapBuilder, ContainerBuilder, KeyToPathBuilder, PodBuilder}
    +
    +import org.apache.spark.deploy.k8s.Constants._
    +import org.apache.spark.deploy.k8s.SparkPod
    +import org.apache.spark.deploy.k8s.security.KubernetesHadoopDelegationTokenManager
    +
    +private[spark] object HadoopBootstrapUtil {
    +
    +   /**
    +    * Mounting the DT secret for both the Driver and the executors
    +    *
    +    * @param dtSecretName Name of the secret that stores the Delegation Token
    +    * @param dtSecretItemKey Name of the Item Key storing the Delegation Token
    +    * @param userName Name of the SparkUser to set SPARK_USER
    +    * @param fileLocation Location of the krb5 file
    +    * @param krb5ConfName Name of the ConfigMap for Krb5
    +    * @param pod Input pod to be appended to
    +    * @return a modified SparkPod
    +    */
    +  def bootstrapKerberosPod(
    +      dtSecretName: String,
    +      dtSecretItemKey: String,
    +      userName: String,
    +      fileLocation: String,
    +      krb5ConfName: String,
    +      pod: SparkPod) : SparkPod = {
    +      val krb5File = new File(fileLocation)
    +      val fileStringPath = krb5File.toPath.getFileName.toString
    +      val kerberizedPod = new PodBuilder(pod.pod)
    +        .editOrNewSpec()
    +          .addNewVolume()
    +            .withName(SPARK_APP_HADOOP_SECRET_VOLUME_NAME)
    +            .withNewSecret()
    +              .withSecretName(dtSecretName)
    +              .endSecret()
    +            .endVolume()
    +          .addNewVolume()
    +            .withName(KRB_FILE_VOLUME)
    +              .withNewConfigMap()
    +                .withName(krb5ConfName)
    +                .withItems(new KeyToPathBuilder()
    +                  .withKey(fileStringPath)
    +                  .withPath(fileStringPath)
    +                  .build())
    +                .endConfigMap()
    +              .endVolume()
    +        // TODO: (ifilonenko) make configurable PU(G)ID
    +          .editOrNewSecurityContext()
    +            .withRunAsUser(1000L)
    +            .withFsGroup(2000L)
    +            .endSecurityContext()
    +          .endSpec()
    +        .build()
    +      val kerberizedContainer = new ContainerBuilder(pod.container)
    +        .addNewVolumeMount()
    +          .withName(SPARK_APP_HADOOP_SECRET_VOLUME_NAME)
    +          .withMountPath(SPARK_APP_HADOOP_CREDENTIALS_BASE_DIR)
    +          .endVolumeMount()
    +        .addNewVolumeMount()
    +          .withName(KRB_FILE_VOLUME)
    +          .withMountPath(KRB_FILE_DIR_PATH)
    --- End diff --
    
    Excellent catch as this was the reason it was unable to read the `/etc/` directory upon doing hadoopUGI login


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org