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

2018-09-04 Thread ifilonenko
Github user ifilonenko commented on a diff in the pull request:

https://github.com/apache/spark/pull/21669#discussion_r215057079
  
--- Diff: core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala ---
@@ -164,7 +164,15 @@ private[spark] class SparkSubmit extends Logging {
 }
 }
   } else {
-runMain(childArgs, childClasspath, sparkConf, childMainClass, 
args.verbose)
+if 
(sparkConf.getOption("spark.kubernetes.kerberos.proxyUser").isDefined) {
--- End diff --

@skonto this will be in a followup PR


---

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



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

2018-09-04 Thread skonto
Github user skonto commented on a diff in the pull request:

https://github.com/apache/spark/pull/21669#discussion_r215020138
  
--- Diff: core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala ---
@@ -164,7 +164,15 @@ private[spark] class SparkSubmit extends Logging {
 }
 }
   } else {
-runMain(childArgs, childClasspath, sparkConf, childMainClass, 
args.verbose)
+if 
(sparkConf.getOption("spark.kubernetes.kerberos.proxyUser").isDefined) {
--- End diff --

@ifilonenko have you implemented proxy user support as well?


---

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



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

2018-09-04 Thread skonto
Github user skonto commented on a diff in the pull request:

https://github.com/apache/spark/pull/21669#discussion_r215019733
  
--- 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 --

If it is read only why it cannot be read. Btw  if we plan to modify the 
configmap then there is a known limitation:  
https://github.com/kubernetes/kubernetes/issues/50345.


---

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



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

2018-09-04 Thread ifilonenko
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-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



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

2018-08-29 Thread viirya
Github user viirya commented on a diff in the pull request:

https://github.com/apache/spark/pull/21669#discussion_r213591711
  
--- Diff: core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala ---
@@ -164,7 +164,15 @@ private[spark] class SparkSubmit extends Logging {
 }
 }
   } else {
-runMain(childArgs, childClasspath, sparkConf, childMainClass, 
args.verbose)
+if 
(sparkConf.getOption("spark.kubernetes.kerberos.proxyUser").isDefined) {
--- End diff --

Can we avoid hard-coding config name?


---

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



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

2018-08-28 Thread suryag10
Github user suryag10 commented on a diff in the pull request:

https://github.com/apache/spark/pull/21669#discussion_r213408857
  
--- 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 --

This makes the "/etc/" mounted as read only. I guess we should use as below:
  .withMountPath(KRB_FILE_DIR_PATH + "/krb5.conf")
  .withSubPath("krb5.conf")



---

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



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

2018-08-25 Thread ifilonenko
Github user ifilonenko commented on a diff in the pull request:

https://github.com/apache/spark/pull/21669#discussion_r212812762
  
--- Diff: 
resource-managers/kubernetes/docker/src/main/dockerfiles/spark/Dockerfile ---
@@ -43,6 +43,10 @@ COPY sbin /opt/spark/sbin
 COPY ${img_path}/spark/entrypoint.sh /opt/
 COPY examples /opt/spark/examples
 COPY data /opt/spark/data
+RUN if [ -z "$SPARK_USER" ]; then \
--- End diff --

Thank you for this catch, I will edit this to see if it solves the 
`javax.security.auth.login.LoginException` 


---

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



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

2018-08-24 Thread holdenk
Github user holdenk commented on a diff in the pull request:

https://github.com/apache/spark/pull/21669#discussion_r212764246
  
--- Diff: 
resource-managers/kubernetes/docker/src/main/dockerfiles/spark/Dockerfile ---
@@ -43,6 +43,10 @@ COPY sbin /opt/spark/sbin
 COPY ${img_path}/spark/entrypoint.sh /opt/
 COPY examples /opt/spark/examples
 COPY data /opt/spark/data
+RUN if [ -z "$SPARK_USER" ]; then \
--- End diff --

So this part I _think_ belongs in the entrypoint where you know which user 
your adding because this is done at image build time where we don't know our 
deal.


---

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



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

2018-08-08 Thread ifilonenko
Github user ifilonenko commented on a diff in the pull request:

https://github.com/apache/spark/pull/21669#discussion_r208558956
  
--- Diff: 
resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala
 ---
@@ -212,6 +212,60 @@ private[spark] object Config extends Logging {
 "Ensure that major Python version is either Python2 or Python3")
   .createWithDefault("2")
 
+  val KUBERNETES_KERBEROS_PROXY_USER =
+ConfigBuilder("spark.kubernetes.kerberos.proxyUser")
+  .doc("Specify the proxy user " +
+"for HadoopUGI login for the Driver + Executors")
+  .internal()
+  .stringConf
+  .createWithDefault("false")
+
+  val KUBERNETES_KERBEROS_KRB5_FILE =
+ConfigBuilder("spark.kubernetes.kerberos.krb5location")
+  .doc("Specify the location of the krb5 file " +
--- End diff --

Yes. Correct. This would be set to your local krb5 file which is usually 
stored locally at etc/krb5.conf  


---

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



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

2018-08-08 Thread rvesse
Github user rvesse commented on a diff in the pull request:

https://github.com/apache/spark/pull/21669#discussion_r208531808
  
--- Diff: core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala ---
@@ -164,7 +164,15 @@ private[spark] class SparkSubmit extends Logging {
 }
 }
   } else {
-runMain(childArgs, childClasspath, sparkConf, childMainClass, 
args.verbose)
+if 
(sparkConf.getOption("spark.kubernetes.kerberos.proxyUser").isDefined) {
+  // scalastyle:off println
+  printStream.println("Running as proxy user in k8s cluster 
mode...")
+  // scalastyle:on println
+  SparkHadoopUtil.get.runAsSparkUser(
+() => runMain(childArgs, childClasspath, sparkConf, 
childMainClass, args.verbose))
--- End diff --

So if I have followed the code flow correctly now when using Kerberos auth 
the value of `SPARK_USER` provided to the driver is the user we're launching 
the job as from a Spark/HDFS perspective?


---

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



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

2018-08-08 Thread rvesse
Github user rvesse commented on a diff in the pull request:

https://github.com/apache/spark/pull/21669#discussion_r208530663
  
--- Diff: 
resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala
 ---
@@ -212,6 +212,60 @@ private[spark] object Config extends Logging {
 "Ensure that major Python version is either Python2 or Python3")
   .createWithDefault("2")
 
+  val KUBERNETES_KERBEROS_PROXY_USER =
+ConfigBuilder("spark.kubernetes.kerberos.proxyUser")
+  .doc("Specify the proxy user " +
+"for HadoopUGI login for the Driver + Executors")
+  .internal()
+  .stringConf
+  .createWithDefault("false")
+
+  val KUBERNETES_KERBEROS_KRB5_FILE =
+ConfigBuilder("spark.kubernetes.kerberos.krb5location")
+  .doc("Specify the location of the krb5 file " +
--- End diff --

Presumably this is referring to the Kerberos configuration file indicating 
the location of the KDC?


---

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



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

2018-08-07 Thread ifilonenko
Github user ifilonenko commented on a diff in the pull request:

https://github.com/apache/spark/pull/21669#discussion_r208257021
  
--- Diff: core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala ---
@@ -336,7 +336,7 @@ private[spark] class SparkSubmit extends Logging {
 val targetDir = Utils.createTempDir()
 
 // assure a keytab is available from any place in a JVM
-if (clusterManager == YARN || clusterManager == LOCAL || 
isMesosClient) {
+if (clusterManager == YARN || clusterManager == LOCAL || isMesosClient 
|| isKubernetesCluster) {
--- End diff --

This check can be removed, but I included it since I believed that the 
keytab shouldn't be stored as a secret for security reasons and should instead 
be only accessible from the JVM. 


---

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



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

2018-08-02 Thread rvesse
Github user rvesse commented on a diff in the pull request:

https://github.com/apache/spark/pull/21669#discussion_r207299324
  
--- Diff: 
resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesClientApplication.scala
 ---
@@ -107,7 +109,14 @@ private[spark] class Client(
   def run(): Unit = {
 val resolvedDriverSpec = builder.buildFromFeatures(kubernetesConf)
 val configMapName = s"$kubernetesResourceNamePrefix-driver-conf-map"
-val configMap = buildConfigMap(configMapName, 
resolvedDriverSpec.systemProperties)
+val isKerberosEnabled = 
kubernetesConf.getTokenManager.isSecurityEnabled
+// HADOOP_SECURITY_AUTHENTICATION is defined as simple for the driver 
and executors as
+// they need only the delegation token to access secure HDFS, no need 
to sign in to Kerberos
+val maybeSimpleAuthentication =
+  if (isKerberosEnabled) Some((s"-D$HADOOP_SECURITY_AUTHENTICATION", 
"simple")) else None
--- End diff --

@ifilonenko Thanks, it does seem to be the case that this is required for 
regular jobs but blocks STS (at least using the old Spark on K8S code base).  
When we disabled it to enable use of STS in our internal fork it then broke 
regular user jobs.


---

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



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

2018-08-02 Thread ifilonenko
Github user ifilonenko commented on a diff in the pull request:

https://github.com/apache/spark/pull/21669#discussion_r207281949
  
--- Diff: 
resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesClientApplication.scala
 ---
@@ -107,7 +109,14 @@ private[spark] class Client(
   def run(): Unit = {
 val resolvedDriverSpec = builder.buildFromFeatures(kubernetesConf)
 val configMapName = s"$kubernetesResourceNamePrefix-driver-conf-map"
-val configMap = buildConfigMap(configMapName, 
resolvedDriverSpec.systemProperties)
+val isKerberosEnabled = 
kubernetesConf.getTokenManager.isSecurityEnabled
+// HADOOP_SECURITY_AUTHENTICATION is defined as simple for the driver 
and executors as
+// they need only the delegation token to access secure HDFS, no need 
to sign in to Kerberos
+val maybeSimpleAuthentication =
+  if (isKerberosEnabled) Some((s"-D$HADOOP_SECURITY_AUTHENTICATION", 
"simple")) else None
--- End diff --

I agree that the uses cases presented above require Kerberos login on the 
driver and executors. I will address these concerns in my followup commit. 


---

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



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

2018-08-01 Thread skonto
Github user skonto commented on a diff in the pull request:

https://github.com/apache/spark/pull/21669#discussion_r206970881
  
--- Diff: 
resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesClientApplication.scala
 ---
@@ -107,7 +109,14 @@ private[spark] class Client(
   def run(): Unit = {
 val resolvedDriverSpec = builder.buildFromFeatures(kubernetesConf)
 val configMapName = s"$kubernetesResourceNamePrefix-driver-conf-map"
-val configMap = buildConfigMap(configMapName, 
resolvedDriverSpec.systemProperties)
+val isKerberosEnabled = 
kubernetesConf.getTokenManager.isSecurityEnabled
+// HADOOP_SECURITY_AUTHENTICATION is defined as simple for the driver 
and executors as
+// they need only the delegation token to access secure HDFS, no need 
to sign in to Kerberos
+val maybeSimpleAuthentication =
+  if (isKerberosEnabled) Some((s"-D$HADOOP_SECURITY_AUTHENTICATION", 
"simple")) else None
--- End diff --

I agree users may want also to have non spark code that fetches a file from 
hdfs at the driver level.


---

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



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

2018-07-31 Thread nrchakradhar
Github user nrchakradhar commented on a diff in the pull request:

https://github.com/apache/spark/pull/21669#discussion_r206447871
  
--- Diff: 
resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesClientApplication.scala
 ---
@@ -107,7 +109,14 @@ private[spark] class Client(
   def run(): Unit = {
 val resolvedDriverSpec = builder.buildFromFeatures(kubernetesConf)
 val configMapName = s"$kubernetesResourceNamePrefix-driver-conf-map"
-val configMap = buildConfigMap(configMapName, 
resolvedDriverSpec.systemProperties)
+val isKerberosEnabled = 
kubernetesConf.getTokenManager.isSecurityEnabled
+// HADOOP_SECURITY_AUTHENTICATION is defined as simple for the driver 
and executors as
+// they need only the delegation token to access secure HDFS, no need 
to sign in to Kerberos
+val maybeSimpleAuthentication =
+  if (isKerberosEnabled) Some((s"-D$HADOOP_SECURITY_AUTHENTICATION", 
"simple")) else None
--- End diff --

This is our observation as well. Setting to "simple" is masking the actual 
setting. When we are trying kerberos with spark thrift server we are facing 
multiple issues and have stopped working on that for some time. We hope we can 
resume it in some time. If there are patches to include, we can give it a try.


---

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



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

2018-07-31 Thread rvesse
Github user rvesse commented on a diff in the pull request:

https://github.com/apache/spark/pull/21669#discussion_r204403925
  
--- Diff: 
resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesClientApplication.scala
 ---
@@ -107,7 +109,14 @@ private[spark] class Client(
   def run(): Unit = {
 val resolvedDriverSpec = builder.buildFromFeatures(kubernetesConf)
 val configMapName = s"$kubernetesResourceNamePrefix-driver-conf-map"
-val configMap = buildConfigMap(configMapName, 
resolvedDriverSpec.systemProperties)
+val isKerberosEnabled = 
kubernetesConf.getTokenManager.isSecurityEnabled
+// HADOOP_SECURITY_AUTHENTICATION is defined as simple for the driver 
and executors as
+// they need only the delegation token to access secure HDFS, no need 
to sign in to Kerberos
+val maybeSimpleAuthentication =
+  if (isKerberosEnabled) Some((s"-D$HADOOP_SECURITY_AUTHENTICATION", 
"simple")) else None
--- End diff --

I am unsure as to why this is needed.  When delegation tokens are available 
in the containers they should get used in preference to the Kerberos login 
anyway by the Hadoop UGI machinery.  And if you want to run applications that 
genuinely need to do a Kerberos login e.g. Spark Thrift Server then this 
prevents that


---

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



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

2018-07-11 Thread ifilonenko
Github user ifilonenko commented on a diff in the pull request:

https://github.com/apache/spark/pull/21669#discussion_r201696463
  
--- Diff: 
resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Constants.scala
 ---
@@ -81,4 +83,35 @@ private[spark] object Constants {
   val KUBERNETES_MASTER_INTERNAL_URL = "https://kubernetes.default.svc";
   val DRIVER_CONTAINER_NAME = "spark-kubernetes-driver"
   val MEMORY_OVERHEAD_MIN_MIB = 384L
+
+  // Hadoop Configuration
+  val HADOOP_FILE_VOLUME = "hadoop-properties"
+  val HADOOP_CONF_DIR_PATH = "/etc/hadoop/conf"
+  val ENV_HADOOP_CONF_DIR = "HADOOP_CONF_DIR"
+  val HADOOP_CONF_DIR_LOC = "spark.kubernetes.hadoop.conf.dir"
+  val HADOOP_CONFIG_MAP_SPARK_CONF_NAME =
+"spark.kubernetes.hadoop.executor.hadoopConfigMapName"
+
+  // Kerberos Configuration
+  val KERBEROS_DELEGEGATION_TOKEN_SECRET_NAME =
+"spark.kubernetes.kerberos.delegation-token-secret-name"
+  val KERBEROS_KEYTAB_SECRET_NAME =
+"spark.kubernetes.kerberos.key-tab-secret-name"
+  val KERBEROS_KEYTAB_SECRET_KEY =
+"spark.kubernetes.kerberos.key-tab-secret-key"
+  val KERBEROS_SPARK_USER_NAME =
+"spark.kubernetes.kerberos.spark-user-name"
+  val KERBEROS_SECRET_LABEL_PREFIX =
+"hadoop-tokens"
+  val SPARK_HADOOP_PREFIX = "spark.hadoop."
+  val HADOOP_SECURITY_AUTHENTICATION =
+SPARK_HADOOP_PREFIX + "hadoop.security.authentication"
+
+  // Kerberos Token-Refresh Server
+  val KERBEROS_REFRESH_LABEL_KEY = "refresh-hadoop-tokens"
--- End diff --

Agreed but that would be out of the scope of this PR as the renewal service 
is a separate micro-service (instead of a running thread), and thereby that 
logic would be housed in a separate PR governing the opinion of the renewal 
service pods "DT retrieving" protocol 


---

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



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

2018-07-11 Thread skonto
Github user skonto commented on a diff in the pull request:

https://github.com/apache/spark/pull/21669#discussion_r201685609
  
--- Diff: 
resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Constants.scala
 ---
@@ -81,4 +83,35 @@ private[spark] object Constants {
   val KUBERNETES_MASTER_INTERNAL_URL = "https://kubernetes.default.svc";
   val DRIVER_CONTAINER_NAME = "spark-kubernetes-driver"
   val MEMORY_OVERHEAD_MIN_MIB = 384L
+
+  // Hadoop Configuration
+  val HADOOP_FILE_VOLUME = "hadoop-properties"
+  val HADOOP_CONF_DIR_PATH = "/etc/hadoop/conf"
+  val ENV_HADOOP_CONF_DIR = "HADOOP_CONF_DIR"
+  val HADOOP_CONF_DIR_LOC = "spark.kubernetes.hadoop.conf.dir"
+  val HADOOP_CONFIG_MAP_SPARK_CONF_NAME =
+"spark.kubernetes.hadoop.executor.hadoopConfigMapName"
+
+  // Kerberos Configuration
+  val KERBEROS_DELEGEGATION_TOKEN_SECRET_NAME =
+"spark.kubernetes.kerberos.delegation-token-secret-name"
+  val KERBEROS_KEYTAB_SECRET_NAME =
+"spark.kubernetes.kerberos.key-tab-secret-name"
+  val KERBEROS_KEYTAB_SECRET_KEY =
+"spark.kubernetes.kerberos.key-tab-secret-key"
+  val KERBEROS_SPARK_USER_NAME =
+"spark.kubernetes.kerberos.spark-user-name"
+  val KERBEROS_SECRET_LABEL_PREFIX =
+"hadoop-tokens"
+  val SPARK_HADOOP_PREFIX = "spark.hadoop."
+  val HADOOP_SECURITY_AUTHENTICATION =
+SPARK_HADOOP_PREFIX + "hadoop.security.authentication"
+
+  // Kerberos Token-Refresh Server
+  val KERBEROS_REFRESH_LABEL_KEY = "refresh-hadoop-tokens"
--- End diff --

It would be good to have that option.


---

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



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

2018-07-05 Thread ifilonenko
Github user ifilonenko commented on a diff in the pull request:

https://github.com/apache/spark/pull/21669#discussion_r200401350
  
--- Diff: 
resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Constants.scala
 ---
@@ -81,4 +83,35 @@ private[spark] object Constants {
   val KUBERNETES_MASTER_INTERNAL_URL = "https://kubernetes.default.svc";
   val DRIVER_CONTAINER_NAME = "spark-kubernetes-driver"
   val MEMORY_OVERHEAD_MIN_MIB = 384L
+
+  // Hadoop Configuration
+  val HADOOP_FILE_VOLUME = "hadoop-properties"
+  val HADOOP_CONF_DIR_PATH = "/etc/hadoop/conf"
+  val ENV_HADOOP_CONF_DIR = "HADOOP_CONF_DIR"
+  val HADOOP_CONF_DIR_LOC = "spark.kubernetes.hadoop.conf.dir"
+  val HADOOP_CONFIG_MAP_SPARK_CONF_NAME =
+"spark.kubernetes.hadoop.executor.hadoopConfigMapName"
+
+  // Kerberos Configuration
+  val KERBEROS_DELEGEGATION_TOKEN_SECRET_NAME =
+"spark.kubernetes.kerberos.delegation-token-secret-name"
+  val KERBEROS_KEYTAB_SECRET_NAME =
+"spark.kubernetes.kerberos.key-tab-secret-name"
+  val KERBEROS_KEYTAB_SECRET_KEY =
+"spark.kubernetes.kerberos.key-tab-secret-key"
+  val KERBEROS_SPARK_USER_NAME =
+"spark.kubernetes.kerberos.spark-user-name"
+  val KERBEROS_SECRET_LABEL_PREFIX =
+"hadoop-tokens"
+  val SPARK_HADOOP_PREFIX = "spark.hadoop."
+  val HADOOP_SECURITY_AUTHENTICATION =
+SPARK_HADOOP_PREFIX + "hadoop.security.authentication"
+
+  // Kerberos Token-Refresh Server
+  val KERBEROS_REFRESH_LABEL_KEY = "refresh-hadoop-tokens"
--- End diff --

Because our original architecture had the opinion that the renewal service 
pod will exist as a separate micro-service, that option could be handled by 
that renewal service. We used this label to detect that this specific secret 
was to be renewed. But if we wished to use another renewal service via some 
existing service, we might be able to just grab an Array[Byte] from some 
DTManager that may exist in their external Hadoop clusters, and store in a 
secret. Thank you for this note in the design doc. 


---

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



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

2018-07-05 Thread ifilonenko
Github user ifilonenko commented on a diff in the pull request:

https://github.com/apache/spark/pull/21669#discussion_r200399452
  
--- Diff: 
resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala
 ---
@@ -211,6 +211,51 @@ private[spark] object Config extends Logging {
 "Ensure that major Python version is either Python2 or Python3")
   .createWithDefault("2")
 
+  val KUBERNETES_KERBEROS_SUPPORT =
+ConfigBuilder("spark.kubernetes.kerberos.enabled")
+  .doc("Specify whether your job is a job that will require a 
Delegation Token to access HDFS")
--- End diff --

agreed


---

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



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

2018-07-03 Thread skonto
Github user skonto commented on a diff in the pull request:

https://github.com/apache/spark/pull/21669#discussion_r199941392
  
--- Diff: 
resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Constants.scala
 ---
@@ -65,11 +65,13 @@ private[spark] object Constants {
   val ENV_CLASSPATH = "SPARK_CLASSPATH"
   val ENV_DRIVER_BIND_ADDRESS = "SPARK_DRIVER_BIND_ADDRESS"
   val ENV_SPARK_CONF_DIR = "SPARK_CONF_DIR"
+  val ENV_SPARK_USER = "SPARK_USER"
--- End diff --

I guess this is for setting the correct user. But I think hadoop libs 
should pick the correct user in SparkContext where `Utils.getCurrentUserName()` 
is used. In addition I think we should allow the pods to run with any username 
(eg. with customized images) and we should have a security context per 
container like:  `SecurityContextBuilder().withRunAsUser()`. In the latter 
scenario hadoop libraries will pick that logged in user as well.


---

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



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

2018-07-03 Thread skonto
Github user skonto commented on a diff in the pull request:

https://github.com/apache/spark/pull/21669#discussion_r199935864
  
--- Diff: 
resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/OptionRequirements.scala
 ---
@@ -0,0 +1,40 @@
+/*
+ * 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
+
+private[spark] object OptionRequirements {
+
+  def requireBothOrNeitherDefined(
+  opt1: Option[_],
+  opt2: Option[_],
+  errMessageWhenFirstIsMissing: String,
+  errMessageWhenSecondIsMissing: String): Unit = {
+requireSecondIfFirstIsDefined(opt1, opt2, 
errMessageWhenSecondIsMissing)
+requireSecondIfFirstIsDefined(opt2, opt1, errMessageWhenFirstIsMissing)
+  }
+
+  def requireSecondIfFirstIsDefined(
+  opt1: Option[_], opt2: Option[_], errMessageWhenSecondIsMissing: 
String): Unit = {
+opt1.foreach { _ =>
+  require(opt2.isDefined, errMessageWhenSecondIsMissing)
+}
+  }
+
+  def requireNandDefined(opt1: Option[_], opt2: Option[_], errMessage: 
String): Unit = {
--- End diff --

`requireNandDefined` is spelling ok?


---

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



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

2018-07-03 Thread skonto
Github user skonto commented on a diff in the pull request:

https://github.com/apache/spark/pull/21669#discussion_r199806583
  
--- Diff: 
resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Constants.scala
 ---
@@ -81,4 +83,35 @@ private[spark] object Constants {
   val KUBERNETES_MASTER_INTERNAL_URL = "https://kubernetes.default.svc";
   val DRIVER_CONTAINER_NAME = "spark-kubernetes-driver"
   val MEMORY_OVERHEAD_MIN_MIB = 384L
+
+  // Hadoop Configuration
+  val HADOOP_FILE_VOLUME = "hadoop-properties"
+  val HADOOP_CONF_DIR_PATH = "/etc/hadoop/conf"
+  val ENV_HADOOP_CONF_DIR = "HADOOP_CONF_DIR"
+  val HADOOP_CONF_DIR_LOC = "spark.kubernetes.hadoop.conf.dir"
+  val HADOOP_CONFIG_MAP_SPARK_CONF_NAME =
+"spark.kubernetes.hadoop.executor.hadoopConfigMapName"
+
+  // Kerberos Configuration
+  val KERBEROS_DELEGEGATION_TOKEN_SECRET_NAME =
+"spark.kubernetes.kerberos.delegation-token-secret-name"
+  val KERBEROS_KEYTAB_SECRET_NAME =
+"spark.kubernetes.kerberos.key-tab-secret-name"
+  val KERBEROS_KEYTAB_SECRET_KEY =
+"spark.kubernetes.kerberos.key-tab-secret-key"
+  val KERBEROS_SPARK_USER_NAME =
+"spark.kubernetes.kerberos.spark-user-name"
+  val KERBEROS_SECRET_LABEL_PREFIX =
+"hadoop-tokens"
+  val SPARK_HADOOP_PREFIX = "spark.hadoop."
+  val HADOOP_SECURITY_AUTHENTICATION =
+SPARK_HADOOP_PREFIX + "hadoop.security.authentication"
+
+  // Kerberos Token-Refresh Server
+  val KERBEROS_REFRESH_LABEL_KEY = "refresh-hadoop-tokens"
--- End diff --

I left a comment also in the design doc, can we also provide the option for 
using an existing renewal service like when integrating with an external hadoop 
cluster where people already have that. This is how it worked for mesos so far.


---

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



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

2018-07-03 Thread skonto
Github user skonto commented on a diff in the pull request:

https://github.com/apache/spark/pull/21669#discussion_r199803715
  
--- Diff: 
resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala
 ---
@@ -211,6 +211,51 @@ private[spark] object Config extends Logging {
 "Ensure that major Python version is either Python2 or Python3")
   .createWithDefault("2")
 
+  val KUBERNETES_KERBEROS_SUPPORT =
+ConfigBuilder("spark.kubernetes.kerberos.enabled")
+  .doc("Specify whether your job is a job that will require a 
Delegation Token to access HDFS")
--- End diff --

I think kerberos goes beyond DTs so it shouldnt be specific to that. Also I 
think you dont need the user to pass that. You just need to call: 
UserGroupInformation.isSecurityEnabled


---

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



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

2018-07-03 Thread skonto
Github user skonto commented on a diff in the pull request:

https://github.com/apache/spark/pull/21669#discussion_r199801815
  
--- Diff: core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala ---
@@ -336,7 +336,7 @@ private[spark] class SparkSubmit extends Logging {
 val targetDir = Utils.createTempDir()
 
 // assure a keytab is available from any place in a JVM
-if (clusterManager == YARN || clusterManager == LOCAL || 
isMesosClient) {
+if (clusterManager == YARN || clusterManager == LOCAL || isMesosClient 
|| isKubernetesCluster) {
--- End diff --

This check has been restrictive for customers in the past. There are cases 
where spark submit should not have the file locally and keytab should be 
mounted as a secret within the cluster.


---

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



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

2018-06-29 Thread ifilonenko
GitHub user ifilonenko opened a pull request:

https://github.com/apache/spark/pull/21669

[SPARK-23257][K8S][WIP] Kerberos Support for Spark on K8S

## What changes were proposed in this pull request?
This is the work on setting up Secure HDFS interaction with Spark-on-K8S.
The architecture is discussed in this community-wide google 
[doc](https://docs.google.com/document/d/1RBnXD9jMDjGonOdKJ2bA1lN4AAV_1RwpU_ewFuCNWKg)
This initiative can be broken down into 4 Stages

**STAGE 1**
- [x] Detecting `HADOOP_CONF_DIR` environmental variable and using Config 
Maps to store all Hadoop config files locally, while also setting 
`HADOOP_CONF_DIR` locally in the driver / executors

**STAGE 2**
- [x] Grabbing `TGT` from `LTC` or using keytabs+principle and creating a 
`DT` that will be mounted as a secret or using a pre-populated secret

**STAGE 3**
- [x] Driver 

**STAGE 3**
- [ ] Executor 

## How was this patch tested?

- [ ] E2E Integration tests
- [ ] Unit tests

## Docs and Error Handling?
- [ ] Docs
- [ ] Error Handling

You can merge this pull request into a Git repository by running:

$ git pull https://github.com/ifilonenko/spark secure-hdfs

Alternatively you can review and apply these changes as the patch at:

https://github.com/apache/spark/pull/21669.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

This closes #21669


commit 73f2777c7ef1b925d323599952c792b5605d5c72
Author: Ilan Filonenko 
Date:   2018-06-29T08:41:18Z

initial Driver logic for Hadoop and Kerberos Support




---

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