Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[SPARK-20156][CORE][SQL][STREAMING][MLLIB] Java String toLowerCase "Turkish locale bug" causes Spark problems #17527

Closed
wants to merge 1 commit into from

Conversation

srowen
Copy link
Member

@srowen srowen commented Apr 4, 2017

What changes were proposed in this pull request?

Add Locale.ROOT to internal calls to String toLowerCase, toUpperCase, to avoid inadvertent locale-sensitive variation in behavior (aka the "Turkish locale problem").

The change looks large but it is just adding Locale.ROOT (the locale with no country or language specified) to every call to these methods.

How was this patch tested?

Existing tests.

@SparkQA
Copy link

SparkQA commented Apr 4, 2017

Test build #75506 has finished for PR 17527 at commit ab5995c.

  • This patch fails Scala style tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Apr 4, 2017

Test build #75507 has finished for PR 17527 at commit af811ce.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@srowen
Copy link
Member Author

srowen commented Apr 5, 2017

Maybe @HyukjinKwon can confirm whether this sounds like a good idea? simple change, just touches so many instances.

@HyukjinKwon
Copy link
Member

HyukjinKwon commented Apr 5, 2017

Thanks for cc'ing me. I support this idea in general. I can at least identify several references, for example,, https://hibernate.atlassian.net/plugins/servlet/mobile#issue/HHH-9722, https://github.com/hibernate/hibernate-orm/pull/931 and https://www.google.co.kr/amp/s/garygregory.wordpress.com/2015/11/03/java-lowercase-conversion-turkey/amp/

Let me investigate possible downside and check the codes within today at my best.

@viirya
Copy link
Member

viirya commented Apr 6, 2017

Out of curious, is there any situation we do really need the locale setting, instead of Locale.ROOT?

Copy link
Member

@HyukjinKwon HyukjinKwon left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

So,, it seems

scala> "I".toLowerCase(java.util.Locale.forLanguageTag("tr"));
res1: String = ı

scala> "i".toUpperCase(java.util.Locale.forLanguageTag("tr"));
res2: String = İ

vs

scala> "I".toLowerCase(java.util.Locale.ROOT);
res3: String = i

scala> "i".toUpperCase(java.util.Locale.ROOT);
res4: String = I

I left my worry on the codes. Personally, support this idea (rather than tracking down such cases every time when we meet the related issues).

@@ -407,7 +408,7 @@ public UTF8String toLowerCase() {
}

