-
Notifications
You must be signed in to change notification settings - Fork 28.4k
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
Conversation
Test build #75848 has finished for PR 17655 at commit
|
Test build #75849 has finished for PR 17655 at commit
|
sql(s"DROP TABLE $tabName") | ||
} | ||
|
||
sql(s"DROP DATABASE $dbName") |
There was a problem hiding this comment.
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") |
There was a problem hiding this comment.
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) |
There was a problem hiding this comment.
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?
There was a problem hiding this comment.
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?
There was a problem hiding this comment.
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.
There was a problem hiding this comment.
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 ==
SELECT 아 FROM 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.
There was a problem hiding this comment.
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?
There was a problem hiding this comment.
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]
There was a problem hiding this comment.
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.
There was a problem hiding this comment.
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.
@gatorsmile, BTW, I just wonder if there test cases we should fix. If there are few, we might better fix them together here. |
This PR is not to fix the test cases. |
Are you going to then open another JIRA and PR (or a followup) to fix test cases related to Turkish locale bug? |
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. |
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? |
If the table name is Updated the test case by adding the column names having 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.
|
Test build #75910 has finished for PR 17655 at commit
|
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.
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
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 = { |
There was a problem hiding this comment.
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.
There was a problem hiding this comment.
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.
Test build #75972 has started for PR 17655 at commit |
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! |
The change looks good to me too. |
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 ? |
Test build #3665 has finished for PR 17655 at commit
|
… 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]>
Merged to master/2.2 |
… 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 apache#17655 from gatorsmile/locale.
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 addLocale.ROOT
to thetoLowerCase
for avoiding inadvertent locale-sensitive variation in behavior (aka the "Turkish locale problem").How was this patch tested?
Added a test case