Skip to content

Commit

Permalink
[SPARK-23850][SQL][BRANCH-2.2] Add separate config for SQL options re…
Browse files Browse the repository at this point in the history
…daction.

The old code was relying on a core configuration and extended its
default value to include things that redact desired things in the
app's environment. Instead, add a SQL-specific option for which
options to redact, and apply both the core and SQL-specific rules
when redacting the options in the save command.

This is a little sub-optimal since it adds another config, but it
retains the current default behavior.

While there I also fixed a typo and a couple of minor config API
usage issues in the related redaction option that SQL already had.

Tested with existing unit tests, plus checking the env page on
a shell UI.

(cherry picked from commit ed7ba7d)

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes apache#21365 from vanzin/SPARK-23850-2.2.
  • Loading branch information
Marcelo Vanzin authored and MatthewRBruce committed Jul 31, 2018
1 parent c5012f7 commit d90e19a
Show file tree
Hide file tree
Showing 5 changed files with 37 additions and 8 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -247,7 +247,7 @@ package object config {
"a property key or value, the value is redacted from the environment UI and various logs " +
"like YARN and event logs.")
.regexConf
.createWithDefault("(?i)secret|password|url|user|username".r)
.createWithDefault("(?i)secret|password".r)

private[spark] val STRING_REDACTION_PATTERN =
ConfigBuilder("spark.redaction.string.regex")
Expand Down
11 changes: 11 additions & 0 deletions core/src/main/scala/org/apache/spark/util/Utils.scala
Original file line number Diff line number Diff line change
Expand Up @@ -2625,6 +2625,17 @@ private[spark] object Utils extends Logging {
}
}

/**
* Redact the sensitive values in the given map. If a map key matches the redaction pattern then
* its value is replaced with a dummy text.
*/
def redact(regex: Option[Regex], kvs: Seq[(String, String)]): Seq[(String, String)] = {
regex match {
case None => kvs
case Some(r) => redact(r, kvs)
}
}

private def redact(redactionPattern: Regex, kvs: Seq[(String, String)]): Seq[(String, String)] = {
// If the sensitive information regex matches with either the key or the value, redact the value
// While the original intent was to only redact the value if the key matched with the regex,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,7 @@ import org.apache.spark.internal.Logging
import org.apache.spark.internal.config._
import org.apache.spark.network.util.ByteUnit
import org.apache.spark.sql.catalyst.analysis.Resolver
import org.apache.spark.util.Utils
import org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter

////////////////////////////////////////////////////////////////////////////////////////////////////
Expand Down Expand Up @@ -819,6 +820,15 @@ object SQLConf {
.intConf
.createWithDefault(UnsafeExternalSorter.DEFAULT_NUM_ELEMENTS_FOR_SPILL_THRESHOLD.toInt)

val SQL_OPTIONS_REDACTION_PATTERN =
buildConf("spark.sql.redaction.options.regex")
.doc("Regex to decide which keys in a Spark SQL command's options map contain sensitive " +
"information. The values of options whose names that match this regex will be redacted " +
"in the explain output. This redaction is applied on top of the global redaction " +
s"configuration defined by ${SECRET_REDACTION_PATTERN.key}.")
.regexConf
.createWithDefault("(?i)url".r)

object Deprecated {
val MAPRED_REDUCE_TASKS = "mapred.reduce.tasks"
}
Expand Down Expand Up @@ -1181,6 +1191,17 @@ class SQLConf extends Serializable with Logging {
}.toSeq
}

/**
* Redacts the given option map according to the description of SQL_OPTIONS_REDACTION_PATTERN.
*/
def redactOptions(options: Map[String, String]): Map[String, String] = {
val regexes = Seq(
getConf(SQL_OPTIONS_REDACTION_PATTERN),
SECRET_REDACTION_PATTERN.readFrom(reader))

regexes.foldLeft(options.toSeq) { case (opts, r) => Utils.redact(Some(r), opts) }.toMap
}

/**
* Return whether a given key is set in this [[SQLConf]].
*/
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,12 +17,10 @@

package org.apache.spark.sql.execution.datasources

import org.apache.spark.SparkEnv
import org.apache.spark.sql.{Dataset, Row, SaveMode, SparkSession}
import org.apache.spark.sql.catalyst.plans.QueryPlan
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
import org.apache.spark.sql.execution.command.RunnableCommand
import org.apache.spark.util.Utils

/**
* Saves the results of `query` in to a data source.
Expand Down Expand Up @@ -53,7 +51,9 @@ case class SaveIntoDataSourceCommand(
}

override def simpleString: String = {
val redacted = Utils.redact(SparkEnv.get.conf, options.toSeq).toMap
s"SaveIntoDataSourceCommand ${provider}, ${partitionColumns}, ${redacted}, ${mode}"
val redacted = SparkSession.getActiveSession
.map(_.sessionState.conf.redactOptions(options))
.getOrElse(Map())
s"SaveIntoDataSourceCommand ${provider}, ${redacted}, ${mode}"
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -23,9 +23,6 @@ import org.apache.spark.sql.test.SharedSQLContext

class SaveIntoDataSourceCommandSuite extends SharedSQLContext {

override protected def sparkConf: SparkConf = super.sparkConf
.set("spark.redaction.regex", "(?i)password|url")

test("simpleString is redacted") {
val URL = "connection.url"
val PASS = "123"
Expand Down

0 comments on commit d90e19a

Please sign in to comment.