Skip to content
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

[SPARK-22791] [SQL] [SS] Redact Output of Explain #19985

Closed
wants to merge 5 commits into from

Conversation

gatorsmile
Copy link
Member

@gatorsmile gatorsmile commented Dec 15, 2017

What changes were proposed in this pull request?

When calling explain on a query, the output can contain sensitive information. We should provide an admin/user to redact such information.

Before this PR, the plan of SS is like this

== Physical Plan ==
*HashAggregate(keys=[value#6], functions=[count(1)], output=[value#6, count(1)#12L])
+- StateStoreSave [value#6], state info [ checkpoint = file:/private/var/folders/vx/j0ydl5rn0gd9mgrh1pljnw900000gn/T/temporary-91c6fac0-609f-4bc8-ad57-52c189f06797/state, runId = 05a4b3af-f02c-40f8-9ff9-a3e18bae496f, opId = 0, ver = 0, numPartitions = 5], Complete, 0
   +- *HashAggregate(keys=[value#6], functions=[merge_count(1)], output=[value#6, count#18L])
      +- StateStoreRestore [value#6], state info [ checkpoint = file:/private/var/folders/vx/j0ydl5rn0gd9mgrh1pljnw900000gn/T/temporary-91c6fac0-609f-4bc8-ad57-52c189f06797/state, runId = 05a4b3af-f02c-40f8-9ff9-a3e18bae496f, opId = 0, ver = 0, numPartitions = 5]
         +- *HashAggregate(keys=[value#6], functions=[merge_count(1)], output=[value#6, count#18L])
            +- Exchange hashpartitioning(value#6, 5)
               +- *HashAggregate(keys=[value#6], functions=[partial_count(1)], output=[value#6, count#18L])
                  +- *SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#6]
                     +- *MapElements <function1>, obj#5: java.lang.String
                        +- *DeserializeToObject value#30.toString, obj#4: java.lang.String
                           +- LocalTableScan [value#30]

After this PR, we can get the following output if users set spark.redaction.string.regex to file:/[\\w_]+

== Physical Plan ==
*HashAggregate(keys=[value#6], functions=[count(1)], output=[value#6, count(1)#12L])
+- StateStoreSave [value#6], state info [ checkpoint = *********(redacted)/var/folders/vx/j0ydl5rn0gd9mgrh1pljnw900000gn/T/temporary-e7da9b7d-3ec0-474d-8b8c-927f7d12ed72/state, runId = 8a9c3761-93d5-4896-ab82-14c06240dcea, opId = 0, ver = 0, numPartitions = 5], Complete, 0
   +- *HashAggregate(keys=[value#6], functions=[merge_count(1)], output=[value#6, count#32L])
      +- StateStoreRestore [value#6], state info [ checkpoint = *********(redacted)/var/folders/vx/j0ydl5rn0gd9mgrh1pljnw900000gn/T/temporary-e7da9b7d-3ec0-474d-8b8c-927f7d12ed72/state, runId = 8a9c3761-93d5-4896-ab82-14c06240dcea, opId = 0, ver = 0, numPartitions = 5]
         +- *HashAggregate(keys=[value#6], functions=[merge_count(1)], output=[value#6, count#32L])
            +- Exchange hashpartitioning(value#6, 5)
               +- *HashAggregate(keys=[value#6], functions=[partial_count(1)], output=[value#6, count#32L])
                  +- *SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#6]
                     +- *MapElements <function1>, obj#5: java.lang.String
                        +- *DeserializeToObject value#27.toString, obj#4: java.lang.String
                           +- LocalTableScan [value#27]

How was this patch tested?

Added a test case

@gatorsmile
Copy link
Member Author

gatorsmile commented Dec 15, 2017

override protected def sparkConf: SparkConf = super.sparkConf
.set("spark.redaction.string.regex", "file:/[\\w_]+")

test("explain - redaction") {
Copy link
Member Author

Choose a reason for hiding this comment

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

This is just a SS example. I believe we have more such cases.

Copy link
Member

Choose a reason for hiding this comment

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

Is it because you want to use explainInternal, so you write a SS test?

Copy link
Member Author

Choose a reason for hiding this comment

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

Just because I found a potential security issue in SS.

@SparkQA
Copy link

SparkQA commented Dec 15, 2017

Test build #84946 has finished for PR 19985 at commit 8f44edf.

  • This patch fails due to an unknown error code, -9.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Dec 15, 2017

Test build #84949 has finished for PR 19985 at commit 8f44edf.

  • This patch fails SparkR unit tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@gatorsmile
Copy link
Member Author

retest this please

* Redact the sensitive information in the given string.
*/
private def withRedaction(message: => String): String = {
Utils.redact(SparkSession.getActiveSession.map(_.sparkContext.conf).orNull, message)
Copy link
Member

Choose a reason for hiding this comment

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

Why not make Utils.redact accept a regex parameter so that we can use session conf?

Copy link
Member

Choose a reason for hiding this comment

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

A different query may need to use a different regex, a session conf will be better and the user can change it during runtime.


val explainWithoutExtended = q.explainInternal(false)
assert(explainWithoutExtended.contains(replacement))
assert(explainWithoutExtended.contains("StateStoreRestore"))
Copy link
Member

Choose a reason for hiding this comment

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

nit: add assert(!explainWithoutExtended.contains("file:/")) to verify it does replace the correct content.

Copy link
Member Author

Choose a reason for hiding this comment

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

done.


val explainWithExtended = q.explainInternal(true)
assert(explainWithExtended.contains(replacement))
assert(explainWithExtended.contains("StateStoreRestore"))
Copy link
Member

Choose a reason for hiding this comment

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

ditto

@zsxwing
Copy link
Member

zsxwing commented Dec 15, 2017

Looks good. My major comment is using a session conf instead.

@SparkQA
Copy link

SparkQA commented Dec 15, 2017

Test build #84968 has finished for PR 19985 at commit 8f44edf.

  • This patch fails SparkR unit tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Dec 16, 2017

Test build #85002 has finished for PR 19985 at commit e246096.

  • This patch fails PySpark unit tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@gatorsmile
Copy link
Member Author

retest this please

@SparkQA
Copy link

SparkQA commented Dec 16, 2017

Test build #85011 has finished for PR 19985 at commit e246096.

  • This patch fails SparkR unit tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@gatorsmile
Copy link
Member Author

retest this please

@gatorsmile
Copy link
Member Author

cc @zsxwing

.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. " +
"When this conf is not set, the value from `spark.sql.redaction.string.regex` is used.")
Copy link
Member

Choose a reason for hiding this comment

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

nit: spark.sql.redaction.string.regex -> spark.redaction.string.regex

Copy link
Member

@zsxwing zsxwing left a comment

Choose a reason for hiding this comment

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

LGTM except one nit

@SparkQA
Copy link

SparkQA commented Dec 19, 2017

Test build #85077 has finished for PR 19985 at commit d50d594.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

/**
* Redact the sensitive information in the given string.
*/
private def withRedaction(message: => String): String = {
Copy link
Contributor

Choose a reason for hiding this comment

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

=> String looks not very useful here, we need to materialize anyway when calling Utils.redact

assert(isIncluded(df.queryExecution, replacement))

assert(isIncluded(df.queryExecution, "FileScan"))
assert(!isIncluded(df.queryExecution, "file:/"))
Copy link
Contributor

Choose a reason for hiding this comment

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

Copy link
Member Author

Choose a reason for hiding this comment

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

Removed

@SparkQA
Copy link

SparkQA commented Dec 19, 2017

Test build #85090 has finished for PR 19985 at commit 75c1479.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@cloud-fan
Copy link
Contributor

thanks, merging to master!

@asfgit asfgit closed this in 2831571 Dec 19, 2017
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

4 participants