[SPARK-30288][SQL]Remove Parquet Column Name Check#26945
[SPARK-30288][SQL]Remove Parquet Column Name Check#26945iRakson wants to merge 1 commit intoapache:masterfrom
Conversation
|
Can one of the admins verify this patch? |
|
cc @HyukjinKwon |
| def checkFieldName(name: String): Unit = { | ||
| // ,;{}()\n\t= and space are special characters in Parquet schema | ||
| checkConversionRequirement( | ||
| !name.matches(".*[ ,;{}()\n\t=].*"), |
There was a problem hiding this comment.
@iRakson Could you run the UT locally for SQL module related to this change?
|
cc @rdblue , @zsxwing , @gengliangwang |
|
The characters that were checked in This change will work for Spark because Parquet files don't use the IDL form of a schema in file metadata (instead it is converted to Thrift objects). But this change will allow users to create schemas that can't be used by anything that requires parsing a Parquet message type, including Parquet's InputFormat. This will break the string schema representation used commonly to pass a schema in Configuration. I'd recommend against this, unless someone updates that parser. I'd support making that change in upstream Parquet and then removing the check here. (I used to think this was a bad idea because it would break Avro, but I've changed my mind.) |
|
+1 for following the advice. |
|
Hello and thanks for making this MR.
The problem is that people will use pandas and create a dataframe with this "invalid" name, but then this doesn't become an issue until it's written to parquet from Spark which could potentially happen after a project is pretty far along. Please let me know, |
What changes were proposed in this pull request?
Removed the old check for column names when creating a parquet table.
Before Changes:
After changes:
Why are the changes needed?
Now parquet supports all the special characters that we were checking for previously. Initially parquet used to throw errors while using these special characters. So this validity check was introduced. Now parquet do not throw any exception for column names with special characters.
In JIRA also, one of user has pasted the output when creating parquet tables in pandas. There it supports special characters in column names.
Does this PR introduce any user-facing change?
Yes. Now Users will be able to create parquet tables with special characters in column names.
How was this patch tested?
Manually.
Will add unit tests soon.