You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by va...@apache.org on 2018/05/18 18:14:29 UTC
spark git commit: [SPARK-23850][SQL] Add separate config for SQL
options redaction.
Repository: spark
Updated Branches:
refs/heads/master 807ba44cb -> ed7ba7db8
[SPARK-23850][SQL] Add separate config for SQL options redaction.
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.
Author: Marcelo Vanzin <va...@cloudera.com>
Closes #21158 from vanzin/SPARK-23850.
Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/ed7ba7db
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/ed7ba7db
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/ed7ba7db
Branch: refs/heads/master
Commit: ed7ba7db8fa344ff182b72d23ae458e711f63432
Parents: 807ba44
Author: Marcelo Vanzin <va...@cloudera.com>
Authored: Fri May 18 11:14:22 2018 -0700
Committer: Marcelo Vanzin <va...@cloudera.com>
Committed: Fri May 18 11:14:22 2018 -0700
----------------------------------------------------------------------
.../apache/spark/internal/config/package.scala | 2 +-
.../org/apache/spark/sql/internal/SQLConf.scala | 24 ++++++++++++++++++--
.../sql/execution/DataSourceScanExec.scala | 2 +-
.../spark/sql/execution/QueryExecution.scala | 2 +-
.../datasources/SaveIntoDataSourceCommand.scala | 5 ++--
.../SaveIntoDataSourceCommandSuite.scala | 3 ---
6 files changed, 27 insertions(+), 11 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/spark/blob/ed7ba7db/core/src/main/scala/org/apache/spark/internal/config/package.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala
index 82f0a04..a54b091 100644
--- a/core/src/main/scala/org/apache/spark/internal/config/package.scala
+++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala
@@ -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)
private[spark] val STRING_REDACTION_PATTERN =
ConfigBuilder("spark.redaction.string.regex")
http://git-wip-us.apache.org/repos/asf/spark/blob/ed7ba7db/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
index 2a673c6..53a5030 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
@@ -1155,8 +1155,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. " +
@@ -1429,7 +1438,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)
@@ -1739,6 +1748,17 @@ class SQLConf extends Serializable with Logging {
}
/**
+ * 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]].
*/
def contains(key: String): Boolean = {
http://git-wip-us.apache.org/repos/asf/spark/blob/ed7ba7db/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala
index 08ff33a..61c14fe 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala
@@ -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)
}
}
http://git-wip-us.apache.org/repos/asf/spark/blob/ed7ba7db/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala
index 15379a0..3112b30 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala
@@ -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)
}
/** A special namespace for commands that can be used to debug query execution. */
http://git-wip-us.apache.org/repos/asf/spark/blob/ed7ba7db/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/SaveIntoDataSourceCommand.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/SaveIntoDataSourceCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/SaveIntoDataSourceCommand.scala
index 568e953..00b1b5d 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/SaveIntoDataSourceCommand.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/SaveIntoDataSourceCommand.scala
@@ -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.
@@ -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}"
}
}
http://git-wip-us.apache.org/repos/asf/spark/blob/ed7ba7db/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SaveIntoDataSourceCommandSuite.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SaveIntoDataSourceCommandSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SaveIntoDataSourceCommandSuite.scala
index 4b3ca8e..a1da3ec 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SaveIntoDataSourceCommandSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SaveIntoDataSourceCommandSuite.scala
@@ -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"
---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@spark.apache.org
For additional commands, e-mail: commits-help@spark.apache.org