Repository: spark
Updated Branches:
  refs/heads/master 39dfaf2fd -> 33e77fa89


[SPARK-24518][CORE] Using Hadoop credential provider API to store password

## 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.

Author: jerryshao <ss...@hortonworks.com>

Closes #21548 from jerryshao/SPARK-24518.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/33e77fa8
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/33e77fa8
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/33e77fa8

Branch: refs/heads/master
Commit: 33e77fa89b5805ecb1066fc534723527f70d37c7
Parents: 39dfaf2
Author: jerryshao <ss...@hortonworks.com>
Authored: Fri Jun 22 10:14:12 2018 -0700
Committer: Marcelo Vanzin <van...@cloudera.com>
Committed: Fri Jun 22 10:14:12 2018 -0700

----------------------------------------------------------------------
 .../scala/org/apache/spark/SSLOptions.scala     | 11 ++-
 .../org/apache/spark/SecurityManager.scala      |  9 ++-
 .../org/apache/spark/SSLOptionsSuite.scala      | 75 ++++++++++++++++++--
 docs/security.md                                | 23 +++++-
 4 files changed, 107 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/spark/blob/33e77fa8/core/src/main/scala/org/apache/spark/SSLOptions.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/SSLOptions.scala 
b/core/src/main/scala/org/apache/spark/SSLOptions.scala
index 04c38f1..1632e0c 100644
--- a/core/src/main/scala/org/apache/spark/SSLOptions.scala
+++ b/core/src/main/scala/org/apache/spark/SSLOptions.scala
@@ -21,6 +21,7 @@ import java.io.File
 import java.security.NoSuchAlgorithmException
 import javax.net.ssl.SSLContext
 
+import org.apache.hadoop.conf.Configuration
 import org.eclipse.jetty.util.ssl.SslContextFactory
 
 import org.apache.spark.internal.Logging
