Skip to content
This repository was archived by the owner on Nov 15, 2024. It is now read-only.

Commit b2fe2e1

Browse files
onursaticiMatthewRBruce
authored andcommitted
[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 apache#19761 from onursatici/os/redact-jdbc-creds-2.2.
1 parent 8865e18 commit b2fe2e1

3 files changed

Lines changed: 53 additions & 1 deletion

File tree

core/src/main/scala/org/apache/spark/internal/config/package.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -247,7 +247,7 @@ package object config {
247247
"a property key or value, the value is redacted from the environment UI and various logs " +
248248
"like YARN and event logs.")
249249
.regexConf
250-
.createWithDefault("(?i)secret|password".r)
250+
.createWithDefault("(?i)secret|password|url|user|username".r)
251251

252252
private[spark] val STRING_REDACTION_PATTERN =
253253
ConfigBuilder("spark.redaction.string.regex")

sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/SaveIntoDataSourceCommand.scala

Lines changed: 7 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -17,10 +17,12 @@
1717

1818
package org.apache.spark.sql.execution.datasources
1919

20+
import org.apache.spark.SparkEnv
2021
import org.apache.spark.sql.{Dataset, Row, SaveMode, SparkSession}
2122
import org.apache.spark.sql.catalyst.plans.QueryPlan
2223
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
2324
import org.apache.spark.sql.execution.command.RunnableCommand
25+
import org.apache.spark.util.Utils
2426

2527
/**
2628
* Saves the results of `query` in to a data source.
@@ -49,4 +51,9 @@ case class SaveIntoDataSourceCommand(
4951

5052
Seq.empty[Row]
5153
}
54+
55+
override def simpleString: String = {
56+
val redacted = Utils.redact(SparkEnv.get.conf, options.toSeq).toMap
57+
s"SaveIntoDataSourceCommand ${provider}, ${partitionColumns}, ${redacted}, ${mode}"
58+
}
5259
}
Lines changed: 45 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,45 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one or more
3+
* contributor license agreements. See the NOTICE file distributed with
4+
* this work for additional information regarding copyright ownership.
5+
* The ASF licenses this file to You under the Apache License, Version 2.0
6+
* (the "License"); you may not use this file except in compliance with
7+
* the License. You may obtain a copy of the License at
8+
*
9+
* http://www.apache.org/licenses/LICENSE-2.0
10+
*
11+
* Unless required by applicable law or agreed to in writing, software
12+
* distributed under the License is distributed on an "AS IS" BASIS,
13+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14+
* See the License for the specific language governing permissions and
15+
* limitations under the License.
16+
*/
17+
18+
package org.apache.spark.sql.execution.datasources
19+
20+
import org.apache.spark.SparkConf
21+
import org.apache.spark.sql.SaveMode
22+
import org.apache.spark.sql.test.SharedSQLContext
23+
24+
class SaveIntoDataSourceCommandSuite extends SharedSQLContext {
25+
26+
override protected def sparkConf: SparkConf = super.sparkConf
27+
.set("spark.redaction.regex", "(?i)password|url")
28+
29+
test("simpleString is redacted") {
30+
val URL = "connection.url"
31+
val PASS = "123"
32+
val DRIVER = "mydriver"
33+
34+
val simpleString = SaveIntoDataSourceCommand(
35+
spark.range(1).logicalPlan,
36+
"jdbc",
37+
Nil,
38+
Map("password" -> PASS, "url" -> URL, "driver" -> DRIVER),
39+
SaveMode.ErrorIfExists).treeString(true)
40+
41+
assert(!simpleString.contains(URL))
42+
assert(!simpleString.contains(PASS))
43+
assert(simpleString.contains(DRIVER))
44+
}
45+
}

0 commit comments

Comments
 (0)