private UTF8String toLowerCaseSlow() {
return fromString(toString().toLowerCase());
return fromString(toString().toLowerCase(Locale.ROOT));
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It seems there are few cases exposed to users. For example, this seems used in Lower and InitCap expressions, where, up to my knowledge, the lower cased ones are exposed into users.

In this case, IIRC the same argument (consistent base + options for variants approach) is applied in SPARK-18076. So, I think it might be fine.

@HyukjinKwon
Copy link
Member

@viirya, I think it is possible. In case of Lower, Upper and InitCap as an example maybe.

@viirya
Copy link
Member

viirya commented Apr 6, 2017

@HyukjinKwon yap, so for such cases exposed to users, I think it is better to leave it out for the default locale?

@HyukjinKwon
Copy link
Member

HyukjinKwon commented Apr 6, 2017

Yea, that's the concern. The downside is when these are exposed to users. However, it might be an advantage as well. The behavior doesn't depend on default JVM locale and is consistent. I think the argument is the same with SPARK-18076.

If it is easy to leave out the exposed ones, I think that might be also an option.

@srowen
Copy link
Member Author

srowen commented Apr 6, 2017

Yes that's certainly the question. Is it more surprising or less surprising for Lower()'s behavior to vary by machine's Locale? Generally I'd say that the varying behavior is bad. However in this case we also don't have a standard way of letting users specify the Locale. It becomes impossible to get the Turkish-lower-cased version of a string via the API.

For that reason I could see backing out changes that affect user application strings, to be conservative. We could decide to change that later. The issue here really stems from lowercasing of purely internal strings.

@viirya
Copy link
Member

viirya commented Apr 6, 2017

@srowen That sounds good to me.

@HyukjinKwon
Copy link
Member

Thank you for clarifying it, @srowen and @viirya . I am okay with it too.

@srowen
Copy link
Member Author

srowen commented Apr 6, 2017

I reverted the changes to UTF8String, StopWordsRemover, and Tokenizer. The rest look like either purely internal string-insensitive comparison, excepting operations on database table and column names. I wonder if those need to be Locale-sensitive or not? it is not a question of input a user program may process in that case, but of the user's application and data schema itself.

@SparkQA
Copy link

SparkQA commented Apr 6, 2017

Test build #75572 has finished for PR 17527 at commit 87cb5de.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

Copy link
Member

@HyukjinKwon HyukjinKwon left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@srowen, I don't know the explicit answer to the question. So, I ran some tests and checked the each code block. I think it seems safer only to leave purely internal string-insensitive comparison as we are not going to do this for ones exposed to users.

I assume we support case sensitive column names. For example, as below:

scala> Seq(Tuple4("a", "b", "c", "d")).toDF("ı", "I", "i", "İ").show()
+---+---+---+---+
|  ı|  I|  i|  İ|
+---+---+---+---+
|  a|  b|  c|  d|
+---+---+---+---+

and also other languages as field names:

scala> spark.range(1).toDF("")
res2: org.apache.spark.sql.DataFrame = [: bigint]

Both ı & I and i & İ are different characters in Turkish (I asked my Turkish friend for sure). So, I think we should differentiate them

I left some comments with some tests and my opinion. I hope this is helpful.

@@ -396,7 +397,7 @@ object PartitioningAwareFileIndex extends Logging {
sessionOpt: Option[SparkSession]): Seq[FileStatus] = {
logTrace(s"Listing $path")
val fs = path.getFileSystem(hadoopConf)
val name = path.getName.toLowerCase
val name = path.getName.toLowerCase(Locale.ROOT)
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

(This variable seems not used.)

@@ -26,11 +28,12 @@ package org.apache.spark.sql.catalyst.util
class CaseInsensitiveMap[T] private (val originalMap: Map[String, T]) extends Map[String, T]
with Serializable {

val keyLowerCasedMap = originalMap.map(kv => kv.copy(_1 = kv._1.toLowerCase))
val keyLowerCasedMap = originalMap.map(kv => kv.copy(_1 = kv._1.toLowerCase(Locale.ROOT)))
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Maybe nitpicking and it is rarely possible I guess. However, up to my knowledge, this will affect the options users set, spark.read.option(...). Namely, I think these case possible as below:

scala> java.util.Locale.setDefault(new java.util.Locale("tr"))

scala> val originalMap = Map("ı" -> 1, "I" -> 2)
originalMap: scala.collection.immutable.Map[String,Int] = Map-> 1, I -> 2)

Before

scala> originalMap.map(kv => kv.copy(_1 = kv._1.toLowerCase))
res6: scala.collection.immutable.Map[String,Int] = Map-> 2)

After

scala> originalMap.map(kv => kv.copy(_1 = kv._1.toLowerCase(java.util.Locale.ROOT)))
res7: scala.collection.immutable.Map[String,Int] = Map-> 1, i -> 2)

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes, though would these keys ever involve a user-defined key string? I think they're all Spark, Hive properties, etc.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes, hardly possible really.. not user but a developer could use that Turkish character for a external data source.. (external data source could use this up to my knowledge, see interfaces.scala#L73-L74 and DataSource.scala#L306).

For example, XML datasource is extending this - DefaultSource.scala#L30. Though.. one case I finally come up is use the map as a kind of arbitrary key and value.

