File tree Expand file tree Collapse file tree 5 files changed +54
-5
lines changed
catalyst/src/main/scala/org/apache/spark/sql/catalyst
main/scala/org/apache/spark/sql/internal
test/scala/org/apache/spark/sql Expand file tree Collapse file tree 5 files changed +54
-5
lines changed Original file line number Diff line number Diff line change 1717
1818package org .apache .spark .sql .catalyst .analysis
1919
20+ import java .util .Locale
21+
2022import org .apache .spark .sql .catalyst .expressions .Expression
2123import org .apache .spark .sql .catalyst .plans .logical .{LogicalPlan , Range }
2224import org .apache .spark .sql .catalyst .rules ._
@@ -103,7 +105,7 @@ object ResolveTableValuedFunctions extends Rule[LogicalPlan] {
103105
104106 override def apply (plan : LogicalPlan ): LogicalPlan = plan resolveOperators {
105107 case u : UnresolvedTableValuedFunction if u.functionArgs.forall(_.resolved) =>
106- builtinFunctions.get(u.functionName.toLowerCase()) match {
108+ builtinFunctions.get(u.functionName.toLowerCase(Locale . ROOT )) match {
107109 case Some (tvf) =>
108110 val resolved = tvf.flatMap { case (argList, resolver) =>
109111 argList.implicitCast(u.functionArgs) match {
Original file line number Diff line number Diff line change @@ -115,14 +115,14 @@ class SessionCatalog(
115115 * Format table name, taking into account case sensitivity.
116116 */
117117 protected [this ] def formatTableName (name : String ): String = {
118- if (conf.caseSensitiveAnalysis) name else name.toLowerCase
118+ if (conf.caseSensitiveAnalysis) name else name.toLowerCase( Locale . ROOT )
119119 }
120120
121121 /**
122122 * Format database name, taking into account case sensitivity.
123123 */
124124 protected [this ] def formatDatabaseName (name : String ): String = {
125- if (conf.caseSensitiveAnalysis) name else name.toLowerCase
125+ if (conf.caseSensitiveAnalysis) name else name.toLowerCase( Locale . ROOT )
126126 }
127127
128128 /**
Original file line number Diff line number Diff line change 1717
1818package org .apache .spark .sql .internal
1919
20+ import java .util .Locale
21+
2022import scala .reflect .ClassTag
2123import scala .util .control .NonFatal
2224
@@ -114,7 +116,7 @@ private[sql] class SharedState(val sparkContext: SparkContext) extends Logging {
114116 // System preserved database should not exists in metastore. However it's hard to guarantee it
115117 // for every session, because case-sensitivity differs. Here we always lowercase it to make our
116118 // life easier.
117- val globalTempDB = sparkContext.conf.get(GLOBAL_TEMP_DATABASE ).toLowerCase
119+ val globalTempDB = sparkContext.conf.get(GLOBAL_TEMP_DATABASE ).toLowerCase( Locale . ROOT )
118120 if (externalCatalog.databaseExists(globalTempDB)) {
119121 throw new SparkException (
120122 s " $globalTempDB is a system preserved database, please rename your existing database " +
Original file line number Diff line number Diff line change @@ -2295,5 +2295,24 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils {
22952295 }
22962296 }
22972297 }
2298+
2299+ test(s " basic DDL using locale tr - caseSensitive $caseSensitive" ) {
2300+ withSQLConf(SQLConf .CASE_SENSITIVE .key -> s " $caseSensitive" ) {
2301+ withLocale(" tr" ) {
2302+ val dbName = " DaTaBaSe_I"
2303+ withDatabase(dbName) {
2304+ sql(s " CREATE DATABASE $dbName" )
2305+ sql(s " USE $dbName" )
2306+
2307+ val tabName = " tAb_I"
2308+ withTable(tabName) {
2309+ sql(s " CREATE TABLE $tabName(col_I int) USING PARQUET " )
2310+ sql(s " INSERT OVERWRITE TABLE $tabName SELECT 1 " )
2311+ checkAnswer(sql(s " SELECT col_I FROM $tabName" ), Row (1 ) :: Nil )
2312+ }
2313+ }
2314+ }
2315+ }
2316+ }
22982317 }
22992318}
Original file line number Diff line number Diff line change @@ -20,7 +20,7 @@ package org.apache.spark.sql.test
2020import java .io .File
2121import java .net .URI
2222import java .nio .file .Files
23- import java .util .UUID
23+ import java .util .{ Locale , UUID }
2424
2525import scala .language .implicitConversions
2626import scala .util .control .NonFatal
@@ -228,6 +228,32 @@ private[sql] trait SQLTestUtils
228228 }
229229 }
230230
231+ /**
232+ * Drops database `dbName` after calling `f`.
233+ */
234+ protected def withDatabase (dbNames : String * )(f : => Unit ): Unit = {
235+ try f finally {
236+ dbNames.foreach { name =>
237+ spark.sql(s " DROP DATABASE IF EXISTS $name" )
238+ }
239+ }
240+ }
241+
242+ /**
243+ * Enables Locale `language` before executing `f`, then switches back to the default locale of JVM
244+ * after `f` returns.
245+ */
246+ protected def withLocale (language : String )(f : => Unit ): Unit = {
247+ val originalLocale = Locale .getDefault
248+ try {
249+ // Add Locale setting
250+ Locale .setDefault(new Locale (language))
251+ f
252+ } finally {
253+ Locale .setDefault(originalLocale)
254+ }
255+ }
256+
231257 /**
232258 * Activates database `db` before executing `f`, then switches back to `default` database after
233259 * `f` returns.
You can’t perform that action at this time.
0 commit comments