-
Notifications
You must be signed in to change notification settings - Fork 28.9k
[SPARK-20156] [SQL] [FOLLOW-UP] Java String toLowerCase "Turkish locale bug" in Database and Table DDLs #17655
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
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -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 | ||
|
|
@@ -228,6 +228,32 @@ private[sql] trait SQLTestUtils | |
| } | ||
| } | ||
|
|
||
| /** | ||
| * Drops database `dbName` after calling `f`. | ||
| */ | ||
| protected def withDatabase(dbNames: String*)(f: => Unit): Unit = { | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe 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.
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe 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. |
||
| 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. | ||
|
|
||
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?
Uh oh!
There was an error while loading. Please reload this page.
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, becausetoLowerCasewill convert it toıwhen the locale istr. 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.
Uh oh!
There was an error while loading. Please reload this page.
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:
Seems parser does not allow such characters though.
EDITED: We can use backquotes instead in this case
It seems we can still select
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?
Uh oh!
There was an error while loading. Please reload this page.
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.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.