diff options
author | Tejas Patil <tejasp@fb.com> | 2016-01-28 13:45:28 -0800 |
---|---|---|
committer | Michael Armbrust <michael@databricks.com> | 2016-01-28 13:45:28 -0800 |
commit | 676803963fcc08aa988aa6f14be3751314e006ca (patch) | |
tree | 2152bbed4c49513c49ff4489e82b30ebff812593 /sql | |
parent | 415d0a859b7a76f3a866ec62ab472c4050f2a01b (diff) | |
download | spark-676803963fcc08aa988aa6f14be3751314e006ca.tar.gz spark-676803963fcc08aa988aa6f14be3751314e006ca.tar.bz2 spark-676803963fcc08aa988aa6f14be3751314e006ca.zip |
[SPARK-12926][SQL] SQLContext to display warning message when non-sql configs are being set
Users unknowingly try to set core Spark configs in SQLContext but later realise that it didn't work. eg. sqlContext.sql("SET spark.shuffle.memoryFraction=0.4"). This PR adds a warning message when such operations are done.
Author: Tejas Patil <tejasp@fb.com>
Closes #10849 from tejasapatil/SPARK-12926.
Diffstat (limited to 'sql')
-rw-r--r-- | sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala | 14 |
1 files changed, 11 insertions, 3 deletions
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala index c9ba670099..eb9da0bd4f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala @@ -24,6 +24,7 @@ import scala.collection.JavaConverters._ import org.apache.parquet.hadoop.ParquetOutputCommitter +import org.apache.spark.Logging import org.apache.spark.sql.catalyst.CatalystConf import org.apache.spark.sql.catalyst.parser.ParserConf import org.apache.spark.util.Utils @@ -519,7 +520,7 @@ private[spark] object SQLConf { * * SQLConf is thread-safe (internally synchronized, so safe to be used in multiple threads). */ -private[sql] class SQLConf extends Serializable with CatalystConf with ParserConf { +private[sql] class SQLConf extends Serializable with CatalystConf with ParserConf with Logging { import SQLConf._ /** Only low degree of contention is expected for conf, thus NOT using ConcurrentHashMap. */ @@ -628,7 +629,7 @@ private[sql] class SQLConf extends Serializable with CatalystConf with ParserCon // Only verify configs in the SQLConf object entry.valueConverter(value) } - settings.put(key, value) + setConfWithCheck(key, value) } /** Set the given Spark SQL configuration property. */ @@ -636,7 +637,7 @@ private[sql] class SQLConf extends Serializable with CatalystConf with ParserCon require(entry != null, "entry cannot be null") require(value != null, s"value cannot be null for key: ${entry.key}") require(sqlConfEntries.get(entry.key) == entry, s"$entry is not registered") - settings.put(entry.key, entry.stringConverter(value)) + setConfWithCheck(entry.key, entry.stringConverter(value)) } /** Return the value of Spark SQL configuration property for the given key. */ @@ -699,6 +700,13 @@ private[sql] class SQLConf extends Serializable with CatalystConf with ParserCon }.toSeq } + private def setConfWithCheck(key: String, value: String): Unit = { + if (key.startsWith("spark.") && !key.startsWith("spark.sql.")) { + logWarning(s"Attempt to set non-Spark SQL config in SQLConf: key = $key, value = $value") + } + settings.put(key, value) + } + private[spark] def unsetConf(key: String): Unit = { settings.remove(key) } |