-
Notifications
You must be signed in to change notification settings - Fork 28.9k
[SPARK-20460][SQL] Make it more consistent to handle column name duplication #17758
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
Conversation
|
Test build #76137 has finished for PR 17758 at commit
|
|
Test build #76138 has finished for PR 17758 at commit
|
|
Test build #76165 has finished for PR 17758 at commit
|
|
Test build #76167 has finished for PR 17758 at commit
|
|
Test build #76864 has finished for PR 17758 at commit
|
|
@gatorsmile Could you check this and give me advise on this? Thanks! |
|
ping |
|
Will review it tomorrow. Ping me if I forgot it. |
|
Thanks alot! |
|
@gatorsmile ping |
|
ping |
|
cc @wzhfy |
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.
Rename the file (SchemaUtil) also as SchemaUtils?
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.
nit: duplicate """
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 header option necessary?
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 also need to add tests for case sensitivity.
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 addition to end-to-end tests, we can also add a suite for SchemaUtils and put the case sensitivity cases in it.
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.
Added SchemaUtilsSuite.
|
Thanks! I'll update in a day |
|
Hi @maropu , I just did some simple search, and found other places also related to duplicate columns. e.g. |
|
ok, I'll also check again. Thanks! |
|
Test build #77971 has finished for PR 17758 at commit
|
|
Test build #77973 has finished for PR 17758 at commit
|
|
Test build #77981 has finished for PR 17758 at commit
|
|
Test build #77986 has finished for PR 17758 at commit
|
|
Test build #77991 has finished for PR 17758 at commit
|
This reverts commit 12159c403955f54066ed8c532ed991f829edfc1f.
|
Test build #79284 has finished for PR 17758 at commit
|
| -- Catch case-sensitive name duplication | ||
| SET spark.sql.caseSensitive=true; | ||
|
|
||
| CREATE TABLE t(c0 STRING, c1 INT, c1 DOUBLE, c0 INT) USING parquet; |
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 didn't have test cases for create table before?
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 DDLSuite, we already have simple tests for duplicate columns. we better moving these tests there?
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 should keep them in one place. For now I think we still need to put them in DDLSuite because we need to run it with and without hive support. Can we pick some typical test cases here and move them to DDLSuite?
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.
ok, will update
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 moved some tests to DDLSuite and removed this file.
|
Test build #79320 has finished for PR 17758 at commit
|
|
Test build #79323 has finished for PR 17758 at commit
|
|
LGTM |
| } | ||
| } | ||
|
|
||
| val errorMsg = intercept[AnalysisException] { |
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.
why is this test related to streaming?
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.
If we have streaming metadata logs, we have the different code path from non-streaming cases in Dataset. So, the original motivation of this tests is to check the duplication even in this path.
|
thanks, merging to master! |
|
@cloud-fan Thanks! BTW, we better handle other duplication (e.g., local temporary tables)?; This changes the existing behaviour though. |
|
We can create a ticket first and discuss there. |
|
ok, I'll file a ticket first. Thanks. |
|
Still when you load the json file through Dataset[String] by doing |
|
yea, I think that is an expected behaviour, actually, some operations of Also, the original motivation of this pr does not intend to change existing behaviour. |
|
Why it's ok to have duplicate columns when you read from RDD/DS and not when you read directly from file? Maybe it's should be configurable option? |
|
Because it is a design of |
…lt datasources ### What changes were proposed in this pull request? When `spark.sql.caseSensitive` is `false` (by default), check that there are not duplicate column names on the same level (top level or nested levels) in reading from in-built datasources Parquet, ORC, Avro and JSON. If such duplicate columns exist, throw the exception: ``` org.apache.spark.sql.AnalysisException: Found duplicate column(s) in the data schema: ``` ### Why are the changes needed? To make handling of duplicate nested columns is similar to handling of duplicate top-level columns i. e. output the same error when `spark.sql.caseSensitive` is `false`: ```Scala org.apache.spark.sql.AnalysisException: Found duplicate column(s) in the data schema: `camelcase` ``` Checking of top-level duplicates was introduced by #17758. ### Does this PR introduce _any_ user-facing change? Yes. For the example from SPARK-32431: ORC: ```scala java.io.IOException: Error reading file: file:/private/var/folders/p3/dfs6mf655d7fnjrsjvldh0tc0000gn/T/spark-c02c2f9a-0cdc-4859-94fc-b9c809ca58b1/part-00001-63e8c3f0-7131-4ec9-be02-30b3fdd276f4-c000.snappy.orc at org.apache.orc.impl.RecordReaderImpl.nextBatch(RecordReaderImpl.java:1329) at org.apache.orc.mapreduce.OrcMapreduceRecordReader.ensureBatch(OrcMapreduceRecordReader.java:78) ... Caused by: java.io.EOFException: Read past end of RLE integer from compressed stream Stream for column 3 kind DATA position: 6 length: 6 range: 0 offset: 12 limit: 12 range 0 = 0 to 6 uncompressed: 3 to 3 at org.apache.orc.impl.RunLengthIntegerReaderV2.readValues(RunLengthIntegerReaderV2.java:61) at org.apache.orc.impl.RunLengthIntegerReaderV2.next(RunLengthIntegerReaderV2.java:323) ``` JSON: ```scala +------------+ |StructColumn| +------------+ | [,,]| +------------+ ``` Parquet: ```scala +------------+ |StructColumn| +------------+ | [0,, 1]| +------------+ ``` Avro: ```scala +------------+ |StructColumn| +------------+ | [,,]| +------------+ ``` After the changes, Parquet, ORC, JSON and Avro output the same error: ```scala Found duplicate column(s) in the data schema: `camelcase`; org.apache.spark.sql.AnalysisException: Found duplicate column(s) in the data schema: `camelcase`; at org.apache.spark.sql.util.SchemaUtils$.checkColumnNameDuplication(SchemaUtils.scala:112) at org.apache.spark.sql.util.SchemaUtils$.checkSchemaColumnNameDuplication(SchemaUtils.scala:51) at org.apache.spark.sql.util.SchemaUtils$.checkSchemaColumnNameDuplication(SchemaUtils.scala:67) ``` ### How was this patch tested? Run modified test suites: ``` $ build/sbt "sql/test:testOnly org.apache.spark.sql.FileBasedDataSourceSuite" $ build/sbt "avro/test:testOnly org.apache.spark.sql.avro.*" ``` and added new UT to `SchemaUtilsSuite`. Closes #29234 from MaxGekk/nested-case-insensitive-column. Authored-by: Max Gekk <[email protected]> Signed-off-by: Wenchen Fan <[email protected]>
…atasource ### What changes were proposed in this pull request? Check that there are not duplicate column names on the same level (top level or nested levels) in reading from JDBC datasource. If such duplicate columns exist, throw the exception: ``` org.apache.spark.sql.AnalysisException: Found duplicate column(s) in the customSchema option value: ``` The check takes into account the SQL config `spark.sql.caseSensitive` (`false` by default). ### Why are the changes needed? To make handling of duplicate nested columns is similar to handling of duplicate top-level columns i. e. output the same error: ```Scala org.apache.spark.sql.AnalysisException: Found duplicate column(s) in the customSchema option value: `camelcase` ``` Checking of top-level duplicates was introduced by #17758, and duplicates in nested structures by #29234. ### Does this PR introduce _any_ user-facing change? Yes. ### How was this patch tested? Added new test suite `JdbcNestedDataSourceSuite`. Closes #29317 from MaxGekk/jdbc-dup-nested-columns. Authored-by: Max Gekk <[email protected]> Signed-off-by: Wenchen Fan <[email protected]>
What changes were proposed in this pull request?
This pr made it more consistent to handle column name duplication. In the current master, error handling is different when hitting column name duplication:
When this patch applied, the results change to;
How was this patch tested?
Added tests in
DataFrameReaderWriterSuiteandSQLQueryTestSuite.