Skip to content

[SPARK-23850][sql] Add separate config for SQL options redaction. #21158

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Closed
wants to merge 2 commits into from
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -342,7 +342,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)
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

If the users want to see the values, they can change the default by themselves, right?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Not sure what you're trying to imply? The default is redacting too much. That needs to be fixed.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Normally, making it stricter is not a big deal. However, if we want to relax it, this could be a big deal since users might expect these variables have been hided by default. cc the original reviewers and author @onursatici @ash211 @hvanhovell @jiangxb1987

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

URIs could contain the access keys and usernames could have potentially personal identifiable information. Could we do not change the default?


private[spark] val STRING_REDACTION_PATTERN =
ConfigBuilder("spark.redaction.string.regex")
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1147,8 +1147,17 @@ object SQLConf {
.booleanConf
.createWithDefault(true)

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)

val SQL_STRING_REDACTION_PATTERN =
ConfigBuilder("spark.sql.redaction.string.regex")
buildConf("spark.sql.redaction.string.regex")
.doc("Regex to decide which parts of strings produced by Spark contain sensitive " +
"information. When this regex matches a string part, that string part is replaced by a " +
"dummy value. This is currently used to redact the output of SQL explain commands. " +
Expand Down Expand Up @@ -1402,7 +1411,7 @@ class SQLConf extends Serializable with Logging {

def fileCompressionFactor: Double = getConf(FILE_COMRESSION_FACTOR)

def stringRedationPattern: Option[Regex] = SQL_STRING_REDACTION_PATTERN.readFrom(reader)
def stringRedactionPattern: Option[Regex] = getConf(SQL_STRING_REDACTION_PATTERN)

def sortBeforeRepartition: Boolean = getConf(SORT_BEFORE_REPARTITION)

Expand Down Expand Up @@ -1709,6 +1718,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 @@ -69,7 +69,7 @@ trait DataSourceScanExec extends LeafExecNode with CodegenSupport {
* Shorthand for calling redactString() without specifying redacting rules
*/
private def redact(text: String): String = {
Utils.redact(sqlContext.sessionState.conf.stringRedationPattern, text)
Utils.redact(sqlContext.sessionState.conf.stringRedactionPattern, text)
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -225,7 +225,7 @@ class QueryExecution(val sparkSession: SparkSession, val logical: LogicalPlan) {
* Redact the sensitive information in the given string.
*/
private def withRedaction(message: String): String = {
Utils.redact(sparkSession.sessionState.conf.stringRedationPattern, message)
Utils.redact(sparkSession.sessionState.conf.stringRedactionPattern, message)
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks for fixing this typo

}

/** A special namespace for commands that can be used to debug query execution. */
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,13 +17,12 @@

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.sql.internal.SQLConf
import org.apache.spark.sql.sources.CreatableRelationProvider
import org.apache.spark.util.Utils

/**
* Saves the results of `query` in to a data source.
Expand All @@ -50,7 +49,7 @@ case class SaveIntoDataSourceCommand(
}

override def simpleString: String = {
val redacted = Utils.redact(SparkEnv.get.conf, options.toSeq).toMap
val redacted = SQLConf.get.redactOptions(options)
s"SaveIntoDataSourceCommand ${dataSource}, ${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