Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[SPARK-5158] [core] [security] Spark standalone mode can authenticate against a Kerberos-secured Hadoop cluster #4106

Closed
wants to merge 1 commit into from

Conversation

mccheah
Copy link
Contributor

@mccheah mccheah commented Jan 19, 2015

Previously, Kerberos secured Hadoop clusters could only be accessed by Spark running on top of YARN. In other words, Spark standalone clusters had no way to read from secure Hadoop clusters. Other solutions were proposed previously, but all of them attempted to perform authentication by obtaining
a token on a single node and passing that token around to all of the other Spark worker nodes. The shipping of the token is risky, and all previous iterations fell short in leaving the token open to man-in-the-middle attacks.

This patch introduces an alternative approach. It assumes that the keytab file has already been distributed to every node in the cluster. When Spark starts in standalone mode, all of the workers individually log in via Kerberos using the principal and keytab file specified in hdfs-site.xml. We can assume this will be well formed because on standalone configurations all of the worker nodes should be using the same hdfs-site.xml configurations as the Hadoop cluster itself. In addition, on basic Hadoop cluster setups the key tab file is often already manually deployed on all of the cluster's nodes; it's not a huge stretch to expect the keytab files to be deployed to the Spark worker nodes as well, if they are not already there.

There are a number of caveats to this approach. Firstly, it assumes that Spark will always authenticate with Kerberos using the same principal and keytab, and the login is done at the very start of the job. Strictly speaking we should be trying to reduce the surface area of the region of code that operate under a logged-in state. Or to put it another way, the authentication should only be performed precisely when files are written or read from HDFS, and after the read or write is performed the subject should be logged out. However this is difficult to write and prone to errors, so this is left for a future refactor.

More concerningly, the code does not actually execute "kinit", and each of the executor nodes need to run kinit manually before starting the job. It is suggested that a call to kinit with the appropriate principal and keytab is done in spark-env.sh, and we should document this as being the case. I remark that UserGroupInformation.loginUserFromKeytab(...) does not actually run kinit, but merely creates a "delegation token", but doing this without running kinit still makes the Spark job crash with an exception message along the lines of "Unable to find tgt...". Any suggestions as to how to actually execute "kinit" in Java are appreciated; a system call is flaky at best due to cross-platform issues.

cc @mingyukim, @pwendell , @ash211

@mccheah
Copy link
Contributor Author

mccheah commented Jan 19, 2015

Suggestions to unit test are welcome. This should not be merged until it is unit-tested.

@mccheah
Copy link
Contributor Author

mccheah commented Jan 19, 2015

One other caveat I forgot to mention, and the commit message should be updated and this reflected in the docs: User proxying needs to be enabled. Basically, the user principal configured to access the Hadoop cluster should not be the user that is running the spark job. Suppose the Hadoop principal is userHadoop and the spark user is userSpark. When I run the Spark job, I need userHadoop's privileges, but don't want to run the job as userHadoop. So we have userHadoop impersonate userSpark by using UserGroupInformation.createProxyUser(). HOWEVER this requires that user proxies are configured properly. For details see http://hadoop.apache.org/docs/r1.2.1/Secure_Impersonation.html

@SparkQA
Copy link

SparkQA commented Jan 19, 2015

Test build #25768 has finished for PR 4106 at commit 5a7bd66.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds the following public classes (experimental):
    • abstract class SparkHadoopUtil extends Logging

