Skip to content
Closed
Show file tree
Hide file tree
Changes from 4 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -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")
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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.
Expand All @@ -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}"
}
}
Original file line number Diff line number Diff line change
@@ -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.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?


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?

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))
}
}