This is an automated email from the ASF dual-hosted git repository.
dongjoon pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git
The following commit(s) were added to refs/heads/master by this push:
new 8e11f5f [SPARK-35576][SQL] Redact the sensitive info in the result of
Set command
8e11f5f is described below
commit 8e11f5f00764b60c7c621aa9d53f3cef0d8709ac
Author: Gengliang Wang <[email protected]>
AuthorDate: Mon May 31 14:50:18 2021 -0700
[SPARK-35576][SQL] Redact the sensitive info in the result of Set command
### What changes were proposed in this pull request?
Currently, the results of following SQL queries are not redacted:
```
SET [KEY];
SET;
```
For example:
```
scala> spark.sql("set javax.jdo.option.ConnectionPassword=123456").show()
+--------------------+------+
| key| value|
+--------------------+------+
|javax.jdo.option....|123456|
+--------------------+------+
scala> spark.sql("set javax.jdo.option.ConnectionPassword").show()
+--------------------+------+
| key| value|
+--------------------+------+
|javax.jdo.option....|123456|
+--------------------+------+
scala> spark.sql("set").show()
+--------------------+--------------------+
| key| value|
+--------------------+--------------------+
|javax.jdo.option....| 123456|
```
We should hide the sensitive information and redact the query output.
### Why are the changes needed?
Security.
### Does this PR introduce _any_ user-facing change?
Yes, the sensitive information in the output of Set commands are redacted
### How was this patch tested?
Unit test
Closes #32712 from gengliangwang/redactSet.
Authored-by: Gengliang Wang <[email protected]>
Signed-off-by: Dongjoon Hyun <[email protected]>
---
.../main/scala/org/apache/spark/sql/internal/SQLConf.scala | 9 ++++++++-
.../org/apache/spark/sql/execution/command/SetCommand.scala | 6 ++++--
.../src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala | 13 +++++++++++++
3 files changed, 25 insertions(+), 3 deletions(-)
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 3dfe4dc..e368ee1 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
@@ -4051,11 +4051,18 @@ class SQLConf extends Serializable with Logging {
* Redacts the given option map according to the description of
SQL_OPTIONS_REDACTION_PATTERN.
*/
def redactOptions[K, V](options: Map[K, V]): Map[K, V] = {
+ redactOptions(options.toSeq).toMap
+ }
+
+ /**
+ * Redacts the given option map according to the description of
SQL_OPTIONS_REDACTION_PATTERN.
+ */
+ def redactOptions[K, V](options: Seq[(K, V)]): Seq[(K, V)] = {
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
+ regexes.foldLeft(options) { case (opts, r) => Utils.redact(Some(r), opts) }
}
/**
diff --git
a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/SetCommand.scala
b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/SetCommand.scala
index c32d1d7..6f35c05 100644
---
a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/SetCommand.scala
+++
b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/SetCommand.scala
@@ -107,7 +107,8 @@ case class SetCommand(kv: Option[(String, Option[String])])
// Queries all key-value pairs that are set in the SQLConf of the
sparkSession.
case None =>
val runFunc = (sparkSession: SparkSession) => {
- sparkSession.conf.getAll.toSeq.sorted.map { case (k, v) => Row(k, v) }
+ val redactedConf = SQLConf.get.redactOptions(sparkSession.conf.getAll)
+ redactedConf.toSeq.sorted.map { case (k, v) => Row(k, v) }
}
(keyValueOutput, runFunc)
@@ -162,7 +163,8 @@ case class SetCommand(kv: Option[(String, Option[String])])
// very likely to change them based the default value they see.
sparkSession.sharedState.hadoopConf.get(key, "<undefined>")
}
- Seq(Row(key, value))
+ val (_, redactedValue) = SQLConf.get.redactOptions(Seq((key,
value))).head
+ Seq(Row(key, redactedValue))
}
(keyValueOutput, runFunc)
}
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala
b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala
index 26af0b9..31ffaeb 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala
@@ -1084,6 +1084,19 @@ class SQLQuerySuite extends QueryTest with
SharedSparkSession with AdaptiveSpark
checkAnswer(sql("SET io.file.buffer.size"), Row("io.file.buffer.size",
"65536"))
}
+ test("SPARK-35576: Set command should redact sensitive data") {
+ val key1 = "test.password"
+ val value1 = "test.value1"
+ val key2 = "test.token"
+ val value2 = "test.value2"
+ withSQLConf (key1 -> value1, key2 -> value2) {
+ checkAnswer(sql(s"SET $key1"), Row(key1, "*********(redacted)"))
+ checkAnswer(sql(s"SET $key2"), Row(key2, "*********(redacted)"))
+ val allValues = sql("SET").collect().map(_.getString(1))
+ assert(!allValues.exists(v => v.contains(value1) || v.contains(value2)))
+ }
+ }
+
test("apply schema") {
withTempView("applySchema1", "applySchema2", "applySchema3") {
val schema1 = StructType(
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]