-
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
Changes from all commits
1e647ee
4467077
33ab217
d8efb9d
11d1818
22e1e4f
743a069
f6eab2d
09da8d6
6d03f31
a0b9b05
91b6424
37ad3f3
d0d9d3e
cbe9c71
c69270f
af959f6
8d3e10a
9b386d5
a878510
be20127
f41bf80
0526391
9e199bc
1ae132d
5c29a75
5ed2c0d
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 |
|---|---|---|
| @@ -0,0 +1,83 @@ | ||
| /* | ||
| * Licensed to the Apache Software Foundation (ASF) under one or more | ||
| * contributor license agreements. See the NOTICE file distributed with | ||
| * this work for additional information regarding copyright ownership. | ||
| * The ASF licenses this file to You under the Apache License, Version 2.0 | ||
| * (the "License"); you may not use this file except in compliance with | ||
| * the License. You may obtain a copy of the License at | ||
| * | ||
| * http://www.apache.org/licenses/LICENSE-2.0 | ||
| * | ||
| * Unless required by applicable law or agreed to in writing, software | ||
| * distributed under the License is distributed on an "AS IS" BASIS, | ||
| * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
| * See the License for the specific language governing permissions and | ||
| * limitations under the License. | ||
| */ | ||
|
|
||
| package org.apache.spark.sql.util | ||
|
|
||
| import org.apache.spark.SparkFunSuite | ||
| import org.apache.spark.sql.AnalysisException | ||
| import org.apache.spark.sql.catalyst.analysis._ | ||
| import org.apache.spark.sql.types.StructType | ||
|
|
||
| class SchemaUtilsSuite extends SparkFunSuite { | ||
|
|
||
| private def resolver(caseSensitiveAnalysis: Boolean): Resolver = { | ||
| if (caseSensitiveAnalysis) { | ||
| caseSensitiveResolution | ||
| } else { | ||
| caseInsensitiveResolution | ||
| } | ||
| } | ||
|
|
||
| Seq((true, ("a", "a"), ("b", "b")), (false, ("a", "A"), ("b", "B"))).foreach { | ||
| case (caseSensitive, (a0, a1), (b0, b1)) => | ||
|
|
||
| val testType = if (caseSensitive) "case-sensitive" else "case-insensitive" | ||
| test(s"Check column name duplication in $testType cases") { | ||
| def checkExceptionCases(schemaStr: String, duplicatedColumns: Seq[String]): Unit = { | ||
| val expectedErrorMsg = "Found duplicate column(s) in SchemaUtilsSuite: " + | ||
| duplicatedColumns.map(c => s"`${c.toLowerCase}`").mkString(", ") | ||
| val schema = StructType.fromDDL(schemaStr) | ||
| var msg = intercept[AnalysisException] { | ||
| SchemaUtils.checkSchemaColumnNameDuplication( | ||
| schema, "in SchemaUtilsSuite", caseSensitiveAnalysis = caseSensitive) | ||
| }.getMessage | ||
| assert(msg.contains(expectedErrorMsg)) | ||
| msg = intercept[AnalysisException] { | ||
| SchemaUtils.checkColumnNameDuplication( | ||
| schema.map(_.name), "in SchemaUtilsSuite", resolver(caseSensitive)) | ||
| }.getMessage | ||
| assert(msg.contains(expectedErrorMsg)) | ||
| msg = intercept[AnalysisException] { | ||
| SchemaUtils.checkColumnNameDuplication( | ||
| schema.map(_.name), "in SchemaUtilsSuite", caseSensitiveAnalysis = caseSensitive) | ||
| }.getMessage | ||
| assert(msg.contains(expectedErrorMsg)) | ||
| } | ||
|
|
||
| checkExceptionCases(s"$a0 INT, b INT, $a1 INT", a0 :: Nil) | ||
| checkExceptionCases(s"$a0 INT, b INT, $a1 INT, $a0 INT", a0 :: Nil) | ||
| checkExceptionCases(s"$a0 INT, $b0 INT, $a1 INT, $a0 INT, $b1 INT", b0 :: a0 :: Nil) | ||
| } | ||
| } | ||
|
|
||
| test("Check no exception thrown for valid schemas") { | ||
| def checkNoExceptionCases(schemaStr: String, caseSensitive: Boolean): Unit = { | ||
| val schema = StructType.fromDDL(schemaStr) | ||
| SchemaUtils.checkSchemaColumnNameDuplication( | ||
| schema, "in SchemaUtilsSuite", caseSensitiveAnalysis = caseSensitive) | ||
| SchemaUtils.checkColumnNameDuplication( | ||
| schema.map(_.name), "in SchemaUtilsSuite", resolver(caseSensitive)) | ||
| SchemaUtils.checkColumnNameDuplication( | ||
| schema.map(_.name), "in SchemaUtilsSuite", caseSensitiveAnalysis = caseSensitive) | ||
| } | ||
|
|
||
| checkNoExceptionCases("a INT, b INT, c INT", caseSensitive = true) | ||
| checkNoExceptionCases("Aa INT, b INT, aA INT", caseSensitive = true) | ||
|
|
||
| checkNoExceptionCases("a INT, b INT, c INT", caseSensitive = false) | ||
| } | ||
| } |
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -21,7 +21,6 @@ import java.io.IOException | |
|
|
||
| import org.apache.hadoop.fs.{FileSystem, Path} | ||
|
|
||
| import org.apache.spark.SparkContext | ||
| import org.apache.spark.internal.io.FileCommitProtocol | ||
| import org.apache.spark.sql._ | ||
| import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogTable, CatalogTablePartition} | ||
|
|
@@ -30,7 +29,7 @@ import org.apache.spark.sql.catalyst.expressions.Attribute | |
| import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan | ||
| import org.apache.spark.sql.execution.SparkPlan | ||
| import org.apache.spark.sql.execution.command._ | ||
| import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} | ||
| import org.apache.spark.sql.util.SchemaUtils | ||
|
|
||
| /** | ||
| * A command for writing data to a [[HadoopFsRelation]]. Supports both overwriting and appending. | ||
|
|
@@ -64,13 +63,10 @@ case class InsertIntoHadoopFsRelationCommand( | |
| assert(children.length == 1) | ||
|
|
||
| // Most formats don't do well with duplicate columns, so lets not allow that | ||
| if (query.schema.fieldNames.length != query.schema.fieldNames.distinct.length) { | ||
| val duplicateColumns = query.schema.fieldNames.groupBy(identity).collect { | ||
| case (x, ys) if ys.length > 1 => "\"" + x + "\"" | ||
| }.mkString(", ") | ||
| throw new AnalysisException(s"Duplicate column(s): $duplicateColumns found, " + | ||
| "cannot save to file.") | ||
| } | ||
| SchemaUtils.checkSchemaColumnNameDuplication( | ||
|
||
| query.schema, | ||
| s"when inserting into $outputPath", | ||
| sparkSession.sessionState.conf.caseSensitiveAnalysis) | ||
|
|
||
| val hadoopConf = sparkSession.sessionState.newHadoopConfWithOptions(options) | ||
| val fs = outputPath.getFileSystem(hadoopConf) | ||
|
|
||
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.
shall we put this check in the constructor of
DataSource? so it works for both read nad write pathUh 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.
Since
dataSchemaandpartitionSchemais firstly fixed by this function (getOrInferFileFormatSchema), IIUC we couldn't easily put this check in the constructer. If we put the check there, we need to move some code to decide schemas inside this function into the constructor. Thought?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.
Also, I think this check (https://github.com/apache/spark/pull/17758/files#diff-f9858c2d9d1a3c3e48753ef675bc865aR66) could cover write cases.
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.
this check also works for file-based data source, how about other data sources?
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 rechecked related code path again though, I couldn't find this issue (
dataSchemaandpartitionSchemahas duplicate column names) in other data sources. Actually, I think this issue happens in file-based data sources only when users directly write partition directories (e.g.,Seq(1, 2, 3).toDF("a").write.parquet(s"$path/a=1")).In catalog tables,
dataSchemaandpartitionSchemacouldn't has duplicate names in write paths;In stream sources, schemas have no partition (so, this issue does not happen) in read paths;
https://github.com/maropu/spark/blob/ad30aded7e95bb51d2028a4a21998c72c0338b3a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala#L210
https://github.com/maropu/spark/blob/ad30aded7e95bb51d2028a4a21998c72c0338b3a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala#L255
In stream sinks, since we assume partition columns is selected from data columns (https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSink.scala#L116), this issue does not happen.
So, IMHO the duplication check in
getOrInferFileFormatSchemais enough for this case.