Skip to content

Commit

Permalink
MapR [SPARK-945] Components can't read keyPassword (apache#883)
Browse files Browse the repository at this point in the history
Co-authored-by: Egor Krivokon <>
  • Loading branch information
ekrivokonmapr authored and Egor Krivokon committed Oct 26, 2021
1 parent 8981e69 commit e6de5c2
Show file tree
Hide file tree
Showing 2 changed files with 32 additions and 15 deletions.
6 changes: 6 additions & 0 deletions conf/log4j.properties.template
Original file line number Diff line number Diff line change
Expand Up @@ -56,6 +56,12 @@ log4j.logger.org.apache.hadoop.hive.metastore.HiveMetastore=ERROR
log4j.logger.org.apache.hadoop.hive.metastore.ObjectStore=ERROR
log4j.logger.org.apache.hive.beeline.SQLCompleter=ERROR

# SPARK-945: Setting to suppress exception when non-cluster admin can not read ssl-server config
log4j.logger.org.apache.hadoop.conf.Configuration=FATAL

# Hide Spark netty rpc error when driver is finished
log4j.logger.org.apache.spark.rpc.netty.Dispatcher=FATAL

# For deploying Spark ThriftServer
# SPARK-34128:Suppress undesirable TTransportException warnings involved in THRIFT-4805
log4j.appender.console.filter.1=org.apache.log4j.varia.StringMatchFilter
Expand Down
41 changes: 26 additions & 15 deletions core/src/main/scala/org/apache/spark/SSLOptions.scala
Original file line number Diff line number Diff line change
Expand Up @@ -19,17 +19,16 @@ package org.apache.spark

import java.io.File
import java.security.NoSuchAlgorithmException
import javax.net.ssl.SSLContext

import scala.util.Try

import com.mapr.web.security.SslConfig.SslConfigScope
import com.mapr.web.security.WebSecurityManager
import javax.net.ssl.SSLContext
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.Path
import org.apache.spark.internal.Logging
import org.eclipse.jetty.util.ssl.SslContextFactory

import org.apache.spark.internal.Logging
import scala.util.Try

/**
* SSLOptions class is a common container for SSL configuration options. It offers methods to
Expand Down Expand Up @@ -195,6 +194,7 @@ private[spark] object SSLOptions extends Logging {
ns: String,
defaults: Option[SSLOptions] = None): SSLOptions = {
val IsSecurityWebUsing = conf.getBoolean("spark.maprSecurityWeb.usage", defaultValue = true)
val defaultSSLKeyStorePassword = "defaultsslpassword"

val enabled = conf.getBoolean(s"$ns.enabled", defaultValue = defaults.exists(_.enabled))

Expand All @@ -204,30 +204,38 @@ private[spark] object SSLOptions extends Logging {
}

val webSecuritySslConfig =
Try(WebSecurityManager.getSslConfig(SslConfigScope.SCOPE_CLIENT_ONLY)).toOption
Try(WebSecurityManager.getSslConfig(SslConfigScope.SCOPE_ALL)).recover {
case _: SecurityException => WebSecurityManager.getSslConfig(SslConfigScope.SCOPE_CLIENT_ONLY)
}.toOption

val newHaoopConf = new Configuration()
val hadoopConfDir = System.getenv("hadoop_conf_dir")
newHaoopConf.addResource(new Path(s"$hadoopConfDir/core-site.xml"))
newHaoopConf.addResource(new Path(s"$hadoopConfDir/ssl-client.xml"))
newHaoopConf.addResource(new Path(s"$hadoopConfDir/ssl-server.xml"))

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

val keyStorePassword = conf.getWithSubstitution(s"$ns.keyStorePassword")
.orElse(if (IsSecurityWebUsing) {
.orElse(Try {if (IsSecurityWebUsing) {
webSecuritySslConfig.map(_.getClientKeystorePassword.mkString)
} else {
Option(newHaoopConf.getPassword("ssl.client.keystore.password")).map(new String(_))
})
Option(new String(newHaoopConf.getPassword("ssl.client.keystore.password")))
}}.getOrElse({
logWarning("SSL keyStore password is not set, using default.")
Option(defaultSSLKeyStorePassword)
}))
.orElse(defaults.flatMap(_.keyStorePassword))

val keyPassword = conf.getWithSubstitution(s"$ns.keyPassword")
.orElse(if (IsSecurityWebUsing) {
.orElse(Try {if (IsSecurityWebUsing) {
webSecuritySslConfig.map(_.getClientKeyPassword.mkString)
} else {
Option(hadoopConf.getPassword(s"ssl.client.keystore.keypassword")).map(new String(_))
})
Option(new String(newHaoopConf.getPassword("ssl.client.keystore.keypassword")))
}}.getOrElse({
logWarning("SSL key password is not set, using default.")
Option(defaultSSLKeyStorePassword)
}))
.orElse(defaults.flatMap(_.keyPassword))

val keyStoreType = conf.getWithSubstitution(s"$ns.keyStoreType")
Expand All @@ -240,11 +248,14 @@ private[spark] object SSLOptions extends Logging {
.orElse(defaults.flatMap(_.trustStore))

val trustStorePassword = conf.getWithSubstitution(s"$ns.trustStorePassword")
.orElse(if (IsSecurityWebUsing) {
.orElse(Try {if (IsSecurityWebUsing) {
webSecuritySslConfig.map(_.getClientTruststorePassword.mkString)
} else {
Option(newHaoopConf.getPassword("ssl.client.truststore.password")).map(new String(_))
})
Option(new String(newHaoopConf.getPassword("ssl.client.truststore.password")))
}}.getOrElse({
logWarning("SSL trustStore password is not set, using default.")
Option(defaultSSLKeyStorePassword)
}))
.orElse(defaults.flatMap(_.trustStorePassword))

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

0 comments on commit e6de5c2

Please sign in to comment.