Someone opened a PR to Spark XML https://github.com/databricks/spark-xml/pull/247 and this proposes to use arbitrary key and value for writing out attributes to the root element.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Really, it sounds nitpicking though.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I am actually fine if you are fond of leaving it. I really think such cases are inappropriate and hardly possible.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I don't think there's a right answer... for consistency it seems like we should treat keys uniformly and my gut says that we want those to be locale-neutral because they are generally never something localized or locale-specific.

@@ -25,7 +27,7 @@ object StringKeyHashMap {
def apply[T](caseSensitive: Boolean): StringKeyHashMap[T] = if (caseSensitive) {
new StringKeyHashMap[T](identity)
} else {
new StringKeyHashMap[T](_.toLowerCase)
new StringKeyHashMap[T](_.toLowerCase(Locale.ROOT))
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This only seems used in SimpleFunctionRegistry. I don't think we have Turkish characters in function names and I don't think users will use other language in the function names. So probably it is fine.

val partColumns = partitionColumnNames.map(_.toLowerCase).toSet
relation.output.filter(a => partColumns.contains(a.name.toLowerCase))
val partColumns = partitionColumnNames.map(_.toLowerCase(Locale.ROOT)).toSet
relation.output.filter(a => partColumns.contains(a.name.toLowerCase(Locale.ROOT)))
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I am little bit worried of this change likewise. For example,

Before

scala> java.util.Locale.setDefault(new java.util.Locale("tr"))

scala>  val partColumns = Seq("I").map(_.toLowerCase).toSet
partColumns: scala.collection.immutable.Set[String] = Set(ı)

scala> Seq("a", "ı", "I").filter(a => partColumns.contains(a.toLowerCase))
res13: Seq[String] = List(ı, I)

After

scala> val partColumns = Seq("I").map(_.toLowerCase(java.util.Locale.ROOT)).toSet
partColumns: scala.collection.immutable.Set[String] = Set(i)

scala> Seq("a", "ı", "I").filter(a => partColumns.contains(a.toLowerCase(java.util.Locale.ROOT)))
res14: Seq[String] = List(I)

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes, the question is whether case-insensitive operations on schema elements, not just data, should be locale-neutral.

The flip side to this argument is that a case-insensitive operation on a schema with the character "I" or "i" suddenly doesn't work as expected on a machine with the Turkish locale.

I don't know which is rarer: schema with these special characters, or a machine with this particular Locale.

I think I'd probably err on the side of not changing the behavior. I'll further revert the changes that touch things like column and table names.

_.name
} else {
_.name.toLowerCase(Locale.ROOT)
}
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think we should leave this out. It seems dataSchema means the schema from the source which is exposed to users. I think this could cause a problem. For example as below:

Before

import collection.mutable

import org.apache.spark.sql.types._

java.util.Locale.setDefault(new java.util.Locale("tr"))

val partitionSchema: StructType = StructType(StructField("I", StringType) :: Nil)
val dataSchema: StructType = StructType(StructField("ı", StringType) :: Nil)

val getColName: (StructField => String) = _.name.toLowerCase

val overlappedPartCols = mutable.Map.empty[String, StructField]
partitionSchema.foreach { partitionField =>
  if (dataSchema.exists(getColName(_) == getColName(partitionField))) {
    overlappedPartCols += getColName(partitionField) -> partitionField
  }
}

val schema = StructType(dataSchema.map(f => overlappedPartCols.getOrElse(getColName(f), f)) ++
  partitionSchema.filterNot(f => overlappedPartCols.contains(getColName(f))))

schema.fieldNames

prints

Array[String] = Array(I)

After

import collection.mutable

import org.apache.spark.sql.types._

java.util.Locale.setDefault(new java.util.Locale("tr"))

val partitionSchema: StructType = StructType(StructField("I", StringType) :: Nil)
val dataSchema: StructType = StructType(StructField("ı", StringType) :: Nil)

val getColName: (StructField => String) = _.name.toLowerCase(java.util.Locale.ROOT)

val overlappedPartCols = mutable.Map.empty[String, StructField]
partitionSchema.foreach { partitionField =>
  if (dataSchema.exists(getColName(_) == getColName(partitionField))) {
    overlappedPartCols += getColName(partitionField) -> partitionField
  }
}

val schema = StructType(dataSchema.map(f => overlappedPartCols.getOrElse(getColName(f), f)) ++
  partitionSchema.filterNot(f => overlappedPartCols.contains(getColName(f))))

schema.fieldNames

prints

Array[String] = Array(ı, I)

@@ -128,7 +128,8 @@ object PartitioningUtils {
// "hdfs://host:9000/invalidPath"
// "hdfs://host:9000/path"
// TODO: Selective case sensitivity.
val discoveredBasePaths = optDiscoveredBasePaths.flatten.map(_.toString.toLowerCase())
val discoveredBasePaths =
optDiscoveredBasePaths.flatten.map(_.toString.toLowerCase(Locale.ROOT))
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I am worried of this one too. It sounds the path could contains Turkish characters I guess..

@@ -328,7 +329,7 @@ object PartitioningUtils {
} else {
// TODO: Selective case sensitivity.
val distinctPartColNames =
pathsWithPartitionValues.map(_._2.columnNames.map(_.toLowerCase())).distinct
pathsWithPartitionValues.map(_._2.columnNames.map(_.toLowerCase(Locale.ROOT))).distinct
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think this might cause a similar problem with https://github.com/apache/spark/pull/17527/files#r110317272.

@@ -222,7 +225,7 @@ case class PreprocessTableCreation(sparkSession: SparkSession) extends Rule[Logi
val columnNames = if (sparkSession.sessionState.conf.caseSensitiveAnalysis) {
schema.map(_.name)
} else {
schema.map(_.name.toLowerCase)
schema.map(_.name.toLowerCase(Locale.ROOT))
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Maybe, it is not good to point the similar instances all but let me just point this out as the change looks big. This maybe the similar instances with https://github.com/apache/spark/pull/17527/files#r110317272.

@srowen
Copy link
Member Author

srowen commented Apr 7, 2017

Thanks @HyukjinKwon I backed out the changes that seem to directly affect column, table and database names.

Copy link
Member

@HyukjinKwon HyukjinKwon left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@srowen, for me LGTM except for single trivial comment.

@@ -910,7 +911,8 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat
// When Hive rename partition for managed tables, it will create the partition location with
// a default path generate by the new spec with lower cased partition column names. This is
// unexpected and we need to rename them manually and alter the partition location.
val hasUpperCasePartitionColumn = partitionColumnNames.exists(col => col.toLowerCase != col)
val hasUpperCasePartitionColumn =
partitionColumnNames.exists(col => col.toLowerCase != col)
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@srowen, it seems mistakenly a newline inserted for this and the one below.

@SparkQA
Copy link

SparkQA commented Apr 7, 2017

Test build #75603 has finished for PR 17527 at commit ecd6d3a.

  • This patch fails Spark unit tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Apr 7, 2017

Test build #75608 has finished for PR 17527 at commit 662f6ae.

  • This patch fails Spark unit tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Apr 7, 2017

Test build #3647 has finished for PR 17527 at commit 662f6ae.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

… to avoid inadvertent locale-sensitive variation in behavior (aka the "Turkish locale problem")
@SparkQA
Copy link

SparkQA commented Apr 10, 2017

Test build #75657 has finished for PR 17527 at commit 2ac5843.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@srowen
Copy link
Member Author

srowen commented Apr 10, 2017

Merged to master

@asfgit asfgit closed this in a26e3ed Apr 10, 2017
@nihavend
Copy link

Thank you very much all of you for all your efforts. Sometimes, facing the same issue different platforms and looking for a way to set jvm options for locale explicitly. But many times there is no way.

And mostly the platform owners do not take this as serious problem and take care as you do.

For this reason, on the platforms leaving user an open door to define the locale manually may solve most of the issues related with the locale problem.

Hope helped some for future.

@srowen srowen deleted the SPARK-20156 branch April 11, 2017 11:21
@gatorsmile
Copy link
Member

I am wondering what is the reason some of toLowerCase is changed, but the others remain unchanged?

@srowen
Copy link
Member Author

srowen commented Apr 11, 2017

The general idea is to leave any lower-casing that affects strings in the user program alone, to use the locale-sensitive toLowerCase(). This is more conservative. All of the changes should only affect internal strings or API values, where there is no reason to be locale-specific. For example: checking a property value against a known list of enum string values in a case-insensitive way. This should address the underlying problem, where lower-casing an internal property results int he wrong result in the Turkish locale, without changing the results of a user program.

@gatorsmile
Copy link
Member

Thank you for your explanation. Just did a few run in our DDL support. We still have a few bugs in the locale support. If we use Turkish locale, a few test cases failed.

Do you know what is the existing locale support for Hive and Hive metastore?

Also cc @cloud-fan

@HyukjinKwon
Copy link
Member

Do you mind if I ask a example case? I just would like to look into this to help.

@gatorsmile
Copy link
Member

@HyukjinKwon You can set the locale to tr. You will see the test failure. The test cases failed because the column names are incorrectly set.

@HyukjinKwon
Copy link
Member

If you refer column names being ı from I in Turkish locale, it might be correct per the discussion above as they are correct lower cases in Turkish locale.

I would like to know and verify which test case failures make you assert we still have a few bugs in the locale support. The burden of proof should be yours if you think we still have the bugs and say this PR is incomplete due to this.

If there are, we should make a followup and I am willing to track it down.

@nihavend
Copy link

I logged the comment for the case below. The column and/or table name(s) are not user or locale dependent as they are internals.

e.g :
"SERDEINFO" lowers to "serdeınfo"
"uniquetable" uppers to "UNİQUETABLE"
work around :
add -Duser.country=US -Duser.language=en to the end of the line
SPARK_SUBMIT_OPTS="$SPARK_SUBMIT_OPTS -Dscala.usejavacp=true"
in spark-shell.sh

@gatorsmile
Copy link
Member

: ) It sounds like you did not get my points. Anyway, I will fix it later.

@nihavend
Copy link

maybe

@gatorsmile
Copy link
Member

Sorry, my previous comment is to @HyukjinKwon

@srowen
Copy link
Member Author

srowen commented Apr 16, 2017

Yes you have a point. It is minor in that it is just a test that is now locale sensitive and supporting the locale in tests is much less important. However ideally whatever fails should be fixed as I suspect it would be some trivial piece we missed.

@HyukjinKwon
Copy link
Member

Ah, sorry, it was only about fixing tests. I thought we have bugs in the main codes.

@gatorsmile
Copy link
Member

Yes. The codes have the bug. For example, when the locale is TR, users are unable to create a table with a table name containing I. This does not make sense to me. I believe we have more issues like this. I can submit a PR to fix this, but I do not think this is the only one.

peter-toth pushed a commit to peter-toth/spark that referenced this pull request Oct 6, 2018
…urkish locale bug" causes Spark problems

Add Locale.ROOT to internal calls to String `toLowerCase`, `toUpperCase`, to avoid inadvertent locale-sensitive variation in behavior (aka the "Turkish locale problem").

The change looks large but it is just adding `Locale.ROOT` (the locale with no country or language specified) to every call to these methods.

Existing tests.

Author: Sean Owen <sowen@cloudera.com>

Closes apache#17527 from srowen/SPARK-20156.

Change-Id: Iba0f12a3c0903d9a98ae98ed9b90784b59e468e2
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
6 participants