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 57e2da8353d6..84315f55a59a 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 @@ -307,7 +307,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") 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 96c84eab1c89..568e953a5db6 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,11 +17,13 @@ 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.sources.CreatableRelationProvider +import org.apache.spark.util.Utils /** * Saves the results of `query` in to a data source. @@ -46,4 +48,9 @@ case class SaveIntoDataSourceCommand( Seq.empty[Row] } + + override def simpleString: String = { + val redacted = Utils.redact(SparkEnv.get.conf, options.toSeq).toMap + s"SaveIntoDataSourceCommand ${dataSource}, ${redacted}, ${mode}" + } } 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 000000000000..4b3ca8e60cab --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SaveIntoDataSourceCommandSuite.scala @@ -0,0 +1,48 @@ +/* + * 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 dataSource = DataSource( + sparkSession = spark, + className = "jdbc", + partitionColumns = Nil, + options = Map("password" -> PASS, "url" -> URL, "driver" -> DRIVER)) + + val logicalPlanString = dataSource + .planForWriting(SaveMode.ErrorIfExists, spark.range(1).logicalPlan) + .treeString(true) + + assert(!logicalPlanString.contains(URL)) + assert(!logicalPlanString.contains(PASS)) + assert(logicalPlanString.contains(DRIVER)) + } +}