@@ -163,11 +164,16 @@ private[spark] object SSLOptions extends Logging {
    * missing in SparkConf, the corresponding setting is used from the default 
configuration.
    *
    * @param conf Spark configuration object where the settings are collected 
from
+   * @param hadoopConf Hadoop configuration to get settings
    * @param ns the namespace name
    * @param defaults the default configuration
    * @return [[org.apache.spark.SSLOptions]] object
    */
-  def parse(conf: SparkConf, ns: String, defaults: Option[SSLOptions] = None): 
SSLOptions = {
+  def parse(
+      conf: SparkConf,
+      hadoopConf: Configuration,
+      ns: String,
+      defaults: Option[SSLOptions] = None): SSLOptions = {
     val enabled = conf.getBoolean(s"$ns.enabled", defaultValue = 
defaults.exists(_.enabled))
 
     val port = conf.getWithSubstitution(s"$ns.port").map(_.toInt)
@@ -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(_)))
         .orElse(defaults.flatMap(_.keyStorePassword))
 
     val keyPassword = conf.getWithSubstitution(s"$ns.keyPassword")
+        .orElse(Option(hadoopConf.getPassword(s"$ns.keyPassword")).map(new 
String(_)))
         .orElse(defaults.flatMap(_.keyPassword))
 
     val keyStoreType = conf.getWithSubstitution(s"$ns.keyStoreType")
@@ -194,6 +202,7 @@ private[spark] object SSLOptions extends Logging {
         .orElse(defaults.flatMap(_.trustStore))
 
     val trustStorePassword = 
conf.getWithSubstitution(s"$ns.trustStorePassword")
+        
.orElse(Option(hadoopConf.getPassword(s"$ns.trustStorePassword")).map(new 
String(_)))
         .orElse(defaults.flatMap(_.trustStorePassword))
 
     val trustStoreType = conf.getWithSubstitution(s"$ns.trustStoreType")

http://git-wip-us.apache.org/repos/asf/spark/blob/33e77fa8/core/src/main/scala/org/apache/spark/SecurityManager.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/SecurityManager.scala 
b/core/src/main/scala/org/apache/spark/SecurityManager.scala
index b874763..3cfafeb 100644
--- a/core/src/main/scala/org/apache/spark/SecurityManager.scala
+++ b/core/src/main/scala/org/apache/spark/SecurityManager.scala
@@ -19,11 +19,11 @@ package org.apache.spark
 
 import java.net.{Authenticator, PasswordAuthentication}
 import java.nio.charset.StandardCharsets.UTF_8
-import javax.net.ssl._
 
 import org.apache.hadoop.io.Text
 import org.apache.hadoop.security.{Credentials, UserGroupInformation}
 
+import org.apache.spark.deploy.SparkHadoopUtil
 import org.apache.spark.internal.Logging
 import org.apache.spark.internal.config._
 import org.apache.spark.launcher.SparkLauncher
@@ -111,11 +111,14 @@ private[spark] class SecurityManager(
     )
   }
 
+  private val hadoopConf = SparkHadoopUtil.get.newConfiguration(sparkConf)
   // the default SSL configuration - it will be used by all communication 
layers unless overwritten
-  private val defaultSSLOptions = SSLOptions.parse(sparkConf, "spark.ssl", 
defaults = None)
+  private val defaultSSLOptions =
+    SSLOptions.parse(sparkConf, hadoopConf, "spark.ssl", defaults = None)
 
   def getSSLOptions(module: String): SSLOptions = {
-    val opts = SSLOptions.parse(sparkConf, s"spark.ssl.$module", 
Some(defaultSSLOptions))
+    val opts =
+      SSLOptions.parse(sparkConf, hadoopConf, s"spark.ssl.$module", 
Some(defaultSSLOptions))
     logDebug(s"Created SSL options for $module: $opts")
     opts
   }

http://git-wip-us.apache.org/repos/asf/spark/blob/33e77fa8/core/src/test/scala/org/apache/spark/SSLOptionsSuite.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/org/apache/spark/SSLOptionsSuite.scala 
b/core/src/test/scala/org/apache/spark/SSLOptionsSuite.scala
index 8eabc2b..5dbfc5c 100644
--- a/core/src/test/scala/org/apache/spark/SSLOptionsSuite.scala
+++ b/core/src/test/scala/org/apache/spark/SSLOptionsSuite.scala
@@ -18,8 +18,11 @@
 package org.apache.spark
 
 import java.io.File
+import java.util.UUID
 import javax.net.ssl.SSLContext
 
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.security.alias.{CredentialProvider, 
CredentialProviderFactory}
 import org.scalatest.BeforeAndAfterAll
 
 import org.apache.spark.util.SparkConfWithEnv
@@ -40,6 +43,7 @@ class SSLOptionsSuite extends SparkFunSuite with 
BeforeAndAfterAll {
       .toSet
 
     val conf = new SparkConf
+    val hadoopConf = new Configuration()
     conf.set("spark.ssl.enabled", "true")
     conf.set("spark.ssl.keyStore", keyStorePath)
     conf.set("spark.ssl.keyStorePassword", "password")
@@ -49,7 +53,7 @@ class SSLOptionsSuite extends SparkFunSuite with 
BeforeAndAfterAll {
     conf.set("spark.ssl.enabledAlgorithms", algorithms.mkString(","))
     conf.set("spark.ssl.protocol", "TLSv1.2")
 
-    val opts = SSLOptions.parse(conf, "spark.ssl")
+    val opts = SSLOptions.parse(conf, hadoopConf, "spark.ssl")
 
     assert(opts.enabled === true)
     assert(opts.trustStore.isDefined === true)
@@ -70,6 +74,7 @@ class SSLOptionsSuite extends SparkFunSuite with 
BeforeAndAfterAll {
     val trustStorePath = new 
File(this.getClass.getResource("/truststore").toURI).getAbsolutePath
 
     val conf = new SparkConf
+    val hadoopConf = new Configuration()
     conf.set("spark.ssl.enabled", "true")
     conf.set("spark.ssl.keyStore", keyStorePath)
     conf.set("spark.ssl.keyStorePassword", "password")
@@ -80,8 +85,8 @@ class SSLOptionsSuite extends SparkFunSuite with 
BeforeAndAfterAll {
       "TLS_RSA_WITH_AES_128_CBC_SHA, TLS_RSA_WITH_AES_256_CBC_SHA")
     conf.set("spark.ssl.protocol", "SSLv3")
 
-    val defaultOpts = SSLOptions.parse(conf, "spark.ssl", defaults = None)
-    val opts = SSLOptions.parse(conf, "spark.ssl.ui", defaults = 
Some(defaultOpts))
+    val defaultOpts = SSLOptions.parse(conf, hadoopConf, "spark.ssl", defaults 
= None)
+    val opts = SSLOptions.parse(conf, hadoopConf, "spark.ssl.ui", defaults = 
Some(defaultOpts))
 
     assert(opts.enabled === true)
     assert(opts.trustStore.isDefined === true)
@@ -103,6 +108,7 @@ class SSLOptionsSuite extends SparkFunSuite with 
BeforeAndAfterAll {
     val trustStorePath = new 
File(this.getClass.getResource("/truststore").toURI).getAbsolutePath
 
     val conf = new SparkConf
+    val hadoopConf = new Configuration()
     conf.set("spark.ssl.enabled", "true")
     conf.set("spark.ssl.ui.enabled", "false")
     conf.set("spark.ssl.ui.port", "4242")
@@ -117,8 +123,8 @@ class SSLOptionsSuite extends SparkFunSuite with 
BeforeAndAfterAll {
     conf.set("spark.ssl.ui.enabledAlgorithms", "ABC, DEF")
     conf.set("spark.ssl.protocol", "SSLv3")
 
-    val defaultOpts = SSLOptions.parse(conf, "spark.ssl", defaults = None)
-    val opts = SSLOptions.parse(conf, "spark.ssl.ui", defaults = 
Some(defaultOpts))
+    val defaultOpts = SSLOptions.parse(conf, hadoopConf, "spark.ssl", defaults 
= None)
+    val opts = SSLOptions.parse(conf, hadoopConf, "spark.ssl.ui", defaults = 
Some(defaultOpts))
 
     assert(opts.enabled === false)
     assert(opts.port === Some(4242))
@@ -139,14 +145,71 @@ class SSLOptionsSuite extends SparkFunSuite with 
BeforeAndAfterAll {
     val conf = new SparkConfWithEnv(Map(
       "ENV1" -> "val1",
       "ENV2" -> "val2"))
+    val hadoopConf = new Configuration()
 
     conf.set("spark.ssl.enabled", "true")
     conf.set("spark.ssl.keyStore", "${env:ENV1}")
     conf.set("spark.ssl.trustStore", "${env:ENV2}")
 
-    val opts = SSLOptions.parse(conf, "spark.ssl", defaults = None)
+    val opts = SSLOptions.parse(conf, hadoopConf, "spark.ssl", defaults = None)
     assert(opts.keyStore === Some(new File("val1")))
     assert(opts.trustStore === Some(new File("val2")))
   }
 
+  test("get password from Hadoop credential provider") {
+    val keyStorePath = new 
File(this.getClass.getResource("/keystore").toURI).getAbsolutePath
+    val trustStorePath = new 
File(this.getClass.getResource("/truststore").toURI).getAbsolutePath
+
+    val conf = new SparkConf
+    val hadoopConf = new Configuration()
+    val tmpPath = s"localjceks://file${sys.props("java.io.tmpdir")}/test-" +
+      s"${UUID.randomUUID().toString}.jceks"
+    val provider = createCredentialProvider(tmpPath, hadoopConf)
+
+    conf.set("spark.ssl.enabled", "true")
+    conf.set("spark.ssl.keyStore", keyStorePath)
+    storePassword(provider, "spark.ssl.keyStorePassword", "password")
+    storePassword(provider, "spark.ssl.keyPassword", "password")
+    conf.set("spark.ssl.trustStore", trustStorePath)
+    storePassword(provider, "spark.ssl.trustStorePassword", "password")
+    conf.set("spark.ssl.enabledAlgorithms",
+      "TLS_RSA_WITH_AES_128_CBC_SHA, TLS_RSA_WITH_AES_256_CBC_SHA")
+    conf.set("spark.ssl.protocol", "SSLv3")
+
+    val defaultOpts = SSLOptions.parse(conf, hadoopConf, "spark.ssl", defaults 
= None)
+    val opts = SSLOptions.parse(conf, hadoopConf, "spark.ssl.ui", defaults = 
Some(defaultOpts))
+
+    assert(opts.enabled === true)
+    assert(opts.trustStore.isDefined === true)
+    assert(opts.trustStore.get.getName === "truststore")
+    assert(opts.trustStore.get.getAbsolutePath === trustStorePath)
+    assert(opts.keyStore.isDefined === true)
+    assert(opts.keyStore.get.getName === "keystore")
+    assert(opts.keyStore.get.getAbsolutePath === keyStorePath)
+    assert(opts.trustStorePassword === Some("password"))
+    assert(opts.keyStorePassword === Some("password"))
+    assert(opts.keyPassword === Some("password"))
+    assert(opts.protocol === Some("SSLv3"))
+    assert(opts.enabledAlgorithms ===
+      Set("TLS_RSA_WITH_AES_128_CBC_SHA", "TLS_RSA_WITH_AES_256_CBC_SHA"))
+  }
+
+  private def createCredentialProvider(tmpPath: String, conf: Configuration): 
CredentialProvider = {
+    conf.set(CredentialProviderFactory.CREDENTIAL_PROVIDER_PATH, tmpPath)
+
+    val provider = CredentialProviderFactory.getProviders(conf).get(0)
+    if (provider == null) {
+      throw new IllegalStateException(s"Fail to get credential provider with 
path $tmpPath")
+    }
+
+    provider
+  }
+
+  private def storePassword(
+      provider: CredentialProvider,
+      passwordKey: String,
+      password: String): Unit = {
+    provider.createCredentialEntry(passwordKey, password.toCharArray)
+    provider.flush()
+  }
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/33e77fa8/docs/security.md
----------------------------------------------------------------------
diff --git a/docs/security.md b/docs/security.md
index 8c0c66f..6ef3a80 100644
--- a/docs/security.md
+++ b/docs/security.md
@@ -177,7 +177,7 @@ ACLs can be configured for either users or groups. 
Configuration entries accept
 lists as input, meaning multiple users or groups can be given the desired 
privileges. This can be
 used if you run on a shared cluster and have a set of administrators or 
developers who need to
 monitor applications they may not have started themselves. A wildcard (`*`) 
added to specific ACL
-means that all users will have the respective pivilege. By default, only the 
user submitting the
+means that all users will have the respective privilege. By default, only the 
user submitting the
 application is added to the ACLs.
 
 Group membership is established by using a configurable group mapping 
provider. The mapper is
@@ -446,6 +446,27 @@ replaced with one of the above namespaces.
   </tr>
 </table>
 
+Spark also supports retrieving `${ns}.keyPassword`, `${ns}.keyStorePassword` 
and `${ns}.trustStorePassword` from
+[Hadoop Credential 
Providers](https://hadoop.apache.org/docs/current/hadoop-project-dist/hadoop-common/CredentialProviderAPI.html).
+User could store password into credential file and make it accessible by 
different components, like:
+
+```
+hadoop credential create spark.ssl.keyPassword -value password \
+    -provider jceks://h...@nn1.example.com:9001/user/backup/ssl.jceks
+```
+
+To configure the location of the credential provider, set the 
`hadoop.security.credential.provider.path`
+config option in the Hadoop configuration used by Spark, like:
+
+```
+  <property>
+    <name>hadoop.security.credential.provider.path</name>
+    <value>jceks://h...@nn1.example.com:9001/user/backup/ssl.jceks</value>
+  </property>
+```
+
+Or via SparkConf 
"spark.hadoop.hadoop.security.credential.provider.path=jceks://h...@nn1.example.com:9001/user/backup/ssl.jceks".
+
 ## Preparing the key stores
 
 Key stores can be generated by `keytool` program. The reference documentation 
for this tool for


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

Reply via email to