array(i) = new HadoopPartition(id, i, inputSplits(i))
}
array
SparkHadoopUtil.get.runAsSparkUser(() => {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I don't think we want to do this here (we definitely don't on yarn), see the comment on runAsSparkUser. Generally you want to do the runAsSparkUser once as high up as possible

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Am I supposed to do this when the driver starts then? Because the driver needs it in this case. I don't know how YARN gets around this.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Ah - if the YARN Client executes this code, it already inherits the permissions set up by the YARN environment, which makes running this unnecessary. Whereas if this is run in standalone mode, we need to explicitly run this code as the YARN User at the driver.

I don't want to wrap all of SparkContext.runJob in a runAsSparkUser call. We might as well localize the things that need HDFS permissions to the bits of code that actually need it.

@mccheah
Copy link
Contributor Author

mccheah commented Jan 21, 2015

That¹s correct. Definitely a work-in-progress so if there¹s another security
model you¹d recommend I¹m all ears!

-Matt Cheah

From: Tom Graves notifications@github.com
Reply-To: apache/spark
<reply+0088f41930c79f00294e606af373daf9a0308eb6f043646792cf0000000110d7995c9
2a169ce03444a22@reply.github.com>
Date: Wednesday, January 21, 2015 at 8:44 AM
To: apache/spark spark@noreply.github.com
Cc: Matt Cheah mcheah@palantir.com
Subject: Re: [spark] [SPARK-5158] [core] [security] Spark standalone mode
can authenticate against a Kerberos-secured Hadoop cluster (#4106)

So are you trying to add security such that spark cluster would run as one
superuser who would have to be configured as proxy user on the hadoop
cluster and then each job would run as the specific user?


Reply to this email directly or view it on GitHub
<https://urldefense.proofpoint.com/v2/url?u=https-3A__github.com_apache_spar
k_pull_4106-23issuecomment-2D70872378&d=AwMCaQ&c=izlc9mHr637UR4lpLEZLFFS3Vn2
UXBrZ4tFb6oOnmz8&r=hzwIMNQ9E99EMYGuqHI0kXhVbvX3nU3OSDadUnJxjAs&m=YgwVV9t2v9R
y5RkEKJsZNBxeNTrj5sVVWOPWffa9b5U&s=zoJbhDpiW5-A9HARNP4H6RbwB1KjGQ1PDiW8G_vOi
9k&e=> .

@mccheah
Copy link
Contributor Author

mccheah commented Feb 2, 2015

Suggestions make sense. I'm currently on a business trip so it might be a bit of time before I can get back to this.

@SparkQA
Copy link

SparkQA commented Feb 2, 2015

Test build #26533 has finished for PR 4106 at commit 5a7bd66.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds the following public classes (experimental):
    • abstract class SparkHadoopUtil extends Logging

@mccheah
Copy link
Contributor Author

mccheah commented Feb 2, 2015

@pwendell do we need this for Spark 1.3.0? Is the feature merge deadline already past? I'm uncertain of what my bandwidth will be like but if it needs to be sped up I can look into squeezing in some bandwidth for this, despite the traveling-work.

cc @mingyukim

@pwendell
Copy link
Contributor

pwendell commented Feb 4, 2015

Hey @mccheah - if you are too busy I think it's fine to let it slip past 1.3, given that there are still several unknowns.

@mccheah
Copy link
Contributor Author

mccheah commented Feb 17, 2015

Able to come back to this now!

@mccheah
Copy link
Contributor Author

mccheah commented Feb 20, 2015

Okay, so I actually ran the history server on the spark Master branch with Kerberos authentication, and without running kinit, I got the following error when launching spark shell:

WARN security.UserGroupInformation: PriviledgedActionException as:mcheah (auth:KERBEROS) cause:javax.security.sasl.SaslException: GSS initiate failed [Caused by GSSException: No valid credentials provided (Mechanism level: Failed to find any Kerberos tgt)]
15/02/20 11:54:57 WARN ipc.Client: Exception encountered while connecting to the server : javax.security.sasl.SaslException: GSS initiate failed [Caused by GSSException: No valid credentials provided (Mechanism level: Failed to find any Kerberos tgt)]
15/02/20 11:54:57 WARN security.UserGroupInformation: PriviledgedActionException as:mcheah (auth:KERBEROS) cause:java.io.IOException: javax.security.sasl.SaslException: GSS initiate failed [Caused by GSSException: No valid credentials provided (Mechanism level: Failed to find any Kerberos tgt)]
java.io.IOException: Failed on local exception: java.io.IOException: javax.security.sasl.SaslException: GSS initiate failed [Caused by GSSException: No valid credentials provided (Mechanism level: Failed to find any Kerberos tgt)]; Host Details : local host is: "mcheah-VirtualBox/127.0.1.1"; destination host is: "localhost":9000;

When I run kinit before launching my spark shell, the context is created successfully and the history server works as normal.

This indicates that the need to run kinit isn't just limited to my patch here. At any rate, as long as the process required to write the event logs to kerberos-secured HDFS, is the same process required to read/write Spark RDDs from kerberos-secured HDFS, then we shouldn't have much to worry about.

@tgravescs
Copy link
Contributor

What does running spark-shell have to do with launching the history server without doing kinit?

history server is a standalone process that simply reads from HDFS and then displays that information back to the user. In order to test that you just start history server without kinit and see if you can view the already existing application logs. Or look in the history server logs to see what its reading.

Running spark-shell with and without running kinit has nothing to do with history server itself.

@mccheah
Copy link
Contributor Author

mccheah commented Feb 20, 2015

Ah, I mixed up the history server with the event log directory. Let me try getting the history server up and see.

@mccheah
Copy link
Contributor Author

mccheah commented Feb 20, 2015

I get the same exception when I try to start the history server without running kinit first. My settings are:

spark.history.kerberos.enabled true
spark.history.kerberos.principal hdfs/localhost@LOCALHOST
spark.history.kerberos.keytab [keytab-file]
spark.history.fs.logDirectory hdfs://localhost:9000/spark-events

The stack trace that appears is very similar to the one I mentioned above already.

If I run kinit -kt [keytab] hdfs/localhost@LOCALHOST first, the history server starts up properly. Furthermore, the history server can move between running and stopped states freely without running kinit again, until I run kdestroy -A, after which I need to run kinit again before starting the history server.

It's possible I just messed up my configuration of HDFS and/or Kerberos in general. However I also know that if kinit is run once, the ticket can live forever if it is set to auto-renew indefinitely. That would explain how for many users who just run kinit before doing hadoop commands on a regular basis anyways, it would appear they need to do nothing special for launching the history server, since it's automatically picking up the self-renewing ticket.

@harishreedharan
Copy link
Contributor

From what I can see, the approach is to login using the same principal and keytab on all machines and login on every machine using that. This has a couple of issues (at least):

  • the KDC will block off the credentials thinking it is a DDoS attack when so many of these logins happen at almost the same time - resulting in the whole thing failing. Hadoop uses delegation tokens (for running user apps) and principals specific to each host to avoid this (for headless users)
  • Distributing the keytabs is an issue as well.
  • Another issue is that since all applications are running as the same user on each machine (spark) which can read all keytabs, one user would be able to write an app that can be used to read another user's keytab. This is a major security issue - and I don't see a way to avoid this. (credit to @vanzin for finding this one)

I think the best way to fix this would be to find a way to ensure that the delegation tokens can be transferred securely (with authentication within the same app, and encryption to avoid snooping). This would fix all of these issues

@mccheah
Copy link
Contributor Author

mccheah commented Feb 24, 2015

When @pwendell and I originally discussed the feature, we wanted to design it to be simple and usable for small dedicated Spark clusters. We also explicitly wanted to avoid the approach of transferring delegation tokens; every attempt to do so in the past was prone to security flaws. So in response to these concerns:

(1) With a cluster of ~10 machines, will the credentials blocking still happen? How many is "many"?

(2) I assumed that the user will be responsible for securely transferring the keytabs to every machine. I don't think that it is necessarily Spark's responsibility to automatically transfer the keytabs securely. If the user wants to leverage this, I would think they're already aware of security measures that need to be taken in transferring keys between hosts. At any rate we can document this explicitly.

(3) I'm confused here - Spark can't read a keytab if the permissions on the keytab file deny access. Again, this comes down to configuration - the Spark user should be configured to be able to access keytabs on a need-to-know basis. And I think the bottom line is that we should still give this to users and the users have the ultimate responsibility of using it wisely. If the Spark cluster is a dedicated set of machines, then only the keytabs that are needed by Spark's work will be on those machines. If they wanted a cluster that uses Spark alongside other things, then YARN is probably a better solution for them already.

@vanzin
Copy link
Contributor

vanzin commented Feb 24, 2015

(3) I'm confused here - Spark can't read a keytab if the permissions on the keytab file deny access.

But in standalone mode all executors run as the "spark" user, whatever that is. If I understood your design, that means that "spark" would need to be able to read the keytabs for all users. Which means users can read each other's keytabs.

I understand this might not be a concern for the scenario where this feature is being proposed, but it might give people the wrong impression. Why have secure HDFS in that case to start with?

@mccheah
Copy link
Contributor Author

mccheah commented Feb 24, 2015

Spark only needs to read the specific keytab for the HDFS Namenode. It does not need to read any arbitrary keytab. I'm pushing a commit that makes this explicit in the Spark configuration as opposed to inferring the credentials from the Hadoop configuration.

@vanzin
Copy link
Contributor

vanzin commented Feb 24, 2015

Spark only needs to read the specific keytab for the HDFS Namenode.

That sounds even worse. Why would you run the Spark job with HDFS super user privileges?

@mccheah
Copy link
Contributor Author

mccheah commented Feb 24, 2015

I'm confused as to how this case is different from anywhere else we use Kerberos authentication to HDFS in Spark. If HDFS is configured with Kerberos authentication, naturally any client that writes and reads from HDFS needs to authenticate using Kerberos, using a particular keytab and a principal associated with that keytab. I used a proxy user to avoid actually running as the HDFS user though. See StandaloneSparkHadoopUtil.

@vanzin
Copy link
Contributor

vanzin commented Feb 24, 2015

I'm confused as to how this case is different from anywhere else we use Kerberos authentication to HDFS in Spark.

You're opening up the possibility that user code will find the keytab to the HDFS super user (since it needs read access to it, otherwise your executor can't log in).

The user can then login with that keytab and voila! Instant super user access. Or it can upload the keytab somewhere else...

@pwendell
Copy link
Contributor

The model I had in mind for this patch was to support dedicated clusters/appliances based on Spark where the Spark cluster itself is fully trusted and not multi-tenant. @harishreedharan - trying to have secured key distribution in the standalone mode is a very large undertaking, this is intentionally designed to avoid that to support a more limited security model. Trying to support arbitrary user code that you don't trust (within the cluster) is also not the intention - there aren't arbitrary users, it's a single embedded application. We've had several requests for some simpler security mechanism in Standalone mode over the years.

The main concerns I see here are that the KDC can't handle frequent authentication requests of the same principle from multiple hosts. That might render this approach untenable. It would be worth testing and understanding the limitations in that regard.

@mccheah
Copy link
Contributor Author

mccheah commented Feb 24, 2015

Sorry about the merge commit - I'm pretty sure I did it wrong, as I'm fairly certain I wasn't supposed to create the storm of commits listed above.

What's the right model for merging, for future reference?

@mccheah
Copy link
Contributor Author

mccheah commented Feb 24, 2015

Wow I really screwed this up, let me fix it. (Edit: Will probably end up me just cherry picking my commits on top of master and force-pushing... seems cleaner to start from scratch...)

@vanzin
Copy link
Contributor

vanzin commented Feb 24, 2015

The model I had in mind for this patch was to support dedicated clusters/appliances based on Spark where the Spark cluster itself is fully trusted and not multi-tenant.

But does trust also extends to the applications being run? Because according to @mccheah you'd be exposing important keytabs to any job running on that cluster. Keytabs that you may not want regular users to have access to.

@mccheah
Copy link
Contributor Author

mccheah commented Feb 24, 2015

I think this raises a core disconnect - Spark jobs themselves do not have an authentication mechanism in place. It would be more secure if we had a way to also authenticate Spark jobs themselves, and the model is that all of those authenticated Spark jobs also have authentication rights to HDFS. At this point however yes this trust does also extend to the applications being run.

@SparkQA
Copy link

SparkQA commented Feb 24, 2015

Test build #27913 has finished for PR 4106 at commit d18fbe7.

  • This patch passes all tests.
  • This patch does not merge cleanly.
  • This patch adds the following public classes (experimental):
    • abstract class SparkHadoopUtil extends Logging

@mccheah
Copy link
Contributor Author

mccheah commented Feb 25, 2015

One other model that would make sense is to require the driver to log in using the principal and the keytab specified in the configuration. If the driver is able to authenticate to Kerberos, then that assumes that the user submitting the Spark job already had access to the keytab independent of Spark anyways, and Spark is just one way they leverage their permissions to access HDFS. I already need the driver to log in at HadoopRDD.getPartitions and PairRDDFunctions.saveAsHadoopDataset.

@SparkQA
Copy link

SparkQA commented Feb 25, 2015

Test build #27915 has finished for PR 4106 at commit c23254d.

  • This patch passes all tests.
  • This patch does not merge cleanly.
  • This patch adds no public classes.

…s-secured Hadoop cluster

Previously, Kerberos secured Hadoop clusters could only be accessed by Spark running on top of YARN.
In other words, Spark standalone clusters had no way to read from secure Hadoop clusters. Other
solutions were proposed previously, but all of them attempted to perform authentication by obtaining
a token on a single node and passing that token around to all of the other Spark worker nodes. The
shipping of the token is risky, and all previous iterations fell short in leaving the token open
to man-in-the-middle attacks.

This patch introduces an alternative approach. It assumes that the keytab file has already been
distributed to every node in the cluster. When Spark starts in standalone mode, all of the workers
individually log in via Kerberos using specified configurations in the driver's SparkConf. In addition, on
basic Hadoop cluster setups the key tab file is often already manually deployed on all of the cluster's
nodes; it's not a huge stretch to expect the keytab files to be deployed to the Spark worker nodes as
well, if they are not already there.

This assumes that Spark will always authenticate with Kerberos using the same principal and keytab,
and the login is done at the very start of the job. Strictly speaking we should be trying to reduce the
surface area of the region of code that operates under a logged-in state. Or to put it another way,
the authentication should only be performed precisely when files are written or read from HDFS, and
after the read or write is performed the subject should be logged out. However this is difficult to
write and prone to errors, so this is left for a future refactor.
if (SparkConf.isExecutorStartupConf(key)) {
driverConf.setIfMissing(key, value)
} else {
driverConf.set(key, value)
}
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Please double check my merge here.

@mccheah
Copy link
Contributor Author

mccheah commented Feb 25, 2015

Okay, I started from scratch from master and cherry-picked my changes over - this PR was in a terrible state when I made an incorrect assumption when merging.

The discussion is still ongoing. Please look over the PR from top to bottom though, I did some major refactoring so it might as well be re-examined from scratch. Sorry about the first bad merge attempt =/

@@ -205,7 +208,7 @@ class SparkHadoopUtil extends Logging {

object SparkHadoopUtil {

private val hadoop = {
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Whoops, this doesn't have to be lazy

@SparkQA
Copy link

SparkQA commented Feb 25, 2015

Test build #27931 has finished for PR 4106 at commit 626318d.

  • This patch fails Spark unit tests.
  • This patch merges cleanly.
  • This patch adds the following public classes (experimental):
    • abstract class SparkHadoopUtil extends Logging

@vanzin
Copy link
Contributor

vanzin commented Feb 25, 2015

Hi @mccheah, just to clarify my comments, I'm thinking about someone looking at this feature and thinking that "hey, Spark Standalone now supports kerberos!", while that's not entirely true. There are many caveats to this approach. It might cover the needs of your use case, and I'm willing to give you the benefit of the doubt about that, even though I have to be frank and say I don't really understand it.

That being said, checking access to the keytab on the driver may not be enough for security purposes. If you do it explicitly when creating the SparkContext, it might work for client mode, although that also means you're trusting the client machine. But it doesn't work at all in cluster mode, where the driver would be run as a child process of a Worker. So there's extra configuration to not allow the Master to service requests from outside the trusted domain, for example.

As for the KDC denying logins, it might be possible to work around it by using user/host@REALM-style principals, instead of the same principal for every worker. You don't need to log in for every executor - just have the Worker manage the kerberos login, and all processes launched by it will automatically inherit the credentials. Having different principals for each Worker means the KDC won't freak out. I believe that HDFS ignores the "host" part when determining the actual user, but I might be wrong there - you'd have to check that out.

@mccheah
Copy link
Contributor Author

mccheah commented Feb 25, 2015

The security model I want to support is: if the client application wants to execute a job that reads and writes from HDFS that has been secured with kerberos, they should be allowed to do so if they have the keytab file at the client side. I also want to do this without token distribution. Is there no way to do this securely without token distribution?

@vanzin
Copy link
Contributor

vanzin commented Feb 25, 2015

if the client application wants to execute a job that reads and writes from HDFS that has been secured with kerberos, they should be allowed to do so if they have the keytab file at the client side.

But that's the part I don't understand: what keytab?

The reason people deploy kerberos is to make sure people properly log in with their own credentials before using services. Those credentials need not be materialized in the form of a keytab - you can use kinit to login using your password, for example.

But the approach here seems to be to have everybody share one set of credentials. Which to me doesn't really make a lot of sense when talking about kerberos.

@mccheah
Copy link
Contributor Author

mccheah commented Feb 25, 2015

Come to think of it, with my current approach, since the keytab is specified in the driver's SparkConf, theoretically different Spark applications can specify different keytabs and principals for the login. So this doesn't necessarily imply that every Spark application has to use the same credentials. But there is the problem which I didn't foresee regarding the Spark user needing to be able to access the credentials.

That being said my use case was that the cluster is a dedicated resource for a select few Spark applications, and the Spark applications serve end users, and the end users themselves do not have access to the credentials.

Part of the reason why this is confusing to me is that when I looked at secure HDFS configurations, there is only one keytab file that can be specified for the namenode, which implied to me that everyone who accesses HDFS must use that keytab file. Perhaps I can be enlightened as to how one accesses HDFS WITHOUT using that specific keytab file?

@vanzin
Copy link
Contributor

vanzin commented Feb 25, 2015

Come to think of it, with my current approach, since the keytab is specified in the driver's SparkConf, theoretically different Spark applications can specify different keytabs and principals for the login.

But that's the first security issue I mentioned. In Standalone mode, all executors will be running as the same local user (the user running the Worker process), so executors would need read access to all those keytabs. So every user can potentially see every other user's keytab.

Part of the reason why this is confusing to me is that when I looked at secure HDFS configurations, there is only one keytab file that can be specified for the namenode

The keytab for the NN and the keytab for users running applications are completely unrelated. The NN needs a keytab so that it can authenticate itself to DNs and vice-versa, so that they know who they're talking to. But users talking to HDFS identify themselves as, well, themselves, and use their own credentials.

@mccheah
Copy link
Contributor Author

mccheah commented Feb 25, 2015

Your views make sense! Thanks a lot =) the discussion was helpful and clarified the pitfalls here. I have learned a lot.

I'm going to defer to @pwendell for the decision to commit to this or not. The way I see it, I think this is still acceptable in the limited use case that Spark applications are trusted for accessing HDFS and the applications can serve end users that don't have access to the HDFS credentials. That being said I started down this path a few months ago, and priorities have shifted since then: specifically users I'm working with want to move to YARN which would make this feature unnecessary for them.

So it's more now a matter of if the community would still need this or not, and our assessment of the risk that users don't understand the security implications.

@ash211
Copy link
Contributor

ash211 commented Apr 27, 2015

@mccheah is this work still active? If not should maybe close this PR

@mccheah mccheah closed this Apr 27, 2015
@mccheah
Copy link
Contributor Author

mccheah commented Apr 27, 2015

No longer in progress. Closing.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
7 participants