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] [SQL] [FOLLOW-UP] Java String toLowerCase "Turkish locale bug" in Database and Table DDLs #17655

Closed
wants to merge 4 commits into from

Conversation

gatorsmile
Copy link
Member

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 toLowerCasefor avoiding inadvertent locale-sensitive variation in behavior (aka the "Turkish locale problem").

How was this patch tested?

Added a test case

@gatorsmile
Copy link
Member Author

@SparkQA
Copy link

SparkQA commented Apr 17, 2017

Test build #75848 has finished for PR 17655 at commit 47771e1.

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

@SparkQA
Copy link

SparkQA commented Apr 17, 2017

Test build #75849 has finished for PR 17655 at commit 65b0ff7.

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

sql(s"DROP TABLE $tabName")
}

sql(s"DROP DATABASE $dbName")
Copy link
Contributor

Choose a reason for hiding this comment

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

is this needed?

sql(s"CREATE TABLE $tabName(c1 int) USING PARQUET")
sql(s"INSERT OVERWRITE TABLE $tabName SELECT 1")
checkAnswer(sql(s"SELECT c1 FROM $tabName"), Row(1) :: Nil)
sql(s"DROP TABLE $tabName")
Copy link
Contributor

Choose a reason for hiding this comment

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

is this needed?

@@ -114,14 +114,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)
Copy link
Member

Choose a reason for hiding this comment

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

The problem I think is that this affects user apps and we were trying to avoid changes like this. The change was only about internal strings.

I would imagine the fix is in a test, not the main code?

Copy link
Member Author

Choose a reason for hiding this comment

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

We have the restrictions on database/table names. That is, the names can only contain ("[a-zA-z_0-9]+").

Without the fixe in this PR, users are not allowed to read/write/create a table whose name containing I, because toLowerCase will convert it to ı when the locale is tr. The names become illegal. Is my understanding right?

Copy link
Member

Choose a reason for hiding this comment

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

Yes you are correct then, if these identifiers always have only alphanumeric characters. There's no case where lower-casing the table name should be locale-sensitive then.

Is this true of column names?

It won't be true of data, and those are the cases I was trying to leave alone along with user-supplied table and col names, but maybe the latter two aren't locale-sensitive.

Copy link
Member

Choose a reason for hiding this comment

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

I don't think column names have such restrictions. Assuming #7165, it seems we support other characters in column names. I can provide several cases that data becomes column names as below:

scala> Seq("").toDF("a").groupBy("a").pivot("a").count().show()
+---+---+
|  a||
+---+---+
||  1|
+---+---+
scala> import org.apache.spark.sql.functions
import org.apache.spark.sql.functions

scala> spark.range(1).select(functions.lit("")).show()
+---+
||
+---+
||
+---+

Seems parser does not allow such characters though.

scala> sql("SELECT 아 FROM tbl")
org.apache.spark.sql.catalyst.parser.ParseException:
no viable alternative at input 'SELECT 아'(line 1, pos 7)

== SQL ==
SELECTFROM tbl
-------^^^

  at org.apache.spark.sql.catalyst.parser.ParseException.withCommand(ParseDriver.scala:210)
  at org.apache.spark.sql.catalyst.parser.AbstractSqlParser.parse(ParseDriver.scala:112)
  at org.apache.spark.sql.execution.SparkSqlParser.parse(SparkSqlParser.scala:48)
  at org.apache.spark.sql.catalyst.parser.AbstractSqlParser.parsePlan(ParseDriver.scala:66)
  at org.apache.spark.sql.SparkSession.sql(SparkSession.scala:622)
  ... 48 elided

EDITED: We can use backquotes instead in this case

scala> sql("SELECT `아` FROM tbl")
res29: org.apache.spark.sql.DataFrame = [: bigint]

It seems we can still select

scala> Seq("").toDF("a").groupBy("a").pivot("a").count().createOrReplaceTempView("tbl")

scala> sql("SELECT * FROM tbl").show()
+---+---+
|  a||
+---+---+
||  1|
+---+---+

If these were mistakenly supported, these should have the restrictions first.

Copy link
Member Author

Choose a reason for hiding this comment

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

Have you tried to use backticks to quote the column names?

Copy link
Member

Choose a reason for hiding this comment

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

Do you mean selectExpr("`아`") via the parser? Ah, Sorry, it seems working if we backquotes.

scala> Seq(1).toDF("").selectExpr("`아`")
res14: org.apache.spark.sql.DataFrame = [: int]

Copy link
Member Author

Choose a reason for hiding this comment

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

Then, it works as expected.

Copy link
Member

Choose a reason for hiding this comment

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

Yup, so, it seems the column names don't have such restrictions. I added the case you mentioned above in my comment.

@HyukjinKwon
Copy link
Member

@gatorsmile, BTW, I just wonder if there test cases we should fix. If there are few, we might better fix them together here.

