Skip to content

Commit 33e77fa

Browse files
jerryshaoMarcelo Vanzin
authored andcommitted
[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 <[email protected]> Closes apache#21548 from jerryshao/SPARK-24518.
1 parent 39dfaf2 commit 33e77fa

File tree

4 files changed

+107
-11
lines changed

4 files changed

+107
-11
lines changed

core/src/main/scala/org/apache/spark/SSLOptions.scala

Lines changed: 10 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -21,6 +21,7 @@ import java.io.File
2121
import java.security.NoSuchAlgorithmException
2222
import javax.net.ssl.SSLContext
2323

24+
import org.apache.hadoop.conf.Configuration
2425
import org.eclipse.jetty.util.ssl.SslContextFactory
2526

2627
import org.apache.spark.internal.Logging
@@ -163,11 +164,16 @@ private[spark] object SSLOptions extends Logging {
163164
* missing in SparkConf, the corresponding setting is used from the default configuration.
164165
*
165166
* @param conf Spark configuration object where the settings are collected from
167+
* @param hadoopConf Hadoop configuration to get settings
166168
* @param ns the namespace name
167169
* @param defaults the default configuration
168170
* @return [[org.apache.spark.SSLOptions]] object
169171
*/
170-
def parse(conf: SparkConf, ns: String, defaults: Option[SSLOptions] = None): SSLOptions = {
172+
def parse(
173+
conf: SparkConf,
174+
hadoopConf: Configuration,
175+
ns: String,
176+
defaults: Option[SSLOptions] = None): SSLOptions = {
171177
val enabled = conf.getBoolean(s"$ns.enabled", defaultValue = defaults.exists(_.enabled))
172178

173179
val port = conf.getWithSubstitution(s"$ns.port").map(_.toInt)
@@ -179,9 +185,11 @@ private[spark] object SSLOptions extends Logging {
179185
.orElse(defaults.flatMap(_.keyStore))
180186

181187
val keyStorePassword = conf.getWithSubstitution(s"$ns.keyStorePassword")
188+
.orElse(Option(hadoopConf.getPassword(s"$ns.keyStorePassword")).map(new String(_)))
182189
.orElse(defaults.flatMap(_.keyStorePassword))
183190

184191
val keyPassword = conf.getWithSubstitution(s"$ns.keyPassword")
192+
.orElse(Option(hadoopConf.getPassword(s"$ns.keyPassword")).map(new String(_)))
185193
.orElse(defaults.flatMap(_.keyPassword))
186194

187195
val keyStoreType = conf.getWithSubstitution(s"$ns.keyStoreType")
@@ -194,6 +202,7 @@ private[spark] object SSLOptions extends Logging {
194202
.orElse(defaults.flatMap(_.trustStore))
195203

196204
val trustStorePassword = conf.getWithSubstitution(s"$ns.trustStorePassword")
205+
.orElse(Option(hadoopConf.getPassword(s"$ns.trustStorePassword")).map(new String(_)))
197206
.orElse(defaults.flatMap(_.trustStorePassword))
198207

199208
val trustStoreType = conf.getWithSubstitution(s"$ns.trustStoreType")

core/src/main/scala/org/apache/spark/SecurityManager.scala

Lines changed: 6 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -19,11 +19,11 @@ package org.apache.spark
1919

2020
import java.net.{Authenticator, PasswordAuthentication}
2121
import java.nio.charset.StandardCharsets.UTF_8
22-
import javax.net.ssl._
2322

2423
import org.apache.hadoop.io.Text
2524
import org.apache.hadoop.security.{Credentials, UserGroupInformation}
2625

26+
import org.apache.spark.deploy.SparkHadoopUtil
2727
import org.apache.spark.internal.Logging
2828
import org.apache.spark.internal.config._
2929
import org.apache.spark.launcher.SparkLauncher
@@ -111,11 +111,14 @@ private[spark] class SecurityManager(
111111
)
112112
}
113113

114+
private val hadoopConf = SparkHadoopUtil.get.newConfiguration(sparkConf)
114115
// the default SSL configuration - it will be used by all communication layers unless overwritten
115-
private val defaultSSLOptions = SSLOptions.parse(sparkConf, "spark.ssl", defaults = None)
116+
private val defaultSSLOptions =
117+
SSLOptions.parse(sparkConf, hadoopConf, "spark.ssl", defaults = None)
116118

117119
def getSSLOptions(module: String): SSLOptions = {
118-
val opts = SSLOptions.parse(sparkConf, s"spark.ssl.$module", Some(defaultSSLOptions))
120+
val opts =
121+
SSLOptions.parse(sparkConf, hadoopConf, s"spark.ssl.$module", Some(defaultSSLOptions))
119122
logDebug(s"Created SSL options for $module: $opts")
120123
opts
121124
}

core/src/test/scala/org/apache/spark/SSLOptionsSuite.scala

Lines changed: 69 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -18,8 +18,11 @@
1818
package org.apache.spark
1919

2020
import java.io.File
21+
import java.util.UUID
2122
import javax.net.ssl.SSLContext
2223

24+
import org.apache.hadoop.conf.Configuration
25+
import org.apache.hadoop.security.alias.{CredentialProvider, CredentialProviderFactory}
2326
import org.scalatest.BeforeAndAfterAll
2427

2528
import org.apache.spark.util.SparkConfWithEnv
@@ -40,6 +43,7 @@ class SSLOptionsSuite extends SparkFunSuite with BeforeAndAfterAll {
4043
.toSet
4144

4245
val conf = new SparkConf
46+
val hadoopConf = new Configuration()
4347
conf.set("spark.ssl.enabled", "true")
4448
conf.set("spark.ssl.keyStore", keyStorePath)
4549
conf.set("spark.ssl.keyStorePassword", "password")
@@ -49,7 +53,7 @@ class SSLOptionsSuite extends SparkFunSuite with BeforeAndAfterAll {
4953
conf.set("spark.ssl.enabledAlgorithms", algorithms.mkString(","))
5054
conf.set("spark.ssl.protocol", "TLSv1.2")
5155

52-
val opts = SSLOptions.parse(conf, "spark.ssl")
56+
val opts = SSLOptions.parse(conf, hadoopConf, "spark.ssl")
5357

5458
assert(opts.enabled === true)
5559
assert(opts.trustStore.isDefined === true)
@@ -70,6 +74,7 @@ class SSLOptionsSuite extends SparkFunSuite with BeforeAndAfterAll {
7074
val trustStorePath = new File(this.getClass.getResource("/truststore").toURI).getAbsolutePath
7175

7276
val conf = new SparkConf
77+
val hadoopConf = new Configuration()
7378
conf.set("spark.ssl.enabled", "true")
7479
conf.set("spark.ssl.keyStore", keyStorePath)
7580
conf.set("spark.ssl.keyStorePassword", "password")
@@ -80,8 +85,8 @@ class SSLOptionsSuite extends SparkFunSuite with BeforeAndAfterAll {
8085
"TLS_RSA_WITH_AES_128_CBC_SHA, TLS_RSA_WITH_AES_256_CBC_SHA")
8186
conf.set("spark.ssl.protocol", "SSLv3")
8287

83-
val defaultOpts = SSLOptions.parse(conf, "spark.ssl", defaults = None)
84-
val opts = SSLOptions.parse(conf, "spark.ssl.ui", defaults = Some(defaultOpts))
88+
val defaultOpts = SSLOptions.parse(conf, hadoopConf, "spark.ssl", defaults = None)
89+
val opts = SSLOptions.parse(conf, hadoopConf, "spark.ssl.ui", defaults = Some(defaultOpts))
8590

8691
assert(opts.enabled === true)
8792
assert(opts.trustStore.isDefined === true)
@@ -103,6 +108,7 @@ class SSLOptionsSuite extends SparkFunSuite with BeforeAndAfterAll {
103108
val trustStorePath = new File(this.getClass.getResource("/truststore").toURI).getAbsolutePath
104109

105110
val conf = new SparkConf
111+
val hadoopConf = new Configuration()
106112
conf.set("spark.ssl.enabled", "true")
107113
conf.set("spark.ssl.ui.enabled", "false")
108114
conf.set("spark.ssl.ui.port", "4242")
@@ -117,8 +123,8 @@ class SSLOptionsSuite extends SparkFunSuite with BeforeAndAfterAll {
117123
conf.set("spark.ssl.ui.enabledAlgorithms", "ABC, DEF")
118124
conf.set("spark.ssl.protocol", "SSLv3")
119125

120-
val defaultOpts = SSLOptions.parse(conf, "spark.ssl", defaults = None)
121-
val opts = SSLOptions.parse(conf, "spark.ssl.ui", defaults = Some(defaultOpts))
126+
val defaultOpts = SSLOptions.parse(conf, hadoopConf, "spark.ssl", defaults = None)
127+
val opts = SSLOptions.parse(conf, hadoopConf, "spark.ssl.ui", defaults = Some(defaultOpts))
122128

123129
assert(opts.enabled === false)
124130
assert(opts.port === Some(4242))
@@ -139,14 +145,71 @@ class SSLOptionsSuite extends SparkFunSuite with BeforeAndAfterAll {
139145
val conf = new SparkConfWithEnv(Map(
140146
"ENV1" -> "val1",
141147
"ENV2" -> "val2"))
148+
val hadoopConf = new Configuration()
142149

143150
conf.set("spark.ssl.enabled", "true")
144151
conf.set("spark.ssl.keyStore", "${env:ENV1}")
145152
conf.set("spark.ssl.trustStore", "${env:ENV2}")
146153

147-
val opts = SSLOptions.parse(conf, "spark.ssl", defaults = None)
154+
val opts = SSLOptions.parse(conf, hadoopConf, "spark.ssl", defaults = None)
148155
assert(opts.keyStore === Some(new File("val1")))
149156
assert(opts.trustStore === Some(new File("val2")))
150157
}
151158

159+
test("get password from Hadoop credential provider") {
160+
val keyStorePath = new File(this.getClass.getResource("/keystore").toURI).getAbsolutePath
161+
val trustStorePath = new File(this.getClass.getResource("/truststore").toURI).getAbsolutePath
162+
163+
val conf = new SparkConf
164+
val hadoopConf = new Configuration()
165+
val tmpPath = s"localjceks://file${sys.props("java.io.tmpdir")}/test-" +
166+
s"${UUID.randomUUID().toString}.jceks"
167+
val provider = createCredentialProvider(tmpPath, hadoopConf)
168+
169+
conf.set("spark.ssl.enabled", "true")
170+
conf.set("spark.ssl.keyStore", keyStorePath)
171+
storePassword(provider, "spark.ssl.keyStorePassword", "password")
172+
storePassword(provider, "spark.ssl.keyPassword", "password")
173+
conf.set("spark.ssl.trustStore", trustStorePath)
174+
storePassword(provider, "spark.ssl.trustStorePassword", "password")
175+
conf.set("spark.ssl.enabledAlgorithms",
176+
"TLS_RSA_WITH_AES_128_CBC_SHA, TLS_RSA_WITH_AES_256_CBC_SHA")
177+
conf.set("spark.ssl.protocol", "SSLv3")
178+
179+
val defaultOpts = SSLOptions.parse(conf, hadoopConf, "spark.ssl", defaults = None)
180+
val opts = SSLOptions.parse(conf, hadoopConf, "spark.ssl.ui", defaults = Some(defaultOpts))
181+
182+
assert(opts.enabled === true)
183+
assert(opts.trustStore.isDefined === true)
184+
assert(opts.trustStore.get.getName === "truststore")
185+
assert(opts.trustStore.get.getAbsolutePath === trustStorePath)
186+
assert(opts.keyStore.isDefined === true)
187+
assert(opts.keyStore.get.getName === "keystore")
188+
assert(opts.keyStore.get.getAbsolutePath === keyStorePath)
189+
assert(opts.trustStorePassword === Some("password"))
190+
assert(opts.keyStorePassword === Some("password"))
191+
assert(opts.keyPassword === Some("password"))
192+
assert(opts.protocol === Some("SSLv3"))
193+
assert(opts.enabledAlgorithms ===
194+
Set("TLS_RSA_WITH_AES_128_CBC_SHA", "TLS_RSA_WITH_AES_256_CBC_SHA"))
195+
}
196+
197+
private def createCredentialProvider(tmpPath: String, conf: Configuration): CredentialProvider = {
198+
conf.set(CredentialProviderFactory.CREDENTIAL_PROVIDER_PATH, tmpPath)
199+
200+
val provider = CredentialProviderFactory.getProviders(conf).get(0)
201+
if (provider == null) {
202+
throw new IllegalStateException(s"Fail to get credential provider with path $tmpPath")
203+
}
204+
205+
provider
206+
}
207+
208+
private def storePassword(
209+
provider: CredentialProvider,
210+
passwordKey: String,
211+
password: String): Unit = {
212+
provider.createCredentialEntry(passwordKey, password.toCharArray)
213+
provider.flush()
214+
}
152215
}

docs/security.md

Lines changed: 22 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -177,7 +177,7 @@ ACLs can be configured for either users or groups. Configuration entries accept
177177
lists as input, meaning multiple users or groups can be given the desired privileges. This can be
178178
used if you run on a shared cluster and have a set of administrators or developers who need to
179179
monitor applications they may not have started themselves. A wildcard (`*`) added to specific ACL
180-
means that all users will have the respective pivilege. By default, only the user submitting the
180+
means that all users will have the respective privilege. By default, only the user submitting the
181181
application is added to the ACLs.
182182

183183
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.
446446
</tr>
447447
</table>
448448

449+
Spark also supports retrieving `${ns}.keyPassword`, `${ns}.keyStorePassword` and `${ns}.trustStorePassword` from
450+
[Hadoop Credential Providers](https://hadoop.apache.org/docs/current/hadoop-project-dist/hadoop-common/CredentialProviderAPI.html).
451+
User could store password into credential file and make it accessible by different components, like:
452+
453+
```
454+
hadoop credential create spark.ssl.keyPassword -value password \
455+
-provider jceks://[email protected]:9001/user/backup/ssl.jceks
456+
```
457+
458+
To configure the location of the credential provider, set the `hadoop.security.credential.provider.path`
459+
config option in the Hadoop configuration used by Spark, like:
460+
461+
```
462+
<property>
463+
<name>hadoop.security.credential.provider.path</name>
464+
<value>jceks://[email protected]:9001/user/backup/ssl.jceks</value>
465+
</property>
466+
```
467+
468+
Or via SparkConf "spark.hadoop.hadoop.security.credential.provider.path=jceks://[email protected]:9001/user/backup/ssl.jceks".
469+
449470
## Preparing the key stores
450471

451472
Key stores can be generated by `keytool` program. The reference documentation for this tool for

0 commit comments

Comments
 (0)