From 04aa9f0363f6202a5358e41587415da4fa5f425e Mon Sep 17 00:00:00 2001 From: osatici Date: Thu, 9 Nov 2017 14:06:05 +0000 Subject: [PATCH 1/7] do not log properties on SaveintoDataSourceCommand.simpleString --- .../sql/execution/datasources/SaveIntoDataSourceCommand.scala | 2 ++ 1 file changed, 2 insertions(+) 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..64b4d6653e3c 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 @@ -46,4 +46,6 @@ case class SaveIntoDataSourceCommand( Seq.empty[Row] } + + override def simpleString: String = s"SaveIntoDataSourceCommand ${dataSource}, ${mode}" } From a0d15df9e02c6185cc7beb486bee1442964c1048 Mon Sep 17 00:00:00 2001 From: osatici Date: Mon, 13 Nov 2017 18:14:47 +0000 Subject: [PATCH 2/7] redact all maps in treenode --- .../main/scala/org/apache/spark/internal/config/package.scala | 2 +- .../scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala | 2 ++ .../sql/execution/datasources/SaveIntoDataSourceCommand.scala | 2 -- 3 files changed, 3 insertions(+), 3 deletions(-) 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/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala index 9c7d47f99ee1..8378e38ecb3d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala @@ -450,6 +450,8 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { case set: Set[_] => Utils.truncatedString(set.toSeq, "{", ", ", "}") :: Nil case array: Array[_] if array.isEmpty => Nil case array: Array[_] => Utils.truncatedString(array, "[", ", ", "]") :: Nil + case map: Map[String, String] if map.isEmpty => Nil + case map: Map[String, String] => Utils.redact(map).toMap :: Nil case null => Nil case None => Nil case Some(null) => Nil 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 64b4d6653e3c..96c84eab1c89 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 @@ -46,6 +46,4 @@ case class SaveIntoDataSourceCommand( Seq.empty[Row] } - - override def simpleString: String = s"SaveIntoDataSourceCommand ${dataSource}, ${mode}" } From 56f48f3aa0c33ddeee86ce330b54b6d3f712ffae Mon Sep 17 00:00:00 2001 From: osatici Date: Mon, 13 Nov 2017 19:57:39 +0000 Subject: [PATCH 3/7] redact options for SaveIntoDataSourceCommand --- .../org/apache/spark/sql/catalyst/trees/TreeNode.scala | 2 -- .../execution/datasources/SaveIntoDataSourceCommand.scala | 7 +++++++ 2 files changed, 7 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala index 8378e38ecb3d..9c7d47f99ee1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala @@ -450,8 +450,6 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { case set: Set[_] => Utils.truncatedString(set.toSeq, "{", ", ", "}") :: Nil case array: Array[_] if array.isEmpty => Nil case array: Array[_] => Utils.truncatedString(array, "[", ", ", "]") :: Nil - case map: Map[String, String] if map.isEmpty => Nil - case map: Map[String, String] => Utils.redact(map).toMap :: Nil case null => Nil case None => Nil case Some(null) => Nil 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}" + } } From db565f6b0a57f436f85c32fe7d05b027908c7a9b Mon Sep 17 00:00:00 2001 From: osatici Date: Tue, 14 Nov 2017 21:24:28 +0000 Subject: [PATCH 4/7] test redaction for saveintodatasourcecommand --- .../SaveIntoDataSourceCommandSuite.scala | 48 +++++++++++++++++++ 1 file changed, 48 insertions(+) create mode 100644 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 000000000000..21a4fafe6b3a --- /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.string.regex", "(?i)password|url") + + test("treeString 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)) + } +} From 2a1204b3ec7f255d4c73f314f3f96da389fedecb Mon Sep 17 00:00:00 2001 From: osatici Date: Wed, 15 Nov 2017 13:54:32 +0000 Subject: [PATCH 5/7] change correct redact conf key --- .../execution/datasources/SaveIntoDataSourceCommandSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 index 21a4fafe6b3a..4520054466e6 100644 --- 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 @@ -24,7 +24,7 @@ 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") + .set("spark.redaction.regex", "(?i)password|url") test("treeString is redacted") { val URL = "connection.url" From 2983bba7d228dd5387b0088ab54a7a59e71e92de Mon Sep 17 00:00:00 2001 From: osatici Date: Wed, 15 Nov 2017 14:21:53 +0000 Subject: [PATCH 6/7] rename test --- .../execution/datasources/SaveIntoDataSourceCommandSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 index 4520054466e6..13fb9ef8b233 100644 --- 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 @@ -26,7 +26,7 @@ class SaveIntoDataSourceCommandSuite extends SharedSQLContext { override protected def sparkConf: SparkConf = super.sparkConf .set("spark.redaction.regex", "(?i)password|url") - test("treeString is redacted") { + test("simpleString is redacted") { val URL = "connection.url" val PASS = "123" val DRIVER = "mydriver" From 294247757538a3f11815dfe130a92a8ae30c1db4 Mon Sep 17 00:00:00 2001 From: osatici Date: Wed, 15 Nov 2017 17:21:47 +0000 Subject: [PATCH 7/7] indent --- .../execution/datasources/SaveIntoDataSourceCommandSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 index 13fb9ef8b233..4b3ca8e60cab 100644 --- 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 @@ -24,7 +24,7 @@ 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") + .set("spark.redaction.regex", "(?i)password|url") test("simpleString is redacted") { val URL = "connection.url"