@gatorsmile
Copy link
Member Author

This PR is not to fix the test cases.

@HyukjinKwon
Copy link
Member

Are you going to then open another JIRA and PR (or a followup) to fix test cases related to Turkish locale bug?

@gatorsmile
Copy link
Member Author

There is another issue in locale support in Avro serde. It is out of our controls.

In the QueryTest, we already set the Locale to US. Thus, it covers DDLSuite and HiveDDLSuite. Thus, I do not think we should do any change.

@srowen
Copy link
Member

srowen commented Apr 18, 2017

Am I right that this PR resolves the following problem: if you create a table name like "Things" and run Spark in the Turkish locale, and rely on case-insensitive table comparison, you'll fail to find the table as "things" because it will lower-case differently in one case vs another?

That's worth fixing. It sounds like it doesn't affect column names in the same way.

Does this PR actually fix it? I'm not clear from the last comments here.

Are there other issues related to Avro serde? that is what we fixed already (in theory). Is it a test-only locale issue?

@gatorsmile
Copy link
Member Author

gatorsmile commented Apr 18, 2017

If the table name is Things, it works well. So far, the only issue I hit is the big I, because toLowerCase in SessionCatalog converts it to a non-alphabic character ı.

Updated the test case by adding the column names having I. I did not go over the code base to prove all the cases work well.

Avro serde issues can be easily reproduced. For example,

    withLocale("tr") {
      withTable("t") {
        Seq(1 -> "a").toDF("i", "j")
          .write.format("hive").option("fileFormat", "avro").saveAsTable("t")
      }
    }

The error is from the underlying avro schema parser. It is not caused by our Spark, I think.

Caused by: org.apache.avro.SchemaParseException: Undefined name: "ınt"
	at org.apache.avro.Schema.parse(Schema.java:1162)

@SparkQA
Copy link

SparkQA commented Apr 18, 2017

Test build #75910 has finished for PR 17655 at commit ba6913f.

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

@srowen
Copy link
Member

srowen commented Apr 19, 2017

This change looks good. You're saying that you think the Avro issue is another instance of the same issue, just in Avro? if so then, yeah, not sure we can fix that unfortunately, not directly.

I did one more pass, while we (you) are cleaning up some loose ends, and spotted two more instances that look questionable to me.

ResolveTableValuedFunctions:

  override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
    case u: UnresolvedTableValuedFunction if u.functionArgs.forall(_.resolved) =>
      builtinFunctions.get(u.functionName.toLowerCase()) match {

I'm guessing this is another instance where the function name should not be locale-sensitive. Not sure how I missed it.

And related to your change here, in SharedState line 117:

    val globalTempDB = sparkContext.conf.get(GLOBAL_TEMP_DATABASE).toLowerCase

should be non-locale-sensitive too?

@@ -229,6 +229,32 @@ private[sql] trait SQLTestUtils
}

/**
* Drops database `dbName` after calling `f`.
*/
protected def withDatabase(dbNames: String*)(f: => Unit): Unit = {
Copy link
Member

Choose a reason for hiding this comment

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

I'm fine with this but there's only one usage of it, and other tests don't seem to bother to drop their tables -- is it necessary within the context of one run? or just inline this? I don't feel strongly, you can leave it too.

Copy link
Member Author

Choose a reason for hiding this comment

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

In the future, we will use it more when refactoring the test cases.

@SparkQA
Copy link

SparkQA commented Apr 20, 2017

Test build #75972 has started for PR 17655 at commit aeeaba5.

@gatorsmile
Copy link
Member Author

Yeah. It sounds the Avro issues are not caused by us. Thus, we are unable to fix it. : (

Fixed the issues you mentioned above. They look right to me, but adding test cases for them might not be simple. Thanks!

@HyukjinKwon
Copy link
Member

The change looks good to me too.

@nihavend
Copy link

I had another issue before about avro https://issues.apache.org/jira/browse/AVRO-1493 that has been fixed.

@gatorsmile should i follow the avro serde case separately ?

@SparkQA
Copy link

SparkQA commented Apr 20, 2017

Test build #3665 has finished for PR 17655 at commit aeeaba5.

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

asfgit pushed a commit that referenced this pull request Apr 20, 2017
… 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 <gatorsmile@gmail.com>

Closes #17655 from gatorsmile/locale.

(cherry picked from commit 55bea56)
Signed-off-by: Sean Owen <sowen@cloudera.com>
@srowen
Copy link
Member

srowen commented Apr 20, 2017

Merged to master/2.2

@asfgit asfgit closed this in 55bea56 Apr 20, 2017
peter-toth pushed a commit to peter-toth/spark that referenced this pull request Oct 6, 2018
… 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 <gatorsmile@gmail.com>

Closes apache#17655 from gatorsmile/locale.
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