Repository: spark Updated Branches: refs/heads/branch-2.2 3ae187b96 -> b17ba3551
[SPARK-22479][SQL][BRANCH-2.2] Exclude credentials from SaveintoDataSourceCommand.simpleString ## 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? new tests Author: osatici <[email protected]> Closes #19761 from onursatici/os/redact-jdbc-creds-2.2. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/b17ba355 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/b17ba355 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/b17ba355 Branch: refs/heads/branch-2.2 Commit: b17ba3551230f34559e8faf1cb59a73edd8505c6 Parents: 3ae187b Author: osatici <[email protected]> Authored: Thu Nov 16 12:49:29 2017 +0100 Committer: Wenchen Fan <[email protected]> Committed: Thu Nov 16 12:49:29 2017 +0100 ---------------------------------------------------------------------- .../apache/spark/internal/config/package.scala | 2 +- .../datasources/SaveIntoDataSourceCommand.scala | 7 +++ .../SaveIntoDataSourceCommandSuite.scala | 45 ++++++++++++++++++++ 3 files changed, 53 insertions(+), 1 deletion(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/spark/blob/b17ba355/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 1588dfe..da3cc04 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 @@ -247,7 +247,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".r) + .createWithDefault("(?i)secret|password|url|user|username".r) private[spark] val STRING_REDACTION_PATTERN = ConfigBuilder("spark.redaction.string.regex") http://git-wip-us.apache.org/repos/asf/spark/blob/b17ba355/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 6f19ea1..53868d4 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,10 +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.util.Utils /** * Saves the results of `query` in to a data source. @@ -49,4 +51,9 @@ case class SaveIntoDataSourceCommand( Seq.empty[Row] } + + override def simpleString: String = { + val redacted = Utils.redact(SparkEnv.get.conf, options.toSeq).toMap + s"SaveIntoDataSourceCommand ${provider}, ${partitionColumns}, ${redacted}, ${mode}" + } } http://git-wip-us.apache.org/repos/asf/spark/blob/b17ba355/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 new file mode 100644 index 0000000..6b9ddb1 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SaveIntoDataSourceCommandSuite.scala @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.datasources + +import org.apache.spark.SparkConf +import org.apache.spark.sql.SaveMode +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" + val DRIVER = "mydriver" + + val simpleString = SaveIntoDataSourceCommand( + spark.range(1).logicalPlan, + "jdbc", + Nil, + Map("password" -> PASS, "url" -> URL, "driver" -> DRIVER), + SaveMode.ErrorIfExists).treeString(true) + + assert(!simpleString.contains(URL)) + assert(!simpleString.contains(PASS)) + assert(simpleString.contains(DRIVER)) + } +} --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
