Skip to content

Commit d3e8ab0

Browse files
Egor Krivokonekrivokonmapr
authored andcommitted
MapR [SPARK-945] Components can't read keyPassword (apache#883)
Co-authored-by: Egor Krivokon <>
1 parent ff6b6e8 commit d3e8ab0

File tree

2 files changed

+31
-11
lines changed

2 files changed

+31
-11
lines changed

conf/log4j.properties.template

Lines changed: 6 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -56,6 +56,12 @@ log4j.logger.org.apache.hadoop.hive.metastore.HiveMetastore=ERROR
5656
log4j.logger.org.apache.hadoop.hive.metastore.ObjectStore=ERROR
5757
log4j.logger.org.apache.hive.beeline.SQLCompleter=ERROR
5858

59+
# SPARK-945: Setting to suppress exception when non-cluster admin can not read ssl-server config
60+
log4j.logger.org.apache.hadoop.conf.Configuration=FATAL
61+
62+
# Hide Spark netty rpc error when driver is finished
63+
log4j.logger.org.apache.spark.rpc.netty.Dispatcher=FATAL
64+
5965
# For deploying Spark ThriftServer
6066
# SPARK-34128:Suppress undesirable TTransportException warnings involved in THRIFT-4805
6167
log4j.appender.console.filter.1=org.apache.log4j.varia.StringMatchFilter

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

Lines changed: 25 additions & 11 deletions
Original file line numberDiff line numberDiff line change
@@ -25,8 +25,10 @@ import scala.util.Try
2525

2626
import com.mapr.web.security.SslConfig.SslConfigScope
2727
import com.mapr.web.security.WebSecurityManager
28+
import javax.net.ssl.SSLContext
2829
import org.apache.hadoop.conf.Configuration
2930
import org.apache.hadoop.fs.Path
31+
import org.apache.spark.internal.Logging
3032
import org.eclipse.jetty.util.ssl.SslContextFactory
3133

3234
import org.eclipse.jetty.util.ssl.SslContextFactory
@@ -188,6 +190,7 @@ private[spark] object SSLOptions extends Logging {
188190
ns: String,
189191
defaults: Option[SSLOptions] = None): SSLOptions = {
190192
val IsSecurityWebUsing = conf.getBoolean("spark.maprSecurityWeb.usage", defaultValue = true)
193+
val defaultSSLKeyStorePassword = "defaultsslpassword"
191194

192195
val enabled = conf.getBoolean(s"$ns.enabled", defaultValue = defaults.exists(_.enabled))
193196
if (!enabled) {
@@ -199,30 +202,38 @@ private[spark] object SSLOptions extends Logging {
199202
}
200203

201204
val webSecuritySslConfig =
202-
Try(WebSecurityManager.getSslConfig(SslConfigScope.SCOPE_CLIENT_ONLY)).toOption
205+
Try(WebSecurityManager.getSslConfig(SslConfigScope.SCOPE_ALL)).recover {
206+
case _: SecurityException => WebSecurityManager.getSslConfig(SslConfigScope.SCOPE_CLIENT_ONLY)
207+
}.toOption
203208

204209
val newHaoopConf = new Configuration()
205210
val hadoopConfDir = System.getenv("hadoop_conf_dir")
206211
newHaoopConf.addResource(new Path(s"$hadoopConfDir/core-site.xml"))
207-
newHaoopConf.addResource(new Path(s"$hadoopConfDir/ssl-client.xml"))
212+
newHaoopConf.addResource(new Path(s"$hadoopConfDir/ssl-server.xml"))
208213

209214
val keyStore = conf.getWithSubstitution(s"$ns.keyStore").map(new File(_))
210215
.orElse(defaults.flatMap(_.keyStore))
211216

212217
val keyStorePassword = conf.getWithSubstitution(s"$ns.keyStorePassword")
213-
.orElse(if (IsSecurityWebUsing) {
218+
.orElse(Try {if (IsSecurityWebUsing) {
214219
webSecuritySslConfig.map(_.getClientKeystorePassword.mkString)
215220
} else {
216-
Option(newHaoopConf.getPassword("ssl.client.keystore.password")).map(new String(_))
217-
})
221+
Option(new String(newHaoopConf.getPassword("ssl.client.keystore.password")))
222+
}}.getOrElse({
223+
logWarning("SSL keyStore password is not set, using default.")
224+
Option(defaultSSLKeyStorePassword)
225+
}))
218226
.orElse(defaults.flatMap(_.keyStorePassword))
219227

220228
val keyPassword = conf.getWithSubstitution(s"$ns.keyPassword")
221-
.orElse(if (IsSecurityWebUsing) {
229+
.orElse(Try {if (IsSecurityWebUsing) {
222230
webSecuritySslConfig.map(_.getClientKeyPassword.mkString)
223231
} else {
224-
Option(hadoopConf.getPassword(s"ssl.client.keystore.keypassword")).map(new String(_))
225-
})
232+
Option(new String(newHaoopConf.getPassword("ssl.client.keystore.keypassword")))
233+
}}.getOrElse({
234+
logWarning("SSL key password is not set, using default.")
235+
Option(defaultSSLKeyStorePassword)
236+
}))
226237
.orElse(defaults.flatMap(_.keyPassword))
227238

228239
val keyStoreType = conf.getWithSubstitution(s"$ns.keyStoreType")
@@ -235,11 +246,14 @@ private[spark] object SSLOptions extends Logging {
235246
.orElse(defaults.flatMap(_.trustStore))
236247

237248
val trustStorePassword = conf.getWithSubstitution(s"$ns.trustStorePassword")
238-
.orElse(if (IsSecurityWebUsing) {
249+
.orElse(Try {if (IsSecurityWebUsing) {
239250
webSecuritySslConfig.map(_.getClientTruststorePassword.mkString)
240251
} else {
241-
Option(newHaoopConf.getPassword("ssl.client.truststore.password")).map(new String(_))
242-
})
252+
Option(new String(newHaoopConf.getPassword("ssl.client.truststore.password")))
253+
}}.getOrElse({
254+
logWarning("SSL trustStore password is not set, using default.")
255+
Option(defaultSSLKeyStorePassword)
256+
}))
243257
.orElse(defaults.flatMap(_.trustStorePassword))
244258

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

0 commit comments

Comments
 (0)