Skip to content

Conversation

@onursatici
Copy link
Contributor

@onursatici onursatici commented Nov 9, 2017

What changes were proposed in this pull request?

Do not include jdbc properties which may contain credentials in logging a logical plan with SaveIntoDataSourceCommand in it.

How was this patch tested?

building locally and trying to reproduce (per the steps in https://issues.apache.org/jira/browse/SPARK-22479):

== Parsed Logical Plan ==
SaveIntoDataSourceCommand org.apache.spark.sql.execution.datasources.jdbc.JdbcRelationProvider@570127fa, Map(dbtable -> test20, driver -> org.postgresql.Driver, url -> *********(redacted), password -> *********(redacted)), ErrorIfExists
   +- Range (0, 100, step=1, splits=Some(8))

== Analyzed Logical Plan ==
SaveIntoDataSourceCommand org.apache.spark.sql.execution.datasources.jdbc.JdbcRelationProvider@570127fa, Map(dbtable -> test20, driver -> org.postgresql.Driver, url -> *********(redacted), password -> *********(redacted)), ErrorIfExists
   +- Range (0, 100, step=1, splits=Some(8))

== Optimized Logical Plan ==
SaveIntoDataSourceCommand org.apache.spark.sql.execution.datasources.jdbc.JdbcRelationProvider@570127fa, Map(dbtable -> test20, driver -> org.postgresql.Driver, url -> *********(redacted), password -> *********(redacted)), ErrorIfExists
   +- Range (0, 100, step=1, splits=Some(8))

== Physical Plan ==
Execute SaveIntoDataSourceCommand
   +- SaveIntoDataSourceCommand org.apache.spark.sql.execution.datasources.jdbc.JdbcRelationProvider@570127fa, Map(dbtable -> test20, driver -> org.postgresql.Driver, url -> *********(redacted), password -> *********(redacted)), ErrorIfExists
         +- Range (0, 100, step=1, splits=Some(8))

@ash211
Copy link
Contributor

ash211 commented Nov 9, 2017

Jenkins, this is ok to test

@SparkQA
Copy link

SparkQA commented Nov 9, 2017

Test build #83652 has finished for PR 19708 at commit 04aa9f0.

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

Seq.empty[Row]
}

override def simpleString: String = s"SaveIntoDataSourceCommand ${dataSource}, ${mode}"
Copy link
Member

Choose a reason for hiding this comment

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

Copy link
Contributor Author

Choose a reason for hiding this comment

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

I can use that, but I would need to expand the default for spark.redaction.regex as the user and url (some drivers allow credentials passed in the connection url) fields might also contain sensitive data. We should also change JDBCRelation::toString to include the redaction regex to be consistent.

I would argue not showing the jdbc properties at all because they provide little value and wrong redaction regex configuration could cause leaks to downstream log collection systems.

let me know if that makes sense, and I can modify this accordingly

Copy link
Member

Choose a reason for hiding this comment

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

SaveIntoDataSourceCommand is not being used for JDBC only.

JDBCRelation::toString was already fixed in #15975 ?

@SparkQA
Copy link

SparkQA commented Nov 13, 2017

Test build #83799 has finished for PR 19708 at commit a0d15df.

  • This patch fails to build.
  • This patch merges cleanly.
  • This patch adds no public classes.

@onursatici onursatici changed the title [SPARK-22479][SQL] Exclude credentials from SaveintoDataSourceCommand.simpleString [WIP][SPARK-22479][SQL] Exclude credentials from SaveintoDataSourceCommand.simpleString Nov 13, 2017
@SparkQA
Copy link

SparkQA commented Nov 13, 2017

Test build #83810 has finished for PR 19708 at commit ae091ec.

  • This patch fails Scala style tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@gatorsmile
Copy link
Member

Just want to confirm whether the examples in the PR description are the ones based on the latest updates?

@gatorsmile
Copy link
Member

Generally, this looks good to me.

cc @cloud-fan @hvanhovell

@hvanhovell
Copy link
Contributor

This looks good. I was wondering if we shouldn't also take a look at data source operations like InsertIntoDataSourceCommand?

Could you also add a test?

@SparkQA
Copy link

SparkQA commented Nov 13, 2017

Test build #83811 has finished for PR 19708 at commit 56f48f3.

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

@onursatici
Copy link
Contributor Author

onursatici commented Nov 14, 2017

Yeah PR description reflects the latest changes.
InsertIntoDataSourceCommand could log credentials via the JDBCRelation it contains, but that class already properly redacts

@onursatici onursatici changed the title [WIP][SPARK-22479][SQL] Exclude credentials from SaveintoDataSourceCommand.simpleString [SPARK-22479][SQL] Exclude credentials from SaveintoDataSourceCommand.simpleString Nov 14, 2017
@SparkQA
Copy link

SparkQA commented Nov 15, 2017

Test build #83864 has finished for PR 19708 at commit db565f6.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds the following public classes (experimental):
  • class SaveIntoDataSourceCommandSuite extends SharedSQLContext

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

test("treeString is redacted") {
Copy link
Contributor

Choose a reason for hiding this comment

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

old test name? we're not modifying the treeString anymore, it's just the SaveIntoDataSourceCommand

Copy link
Contributor Author

Choose a reason for hiding this comment

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

I followed the naming convention here:

we are essentially redacting SaveIntoDataSourceCommand::simpleString which is called in SaceIntoDataSourceCommand::treeString

Copy link
Contributor

Choose a reason for hiding this comment

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

That is not really a convention. Can you just call it simpleString is redacted?

class SaveIntoDataSourceCommandSuite extends SharedSQLContext {

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

Choose a reason for hiding this comment

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

Shoundn't this be spark.redaction.regex instead of spark.redaction.string.regex?

Copy link
Contributor

@jiangxb1987 jiangxb1987 left a comment

Choose a reason for hiding this comment

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

LGTM pending jenkins

class SaveIntoDataSourceCommandSuite extends SharedSQLContext {

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

Choose a reason for hiding this comment

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

Nit: indents.

@SparkQA
Copy link

SparkQA commented Nov 15, 2017

Test build #83895 has finished for PR 19708 at commit 2a1204b.

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

@gatorsmile
Copy link
Member

LGTM

@SparkQA
Copy link

SparkQA commented Nov 15, 2017

Test build #83896 has finished for PR 19708 at commit 2983bba.

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

Copy link
Contributor

@ash211 ash211 left a comment

Choose a reason for hiding this comment

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

LGTM

@SparkQA
Copy link

SparkQA commented Nov 15, 2017

Test build #83901 has finished for PR 19708 at commit 2942477.

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

@asfgit asfgit closed this in 2014e7a Nov 15, 2017
@gatorsmile
Copy link
Member

Thanks! Merged to master.

@onursatici Could you submit a separate PR for 2.2?

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.

6 participants