aboutsummaryrefslogtreecommitdiff
path: root/core/src
diff options
context:
space:
mode:
authorMark Grover <mark@apache.org>2017-03-02 10:33:56 -0800
committerMarcelo Vanzin <vanzin@cloudera.com>2017-03-02 10:33:56 -0800
commit5ae3516bfb7716f1793eb76b4fdc720b31829d07 (patch)
treea3382825bc898cc4932b1e998072bd8e9df21b09 /core/src
parent9cca3dbf4add9004a769dee1a556987e37230294 (diff)
downloadspark-5ae3516bfb7716f1793eb76b4fdc720b31829d07.tar.gz
spark-5ae3516bfb7716f1793eb76b4fdc720b31829d07.tar.bz2
spark-5ae3516bfb7716f1793eb76b4fdc720b31829d07.zip
[SPARK-19720][CORE] Redact sensitive information from SparkSubmit console
## What changes were proposed in this pull request? This change redacts senstive information (based on `spark.redaction.regex` property) from the Spark Submit console logs. Such sensitive information is already being redacted from event logs and yarn logs, etc. ## How was this patch tested? Testing was done manually to make sure that the console logs were not printing any sensitive information. Here's some output from the console: ``` Spark properties used, including those specified through --conf and those from the properties file /etc/spark2/conf/spark-defaults.conf: (spark.yarn.appMasterEnv.HADOOP_CREDSTORE_PASSWORD,*********(redacted)) (spark.authenticate,false) (spark.executorEnv.HADOOP_CREDSTORE_PASSWORD,*********(redacted)) ``` ``` System properties: (spark.yarn.appMasterEnv.HADOOP_CREDSTORE_PASSWORD,*********(redacted)) (spark.authenticate,false) (spark.executorEnv.HADOOP_CREDSTORE_PASSWORD,*********(redacted)) ``` There is a risk if new print statements were added to the console down the road, sensitive information may still get leaked, since there is no test that asserts on the console log output. I considered it out of the scope of this JIRA to write an integration test to make sure new leaks don't happen in the future. Running unit tests to make sure nothing else is broken by this change. Author: Mark Grover <mark@apache.org> Closes #17047 from markgrover/master_redaction.
Diffstat (limited to 'core/src')
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala3
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala12
-rw-r--r--core/src/main/scala/org/apache/spark/util/Utils.scala21
3 files changed, 31 insertions, 5 deletions
diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
index 5ffdedd165..1e50eb6635 100644
--- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
@@ -665,7 +665,8 @@ object SparkSubmit extends CommandLineUtils {
if (verbose) {
printStream.println(s"Main class:\n$childMainClass")
printStream.println(s"Arguments:\n${childArgs.mkString("\n")}")
- printStream.println(s"System properties:\n${sysProps.mkString("\n")}")
+ // sysProps may contain sensitive information, so redact before printing
+ printStream.println(s"System properties:\n${Utils.redact(sysProps).mkString("\n")}")
printStream.println(s"Classpath elements:\n${childClasspath.mkString("\n")}")
printStream.println("\n")
}
diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala
index dee77343d8..0614d80b60 100644
--- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala
@@ -84,9 +84,15 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S
// scalastyle:off println
if (verbose) SparkSubmit.printStream.println(s"Using properties file: $propertiesFile")
Option(propertiesFile).foreach { filename =>
- Utils.getPropertiesFromFile(filename).foreach { case (k, v) =>
+ val properties = Utils.getPropertiesFromFile(filename)
+ properties.foreach { case (k, v) =>
defaultProperties(k) = v
- if (verbose) SparkSubmit.printStream.println(s"Adding default property: $k=$v")
+ }
+ // Property files may contain sensitive information, so redact before printing
+ if (verbose) {
+ Utils.redact(properties).foreach { case (k, v) =>
+ SparkSubmit.printStream.println(s"Adding default property: $k=$v")
+ }
}
}
// scalastyle:on println
@@ -318,7 +324,7 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S
|
|Spark properties used, including those specified through
| --conf and those from the properties file $propertiesFile:
- |${sparkProperties.mkString(" ", "\n ", "\n")}
+ |${Utils.redact(sparkProperties).mkString(" ", "\n ", "\n")}
""".stripMargin
}
diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala
index 10e5233679..1af34e3da2 100644
--- a/core/src/main/scala/org/apache/spark/util/Utils.scala
+++ b/core/src/main/scala/org/apache/spark/util/Utils.scala
@@ -39,6 +39,7 @@ import scala.io.Source
import scala.reflect.ClassTag
import scala.util.Try
import scala.util.control.{ControlThrowable, NonFatal}
+import scala.util.matching.Regex
import _root_.io.netty.channel.unix.Errors.NativeIoException
import com.google.common.cache.{CacheBuilder, CacheLoader, LoadingCache}
@@ -2588,13 +2589,31 @@ private[spark] object Utils extends Logging {
def redact(conf: SparkConf, kvs: Seq[(String, String)]): Seq[(String, String)] = {
val redactionPattern = conf.get(SECRET_REDACTION_PATTERN).r
+ redact(redactionPattern, kvs)
+ }
+
+ private def redact(redactionPattern: Regex, kvs: Seq[(String, String)]): Seq[(String, String)] = {
kvs.map { kv =>
redactionPattern.findFirstIn(kv._1)
- .map { ignore => (kv._1, REDACTION_REPLACEMENT_TEXT) }
+ .map { _ => (kv._1, REDACTION_REPLACEMENT_TEXT) }
.getOrElse(kv)
}
}
+ /**
+ * Looks up the redaction regex from within the key value pairs and uses it to redact the rest
+ * of the key value pairs. No care is taken to make sure the redaction property itself is not
+ * redacted. So theoretically, the property itself could be configured to redact its own value
+ * when printing.
+ */
+ def redact(kvs: Map[String, String]): Seq[(String, String)] = {
+ val redactionPattern = kvs.getOrElse(
+ SECRET_REDACTION_PATTERN.key,
+ SECRET_REDACTION_PATTERN.defaultValueString
+ ).r
+ redact(redactionPattern, kvs.toArray)
+ }
+
}
private[util] object CallerContext extends Logging {