Skip to content

Commit d01122d

Browse files
gatorsmilesrowen
authored andcommitted
[SPARK-20156][SQL][FOLLOW-UP] Java String toLowerCase "Turkish locale bug" in Database and Table DDLs
### What changes were proposed in this pull request? Database and Table names conform the Hive standard ("[a-zA-z_0-9]+"), i.e. if this name only contains characters, numbers, and _. When calling `toLowerCase` on the names, we should add `Locale.ROOT` to the `toLowerCase`for avoiding inadvertent locale-sensitive variation in behavior (aka the "Turkish locale problem"). ### How was this patch tested? Added a test case Author: Xiao Li <[email protected]> Closes #17655 from gatorsmile/locale. (cherry picked from commit 55bea56) Signed-off-by: Sean Owen <[email protected]>
1 parent 8d658b9 commit d01122d

File tree

5 files changed

+54
-5
lines changed

5 files changed

+54
-5
lines changed

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveTableValuedFunctions.scala

Lines changed: 3 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -17,6 +17,8 @@
1717

1818
package org.apache.spark.sql.catalyst.analysis
1919

20+
import java.util.Locale
21+
2022
import org.apache.spark.sql.catalyst.expressions.Expression
2123
import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Range}
2224
import 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 {

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff 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
/**

sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala

Lines changed: 3 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -17,6 +17,8 @@
1717

1818
package org.apache.spark.sql.internal
1919

20+
import java.util.Locale
21+
2022
import scala.reflect.ClassTag
2123
import 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 " +

sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala

Lines changed: 19 additions & 0 deletions
Original file line numberDiff line numberDiff 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
}

sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala

Lines changed: 27 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -20,7 +20,7 @@ package org.apache.spark.sql.test
2020
import java.io.File
2121
import java.net.URI
2222
import java.nio.file.Files
23-
import java.util.UUID
23+
import java.util.{Locale, UUID}
2424

2525
import scala.language.implicitConversions
2626
import 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.

0 commit comments

Comments
 (0)