You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by arinconstrio <gi...@git.apache.org> on 2017/02/03 14:21:54 UTC

[GitHub] spark pull request #16788: [WIP][SPARK-16742] Kerberos impersonation support

GitHub user arinconstrio opened a pull request:

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

    [WIP][SPARK-16742] Kerberos impersonation support

    ## What changes were proposed in this pull request?
    
    Solution for use kerberized hdfs and use realtime impersonation.
    
    Document at jira [https://issues.apache.org/jira/browse/SPARK-16742](https://issues.apache.org/jira/browse/SPARK-16742)
    


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

    $ git pull https://github.com/Stratio/spark kerberos-support

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

    https://github.com/apache/spark/pull/16788.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 #16788
    
----
commit 23157cc580ca7c99f7a7e9115a28f99688f87352
Author: arinconstrio <ar...@stratio.com>
Date:   2016-08-31T11:48:05Z

    Kerberos impersonation support

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #16788: [SPARK-16742] Kerberos impersonation support

Posted by mridulm <gi...@git.apache.org>.
Github user mridulm commented on the issue:

    https://github.com/apache/spark/pull/16788
  
    @tgravescs , @vanzin - this PR for mesos changes how spark handles kerberos tokens fundamentally; would be good to have your views.
    
    +CC @jerryshao to also look at the PR, since you have worked on yarn security changes


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #16788: [SPARK-16742] Kerberos impersonation support

Posted by arinconstrio <gi...@git.apache.org>.
Github user arinconstrio commented on the issue:

    https://github.com/apache/spark/pull/16788
  
    We are going to continue working on our solution which implies much more than a feature for Mesos, so I close this PR, and create a new one in the short future. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #16788: [SPARK-16742] Kerberos impersonation support

Posted by jerryshao <gi...@git.apache.org>.
Github user jerryshao commented on the issue:

    https://github.com/apache/spark/pull/16788
  
    >Trying to put it differently: if Spark had its own, secure method for distributing the initial set of delegation tokens needed by the executors (+ AM in case of YARN), then the YARN backend wouldn't need to use amContainer.setTokens at all. What I'm suggesting here is that this method be the base of the Mesos / Kerberos integration; and later we could change YARN to also use it.
    
    >This particular code is pretty self-contained and is the base of what you need here to bootstrap things. Moving it to "core" wouldn't be that hard, I think. The main thing would be to work on how the initial set of tokens is sent to executors, since that is the only thing YARN does for Spark right now.
    
    Agreed, I'm also thinking about it, the main thing currently only Spark on YARN can support DT (delegation token) is that yarn could help propagate DTs in bootstrapping. If Spark has a common solution for this, then Spark could support accessing kerberized services under different cluster manages. One simple way as I prototyped before is to pass serialized credentials as executor launch command argument, then when executor launched, deserialize the credential and set to UGI.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #16788: [SPARK-16742] Kerberos impersonation support

Posted by mridulm <gi...@git.apache.org>.
Github user mridulm commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16788#discussion_r105319198
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala ---
    @@ -151,9 +152,13 @@ object SparkSubmit extends CommandLineUtils {
         val (childArgs, childClasspath, sysProps, childMainClass) = prepareSubmitEnvironment(args)
     
         def doRunMain(): Unit = {
    +      if (args.principal != null && args.keytab!= null) {
    +        KerberosUser.securize(args.principal, args.keytab)
    --- End diff --
    
    This will cause multiple UGI.loginUserFromKeytab (in yarn case it happens in SparkSubmit.prepareSubmitEnvironment) - which causes various issues.
    In an application, there must be only one call to UGI.loginUserFromKeytab; if more than one, random things fail (dfs client, metastore, etc) due to the way the loginUser is cached/used and hadoop ipc renews unfortunately.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #16788: [WIP][SPARK-16742] Kerberos impersonation support

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/16788
  
    Can one of the admins verify this patch?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #16788: [SPARK-16742] Kerberos impersonation support

Posted by mridulm <gi...@git.apache.org>.
Github user mridulm commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16788#discussion_r105319503
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala ---
    @@ -1028,7 +1028,7 @@ class DAGScheduler(
                 val locs = taskIdToLocations(id)
                 new ResultTask(stage.id, stage.latestInfo.attemptId,
                   taskBinary, part, locs, id, properties, serializedTaskMetrics,
    -              Option(jobId), Option(sc.applicationId), sc.applicationAttemptId)
    +              Option(jobId), Option(sc.applicationId), sc.applicationAttemptId, Option(tokens))
               }
    --- End diff --
    
    Current spark model in yarn for managing tokens is to do it out of band with the actual tasks (unlike tez/MR iirc : where then the execution model is itself different).
    The tasks themselves do not propagate the credentials - the executors directly update the credentials based on driver updates.
    
    This allows for very long running spark tasks (> 24 hours for example) to run - which per task tokens might not allow.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #16788: [SPARK-16742] Kerberos impersonation support

Posted by arinconstrio <gi...@git.apache.org>.
Github user arinconstrio closed the pull request at:

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #16788: [SPARK-16742] Kerberos impersonation support

Posted by mgummelt <gi...@git.apache.org>.
Github user mgummelt commented on the issue:

    https://github.com/apache/spark/pull/16788
  
    @vanzin When you say "distributing the principal's credentials", I take it you mean that the driver logs in via Kerberos, and submits the resulting token (TGT?) via `amContainer.setTokens`.  That's what I understand from reading the code.  Whereas the Hadoop delegation tokens are distributed via HDFS itself.  Is this correct?
    
    I think this is necessary for YARN, because in both client and cluster mode, the `ApplicationMaster` runs remotely, correct?  In Mesos client mode, the scheduler runs in the same process as `spark-submit` (the driver), so there's no need for Kerberos token distribution.  The scheduler can simply use the `UserGroupInformation` the user initially logged in with.
    
    We would need some method of Kerberos token distribution in cluster mode, but we can punt on that.  Users have many ways of running Spark jobs asynchronously, and we'll have to take those one by one.  I think we can just focus on solving this in client mode for now.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #16788: [SPARK-16742] Kerberos impersonation support

Posted by mridulm <gi...@git.apache.org>.
Github user mridulm commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16788#discussion_r105526180
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala ---
    @@ -1006,7 +1006,7 @@ class DAGScheduler(
             runningStages -= stage
             return
         }
    -
    +    val tokens = KerberosUtil.getHadoopDelegationTokens
    --- End diff --
    
    
    See my comment about below about how current spark (on yarn) handles security https://github.com/apache/spark/pull/16788/files#r105319503.
    
    @tgravescs or @vanzin can correct me if I am wrong (in case I am misremembering) - 
    In a secure hdfs, it is not necessary for principal/keytab to be provided - if the job will finishes before token renewal is necessitated.
    
    Given above : The call chain in KerberosUtil.getHadoopDelegationTokens will throw an exception if they are missing if ugi security is enabled.
    I am not sure if this is a requirement in mesos, but it is not for yarn.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #16788: [SPARK-16742] Kerberos impersonation support

Posted by mgummelt <gi...@git.apache.org>.
Github user mgummelt commented on the issue:

    https://github.com/apache/spark/pull/16788
  
    A few high-level issues:
    
    - The title of this PR seems misleading.  This is about Kerberos support in general, not just proxy user support.
    - There's no delegation token renewer.  Executors will start failing to read data once their delegation tokens expire.
    - This conflicts with how YARN handles kerberos auth, which has the `ApplicationMaster` login via the keytab, and executors read the delegation tokens from HDFS via the `CredentialUpdater`.  It's going to be very hard to generalize that solution to Mesos.  Since the subject of this JIRA is "Kerberos support on Mesos", I prefer we keep this solution Mesos specific, and just aim for a consistent user interface (--keytab, --principal, --proxy-user). 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #16788: [SPARK-16742] Kerberos impersonation support

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on the issue:

    https://github.com/apache/spark/pull/16788
  
    > Ah, I didn't realize that the --keytab parameter was expected to be an HDFS location. Thanks.
    
    It's not. It's just how the YARN module chose to distribute the keytab.
    
    But the main point is that support for kerberos should not require a keytab. All you need is a kerberos login, which /does not require a keytab/ (you can use "kinit" to log in with your password).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #16788: [SPARK-16742] Kerberos impersonation support

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on the issue:

    https://github.com/apache/spark/pull/16788
  
    > I take it you mean that the driver logs in via Kerberos, and submits the resulting token (TGT?) via amContainer.setTokens
    
    No. `amContainer.setTokens` is used to distribute delegation tokens; the TGT remains only on the "gateway" node (the driver in client mode, or the "launcher" in cluster mode).
    
    > Whereas the Hadoop delegation tokens are distributed via HDFS itself.
    
    That's separate. That's Spark-specific code that distributes new delegation tokens and doesn't really depend on YARN. Which is a reason why I suggested the refactoring, since once you solve the initial token distribution, that code should work for Mesos without the need to change anything.
    
    Trying to put it differently: if Spark had its own, secure method for distributing the initial set of delegation tokens needed by the executors (+ AM in case of YARN), then the YARN backend wouldn't need to use `amContainer.setTokens` at all. What I'm suggesting here is that this method be the base of the Mesos / Kerberos integration; and later we could change YARN to also use it.
    
    This particular code is pretty self-contained and is the base of what you need here to bootstrap things. Moving it to "core" wouldn't be that hard, I think. The main thing would be to work on how the initial set of tokens is sent to executors, since *that* is the only thing YARN does for Spark right now.
    
    > I'm worried it's going to a) be quite a chore to factor out the YARN Kerberos code
    
    I'm not saying that it will be a walk in the park, but it's a much better solution than creating a completely separate way of dealing with Kerberos just for Mesos.
    
    >  I've never setup a YARN cluster. How difficult is it?
    
    Manually, probably complicated; I've only ever done it using our internal tools (based on Cloudera Manager).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #16788: [SPARK-16742] Kerberos impersonation support

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on the issue:

    https://github.com/apache/spark/pull/16788
  
    Hello all, if you're not going to update this PR then it should be closed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #16788: [SPARK-16742] Kerberos impersonation support

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on the issue:

    https://github.com/apache/spark/pull/16788
  
    I took a very quick look at this; Mridul and Saisai have already raised good questions about this.
    
    But I'm a little worried that this is creating yet another way of dealing with Kerberos that will lead to two different code paths that don't share much with each other.
    
    Spark-on-YARN's support for Kerberos is really nothing complicated. The only thing that Spark needs from YARN is a method of distributing the principal's credentials. YARN just happens to have an API for that (see `setupSecurityToken` in Client.scala).
    
    On the other side, YARN just writes the credentials to a file and sets `HADOOP_TOKEN_FILE_LOCATION` in the container process's environment to point to the file; `UserGroupInformation` reads that automatically, and things work.
    
    So it seems to me that all that would be needed from Mesos is a way to replicate that behavior (i.e. distribute these credentials securely). The rest (including the code to deal with delegation tokens) could just be factored out of the YARN module, which should be easy now that Spark depends on Hadoop 2.6.
    
    Also, note that `--principal` and `--keytab` are *not* required to run a Spark application in a kerberized cluster. Things should also work when you just login using `kinit` and run the Spark app. `UserGroupInformation` already handles that case for you.



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #16788: [SPARK-16742] Kerberos impersonation support

Posted by mridulm <gi...@git.apache.org>.
Github user mridulm commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16788#discussion_r105319759
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/KerberosUtil.scala ---
    @@ -0,0 +1,108 @@
    +/*
    + * 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.scheduler
    +
    +import java.security.PrivilegedExceptionAction
    +
    +import org.apache.hadoop.conf.Configuration
    +import org.apache.hadoop.fs.{FileSystem, Path}
    +import org.apache.hadoop.mapred.Master
    +import org.apache.hadoop.security.{Credentials, UserGroupInformation}
    +import org.apache.spark.deploy.SparkHadoopUtil
    +import org.apache.spark.{SparkConf, SparkEnv, SparkException}
    +import org.apache.spark.internal.Logging
    +
    +object KerberosUtil  extends Logging {
    +  var proxyUser : Option[UserGroupInformation] = None
    +  def securize (principal: String, keytab: String) : Unit = {
    +    val hadoopConf = SparkHadoopUtil.get.newConfiguration(new SparkConf())
    +    hadoopConf.set("hadoop.security.authentication", "Kerberos")
    +    UserGroupInformation.setConfiguration(hadoopConf)
    +    UserGroupInformation.loginUserFromKeytab(principal, keytab)
    +  }
    +
    +
    +  def getHadoopDelegationTokens : Array[Byte] = {
    --- End diff --
    
    Given the frequency (and where it is invoked from) of this method - how expensive is it ?
    +CC @tgravescs, @vanzin  


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #16788: [SPARK-16742] Kerberos impersonation support

Posted by mgummelt <gi...@git.apache.org>.
Github user mgummelt commented on the issue:

    https://github.com/apache/spark/pull/16788
  
    Ah, I didn't realize that the --keytab parameter was expected to be an HDFS location.  Thanks.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #16788: [SPARK-16742] Kerberos impersonation support

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on the issue:

    https://github.com/apache/spark/pull/16788
  
    > In order to renew delegation tokens, the ApplicationMaster needs access to the keytab, right? 
    
    Yes.
    
    > So why must the driver send delegation tokens to the ApplicationMaster, if the ApplicationMaster already has access to the keytab, and can thus fetch delegation tokens itself?
    
    The AM reads the keytab from HDFS. Without the initial delegation token, the AM cannot access HDFS to read the keytab. Chicken & egg.
    
    Also note, as I said before, that `--principal` and `--keytab` are *not* required for kerberos support. They are only required for creating new delegation tokens after the original ones exceed their maximum life. If you apps won't run for more than that, you don't need `--principal` and `--keytab` at all.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #16788: [SPARK-16742] Kerberos impersonation support

Posted by jerryshao <gi...@git.apache.org>.
Github user jerryshao commented on the issue:

    https://github.com/apache/spark/pull/16788
  
    Shipping tokens with tasks may have a big issue as discussed with @mridulm before. Some Spark applications has long running out-band operations to community with HDFS, like Spark Streaming's WAL, Structured Streaming's checkpoint. What if token is expired and new token which shipped with task hasn't fetched, the operations will be failed. So shipping with tasks is not workable.
    
    Also here only handles Hadoop FileSystem DTs, and only supports default FS, this is not feature comparable to Spark on YARN.
    
    And your changes will also effect standalone mode, is that your expectation?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #16788: [SPARK-16742] Kerberos impersonation support

Posted by mridulm <gi...@git.apache.org>.
Github user mridulm commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16788#discussion_r105318927
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala ---
    @@ -257,10 +257,6 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S
               "either HADOOP_CONF_DIR or YARN_CONF_DIR must be set in the environment.")
           }
         }
    -
    -    if (proxyUser != null && principal != null) {
    -      SparkSubmit.printErrorAndExit("Only one of --proxy-user or --principal can be provided.")
    -    }
    --- End diff --
    
    This validation is relevant for spark yarn support, and should not be removed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #16788: [SPARK-16742] Kerberos impersonation support

Posted by mgummelt <gi...@git.apache.org>.
Github user mgummelt commented on the issue:

    https://github.com/apache/spark/pull/16788
  
    @vanzin Consolidating a Mesos and YARN kerberos solution sounds nice, but it does worry me.  I'm worried it's going to a) be quite a chore to factor out the YARN Kerberos code, and more importantly b) be difficult to test on YARN to ensure we haven't broken anything.  I've never setup a YARN cluster.  How difficult is it?
    
    I already have a method for bringing up a Kerberized HDFS cluster.  Should we be able to test Spark on YARN against this cluster, or does HDFS need to be running under the same YARN ResourceManager (I don't see any reason why it should)?
    



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #16788: [SPARK-16742] Kerberos impersonation support

Posted by mridulm <gi...@git.apache.org>.
Github user mridulm commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16788#discussion_r105526086
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/KerberosUser.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.scheduler
    +
    +import org.apache.hadoop.security.{Credentials, UserGroupInformation}
    +import org.apache.spark.{SparkConf, SparkEnv, SparkException}
    +import org.apache.spark.deploy.SparkHadoopUtil
    +import org.apache.spark.internal.Logging
    +
    +object KerberosUser extends Logging {
    +
    +  def securize (principal: String, keytab: String) : Unit = {
    +    val hadoopConf = SparkHadoopUtil.get.newConfiguration(new SparkConf())
    +    hadoopConf.set("hadoop.security.authentication", "Kerberos")
    +    UserGroupInformation.setConfiguration(hadoopConf)
    +    UserGroupInformation.loginUserFromKeytab(principal, keytab)
    +  }
    --- End diff --
    
    This method is duplicated in KerberosUtil ?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #16788: [SPARK-16742] Kerberos impersonation support

Posted by mridulm <gi...@git.apache.org>.
Github user mridulm commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16788#discussion_r105526225
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala ---
    @@ -60,9 +60,10 @@ private[spark] class ResultTask[T, U](
         serializedTaskMetrics: Array[Byte],
         jobId: Option[Int] = None,
         appId: Option[String] = None,
    -    appAttemptId: Option[String] = None)
    +    appAttemptId: Option[String] = None,
    +    tokens: Option[Array[Byte]] = None)
       extends Task[U](stageId, stageAttemptId, partition.index, localProperties, serializedTaskMetrics,
    -    jobId, appId, appAttemptId)
    +    jobId, appId, appAttemptId,tokens)
    --- End diff --
    
    Changes to both *Task.scala is changing the security model for not just mesos, but also yarn - and this is incompatible with existing public api (credential managers, etc)  : unless the PR is planning to overhaul the security in spark for all cluster managers.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #16788: [SPARK-16742] Kerberos impersonation support

Posted by mgummelt <gi...@git.apache.org>.
Github user mgummelt commented on the issue:

    https://github.com/apache/spark/pull/16788
  
    In order to renew delegation tokens, the `ApplicationMaster` needs access to the keytab, right? https://github.com/apache/spark/blob/master/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/AMCredentialRenewer.scala#L84
    
    So why must the driver send delegation tokens to the ApplicationMaster, if the ApplicationMaster already has access to the keytab, and can thus fetch delegation tokens itself?  Unless maybe those tokens are only used for the non-renewal case, when the keytab isn't specified?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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