18
18
package org .apache .spark
19
19
20
20
import java .io .File
21
+ import java .util .UUID
21
22
import javax .net .ssl .SSLContext
22
23
24
+ import org .apache .hadoop .conf .Configuration
25
+ import org .apache .hadoop .security .alias .{CredentialProvider , CredentialProviderFactory }
23
26
import org .scalatest .BeforeAndAfterAll
24
27
25
28
import org .apache .spark .util .SparkConfWithEnv
@@ -40,6 +43,7 @@ class SSLOptionsSuite extends SparkFunSuite with BeforeAndAfterAll {
40
43
.toSet
41
44
42
45
val conf = new SparkConf
46
+ val hadoopConf = new Configuration ()
43
47
conf.set(" spark.ssl.enabled" , " true" )
44
48
conf.set(" spark.ssl.keyStore" , keyStorePath)
45
49
conf.set(" spark.ssl.keyStorePassword" , " password" )
@@ -49,7 +53,7 @@ class SSLOptionsSuite extends SparkFunSuite with BeforeAndAfterAll {
49
53
conf.set(" spark.ssl.enabledAlgorithms" , algorithms.mkString(" ," ))
50
54
conf.set(" spark.ssl.protocol" , " TLSv1.2" )
51
55
52
- val opts = SSLOptions .parse(conf, " spark.ssl" )
56
+ val opts = SSLOptions .parse(conf, hadoopConf, " spark.ssl" )
53
57
54
58
assert(opts.enabled === true )
55
59
assert(opts.trustStore.isDefined === true )
@@ -70,6 +74,7 @@ class SSLOptionsSuite extends SparkFunSuite with BeforeAndAfterAll {
70
74
val trustStorePath = new File (this .getClass.getResource(" /truststore" ).toURI).getAbsolutePath
71
75
72
76
val conf = new SparkConf
77
+ val hadoopConf = new Configuration ()
73
78
conf.set(" spark.ssl.enabled" , " true" )
74
79
conf.set(" spark.ssl.keyStore" , keyStorePath)
75
80
conf.set(" spark.ssl.keyStorePassword" , " password" )
@@ -80,8 +85,8 @@ class SSLOptionsSuite extends SparkFunSuite with BeforeAndAfterAll {
80
85
" TLS_RSA_WITH_AES_128_CBC_SHA, TLS_RSA_WITH_AES_256_CBC_SHA" )
81
86
conf.set(" spark.ssl.protocol" , " SSLv3" )
82
87
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))
85
90
86
91
assert(opts.enabled === true )
87
92
assert(opts.trustStore.isDefined === true )
@@ -103,6 +108,7 @@ class SSLOptionsSuite extends SparkFunSuite with BeforeAndAfterAll {
103
108
val trustStorePath = new File (this .getClass.getResource(" /truststore" ).toURI).getAbsolutePath
104
109
105
110
val conf = new SparkConf
111
+ val hadoopConf = new Configuration ()
106
112
conf.set(" spark.ssl.enabled" , " true" )
107
113
conf.set(" spark.ssl.ui.enabled" , " false" )
108
114
conf.set(" spark.ssl.ui.port" , " 4242" )
@@ -117,8 +123,8 @@ class SSLOptionsSuite extends SparkFunSuite with BeforeAndAfterAll {
117
123
conf.set(" spark.ssl.ui.enabledAlgorithms" , " ABC, DEF" )
118
124
conf.set(" spark.ssl.protocol" , " SSLv3" )
119
125
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))
122
128
123
129
assert(opts.enabled === false )
124
130
assert(opts.port === Some (4242 ))
@@ -139,14 +145,71 @@ class SSLOptionsSuite extends SparkFunSuite with BeforeAndAfterAll {
139
145
val conf = new SparkConfWithEnv (Map (
140
146
" ENV1" -> " val1" ,
141
147
" ENV2" -> " val2" ))
148
+ val hadoopConf = new Configuration ()
142
149
143
150
conf.set(" spark.ssl.enabled" , " true" )
144
151
conf.set(" spark.ssl.keyStore" , " ${env:ENV1}" )
145
152
conf.set(" spark.ssl.trustStore" , " ${env:ENV2}" )
146
153
147
- val opts = SSLOptions .parse(conf, " spark.ssl" , defaults = None )
154
+ val opts = SSLOptions .parse(conf, hadoopConf, " spark.ssl" , defaults = None )
148
155
assert(opts.keyStore === Some (new File (" val1" )))
149
156
assert(opts.trustStore === Some (new File (" val2" )))
150
157
}
151
158
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
+ }
152
215
}
0 commit comments