Skip to content

Commit

Permalink
[SPARK-20156][SQL][FOLLOW-UP] Java String toLowerCase "Turkish locale…
Browse files Browse the repository at this point in the history
… 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.
  • Loading branch information
gatorsmile authored and srowen committed Apr 20, 2017
1 parent 46c5749 commit 55bea56
Show file tree
Hide file tree
Showing 5 changed files with 54 additions and 5 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,8 @@

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

import java.util.Locale

import org.apache.spark.sql.catalyst.expressions.Expression
import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Range}
import org.apache.spark.sql.catalyst.rules._
Expand Down Expand Up @@ -103,7 +105,7 @@ object ResolveTableValuedFunctions extends Rule[LogicalPlan] {

override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
case u: UnresolvedTableValuedFunction if u.functionArgs.forall(_.resolved) =>
builtinFunctions.get(u.functionName.toLowerCase()) match {
builtinFunctions.get(u.functionName.toLowerCase(Locale.ROOT)) match {
case Some(tvf) =>
val resolved = tvf.flatMap { case (argList, resolver) =>
argList.implicitCast(u.functionArgs) match {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -115,14 +115,14 @@ class SessionCatalog(
* Format table name, taking into account case sensitivity.
*/
protected[this] def formatTableName(name: String): String = {
if (conf.caseSensitiveAnalysis) name else name.toLowerCase
if (conf.caseSensitiveAnalysis) name else name.toLowerCase(Locale.ROOT)
}

/**
* Format database name, taking into account case sensitivity.
*/
protected[this] def formatDatabaseName(name: String): String = {
if (conf.caseSensitiveAnalysis) name else name.toLowerCase
if (conf.caseSensitiveAnalysis) name else name.toLowerCase(Locale.ROOT)
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,8 @@

package org.apache.spark.sql.internal

import java.util.Locale

import scala.reflect.ClassTag
import scala.util.control.NonFatal

Expand Down Expand Up @@ -114,7 +116,7 @@ private[sql] class SharedState(val sparkContext: SparkContext) extends Logging {
// 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 = sparkContext.conf.get(GLOBAL_TEMP_DATABASE).toLowerCase
val globalTempDB = sparkContext.conf.get(GLOBAL_TEMP_DATABASE).toLowerCase(Locale.ROOT)
if (externalCatalog.databaseExists(globalTempDB)) {
throw new SparkException(
s"$globalTempDB is a system preserved database, please rename your existing database " +
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -2295,5 +2295,24 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils {
}
}
}

test(s"basic DDL using locale tr - caseSensitive $caseSensitive") {
withSQLConf(SQLConf.CASE_SENSITIVE.key -> s"$caseSensitive") {
withLocale("tr") {
val dbName = "DaTaBaSe_I"
withDatabase(dbName) {
sql(s"CREATE DATABASE $dbName")
sql(s"USE $dbName")

val tabName = "tAb_I"
withTable(tabName) {
sql(s"CREATE TABLE $tabName(col_I int) USING PARQUET")
sql(s"INSERT OVERWRITE TABLE $tabName SELECT 1")
checkAnswer(sql(s"SELECT col_I FROM $tabName"), Row(1) :: Nil)
}
}
}
}
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,7 @@ package org.apache.spark.sql.test
import java.io.File
import java.net.URI
import java.nio.file.Files
import java.util.UUID
import java.util.{Locale, UUID}

import scala.language.implicitConversions
import scala.util.control.NonFatal
Expand Down Expand Up @@ -228,6 +228,32 @@ private[sql] trait SQLTestUtils
}
}

/**
* Drops database `dbName` after calling `f`.
*/
protected def withDatabase(dbNames: String*)(f: => Unit): Unit = {
try f finally {
dbNames.foreach { name =>
spark.sql(s"DROP DATABASE IF EXISTS $name")
}
}
}

/**
* Enables Locale `language` before executing `f`, then switches back to the default locale of JVM
* after `f` returns.
*/
protected def withLocale(language: String)(f: => Unit): Unit = {
val originalLocale = Locale.getDefault
try {
// Add Locale setting
Locale.setDefault(new Locale(language))
f
} finally {
Locale.setDefault(originalLocale)
}
}

/**
* Activates database `db` before executing `f`, then switches back to `default` database after
* `f` returns.
Expand Down

0 comments on commit 55bea56

Please sign in to comment.