This is an automated email from the ASF dual-hosted git repository. gurwls223 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git
The following commit(s) were added to refs/heads/master by this push: new 472629a [SPARK-37727][SQL] Show ignored confs & hide warnings for conf already set in SparkSession.builder.getOrCreate 472629a is described below commit 472629afcc0518ad5b9f5042bf4d9a791822d378 Author: Hyukjin Kwon <gurwls...@apache.org> AuthorDate: Fri Dec 24 12:51:45 2021 +0900 [SPARK-37727][SQL] Show ignored confs & hide warnings for conf already set in SparkSession.builder.getOrCreate ### What changes were proposed in this pull request? This PR proposes to show ignored configurations and hide the warnings for configurations that are already set when invoking `SparkSession.builder.getOrCreate`. ### Why are the changes needed? Currently, `SparkSession.builder.getOrCreate()` is too noisy even when duplicate configurations are set. Users cannot easily tell which configurations are to fix. See the example below: ```bash ./bin/spark-shell --conf spark.abc=abc ``` ```scala import org.apache.spark.sql.SparkSession spark.sparkContext.setLogLevel("DEBUG") SparkSession.builder.config("spark.abc", "abc").getOrCreate ``` ``` 21:04:01.670 [main] WARN org.apache.spark.sql.SparkSession - Using an existing SparkSession; some spark core configurations may not take effect. ``` It is straitforward when there are few configurations but it is difficult for users to figure out when there are too many configurations especially when these configurations are defined in a property file like 'spark-default.conf' maintained separately by system admins in production. See also https://github.com/apache/spark/pull/34757#discussion_r769248275. ### Does this PR introduce _any_ user-facing change? Yes. 1. Show ignored configurations in debug level logs: ```bash ./bin/spark-shell --conf spark.abc=abc ``` ```scala import org.apache.spark.sql.SparkSession spark.sparkContext.setLogLevel("DEBUG") SparkSession.builder .config("spark.sql.warehouse.dir", "2") .config("spark.abc", "abcb") .config("spark.abcd", "abcb4") .getOrCreate ``` **Before:** ``` 21:13:28.360 [main] WARN org.apache.spark.sql.SparkSession - Using an existing SparkSession; the static sql configurations will not take effect. 21:13:28.360 [main] WARN org.apache.spark.sql.SparkSession - Using an existing SparkSession; some spark core configurations may not take effect. ``` **After**: ``` 20:34:30.619 [main] WARN org.apache.spark.sql.SparkSession - Using an existing Spark session; only runtime SQL configurations will take effect. 20:34:30.622 [main] DEBUG org.apache.spark.sql.SparkSession - Ignored static SQL configurations: spark.sql.warehouse.dir=2 20:34:30.623 [main] DEBUG org.apache.spark.sql.SparkSession - Configurations that might not take effect: spark.abcd=abcb4 spark.abc=abcb ``` 2. Do not issue a warning and hide a configuration already explicitly set (with the same value) before. ```bash ./bin/spark-shell --conf spark.abc=abc ``` ```scala import org.apache.spark.sql.SparkSession spark.sparkContext.setLogLevel("DEBUG") SparkSession.builder.config("spark.abc", "abc").getOrCreate // **Ignore** warnings because it's already set in --conf SparkSession.builder.config("spark.abc.new", "abc").getOrCreate // **Show** warnings for only configuration newly set. SparkSession.builder.config("spark.abc.new", "abc").getOrCreate // **Ignore** warnings because it's set ^. ``` **Before**: ``` 21:13:56.183 [main] WARN org.apache.spark.sql.SparkSession - Using an existing SparkSession; some spark core configurations may not take effect. 21:13:56.356 [main] WARN org.apache.spark.sql.SparkSession - Using an existing SparkSession; some spark core configurations may not take effect. 21:13:56.476 [main] WARN org.apache.spark.sql.SparkSession - Using an existing SparkSession; some spark core configurations may not take effect. ``` **After:** ``` 20:36:36.251 [main] WARN org.apache.spark.sql.SparkSession - Using an existing Spark session; only runtime SQL configurations will take effect. 20:36:36.253 [main] DEBUG org.apache.spark.sql.SparkSession - Configurations that might not take effect: spark.abc.new=abc ``` 3. Do not issue a warning and hide runtime SQL configurations in debug log: ```bash ./bin/spark-shell ``` ```scala import org.apache.spark.sql.SparkSession spark.sparkContext.setLogLevel("DEBUG") SparkSession.builder.config("spark.sql.ansi.enabled", "true").getOrCreate // **Ignore** warnings for runtime SQL configurations SparkSession.builder.config("spark.buffer.size", "1234").getOrCreate // **Show** warnings for Spark core configuration SparkSession.builder.config("spark.sql.source.specific", "abc").getOrCreate // **Show** warnings for custom runtime options SparkSession.builder.config("spark.sql.warehouse.dir", "xyz").getOrCreate // **Show** warnings for static SQL configurations ``` **Before**: ``` 11:11:40.846 [main] WARN org.apache.spark.sql.SparkSession - Using an existing SparkSession; some spark core configurations may not take effect. 11:11:41.037 [main] WARN org.apache.spark.sql.SparkSession - Using an existing SparkSession; some spark core configurations may not take effect. 11:11:41.167 [main] WARN org.apache.spark.sql.SparkSession - Using an existing SparkSession; some spark core configurations may not take effect. 11:11:41.318 [main] WARN org.apache.spark.sql.SparkSession - Using an existing SparkSession; the static sql configurations will not take effect. ``` **After**: ``` 10:39:54.870 [main] WARN org.apache.spark.sql.SparkSession - Using an existing Spark session; only runtime SQL configurations will take effect. 10:39:54.872 [main] DEBUG org.apache.spark.sql.SparkSession - Configurations that might not take effect: spark.buffer.size=1234 10:39:54.988 [main] WARN org.apache.spark.sql.SparkSession - Using an existing Spark session; only runtime SQL configurations will take effect. 10:39:54.988 [main] DEBUG org.apache.spark.sql.SparkSession - Configurations that might not take effect: spark.sql.source.specific=abc 10:39:55.107 [main] WARN org.apache.spark.sql.SparkSession - Using an existing Spark session; only runtime SQL configurations will take effect. 10:39:55.108 [main] DEBUG org.apache.spark.sql.SparkSession - Ignored static SQL configurations: spark.sql.warehouse.dir=xyz ``` Note that there is no behaviour change on session state initialization when configurations are not set. For example: ```scala import org.apache.spark.sql.SparkSession spark.sparkContext.setLogLevel("DEBUG") SparkSession.builder.getOrCreate ``` But the session state initialization can be triggered now for static SQL configurations set after this PR. Previously, it was not triggered. This would not introduce something user-facing or a bug but worth noting it. For runtime SQL configurations, the session state initialization in this code path was introduced at https://github.com/apache/spark/pull/15295. ### How was this patch tested? It was manually tested as shown above. Closes #35001 from HyukjinKwon/SPARK-37727. Authored-by: Hyukjin Kwon <gurwls...@apache.org> Signed-off-by: Hyukjin Kwon <gurwls...@apache.org> --- .../scala/org/apache/spark/sql/SparkSession.scala | 31 +++++++++++++++++----- 1 file changed, 24 insertions(+), 7 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala index cd101e5..cdd57d7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -1065,18 +1065,35 @@ object SparkSession extends Logging { private[sql] def applyModifiableSettings( session: SparkSession, options: java.util.HashMap[String, String]): Unit = { + // Lazy val to avoid an unnecessary session state initialization + lazy val conf = session.sessionState.conf + + val dedupOptions = if (options.isEmpty) Map.empty[String, String] else ( + options.asScala.toSet -- conf.getAllConfs.toSet).toMap val (staticConfs, otherConfs) = - options.asScala.partition(kv => SQLConf.isStaticConfigKey(kv._1)) + dedupOptions.partition(kv => SQLConf.isStaticConfigKey(kv._1)) + + otherConfs.foreach { case (k, v) => conf.setConfString(k, v) } - otherConfs.foreach { case (k, v) => session.sessionState.conf.setConfString(k, v) } + // Note that other runtime SQL options, for example, for other third-party datasource + // can be marked as an ignored configuration here. + val maybeIgnoredConfs = otherConfs.filterNot { case (k, _) => conf.isModifiable(k) } + if (staticConfs.nonEmpty || maybeIgnoredConfs.nonEmpty) { + logWarning( + "Using an existing Spark session; only runtime SQL configurations will take effect.") + } if (staticConfs.nonEmpty) { - logWarning("Using an existing SparkSession; the static sql configurations will not take" + - " effect.") + logDebug("Ignored static SQL configurations:\n " + + conf.redactOptions(staticConfs).toSeq.map { case (k, v) => s"$k=$v" }.mkString("\n ")) } - if (otherConfs.nonEmpty) { - logWarning("Using an existing SparkSession; some spark core configurations may not take" + - " effect.") + if (maybeIgnoredConfs.nonEmpty) { + // Only print out non-static and non-runtime SQL configurations. + // Note that this might show core configurations or source specific + // options defined in the third-party datasource. + logDebug("Configurations that might not take effect:\n " + + conf.redactOptions( + maybeIgnoredConfs).toSeq.map { case (k, v) => s"$k=$v" }.mkString("\n ")) } } --------------------------------------------------------------------- To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org