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-24518][CORE] Using Hadoop credential provider API to store password #21548

Closed
wants to merge 3 commits into from

Conversation

jerryshao
Copy link
Contributor

@jerryshao jerryshao commented Jun 13, 2018

What changes were proposed in this pull request?

In our distribution, because we don't do such fine-grained access control of config file, also configuration file is world readable shared between different components, so password may leak to different users.

Hadoop credential provider API support storing password in a secure way, in which Spark could read it in a secure way, so here propose to add support of using credential provider API to get password.

How was this patch tested?

Adding tests and verified locally.

Change-Id: Ie774eeb9376f8b5d7379f1976826e12e9c529be3
@jerryshao jerryshao changed the title [SPARK-24518] Using Hadoop credential provider API to store password [SPARK-24518][CORE] Using Hadoop credential provider API to store password Jun 13, 2018
@SparkQA
Copy link

SparkQA commented Jun 13, 2018

Test build #91747 has finished for PR 21548 at commit 575152b.

  • This patch fails due to an unknown error code, -9.
  • This patch merges cleanly.
  • This patch adds no public classes.

@jerryshao
Copy link
Contributor Author

Jenkins, retest this please.

@SparkQA
Copy link

SparkQA commented Jun 13, 2018

Test build #91757 has finished for PR 21548 at commit 575152b.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@jerryshao
Copy link
Contributor Author

CC @vanzin @tgravescs would you please help to review, thanks!

@tgravescs
Copy link
Contributor

we would definitely want to update the docs on how user would do this. I don't see a test that actually tests reading from the hadoopConf either, so we should add one.

I need to look at the hadoop api in more depth to do a full review

@jerryshao
Copy link
Contributor Author

jerryshao commented Jun 19, 2018

Thanks @tgravescs I will add the docs about how to use it. I was thinking to add a test case, but it looks like may not be easy to add one, I will try to add one.

Change-Id: I38146ee45a4565295fa6fd297f591c368d6b250a
@SparkQA
Copy link

SparkQA commented Jun 20, 2018

Test build #92120 has finished for PR 21548 at commit c7ef15e.

  • This patch fails due to an unknown error code, -9.
  • This patch merges cleanly.
  • This patch adds no public classes.

@jerryshao
Copy link
Contributor Author

Jenkins, retest this please.

@SparkQA
Copy link

SparkQA commented Jun 20, 2018

Test build #92129 has finished for PR 21548 at commit c7ef15e.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

Copy link
Contributor

@vanzin vanzin left a comment

Choose a reason for hiding this comment

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

Looks OK. I'm not a huge fan of the hadoop conf in the SecurityManager, but that's the easiest way to achieve this...

I just have an issue with your summary. It makes it sound like there's no way to securely configure these things today, and that's not true.

You can have the password in the config file and have it only readable by authorized users, which is basically what the credential provider does.

Or you can have the password in an env variable, and reference the env variable in the Spark config.

This just adds a third way.

@@ -179,9 +185,11 @@ private[spark] object SSLOptions extends Logging {
.orElse(defaults.flatMap(_.keyStore))

val keyStorePassword = conf.getWithSubstitution(s"$ns.keyStorePassword")
.orElse(Option(hadoopConf.getPassword(s"$ns.keyStorePassword")).map(new String(_)))
Copy link
Contributor

Choose a reason for hiding this comment

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

Needs charset (also in others).

Copy link
Contributor Author

Choose a reason for hiding this comment

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

@vanzin the return value of hadoopConf#getPassword is char array, so there's no way to specify the charset here.

Copy link
Contributor

Choose a reason for hiding this comment

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

new String takes a charset. (In fact the constructor you're calling should be deprecated...)

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Hi @vanzin , I checked jdk8 doc again, I don't find a String constructor which takes both char array and charset as parameters.

Copy link
Contributor

Choose a reason for hiding this comment

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

Oh, my bad, that's a char array, not a byte array. All is good then.

docs/security.md Outdated
-provider jceks://[email protected]:9001/user/backup/ssl.jceks
```

In the meantime, adding configuration "hadoop.security.credential.provider.path=jceks://[email protected]:9001/user/backup/ssl.jceks"
Copy link
Contributor

Choose a reason for hiding this comment

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

Rephrase:

"To configure the location of the credential provider, set the hadoop.security.credential.provider.path config option in the Hadoop configuration used by Spark."

Your example also kinda looks like a Spark config (which would be "spark.hadoop.blah"), since Hadoop configs are generally in XML.

@jerryshao
Copy link
Contributor Author

I just have an issue with your summary. It makes it sound like there's no way to securely configure these things today, and that's not true.

You can have the password in the config file and have it only readable by authorized users, which is basically what the credential provider does.

Or you can have the password in an env variable, and reference the env variable in the Spark config.

I see, thanks for explanation @vanzin . This might be the problem in our distribution, because we don't do such fine-grained access control of config file, also configuration file is world readable shared between different components, that's why we're seeking to use Hadoop credential provide to secure this thing.

@SparkQA
Copy link

SparkQA commented Jun 21, 2018

Test build #92164 has finished for PR 21548 at commit 1ab28d0.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@vanzin
Copy link
Contributor

vanzin commented Jun 21, 2018

Could you update the summary so that it doesn't sound like this is an existing security issue?

@vanzin
Copy link
Contributor

vanzin commented Jun 22, 2018

Merging to master.

@asfgit asfgit closed this in 33e77fa Jun 22, 2018
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

4 participants