diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala index e49e54f9bb31..dcc62989d9d2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala @@ -39,6 +39,7 @@ import org.apache.spark.sql.catalyst.parser.{CatalystSqlParser, ParserInterface} import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SubqueryAlias, View} import org.apache.spark.sql.catalyst.util.StringUtils import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.internal.StaticSQLConf.GLOBAL_TEMP_DATABASE import org.apache.spark.sql.types.StructType import org.apache.spark.util.Utils @@ -71,7 +72,7 @@ class SessionCatalog( conf: SQLConf) { this( () => externalCatalog, - () => new GlobalTempViewManager("global_temp"), + () => new GlobalTempViewManager(conf.getConf(GLOBAL_TEMP_DATABASE)), functionRegistry, conf, new Configuration(), diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/StaticSQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/StaticSQLConf.scala index 3fda4c806a0b..d665d16ae419 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/StaticSQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/StaticSQLConf.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.internal +import java.util.Locale + import org.apache.spark.util.Utils @@ -42,6 +44,7 @@ object StaticSQLConf { val GLOBAL_TEMP_DATABASE = buildStaticConf("spark.sql.globalTempDatabase") .internal() .stringConf + .transform(_.toLowerCase(Locale.ROOT)) .createWithDefault("global_temp") // This is used to control when we will split a schema's JSON string to multiple pieces diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala index e6c40bd93174..f1a648176c3b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala @@ -158,7 +158,7 @@ private[sql] class SharedState( // System preserved database should not exists in metastore. However it's hard to guarantee it // for every session, because case-sensitivity differs. Here we always lowercase it to make our // life easier. - val globalTempDB = conf.get(GLOBAL_TEMP_DATABASE).toLowerCase(Locale.ROOT) + val globalTempDB = conf.get(GLOBAL_TEMP_DATABASE) if (externalCatalog.databaseExists(globalTempDB)) { throw new SparkException( s"$globalTempDB is a system preserved database, please rename your existing database " + diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala index cc1ead931bf7..acfb84ede7ad 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala @@ -39,6 +39,7 @@ import org.apache.spark.sql.functions._ import org.apache.spark.sql.hive.{HiveExternalCatalog, HiveUtils} import org.apache.spark.sql.hive.test.{HiveTestUtils, TestHiveSingleton} import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.internal.StaticSQLConf.GLOBAL_TEMP_DATABASE import org.apache.spark.sql.test.SQLTestUtils import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.CalendarInterval @@ -73,13 +74,13 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { test("query global temp view") { val df = Seq(1).toDF("i1") df.createGlobalTempView("tbl1") - val global_temp_db = spark.conf.get("spark.sql.globalTempDatabase") + val global_temp_db = spark.conf.get(GLOBAL_TEMP_DATABASE) checkAnswer(spark.sql(s"select * from ${global_temp_db}.tbl1"), Row(1)) spark.sql(s"drop view ${global_temp_db}.tbl1") } test("non-existent global temp view") { - val global_temp_db = spark.conf.get("spark.sql.globalTempDatabase") + val global_temp_db = spark.conf.get(GLOBAL_TEMP_DATABASE) val message = intercept[AnalysisException] { spark.sql(s"select * from ${global_temp_db}.nonexistentview") }.getMessage