From 1e647ee9c779d5d60fcadfe70a3dac05d48d8ccc Mon Sep 17 00:00:00 2001 From: Takeshi Yamamuro Date: Tue, 25 Apr 2017 14:24:48 +0900 Subject: [PATCH 01/27] Share code to check column name duplication --- .../execution/datasources/DataSource.scala | 6 +++ .../datasources/json/JsonDataSource.scala | 15 +----- .../sql/execution/datasources/rules.scala | 37 +++++++------- .../apache/spark/sql/util/SchemaUtil.scala | 51 +++++++++++++++++++ .../resources/sql-tests/inputs/create.sql | 5 ++ .../sql-tests/results/create.sql.out | 20 ++++++++ .../sql/execution/command/DDLSuite.scala | 8 +-- .../sql/test/DataFrameReaderWriterSuite.scala | 42 +++++++++++++++ .../spark/sql/hive/HiveMetastoreCatalog.scala | 5 ++ .../sql/hive/execution/HiveDDLSuite.scala | 2 +- 10 files changed, 154 insertions(+), 37 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/util/SchemaUtil.scala create mode 100644 sql/core/src/test/resources/sql-tests/inputs/create.sql create mode 100644 sql/core/src/test/resources/sql-tests/results/create.sql.out diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala index 75e530607570..3b04c207d540 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala @@ -188,6 +188,9 @@ case class DataSource( (dataSchema ++ partitionSchema).map(_.name), "in the data schema and the partition schema", sparkSession.sessionState.conf.caseSensitiveAnalysis) + SchemaUtils.checkSchemaColumnNameDuplication( + dataSchema, "datasource", sparkSession.sessionState.conf.caseSensitiveAnalysis) + (dataSchema, partitionSchema) } @@ -334,6 +337,9 @@ case class DataSource( "It must be specified manually") } + SchemaUtils.checkSchemaColumnNameDuplication( + dataSchema, "datasource", sparkSession.sessionState.conf.caseSensitiveAnalysis) + HadoopFsRelation( fileCatalog, partitionSchema = fileCatalog.partitionSchema, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala index 5a92a71d19e7..8b7c2709afde 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala @@ -59,9 +59,7 @@ abstract class JsonDataSource extends Serializable { inputPaths: Seq[FileStatus], parsedOptions: JSONOptions): Option[StructType] = { if (inputPaths.nonEmpty) { - val jsonSchema = infer(sparkSession, inputPaths, parsedOptions) - checkConstraints(jsonSchema) - Some(jsonSchema) + Some(infer(sparkSession, inputPaths, parsedOptions)) } else { None } @@ -71,17 +69,6 @@ abstract class JsonDataSource extends Serializable { sparkSession: SparkSession, inputPaths: Seq[FileStatus], parsedOptions: JSONOptions): StructType - - /** Constraints to be imposed on schema to be stored. */ - private def checkConstraints(schema: StructType): Unit = { - if (schema.fieldNames.length != schema.fieldNames.distinct.length) { - val duplicateColumns = schema.fieldNames.groupBy(identity).collect { - case (x, ys) if ys.length > 1 => "\"" + x + "\"" - }.mkString(", ") - throw new AnalysisException(s"Duplicate column(s) : $duplicateColumns found, " + - s"cannot save to JSON format") - } - } } object JsonDataSource { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala index 3f4a78580f1e..23aa14b72337 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala @@ -29,6 +29,7 @@ import org.apache.spark.sql.execution.command.DDLUtils import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources.InsertableRelation import org.apache.spark.sql.types.{AtomicType, StructType} +import org.apache.spark.sql.util.SchemaUtils /** * Try to replaces [[UnresolvedRelation]]s if the plan is for direct query on files. @@ -222,12 +223,10 @@ case class PreprocessTableCreation(sparkSession: SparkSession) extends Rule[Logi } private def normalizeCatalogTable(schema: StructType, table: CatalogTable): CatalogTable = { - val columnNames = if (sparkSession.sessionState.conf.caseSensitiveAnalysis) { - schema.map(_.name) - } else { - schema.map(_.name.toLowerCase) - } - checkDuplication(columnNames, "table definition of " + table.identifier) + SchemaUtils.checkSchemaColumnNameDuplication( + schema, + "table definition of " + table.identifier, + sparkSession.sessionState.conf.caseSensitiveAnalysis) val normalizedPartCols = normalizePartitionColumns(schema, table) val normalizedBucketSpec = normalizeBucketSpec(schema, table) @@ -253,7 +252,10 @@ case class PreprocessTableCreation(sparkSession: SparkSession) extends Rule[Logi partCols = table.partitionColumnNames, resolver = sparkSession.sessionState.conf.resolver) - checkDuplication(normalizedPartitionCols, "partition") + SchemaUtils.checkColumnNameDuplication( + normalizedPartitionCols, + "partition", + sparkSession.sessionState.conf.caseSensitiveAnalysis) if (schema.nonEmpty && normalizedPartitionCols.length == schema.length) { if (DDLUtils.isHiveTable(table)) { @@ -283,8 +285,16 @@ case class PreprocessTableCreation(sparkSession: SparkSession) extends Rule[Logi tableCols = schema.map(_.name), bucketSpec = bucketSpec, resolver = sparkSession.sessionState.conf.resolver) - checkDuplication(normalizedBucketSpec.bucketColumnNames, "bucket") - checkDuplication(normalizedBucketSpec.sortColumnNames, "sort") + + val caseSensitiveAnalysis = sparkSession.sessionState.conf.caseSensitiveAnalysis + SchemaUtils.checkColumnNameDuplication( + normalizedBucketSpec.bucketColumnNames, + "bucket", + caseSensitiveAnalysis) + SchemaUtils.checkColumnNameDuplication( + normalizedBucketSpec.sortColumnNames, + "sort", + caseSensitiveAnalysis) normalizedBucketSpec.sortColumnNames.map(schema(_)).map(_.dataType).foreach { case dt if RowOrdering.isOrderable(dt) => // OK @@ -297,15 +307,6 @@ case class PreprocessTableCreation(sparkSession: SparkSession) extends Rule[Logi } } - private def checkDuplication(colNames: Seq[String], colType: String): Unit = { - if (colNames.distinct.length != colNames.length) { - val duplicateColumns = colNames.groupBy(identity).collect { - case (x, ys) if ys.length > 1 => x - } - failAnalysis(s"Found duplicate column(s) in $colType: ${duplicateColumns.mkString(", ")}") - } - } - private def failAnalysis(msg: String) = throw new AnalysisException(msg) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/util/SchemaUtil.scala b/sql/core/src/main/scala/org/apache/spark/sql/util/SchemaUtil.scala new file mode 100644 index 000000000000..d4de97133738 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/util/SchemaUtil.scala @@ -0,0 +1,51 @@ +/* + * 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.sql.AnalysisException +import org.apache.spark.sql.types.StructType + + +/** + * Utils for handling schemas. + * + * TODO: Merge this file with [[org.apache.spark.ml.util.SchemaUtils]]. + */ +private[spark] object SchemaUtils { + + def checkSchemaColumnNameDuplication( + schema: StructType, colType: String, caseSensitiveAnalysis: Boolean = false): Unit = { + checkColumnNameDuplication(schema.map(_.name), colType, caseSensitiveAnalysis) + } + + def checkColumnNameDuplication( + names: Seq[String], colType: String, caseSensitiveAnalysis: Boolean = false): Unit = { + val colNames = if (caseSensitiveAnalysis) { + names + } else { + names.map(_.toLowerCase) + } + if (colNames.distinct.length != colNames.length) { + val duplicateColumns = colNames.groupBy(identity).collect { + case (x, ys) if ys.length > 1 => "\"" + x + "\"" + } + throw new AnalysisException(s"Found duplicate column(s) in $colType: " + + duplicateColumns.mkString(", ")) + } + } +} diff --git a/sql/core/src/test/resources/sql-tests/inputs/create.sql b/sql/core/src/test/resources/sql-tests/inputs/create.sql new file mode 100644 index 000000000000..dafcb8f81da0 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/create.sql @@ -0,0 +1,5 @@ +-- Check name duplication in a regular case +CREATE TABLE t (c STRING, c INT) USING parquet; + +-- Check multiple name duplication +CREATE TABLE t (c0 STRING, c1 INT, c1 DOUBLE, c0 INT) USING parquet; diff --git a/sql/core/src/test/resources/sql-tests/results/create.sql.out b/sql/core/src/test/resources/sql-tests/results/create.sql.out new file mode 100644 index 000000000000..427205469468 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/create.sql.out @@ -0,0 +1,20 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 2 + + +-- !query 0 +CREATE TABLE t (c STRING, c INT) USING parquet +-- !query 0 schema +struct<> +-- !query 0 output +org.apache.spark.sql.AnalysisException +Found duplicate column(s) in table definition of `t`: "c"; + + +-- !query 1 +CREATE TABLE t (c0 STRING, c1 INT, c1 DOUBLE, c0 INT) USING parquet +-- !query 1 schema +struct<> +-- !query 1 output +org.apache.spark.sql.AnalysisException +Found duplicate column(s) in table definition of `t`: "c1", "c0"; diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala index 5c40d8bb4b1e..f482d3846ead 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala @@ -439,13 +439,13 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils { val e = intercept[AnalysisException] { sql("CREATE TABLE tbl(a int, a string) USING json") } - assert(e.message == "Found duplicate column(s) in table definition of `tbl`: a") + assert(e.message == """Found duplicate column(s) in table definition of `tbl`: "a"""") withSQLConf(SQLConf.CASE_SENSITIVE.key -> "false") { val e2 = intercept[AnalysisException] { sql("CREATE TABLE tbl(a int, A string) USING json") } - assert(e2.message == "Found duplicate column(s) in table definition of `tbl`: a") + assert(e2.message == """Found duplicate column(s) in table definition of `tbl`: "a"""") } } @@ -469,14 +469,14 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils { val e = intercept[AnalysisException] { sql("CREATE TABLE tbl(a int) USING json PARTITIONED BY (a, a)") } - assert(e.message == "Found duplicate column(s) in partition: a") + assert(e.message == """Found duplicate column(s) in partition: "a"""") } test("create table - column repeated in bucket columns") { val e = intercept[AnalysisException] { sql("CREATE TABLE tbl(a int) USING json CLUSTERED BY (a, a) INTO 4 BUCKETS") } - assert(e.message == "Found duplicate column(s) in bucket: a") + assert(e.message == """Found duplicate column(s) in bucket: "a"""") } test("Refresh table after changing the data source table partitioning") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala index 306aecb5bbc8..cfed0d00e4c9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala @@ -687,4 +687,46 @@ class DataFrameReaderWriterSuite extends QueryTest with SharedSQLContext with Be testRead(spark.read.schema(userSchemaString).text(dir, dir), data ++ data, userSchema) testRead(spark.read.schema(userSchemaString).text(Seq(dir, dir): _*), data ++ data, userSchema) } + + test("SPARK-20460 Check name duplication in schema") { + withTempDir { src => + val columnDuplicateSchema = StructType( + StructField("a", IntegerType) :: + StructField("a", IntegerType) :: + Nil) + + // Check CSV format + Seq("a,a", "1,1").toDF().coalesce(1).write.mode("overwrite").text(src.toString) + val e1 = intercept[AnalysisException] { + spark.read.format("csv").schema(columnDuplicateSchema).option("header", false) + .load(src.toString) + } + assert(e1.getMessage.contains("""Found duplicate column(s) in datasource: "a";""")) + + // If `inferSchema` is true, a CSV format is duplicate-safe (See SPARK-16896) + val df = spark.read.format("csv").option("inferSchema", true).option("header", true) + .load(src.toString) + checkAnswer(df, Row(1, 1)) + + // Check JSON format + Seq("""{"a":1, "a":1}"""""").toDF().coalesce(1).write.mode("overwrite").text(src.toString) + val e2 = intercept[AnalysisException] { + spark.read.format("json").schema(columnDuplicateSchema).option("header", false) + .load(src.toString) + } + assert(e2.getMessage.contains("""Found duplicate column(s) in datasource: "a";""")) + + val e3 = intercept[AnalysisException] { + spark.read.format("json").option("inferSchema", true).load(src.toString) + } + assert(e3.getMessage.contains("""Found duplicate column(s) in datasource: "a";""")) + + // Check Paruqet format + Seq((1, 1)).toDF("a", "b").coalesce(1).write.mode("overwrite").parquet(src.toString) + val e4 = intercept[AnalysisException] { + spark.read.format("parquet").schema(columnDuplicateSchema).load(src.toString) + } + assert(e4.getMessage.contains("""Found duplicate column(s) in datasource: "a";""")) + } + } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index 808dc013f170..e404517e8c56 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -31,6 +31,7 @@ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.internal.SQLConf.HiveCaseSensitiveInferenceMode._ import org.apache.spark.sql.types._ +import org.apache.spark.sql.util.SchemaUtils /** * Legacy catalog for interacting with the Hive metastore. @@ -248,6 +249,10 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log if (inferenceMode == INFER_AND_SAVE) { updateCatalogSchema(relation.tableMeta.identifier, schema) } + + SchemaUtils.checkSchemaColumnNameDuplication( + schema, "hive serde table", sparkSession.sessionState.conf.caseSensitiveAnalysis) + (schema, relation.tableMeta.copy(schema = schema)) case None => logWarning(s"Unable to infer schema for table $tableName from file format " + diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala index 31fa3d244746..0b81e60887a5 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala @@ -345,7 +345,7 @@ class HiveDDLSuite val e = intercept[AnalysisException] { sql("CREATE TABLE tbl(a int) PARTITIONED BY (a string)") } - assert(e.message == "Found duplicate column(s) in table definition of `default`.`tbl`: a") + assert(e.message == """Found duplicate column(s) in table definition of `default`.`tbl`: "a"""") } test("add/drop partition with location - managed table") { From 4467077b1309222c99a9ba56ee9c442689cc3624 Mon Sep 17 00:00:00 2001 From: Takeshi Yamamuro Date: Tue, 13 Jun 2017 15:26:04 +0900 Subject: [PATCH 02/27] Apply reviews --- .../{SchemaUtil.scala => SchemaUtils.scala} | 0 .../resources/sql-tests/inputs/create.sql | 4 + .../sql-tests/results/create.sql.out | 19 ++++- .../sql/test/DataFrameReaderWriterSuite.scala | 55 +++++++------ .../spark/sql/util/SchemaUtilsSuite.scala | 81 +++++++++++++++++++ 5 files changed, 134 insertions(+), 25 deletions(-) rename sql/core/src/main/scala/org/apache/spark/sql/util/{SchemaUtil.scala => SchemaUtils.scala} (100%) create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/util/SchemaUtilsSuite.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/util/SchemaUtil.scala b/sql/core/src/main/scala/org/apache/spark/sql/util/SchemaUtils.scala similarity index 100% rename from sql/core/src/main/scala/org/apache/spark/sql/util/SchemaUtil.scala rename to sql/core/src/main/scala/org/apache/spark/sql/util/SchemaUtils.scala diff --git a/sql/core/src/test/resources/sql-tests/inputs/create.sql b/sql/core/src/test/resources/sql-tests/inputs/create.sql index dafcb8f81da0..60177ea504a2 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/create.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/create.sql @@ -3,3 +3,7 @@ CREATE TABLE t (c STRING, c INT) USING parquet; -- Check multiple name duplication CREATE TABLE t (c0 STRING, c1 INT, c1 DOUBLE, c0 INT) USING parquet; + +-- Catch case-insensitive name duplication +SET spark.sql.caseSensitive=false; +CREATE TABLE t (ab STRING, cd INT, ef DOUBLE, Ab INT) USING parquet; diff --git a/sql/core/src/test/resources/sql-tests/results/create.sql.out b/sql/core/src/test/resources/sql-tests/results/create.sql.out index 427205469468..5fe9f81d7e41 100644 --- a/sql/core/src/test/resources/sql-tests/results/create.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/create.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 2 +-- Number of queries: 4 -- !query 0 @@ -18,3 +18,20 @@ struct<> -- !query 1 output org.apache.spark.sql.AnalysisException Found duplicate column(s) in table definition of `t`: "c1", "c0"; + + +-- !query 2 +SET spark.sql.caseSensitive=false +-- !query 2 schema +struct +-- !query 2 output +spark.sql.caseSensitive false + + +-- !query 3 +CREATE TABLE t (ab STRING, cd INT, ef DOUBLE, Ab INT) USING parquet +-- !query 3 schema +struct<> +-- !query 3 output +org.apache.spark.sql.AnalysisException +Found duplicate column(s) in table definition of `t`: "ab"; diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala index cfed0d00e4c9..3ef94fb8274f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala @@ -27,6 +27,7 @@ import org.apache.spark.internal.io.FileCommitProtocol.TaskCommitMessage import org.apache.spark.internal.io.HadoopMapReduceCommitProtocol import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources._ import org.apache.spark.sql.types._ import org.apache.spark.util.Utils @@ -690,43 +691,49 @@ class DataFrameReaderWriterSuite extends QueryTest with SharedSQLContext with Be test("SPARK-20460 Check name duplication in schema") { withTempDir { src => - val columnDuplicateSchema = StructType( - StructField("a", IntegerType) :: - StructField("a", IntegerType) :: - Nil) - // Check CSV format - Seq("a,a", "1,1").toDF().coalesce(1).write.mode("overwrite").text(src.toString) - val e1 = intercept[AnalysisException] { - spark.read.format("csv").schema(columnDuplicateSchema).option("header", false) - .load(src.toString) + Seq("1,1").toDF().coalesce(1).write.mode("overwrite").text(src.toString) + Seq((true, "a INT, a INT"), (false, "aA INT, Aa INT")).map { case (caseSensitive, schema) => + withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive.toString) { + val e = intercept[AnalysisException] { + spark.read.format("csv").schema(schema).load(src.toString) + } + assert(e.getMessage.contains("Found duplicate column(s) in datasource: ")) + } } - assert(e1.getMessage.contains("""Found duplicate column(s) in datasource: "a";""")) // If `inferSchema` is true, a CSV format is duplicate-safe (See SPARK-16896) + Seq("a,a", "1,1").toDF().coalesce(1).write.mode("overwrite").text(src.toString) val df = spark.read.format("csv").option("inferSchema", true).option("header", true) .load(src.toString) checkAnswer(df, Row(1, 1)) // Check JSON format - Seq("""{"a":1, "a":1}"""""").toDF().coalesce(1).write.mode("overwrite").text(src.toString) - val e2 = intercept[AnalysisException] { - spark.read.format("json").schema(columnDuplicateSchema).option("header", false) - .load(src.toString) - } - assert(e2.getMessage.contains("""Found duplicate column(s) in datasource: "a";""")) - - val e3 = intercept[AnalysisException] { - spark.read.format("json").option("inferSchema", true).load(src.toString) + Seq((true, ("a", "a")), (false, ("aA", "Aa"))).map { case (caseSensitive, (c0, c1)) => + Seq(s"""{"$c0":1, "$c1":1}""").toDF().coalesce(1).write.mode("overwrite").text(src.toString) + withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive.toString) { + val e1 = intercept[AnalysisException] { + spark.read.format("json").schema(s"$c0 INT, $c1 INT").load(src.toString) + } + assert(e1.getMessage.contains("Found duplicate column(s) in datasource: ")) + + val e2 = intercept[AnalysisException] { + spark.read.format("json").option("inferSchema", true).load(src.toString) + } + assert(e2.getMessage.contains("Found duplicate column(s) in datasource: ")) + } } - assert(e3.getMessage.contains("""Found duplicate column(s) in datasource: "a";""")) // Check Paruqet format - Seq((1, 1)).toDF("a", "b").coalesce(1).write.mode("overwrite").parquet(src.toString) - val e4 = intercept[AnalysisException] { - spark.read.format("parquet").schema(columnDuplicateSchema).load(src.toString) + Seq((1, 1)).toDF("c0", "c1").coalesce(1).write.mode("overwrite").parquet(src.toString) + Seq((true, "a INT, a INT"), (false, "aA INT, Aa INT")).map { case (caseSensitive, schema) => + withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive.toString) { + val e = intercept[AnalysisException] { + spark.read.format("parquet").schema(schema).load(src.toString) + } + assert(e.getMessage.contains("Found duplicate column(s) in datasource: ")) + } } - assert(e4.getMessage.contains("""Found duplicate column(s) in datasource: "a";""")) } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/util/SchemaUtilsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/util/SchemaUtilsSuite.scala new file mode 100644 index 000000000000..1cec6c35beab --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/util/SchemaUtilsSuite.scala @@ -0,0 +1,81 @@ +/* + * 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.types.StructType + +class SchemaUtilsSuite extends SparkFunSuite { + + test("Check column name duplication") { + // Case-sensitive case + val msg1 = intercept[AnalysisException] { + SchemaUtils.checkSchemaColumnNameDuplication( + StructType.fromDDL("a INT, b INT, a INT"), "SchemaUtilsSuite", + caseSensitiveAnalysis = true) + }.getMessage + assert(msg1.contains("""Found duplicate column(s) in SchemaUtilsSuite: "a";""")) + val msg2 = intercept[AnalysisException] { + SchemaUtils.checkColumnNameDuplication( + "a" :: "b" :: "a" :: Nil, "SchemaUtilsSuite", caseSensitiveAnalysis = true) + }.getMessage + assert(msg2.contains("""Found duplicate column(s) in SchemaUtilsSuite: "a";""")) + + // Check no exception thrown + SchemaUtils.checkSchemaColumnNameDuplication( + StructType.fromDDL("a INT, b INT, c INT"), "SchemaUtilsSuite", caseSensitiveAnalysis = true) + SchemaUtils.checkColumnNameDuplication( + "a" :: "b" :: "c" :: Nil, "SchemaUtilsSuite", caseSensitiveAnalysis = true) + SchemaUtils.checkSchemaColumnNameDuplication( + StructType.fromDDL("Aa INT, b INT, aA INT"), "SchemaUtilsSuite", caseSensitiveAnalysis = true) + SchemaUtils.checkColumnNameDuplication( + "Aa" :: "b" :: "aA" :: Nil, "SchemaUtilsSuite", caseSensitiveAnalysis = true) + + // Case-insensitive case + val msg3 = intercept[AnalysisException] { + SchemaUtils.checkSchemaColumnNameDuplication( + StructType.fromDDL("Aa INT, b INT, Aa INT"), "SchemaUtilsSuite", + caseSensitiveAnalysis = false) + }.getMessage + assert(msg3.contains("""Found duplicate column(s) in SchemaUtilsSuite: "aa";""")) + val msg4 = intercept[AnalysisException] { + SchemaUtils.checkColumnNameDuplication( + "Aa" :: "b" :: "Aa" :: Nil, "SchemaUtilsSuite", caseSensitiveAnalysis = false) + }.getMessage + assert(msg4.contains("""Found duplicate column(s) in SchemaUtilsSuite: "aa";""")) + + val msg5 = intercept[AnalysisException] { + SchemaUtils.checkSchemaColumnNameDuplication( + StructType.fromDDL("a INT, bB INT, Bb INT"), "SchemaUtilsSuite", + caseSensitiveAnalysis = false) + }.getMessage + assert(msg5.contains("""Found duplicate column(s) in SchemaUtilsSuite: "bb";""")) + val msg6 = intercept[AnalysisException] { + SchemaUtils.checkColumnNameDuplication( + "a" :: "bB" :: "Bb" :: Nil, "SchemaUtilsSuite", caseSensitiveAnalysis = false) + }.getMessage + assert(msg6.contains("""Found duplicate column(s) in SchemaUtilsSuite: "bb";""")) + + // Check no exception thrown + SchemaUtils.checkSchemaColumnNameDuplication( + StructType.fromDDL("a INT, b INT, c INT"), "SchemaUtilsSuite", caseSensitiveAnalysis = false) + SchemaUtils.checkColumnNameDuplication( + "a" :: "b" :: "c" :: Nil, "SchemaUtilsSuite", caseSensitiveAnalysis = false) + } +} From 33ab2173394265d021f7dc0d3e2f390af291b5d7 Mon Sep 17 00:00:00 2001 From: Takeshi Yamamuro Date: Tue, 13 Jun 2017 18:04:25 +0900 Subject: [PATCH 03/27] Make code more consistent --- .../sql/catalyst/catalog/SessionCatalog.scala | 20 ++------ .../apache/spark/sql/util/SchemaUtils.scala | 36 +++++++------ .../spark/sql/util/SchemaUtilsSuite.scala | 0 .../spark/sql/execution/command/ddl.scala | 12 +++-- .../spark/sql/execution/command/tables.scala | 3 +- .../spark/sql/execution/command/views.scala | 7 +-- .../InsertIntoHadoopFsRelationCommand.scala | 10 ++-- .../datasources/PartitioningUtils.scala | 13 ++--- .../datasources/jdbc/JdbcUtils.scala | 10 +--- .../sql/execution/datasources/rules.scala | 2 +- .../apache/spark/sql/util/SchemaUtils.scala | 51 ------------------- .../org/apache/spark/sql/DataFrameSuite.scala | 4 +- .../spark/sql/jdbc/JDBCWriteSuite.scala | 2 +- 13 files changed, 50 insertions(+), 120 deletions(-) rename sql/{core => catalyst}/src/test/scala/org/apache/spark/sql/util/SchemaUtilsSuite.scala (100%) delete mode 100644 sql/core/src/main/scala/org/apache/spark/sql/util/SchemaUtils.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala index 336d3d65d0dd..2fcd103a87fd 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala @@ -39,7 +39,8 @@ import org.apache.spark.sql.catalyst.parser.{CatalystSqlParser, ParserInterface} import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SubqueryAlias, View} import org.apache.spark.sql.catalyst.util.StringUtils import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.types.{StructField, StructType} +import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.util.SchemaUtils object SessionCatalog { val DEFAULT_DATABASE = "default" @@ -188,19 +189,6 @@ class SessionCatalog( } } - private def checkDuplication(fields: Seq[StructField]): Unit = { - val columnNames = if (conf.caseSensitiveAnalysis) { - fields.map(_.name) - } else { - fields.map(_.name.toLowerCase) - } - if (columnNames.distinct.length != columnNames.length) { - val duplicateColumns = columnNames.groupBy(identity).collect { - case (x, ys) if ys.length > 1 => x - } - throw new AnalysisException(s"Found duplicate column(s): ${duplicateColumns.mkString(", ")}") - } - } // ---------------------------------------------------------------------------- // Databases // ---------------------------------------------------------------------------- @@ -353,7 +341,9 @@ class SessionCatalog( val tableIdentifier = TableIdentifier(table, Some(db)) requireDbExists(db) requireTableExists(tableIdentifier) - checkDuplication(newSchema) + + SchemaUtils.checkSchemaColumnNameDuplication( + newSchema, "table definition of " + tableIdentifier.identifier, conf.caseSensitiveAnalysis) val catalogTable = externalCatalog.getTable(db, table) val oldSchema = catalogTable.schema diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/util/SchemaUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/util/SchemaUtils.scala index e881685ce626..d4de97133738 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/util/SchemaUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/util/SchemaUtils.scala @@ -17,7 +17,8 @@ package org.apache.spark.sql.util -import org.apache.spark.internal.Logging +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.types.StructType /** @@ -25,29 +26,26 @@ import org.apache.spark.internal.Logging * * TODO: Merge this file with [[org.apache.spark.ml.util.SchemaUtils]]. */ -private[spark] object SchemaUtils extends Logging { +private[spark] object SchemaUtils { + + def checkSchemaColumnNameDuplication( + schema: StructType, colType: String, caseSensitiveAnalysis: Boolean = false): Unit = { + checkColumnNameDuplication(schema.map(_.name), colType, caseSensitiveAnalysis) + } - /** - * Checks if input column names have duplicate identifiers. Prints a warning message if - * the duplication exists. - * - * @param columnNames column names to check - * @param colType column type name, used in a warning message - * @param caseSensitiveAnalysis whether duplication checks should be case sensitive or not - */ def checkColumnNameDuplication( - columnNames: Seq[String], colType: String, caseSensitiveAnalysis: Boolean): Unit = { - val names = if (caseSensitiveAnalysis) { - columnNames + names: Seq[String], colType: String, caseSensitiveAnalysis: Boolean = false): Unit = { + val colNames = if (caseSensitiveAnalysis) { + names } else { - columnNames.map(_.toLowerCase) + names.map(_.toLowerCase) } - if (names.distinct.length != names.length) { - val duplicateColumns = names.groupBy(identity).collect { - case (x, ys) if ys.length > 1 => s"`$x`" + if (colNames.distinct.length != colNames.length) { + val duplicateColumns = colNames.groupBy(identity).collect { + case (x, ys) if ys.length > 1 => "\"" + x + "\"" } - logWarning(s"Found duplicate column(s) $colType: ${duplicateColumns.mkString(", ")}. " + - "You might need to assign different column names.") + throw new AnalysisException(s"Found duplicate column(s) in $colType: " + + duplicateColumns.mkString(", ")) } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/util/SchemaUtilsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/util/SchemaUtilsSuite.scala similarity index 100% rename from sql/core/src/test/scala/org/apache/spark/sql/util/SchemaUtilsSuite.scala rename to sql/catalyst/src/test/scala/org/apache/spark/sql/util/SchemaUtilsSuite.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala index ba7ca84f229f..91daf5328e98 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala @@ -430,7 +430,8 @@ case class AlterTableAddPartitionCommand( spec, table.partitionColumnNames, table.identifier.quotedString, - sparkSession.sessionState.conf.resolver) + sparkSession.sessionState.conf.resolver, + sparkSession.sessionState.conf.caseSensitiveAnalysis) // inherit table storage format (possibly except for location) CatalogTablePartition(normalizedSpec, table.storage.copy( locationUri = location.map(CatalogUtils.stringToURI))) @@ -480,13 +481,15 @@ case class AlterTableRenamePartitionCommand( oldPartition, table.partitionColumnNames, table.identifier.quotedString, - sparkSession.sessionState.conf.resolver) + sparkSession.sessionState.conf.resolver, + sparkSession.sessionState.conf.caseSensitiveAnalysis) val normalizedNewPartition = PartitioningUtils.normalizePartitionSpec( newPartition, table.partitionColumnNames, table.identifier.quotedString, - sparkSession.sessionState.conf.resolver) + sparkSession.sessionState.conf.resolver, + sparkSession.sessionState.conf.caseSensitiveAnalysis) catalog.renamePartitions( tableName, Seq(normalizedOldPartition), Seq(normalizedNewPartition)) @@ -528,7 +531,8 @@ case class AlterTableDropPartitionCommand( spec, table.partitionColumnNames, table.identifier.quotedString, - sparkSession.sessionState.conf.resolver) + sparkSession.sessionState.conf.resolver, + sparkSession.sessionState.conf.caseSensitiveAnalysis) } catalog.dropPartitions( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala index 8ded1060f7bf..1c98355a69e3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala @@ -449,7 +449,8 @@ case class TruncateTableCommand( spec, partCols, table.identifier.quotedString, - spark.sessionState.conf.resolver) + spark.sessionState.conf.resolver, + spark.sessionState.conf.caseSensitiveAnalysis) } val partLocations = catalog.listPartitions(table.identifier, normalizedSpec).map(_.storage.locationUri) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala index a6d56ca91a3e..c8fab4f992f9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala @@ -27,6 +27,7 @@ import org.apache.spark.sql.catalyst.expressions.{Alias, SubqueryExpression} import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project, View} import org.apache.spark.sql.types.MetadataBuilder +import org.apache.spark.sql.util.SchemaUtils /** @@ -357,12 +358,12 @@ object ViewHelper { analyzedPlan: LogicalPlan): Map[String, String] = { // Generate the query column names, throw an AnalysisException if there exists duplicate column // names. - val queryOutput = analyzedPlan.schema.fieldNames - assert(queryOutput.distinct.size == queryOutput.size, - s"The view output ${queryOutput.mkString("(", ",", ")")} contains duplicate column name.") + SchemaUtils.checkSchemaColumnNameDuplication( + analyzedPlan.schema, "view", session.sessionState.conf.caseSensitiveAnalysis) // Generate the view default database name. val viewDefaultDatabase = session.sessionState.catalog.getCurrentDatabase + val queryOutput = analyzedPlan.schema.fieldNames removeQueryColumnNames(properties) ++ generateViewDefaultDatabase(viewDefaultDatabase) ++ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala index 0031567d3d28..05d2bd92de53 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala @@ -31,6 +31,7 @@ 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 +65,8 @@ 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, "query", sparkSession.sessionState.conf.caseSensitiveAnalysis) val hadoopConf = sparkSession.sessionState.newHadoopConfWithOptions(options) val fs = outputPath.getFileSystem(hadoopConf) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala index f61c673baaa5..45eecfa6e8f1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala @@ -33,6 +33,7 @@ import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec import org.apache.spark.sql.catalyst.expressions.{Cast, Literal} import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.types._ +import org.apache.spark.sql.util.SchemaUtils // TODO: We should tighten up visibility of the classes here once we clean up Hive coupling. @@ -293,7 +294,8 @@ object PartitioningUtils { partitionSpec: Map[String, T], partColNames: Seq[String], tblName: String, - resolver: Resolver): Map[String, T] = { + resolver: Resolver, + caseSensitiveAnalysis: Boolean): Map[String, T] = { val normalizedPartSpec = partitionSpec.toSeq.map { case (key, value) => val normalizedKey = partColNames.find(resolver(_, key)).getOrElse { throw new AnalysisException(s"$key is not a valid partition column in table $tblName.") @@ -301,13 +303,8 @@ object PartitioningUtils { normalizedKey -> value } - if (normalizedPartSpec.map(_._1).distinct.length != normalizedPartSpec.length) { - val duplicateColumns = normalizedPartSpec.map(_._1).groupBy(identity).collect { - case (x, ys) if ys.length > 1 => x - } - throw new AnalysisException(s"Found duplicated columns in partition specification: " + - duplicateColumns.mkString(", ")) - } + SchemaUtils.checkColumnNameDuplication( + normalizedPartSpec.map(_._1), "partition specification", caseSensitiveAnalysis) normalizedPartSpec.toMap } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala index 55b2539c1338..d7235ca27979 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala @@ -35,6 +35,7 @@ import org.apache.spark.sql.catalyst.parser.CatalystSqlParser import org.apache.spark.sql.catalyst.util.{CaseInsensitiveMap, DateTimeUtils, GenericArrayData} import org.apache.spark.sql.jdbc.{JdbcDialect, JdbcDialects, JdbcType} import org.apache.spark.sql.types._ +import org.apache.spark.sql.util.SchemaUtils import org.apache.spark.unsafe.types.UTF8String import org.apache.spark.util.NextIterator @@ -749,14 +750,7 @@ object JdbcUtils extends Logging { val nameEquality = df.sparkSession.sessionState.conf.resolver // checks duplicate columns in the user specified column types. - userSchema.fieldNames.foreach { col => - val duplicatesCols = userSchema.fieldNames.filter(nameEquality(_, col)) - if (duplicatesCols.size >= 2) { - throw new AnalysisException( - "Found duplicate column(s) in createTableColumnTypes option value: " + - duplicatesCols.mkString(", ")) - } - } + SchemaUtils.checkSchemaColumnNameDuplication(userSchema, "createTableColumnTypes option value") // checks if user specified column names exist in the DataFrame schema userSchema.fieldNames.foreach { col => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala index 23aa14b72337..783726f3ca21 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala @@ -323,7 +323,7 @@ case class PreprocessTableInsertion(conf: SQLConf) extends Rule[LogicalPlan] wit partColNames: Seq[String]): InsertIntoTable = { val normalizedPartSpec = PartitioningUtils.normalizePartitionSpec( - insert.partition, partColNames, tblName, conf.resolver) + insert.partition, partColNames, tblName, conf.resolver, conf.caseSensitiveAnalysis) val staticPartCols = normalizedPartSpec.filter(_._2.isDefined).keySet val expectedColumns = insert.table.output.filterNot(a => staticPartCols.contains(a.name)) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/util/SchemaUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/util/SchemaUtils.scala deleted file mode 100644 index d4de97133738..000000000000 --- a/sql/core/src/main/scala/org/apache/spark/sql/util/SchemaUtils.scala +++ /dev/null @@ -1,51 +0,0 @@ -/* - * 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.sql.AnalysisException -import org.apache.spark.sql.types.StructType - - -/** - * Utils for handling schemas. - * - * TODO: Merge this file with [[org.apache.spark.ml.util.SchemaUtils]]. - */ -private[spark] object SchemaUtils { - - def checkSchemaColumnNameDuplication( - schema: StructType, colType: String, caseSensitiveAnalysis: Boolean = false): Unit = { - checkColumnNameDuplication(schema.map(_.name), colType, caseSensitiveAnalysis) - } - - def checkColumnNameDuplication( - names: Seq[String], colType: String, caseSensitiveAnalysis: Boolean = false): Unit = { - val colNames = if (caseSensitiveAnalysis) { - names - } else { - names.map(_.toLowerCase) - } - if (colNames.distinct.length != colNames.length) { - val duplicateColumns = colNames.groupBy(identity).collect { - case (x, ys) if ys.length > 1 => "\"" + x + "\"" - } - throw new AnalysisException(s"Found duplicate column(s) in $colType: " + - duplicateColumns.mkString(", ")) - } - } -} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala index 9ea9951c24ef..dff4d516ecbc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala @@ -1123,7 +1123,7 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { Seq((1, 2, 3), (2, 3, 4), (3, 4, 5)).toDF("column1", "column2", "column1") .write.format("parquet").save("temp") } - assert(e.getMessage.contains("Duplicate column(s)")) + assert(e.getMessage.contains("Found duplicate column(s) in query: ")) assert(e.getMessage.contains("column1")) assert(!e.getMessage.contains("column2")) @@ -1133,7 +1133,7 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { .toDF("column1", "column2", "column3", "column1", "column3") .write.format("json").save("temp") } - assert(f.getMessage.contains("Duplicate column(s)")) + assert(f.getMessage.contains("Found duplicate column(s) in query: ")) assert(f.getMessage.contains("column1")) assert(f.getMessage.contains("column3")) assert(!f.getMessage.contains("column2")) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala index 92f50a095f19..3ca2804f5b96 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala @@ -479,7 +479,7 @@ class JDBCWriteSuite extends SharedSQLContext with BeforeAndAfter { .jdbc(url1, "TEST.USERDBTYPETEST", properties) }.getMessage() assert(msg.contains( - "Found duplicate column(s) in createTableColumnTypes option value: name, NaMe")) + """Found duplicate column(s) in createTableColumnTypes option value: "name";""")) } } From d8efb9d17fbc75d4d756f66bd84a3d9697ee0c0b Mon Sep 17 00:00:00 2001 From: Takeshi Yamamuro Date: Fri, 16 Jun 2017 10:21:37 +0900 Subject: [PATCH 04/27] Apply review comments --- .../apache/spark/sql/util/SchemaUtils.scala | 29 ++++++++++++------- .../spark/sql/util/SchemaUtilsSuite.scala | 21 +++++++------- .../spark/sql/execution/command/ddl.scala | 12 +++----- .../spark/sql/execution/command/tables.scala | 3 +- .../datasources/PartitioningUtils.scala | 5 ++-- .../datasources/jdbc/JdbcUtils.scala | 3 +- .../sql/execution/datasources/rules.scala | 9 +++--- .../resources/sql-tests/inputs/create.sql | 8 +++-- .../sql-tests/results/create.sql.out | 22 +++++++++----- 9 files changed, 62 insertions(+), 50 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/util/SchemaUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/util/SchemaUtils.scala index d4de97133738..33280fbb369f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/util/SchemaUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/util/SchemaUtils.scala @@ -17,7 +17,10 @@ package org.apache.spark.sql.util +import scala.collection.mutable + import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.types.StructType @@ -30,22 +33,26 @@ private[spark] object SchemaUtils { def checkSchemaColumnNameDuplication( schema: StructType, colType: String, caseSensitiveAnalysis: Boolean = false): Unit = { - checkColumnNameDuplication(schema.map(_.name), colType, caseSensitiveAnalysis) + val resolver = if (caseSensitiveAnalysis) { + caseSensitiveResolution + } else { + caseInsensitiveResolution + } + checkColumnNameDuplication(schema.map(_.name), colType, resolver) } def checkColumnNameDuplication( - names: Seq[String], colType: String, caseSensitiveAnalysis: Boolean = false): Unit = { - val colNames = if (caseSensitiveAnalysis) { - names - } else { - names.map(_.toLowerCase) - } - if (colNames.distinct.length != colNames.length) { - val duplicateColumns = colNames.groupBy(identity).collect { - case (x, ys) if ys.length > 1 => "\"" + x + "\"" + names: Seq[String], colType: String, resolver: Resolver): Unit = { + val duplicateColumns = mutable.ArrayBuffer[String]() + names.foreach { name => + val sameColNames = names.filter(resolver(_, name)) + if (sameColNames.size > 1 && !duplicateColumns.exists(resolver(_, name))) { + duplicateColumns.append(name) } + } + if (duplicateColumns.size > 0) { throw new AnalysisException(s"Found duplicate column(s) in $colType: " + - duplicateColumns.mkString(", ")) + duplicateColumns.map(colName => s""""$colName"""").mkString(", ")) } } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/util/SchemaUtilsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/util/SchemaUtilsSuite.scala index 1cec6c35beab..21e38fabed7d 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/util/SchemaUtilsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/util/SchemaUtilsSuite.scala @@ -19,6 +19,7 @@ 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 { @@ -33,7 +34,7 @@ class SchemaUtilsSuite extends SparkFunSuite { assert(msg1.contains("""Found duplicate column(s) in SchemaUtilsSuite: "a";""")) val msg2 = intercept[AnalysisException] { SchemaUtils.checkColumnNameDuplication( - "a" :: "b" :: "a" :: Nil, "SchemaUtilsSuite", caseSensitiveAnalysis = true) + "a" :: "b" :: "a" :: Nil, "SchemaUtilsSuite", caseSensitiveResolution) }.getMessage assert(msg2.contains("""Found duplicate column(s) in SchemaUtilsSuite: "a";""")) @@ -41,11 +42,11 @@ class SchemaUtilsSuite extends SparkFunSuite { SchemaUtils.checkSchemaColumnNameDuplication( StructType.fromDDL("a INT, b INT, c INT"), "SchemaUtilsSuite", caseSensitiveAnalysis = true) SchemaUtils.checkColumnNameDuplication( - "a" :: "b" :: "c" :: Nil, "SchemaUtilsSuite", caseSensitiveAnalysis = true) + "a" :: "b" :: "c" :: Nil, "SchemaUtilsSuite", caseSensitiveResolution) SchemaUtils.checkSchemaColumnNameDuplication( StructType.fromDDL("Aa INT, b INT, aA INT"), "SchemaUtilsSuite", caseSensitiveAnalysis = true) SchemaUtils.checkColumnNameDuplication( - "Aa" :: "b" :: "aA" :: Nil, "SchemaUtilsSuite", caseSensitiveAnalysis = true) + "Aa" :: "b" :: "aA" :: Nil, "SchemaUtilsSuite", caseSensitiveResolution) // Case-insensitive case val msg3 = intercept[AnalysisException] { @@ -53,29 +54,29 @@ class SchemaUtilsSuite extends SparkFunSuite { StructType.fromDDL("Aa INT, b INT, Aa INT"), "SchemaUtilsSuite", caseSensitiveAnalysis = false) }.getMessage - assert(msg3.contains("""Found duplicate column(s) in SchemaUtilsSuite: "aa";""")) + assert(msg3.contains("""Found duplicate column(s) in SchemaUtilsSuite: "Aa";""")) val msg4 = intercept[AnalysisException] { SchemaUtils.checkColumnNameDuplication( - "Aa" :: "b" :: "Aa" :: Nil, "SchemaUtilsSuite", caseSensitiveAnalysis = false) + "Aa" :: "b" :: "Aa" :: Nil, "SchemaUtilsSuite", caseInsensitiveResolution) }.getMessage - assert(msg4.contains("""Found duplicate column(s) in SchemaUtilsSuite: "aa";""")) + assert(msg4.contains("""Found duplicate column(s) in SchemaUtilsSuite: "Aa";""")) val msg5 = intercept[AnalysisException] { SchemaUtils.checkSchemaColumnNameDuplication( StructType.fromDDL("a INT, bB INT, Bb INT"), "SchemaUtilsSuite", caseSensitiveAnalysis = false) }.getMessage - assert(msg5.contains("""Found duplicate column(s) in SchemaUtilsSuite: "bb";""")) + assert(msg5.contains("""Found duplicate column(s) in SchemaUtilsSuite: "bB";""")) val msg6 = intercept[AnalysisException] { SchemaUtils.checkColumnNameDuplication( - "a" :: "bB" :: "Bb" :: Nil, "SchemaUtilsSuite", caseSensitiveAnalysis = false) + "a" :: "bB" :: "Bb" :: Nil, "SchemaUtilsSuite", caseInsensitiveResolution) }.getMessage - assert(msg6.contains("""Found duplicate column(s) in SchemaUtilsSuite: "bb";""")) + assert(msg6.contains("""Found duplicate column(s) in SchemaUtilsSuite: "bB";""")) // Check no exception thrown SchemaUtils.checkSchemaColumnNameDuplication( StructType.fromDDL("a INT, b INT, c INT"), "SchemaUtilsSuite", caseSensitiveAnalysis = false) SchemaUtils.checkColumnNameDuplication( - "a" :: "b" :: "c" :: Nil, "SchemaUtilsSuite", caseSensitiveAnalysis = false) + "a" :: "b" :: "c" :: Nil, "SchemaUtilsSuite", caseInsensitiveResolution) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala index 91daf5328e98..ba7ca84f229f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala @@ -430,8 +430,7 @@ case class AlterTableAddPartitionCommand( spec, table.partitionColumnNames, table.identifier.quotedString, - sparkSession.sessionState.conf.resolver, - sparkSession.sessionState.conf.caseSensitiveAnalysis) + sparkSession.sessionState.conf.resolver) // inherit table storage format (possibly except for location) CatalogTablePartition(normalizedSpec, table.storage.copy( locationUri = location.map(CatalogUtils.stringToURI))) @@ -481,15 +480,13 @@ case class AlterTableRenamePartitionCommand( oldPartition, table.partitionColumnNames, table.identifier.quotedString, - sparkSession.sessionState.conf.resolver, - sparkSession.sessionState.conf.caseSensitiveAnalysis) + sparkSession.sessionState.conf.resolver) val normalizedNewPartition = PartitioningUtils.normalizePartitionSpec( newPartition, table.partitionColumnNames, table.identifier.quotedString, - sparkSession.sessionState.conf.resolver, - sparkSession.sessionState.conf.caseSensitiveAnalysis) + sparkSession.sessionState.conf.resolver) catalog.renamePartitions( tableName, Seq(normalizedOldPartition), Seq(normalizedNewPartition)) @@ -531,8 +528,7 @@ case class AlterTableDropPartitionCommand( spec, table.partitionColumnNames, table.identifier.quotedString, - sparkSession.sessionState.conf.resolver, - sparkSession.sessionState.conf.caseSensitiveAnalysis) + sparkSession.sessionState.conf.resolver) } catalog.dropPartitions( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala index 1c98355a69e3..8ded1060f7bf 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala @@ -449,8 +449,7 @@ case class TruncateTableCommand( spec, partCols, table.identifier.quotedString, - spark.sessionState.conf.resolver, - spark.sessionState.conf.caseSensitiveAnalysis) + spark.sessionState.conf.resolver) } val partLocations = catalog.listPartitions(table.identifier, normalizedSpec).map(_.storage.locationUri) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala index 45eecfa6e8f1..4711778ac9ef 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala @@ -294,8 +294,7 @@ object PartitioningUtils { partitionSpec: Map[String, T], partColNames: Seq[String], tblName: String, - resolver: Resolver, - caseSensitiveAnalysis: Boolean): Map[String, T] = { + resolver: Resolver): Map[String, T] = { val normalizedPartSpec = partitionSpec.toSeq.map { case (key, value) => val normalizedKey = partColNames.find(resolver(_, key)).getOrElse { throw new AnalysisException(s"$key is not a valid partition column in table $tblName.") @@ -304,7 +303,7 @@ object PartitioningUtils { } SchemaUtils.checkColumnNameDuplication( - normalizedPartSpec.map(_._1), "partition specification", caseSensitiveAnalysis) + normalizedPartSpec.map(_._1), "partition specification", resolver) normalizedPartSpec.toMap } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala index d7235ca27979..dce02648f6b3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala @@ -750,7 +750,8 @@ object JdbcUtils extends Logging { val nameEquality = df.sparkSession.sessionState.conf.resolver // checks duplicate columns in the user specified column types. - SchemaUtils.checkSchemaColumnNameDuplication(userSchema, "createTableColumnTypes option value") + SchemaUtils.checkColumnNameDuplication( + userSchema.map(_.name), "createTableColumnTypes option value", nameEquality) // checks if user specified column names exist in the DataFrame schema userSchema.fieldNames.foreach { col => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala index 783726f3ca21..35ecd6a190ba 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala @@ -255,7 +255,7 @@ case class PreprocessTableCreation(sparkSession: SparkSession) extends Rule[Logi SchemaUtils.checkColumnNameDuplication( normalizedPartitionCols, "partition", - sparkSession.sessionState.conf.caseSensitiveAnalysis) + sparkSession.sessionState.conf.resolver) if (schema.nonEmpty && normalizedPartitionCols.length == schema.length) { if (DDLUtils.isHiveTable(table)) { @@ -286,15 +286,14 @@ case class PreprocessTableCreation(sparkSession: SparkSession) extends Rule[Logi bucketSpec = bucketSpec, resolver = sparkSession.sessionState.conf.resolver) - val caseSensitiveAnalysis = sparkSession.sessionState.conf.caseSensitiveAnalysis SchemaUtils.checkColumnNameDuplication( normalizedBucketSpec.bucketColumnNames, "bucket", - caseSensitiveAnalysis) + sparkSession.sessionState.conf.resolver) SchemaUtils.checkColumnNameDuplication( normalizedBucketSpec.sortColumnNames, "sort", - caseSensitiveAnalysis) + sparkSession.sessionState.conf.resolver) normalizedBucketSpec.sortColumnNames.map(schema(_)).map(_.dataType).foreach { case dt if RowOrdering.isOrderable(dt) => // OK @@ -323,7 +322,7 @@ case class PreprocessTableInsertion(conf: SQLConf) extends Rule[LogicalPlan] wit partColNames: Seq[String]): InsertIntoTable = { val normalizedPartSpec = PartitioningUtils.normalizePartitionSpec( - insert.partition, partColNames, tblName, conf.resolver, conf.caseSensitiveAnalysis) + insert.partition, partColNames, tblName, conf.resolver) val staticPartCols = normalizedPartSpec.filter(_._2.isDefined).keySet val expectedColumns = insert.table.output.filterNot(a => staticPartCols.contains(a.name)) diff --git a/sql/core/src/test/resources/sql-tests/inputs/create.sql b/sql/core/src/test/resources/sql-tests/inputs/create.sql index 60177ea504a2..80ed69570c25 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/create.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/create.sql @@ -1,9 +1,11 @@ --- Check name duplication in a regular case -CREATE TABLE t (c STRING, c INT) USING parquet; +-- Catch case-sensitive name duplication +SET spark.sql.caseSensitive=true; --- Check multiple name duplication CREATE TABLE t (c0 STRING, c1 INT, c1 DOUBLE, c0 INT) USING parquet; -- Catch case-insensitive name duplication SET spark.sql.caseSensitive=false; + +CREATE TABLE t (c0 STRING, c1 INT, c1 DOUBLE, c0 INT) USING parquet; + CREATE TABLE t (ab STRING, cd INT, ef DOUBLE, Ab INT) USING parquet; diff --git a/sql/core/src/test/resources/sql-tests/results/create.sql.out b/sql/core/src/test/resources/sql-tests/results/create.sql.out index 5fe9f81d7e41..03a465cd4da3 100644 --- a/sql/core/src/test/resources/sql-tests/results/create.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/create.sql.out @@ -1,14 +1,13 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 4 +-- Number of queries: 5 -- !query 0 -CREATE TABLE t (c STRING, c INT) USING parquet +SET spark.sql.caseSensitive=true -- !query 0 schema -struct<> +struct -- !query 0 output -org.apache.spark.sql.AnalysisException -Found duplicate column(s) in table definition of `t`: "c"; +spark.sql.caseSensitive true -- !query 1 @@ -17,7 +16,7 @@ CREATE TABLE t (c0 STRING, c1 INT, c1 DOUBLE, c0 INT) USING parquet struct<> -- !query 1 output org.apache.spark.sql.AnalysisException -Found duplicate column(s) in table definition of `t`: "c1", "c0"; +Found duplicate column(s) in table definition of `t`: "c0", "c1"; -- !query 2 @@ -29,9 +28,18 @@ spark.sql.caseSensitive false -- !query 3 -CREATE TABLE t (ab STRING, cd INT, ef DOUBLE, Ab INT) USING parquet +CREATE TABLE t (c0 STRING, c1 INT, c1 DOUBLE, c0 INT) USING parquet -- !query 3 schema struct<> -- !query 3 output org.apache.spark.sql.AnalysisException +Found duplicate column(s) in table definition of `t`: "c0", "c1"; + + +-- !query 4 +CREATE TABLE t (ab STRING, cd INT, ef DOUBLE, Ab INT) USING parquet +-- !query 4 schema +struct<> +-- !query 4 output +org.apache.spark.sql.AnalysisException Found duplicate column(s) in table definition of `t`: "ab"; From 11d1818fdf41a69f81beb9a2c1e1c17338d9015d Mon Sep 17 00:00:00 2001 From: Takeshi Yamamuro Date: Fri, 16 Jun 2017 18:50:01 +0900 Subject: [PATCH 05/27] Apply xiao's reviews --- .../apache/spark/sql/util/SchemaUtils.scala | 24 +++++++++++++++---- .../spark/sql/util/SchemaUtilsSuite.scala | 21 ++++++++-------- .../sql-tests/results/create.sql.out | 6 ++--- .../sql/execution/command/DDLSuite.scala | 8 +++---- .../spark/sql/jdbc/JDBCWriteSuite.scala | 2 +- .../sql/hive/execution/HiveDDLSuite.scala | 2 +- 6 files changed, 39 insertions(+), 24 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/util/SchemaUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/util/SchemaUtils.scala index 33280fbb369f..d03d91953b26 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/util/SchemaUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/util/SchemaUtils.scala @@ -31,6 +31,14 @@ import org.apache.spark.sql.types.StructType */ private[spark] object SchemaUtils { + /** + * Checks if an input schema has duplicate column names. This throws an exception if the + * duplication exists. + * + * @param schema schema to check + * @param colType column type name, used in an exception message + * @param caseSensitiveAnalysis whether duplication checks should be case sensitive or not + */ def checkSchemaColumnNameDuplication( schema: StructType, colType: String, caseSensitiveAnalysis: Boolean = false): Unit = { val resolver = if (caseSensitiveAnalysis) { @@ -41,18 +49,26 @@ private[spark] object SchemaUtils { checkColumnNameDuplication(schema.map(_.name), colType, resolver) } + /** + * Checks if input column names have duplicate identifiers. This throws an exception if + * the duplication exists. + * + * @param colNames column names to check + * @param colType column type name, used in an exception message + * @param resolver resolver used to determine if two identifiers are equal + */ def checkColumnNameDuplication( - names: Seq[String], colType: String, resolver: Resolver): Unit = { + colNames: Seq[String], colType: String, resolver: Resolver): Unit = { val duplicateColumns = mutable.ArrayBuffer[String]() - names.foreach { name => - val sameColNames = names.filter(resolver(_, name)) + colNames.foreach { name => + val sameColNames = colNames.filter(resolver(_, name)) if (sameColNames.size > 1 && !duplicateColumns.exists(resolver(_, name))) { duplicateColumns.append(name) } } if (duplicateColumns.size > 0) { throw new AnalysisException(s"Found duplicate column(s) in $colType: " + - duplicateColumns.map(colName => s""""$colName"""").mkString(", ")) + duplicateColumns.map(colName => s"`$colName`").mkString(", ")) } } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/util/SchemaUtilsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/util/SchemaUtilsSuite.scala index 21e38fabed7d..3f3cde98b6d4 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/util/SchemaUtilsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/util/SchemaUtilsSuite.scala @@ -24,19 +24,17 @@ import org.apache.spark.sql.types.StructType class SchemaUtilsSuite extends SparkFunSuite { - test("Check column name duplication") { - // Case-sensitive case + test("Check column name duplication in case-sensitive cases") { val msg1 = intercept[AnalysisException] { SchemaUtils.checkSchemaColumnNameDuplication( - StructType.fromDDL("a INT, b INT, a INT"), "SchemaUtilsSuite", - caseSensitiveAnalysis = true) + StructType.fromDDL("a INT, b INT, a INT"), "SchemaUtilsSuite", caseSensitiveAnalysis = true) }.getMessage - assert(msg1.contains("""Found duplicate column(s) in SchemaUtilsSuite: "a";""")) + assert(msg1.contains("""Found duplicate column(s) in SchemaUtilsSuite: `a`;""")) val msg2 = intercept[AnalysisException] { SchemaUtils.checkColumnNameDuplication( "a" :: "b" :: "a" :: Nil, "SchemaUtilsSuite", caseSensitiveResolution) }.getMessage - assert(msg2.contains("""Found duplicate column(s) in SchemaUtilsSuite: "a";""")) + assert(msg2.contains("""Found duplicate column(s) in SchemaUtilsSuite: `a`;""")) // Check no exception thrown SchemaUtils.checkSchemaColumnNameDuplication( @@ -47,31 +45,32 @@ class SchemaUtilsSuite extends SparkFunSuite { StructType.fromDDL("Aa INT, b INT, aA INT"), "SchemaUtilsSuite", caseSensitiveAnalysis = true) SchemaUtils.checkColumnNameDuplication( "Aa" :: "b" :: "aA" :: Nil, "SchemaUtilsSuite", caseSensitiveResolution) + } - // Case-insensitive case + test("Check column name duplication in case-insensitive cases") { val msg3 = intercept[AnalysisException] { SchemaUtils.checkSchemaColumnNameDuplication( StructType.fromDDL("Aa INT, b INT, Aa INT"), "SchemaUtilsSuite", caseSensitiveAnalysis = false) }.getMessage - assert(msg3.contains("""Found duplicate column(s) in SchemaUtilsSuite: "Aa";""")) + assert(msg3.contains("""Found duplicate column(s) in SchemaUtilsSuite: `Aa`;""")) val msg4 = intercept[AnalysisException] { SchemaUtils.checkColumnNameDuplication( "Aa" :: "b" :: "Aa" :: Nil, "SchemaUtilsSuite", caseInsensitiveResolution) }.getMessage - assert(msg4.contains("""Found duplicate column(s) in SchemaUtilsSuite: "Aa";""")) + assert(msg4.contains("""Found duplicate column(s) in SchemaUtilsSuite: `Aa`;""")) val msg5 = intercept[AnalysisException] { SchemaUtils.checkSchemaColumnNameDuplication( StructType.fromDDL("a INT, bB INT, Bb INT"), "SchemaUtilsSuite", caseSensitiveAnalysis = false) }.getMessage - assert(msg5.contains("""Found duplicate column(s) in SchemaUtilsSuite: "bB";""")) + assert(msg5.contains("""Found duplicate column(s) in SchemaUtilsSuite: `bB`;""")) val msg6 = intercept[AnalysisException] { SchemaUtils.checkColumnNameDuplication( "a" :: "bB" :: "Bb" :: Nil, "SchemaUtilsSuite", caseInsensitiveResolution) }.getMessage - assert(msg6.contains("""Found duplicate column(s) in SchemaUtilsSuite: "bB";""")) + assert(msg6.contains("""Found duplicate column(s) in SchemaUtilsSuite: `bB`;""")) // Check no exception thrown SchemaUtils.checkSchemaColumnNameDuplication( diff --git a/sql/core/src/test/resources/sql-tests/results/create.sql.out b/sql/core/src/test/resources/sql-tests/results/create.sql.out index 03a465cd4da3..5326711babff 100644 --- a/sql/core/src/test/resources/sql-tests/results/create.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/create.sql.out @@ -16,7 +16,7 @@ CREATE TABLE t (c0 STRING, c1 INT, c1 DOUBLE, c0 INT) USING parquet struct<> -- !query 1 output org.apache.spark.sql.AnalysisException -Found duplicate column(s) in table definition of `t`: "c0", "c1"; +Found duplicate column(s) in table definition of `t`: `c0`, `c1`; -- !query 2 @@ -33,7 +33,7 @@ CREATE TABLE t (c0 STRING, c1 INT, c1 DOUBLE, c0 INT) USING parquet struct<> -- !query 3 output org.apache.spark.sql.AnalysisException -Found duplicate column(s) in table definition of `t`: "c0", "c1"; +Found duplicate column(s) in table definition of `t`: `c0`, `c1`; -- !query 4 @@ -42,4 +42,4 @@ CREATE TABLE t (ab STRING, cd INT, ef DOUBLE, Ab INT) USING parquet struct<> -- !query 4 output org.apache.spark.sql.AnalysisException -Found duplicate column(s) in table definition of `t`: "ab"; +Found duplicate column(s) in table definition of `t`: `ab`; diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala index f482d3846ead..b6062ba66fcf 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala @@ -439,13 +439,13 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils { val e = intercept[AnalysisException] { sql("CREATE TABLE tbl(a int, a string) USING json") } - assert(e.message == """Found duplicate column(s) in table definition of `tbl`: "a"""") + assert(e.message == """Found duplicate column(s) in table definition of `tbl`: `a`""") withSQLConf(SQLConf.CASE_SENSITIVE.key -> "false") { val e2 = intercept[AnalysisException] { sql("CREATE TABLE tbl(a int, A string) USING json") } - assert(e2.message == """Found duplicate column(s) in table definition of `tbl`: "a"""") + assert(e2.message == """Found duplicate column(s) in table definition of `tbl`: `a`""") } } @@ -469,14 +469,14 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils { val e = intercept[AnalysisException] { sql("CREATE TABLE tbl(a int) USING json PARTITIONED BY (a, a)") } - assert(e.message == """Found duplicate column(s) in partition: "a"""") + assert(e.message == """Found duplicate column(s) in partition: `a`""") } test("create table - column repeated in bucket columns") { val e = intercept[AnalysisException] { sql("CREATE TABLE tbl(a int) USING json CLUSTERED BY (a, a) INTO 4 BUCKETS") } - assert(e.message == """Found duplicate column(s) in bucket: "a"""") + assert(e.message == """Found duplicate column(s) in bucket: `a`""") } test("Refresh table after changing the data source table partitioning") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala index 3ca2804f5b96..e05dc076f9c7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala @@ -479,7 +479,7 @@ class JDBCWriteSuite extends SharedSQLContext with BeforeAndAfter { .jdbc(url1, "TEST.USERDBTYPETEST", properties) }.getMessage() assert(msg.contains( - """Found duplicate column(s) in createTableColumnTypes option value: "name";""")) + """Found duplicate column(s) in createTableColumnTypes option value: `name`;""")) } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala index 0b81e60887a5..7528e643681f 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala @@ -345,7 +345,7 @@ class HiveDDLSuite val e = intercept[AnalysisException] { sql("CREATE TABLE tbl(a int) PARTITIONED BY (a string)") } - assert(e.message == """Found duplicate column(s) in table definition of `default`.`tbl`: "a"""") + assert(e.message == """Found duplicate column(s) in table definition of `default`.`tbl`: `a`""") } test("add/drop partition with location - managed table") { From 22e1e4fab65b0ef641901024c1278a2e37556b6d Mon Sep 17 00:00:00 2001 From: Takeshi Yamamuro Date: Sat, 17 Jun 2017 11:22:59 +0900 Subject: [PATCH 06/27] Apply more xiao's reviews --- .../sql/catalyst/catalog/SessionCatalog.scala | 3 +- .../spark/sql/util/SchemaUtilsSuite.scala | 97 ++++++++++++++++--- .../spark/sql/execution/command/views.scala | 2 +- .../execution/datasources/DataSource.scala | 4 +- .../InsertIntoHadoopFsRelationCommand.scala | 2 +- .../datasources/PartitioningUtils.scala | 2 +- .../datasources/jdbc/JdbcUtils.scala | 2 +- .../sql/execution/datasources/rules.scala | 8 +- .../sql-tests/results/create.sql.out | 6 +- .../org/apache/spark/sql/DataFrameSuite.scala | 4 +- .../sql/execution/command/DDLSuite.scala | 8 +- .../spark/sql/jdbc/JDBCWriteSuite.scala | 2 +- .../sql/test/DataFrameReaderWriterSuite.scala | 8 +- .../spark/sql/hive/HiveMetastoreCatalog.scala | 2 +- .../sql/hive/execution/HiveDDLSuite.scala | 2 +- 15 files changed, 113 insertions(+), 39 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala index 2fcd103a87fd..84771d627fcd 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala @@ -343,7 +343,8 @@ class SessionCatalog( requireTableExists(tableIdentifier) SchemaUtils.checkSchemaColumnNameDuplication( - newSchema, "table definition of " + tableIdentifier.identifier, conf.caseSensitiveAnalysis) + newSchema, "the table definition of " + tableIdentifier.identifier, + conf.caseSensitiveAnalysis) val catalogTable = externalCatalog.getTable(db, table) val oldSchema = catalogTable.schema diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/util/SchemaUtilsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/util/SchemaUtilsSuite.scala index 3f3cde98b6d4..e40e99b71d61 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/util/SchemaUtilsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/util/SchemaUtilsSuite.scala @@ -25,16 +25,52 @@ import org.apache.spark.sql.types.StructType class SchemaUtilsSuite extends SparkFunSuite { test("Check column name duplication in case-sensitive cases") { - val msg1 = intercept[AnalysisException] { + var msg = intercept[AnalysisException] { SchemaUtils.checkSchemaColumnNameDuplication( StructType.fromDDL("a INT, b INT, a INT"), "SchemaUtilsSuite", caseSensitiveAnalysis = true) }.getMessage - assert(msg1.contains("""Found duplicate column(s) in SchemaUtilsSuite: `a`;""")) - val msg2 = intercept[AnalysisException] { + assert(msg.contains("Found duplicate column(s) in SchemaUtilsSuite: `a`;")) + msg = intercept[AnalysisException] { SchemaUtils.checkColumnNameDuplication( "a" :: "b" :: "a" :: Nil, "SchemaUtilsSuite", caseSensitiveResolution) }.getMessage - assert(msg2.contains("""Found duplicate column(s) in SchemaUtilsSuite: `a`;""")) + assert(msg.contains("Found duplicate column(s) in SchemaUtilsSuite: `a`;")) + + msg = intercept[AnalysisException] { + SchemaUtils.checkSchemaColumnNameDuplication( + StructType.fromDDL("a INT, b INT, a INT, a INT"), "SchemaUtilsSuite", + caseSensitiveAnalysis = true) + }.getMessage + assert(msg.contains("Found duplicate column(s) in SchemaUtilsSuite: `a`;")) + msg = intercept[AnalysisException] { + SchemaUtils.checkColumnNameDuplication( + "a" :: "b" :: "a" :: "a" :: Nil, "SchemaUtilsSuite", caseSensitiveResolution) + }.getMessage + assert(msg.contains("Found duplicate column(s) in SchemaUtilsSuite: `a`;")) + + msg = intercept[AnalysisException] { + SchemaUtils.checkSchemaColumnNameDuplication( + StructType.fromDDL("a INT, b INT, a INT, b INT"), "SchemaUtilsSuite", + caseSensitiveAnalysis = true) + }.getMessage + assert(msg.contains("Found duplicate column(s) in SchemaUtilsSuite: `a`, `b`;")) + msg = intercept[AnalysisException] { + SchemaUtils.checkColumnNameDuplication( + "a" :: "b" :: "a" :: "b" :: Nil, "SchemaUtilsSuite", caseSensitiveResolution) + }.getMessage + assert(msg.contains("Found duplicate column(s) in SchemaUtilsSuite: `a`, `b`;")) + + msg = intercept[AnalysisException] { + SchemaUtils.checkSchemaColumnNameDuplication( + StructType.fromDDL("a INT, c INT, b INT, a INT, b INT, c INT"), "SchemaUtilsSuite", + caseSensitiveAnalysis = true) + }.getMessage + assert(msg.contains("Found duplicate column(s) in SchemaUtilsSuite: `a`, `c`, `b`;")) + msg = intercept[AnalysisException] { + SchemaUtils.checkColumnNameDuplication( + "a" :: "c" :: "b" :: "a" :: "b" :: "c" :: Nil, "SchemaUtilsSuite", caseSensitiveResolution) + }.getMessage + assert(msg.contains("Found duplicate column(s) in SchemaUtilsSuite: `a`, `c`, `b`;")) // Check no exception thrown SchemaUtils.checkSchemaColumnNameDuplication( @@ -48,29 +84,66 @@ class SchemaUtilsSuite extends SparkFunSuite { } test("Check column name duplication in case-insensitive cases") { - val msg3 = intercept[AnalysisException] { + var msg = intercept[AnalysisException] { SchemaUtils.checkSchemaColumnNameDuplication( StructType.fromDDL("Aa INT, b INT, Aa INT"), "SchemaUtilsSuite", caseSensitiveAnalysis = false) }.getMessage - assert(msg3.contains("""Found duplicate column(s) in SchemaUtilsSuite: `Aa`;""")) - val msg4 = intercept[AnalysisException] { + assert(msg.contains("Found duplicate column(s) in SchemaUtilsSuite: `Aa`;")) + msg = intercept[AnalysisException] { SchemaUtils.checkColumnNameDuplication( "Aa" :: "b" :: "Aa" :: Nil, "SchemaUtilsSuite", caseInsensitiveResolution) }.getMessage - assert(msg4.contains("""Found duplicate column(s) in SchemaUtilsSuite: `Aa`;""")) + assert(msg.contains("Found duplicate column(s) in SchemaUtilsSuite: `Aa`;")) - val msg5 = intercept[AnalysisException] { + msg = intercept[AnalysisException] { SchemaUtils.checkSchemaColumnNameDuplication( StructType.fromDDL("a INT, bB INT, Bb INT"), "SchemaUtilsSuite", caseSensitiveAnalysis = false) }.getMessage - assert(msg5.contains("""Found duplicate column(s) in SchemaUtilsSuite: `bB`;""")) - val msg6 = intercept[AnalysisException] { + assert(msg.contains("Found duplicate column(s) in SchemaUtilsSuite: `bB`;")) + msg = intercept[AnalysisException] { SchemaUtils.checkColumnNameDuplication( "a" :: "bB" :: "Bb" :: Nil, "SchemaUtilsSuite", caseInsensitiveResolution) }.getMessage - assert(msg6.contains("""Found duplicate column(s) in SchemaUtilsSuite: `bB`;""")) + assert(msg.contains("Found duplicate column(s) in SchemaUtilsSuite: `bB`;")) + + msg = intercept[AnalysisException] { + SchemaUtils.checkSchemaColumnNameDuplication( + StructType.fromDDL("Aa INT, b INT, Aa INT, c INT, aa INT"), "SchemaUtilsSuite", + caseSensitiveAnalysis = false) + }.getMessage + assert(msg.contains("Found duplicate column(s) in SchemaUtilsSuite: `Aa`;")) + msg = intercept[AnalysisException] { + SchemaUtils.checkColumnNameDuplication( + "Aa" :: "b" :: "Aa" :: "c" :: "aa" :: Nil, "SchemaUtilsSuite", caseInsensitiveResolution) + }.getMessage + assert(msg.contains("Found duplicate column(s) in SchemaUtilsSuite: `Aa`;")) + + msg = intercept[AnalysisException] { + SchemaUtils.checkSchemaColumnNameDuplication( + StructType.fromDDL("Aa INT, bB INT, Bb INT, aa INT"), "SchemaUtilsSuite", + caseSensitiveAnalysis = false) + }.getMessage + assert(msg.contains("Found duplicate column(s) in SchemaUtilsSuite: `Aa`, `bB`;")) + msg = intercept[AnalysisException] { + SchemaUtils.checkColumnNameDuplication( + "Aa" :: "bB" :: "Bb" :: "aa" :: Nil, "SchemaUtilsSuite", caseInsensitiveResolution) + }.getMessage + assert(msg.contains("Found duplicate column(s) in SchemaUtilsSuite: `Aa`, `bB`;")) + + msg = intercept[AnalysisException] { + SchemaUtils.checkSchemaColumnNameDuplication( + StructType.fromDDL("Aa INT, cc INT, bB INT, cC INT, Bb INT, aa INT"), "SchemaUtilsSuite", + caseSensitiveAnalysis = false) + }.getMessage + assert(msg.contains("Found duplicate column(s) in SchemaUtilsSuite: `Aa`, `cc`, `bB`;")) + msg = intercept[AnalysisException] { + SchemaUtils.checkColumnNameDuplication( + "Aa" :: "cc" :: "bB" :: "cC" :: "Bb" :: "aa" :: Nil, "SchemaUtilsSuite", + caseInsensitiveResolution) + }.getMessage + assert(msg.contains("Found duplicate column(s) in SchemaUtilsSuite: `Aa`, `cc`, `bB`;")) // Check no exception thrown SchemaUtils.checkSchemaColumnNameDuplication( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala index c8fab4f992f9..ee7a877a9134 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala @@ -359,7 +359,7 @@ object ViewHelper { // Generate the query column names, throw an AnalysisException if there exists duplicate column // names. SchemaUtils.checkSchemaColumnNameDuplication( - analyzedPlan.schema, "view", session.sessionState.conf.caseSensitiveAnalysis) + analyzedPlan.schema, "the view", session.sessionState.conf.caseSensitiveAnalysis) // Generate the view default database name. val viewDefaultDatabase = session.sessionState.catalog.getCurrentDatabase diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala index 3b04c207d540..6d8b8e31c7d4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala @@ -189,7 +189,7 @@ case class DataSource( sparkSession.sessionState.conf.caseSensitiveAnalysis) SchemaUtils.checkSchemaColumnNameDuplication( - dataSchema, "datasource", sparkSession.sessionState.conf.caseSensitiveAnalysis) + dataSchema, "the datasource", sparkSession.sessionState.conf.caseSensitiveAnalysis) (dataSchema, partitionSchema) } @@ -338,7 +338,7 @@ case class DataSource( } SchemaUtils.checkSchemaColumnNameDuplication( - dataSchema, "datasource", sparkSession.sessionState.conf.caseSensitiveAnalysis) + dataSchema, "the datasource", sparkSession.sessionState.conf.caseSensitiveAnalysis) HadoopFsRelation( fileCatalog, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala index 05d2bd92de53..666ea062d6fe 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala @@ -66,7 +66,7 @@ case class InsertIntoHadoopFsRelationCommand( // Most formats don't do well with duplicate columns, so lets not allow that SchemaUtils.checkSchemaColumnNameDuplication( - query.schema, "query", sparkSession.sessionState.conf.caseSensitiveAnalysis) + query.schema, "the query", sparkSession.sessionState.conf.caseSensitiveAnalysis) val hadoopConf = sparkSession.sessionState.newHadoopConfWithOptions(options) val fs = outputPath.getFileSystem(hadoopConf) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala index 4711778ac9ef..e72e33d6784d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala @@ -303,7 +303,7 @@ object PartitioningUtils { } SchemaUtils.checkColumnNameDuplication( - normalizedPartSpec.map(_._1), "partition specification", resolver) + normalizedPartSpec.map(_._1), "the partition specification", resolver) normalizedPartSpec.toMap } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala index dce02648f6b3..9932a2a7da12 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala @@ -751,7 +751,7 @@ object JdbcUtils extends Logging { // checks duplicate columns in the user specified column types. SchemaUtils.checkColumnNameDuplication( - userSchema.map(_.name), "createTableColumnTypes option value", nameEquality) + userSchema.map(_.name), "the createTableColumnTypes option value", nameEquality) // checks if user specified column names exist in the DataFrame schema userSchema.fieldNames.foreach { col => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala index 35ecd6a190ba..9df04f38115a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala @@ -225,7 +225,7 @@ case class PreprocessTableCreation(sparkSession: SparkSession) extends Rule[Logi private def normalizeCatalogTable(schema: StructType, table: CatalogTable): CatalogTable = { SchemaUtils.checkSchemaColumnNameDuplication( schema, - "table definition of " + table.identifier, + "the table definition of " + table.identifier, sparkSession.sessionState.conf.caseSensitiveAnalysis) val normalizedPartCols = normalizePartitionColumns(schema, table) @@ -254,7 +254,7 @@ case class PreprocessTableCreation(sparkSession: SparkSession) extends Rule[Logi SchemaUtils.checkColumnNameDuplication( normalizedPartitionCols, - "partition", + "the partition", sparkSession.sessionState.conf.resolver) if (schema.nonEmpty && normalizedPartitionCols.length == schema.length) { @@ -288,11 +288,11 @@ case class PreprocessTableCreation(sparkSession: SparkSession) extends Rule[Logi SchemaUtils.checkColumnNameDuplication( normalizedBucketSpec.bucketColumnNames, - "bucket", + "the bucket", sparkSession.sessionState.conf.resolver) SchemaUtils.checkColumnNameDuplication( normalizedBucketSpec.sortColumnNames, - "sort", + "the sort", sparkSession.sessionState.conf.resolver) normalizedBucketSpec.sortColumnNames.map(schema(_)).map(_.dataType).foreach { diff --git a/sql/core/src/test/resources/sql-tests/results/create.sql.out b/sql/core/src/test/resources/sql-tests/results/create.sql.out index 5326711babff..c4fc4c1343c3 100644 --- a/sql/core/src/test/resources/sql-tests/results/create.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/create.sql.out @@ -16,7 +16,7 @@ CREATE TABLE t (c0 STRING, c1 INT, c1 DOUBLE, c0 INT) USING parquet struct<> -- !query 1 output org.apache.spark.sql.AnalysisException -Found duplicate column(s) in table definition of `t`: `c0`, `c1`; +Found duplicate column(s) in the table definition of `t`: `c0`, `c1`; -- !query 2 @@ -33,7 +33,7 @@ CREATE TABLE t (c0 STRING, c1 INT, c1 DOUBLE, c0 INT) USING parquet struct<> -- !query 3 output org.apache.spark.sql.AnalysisException -Found duplicate column(s) in table definition of `t`: `c0`, `c1`; +Found duplicate column(s) in the table definition of `t`: `c0`, `c1`; -- !query 4 @@ -42,4 +42,4 @@ CREATE TABLE t (ab STRING, cd INT, ef DOUBLE, Ab INT) USING parquet struct<> -- !query 4 output org.apache.spark.sql.AnalysisException -Found duplicate column(s) in table definition of `t`: `ab`; +Found duplicate column(s) in the table definition of `t`: `ab`; diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala index dff4d516ecbc..8c73d20cef03 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala @@ -1123,7 +1123,7 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { Seq((1, 2, 3), (2, 3, 4), (3, 4, 5)).toDF("column1", "column2", "column1") .write.format("parquet").save("temp") } - assert(e.getMessage.contains("Found duplicate column(s) in query: ")) + assert(e.getMessage.contains("Found duplicate column(s) in the query: ")) assert(e.getMessage.contains("column1")) assert(!e.getMessage.contains("column2")) @@ -1133,7 +1133,7 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { .toDF("column1", "column2", "column3", "column1", "column3") .write.format("json").save("temp") } - assert(f.getMessage.contains("Found duplicate column(s) in query: ")) + assert(f.getMessage.contains("Found duplicate column(s) in the query: ")) assert(f.getMessage.contains("column1")) assert(f.getMessage.contains("column3")) assert(!f.getMessage.contains("column2")) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala index b6062ba66fcf..b40eac5d5c77 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala @@ -439,13 +439,13 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils { val e = intercept[AnalysisException] { sql("CREATE TABLE tbl(a int, a string) USING json") } - assert(e.message == """Found duplicate column(s) in table definition of `tbl`: `a`""") + assert(e.message == "Found duplicate column(s) in the table definition of `tbl`: `a`") withSQLConf(SQLConf.CASE_SENSITIVE.key -> "false") { val e2 = intercept[AnalysisException] { sql("CREATE TABLE tbl(a int, A string) USING json") } - assert(e2.message == """Found duplicate column(s) in table definition of `tbl`: `a`""") + assert(e2.message == "Found duplicate column(s) in the table definition of `tbl`: `a`") } } @@ -469,14 +469,14 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils { val e = intercept[AnalysisException] { sql("CREATE TABLE tbl(a int) USING json PARTITIONED BY (a, a)") } - assert(e.message == """Found duplicate column(s) in partition: `a`""") + assert(e.message == "Found duplicate column(s) in the partition: `a`") } test("create table - column repeated in bucket columns") { val e = intercept[AnalysisException] { sql("CREATE TABLE tbl(a int) USING json CLUSTERED BY (a, a) INTO 4 BUCKETS") } - assert(e.message == """Found duplicate column(s) in bucket: `a`""") + assert(e.message == "Found duplicate column(s) in the bucket: `a`") } test("Refresh table after changing the data source table partitioning") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala index e05dc076f9c7..eac773057251 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala @@ -479,7 +479,7 @@ class JDBCWriteSuite extends SharedSQLContext with BeforeAndAfter { .jdbc(url1, "TEST.USERDBTYPETEST", properties) }.getMessage() assert(msg.contains( - """Found duplicate column(s) in createTableColumnTypes option value: `name`;""")) + "Found duplicate column(s) in the createTableColumnTypes option value: `name`;")) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala index 3ef94fb8274f..32c630dfc94c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala @@ -698,7 +698,7 @@ class DataFrameReaderWriterSuite extends QueryTest with SharedSQLContext with Be val e = intercept[AnalysisException] { spark.read.format("csv").schema(schema).load(src.toString) } - assert(e.getMessage.contains("Found duplicate column(s) in datasource: ")) + assert(e.getMessage.contains("Found duplicate column(s) in the datasource: ")) } } @@ -715,12 +715,12 @@ class DataFrameReaderWriterSuite extends QueryTest with SharedSQLContext with Be val e1 = intercept[AnalysisException] { spark.read.format("json").schema(s"$c0 INT, $c1 INT").load(src.toString) } - assert(e1.getMessage.contains("Found duplicate column(s) in datasource: ")) + assert(e1.getMessage.contains("Found duplicate column(s) in the datasource: ")) val e2 = intercept[AnalysisException] { spark.read.format("json").option("inferSchema", true).load(src.toString) } - assert(e2.getMessage.contains("Found duplicate column(s) in datasource: ")) + assert(e2.getMessage.contains("Found duplicate column(s) in the datasource: ")) } } @@ -731,7 +731,7 @@ class DataFrameReaderWriterSuite extends QueryTest with SharedSQLContext with Be val e = intercept[AnalysisException] { spark.read.format("parquet").schema(schema).load(src.toString) } - assert(e.getMessage.contains("Found duplicate column(s) in datasource: ")) + assert(e.getMessage.contains("Found duplicate column(s) in the datasource: ")) } } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index e404517e8c56..5afe6c144f81 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -251,7 +251,7 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log } SchemaUtils.checkSchemaColumnNameDuplication( - schema, "hive serde table", sparkSession.sessionState.conf.caseSensitiveAnalysis) + schema, "the hive serde table", sparkSession.sessionState.conf.caseSensitiveAnalysis) (schema, relation.tableMeta.copy(schema = schema)) case None => diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala index 7528e643681f..12daf3af11ab 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala @@ -345,7 +345,7 @@ class HiveDDLSuite val e = intercept[AnalysisException] { sql("CREATE TABLE tbl(a int) PARTITIONED BY (a string)") } - assert(e.message == """Found duplicate column(s) in table definition of `default`.`tbl`: `a`""") + assert(e.message == "Found duplicate column(s) in the table definition of `default`.`tbl`: `a`") } test("add/drop partition with location - managed table") { From 743a069db250625f0b5c67160dc1096029e9eaa6 Mon Sep 17 00:00:00 2001 From: Takeshi Yamamuro Date: Tue, 20 Jun 2017 09:17:20 +0900 Subject: [PATCH 07/27] Replace map with foreach --- .../spark/sql/test/DataFrameReaderWriterSuite.scala | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala index 32c630dfc94c..4a6cf038891c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala @@ -693,7 +693,8 @@ class DataFrameReaderWriterSuite extends QueryTest with SharedSQLContext with Be withTempDir { src => // Check CSV format Seq("1,1").toDF().coalesce(1).write.mode("overwrite").text(src.toString) - Seq((true, "a INT, a INT"), (false, "aA INT, Aa INT")).map { case (caseSensitive, schema) => + Seq((true, "a INT, a INT"), (false, "aA INT, Aa INT")) + .foreach { case (caseSensitive, schema) => withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive.toString) { val e = intercept[AnalysisException] { spark.read.format("csv").schema(schema).load(src.toString) @@ -709,7 +710,8 @@ class DataFrameReaderWriterSuite extends QueryTest with SharedSQLContext with Be checkAnswer(df, Row(1, 1)) // Check JSON format - Seq((true, ("a", "a")), (false, ("aA", "Aa"))).map { case (caseSensitive, (c0, c1)) => + Seq((true, ("a", "a")), (false, ("aA", "Aa"))) + .foreach { case (caseSensitive, (c0, c1)) => Seq(s"""{"$c0":1, "$c1":1}""").toDF().coalesce(1).write.mode("overwrite").text(src.toString) withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive.toString) { val e1 = intercept[AnalysisException] { @@ -726,7 +728,8 @@ class DataFrameReaderWriterSuite extends QueryTest with SharedSQLContext with Be // Check Paruqet format Seq((1, 1)).toDF("c0", "c1").coalesce(1).write.mode("overwrite").parquet(src.toString) - Seq((true, "a INT, a INT"), (false, "aA INT, Aa INT")).map { case (caseSensitive, schema) => + Seq((true, "a INT, a INT"), (false, "aA INT, Aa INT")) + .foreach { case (caseSensitive, schema) => withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive.toString) { val e = intercept[AnalysisException] { spark.read.format("parquet").schema(schema).load(src.toString) From f6eab2df50e15d50a0b60807aa503fcecf024a76 Mon Sep 17 00:00:00 2001 From: Takeshi Yamamuro Date: Tue, 20 Jun 2017 16:26:16 +0900 Subject: [PATCH 08/27] Add tests for data schema + parititon schema --- .../execution/datasources/DataSource.scala | 10 +-- .../sql/test/DataFrameReaderWriterSuite.scala | 89 +++++++++++++------ 2 files changed, 65 insertions(+), 34 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala index 6d8b8e31c7d4..cb7c03303499 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala @@ -128,11 +128,11 @@ case class DataSource( }.toArray new InMemoryFileIndex(sparkSession, globbedPaths, options, None, fileStatusCache) } + val equality = sparkSession.sessionState.conf.resolver val partitionSchema = if (partitionColumns.isEmpty) { // Try to infer partitioning, because no DataSource in the read path provides the partitioning // columns properly unless it is a Hive DataSource val resolved = tempFileIndex.partitionSchema.map { partitionField => - val equality = sparkSession.sessionState.conf.resolver // SPARK-18510: try to get schema from userSpecifiedSchema, otherwise fallback to inferred userSpecifiedSchema.flatMap(_.find(f => equality(f.name, partitionField.name))).getOrElse( partitionField) @@ -146,7 +146,6 @@ case class DataSource( inferredPartitions } else { val partitionFields = partitionColumns.map { partitionColumn => - val equality = sparkSession.sessionState.conf.resolver userSpecifiedSchema.flatMap(_.find(c => equality(c.name, partitionColumn))).orElse { val inferredPartitions = tempFileIndex.partitionSchema val inferredOpt = inferredPartitions.find(p => equality(p.name, partitionColumn)) @@ -172,7 +171,6 @@ case class DataSource( } val dataSchema = userSpecifiedSchema.map { schema => - val equality = sparkSession.sessionState.conf.resolver StructType(schema.filterNot(f => partitionSchema.exists(p => equality(p.name, f.name)))) }.orElse { format.inferSchema( @@ -186,10 +184,10 @@ case class DataSource( SchemaUtils.checkColumnNameDuplication( (dataSchema ++ partitionSchema).map(_.name), "in the data schema and the partition schema", - sparkSession.sessionState.conf.caseSensitiveAnalysis) + equality) - SchemaUtils.checkSchemaColumnNameDuplication( - dataSchema, "the datasource", sparkSession.sessionState.conf.caseSensitiveAnalysis) + SchemaUtils.checkColumnNameDuplication( + dataSchema.map(_.name), "the datasource", equality) (dataSchema, partitionSchema) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala index 4a6cf038891c..5e38d96c4243 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala @@ -690,49 +690,82 @@ class DataFrameReaderWriterSuite extends QueryTest with SharedSQLContext with Be } test("SPARK-20460 Check name duplication in schema") { - withTempDir { src => - // Check CSV format - Seq("1,1").toDF().coalesce(1).write.mode("overwrite").text(src.toString) - Seq((true, "a INT, a INT"), (false, "aA INT, Aa INT")) - .foreach { case (caseSensitive, schema) => - withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive.toString) { - val e = intercept[AnalysisException] { - spark.read.format("csv").schema(schema).load(src.toString) + // Check CSV format + Seq((true, ("a", "a")), (false, ("aA", "Aa"))).foreach { case (caseSensitive, (c0, c1)) => + withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive.toString) { + withTempDir { src => + Seq("1,1").toDF().write.mode("overwrite").text(src.toString) + + // data schema only + var e = intercept[AnalysisException] { + spark.read.schema(s"$c0 INT, $c1 INT").csv(src.toString) + } + assert(e.getMessage.contains("Found duplicate column(s) in the datasource: ")) + + // data schema + partition schema + val partTestDir = Utils.createDirectory(src.toString) + Seq(1).toDF(c0).write.option("header", true).mode("overwrite") + .csv(s"${partTestDir.getAbsolutePath}/$c1=1") + e = intercept[AnalysisException] { + spark.read.option("header", true).csv(s"${partTestDir.getAbsolutePath}") } assert(e.getMessage.contains("Found duplicate column(s) in the datasource: ")) } } + } + withTempDir { src => // If `inferSchema` is true, a CSV format is duplicate-safe (See SPARK-16896) Seq("a,a", "1,1").toDF().coalesce(1).write.mode("overwrite").text(src.toString) - val df = spark.read.format("csv").option("inferSchema", true).option("header", true) - .load(src.toString) + val df = spark.read.option("inferSchema", true).option("header", true).csv(src.toString) checkAnswer(df, Row(1, 1)) + } - // Check JSON format - Seq((true, ("a", "a")), (false, ("aA", "Aa"))) - .foreach { case (caseSensitive, (c0, c1)) => - Seq(s"""{"$c0":1, "$c1":1}""").toDF().coalesce(1).write.mode("overwrite").text(src.toString) - withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive.toString) { - val e1 = intercept[AnalysisException] { - spark.read.format("json").schema(s"$c0 INT, $c1 INT").load(src.toString) + // Check JSON format + Seq((true, ("a", "a")), (false, ("aA", "Aa"))).foreach { case (caseSensitive, (c0, c1)) => + withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive.toString) { + withTempDir { src => + Seq(s"""{"$c0":1, "$c1":1}""").toDF().write.mode("overwrite").text(src.toString) + + // data schema only + var e = intercept[AnalysisException] { + spark.read.schema(s"$c0 INT, $c1 INT").json(src.toString) } - assert(e1.getMessage.contains("Found duplicate column(s) in the datasource: ")) + assert(e.getMessage.contains("Found duplicate column(s) in the datasource: ")) - val e2 = intercept[AnalysisException] { - spark.read.format("json").option("inferSchema", true).load(src.toString) + e = intercept[AnalysisException] { + spark.read.option("inferSchema", true).json(src.toString) } - assert(e2.getMessage.contains("Found duplicate column(s) in the datasource: ")) + assert(e.getMessage.contains("Found duplicate column(s) in the datasource: ")) + + // data schema + partition schema only + val partTestDir = Utils.createDirectory(src.toString) + Seq(1).toDF(c0).write.mode("overwrite").json(s"${partTestDir.getAbsolutePath}/$c1=1") + e = intercept[AnalysisException] { + spark.read.json(s"${partTestDir.getAbsolutePath}") + } + assert(e.getMessage.contains("Found duplicate column(s) in the datasource: ")) } } + } + + // Check Parquet format + Seq((true, ("a", "a")), (false, ("aA", "Aa"))).foreach { case (caseSensitive, (c0, c1)) => + withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive.toString) { + withTempDir { src => + Seq((1, 1)).toDF("c0", "c1").write.mode("overwrite").parquet(src.toString) + + // data schema only + var e = intercept[AnalysisException] { + spark.read.schema(s"$c0 INT, $c1 INT").parquet(src.toString) + } + assert(e.getMessage.contains("Found duplicate column(s) in the datasource: ")) - // Check Paruqet format - Seq((1, 1)).toDF("c0", "c1").coalesce(1).write.mode("overwrite").parquet(src.toString) - Seq((true, "a INT, a INT"), (false, "aA INT, Aa INT")) - .foreach { case (caseSensitive, schema) => - withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive.toString) { - val e = intercept[AnalysisException] { - spark.read.format("parquet").schema(schema).load(src.toString) + // data schema + partition schema only + val partTestDir = Utils.createDirectory(src.toString) + Seq(1).toDF(c0).write.mode("overwrite").parquet(s"${partTestDir.getAbsolutePath}/$c1=1") + e = intercept[AnalysisException] { + spark.read.parquet(s"${partTestDir.getAbsolutePath}") } assert(e.getMessage.contains("Found duplicate column(s) in the datasource: ")) } From 09da8d6611ada267076b2ce8f5ddde6ed661aec7 Mon Sep 17 00:00:00 2001 From: Takeshi Yamamuro Date: Tue, 20 Jun 2017 16:30:09 +0900 Subject: [PATCH 09/27] Drop name dplication checks in HiveMetastoreCatalog.scala --- .../org/apache/spark/sql/hive/HiveMetastoreCatalog.scala | 4 ---- 1 file changed, 4 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index 5afe6c144f81..56cf26643b08 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -249,10 +249,6 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log if (inferenceMode == INFER_AND_SAVE) { updateCatalogSchema(relation.tableMeta.identifier, schema) } - - SchemaUtils.checkSchemaColumnNameDuplication( - schema, "the hive serde table", sparkSession.sessionState.conf.caseSensitiveAnalysis) - (schema, relation.tableMeta.copy(schema = schema)) case None => logWarning(s"Unable to infer schema for table $tableName from file format " + From 6d03f31fb6e5872f2e5f22362c8c9a87d0d08fee Mon Sep 17 00:00:00 2001 From: Takeshi Yamamuro Date: Tue, 20 Jun 2017 17:13:57 +0900 Subject: [PATCH 10/27] Modify exception messages --- .../sql/catalyst/catalog/SessionCatalog.scala | 2 +- .../apache/spark/sql/util/SchemaUtils.scala | 2 +- .../spark/sql/util/SchemaUtilsSuite.scala | 53 ++++++++++--------- .../spark/sql/execution/command/views.scala | 8 +-- .../execution/datasources/DataSource.scala | 2 +- .../InsertIntoHadoopFsRelationCommand.scala | 4 +- .../datasources/PartitioningUtils.scala | 2 +- .../datasources/jdbc/JdbcUtils.scala | 2 +- .../sql/execution/datasources/rules.scala | 8 +-- .../org/apache/spark/sql/DataFrameSuite.scala | 4 +- .../spark/sql/hive/HiveMetastoreCatalog.scala | 1 - 11 files changed, 47 insertions(+), 41 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala index 84771d627fcd..0a7ca4280bd0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala @@ -343,7 +343,7 @@ class SessionCatalog( requireTableExists(tableIdentifier) SchemaUtils.checkSchemaColumnNameDuplication( - newSchema, "the table definition of " + tableIdentifier.identifier, + newSchema, "in the table definition of " + tableIdentifier.identifier, conf.caseSensitiveAnalysis) val catalogTable = externalCatalog.getTable(db, table) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/util/SchemaUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/util/SchemaUtils.scala index d03d91953b26..a89db3f2713f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/util/SchemaUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/util/SchemaUtils.scala @@ -67,7 +67,7 @@ private[spark] object SchemaUtils { } } if (duplicateColumns.size > 0) { - throw new AnalysisException(s"Found duplicate column(s) in $colType: " + + throw new AnalysisException(s"Found duplicate column(s) $colType: " + duplicateColumns.map(colName => s"`$colName`").mkString(", ")) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/util/SchemaUtilsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/util/SchemaUtilsSuite.scala index e40e99b71d61..8ccd827a63a0 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/util/SchemaUtilsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/util/SchemaUtilsSuite.scala @@ -27,128 +27,133 @@ class SchemaUtilsSuite extends SparkFunSuite { test("Check column name duplication in case-sensitive cases") { var msg = intercept[AnalysisException] { SchemaUtils.checkSchemaColumnNameDuplication( - StructType.fromDDL("a INT, b INT, a INT"), "SchemaUtilsSuite", caseSensitiveAnalysis = true) + StructType.fromDDL("a INT, b INT, a INT"), "in SchemaUtilsSuite", + caseSensitiveAnalysis = true) }.getMessage assert(msg.contains("Found duplicate column(s) in SchemaUtilsSuite: `a`;")) msg = intercept[AnalysisException] { SchemaUtils.checkColumnNameDuplication( - "a" :: "b" :: "a" :: Nil, "SchemaUtilsSuite", caseSensitiveResolution) + "a" :: "b" :: "a" :: Nil, "in SchemaUtilsSuite", caseSensitiveResolution) }.getMessage assert(msg.contains("Found duplicate column(s) in SchemaUtilsSuite: `a`;")) msg = intercept[AnalysisException] { SchemaUtils.checkSchemaColumnNameDuplication( - StructType.fromDDL("a INT, b INT, a INT, a INT"), "SchemaUtilsSuite", + StructType.fromDDL("a INT, b INT, a INT, a INT"), "in SchemaUtilsSuite", caseSensitiveAnalysis = true) }.getMessage assert(msg.contains("Found duplicate column(s) in SchemaUtilsSuite: `a`;")) msg = intercept[AnalysisException] { SchemaUtils.checkColumnNameDuplication( - "a" :: "b" :: "a" :: "a" :: Nil, "SchemaUtilsSuite", caseSensitiveResolution) + "a" :: "b" :: "a" :: "a" :: Nil, "in SchemaUtilsSuite", caseSensitiveResolution) }.getMessage assert(msg.contains("Found duplicate column(s) in SchemaUtilsSuite: `a`;")) msg = intercept[AnalysisException] { SchemaUtils.checkSchemaColumnNameDuplication( - StructType.fromDDL("a INT, b INT, a INT, b INT"), "SchemaUtilsSuite", + StructType.fromDDL("a INT, b INT, a INT, b INT"), "in SchemaUtilsSuite", caseSensitiveAnalysis = true) }.getMessage assert(msg.contains("Found duplicate column(s) in SchemaUtilsSuite: `a`, `b`;")) msg = intercept[AnalysisException] { SchemaUtils.checkColumnNameDuplication( - "a" :: "b" :: "a" :: "b" :: Nil, "SchemaUtilsSuite", caseSensitiveResolution) + "a" :: "b" :: "a" :: "b" :: Nil, "in SchemaUtilsSuite", caseSensitiveResolution) }.getMessage assert(msg.contains("Found duplicate column(s) in SchemaUtilsSuite: `a`, `b`;")) msg = intercept[AnalysisException] { SchemaUtils.checkSchemaColumnNameDuplication( - StructType.fromDDL("a INT, c INT, b INT, a INT, b INT, c INT"), "SchemaUtilsSuite", + StructType.fromDDL("a INT, c INT, b INT, a INT, b INT, c INT"), "in SchemaUtilsSuite", caseSensitiveAnalysis = true) }.getMessage assert(msg.contains("Found duplicate column(s) in SchemaUtilsSuite: `a`, `c`, `b`;")) msg = intercept[AnalysisException] { SchemaUtils.checkColumnNameDuplication( - "a" :: "c" :: "b" :: "a" :: "b" :: "c" :: Nil, "SchemaUtilsSuite", caseSensitiveResolution) + "a" :: "c" :: "b" :: "a" :: "b" :: "c" :: Nil, "in SchemaUtilsSuite", + caseSensitiveResolution) }.getMessage assert(msg.contains("Found duplicate column(s) in SchemaUtilsSuite: `a`, `c`, `b`;")) // Check no exception thrown SchemaUtils.checkSchemaColumnNameDuplication( - StructType.fromDDL("a INT, b INT, c INT"), "SchemaUtilsSuite", caseSensitiveAnalysis = true) + StructType.fromDDL("a INT, b INT, c INT"), "in SchemaUtilsSuite", + caseSensitiveAnalysis = true) SchemaUtils.checkColumnNameDuplication( - "a" :: "b" :: "c" :: Nil, "SchemaUtilsSuite", caseSensitiveResolution) + "a" :: "b" :: "c" :: Nil, "in SchemaUtilsSuite", caseSensitiveResolution) SchemaUtils.checkSchemaColumnNameDuplication( - StructType.fromDDL("Aa INT, b INT, aA INT"), "SchemaUtilsSuite", caseSensitiveAnalysis = true) + StructType.fromDDL("Aa INT, b INT, aA INT"), "in SchemaUtilsSuite", + caseSensitiveAnalysis = true) SchemaUtils.checkColumnNameDuplication( - "Aa" :: "b" :: "aA" :: Nil, "SchemaUtilsSuite", caseSensitiveResolution) + "Aa" :: "b" :: "aA" :: Nil, "in SchemaUtilsSuite", caseSensitiveResolution) } test("Check column name duplication in case-insensitive cases") { var msg = intercept[AnalysisException] { SchemaUtils.checkSchemaColumnNameDuplication( - StructType.fromDDL("Aa INT, b INT, Aa INT"), "SchemaUtilsSuite", + StructType.fromDDL("Aa INT, b INT, Aa INT"), "in SchemaUtilsSuite", caseSensitiveAnalysis = false) }.getMessage assert(msg.contains("Found duplicate column(s) in SchemaUtilsSuite: `Aa`;")) msg = intercept[AnalysisException] { SchemaUtils.checkColumnNameDuplication( - "Aa" :: "b" :: "Aa" :: Nil, "SchemaUtilsSuite", caseInsensitiveResolution) + "Aa" :: "b" :: "Aa" :: Nil, "in SchemaUtilsSuite", caseInsensitiveResolution) }.getMessage assert(msg.contains("Found duplicate column(s) in SchemaUtilsSuite: `Aa`;")) msg = intercept[AnalysisException] { SchemaUtils.checkSchemaColumnNameDuplication( - StructType.fromDDL("a INT, bB INT, Bb INT"), "SchemaUtilsSuite", + StructType.fromDDL("a INT, bB INT, Bb INT"), "in SchemaUtilsSuite", caseSensitiveAnalysis = false) }.getMessage assert(msg.contains("Found duplicate column(s) in SchemaUtilsSuite: `bB`;")) msg = intercept[AnalysisException] { SchemaUtils.checkColumnNameDuplication( - "a" :: "bB" :: "Bb" :: Nil, "SchemaUtilsSuite", caseInsensitiveResolution) + "a" :: "bB" :: "Bb" :: Nil, "in SchemaUtilsSuite", caseInsensitiveResolution) }.getMessage assert(msg.contains("Found duplicate column(s) in SchemaUtilsSuite: `bB`;")) msg = intercept[AnalysisException] { SchemaUtils.checkSchemaColumnNameDuplication( - StructType.fromDDL("Aa INT, b INT, Aa INT, c INT, aa INT"), "SchemaUtilsSuite", + StructType.fromDDL("Aa INT, b INT, Aa INT, c INT, aa INT"), "in SchemaUtilsSuite", caseSensitiveAnalysis = false) }.getMessage assert(msg.contains("Found duplicate column(s) in SchemaUtilsSuite: `Aa`;")) msg = intercept[AnalysisException] { SchemaUtils.checkColumnNameDuplication( - "Aa" :: "b" :: "Aa" :: "c" :: "aa" :: Nil, "SchemaUtilsSuite", caseInsensitiveResolution) + "Aa" :: "b" :: "Aa" :: "c" :: "aa" :: Nil, "in SchemaUtilsSuite", caseInsensitiveResolution) }.getMessage assert(msg.contains("Found duplicate column(s) in SchemaUtilsSuite: `Aa`;")) msg = intercept[AnalysisException] { SchemaUtils.checkSchemaColumnNameDuplication( - StructType.fromDDL("Aa INT, bB INT, Bb INT, aa INT"), "SchemaUtilsSuite", + StructType.fromDDL("Aa INT, bB INT, Bb INT, aa INT"), "in SchemaUtilsSuite", caseSensitiveAnalysis = false) }.getMessage assert(msg.contains("Found duplicate column(s) in SchemaUtilsSuite: `Aa`, `bB`;")) msg = intercept[AnalysisException] { SchemaUtils.checkColumnNameDuplication( - "Aa" :: "bB" :: "Bb" :: "aa" :: Nil, "SchemaUtilsSuite", caseInsensitiveResolution) + "Aa" :: "bB" :: "Bb" :: "aa" :: Nil, "in SchemaUtilsSuite", caseInsensitiveResolution) }.getMessage assert(msg.contains("Found duplicate column(s) in SchemaUtilsSuite: `Aa`, `bB`;")) msg = intercept[AnalysisException] { SchemaUtils.checkSchemaColumnNameDuplication( - StructType.fromDDL("Aa INT, cc INT, bB INT, cC INT, Bb INT, aa INT"), "SchemaUtilsSuite", + StructType.fromDDL("Aa INT, cc INT, bB INT, cC INT, Bb INT, aa INT"), "in SchemaUtilsSuite", caseSensitiveAnalysis = false) }.getMessage assert(msg.contains("Found duplicate column(s) in SchemaUtilsSuite: `Aa`, `cc`, `bB`;")) msg = intercept[AnalysisException] { SchemaUtils.checkColumnNameDuplication( - "Aa" :: "cc" :: "bB" :: "cC" :: "Bb" :: "aa" :: Nil, "SchemaUtilsSuite", + "Aa" :: "cc" :: "bB" :: "cC" :: "Bb" :: "aa" :: Nil, "in SchemaUtilsSuite", caseInsensitiveResolution) }.getMessage assert(msg.contains("Found duplicate column(s) in SchemaUtilsSuite: `Aa`, `cc`, `bB`;")) // Check no exception thrown SchemaUtils.checkSchemaColumnNameDuplication( - StructType.fromDDL("a INT, b INT, c INT"), "SchemaUtilsSuite", caseSensitiveAnalysis = false) + StructType.fromDDL("a INT, b INT, c INT"), "in SchemaUtilsSuite", + caseSensitiveAnalysis = false) SchemaUtils.checkColumnNameDuplication( - "a" :: "b" :: "c" :: Nil, "SchemaUtilsSuite", caseInsensitiveResolution) + "a" :: "b" :: "c" :: Nil, "in SchemaUtilsSuite", caseInsensitiveResolution) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala index ee7a877a9134..e167dc1b6a34 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala @@ -356,15 +356,15 @@ object ViewHelper { properties: Map[String, String], session: SparkSession, analyzedPlan: LogicalPlan): Map[String, String] = { + val queryOutput = analyzedPlan.schema.fieldNames + // Generate the query column names, throw an AnalysisException if there exists duplicate column // names. - SchemaUtils.checkSchemaColumnNameDuplication( - analyzedPlan.schema, "the view", session.sessionState.conf.caseSensitiveAnalysis) + SchemaUtils.checkColumnNameDuplication( + queryOutput, "the view", session.sessionState.conf.resolver) // Generate the view default database name. val viewDefaultDatabase = session.sessionState.catalog.getCurrentDatabase - val queryOutput = analyzedPlan.schema.fieldNames - removeQueryColumnNames(properties) ++ generateViewDefaultDatabase(viewDefaultDatabase) ++ generateQueryColumnNames(queryOutput) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala index cb7c03303499..960b58274d2f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala @@ -336,7 +336,7 @@ case class DataSource( } SchemaUtils.checkSchemaColumnNameDuplication( - dataSchema, "the datasource", sparkSession.sessionState.conf.caseSensitiveAnalysis) + dataSchema, "in the datasource", sparkSession.sessionState.conf.caseSensitiveAnalysis) HadoopFsRelation( fileCatalog, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala index 666ea062d6fe..a98cd3902214 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala @@ -66,7 +66,9 @@ case class InsertIntoHadoopFsRelationCommand( // Most formats don't do well with duplicate columns, so lets not allow that SchemaUtils.checkSchemaColumnNameDuplication( - query.schema, "the query", sparkSession.sessionState.conf.caseSensitiveAnalysis) + query.schema, + s"when inserting into $outputPath", + sparkSession.sessionState.conf.caseSensitiveAnalysis) val hadoopConf = sparkSession.sessionState.newHadoopConfWithOptions(options) val fs = outputPath.getFileSystem(hadoopConf) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala index e72e33d6784d..54cdd973420b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala @@ -303,7 +303,7 @@ object PartitioningUtils { } SchemaUtils.checkColumnNameDuplication( - normalizedPartSpec.map(_._1), "the partition specification", resolver) + normalizedPartSpec.map(_._1), "in the partition specification", resolver) normalizedPartSpec.toMap } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala index 9932a2a7da12..bbe9024f13a4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala @@ -751,7 +751,7 @@ object JdbcUtils extends Logging { // checks duplicate columns in the user specified column types. SchemaUtils.checkColumnNameDuplication( - userSchema.map(_.name), "the createTableColumnTypes option value", nameEquality) + userSchema.map(_.name), "in the createTableColumnTypes option value", nameEquality) // checks if user specified column names exist in the DataFrame schema userSchema.fieldNames.foreach { col => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala index 9df04f38115a..85feef9e5788 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala @@ -225,7 +225,7 @@ case class PreprocessTableCreation(sparkSession: SparkSession) extends Rule[Logi private def normalizeCatalogTable(schema: StructType, table: CatalogTable): CatalogTable = { SchemaUtils.checkSchemaColumnNameDuplication( schema, - "the table definition of " + table.identifier, + "in the table definition of " + table.identifier, sparkSession.sessionState.conf.caseSensitiveAnalysis) val normalizedPartCols = normalizePartitionColumns(schema, table) @@ -254,7 +254,7 @@ case class PreprocessTableCreation(sparkSession: SparkSession) extends Rule[Logi SchemaUtils.checkColumnNameDuplication( normalizedPartitionCols, - "the partition", + "in the partition", sparkSession.sessionState.conf.resolver) if (schema.nonEmpty && normalizedPartitionCols.length == schema.length) { @@ -288,11 +288,11 @@ case class PreprocessTableCreation(sparkSession: SparkSession) extends Rule[Logi SchemaUtils.checkColumnNameDuplication( normalizedBucketSpec.bucketColumnNames, - "the bucket", + "in the bucket", sparkSession.sessionState.conf.resolver) SchemaUtils.checkColumnNameDuplication( normalizedBucketSpec.sortColumnNames, - "the sort", + "in the sort", sparkSession.sessionState.conf.resolver) normalizedBucketSpec.sortColumnNames.map(schema(_)).map(_.dataType).foreach { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala index 8c73d20cef03..3c08b40a2652 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala @@ -1123,7 +1123,7 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { Seq((1, 2, 3), (2, 3, 4), (3, 4, 5)).toDF("column1", "column2", "column1") .write.format("parquet").save("temp") } - assert(e.getMessage.contains("Found duplicate column(s) in the query: ")) + assert(e.getMessage.contains("Found duplicate column(s) when inserting into")) assert(e.getMessage.contains("column1")) assert(!e.getMessage.contains("column2")) @@ -1133,7 +1133,7 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { .toDF("column1", "column2", "column3", "column1", "column3") .write.format("json").save("temp") } - assert(f.getMessage.contains("Found duplicate column(s) in the query: ")) + assert(f.getMessage.contains("Found duplicate column(s) when inserting into")) assert(f.getMessage.contains("column1")) assert(f.getMessage.contains("column3")) assert(!f.getMessage.contains("column2")) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index 56cf26643b08..808dc013f170 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -31,7 +31,6 @@ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.internal.SQLConf.HiveCaseSensitiveInferenceMode._ import org.apache.spark.sql.types._ -import org.apache.spark.sql.util.SchemaUtils /** * Legacy catalog for interacting with the Hive metastore. From a0b9b0578d6c0dd556240fc471e857b2c95d0848 Mon Sep 17 00:00:00 2001 From: Takeshi Yamamuro Date: Tue, 20 Jun 2017 18:47:27 +0900 Subject: [PATCH 11/27] Revert logic to check name duplication --- .../apache/spark/sql/util/SchemaUtils.scala | 53 ++++-- .../spark/sql/util/SchemaUtilsSuite.scala | 178 ++++++------------ 2 files changed, 98 insertions(+), 133 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/util/SchemaUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/util/SchemaUtils.scala index a89db3f2713f..716a0912bdd5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/util/SchemaUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/util/SchemaUtils.scala @@ -17,8 +17,6 @@ package org.apache.spark.sql.util -import scala.collection.mutable - import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.types.StructType @@ -41,34 +39,55 @@ private[spark] object SchemaUtils { */ def checkSchemaColumnNameDuplication( schema: StructType, colType: String, caseSensitiveAnalysis: Boolean = false): Unit = { - val resolver = if (caseSensitiveAnalysis) { - caseSensitiveResolution + checkColumnNameDuplication(schema.map(_.name), colType, caseSensitiveAnalysis) + } + + // Returns true if a given resolver is case-sensitive + private def isCaseSensitiveAnalysis(resolver: Resolver): Boolean = { + if (resolver == caseSensitiveResolution) { + true + } else if (resolver == caseInsensitiveResolution) { + false } else { - caseInsensitiveResolution + sys.error("A resolver to check if two identifiers are equal must be " + + "`caseSensitiveResolution` or `caseInsensitiveResolution` in o.a.s.sql.catalyst.") } - checkColumnNameDuplication(schema.map(_.name), colType, resolver) } /** * Checks if input column names have duplicate identifiers. This throws an exception if * the duplication exists. * - * @param colNames column names to check + * @param columnNames column names to check * @param colType column type name, used in an exception message * @param resolver resolver used to determine if two identifiers are equal */ def checkColumnNameDuplication( - colNames: Seq[String], colType: String, resolver: Resolver): Unit = { - val duplicateColumns = mutable.ArrayBuffer[String]() - colNames.foreach { name => - val sameColNames = colNames.filter(resolver(_, name)) - if (sameColNames.size > 1 && !duplicateColumns.exists(resolver(_, name))) { - duplicateColumns.append(name) - } + columnNames: Seq[String], colType: String, resolver: Resolver): Unit = { + checkColumnNameDuplication(columnNames, colType, isCaseSensitiveAnalysis(resolver)) + } + + /** + * Checks if input column names have duplicate identifiers. This throws an exception if + * the duplication exists. + * + * @param columnNames column names to check + * @param colType column type name, used in an exception message + * @param caseSensitiveAnalysis whether duplication checks should be case sensitive or not + */ + def checkColumnNameDuplication( + columnNames: Seq[String], colType: String, caseSensitiveAnalysis: Boolean): Unit = { + val names = if (caseSensitiveAnalysis) { + columnNames + } else { + columnNames.map(_.toLowerCase) } - if (duplicateColumns.size > 0) { - throw new AnalysisException(s"Found duplicate column(s) $colType: " + - duplicateColumns.map(colName => s"`$colName`").mkString(", ")) + if (names.distinct.length != names.length) { + val duplicateColumns = names.groupBy(identity).collect { + case (x, ys) if ys.length > 1 => s"`$x`" + } + throw new AnalysisException( + s"Found duplicate column(s) $colType: ${duplicateColumns.mkString(", ")}") } } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/util/SchemaUtilsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/util/SchemaUtilsSuite.scala index 8ccd827a63a0..68fca706f68e 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/util/SchemaUtilsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/util/SchemaUtilsSuite.scala @@ -25,135 +25,81 @@ import org.apache.spark.sql.types.StructType class SchemaUtilsSuite extends SparkFunSuite { test("Check column name duplication in case-sensitive cases") { - var msg = intercept[AnalysisException] { - SchemaUtils.checkSchemaColumnNameDuplication( - StructType.fromDDL("a INT, b INT, a INT"), "in SchemaUtilsSuite", - caseSensitiveAnalysis = true) - }.getMessage - assert(msg.contains("Found duplicate column(s) in SchemaUtilsSuite: `a`;")) - msg = intercept[AnalysisException] { - SchemaUtils.checkColumnNameDuplication( - "a" :: "b" :: "a" :: Nil, "in SchemaUtilsSuite", caseSensitiveResolution) - }.getMessage - assert(msg.contains("Found duplicate column(s) in SchemaUtilsSuite: `a`;")) + def checkCaseSensitiveExceptionCases(schemaStr: String, duplicatedColumns: String): Unit = { + val expectedErrorMsg = s"Found duplicate column(s) in SchemaUtilsSuite: $duplicatedColumns" + val schema = StructType.fromDDL(schemaStr) + var msg = intercept[AnalysisException] { + SchemaUtils.checkSchemaColumnNameDuplication( + schema, "in SchemaUtilsSuite", caseSensitiveAnalysis = true) + }.getMessage + assert(msg.contains(expectedErrorMsg)) + msg = intercept[AnalysisException] { + SchemaUtils.checkColumnNameDuplication( + schema.map(_.name), "in SchemaUtilsSuite", caseSensitiveResolution) + }.getMessage + assert(msg.contains(expectedErrorMsg)) + msg = intercept[AnalysisException] { + SchemaUtils.checkColumnNameDuplication( + schema.map(_.name), "in SchemaUtilsSuite", caseSensitiveAnalysis = true) + }.getMessage + assert(msg.contains(expectedErrorMsg)) + } - msg = intercept[AnalysisException] { - SchemaUtils.checkSchemaColumnNameDuplication( - StructType.fromDDL("a INT, b INT, a INT, a INT"), "in SchemaUtilsSuite", - caseSensitiveAnalysis = true) - }.getMessage - assert(msg.contains("Found duplicate column(s) in SchemaUtilsSuite: `a`;")) - msg = intercept[AnalysisException] { - SchemaUtils.checkColumnNameDuplication( - "a" :: "b" :: "a" :: "a" :: Nil, "in SchemaUtilsSuite", caseSensitiveResolution) - }.getMessage - assert(msg.contains("Found duplicate column(s) in SchemaUtilsSuite: `a`;")) + checkCaseSensitiveExceptionCases("a INT, b INT, a INT", "`a`") + checkCaseSensitiveExceptionCases("a INT, b INT, a INT, a INT", "`a`") + checkCaseSensitiveExceptionCases("a INT, b INT, a INT, b INT", "`b`, `a`") + checkCaseSensitiveExceptionCases("a INT, c INT, b INT, a INT, b INT, c INT", "`b`, `a`, `c`") - msg = intercept[AnalysisException] { + // Check no exception thrown + def checkCaseSensitiveNoExceptionCases(schemaStr: String): Unit = { + val schema = StructType.fromDDL(schemaStr) SchemaUtils.checkSchemaColumnNameDuplication( - StructType.fromDDL("a INT, b INT, a INT, b INT"), "in SchemaUtilsSuite", - caseSensitiveAnalysis = true) - }.getMessage - assert(msg.contains("Found duplicate column(s) in SchemaUtilsSuite: `a`, `b`;")) - msg = intercept[AnalysisException] { + schema, "in SchemaUtilsSuite", caseSensitiveAnalysis = true) SchemaUtils.checkColumnNameDuplication( - "a" :: "b" :: "a" :: "b" :: Nil, "in SchemaUtilsSuite", caseSensitiveResolution) - }.getMessage - assert(msg.contains("Found duplicate column(s) in SchemaUtilsSuite: `a`, `b`;")) - - msg = intercept[AnalysisException] { - SchemaUtils.checkSchemaColumnNameDuplication( - StructType.fromDDL("a INT, c INT, b INT, a INT, b INT, c INT"), "in SchemaUtilsSuite", - caseSensitiveAnalysis = true) - }.getMessage - assert(msg.contains("Found duplicate column(s) in SchemaUtilsSuite: `a`, `c`, `b`;")) - msg = intercept[AnalysisException] { + schema.map(_.name), "in SchemaUtilsSuite", caseSensitiveResolution) SchemaUtils.checkColumnNameDuplication( - "a" :: "c" :: "b" :: "a" :: "b" :: "c" :: Nil, "in SchemaUtilsSuite", - caseSensitiveResolution) - }.getMessage - assert(msg.contains("Found duplicate column(s) in SchemaUtilsSuite: `a`, `c`, `b`;")) + schema.map(_.name), "in SchemaUtilsSuite", caseSensitiveAnalysis = true) + } - // Check no exception thrown - SchemaUtils.checkSchemaColumnNameDuplication( - StructType.fromDDL("a INT, b INT, c INT"), "in SchemaUtilsSuite", - caseSensitiveAnalysis = true) - SchemaUtils.checkColumnNameDuplication( - "a" :: "b" :: "c" :: Nil, "in SchemaUtilsSuite", caseSensitiveResolution) - SchemaUtils.checkSchemaColumnNameDuplication( - StructType.fromDDL("Aa INT, b INT, aA INT"), "in SchemaUtilsSuite", - caseSensitiveAnalysis = true) - SchemaUtils.checkColumnNameDuplication( - "Aa" :: "b" :: "aA" :: Nil, "in SchemaUtilsSuite", caseSensitiveResolution) + checkCaseSensitiveNoExceptionCases("a INT, b INT, c INT") + checkCaseSensitiveNoExceptionCases("Aa INT, b INT, aA INT") } test("Check column name duplication in case-insensitive cases") { - var msg = intercept[AnalysisException] { - SchemaUtils.checkSchemaColumnNameDuplication( - StructType.fromDDL("Aa INT, b INT, Aa INT"), "in SchemaUtilsSuite", - caseSensitiveAnalysis = false) - }.getMessage - assert(msg.contains("Found duplicate column(s) in SchemaUtilsSuite: `Aa`;")) - msg = intercept[AnalysisException] { - SchemaUtils.checkColumnNameDuplication( - "Aa" :: "b" :: "Aa" :: Nil, "in SchemaUtilsSuite", caseInsensitiveResolution) - }.getMessage - assert(msg.contains("Found duplicate column(s) in SchemaUtilsSuite: `Aa`;")) + def checkCaseInsensitiveExceptionCases(schemaStr: String, duplicatedColumns: String): Unit = { + val expectedErrorMsg = s"Found duplicate column(s) in SchemaUtilsSuite: $duplicatedColumns" + val schema = StructType.fromDDL(schemaStr) + var msg = intercept[AnalysisException] { + SchemaUtils.checkSchemaColumnNameDuplication( + schema, "in SchemaUtilsSuite", caseSensitiveAnalysis = false) + }.getMessage + assert(msg.contains(expectedErrorMsg)) + msg = intercept[AnalysisException] { + SchemaUtils.checkColumnNameDuplication( + schema.map(_.name), "in SchemaUtilsSuite", caseInsensitiveResolution) + }.getMessage + assert(msg.contains(expectedErrorMsg)) + msg = intercept[AnalysisException] { + SchemaUtils.checkColumnNameDuplication( + schema.map(_.name), "in SchemaUtilsSuite", caseSensitiveAnalysis = false) + }.getMessage + assert(msg.contains(expectedErrorMsg)) + } - msg = intercept[AnalysisException] { - SchemaUtils.checkSchemaColumnNameDuplication( - StructType.fromDDL("a INT, bB INT, Bb INT"), "in SchemaUtilsSuite", - caseSensitiveAnalysis = false) - }.getMessage - assert(msg.contains("Found duplicate column(s) in SchemaUtilsSuite: `bB`;")) - msg = intercept[AnalysisException] { - SchemaUtils.checkColumnNameDuplication( - "a" :: "bB" :: "Bb" :: Nil, "in SchemaUtilsSuite", caseInsensitiveResolution) - }.getMessage - assert(msg.contains("Found duplicate column(s) in SchemaUtilsSuite: `bB`;")) - - msg = intercept[AnalysisException] { - SchemaUtils.checkSchemaColumnNameDuplication( - StructType.fromDDL("Aa INT, b INT, Aa INT, c INT, aa INT"), "in SchemaUtilsSuite", - caseSensitiveAnalysis = false) - }.getMessage - assert(msg.contains("Found duplicate column(s) in SchemaUtilsSuite: `Aa`;")) - msg = intercept[AnalysisException] { - SchemaUtils.checkColumnNameDuplication( - "Aa" :: "b" :: "Aa" :: "c" :: "aa" :: Nil, "in SchemaUtilsSuite", caseInsensitiveResolution) - }.getMessage - assert(msg.contains("Found duplicate column(s) in SchemaUtilsSuite: `Aa`;")) - - msg = intercept[AnalysisException] { - SchemaUtils.checkSchemaColumnNameDuplication( - StructType.fromDDL("Aa INT, bB INT, Bb INT, aa INT"), "in SchemaUtilsSuite", - caseSensitiveAnalysis = false) - }.getMessage - assert(msg.contains("Found duplicate column(s) in SchemaUtilsSuite: `Aa`, `bB`;")) - msg = intercept[AnalysisException] { - SchemaUtils.checkColumnNameDuplication( - "Aa" :: "bB" :: "Bb" :: "aa" :: Nil, "in SchemaUtilsSuite", caseInsensitiveResolution) - }.getMessage - assert(msg.contains("Found duplicate column(s) in SchemaUtilsSuite: `Aa`, `bB`;")) - - msg = intercept[AnalysisException] { - SchemaUtils.checkSchemaColumnNameDuplication( - StructType.fromDDL("Aa INT, cc INT, bB INT, cC INT, Bb INT, aa INT"), "in SchemaUtilsSuite", - caseSensitiveAnalysis = false) - }.getMessage - assert(msg.contains("Found duplicate column(s) in SchemaUtilsSuite: `Aa`, `cc`, `bB`;")) - msg = intercept[AnalysisException] { - SchemaUtils.checkColumnNameDuplication( - "Aa" :: "cc" :: "bB" :: "cC" :: "Bb" :: "aa" :: Nil, "in SchemaUtilsSuite", - caseInsensitiveResolution) - }.getMessage - assert(msg.contains("Found duplicate column(s) in SchemaUtilsSuite: `Aa`, `cc`, `bB`;")) + checkCaseInsensitiveExceptionCases("Aa INT, b INT, Aa INT", "`aa`") + checkCaseInsensitiveExceptionCases("a INT, bB INT, Bb INT", "`bb`") + checkCaseInsensitiveExceptionCases("Aa INT, b INT, Aa INT, c INT, aa INT", "`aa`") + checkCaseInsensitiveExceptionCases("Aa INT, bB INT, Bb INT, aa INT", "`bb`, `aa`") + checkCaseInsensitiveExceptionCases( + "Aa INT, cc INT, bB INT, cC INT, Bb INT, aa INT", "`bb`, `cc`, `aa`") // Check no exception thrown + val schema = StructType.fromDDL("a INT, b INT, c INT") SchemaUtils.checkSchemaColumnNameDuplication( - StructType.fromDDL("a INT, b INT, c INT"), "in SchemaUtilsSuite", - caseSensitiveAnalysis = false) + schema, "in SchemaUtilsSuite", caseSensitiveAnalysis = false) + SchemaUtils.checkColumnNameDuplication( + schema.map(_.name), "in SchemaUtilsSuite", caseInsensitiveResolution) SchemaUtils.checkColumnNameDuplication( - "a" :: "b" :: "c" :: Nil, "in SchemaUtilsSuite", caseInsensitiveResolution) + schema.map(_.name), "in SchemaUtilsSuite", caseSensitiveAnalysis = false) } } From 91b64245daa285778de81e5499fe9f52d79dbf2b Mon Sep 17 00:00:00 2001 From: Takeshi Yamamuro Date: Wed, 21 Jun 2017 12:13:09 +0900 Subject: [PATCH 12/27] Add tests for write paths --- .../sql/test/DataFrameReaderWriterSuite.scala | 27 ++++++++++++++++--- 1 file changed, 24 insertions(+), 3 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala index 5e38d96c4243..fd27ec0f2a49 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala @@ -694,10 +694,17 @@ class DataFrameReaderWriterSuite extends QueryTest with SharedSQLContext with Be Seq((true, ("a", "a")), (false, ("aA", "Aa"))).foreach { case (caseSensitive, (c0, c1)) => withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive.toString) { withTempDir { src => + // Check write path + var e = intercept[AnalysisException] { + Seq((1, 1)).toDF(c0, c1).write.mode("overwrite").csv(src.toString) + } + assert(e.getMessage.contains("Found duplicate column(s) when inserting into")) + + // Check read path Seq("1,1").toDF().write.mode("overwrite").text(src.toString) // data schema only - var e = intercept[AnalysisException] { + e = intercept[AnalysisException] { spark.read.schema(s"$c0 INT, $c1 INT").csv(src.toString) } assert(e.getMessage.contains("Found duplicate column(s) in the datasource: ")) @@ -725,10 +732,17 @@ class DataFrameReaderWriterSuite extends QueryTest with SharedSQLContext with Be Seq((true, ("a", "a")), (false, ("aA", "Aa"))).foreach { case (caseSensitive, (c0, c1)) => withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive.toString) { withTempDir { src => + // Check write path + var e = intercept[AnalysisException] { + Seq((1, 1)).toDF(c0, c1).write.mode("overwrite").csv(src.toString) + } + assert(e.getMessage.contains("Found duplicate column(s) when inserting into")) + + // Check read path Seq(s"""{"$c0":1, "$c1":1}""").toDF().write.mode("overwrite").text(src.toString) // data schema only - var e = intercept[AnalysisException] { + e = intercept[AnalysisException] { spark.read.schema(s"$c0 INT, $c1 INT").json(src.toString) } assert(e.getMessage.contains("Found duplicate column(s) in the datasource: ")) @@ -753,10 +767,17 @@ class DataFrameReaderWriterSuite extends QueryTest with SharedSQLContext with Be Seq((true, ("a", "a")), (false, ("aA", "Aa"))).foreach { case (caseSensitive, (c0, c1)) => withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive.toString) { withTempDir { src => + // Check write path + var e = intercept[AnalysisException] { + Seq((1, 1)).toDF(c0, c1).write.mode("overwrite").csv(src.toString) + } + assert(e.getMessage.contains("Found duplicate column(s) when inserting into")) + + // Check read path Seq((1, 1)).toDF("c0", "c1").write.mode("overwrite").parquet(src.toString) // data schema only - var e = intercept[AnalysisException] { + e = intercept[AnalysisException] { spark.read.schema(s"$c0 INT, $c1 INT").parquet(src.toString) } assert(e.getMessage.contains("Found duplicate column(s) in the datasource: ")) From 37ad3f36818367620946a2058066c0bd5208c8e2 Mon Sep 17 00:00:00 2001 From: Takeshi Yamamuro Date: Wed, 21 Jun 2017 16:58:35 +0900 Subject: [PATCH 13/27] Add tests for stream sink paths --- .../sql/streaming/FileStreamSinkSuite.scala | 37 +++++++++++++++++++ 1 file changed, 37 insertions(+) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala index bb6a27803bb2..998d33e73ca8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala @@ -26,6 +26,7 @@ import org.apache.spark.sql.execution.DataSourceScanExec import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.functions._ +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.{IntegerType, StructField, StructType} import org.apache.spark.util.Utils @@ -352,4 +353,40 @@ class FileStreamSinkSuite extends StreamTest { assertAncestorIsNotMetadataDirectory(s"/a/b/c") assertAncestorIsNotMetadataDirectory(s"/a/b/c/${FileStreamSink.metadataDir}extra") } + + test("SPARK-20460 Check name duplication in schema") { + Seq((true, ("a", "a")), (false, ("aA", "Aa"))).foreach { case (caseSensitive, (c0, c1)) => + withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive.toString) { + val inputData = MemoryStream[(Int, Int)] + val df = inputData.toDF() + + val outputDir = Utils.createTempDir(namePrefix = "stream.output").getCanonicalPath + val checkpointDir = Utils.createTempDir(namePrefix = "stream.checkpoint").getCanonicalPath + + var query: StreamingQuery = null + try { + query = + df.writeStream + .option("checkpointLocation", checkpointDir) + .format("json") + .start(outputDir) + + inputData.addData((1, 1)) + + failAfter(streamingTimeout) { + query.processAllAvailable() + } + } finally { + if (query != null) { + query.stop() + } + } + + val errorMsg = intercept[AnalysisException] { + spark.read.schema(s"$c0 INT, $c1 INT").json(outputDir).as[(Int, Int)] + }.getMessage + assert(errorMsg.contains("Found duplicate column(s) in the datasource: ")) + } + } + } } From d0d9d3e7e0fba0257879b351106b2480c8e0240d Mon Sep 17 00:00:00 2001 From: Takeshi Yamamuro Date: Sun, 25 Jun 2017 18:14:13 +0900 Subject: [PATCH 14/27] Burhs up code and adds more tests --- .../apache/spark/sql/util/SchemaUtils.scala | 6 +- .../execution/datasources/DataSource.scala | 18 ++- .../sql/test/DataFrameReaderWriterSuite.scala | 147 +++++++----------- 3 files changed, 74 insertions(+), 97 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/util/SchemaUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/util/SchemaUtils.scala index 716a0912bdd5..41ca270095ff 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/util/SchemaUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/util/SchemaUtils.scala @@ -77,11 +77,7 @@ private[spark] object SchemaUtils { */ def checkColumnNameDuplication( columnNames: Seq[String], colType: String, caseSensitiveAnalysis: Boolean): Unit = { - val names = if (caseSensitiveAnalysis) { - columnNames - } else { - columnNames.map(_.toLowerCase) - } + val names = if (caseSensitiveAnalysis) columnNames else columnNames.map(_.toLowerCase) if (names.distinct.length != names.length) { val duplicateColumns = names.groupBy(identity).collect { case (x, ys) if ys.length > 1 => s"`$x`" diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala index 960b58274d2f..7f3d463c73ed 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala @@ -183,11 +183,21 @@ case class DataSource( } SchemaUtils.checkColumnNameDuplication( - (dataSchema ++ partitionSchema).map(_.name), "in the data schema and the partition schema", - equality) - + dataSchema.map(_.name), "in the data schema", equality) SchemaUtils.checkColumnNameDuplication( - dataSchema.map(_.name), "the datasource", equality) + partitionSchema.map(_.name), "in the partition schema", equality) + + // We just print a waring message if the data schema and partition schema have the duplicate + // columns. This is because we allow users to do so in the previous Spark releases and + // we have the existing tests for the cases (e.g., `ParquetHadoopFsRelationSuite`). + // See SPARK-18108 and SPARK-21144 for related discussions. + try { + SchemaUtils.checkColumnNameDuplication( + (dataSchema ++ partitionSchema).map(_.name), "in the data schema and the partition schema", + equality) + } catch { + case e: AnalysisException => logWarning(e.getMessage) + } (dataSchema, partitionSchema) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala index fd27ec0f2a49..9c1f1a50591a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala @@ -690,105 +690,76 @@ class DataFrameReaderWriterSuite extends QueryTest with SharedSQLContext with Be } test("SPARK-20460 Check name duplication in schema") { - // Check CSV format - Seq((true, ("a", "a")), (false, ("aA", "Aa"))).foreach { case (caseSensitive, (c0, c1)) => - withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive.toString) { - withTempDir { src => - // Check write path - var e = intercept[AnalysisException] { - Seq((1, 1)).toDF(c0, c1).write.mode("overwrite").csv(src.toString) - } - assert(e.getMessage.contains("Found duplicate column(s) when inserting into")) - - // Check read path - Seq("1,1").toDF().write.mode("overwrite").text(src.toString) - - // data schema only - e = intercept[AnalysisException] { - spark.read.schema(s"$c0 INT, $c1 INT").csv(src.toString) - } - assert(e.getMessage.contains("Found duplicate column(s) in the datasource: ")) - - // data schema + partition schema - val partTestDir = Utils.createDirectory(src.toString) - Seq(1).toDF(c0).write.option("header", true).mode("overwrite") - .csv(s"${partTestDir.getAbsolutePath}/$c1=1") - e = intercept[AnalysisException] { - spark.read.option("header", true).csv(s"${partTestDir.getAbsolutePath}") - } - assert(e.getMessage.contains("Found duplicate column(s) in the datasource: ")) - } + def checkWriteDataColumnDuplication( + format: String, colName0: String, colName1: String, tempDir: File): Unit = { + val e = intercept[AnalysisException] { + Seq((1, 1)).toDF(colName0, colName1).write.format(format).mode("overwrite") + .save(tempDir.getAbsolutePath) } + assert(e.getMessage.contains("Found duplicate column(s) when inserting into")) } - withTempDir { src => - // If `inferSchema` is true, a CSV format is duplicate-safe (See SPARK-16896) - Seq("a,a", "1,1").toDF().coalesce(1).write.mode("overwrite").text(src.toString) - val df = spark.read.option("inferSchema", true).option("header", true).csv(src.toString) - checkAnswer(df, Row(1, 1)) + def checkReadUserSpecifiedDataColumnDuplication( + df: DataFrame, format: String, colName0: String, colName1: String, tempDir: File): Unit = { + val testDir = Utils.createTempDir(tempDir.getAbsolutePath) + df.write.format(format).mode("overwrite").save(testDir.getAbsolutePath) + val e = intercept[AnalysisException] { + spark.read.format(format).schema(s"$colName0 INT, $colName1 INT") + .load(testDir.getAbsolutePath) + } + assert(e.getMessage.contains("Found duplicate column(s) in the data schema:")) } - // Check JSON format - Seq((true, ("a", "a")), (false, ("aA", "Aa"))).foreach { case (caseSensitive, (c0, c1)) => - withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive.toString) { - withTempDir { src => - // Check write path - var e = intercept[AnalysisException] { - Seq((1, 1)).toDF(c0, c1).write.mode("overwrite").csv(src.toString) - } - assert(e.getMessage.contains("Found duplicate column(s) when inserting into")) - - // Check read path - Seq(s"""{"$c0":1, "$c1":1}""").toDF().write.mode("overwrite").text(src.toString) - - // data schema only - e = intercept[AnalysisException] { - spark.read.schema(s"$c0 INT, $c1 INT").json(src.toString) - } - assert(e.getMessage.contains("Found duplicate column(s) in the datasource: ")) - - e = intercept[AnalysisException] { - spark.read.option("inferSchema", true).json(src.toString) - } - assert(e.getMessage.contains("Found duplicate column(s) in the datasource: ")) - - // data schema + partition schema only - val partTestDir = Utils.createDirectory(src.toString) - Seq(1).toDF(c0).write.mode("overwrite").json(s"${partTestDir.getAbsolutePath}/$c1=1") - e = intercept[AnalysisException] { - spark.read.json(s"${partTestDir.getAbsolutePath}") - } - assert(e.getMessage.contains("Found duplicate column(s) in the datasource: ")) - } + def checkReadInferredDataColumnDuplication( + df: DataFrame, format: String, colName0: String, colName1: String, tempDir: File): Unit = { + val testDir = Utils.createTempDir(tempDir.getAbsolutePath) + df.toDF().write.mode("overwrite").text(testDir.getAbsolutePath) + val e = intercept[AnalysisException] { + spark.read.format(format).option("inferSchema", true).load(testDir.getAbsolutePath) + + } + assert(e.getMessage.contains("Found duplicate column(s) in the data schema:")) + } + + def checkReadPartitionColumnDuplication( + format: String, colName0: String, colName1: String, tempDir: File): Unit = { + val testDir = Utils.createTempDir(tempDir.getAbsolutePath) + Seq(1).toDF("col").write.format(format).mode("overwrite") + .save(s"${testDir.getAbsolutePath}/$colName0=1/$colName1=1") + val e = intercept[AnalysisException] { + spark.read.format(format).load(testDir.getAbsolutePath) } + assert(e.getMessage.contains("Found duplicate column(s) in the partition schema:")) } - // Check Parquet format Seq((true, ("a", "a")), (false, ("aA", "Aa"))).foreach { case (caseSensitive, (c0, c1)) => withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive.toString) { withTempDir { src => - // Check write path - var e = intercept[AnalysisException] { - Seq((1, 1)).toDF(c0, c1).write.mode("overwrite").csv(src.toString) - } - assert(e.getMessage.contains("Found duplicate column(s) when inserting into")) - - // Check read path - Seq((1, 1)).toDF("c0", "c1").write.mode("overwrite").parquet(src.toString) - - // data schema only - e = intercept[AnalysisException] { - spark.read.schema(s"$c0 INT, $c1 INT").parquet(src.toString) - } - assert(e.getMessage.contains("Found duplicate column(s) in the datasource: ")) - - // data schema + partition schema only - val partTestDir = Utils.createDirectory(src.toString) - Seq(1).toDF(c0).write.mode("overwrite").parquet(s"${partTestDir.getAbsolutePath}/$c1=1") - e = intercept[AnalysisException] { - spark.read.parquet(s"${partTestDir.getAbsolutePath}") - } - assert(e.getMessage.contains("Found duplicate column(s) in the datasource: ")) + // Check CSV format + checkWriteDataColumnDuplication("csv", c0, c1, src) + checkReadUserSpecifiedDataColumnDuplication( + Seq((1, 1)).toDF("c0", "c1"), "csv", c0, c1, src) + // If `inferSchema` is true, a CSV format is duplicate-safe (See SPARK-16896) + val testDir = Utils.createTempDir(src.getAbsolutePath) + Seq("a,a", "1,1").toDF().coalesce(1).write.mode("overwrite").text(testDir.getAbsolutePath) + val df = spark.read.format("csv").option("inferSchema", true).option("header", true) + .load(testDir.getAbsolutePath) + checkAnswer(df, Row(1, 1)) + checkReadPartitionColumnDuplication("csv", c0, c1, src) + + // Check JSON format + checkWriteDataColumnDuplication("json", c0, c1, src) + checkReadUserSpecifiedDataColumnDuplication( + Seq((1, 1)).toDF("c0", "c1"), "json", c0, c1, src) + checkReadInferredDataColumnDuplication( + Seq(s"""{"$c0":3, "$c1":5}""").toDF(), "json", c0, c1, src) + checkReadPartitionColumnDuplication("json", c0, c1, src) + + // Check Parquet format + checkWriteDataColumnDuplication("parquet", c0, c1, src) + checkReadUserSpecifiedDataColumnDuplication( + Seq((1, 1)).toDF("c0", "c1"), "parquet", c0, c1, src) + checkReadPartitionColumnDuplication("parquet", c0, c1, src) } } } From cbe9c71b3c6d131f698e9ab580122120dd98b626 Mon Sep 17 00:00:00 2001 From: Takeshi Yamamuro Date: Mon, 26 Jun 2017 17:42:53 +0900 Subject: [PATCH 15/27] Apply reviews --- .../spark/sql/util/SchemaUtilsSuite.scala | 112 +++++++----------- .../sql-tests/results/create.sql.out | 4 +- .../sql/test/DataFrameReaderWriterSuite.scala | 1 - 3 files changed, 47 insertions(+), 70 deletions(-) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/util/SchemaUtilsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/util/SchemaUtilsSuite.scala index 68fca706f68e..a25be2fe61db 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/util/SchemaUtilsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/util/SchemaUtilsSuite.scala @@ -24,82 +24,60 @@ import org.apache.spark.sql.types.StructType class SchemaUtilsSuite extends SparkFunSuite { - test("Check column name duplication in case-sensitive cases") { - def checkCaseSensitiveExceptionCases(schemaStr: String, duplicatedColumns: String): Unit = { - val expectedErrorMsg = s"Found duplicate column(s) in SchemaUtilsSuite: $duplicatedColumns" - val schema = StructType.fromDDL(schemaStr) - var msg = intercept[AnalysisException] { - SchemaUtils.checkSchemaColumnNameDuplication( - schema, "in SchemaUtilsSuite", caseSensitiveAnalysis = true) - }.getMessage - assert(msg.contains(expectedErrorMsg)) - msg = intercept[AnalysisException] { - SchemaUtils.checkColumnNameDuplication( - schema.map(_.name), "in SchemaUtilsSuite", caseSensitiveResolution) - }.getMessage - assert(msg.contains(expectedErrorMsg)) - msg = intercept[AnalysisException] { - SchemaUtils.checkColumnNameDuplication( - schema.map(_.name), "in SchemaUtilsSuite", caseSensitiveAnalysis = true) - }.getMessage - assert(msg.contains(expectedErrorMsg)) + private def resolver(caseSensitiveAnalysis: Boolean): Resolver = { + if (caseSensitiveAnalysis) { + caseSensitiveResolution + } else { + caseInsensitiveResolution } + } - checkCaseSensitiveExceptionCases("a INT, b INT, a INT", "`a`") - checkCaseSensitiveExceptionCases("a INT, b INT, a INT, a INT", "`a`") - checkCaseSensitiveExceptionCases("a INT, b INT, a INT, b INT", "`b`, `a`") - checkCaseSensitiveExceptionCases("a INT, c INT, b INT, a INT, b INT, c INT", "`b`, `a`, `c`") + Seq((true, ("a", "a"), ("b", "b")), (false, ("a", "A"), ("b", "B"))).foreach { + case (caseSensitive, (a0, a1), (b0, b1)) => - // Check no exception thrown - def checkCaseSensitiveNoExceptionCases(schemaStr: String): Unit = { - val schema = StructType.fromDDL(schemaStr) - SchemaUtils.checkSchemaColumnNameDuplication( - schema, "in SchemaUtilsSuite", caseSensitiveAnalysis = true) - SchemaUtils.checkColumnNameDuplication( - schema.map(_.name), "in SchemaUtilsSuite", caseSensitiveResolution) - SchemaUtils.checkColumnNameDuplication( - schema.map(_.name), "in SchemaUtilsSuite", caseSensitiveAnalysis = true) - } + 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)) + } - checkCaseSensitiveNoExceptionCases("a INT, b INT, c INT") - checkCaseSensitiveNoExceptionCases("Aa INT, b INT, aA INT") + 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 column name duplication in case-insensitive cases") { - def checkCaseInsensitiveExceptionCases(schemaStr: String, duplicatedColumns: String): Unit = { - val expectedErrorMsg = s"Found duplicate column(s) in SchemaUtilsSuite: $duplicatedColumns" + test("Check no exception thrown for valid schemas") { + def checkNoExceptionCases(schemaStr: String, caseSensitive: Boolean): Unit = { val schema = StructType.fromDDL(schemaStr) - var msg = intercept[AnalysisException] { - SchemaUtils.checkSchemaColumnNameDuplication( - schema, "in SchemaUtilsSuite", caseSensitiveAnalysis = false) - }.getMessage - assert(msg.contains(expectedErrorMsg)) - msg = intercept[AnalysisException] { - SchemaUtils.checkColumnNameDuplication( - schema.map(_.name), "in SchemaUtilsSuite", caseInsensitiveResolution) - }.getMessage - assert(msg.contains(expectedErrorMsg)) - msg = intercept[AnalysisException] { - SchemaUtils.checkColumnNameDuplication( - schema.map(_.name), "in SchemaUtilsSuite", caseSensitiveAnalysis = false) - }.getMessage - assert(msg.contains(expectedErrorMsg)) + 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) } - checkCaseInsensitiveExceptionCases("Aa INT, b INT, Aa INT", "`aa`") - checkCaseInsensitiveExceptionCases("a INT, bB INT, Bb INT", "`bb`") - checkCaseInsensitiveExceptionCases("Aa INT, b INT, Aa INT, c INT, aa INT", "`aa`") - checkCaseInsensitiveExceptionCases("Aa INT, bB INT, Bb INT, aa INT", "`bb`, `aa`") - checkCaseInsensitiveExceptionCases( - "Aa INT, cc INT, bB INT, cC INT, Bb INT, aa INT", "`bb`, `cc`, `aa`") + checkNoExceptionCases("a INT, b INT, c INT", caseSensitive = true) + checkNoExceptionCases("Aa INT, b INT, aA INT", caseSensitive = true) - // Check no exception thrown - val schema = StructType.fromDDL("a INT, b INT, c INT") - SchemaUtils.checkSchemaColumnNameDuplication( - schema, "in SchemaUtilsSuite", caseSensitiveAnalysis = false) - SchemaUtils.checkColumnNameDuplication( - schema.map(_.name), "in SchemaUtilsSuite", caseInsensitiveResolution) - SchemaUtils.checkColumnNameDuplication( - schema.map(_.name), "in SchemaUtilsSuite", caseSensitiveAnalysis = false) + checkNoExceptionCases("a INT, b INT, c INT", caseSensitive = false) } } diff --git a/sql/core/src/test/resources/sql-tests/results/create.sql.out b/sql/core/src/test/resources/sql-tests/results/create.sql.out index c4fc4c1343c3..91a5ba5799fd 100644 --- a/sql/core/src/test/resources/sql-tests/results/create.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/create.sql.out @@ -16,7 +16,7 @@ CREATE TABLE t (c0 STRING, c1 INT, c1 DOUBLE, c0 INT) USING parquet struct<> -- !query 1 output org.apache.spark.sql.AnalysisException -Found duplicate column(s) in the table definition of `t`: `c0`, `c1`; +Found duplicate column(s) in the table definition of `t`: `c1`, `c0`; -- !query 2 @@ -33,7 +33,7 @@ CREATE TABLE t (c0 STRING, c1 INT, c1 DOUBLE, c0 INT) USING parquet struct<> -- !query 3 output org.apache.spark.sql.AnalysisException -Found duplicate column(s) in the table definition of `t`: `c0`, `c1`; +Found duplicate column(s) in the table definition of `t`: `c1`, `c0`; -- !query 4 diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala index 9c1f1a50591a..e8e081f1b4b4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala @@ -716,7 +716,6 @@ class DataFrameReaderWriterSuite extends QueryTest with SharedSQLContext with Be df.toDF().write.mode("overwrite").text(testDir.getAbsolutePath) val e = intercept[AnalysisException] { spark.read.format(format).option("inferSchema", true).load(testDir.getAbsolutePath) - } assert(e.getMessage.contains("Found duplicate column(s) in the data schema:")) } From c69270f3d5cf69042abad67184b5aa671096cfbb Mon Sep 17 00:00:00 2001 From: Takeshi Yamamuro Date: Tue, 27 Jun 2017 21:04:20 +0900 Subject: [PATCH 16/27] Apply more comments --- .../sql/test/DataFrameReaderWriterSuite.scala | 22 ++++++++----------- 1 file changed, 9 insertions(+), 13 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala index e8e081f1b4b4..d741d125ffd5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala @@ -710,16 +710,6 @@ class DataFrameReaderWriterSuite extends QueryTest with SharedSQLContext with Be assert(e.getMessage.contains("Found duplicate column(s) in the data schema:")) } - def checkReadInferredDataColumnDuplication( - df: DataFrame, format: String, colName0: String, colName1: String, tempDir: File): Unit = { - val testDir = Utils.createTempDir(tempDir.getAbsolutePath) - df.toDF().write.mode("overwrite").text(testDir.getAbsolutePath) - val e = intercept[AnalysisException] { - spark.read.format(format).option("inferSchema", true).load(testDir.getAbsolutePath) - } - assert(e.getMessage.contains("Found duplicate column(s) in the data schema:")) - } - def checkReadPartitionColumnDuplication( format: String, colName0: String, colName1: String, tempDir: File): Unit = { val testDir = Utils.createTempDir(tempDir.getAbsolutePath) @@ -739,7 +729,7 @@ class DataFrameReaderWriterSuite extends QueryTest with SharedSQLContext with Be checkReadUserSpecifiedDataColumnDuplication( Seq((1, 1)).toDF("c0", "c1"), "csv", c0, c1, src) // If `inferSchema` is true, a CSV format is duplicate-safe (See SPARK-16896) - val testDir = Utils.createTempDir(src.getAbsolutePath) + var testDir = Utils.createTempDir(src.getAbsolutePath) Seq("a,a", "1,1").toDF().coalesce(1).write.mode("overwrite").text(testDir.getAbsolutePath) val df = spark.read.format("csv").option("inferSchema", true).option("header", true) .load(testDir.getAbsolutePath) @@ -750,8 +740,14 @@ class DataFrameReaderWriterSuite extends QueryTest with SharedSQLContext with Be checkWriteDataColumnDuplication("json", c0, c1, src) checkReadUserSpecifiedDataColumnDuplication( Seq((1, 1)).toDF("c0", "c1"), "json", c0, c1, src) - checkReadInferredDataColumnDuplication( - Seq(s"""{"$c0":3, "$c1":5}""").toDF(), "json", c0, c1, src) + // Inferred schema cases + testDir = Utils.createTempDir(src.getAbsolutePath) + Seq(s"""{"$c0":3, "$c1":5}""").toDF().write.mode("overwrite") + .text(testDir.getAbsolutePath) + val e = intercept[AnalysisException] { + spark.read.format("json").option("inferSchema", true).load(testDir.getAbsolutePath) + } + assert(e.getMessage.contains("Found duplicate column(s) in the data schema:")) checkReadPartitionColumnDuplication("json", c0, c1, src) // Check Parquet format From af959f6a55937e25db52cd2d94fc65af7f3a40d6 Mon Sep 17 00:00:00 2001 From: Takeshi Yamamuro Date: Wed, 28 Jun 2017 00:17:09 +0900 Subject: [PATCH 17/27] Add more tests in create.sql --- .../resources/sql-tests/inputs/create.sql | 14 +++-- .../sql-tests/results/create.sql.out | 54 +++++++++++++++---- 2 files changed, 56 insertions(+), 12 deletions(-) diff --git a/sql/core/src/test/resources/sql-tests/inputs/create.sql b/sql/core/src/test/resources/sql-tests/inputs/create.sql index 80ed69570c25..a716923edc91 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/create.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/create.sql @@ -1,11 +1,19 @@ -- Catch case-sensitive name duplication SET spark.sql.caseSensitive=true; -CREATE TABLE t (c0 STRING, c1 INT, c1 DOUBLE, c0 INT) USING parquet; +CREATE TABLE t(c0 STRING, c1 INT, c1 DOUBLE, c0 INT) USING parquet; + +CREATE TABLE t(c0 INT, c1 INT) PARTITIONED BY (c1 INT) STORED AS parquet; + +CREATE TABLE t(c0 INT) PARTITIONED BY (c1 INT, c1 INT) STORED AS parquet; -- Catch case-insensitive name duplication SET spark.sql.caseSensitive=false; -CREATE TABLE t (c0 STRING, c1 INT, c1 DOUBLE, c0 INT) USING parquet; +CREATE TABLE t(c0 STRING, c1 INT, c1 DOUBLE, c0 INT) USING parquet; + +CREATE TABLE t(ab STRING, cd INT, ef DOUBLE, Ab INT) USING parquet; + +CREATE TABLE t(ab INT, cd INT) PARTITIONED BY (Ab INT) STORED AS parquet; -CREATE TABLE t (ab STRING, cd INT, ef DOUBLE, Ab INT) USING parquet; +CREATE TABLE t(ab INT, cd INT) PARTITIONED BY (Ab INT, aB INT) STORED AS parquet; diff --git a/sql/core/src/test/resources/sql-tests/results/create.sql.out b/sql/core/src/test/resources/sql-tests/results/create.sql.out index 91a5ba5799fd..d531e320aa19 100644 --- a/sql/core/src/test/resources/sql-tests/results/create.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/create.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 5 +-- Number of queries: 9 -- !query 0 @@ -11,7 +11,7 @@ spark.sql.caseSensitive true -- !query 1 -CREATE TABLE t (c0 STRING, c1 INT, c1 DOUBLE, c0 INT) USING parquet +CREATE TABLE t(c0 STRING, c1 INT, c1 DOUBLE, c0 INT) USING parquet -- !query 1 schema struct<> -- !query 1 output @@ -20,26 +20,62 @@ Found duplicate column(s) in the table definition of `t`: `c1`, `c0`; -- !query 2 -SET spark.sql.caseSensitive=false +CREATE TABLE t(c0 INT, c1 INT) PARTITIONED BY (c1 INT) STORED AS parquet -- !query 2 schema -struct +struct<> -- !query 2 output -spark.sql.caseSensitive false +org.apache.spark.sql.AnalysisException +Found duplicate column(s) in the table definition of `t`: `c1`; -- !query 3 -CREATE TABLE t (c0 STRING, c1 INT, c1 DOUBLE, c0 INT) USING parquet +CREATE TABLE t(c0 INT) PARTITIONED BY (c1 INT, c1 INT) STORED AS parquet -- !query 3 schema struct<> -- !query 3 output org.apache.spark.sql.AnalysisException -Found duplicate column(s) in the table definition of `t`: `c1`, `c0`; +Found duplicate column(s) in the table definition of `t`: `c1`; -- !query 4 -CREATE TABLE t (ab STRING, cd INT, ef DOUBLE, Ab INT) USING parquet +SET spark.sql.caseSensitive=false -- !query 4 schema -struct<> +struct -- !query 4 output +spark.sql.caseSensitive false + + +-- !query 5 +CREATE TABLE t(c0 STRING, c1 INT, c1 DOUBLE, c0 INT) USING parquet +-- !query 5 schema +struct<> +-- !query 5 output +org.apache.spark.sql.AnalysisException +Found duplicate column(s) in the table definition of `t`: `c1`, `c0`; + + +-- !query 6 +CREATE TABLE t(ab STRING, cd INT, ef DOUBLE, Ab INT) USING parquet +-- !query 6 schema +struct<> +-- !query 6 output +org.apache.spark.sql.AnalysisException +Found duplicate column(s) in the table definition of `t`: `ab`; + + +-- !query 7 +CREATE TABLE t(ab INT, cd INT) PARTITIONED BY (Ab INT) STORED AS parquet +-- !query 7 schema +struct<> +-- !query 7 output +org.apache.spark.sql.AnalysisException +Found duplicate column(s) in the table definition of `t`: `ab`; + + +-- !query 8 +CREATE TABLE t(ab INT, cd INT) PARTITIONED BY (Ab INT, aB INT) STORED AS parquet +-- !query 8 schema +struct<> +-- !query 8 output org.apache.spark.sql.AnalysisException Found duplicate column(s) in the table definition of `t`: `ab`; From 8d3e10ad6a76d8313a083effde1a49bc76930959 Mon Sep 17 00:00:00 2001 From: Takeshi Yamamuro Date: Thu, 29 Jun 2017 17:56:09 +0900 Subject: [PATCH 18/27] Move duplication checks in constructor --- .../spark/sql/execution/datasources/DataSource.scala | 9 ++++++++- .../spark/sql/sources/ResolvedDataSourceSuite.scala | 5 +++-- .../apache/spark/sql/streaming/FileStreamSinkSuite.scala | 2 +- 3 files changed, 12 insertions(+), 4 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala index 7f3d463c73ed..a92408ae9683 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala @@ -87,6 +87,14 @@ case class DataSource( lazy val providingClass: Class[_] = DataSource.lookupDataSource(className) lazy val sourceInfo: SourceInfo = sourceSchema() private val caseInsensitiveOptions = CaseInsensitiveMap(options) + private val equality = sparkSession.sessionState.conf.resolver + + userSpecifiedSchema.foreach { dataSchema => + SchemaUtils.checkColumnNameDuplication( + dataSchema.map(_.name), "in the data schema", equality) + } + SchemaUtils.checkColumnNameDuplication( + partitionColumns, "in the partition schema", equality) /** * Get the schema of the given FileFormat, if provided by `userSpecifiedSchema`, or try to infer @@ -128,7 +136,6 @@ case class DataSource( }.toArray new InMemoryFileIndex(sparkSession, globbedPaths, options, None, fileStatusCache) } - val equality = sparkSession.sessionState.conf.resolver val partitionSchema = if (partitionColumns.isEmpty) { // Try to infer partitioning, because no DataSource in the read path provides the partitioning // columns properly unless it is a Hive DataSource diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/ResolvedDataSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/ResolvedDataSourceSuite.scala index 0f97fd78d2ff..308c5079c44b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/ResolvedDataSourceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/ResolvedDataSourceSuite.scala @@ -21,11 +21,12 @@ import org.apache.spark.SparkFunSuite import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.execution.datasources.DataSource +import org.apache.spark.sql.test.SharedSQLContext -class ResolvedDataSourceSuite extends SparkFunSuite { +class ResolvedDataSourceSuite extends SparkFunSuite with SharedSQLContext { private def getProvidingClass(name: String): Class[_] = DataSource( - sparkSession = null, + sparkSession = spark, className = name, options = Map(DateTimeUtils.TIMEZONE_OPTION -> DateTimeUtils.defaultTimeZone().getID) ).providingClass diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala index 998d33e73ca8..6676099d426b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala @@ -385,7 +385,7 @@ class FileStreamSinkSuite extends StreamTest { val errorMsg = intercept[AnalysisException] { spark.read.schema(s"$c0 INT, $c1 INT").json(outputDir).as[(Int, Int)] }.getMessage - assert(errorMsg.contains("Found duplicate column(s) in the datasource: ")) + assert(errorMsg.contains("Found duplicate column(s) in the data schema: ")) } } } From 9b386d56fd2ee12278bc2cfc4166a4eef0b9919c Mon Sep 17 00:00:00 2001 From: Takeshi Yamamuro Date: Sat, 1 Jul 2017 00:57:50 +0900 Subject: [PATCH 19/27] Brush up code --- .../sql/catalyst/catalog/SessionCatalog.scala | 5 - .../spark/sql/execution/command/views.scala | 131 ++---------- .../execution/datasources/DataSource.scala | 3 - .../InsertIntoHadoopFsRelationCommand.scala | 7 - .../datasources/PartitioningUtils.scala | 4 - .../sql/execution/datasources/rules.scala | 192 +++++++++++++++++- .../internal/BaseSessionStateBuilder.scala | 2 +- .../resources/sql-tests/inputs/create.sql | 8 - .../sql-tests/results/create.sql.out | 52 +---- .../sql/execution/command/DDLSuite.scala | 4 +- .../sql/hive/HiveSessionStateBuilder.scala | 2 +- 11 files changed, 207 insertions(+), 203 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala index 0a7ca4280bd0..24fff3ad563a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala @@ -40,7 +40,6 @@ import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SubqueryAlias, import org.apache.spark.sql.catalyst.util.StringUtils import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.StructType -import org.apache.spark.sql.util.SchemaUtils object SessionCatalog { val DEFAULT_DATABASE = "default" @@ -342,10 +341,6 @@ class SessionCatalog( requireDbExists(db) requireTableExists(tableIdentifier) - SchemaUtils.checkSchemaColumnNameDuplication( - newSchema, "in the table definition of " + tableIdentifier.identifier, - conf.caseSensitiveAnalysis) - val catalogTable = externalCatalog.getTable(db, table) val oldSchema = catalogTable.schema diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala index e167dc1b6a34..bffe8e4bbc26 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala @@ -21,13 +21,11 @@ import scala.collection.mutable import org.apache.spark.sql.{AnalysisException, Row, SparkSession} import org.apache.spark.sql.catalyst.TableIdentifier -import org.apache.spark.sql.catalyst.analysis.{UnresolvedFunction, UnresolvedRelation} import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable, CatalogTableType} -import org.apache.spark.sql.catalyst.expressions.{Alias, SubqueryExpression} +import org.apache.spark.sql.catalyst.expressions.Alias import org.apache.spark.sql.catalyst.plans.QueryPlan -import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project, View} +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project} import org.apache.spark.sql.types.MetadataBuilder -import org.apache.spark.sql.util.SchemaUtils /** @@ -124,28 +122,19 @@ case class CreateViewCommand( } override def run(sparkSession: SparkSession): Seq[Row] = { - // If the plan cannot be analyzed, throw an exception and don't proceed. - val qe = sparkSession.sessionState.executePlan(child) - qe.assertAnalyzed() - val analyzedPlan = qe.analyzed - if (userSpecifiedColumns.nonEmpty && - userSpecifiedColumns.length != analyzedPlan.output.length) { + userSpecifiedColumns.length != child.output.length) { throw new AnalysisException(s"The number of columns produced by the SELECT clause " + - s"(num: `${analyzedPlan.output.length}`) does not match the number of column names " + + s"(num: `${child.output.length}`) does not match the number of column names " + s"specified by CREATE VIEW (num: `${userSpecifiedColumns.length}`).") } - // When creating a permanent view, not allowed to reference temporary objects. - // This should be called after `qe.assertAnalyzed()` (i.e., `child` can be resolved) - verifyTemporaryObjectsNotExists(sparkSession) - val catalog = sparkSession.sessionState.catalog if (viewType == LocalTempView) { - val aliasedPlan = aliasPlan(sparkSession, analyzedPlan) + val aliasedPlan = aliasPlan(sparkSession, child) catalog.createTempView(name.table, aliasedPlan, overrideIfExists = replace) } else if (viewType == GlobalTempView) { - val aliasedPlan = aliasPlan(sparkSession, analyzedPlan) + val aliasedPlan = aliasPlan(sparkSession, child) catalog.createGlobalTempView(name.table, aliasedPlan, overrideIfExists = replace) } else if (catalog.tableExists(name)) { val tableMetadata = catalog.getTableMetadata(name) @@ -155,14 +144,11 @@ case class CreateViewCommand( } else if (tableMetadata.tableType != CatalogTableType.VIEW) { throw new AnalysisException(s"$name is not a view") } else if (replace) { - // Detect cyclic view reference on CREATE OR REPLACE VIEW. - val viewIdent = tableMetadata.identifier - checkCyclicViewReference(analyzedPlan, Seq(viewIdent), viewIdent) - // Handles `CREATE OR REPLACE VIEW v0 AS SELECT ...` // Nothing we need to retain from the old view, so just drop and create a new one + val viewIdent = tableMetadata.identifier catalog.dropTable(viewIdent, ignoreIfNotExists = false, purge = false) - catalog.createTable(prepareTable(sparkSession, analyzedPlan), ignoreIfExists = false) + catalog.createTable(prepareTable(sparkSession, child), ignoreIfExists = false) } else { // Handles `CREATE VIEW v0 AS SELECT ...`. Throws exception when the target view already // exists. @@ -172,39 +158,11 @@ case class CreateViewCommand( } } else { // Create the view if it doesn't exist. - catalog.createTable(prepareTable(sparkSession, analyzedPlan), ignoreIfExists = false) + catalog.createTable(prepareTable(sparkSession, child), ignoreIfExists = false) } Seq.empty[Row] } - /** - * Permanent views are not allowed to reference temp objects, including temp function and views - */ - private def verifyTemporaryObjectsNotExists(sparkSession: SparkSession): Unit = { - if (!isTemporary) { - // This func traverses the unresolved plan `child`. Below are the reasons: - // 1) Analyzer replaces unresolved temporary views by a SubqueryAlias with the corresponding - // logical plan. After replacement, it is impossible to detect whether the SubqueryAlias is - // added/generated from a temporary view. - // 2) The temp functions are represented by multiple classes. Most are inaccessible from this - // package (e.g., HiveGenericUDF). - child.collect { - // Disallow creating permanent views based on temporary views. - case s: UnresolvedRelation - if sparkSession.sessionState.catalog.isTemporaryTable(s.tableIdentifier) => - throw new AnalysisException(s"Not allowed to create a permanent view $name by " + - s"referencing a temporary view ${s.tableIdentifier}") - case other if !other.resolved => other.expressions.flatMap(_.collect { - // Disallow creating permanent views based on temporary UDFs. - case e: UnresolvedFunction - if sparkSession.sessionState.catalog.isTemporaryFunction(e.name) => - throw new AnalysisException(s"Not allowed to create a permanent view $name by " + - s"referencing a temporary function `${e.name}`") - }) - } - } - } - /** * If `userSpecifiedColumns` is defined, alias the analyzed plan to the user specified columns, * else return the analyzed plan directly. @@ -270,15 +228,10 @@ case class AlterViewAsCommand( override def innerChildren: Seq[QueryPlan[_]] = Seq(query) override def run(session: SparkSession): Seq[Row] = { - // If the plan cannot be analyzed, throw an exception and don't proceed. - val qe = session.sessionState.executePlan(query) - qe.assertAnalyzed() - val analyzedPlan = qe.analyzed - - if (session.sessionState.catalog.alterTempViewDefinition(name, analyzedPlan)) { + if (session.sessionState.catalog.alterTempViewDefinition(name, query)) { // a local/global temp view has been altered, we are done. } else { - alterPermanentView(session, analyzedPlan) + alterPermanentView(session, query) } Seq.empty[Row] @@ -290,10 +243,6 @@ case class AlterViewAsCommand( throw new AnalysisException(s"${viewMeta.identifier} is not a view.") } - // Detect cyclic view reference on ALTER VIEW. - val viewIdent = viewMeta.identifier - checkCyclicViewReference(analyzedPlan, Seq(viewIdent), viewIdent) - val newProperties = generateViewProperties(viewMeta.properties, session, analyzedPlan) val updatedViewMeta = viewMeta.copy( @@ -356,66 +305,12 @@ object ViewHelper { properties: Map[String, String], session: SparkSession, analyzedPlan: LogicalPlan): Map[String, String] = { - val queryOutput = analyzedPlan.schema.fieldNames - - // Generate the query column names, throw an AnalysisException if there exists duplicate column - // names. - SchemaUtils.checkColumnNameDuplication( - queryOutput, "the view", session.sessionState.conf.resolver) - // Generate the view default database name. + val queryOutput = analyzedPlan.schema.fieldNames val viewDefaultDatabase = session.sessionState.catalog.getCurrentDatabase + removeQueryColumnNames(properties) ++ generateViewDefaultDatabase(viewDefaultDatabase) ++ generateQueryColumnNames(queryOutput) } - - /** - * Recursively search the logical plan to detect cyclic view references, throw an - * AnalysisException if cycle detected. - * - * A cyclic view reference is a cycle of reference dependencies, for example, if the following - * statements are executed: - * CREATE VIEW testView AS SELECT id FROM tbl - * CREATE VIEW testView2 AS SELECT id FROM testView - * ALTER VIEW testView AS SELECT * FROM testView2 - * The view `testView` references `testView2`, and `testView2` also references `testView`, - * therefore a reference cycle (testView -> testView2 -> testView) exists. - * - * @param plan the logical plan we detect cyclic view references from. - * @param path the path between the altered view and current node. - * @param viewIdent the table identifier of the altered view, we compare two views by the - * `desc.identifier`. - */ - def checkCyclicViewReference( - plan: LogicalPlan, - path: Seq[TableIdentifier], - viewIdent: TableIdentifier): Unit = { - plan match { - case v: View => - val ident = v.desc.identifier - val newPath = path :+ ident - // If the table identifier equals to the `viewIdent`, current view node is the same with - // the altered view. We detect a view reference cycle, should throw an AnalysisException. - if (ident == viewIdent) { - throw new AnalysisException(s"Recursive view $viewIdent detected " + - s"(cycle: ${newPath.mkString(" -> ")})") - } else { - v.children.foreach { child => - checkCyclicViewReference(child, newPath, viewIdent) - } - } - case _ => - plan.children.foreach(child => checkCyclicViewReference(child, path, viewIdent)) - } - - // Detect cyclic references from subqueries. - plan.expressions.foreach { expr => - expr match { - case s: SubqueryExpression => - checkCyclicViewReference(s.plan, path, viewIdent) - case _ => // Do nothing. - } - } - } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala index a92408ae9683..c21916ad3251 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala @@ -352,9 +352,6 @@ case class DataSource( "It must be specified manually") } - SchemaUtils.checkSchemaColumnNameDuplication( - dataSchema, "in the datasource", sparkSession.sessionState.conf.caseSensitiveAnalysis) - HadoopFsRelation( fileCatalog, partitionSchema = fileCatalog.partitionSchema, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala index a98cd3902214..f71733a408ae 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala @@ -63,13 +63,6 @@ case class InsertIntoHadoopFsRelationCommand( override def run(sparkSession: SparkSession, children: Seq[SparkPlan]): Seq[Row] = { assert(children.length == 1) - - // Most formats don't do well with duplicate columns, so lets not allow that - SchemaUtils.checkSchemaColumnNameDuplication( - query.schema, - s"when inserting into $outputPath", - sparkSession.sessionState.conf.caseSensitiveAnalysis) - val hadoopConf = sparkSession.sessionState.newHadoopConfWithOptions(options) val fs = outputPath.getFileSystem(hadoopConf) val qualifiedOutputPath = outputPath.makeQualified(fs.getUri, fs.getWorkingDirectory) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala index 54cdd973420b..2349115fd043 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala @@ -33,7 +33,6 @@ import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec import org.apache.spark.sql.catalyst.expressions.{Cast, Literal} import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.types._ -import org.apache.spark.sql.util.SchemaUtils // TODO: We should tighten up visibility of the classes here once we clean up Hive coupling. @@ -302,9 +301,6 @@ object PartitioningUtils { normalizedKey -> value } - SchemaUtils.checkColumnNameDuplication( - normalizedPartSpec.map(_._1), "in the partition specification", resolver) - normalizedPartSpec.toMap } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala index 85feef9e5788..508253433d38 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala @@ -20,12 +20,16 @@ package org.apache.spark.sql.execution.datasources import java.util.Locale import org.apache.spark.sql.{AnalysisException, SaveMode, SparkSession} +import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.catalog._ -import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, RowOrdering} +import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, RowOrdering, SubqueryExpression} import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.execution.command.DDLUtils +import org.apache.spark.sql.execution.command._ +import org.apache.spark.sql.execution.datasources.csv.CSVFileFormat +import org.apache.spark.sql.execution.datasources.json.JsonFileFormat +import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources.InsertableRelation import org.apache.spark.sql.types.{AtomicType, StructType} @@ -67,9 +71,10 @@ class ResolveSQLOnFile(sparkSession: SparkSession) extends Rule[LogicalPlan] { } /** - * Preprocess [[CreateTable]], to do some normalization and checking. + * Preprocess DDL commands (e.g., [[CreateTable]] and [[AlterTableAddColumnsCommand]]), to do some + * normalization and checking. */ -case class PreprocessTableCreation(sparkSession: SparkSession) extends Rule[LogicalPlan] { +case class PreprocessDDLCommands(sparkSession: SparkSession) extends Rule[LogicalPlan] { // catalog is a def and not a val/lazy val as the latter would introduce a circular reference private def catalog = sparkSession.sessionState.catalog @@ -220,12 +225,120 @@ case class PreprocessTableCreation(sparkSession: SparkSession) extends Rule[Logi c.copy(tableDesc = normalizedTable.copy(schema = reorderedSchema)) } + + case a @ AlterTableAddColumnsCommand(table, columns) => + val catalog = sparkSession.sessionState.catalog + val catalogTable = catalog.getTempViewOrPermanentTableMetadata(table) + val newSchema = catalogTable.dataSchema ++ columns ++ catalogTable.partitionSchema + SchemaUtils.checkColumnNameDuplication( + newSchema.map(_.name), s"in the table definition of ${table.identifier}", + sparkSession.sessionState.conf.caseSensitiveAnalysis) + a + + case a @ AlterTableAddPartitionCommand(tableName, partitionSpecsAndLocs, _) => + partitionSpecsAndLocs.foreach { case (spec, _) => + val partCols = spec.toSeq.map(_._1) + SchemaUtils.checkColumnNameDuplication( + partCols, s"the table definition of ${tableName.identifier}", + sparkSession.sessionState.conf.caseSensitiveAnalysis) + } + a + + case a @ AlterTableRenamePartitionCommand(tableName, _, newPart) => + val newPartCols = newPart.toSeq.map(_._1) + SchemaUtils.checkColumnNameDuplication( + newPartCols, s"the table definition of ${tableName.identifier}", + sparkSession.sessionState.conf.caseSensitiveAnalysis) + a + + case a @ AlterTableDropPartitionCommand(tableName, partitionSpecs, _, _, _) => + partitionSpecs.foreach { spec => + val partCols = spec.toSeq.map(_._1) + SchemaUtils.checkColumnNameDuplication( + partCols, s"the table definition of ${tableName.identifier}", + sparkSession.sessionState.conf.caseSensitiveAnalysis) + } + a + + case t @ TruncateTableCommand(tableName, Some(partitionSpec)) => + val partCols = partitionSpec.toSeq.map(_._1) + SchemaUtils.checkColumnNameDuplication( + partCols, s"the table definition of ${tableName.identifier}", + sparkSession.sessionState.conf.caseSensitiveAnalysis) + t + + case c @ CreateViewCommand(name, _, _, _, _, child, _, replace, viewType) => + // If the plan cannot be analyzed, throw an exception and don't proceed. + val qe = sparkSession.sessionState.executePlan(child) + qe.assertAnalyzed() + val analyzedPlan = qe.analyzed + def isTemporary = viewType == LocalTempView || viewType == GlobalTempView + if (!isTemporary) { + if (catalog.tableExists(name) && replace) { + // Detect cyclic view reference on CREATE OR REPLACE VIEW. + val tableMetadata = catalog.getTableMetadata(name) + val viewIdent = tableMetadata.identifier + PreprocessDDLCommands.checkCyclicViewReference(analyzedPlan, Seq(viewIdent), viewIdent) + } + + SchemaUtils.checkSchemaColumnNameDuplication( + analyzedPlan.schema, s"the view definition of ${name.identifier}", + sparkSession.sessionState.conf.caseSensitiveAnalysis) + + // When creating a permanent view, not allowed to reference temporary objects. + // This should be called after `qe.assertAnalyzed()` (i.e., `child` can be resolved) + verifyTemporaryObjectsNotExists(name, child) + } + c.copy(child = analyzedPlan) + + case a @ AlterViewAsCommand(name, _, query) => + // If the plan cannot be analyzed, throw an exception and don't proceed. + val qe = sparkSession.sessionState.executePlan(query) + qe.assertAnalyzed() + val analyzedPlan = qe.analyzed + + // First, detect cyclic view reference on ALTER VIEW. + val viewMeta = sparkSession.sessionState.catalog.getTempViewOrPermanentTableMetadata(name) + val viewIdent = viewMeta.identifier + PreprocessDDLCommands.checkCyclicViewReference(analyzedPlan, Seq(viewIdent), viewIdent) + + // Then, check name duplication in the new view table + SchemaUtils.checkSchemaColumnNameDuplication( + analyzedPlan.schema, s"the view definition of ${name.identifier}", + sparkSession.sessionState.conf.caseSensitiveAnalysis) + + a.copy(query = analyzedPlan) + } + + /** + * Permanent views are not allowed to reference temp objects, including temp function and views + */ + private def verifyTemporaryObjectsNotExists(name: TableIdentifier, plan: LogicalPlan): Unit = { + // This func traverses the unresolved plan `child`. Below are the reasons: + // 1) Analyzer replaces unresolved temporary views by a SubqueryAlias with the corresponding + // logical plan. After replacement, it is impossible to detect whether the SubqueryAlias is + // added/generated from a temporary view. + // 2) The temp functions are represented by multiple classes. Most are inaccessible from this + // package (e.g., HiveGenericUDF). + plan.collect { + // Disallow creating permanent views based on temporary views. + case s: UnresolvedRelation + if sparkSession.sessionState.catalog.isTemporaryTable(s.tableIdentifier) => + throw new AnalysisException(s"Not allowed to create a permanent view $name by " + + s"referencing a temporary view ${s.tableIdentifier}") + case other if !other.resolved => other.expressions.flatMap(_.collect { + // Disallow creating permanent views based on temporary UDFs. + case e: UnresolvedFunction + if sparkSession.sessionState.catalog.isTemporaryFunction(e.name) => + throw new AnalysisException(s"Not allowed to create a permanent view $name by " + + s"referencing a temporary function `${e.name}`") + }) + } } private def normalizeCatalogTable(schema: StructType, table: CatalogTable): CatalogTable = { SchemaUtils.checkSchemaColumnNameDuplication( - schema, - "in the table definition of " + table.identifier, + schema, s"in the table definition of ${table.identifier}", sparkSession.sessionState.conf.caseSensitiveAnalysis) val normalizedPartCols = normalizePartitionColumns(schema, table) @@ -253,8 +366,7 @@ case class PreprocessTableCreation(sparkSession: SparkSession) extends Rule[Logi resolver = sparkSession.sessionState.conf.resolver) SchemaUtils.checkColumnNameDuplication( - normalizedPartitionCols, - "in the partition", + normalizedPartitionCols, s"in the partition columns of ${table.identifier}", sparkSession.sessionState.conf.resolver) if (schema.nonEmpty && normalizedPartitionCols.length == schema.length) { @@ -288,11 +400,11 @@ case class PreprocessTableCreation(sparkSession: SparkSession) extends Rule[Logi SchemaUtils.checkColumnNameDuplication( normalizedBucketSpec.bucketColumnNames, - "in the bucket", + s"in the bucket column names of ${table.identifier}", sparkSession.sessionState.conf.resolver) SchemaUtils.checkColumnNameDuplication( normalizedBucketSpec.sortColumnNames, - "in the sort", + s"in the sort column names of ${table.identifier}", sparkSession.sessionState.conf.resolver) normalizedBucketSpec.sortColumnNames.map(schema(_)).map(_.dataType).foreach { @@ -309,6 +421,58 @@ case class PreprocessTableCreation(sparkSession: SparkSession) extends Rule[Logi private def failAnalysis(msg: String) = throw new AnalysisException(msg) } +object PreprocessDDLCommands { + + /** + * Recursively search the logical plan to detect cyclic view references, throw an + * AnalysisException if cycle detected. + * + * A cyclic view reference is a cycle of reference dependencies, for example, if the following + * statements are executed: + * CREATE VIEW testView AS SELECT id FROM tbl + * CREATE VIEW testView2 AS SELECT id FROM testView + * ALTER VIEW testView AS SELECT * FROM testView2 + * The view `testView` references `testView2`, and `testView2` also references `testView`, + * therefore a reference cycle (testView -> testView2 -> testView) exists. + * + * @param plan the logical plan we detect cyclic view references from. + * @param path the path between the altered view and current node. + * @param viewIdent the table identifier of the altered view, we compare two views by the + * `desc.identifier`. + */ + def checkCyclicViewReference( + plan: LogicalPlan, + path: Seq[TableIdentifier], + viewIdent: TableIdentifier): Unit = { + plan match { + case v: View => + val ident = v.desc.identifier + val newPath = path :+ ident + // If the table identifier equals to the `viewIdent`, current view node is the same with + // the altered view. We detect a view reference cycle, should throw an AnalysisException. + if (ident == viewIdent) { + throw new AnalysisException(s"Recursive view $viewIdent detected " + + s"(cycle: ${newPath.mkString(" -> ")})") + } else { + v.children.foreach { child => + checkCyclicViewReference(child, newPath, viewIdent) + } + } + case _ => + plan.children.foreach(child => checkCyclicViewReference(child, path, viewIdent)) + } + + // Detect cyclic references from subqueries. + plan.expressions.foreach { expr => + expr match { + case s: SubqueryExpression => + checkCyclicViewReference(s.plan, path, viewIdent) + case _ => // Do nothing. + } + } + } +} + /** * Preprocess the [[InsertIntoTable]] plan. Throws exception if the number of columns mismatch, or * specified partition columns are different from the existing partition columns in the target @@ -320,6 +484,8 @@ case class PreprocessTableInsertion(conf: SQLConf) extends Rule[LogicalPlan] wit insert: InsertIntoTable, tblName: String, partColNames: Seq[String]): InsertIntoTable = { + SchemaUtils.checkColumnNameDuplication( + insert.partition.toSeq.map(_._1), s"when inserting into $tblName", conf.caseSensitiveAnalysis) val normalizedPartSpec = PartitioningUtils.normalizePartitionSpec( insert.partition, partColNames, tblName, conf.resolver) @@ -393,6 +559,12 @@ case class PreprocessTableInsertion(conf: SQLConf) extends Rule[LogicalPlan] wit preprocess(i, tblName, Nil) case _ => i } + + case i @ InsertIntoHadoopFsRelationCommand(outputPath, _, _, _, _, _, _, query, _, _, _) => + // Most formats don't do well with duplicate columns, so lets not allow that + SchemaUtils.checkSchemaColumnNameDuplication( + query.schema, s"when inserting into $outputPath", conf.caseSensitiveAnalysis) + i } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala index 72d0ddc62303..0a043ae504cd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala @@ -161,7 +161,7 @@ abstract class BaseSessionStateBuilder( customResolutionRules override val postHocResolutionRules: Seq[Rule[LogicalPlan]] = - PreprocessTableCreation(session) +: + PreprocessDDLCommands(session) +: PreprocessTableInsertion(conf) +: DataSourceAnalysis(conf) +: customPostHocResolutionRules diff --git a/sql/core/src/test/resources/sql-tests/inputs/create.sql b/sql/core/src/test/resources/sql-tests/inputs/create.sql index a716923edc91..5a16df270a9c 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/create.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/create.sql @@ -3,17 +3,9 @@ SET spark.sql.caseSensitive=true; CREATE TABLE t(c0 STRING, c1 INT, c1 DOUBLE, c0 INT) USING parquet; -CREATE TABLE t(c0 INT, c1 INT) PARTITIONED BY (c1 INT) STORED AS parquet; - -CREATE TABLE t(c0 INT) PARTITIONED BY (c1 INT, c1 INT) STORED AS parquet; - -- Catch case-insensitive name duplication SET spark.sql.caseSensitive=false; CREATE TABLE t(c0 STRING, c1 INT, c1 DOUBLE, c0 INT) USING parquet; CREATE TABLE t(ab STRING, cd INT, ef DOUBLE, Ab INT) USING parquet; - -CREATE TABLE t(ab INT, cd INT) PARTITIONED BY (Ab INT) STORED AS parquet; - -CREATE TABLE t(ab INT, cd INT) PARTITIONED BY (Ab INT, aB INT) STORED AS parquet; diff --git a/sql/core/src/test/resources/sql-tests/results/create.sql.out b/sql/core/src/test/resources/sql-tests/results/create.sql.out index d531e320aa19..7d7de4f36204 100644 --- a/sql/core/src/test/resources/sql-tests/results/create.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/create.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 9 +-- Number of queries: 5 -- !query 0 @@ -20,62 +20,26 @@ Found duplicate column(s) in the table definition of `t`: `c1`, `c0`; -- !query 2 -CREATE TABLE t(c0 INT, c1 INT) PARTITIONED BY (c1 INT) STORED AS parquet +SET spark.sql.caseSensitive=false -- !query 2 schema -struct<> +struct -- !query 2 output -org.apache.spark.sql.AnalysisException -Found duplicate column(s) in the table definition of `t`: `c1`; +spark.sql.caseSensitive false -- !query 3 -CREATE TABLE t(c0 INT) PARTITIONED BY (c1 INT, c1 INT) STORED AS parquet +CREATE TABLE t(c0 STRING, c1 INT, c1 DOUBLE, c0 INT) USING parquet -- !query 3 schema struct<> -- !query 3 output org.apache.spark.sql.AnalysisException -Found duplicate column(s) in the table definition of `t`: `c1`; - - --- !query 4 -SET spark.sql.caseSensitive=false --- !query 4 schema -struct --- !query 4 output -spark.sql.caseSensitive false - - --- !query 5 -CREATE TABLE t(c0 STRING, c1 INT, c1 DOUBLE, c0 INT) USING parquet --- !query 5 schema -struct<> --- !query 5 output -org.apache.spark.sql.AnalysisException Found duplicate column(s) in the table definition of `t`: `c1`, `c0`; --- !query 6 +-- !query 4 CREATE TABLE t(ab STRING, cd INT, ef DOUBLE, Ab INT) USING parquet --- !query 6 schema -struct<> --- !query 6 output -org.apache.spark.sql.AnalysisException -Found duplicate column(s) in the table definition of `t`: `ab`; - - --- !query 7 -CREATE TABLE t(ab INT, cd INT) PARTITIONED BY (Ab INT) STORED AS parquet --- !query 7 schema -struct<> --- !query 7 output -org.apache.spark.sql.AnalysisException -Found duplicate column(s) in the table definition of `t`: `ab`; - - --- !query 8 -CREATE TABLE t(ab INT, cd INT) PARTITIONED BY (Ab INT, aB INT) STORED AS parquet --- !query 8 schema +-- !query 4 schema struct<> --- !query 8 output +-- !query 4 output org.apache.spark.sql.AnalysisException Found duplicate column(s) in the table definition of `t`: `ab`; diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala index b40eac5d5c77..7b18453105f5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala @@ -469,14 +469,14 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils { val e = intercept[AnalysisException] { sql("CREATE TABLE tbl(a int) USING json PARTITIONED BY (a, a)") } - assert(e.message == "Found duplicate column(s) in the partition: `a`") + assert(e.message == "Found duplicate column(s) in the partition columns of `tbl`: `a`") } test("create table - column repeated in bucket columns") { val e = intercept[AnalysisException] { sql("CREATE TABLE tbl(a int) USING json CLUSTERED BY (a, a) INTO 4 BUCKETS") } - assert(e.message == "Found duplicate column(s) in the bucket: `a`") + assert(e.message == "Found duplicate column(s) in the bucket column names of `tbl`: `a`") } test("Refresh table after changing the data source table partitioning") { diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala index e16c9e46b772..d1301e0cac2d 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala @@ -76,7 +76,7 @@ class HiveSessionStateBuilder(session: SparkSession, parentState: Option[Session override val postHocResolutionRules: Seq[Rule[LogicalPlan]] = new DetermineTableStats(session) +: RelationConversions(conf, catalog) +: - PreprocessTableCreation(session) +: + PreprocessDDLCommands(session) +: PreprocessTableInsertion(conf) +: DataSourceAnalysis(conf) +: HiveAnalysis +: From a878510b61a3e6a2488cc3f5e2c0f4a610eaa32b Mon Sep 17 00:00:00 2001 From: Takeshi Yamamuro Date: Mon, 3 Jul 2017 19:33:21 +0900 Subject: [PATCH 20/27] [WIP] Add DataSourceValidator trait to validate schema in write path --- .../execution/datasources/DataSource.scala | 8 +++- .../datasources/DataSourceValidator.scala | 36 +++++++++++++++ .../jdbc/JdbcRelationProvider.scala | 35 +++++++++++++-- .../datasources/jdbc/JdbcUtils.scala | 44 +++++++------------ .../sql/execution/datasources/rules.scala | 3 -- .../org/apache/spark/sql/jdbc/JDBCSuite.scala | 4 +- .../spark/sql/jdbc/JDBCWriteSuite.scala | 5 ++- 7 files changed, 97 insertions(+), 38 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceValidator.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala index c21916ad3251..35ee056caff5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala @@ -486,7 +486,13 @@ case class DataSource( throw new AnalysisException("Cannot save interval data type into external storage.") } - providingClass.newInstance() match { + val resolvedRelation = providingClass.newInstance() match { + case relationToCheck: DataSourceValidator => + relationToCheck.validateSchema(data.schema, partitionColumns, equality, options) + relationToCheck + case r => r + } + resolvedRelation match { case dataSource: CreatableRelationProvider => SaveIntoDataSourceCommand(data, dataSource, caseInsensitiveOptions, mode) case format: FileFormat => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceValidator.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceValidator.scala new file mode 100644 index 000000000000..2d3ad0168461 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceValidator.scala @@ -0,0 +1,36 @@ +/* + * 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.execution.datasources + +import org.apache.spark.sql.catalyst.analysis.Resolver +import org.apache.spark.sql.types.StructType + + +/** + * Used to validate an input schema for datasource-specific cases (e.g., schema definition + * from an datasource option and unsupported field names). `validateSchema` is called + * in [[DataSource]] for the checking. + */ +trait DataSourceValidator { + + def validateSchema( + schema: StructType, + partitionColumns: Seq[String], + resolver: Resolver, + options: Map[String, String]): Unit +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcRelationProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcRelationProvider.scala index 74dcfb06f5c2..fd8f27a7e5fe 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcRelationProvider.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcRelationProvider.scala @@ -18,14 +18,41 @@ package org.apache.spark.sql.execution.datasources.jdbc import org.apache.spark.sql.{AnalysisException, DataFrame, SaveMode, SQLContext} +import org.apache.spark.sql.catalyst.analysis.Resolver +import org.apache.spark.sql.catalyst.parser.CatalystSqlParser +import org.apache.spark.sql.execution.datasources.DataSourceValidator import org.apache.spark.sql.execution.datasources.jdbc.JdbcUtils._ import org.apache.spark.sql.sources.{BaseRelation, CreatableRelationProvider, DataSourceRegister, RelationProvider} +import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.util.SchemaUtils class JdbcRelationProvider extends CreatableRelationProvider - with RelationProvider with DataSourceRegister { + with RelationProvider with DataSourceValidator with DataSourceRegister { override def shortName(): String = "jdbc" + override def validateSchema( + schema: StructType, partCols: Seq[String], resolver: Resolver, options: Map[String, String]) + : Unit = { + val jdbcOptions = new JDBCOptions(options) + jdbcOptions.createTableColumnTypes.foreach { createTableColumnTypes => + val userSchema = CatalystSqlParser.parseTableSchema(createTableColumnTypes) + + // Checks duplicate columns in the user specified column types + SchemaUtils.checkColumnNameDuplication( + userSchema.map(_.name), "in the createTableColumnTypes option value", resolver) + + // Checks if user specified column names exist in the DataFrame schema + userSchema.fieldNames.foreach { col => + schema.find(f => resolver(f.name, col)).getOrElse { + throw new AnalysisException( + s"createTableColumnTypes option column $col not found in schema " + + schema.catalogString) + } + } + } + } + override def createRelation( sqlContext: SQLContext, parameters: Map[String, String]): BaseRelation = { @@ -69,7 +96,8 @@ class JdbcRelationProvider extends CreatableRelationProvider } else { // Otherwise, do not truncate the table, instead drop and recreate it dropTable(conn, options.table) - createTable(conn, df, options) + val caseSensitiveAnalysis = df.sparkSession.sessionState.conf.caseSensitiveAnalysis + createTable(conn, df.schema, caseSensitiveAnalysis, options) saveTable(df, Some(df.schema), isCaseSensitive, options) } @@ -87,7 +115,8 @@ class JdbcRelationProvider extends CreatableRelationProvider // Therefore, it is okay to do nothing here and then just return the relation below. } } else { - createTable(conn, df, options) + val caseSensitiveAnalysis = df.sparkSession.sessionState.conf.caseSensitiveAnalysis + createTable(conn, df.schema, caseSensitiveAnalysis, options) saveTable(df, Some(df.schema), isCaseSensitive, options) } } finally { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala index bbe9024f13a4..ccd62567ce22 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala @@ -35,7 +35,6 @@ import org.apache.spark.sql.catalyst.parser.CatalystSqlParser import org.apache.spark.sql.catalyst.util.{CaseInsensitiveMap, DateTimeUtils, GenericArrayData} import org.apache.spark.sql.jdbc.{JdbcDialect, JdbcDialects, JdbcType} import org.apache.spark.sql.types._ -import org.apache.spark.sql.util.SchemaUtils import org.apache.spark.unsafe.types.UTF8String import org.apache.spark.util.NextIterator @@ -117,12 +116,12 @@ object JdbcUtils extends Logging { table: String, rddSchema: StructType, tableSchema: Option[StructType], - isCaseSensitive: Boolean, + caseSensitiveAnalysis: Boolean, dialect: JdbcDialect): String = { val columns = if (tableSchema.isEmpty) { rddSchema.fields.map(x => dialect.quoteIdentifier(x.name)).mkString(",") } else { - val columnNameEquality = if (isCaseSensitive) { + val columnNameEquality = if (caseSensitiveAnalysis) { org.apache.spark.sql.catalyst.analysis.caseSensitiveResolution } else { org.apache.spark.sql.catalyst.analysis.caseInsensitiveResolution @@ -710,15 +709,16 @@ object JdbcUtils extends Logging { * Compute the schema string for this RDD. */ def schemaString( - df: DataFrame, + schema: StructType, + caseSensitiveAnalysis: Boolean, url: String, createTableColumnTypes: Option[String] = None): String = { val sb = new StringBuilder() val dialect = JdbcDialects.get(url) val userSpecifiedColTypesMap = createTableColumnTypes - .map(parseUserSpecifiedCreateTableColumnTypes(df, _)) + .map(parseUserSpecifiedCreateTableColumnTypes(schema, caseSensitiveAnalysis, _)) .getOrElse(Map.empty[String, String]) - df.schema.fields.foreach { field => + schema.fields.foreach { field => val name = dialect.quoteIdentifier(field.name) val typ = userSpecifiedColTypesMap .getOrElse(field.name, getJdbcType(field.dataType, dialect).databaseTypeDefinition) @@ -734,8 +734,8 @@ object JdbcUtils extends Logging { * use in-place of the default data type. */ private def parseUserSpecifiedCreateTableColumnTypes( - df: DataFrame, - createTableColumnTypes: String): Map[String, String] = { + schema: StructType, caseSensitiveAnalysis: Boolean, createTableColumnTypes: String) + : Map[String, String] = { def typeName(f: StructField): String = { // char/varchar gets translated to string type. Real data type specified by the user // is available in the field metadata as HIVE_TYPE_STRING @@ -747,24 +747,8 @@ object JdbcUtils extends Logging { } val userSchema = CatalystSqlParser.parseTableSchema(createTableColumnTypes) - val nameEquality = df.sparkSession.sessionState.conf.resolver - - // checks duplicate columns in the user specified column types. - SchemaUtils.checkColumnNameDuplication( - userSchema.map(_.name), "in the createTableColumnTypes option value", nameEquality) - - // checks if user specified column names exist in the DataFrame schema - userSchema.fieldNames.foreach { col => - df.schema.find(f => nameEquality(f.name, col)).getOrElse { - throw new AnalysisException( - s"createTableColumnTypes option column $col not found in schema " + - df.schema.catalogString) - } - } - val userSchemaMap = userSchema.fields.map(f => f.name -> typeName(f)).toMap - val isCaseSensitive = df.sparkSession.sessionState.conf.caseSensitiveAnalysis - if (isCaseSensitive) userSchemaMap else CaseInsensitiveMap(userSchemaMap) + if (caseSensitiveAnalysis) userSchemaMap else CaseInsensitiveMap(userSchemaMap) } /** @@ -773,7 +757,7 @@ object JdbcUtils extends Logging { def saveTable( df: DataFrame, tableSchema: Option[StructType], - isCaseSensitive: Boolean, + caseSensitiveAnalysis: Boolean, options: JDBCOptions): Unit = { val url = options.url val table = options.table @@ -783,7 +767,8 @@ object JdbcUtils extends Logging { val batchSize = options.batchSize val isolationLevel = options.isolationLevel - val insertStmt = getInsertStatement(table, rddSchema, tableSchema, isCaseSensitive, dialect) + val insertStmt = getInsertStatement( + table, rddSchema, tableSchema, caseSensitiveAnalysis, dialect) val repartitionedDF = options.numPartitions match { case Some(n) if n <= 0 => throw new IllegalArgumentException( s"Invalid value `$n` for parameter `${JDBCOptions.JDBC_NUM_PARTITIONS}` in table writing " + @@ -801,10 +786,11 @@ object JdbcUtils extends Logging { */ def createTable( conn: Connection, - df: DataFrame, + schema: StructType, + caseSensitiveAnalysis: Boolean, options: JDBCOptions): Unit = { val strSchema = schemaString( - df, options.url, options.createTableColumnTypes) + schema, caseSensitiveAnalysis, options.url, options.createTableColumnTypes) val table = options.table val createTableOptions = options.createTableOptions // Create the table if the table does not exist. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala index 508253433d38..27c71027f7f4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala @@ -27,9 +27,6 @@ import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, RowOrdering, import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.command._ -import org.apache.spark.sql.execution.datasources.csv.CSVFileFormat -import org.apache.spark.sql.execution.datasources.json.JsonFileFormat -import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources.InsertableRelation import org.apache.spark.sql.types.{AtomicType, StructType} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala index d1daf860fdff..bc4d3e9cb354 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala @@ -883,7 +883,9 @@ class JDBCSuite extends SparkFunSuite test("SPARK-16387: Reserved SQL words are not escaped by JDBC writer") { val df = spark.createDataset(Seq("a", "b", "c")).toDF("order") - val schema = JdbcUtils.schemaString(df, "jdbc:mysql://localhost:3306/temp") + val caseSensitiveAnalysis = df.sparkSession.sessionState.conf.caseSensitiveAnalysis + val schema = JdbcUtils.schemaString( + df.schema, caseSensitiveAnalysis, "jdbc:mysql://localhost:3306/temp") assert(schema.contains("`order` TEXT")) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala index eac773057251..410ae13d8306 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala @@ -377,7 +377,10 @@ class JDBCWriteSuite extends SharedSQLContext with BeforeAndAfter { val expectedSchemaStr = colTypes.map { case (col, dataType) => s""""$col" $dataType """ }.mkString(", ") - assert(JdbcUtils.schemaString(df, url1, Option(createTableColTypes)) == expectedSchemaStr) + val caseSensitiveAnalysis = df.sparkSession.sessionState.conf.caseSensitiveAnalysis + val schemaStr = JdbcUtils.schemaString( + df.schema, caseSensitiveAnalysis, url1, Option(createTableColTypes)) + assert(schemaStr == expectedSchemaStr) } testCreateTableColDataTypes(Seq("boolean")) From be201279949b32015ff0d9485fbf44a2c44c109b Mon Sep 17 00:00:00 2001 From: Takeshi Yamamuro Date: Mon, 3 Jul 2017 20:06:29 +0900 Subject: [PATCH 21/27] Revert "Brush up code" This reverts commit 12159c403955f54066ed8c532ed991f829edfc1f. --- .../sql/catalyst/catalog/SessionCatalog.scala | 5 + .../spark/sql/execution/command/views.scala | 131 ++++++++++-- .../execution/datasources/DataSource.scala | 3 + .../InsertIntoHadoopFsRelationCommand.scala | 7 + .../datasources/PartitioningUtils.scala | 4 + .../sql/execution/datasources/rules.scala | 189 +----------------- .../internal/BaseSessionStateBuilder.scala | 2 +- .../resources/sql-tests/inputs/create.sql | 8 + .../sql-tests/results/create.sql.out | 52 ++++- .../sql/execution/command/DDLSuite.scala | 4 +- .../sql/hive/HiveSessionStateBuilder.scala | 2 +- 11 files changed, 203 insertions(+), 204 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala index 24fff3ad563a..0a7ca4280bd0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala @@ -40,6 +40,7 @@ import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SubqueryAlias, import org.apache.spark.sql.catalyst.util.StringUtils import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.util.SchemaUtils object SessionCatalog { val DEFAULT_DATABASE = "default" @@ -341,6 +342,10 @@ class SessionCatalog( requireDbExists(db) requireTableExists(tableIdentifier) + SchemaUtils.checkSchemaColumnNameDuplication( + newSchema, "in the table definition of " + tableIdentifier.identifier, + conf.caseSensitiveAnalysis) + val catalogTable = externalCatalog.getTable(db, table) val oldSchema = catalogTable.schema diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala index bffe8e4bbc26..e167dc1b6a34 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala @@ -21,11 +21,13 @@ import scala.collection.mutable import org.apache.spark.sql.{AnalysisException, Row, SparkSession} import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.analysis.{UnresolvedFunction, UnresolvedRelation} import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable, CatalogTableType} -import org.apache.spark.sql.catalyst.expressions.Alias +import org.apache.spark.sql.catalyst.expressions.{Alias, SubqueryExpression} import org.apache.spark.sql.catalyst.plans.QueryPlan -import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project} +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project, View} import org.apache.spark.sql.types.MetadataBuilder +import org.apache.spark.sql.util.SchemaUtils /** @@ -122,19 +124,28 @@ case class CreateViewCommand( } override def run(sparkSession: SparkSession): Seq[Row] = { + // If the plan cannot be analyzed, throw an exception and don't proceed. + val qe = sparkSession.sessionState.executePlan(child) + qe.assertAnalyzed() + val analyzedPlan = qe.analyzed + if (userSpecifiedColumns.nonEmpty && - userSpecifiedColumns.length != child.output.length) { + userSpecifiedColumns.length != analyzedPlan.output.length) { throw new AnalysisException(s"The number of columns produced by the SELECT clause " + - s"(num: `${child.output.length}`) does not match the number of column names " + + s"(num: `${analyzedPlan.output.length}`) does not match the number of column names " + s"specified by CREATE VIEW (num: `${userSpecifiedColumns.length}`).") } + // When creating a permanent view, not allowed to reference temporary objects. + // This should be called after `qe.assertAnalyzed()` (i.e., `child` can be resolved) + verifyTemporaryObjectsNotExists(sparkSession) + val catalog = sparkSession.sessionState.catalog if (viewType == LocalTempView) { - val aliasedPlan = aliasPlan(sparkSession, child) + val aliasedPlan = aliasPlan(sparkSession, analyzedPlan) catalog.createTempView(name.table, aliasedPlan, overrideIfExists = replace) } else if (viewType == GlobalTempView) { - val aliasedPlan = aliasPlan(sparkSession, child) + val aliasedPlan = aliasPlan(sparkSession, analyzedPlan) catalog.createGlobalTempView(name.table, aliasedPlan, overrideIfExists = replace) } else if (catalog.tableExists(name)) { val tableMetadata = catalog.getTableMetadata(name) @@ -144,11 +155,14 @@ case class CreateViewCommand( } else if (tableMetadata.tableType != CatalogTableType.VIEW) { throw new AnalysisException(s"$name is not a view") } else if (replace) { + // Detect cyclic view reference on CREATE OR REPLACE VIEW. + val viewIdent = tableMetadata.identifier + checkCyclicViewReference(analyzedPlan, Seq(viewIdent), viewIdent) + // Handles `CREATE OR REPLACE VIEW v0 AS SELECT ...` // Nothing we need to retain from the old view, so just drop and create a new one - val viewIdent = tableMetadata.identifier catalog.dropTable(viewIdent, ignoreIfNotExists = false, purge = false) - catalog.createTable(prepareTable(sparkSession, child), ignoreIfExists = false) + catalog.createTable(prepareTable(sparkSession, analyzedPlan), ignoreIfExists = false) } else { // Handles `CREATE VIEW v0 AS SELECT ...`. Throws exception when the target view already // exists. @@ -158,11 +172,39 @@ case class CreateViewCommand( } } else { // Create the view if it doesn't exist. - catalog.createTable(prepareTable(sparkSession, child), ignoreIfExists = false) + catalog.createTable(prepareTable(sparkSession, analyzedPlan), ignoreIfExists = false) } Seq.empty[Row] } + /** + * Permanent views are not allowed to reference temp objects, including temp function and views + */ + private def verifyTemporaryObjectsNotExists(sparkSession: SparkSession): Unit = { + if (!isTemporary) { + // This func traverses the unresolved plan `child`. Below are the reasons: + // 1) Analyzer replaces unresolved temporary views by a SubqueryAlias with the corresponding + // logical plan. After replacement, it is impossible to detect whether the SubqueryAlias is + // added/generated from a temporary view. + // 2) The temp functions are represented by multiple classes. Most are inaccessible from this + // package (e.g., HiveGenericUDF). + child.collect { + // Disallow creating permanent views based on temporary views. + case s: UnresolvedRelation + if sparkSession.sessionState.catalog.isTemporaryTable(s.tableIdentifier) => + throw new AnalysisException(s"Not allowed to create a permanent view $name by " + + s"referencing a temporary view ${s.tableIdentifier}") + case other if !other.resolved => other.expressions.flatMap(_.collect { + // Disallow creating permanent views based on temporary UDFs. + case e: UnresolvedFunction + if sparkSession.sessionState.catalog.isTemporaryFunction(e.name) => + throw new AnalysisException(s"Not allowed to create a permanent view $name by " + + s"referencing a temporary function `${e.name}`") + }) + } + } + } + /** * If `userSpecifiedColumns` is defined, alias the analyzed plan to the user specified columns, * else return the analyzed plan directly. @@ -228,10 +270,15 @@ case class AlterViewAsCommand( override def innerChildren: Seq[QueryPlan[_]] = Seq(query) override def run(session: SparkSession): Seq[Row] = { - if (session.sessionState.catalog.alterTempViewDefinition(name, query)) { + // If the plan cannot be analyzed, throw an exception and don't proceed. + val qe = session.sessionState.executePlan(query) + qe.assertAnalyzed() + val analyzedPlan = qe.analyzed + + if (session.sessionState.catalog.alterTempViewDefinition(name, analyzedPlan)) { // a local/global temp view has been altered, we are done. } else { - alterPermanentView(session, query) + alterPermanentView(session, analyzedPlan) } Seq.empty[Row] @@ -243,6 +290,10 @@ case class AlterViewAsCommand( throw new AnalysisException(s"${viewMeta.identifier} is not a view.") } + // Detect cyclic view reference on ALTER VIEW. + val viewIdent = viewMeta.identifier + checkCyclicViewReference(analyzedPlan, Seq(viewIdent), viewIdent) + val newProperties = generateViewProperties(viewMeta.properties, session, analyzedPlan) val updatedViewMeta = viewMeta.copy( @@ -305,12 +356,66 @@ object ViewHelper { properties: Map[String, String], session: SparkSession, analyzedPlan: LogicalPlan): Map[String, String] = { - // Generate the view default database name. val queryOutput = analyzedPlan.schema.fieldNames - val viewDefaultDatabase = session.sessionState.catalog.getCurrentDatabase + // Generate the query column names, throw an AnalysisException if there exists duplicate column + // names. + SchemaUtils.checkColumnNameDuplication( + queryOutput, "the view", session.sessionState.conf.resolver) + + // Generate the view default database name. + val viewDefaultDatabase = session.sessionState.catalog.getCurrentDatabase removeQueryColumnNames(properties) ++ generateViewDefaultDatabase(viewDefaultDatabase) ++ generateQueryColumnNames(queryOutput) } + + /** + * Recursively search the logical plan to detect cyclic view references, throw an + * AnalysisException if cycle detected. + * + * A cyclic view reference is a cycle of reference dependencies, for example, if the following + * statements are executed: + * CREATE VIEW testView AS SELECT id FROM tbl + * CREATE VIEW testView2 AS SELECT id FROM testView + * ALTER VIEW testView AS SELECT * FROM testView2 + * The view `testView` references `testView2`, and `testView2` also references `testView`, + * therefore a reference cycle (testView -> testView2 -> testView) exists. + * + * @param plan the logical plan we detect cyclic view references from. + * @param path the path between the altered view and current node. + * @param viewIdent the table identifier of the altered view, we compare two views by the + * `desc.identifier`. + */ + def checkCyclicViewReference( + plan: LogicalPlan, + path: Seq[TableIdentifier], + viewIdent: TableIdentifier): Unit = { + plan match { + case v: View => + val ident = v.desc.identifier + val newPath = path :+ ident + // If the table identifier equals to the `viewIdent`, current view node is the same with + // the altered view. We detect a view reference cycle, should throw an AnalysisException. + if (ident == viewIdent) { + throw new AnalysisException(s"Recursive view $viewIdent detected " + + s"(cycle: ${newPath.mkString(" -> ")})") + } else { + v.children.foreach { child => + checkCyclicViewReference(child, newPath, viewIdent) + } + } + case _ => + plan.children.foreach(child => checkCyclicViewReference(child, path, viewIdent)) + } + + // Detect cyclic references from subqueries. + plan.expressions.foreach { expr => + expr match { + case s: SubqueryExpression => + checkCyclicViewReference(s.plan, path, viewIdent) + case _ => // Do nothing. + } + } + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala index 35ee056caff5..6c43c3eca301 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala @@ -352,6 +352,9 @@ case class DataSource( "It must be specified manually") } + SchemaUtils.checkSchemaColumnNameDuplication( + dataSchema, "in the datasource", sparkSession.sessionState.conf.caseSensitiveAnalysis) + HadoopFsRelation( fileCatalog, partitionSchema = fileCatalog.partitionSchema, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala index f71733a408ae..a98cd3902214 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala @@ -63,6 +63,13 @@ case class InsertIntoHadoopFsRelationCommand( override def run(sparkSession: SparkSession, children: Seq[SparkPlan]): Seq[Row] = { assert(children.length == 1) + + // Most formats don't do well with duplicate columns, so lets not allow that + SchemaUtils.checkSchemaColumnNameDuplication( + query.schema, + s"when inserting into $outputPath", + sparkSession.sessionState.conf.caseSensitiveAnalysis) + val hadoopConf = sparkSession.sessionState.newHadoopConfWithOptions(options) val fs = outputPath.getFileSystem(hadoopConf) val qualifiedOutputPath = outputPath.makeQualified(fs.getUri, fs.getWorkingDirectory) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala index 2349115fd043..54cdd973420b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala @@ -33,6 +33,7 @@ import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec import org.apache.spark.sql.catalyst.expressions.{Cast, Literal} import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.types._ +import org.apache.spark.sql.util.SchemaUtils // TODO: We should tighten up visibility of the classes here once we clean up Hive coupling. @@ -301,6 +302,9 @@ object PartitioningUtils { normalizedKey -> value } + SchemaUtils.checkColumnNameDuplication( + normalizedPartSpec.map(_._1), "in the partition specification", resolver) + normalizedPartSpec.toMap } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala index 27c71027f7f4..85feef9e5788 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala @@ -20,13 +20,12 @@ package org.apache.spark.sql.execution.datasources import java.util.Locale import org.apache.spark.sql.{AnalysisException, SaveMode, SparkSession} -import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.catalog._ -import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, RowOrdering, SubqueryExpression} +import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, RowOrdering} import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.execution.command._ +import org.apache.spark.sql.execution.command.DDLUtils import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources.InsertableRelation import org.apache.spark.sql.types.{AtomicType, StructType} @@ -68,10 +67,9 @@ class ResolveSQLOnFile(sparkSession: SparkSession) extends Rule[LogicalPlan] { } /** - * Preprocess DDL commands (e.g., [[CreateTable]] and [[AlterTableAddColumnsCommand]]), to do some - * normalization and checking. + * Preprocess [[CreateTable]], to do some normalization and checking. */ -case class PreprocessDDLCommands(sparkSession: SparkSession) extends Rule[LogicalPlan] { +case class PreprocessTableCreation(sparkSession: SparkSession) extends Rule[LogicalPlan] { // catalog is a def and not a val/lazy val as the latter would introduce a circular reference private def catalog = sparkSession.sessionState.catalog @@ -222,120 +220,12 @@ case class PreprocessDDLCommands(sparkSession: SparkSession) extends Rule[Logica c.copy(tableDesc = normalizedTable.copy(schema = reorderedSchema)) } - - case a @ AlterTableAddColumnsCommand(table, columns) => - val catalog = sparkSession.sessionState.catalog - val catalogTable = catalog.getTempViewOrPermanentTableMetadata(table) - val newSchema = catalogTable.dataSchema ++ columns ++ catalogTable.partitionSchema - SchemaUtils.checkColumnNameDuplication( - newSchema.map(_.name), s"in the table definition of ${table.identifier}", - sparkSession.sessionState.conf.caseSensitiveAnalysis) - a - - case a @ AlterTableAddPartitionCommand(tableName, partitionSpecsAndLocs, _) => - partitionSpecsAndLocs.foreach { case (spec, _) => - val partCols = spec.toSeq.map(_._1) - SchemaUtils.checkColumnNameDuplication( - partCols, s"the table definition of ${tableName.identifier}", - sparkSession.sessionState.conf.caseSensitiveAnalysis) - } - a - - case a @ AlterTableRenamePartitionCommand(tableName, _, newPart) => - val newPartCols = newPart.toSeq.map(_._1) - SchemaUtils.checkColumnNameDuplication( - newPartCols, s"the table definition of ${tableName.identifier}", - sparkSession.sessionState.conf.caseSensitiveAnalysis) - a - - case a @ AlterTableDropPartitionCommand(tableName, partitionSpecs, _, _, _) => - partitionSpecs.foreach { spec => - val partCols = spec.toSeq.map(_._1) - SchemaUtils.checkColumnNameDuplication( - partCols, s"the table definition of ${tableName.identifier}", - sparkSession.sessionState.conf.caseSensitiveAnalysis) - } - a - - case t @ TruncateTableCommand(tableName, Some(partitionSpec)) => - val partCols = partitionSpec.toSeq.map(_._1) - SchemaUtils.checkColumnNameDuplication( - partCols, s"the table definition of ${tableName.identifier}", - sparkSession.sessionState.conf.caseSensitiveAnalysis) - t - - case c @ CreateViewCommand(name, _, _, _, _, child, _, replace, viewType) => - // If the plan cannot be analyzed, throw an exception and don't proceed. - val qe = sparkSession.sessionState.executePlan(child) - qe.assertAnalyzed() - val analyzedPlan = qe.analyzed - def isTemporary = viewType == LocalTempView || viewType == GlobalTempView - if (!isTemporary) { - if (catalog.tableExists(name) && replace) { - // Detect cyclic view reference on CREATE OR REPLACE VIEW. - val tableMetadata = catalog.getTableMetadata(name) - val viewIdent = tableMetadata.identifier - PreprocessDDLCommands.checkCyclicViewReference(analyzedPlan, Seq(viewIdent), viewIdent) - } - - SchemaUtils.checkSchemaColumnNameDuplication( - analyzedPlan.schema, s"the view definition of ${name.identifier}", - sparkSession.sessionState.conf.caseSensitiveAnalysis) - - // When creating a permanent view, not allowed to reference temporary objects. - // This should be called after `qe.assertAnalyzed()` (i.e., `child` can be resolved) - verifyTemporaryObjectsNotExists(name, child) - } - c.copy(child = analyzedPlan) - - case a @ AlterViewAsCommand(name, _, query) => - // If the plan cannot be analyzed, throw an exception and don't proceed. - val qe = sparkSession.sessionState.executePlan(query) - qe.assertAnalyzed() - val analyzedPlan = qe.analyzed - - // First, detect cyclic view reference on ALTER VIEW. - val viewMeta = sparkSession.sessionState.catalog.getTempViewOrPermanentTableMetadata(name) - val viewIdent = viewMeta.identifier - PreprocessDDLCommands.checkCyclicViewReference(analyzedPlan, Seq(viewIdent), viewIdent) - - // Then, check name duplication in the new view table - SchemaUtils.checkSchemaColumnNameDuplication( - analyzedPlan.schema, s"the view definition of ${name.identifier}", - sparkSession.sessionState.conf.caseSensitiveAnalysis) - - a.copy(query = analyzedPlan) - } - - /** - * Permanent views are not allowed to reference temp objects, including temp function and views - */ - private def verifyTemporaryObjectsNotExists(name: TableIdentifier, plan: LogicalPlan): Unit = { - // This func traverses the unresolved plan `child`. Below are the reasons: - // 1) Analyzer replaces unresolved temporary views by a SubqueryAlias with the corresponding - // logical plan. After replacement, it is impossible to detect whether the SubqueryAlias is - // added/generated from a temporary view. - // 2) The temp functions are represented by multiple classes. Most are inaccessible from this - // package (e.g., HiveGenericUDF). - plan.collect { - // Disallow creating permanent views based on temporary views. - case s: UnresolvedRelation - if sparkSession.sessionState.catalog.isTemporaryTable(s.tableIdentifier) => - throw new AnalysisException(s"Not allowed to create a permanent view $name by " + - s"referencing a temporary view ${s.tableIdentifier}") - case other if !other.resolved => other.expressions.flatMap(_.collect { - // Disallow creating permanent views based on temporary UDFs. - case e: UnresolvedFunction - if sparkSession.sessionState.catalog.isTemporaryFunction(e.name) => - throw new AnalysisException(s"Not allowed to create a permanent view $name by " + - s"referencing a temporary function `${e.name}`") - }) - } } private def normalizeCatalogTable(schema: StructType, table: CatalogTable): CatalogTable = { SchemaUtils.checkSchemaColumnNameDuplication( - schema, s"in the table definition of ${table.identifier}", + schema, + "in the table definition of " + table.identifier, sparkSession.sessionState.conf.caseSensitiveAnalysis) val normalizedPartCols = normalizePartitionColumns(schema, table) @@ -363,7 +253,8 @@ case class PreprocessDDLCommands(sparkSession: SparkSession) extends Rule[Logica resolver = sparkSession.sessionState.conf.resolver) SchemaUtils.checkColumnNameDuplication( - normalizedPartitionCols, s"in the partition columns of ${table.identifier}", + normalizedPartitionCols, + "in the partition", sparkSession.sessionState.conf.resolver) if (schema.nonEmpty && normalizedPartitionCols.length == schema.length) { @@ -397,11 +288,11 @@ case class PreprocessDDLCommands(sparkSession: SparkSession) extends Rule[Logica SchemaUtils.checkColumnNameDuplication( normalizedBucketSpec.bucketColumnNames, - s"in the bucket column names of ${table.identifier}", + "in the bucket", sparkSession.sessionState.conf.resolver) SchemaUtils.checkColumnNameDuplication( normalizedBucketSpec.sortColumnNames, - s"in the sort column names of ${table.identifier}", + "in the sort", sparkSession.sessionState.conf.resolver) normalizedBucketSpec.sortColumnNames.map(schema(_)).map(_.dataType).foreach { @@ -418,58 +309,6 @@ case class PreprocessDDLCommands(sparkSession: SparkSession) extends Rule[Logica private def failAnalysis(msg: String) = throw new AnalysisException(msg) } -object PreprocessDDLCommands { - - /** - * Recursively search the logical plan to detect cyclic view references, throw an - * AnalysisException if cycle detected. - * - * A cyclic view reference is a cycle of reference dependencies, for example, if the following - * statements are executed: - * CREATE VIEW testView AS SELECT id FROM tbl - * CREATE VIEW testView2 AS SELECT id FROM testView - * ALTER VIEW testView AS SELECT * FROM testView2 - * The view `testView` references `testView2`, and `testView2` also references `testView`, - * therefore a reference cycle (testView -> testView2 -> testView) exists. - * - * @param plan the logical plan we detect cyclic view references from. - * @param path the path between the altered view and current node. - * @param viewIdent the table identifier of the altered view, we compare two views by the - * `desc.identifier`. - */ - def checkCyclicViewReference( - plan: LogicalPlan, - path: Seq[TableIdentifier], - viewIdent: TableIdentifier): Unit = { - plan match { - case v: View => - val ident = v.desc.identifier - val newPath = path :+ ident - // If the table identifier equals to the `viewIdent`, current view node is the same with - // the altered view. We detect a view reference cycle, should throw an AnalysisException. - if (ident == viewIdent) { - throw new AnalysisException(s"Recursive view $viewIdent detected " + - s"(cycle: ${newPath.mkString(" -> ")})") - } else { - v.children.foreach { child => - checkCyclicViewReference(child, newPath, viewIdent) - } - } - case _ => - plan.children.foreach(child => checkCyclicViewReference(child, path, viewIdent)) - } - - // Detect cyclic references from subqueries. - plan.expressions.foreach { expr => - expr match { - case s: SubqueryExpression => - checkCyclicViewReference(s.plan, path, viewIdent) - case _ => // Do nothing. - } - } - } -} - /** * Preprocess the [[InsertIntoTable]] plan. Throws exception if the number of columns mismatch, or * specified partition columns are different from the existing partition columns in the target @@ -481,8 +320,6 @@ case class PreprocessTableInsertion(conf: SQLConf) extends Rule[LogicalPlan] wit insert: InsertIntoTable, tblName: String, partColNames: Seq[String]): InsertIntoTable = { - SchemaUtils.checkColumnNameDuplication( - insert.partition.toSeq.map(_._1), s"when inserting into $tblName", conf.caseSensitiveAnalysis) val normalizedPartSpec = PartitioningUtils.normalizePartitionSpec( insert.partition, partColNames, tblName, conf.resolver) @@ -556,12 +393,6 @@ case class PreprocessTableInsertion(conf: SQLConf) extends Rule[LogicalPlan] wit preprocess(i, tblName, Nil) case _ => i } - - case i @ InsertIntoHadoopFsRelationCommand(outputPath, _, _, _, _, _, _, query, _, _, _) => - // Most formats don't do well with duplicate columns, so lets not allow that - SchemaUtils.checkSchemaColumnNameDuplication( - query.schema, s"when inserting into $outputPath", conf.caseSensitiveAnalysis) - i } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala index 0a043ae504cd..72d0ddc62303 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala @@ -161,7 +161,7 @@ abstract class BaseSessionStateBuilder( customResolutionRules override val postHocResolutionRules: Seq[Rule[LogicalPlan]] = - PreprocessDDLCommands(session) +: + PreprocessTableCreation(session) +: PreprocessTableInsertion(conf) +: DataSourceAnalysis(conf) +: customPostHocResolutionRules diff --git a/sql/core/src/test/resources/sql-tests/inputs/create.sql b/sql/core/src/test/resources/sql-tests/inputs/create.sql index 5a16df270a9c..a716923edc91 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/create.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/create.sql @@ -3,9 +3,17 @@ SET spark.sql.caseSensitive=true; CREATE TABLE t(c0 STRING, c1 INT, c1 DOUBLE, c0 INT) USING parquet; +CREATE TABLE t(c0 INT, c1 INT) PARTITIONED BY (c1 INT) STORED AS parquet; + +CREATE TABLE t(c0 INT) PARTITIONED BY (c1 INT, c1 INT) STORED AS parquet; + -- Catch case-insensitive name duplication SET spark.sql.caseSensitive=false; CREATE TABLE t(c0 STRING, c1 INT, c1 DOUBLE, c0 INT) USING parquet; CREATE TABLE t(ab STRING, cd INT, ef DOUBLE, Ab INT) USING parquet; + +CREATE TABLE t(ab INT, cd INT) PARTITIONED BY (Ab INT) STORED AS parquet; + +CREATE TABLE t(ab INT, cd INT) PARTITIONED BY (Ab INT, aB INT) STORED AS parquet; diff --git a/sql/core/src/test/resources/sql-tests/results/create.sql.out b/sql/core/src/test/resources/sql-tests/results/create.sql.out index 7d7de4f36204..d531e320aa19 100644 --- a/sql/core/src/test/resources/sql-tests/results/create.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/create.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 5 +-- Number of queries: 9 -- !query 0 @@ -20,26 +20,62 @@ Found duplicate column(s) in the table definition of `t`: `c1`, `c0`; -- !query 2 -SET spark.sql.caseSensitive=false +CREATE TABLE t(c0 INT, c1 INT) PARTITIONED BY (c1 INT) STORED AS parquet -- !query 2 schema -struct +struct<> -- !query 2 output -spark.sql.caseSensitive false +org.apache.spark.sql.AnalysisException +Found duplicate column(s) in the table definition of `t`: `c1`; -- !query 3 -CREATE TABLE t(c0 STRING, c1 INT, c1 DOUBLE, c0 INT) USING parquet +CREATE TABLE t(c0 INT) PARTITIONED BY (c1 INT, c1 INT) STORED AS parquet -- !query 3 schema struct<> -- !query 3 output org.apache.spark.sql.AnalysisException -Found duplicate column(s) in the table definition of `t`: `c1`, `c0`; +Found duplicate column(s) in the table definition of `t`: `c1`; -- !query 4 -CREATE TABLE t(ab STRING, cd INT, ef DOUBLE, Ab INT) USING parquet +SET spark.sql.caseSensitive=false -- !query 4 schema -struct<> +struct -- !query 4 output +spark.sql.caseSensitive false + + +-- !query 5 +CREATE TABLE t(c0 STRING, c1 INT, c1 DOUBLE, c0 INT) USING parquet +-- !query 5 schema +struct<> +-- !query 5 output +org.apache.spark.sql.AnalysisException +Found duplicate column(s) in the table definition of `t`: `c1`, `c0`; + + +-- !query 6 +CREATE TABLE t(ab STRING, cd INT, ef DOUBLE, Ab INT) USING parquet +-- !query 6 schema +struct<> +-- !query 6 output +org.apache.spark.sql.AnalysisException +Found duplicate column(s) in the table definition of `t`: `ab`; + + +-- !query 7 +CREATE TABLE t(ab INT, cd INT) PARTITIONED BY (Ab INT) STORED AS parquet +-- !query 7 schema +struct<> +-- !query 7 output +org.apache.spark.sql.AnalysisException +Found duplicate column(s) in the table definition of `t`: `ab`; + + +-- !query 8 +CREATE TABLE t(ab INT, cd INT) PARTITIONED BY (Ab INT, aB INT) STORED AS parquet +-- !query 8 schema +struct<> +-- !query 8 output org.apache.spark.sql.AnalysisException Found duplicate column(s) in the table definition of `t`: `ab`; diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala index 7b18453105f5..b40eac5d5c77 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala @@ -469,14 +469,14 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils { val e = intercept[AnalysisException] { sql("CREATE TABLE tbl(a int) USING json PARTITIONED BY (a, a)") } - assert(e.message == "Found duplicate column(s) in the partition columns of `tbl`: `a`") + assert(e.message == "Found duplicate column(s) in the partition: `a`") } test("create table - column repeated in bucket columns") { val e = intercept[AnalysisException] { sql("CREATE TABLE tbl(a int) USING json CLUSTERED BY (a, a) INTO 4 BUCKETS") } - assert(e.message == "Found duplicate column(s) in the bucket column names of `tbl`: `a`") + assert(e.message == "Found duplicate column(s) in the bucket: `a`") } test("Refresh table after changing the data source table partitioning") { diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala index d1301e0cac2d..e16c9e46b772 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala @@ -76,7 +76,7 @@ class HiveSessionStateBuilder(session: SparkSession, parentState: Option[Session override val postHocResolutionRules: Seq[Rule[LogicalPlan]] = new DetermineTableStats(session) +: RelationConversions(conf, catalog) +: - PreprocessDDLCommands(session) +: + PreprocessTableCreation(session) +: PreprocessTableInsertion(conf) +: DataSourceAnalysis(conf) +: HiveAnalysis +: From f41bf80db71eb6bfb152412a7b7739b70febf969 Mon Sep 17 00:00:00 2001 From: Takeshi Yamamuro Date: Tue, 4 Jul 2017 21:13:23 +0900 Subject: [PATCH 22/27] Fix more issues --- .../sql/catalyst/catalog/SessionCatalog.scala | 5 -- .../command/createDataSourceTables.scala | 2 - .../spark/sql/execution/command/tables.scala | 8 ++- .../execution/datasources/DataSource.scala | 16 +++--- .../datasources/PartitioningUtils.scala | 2 +- .../sql/execution/datasources/rules.scala | 6 +-- .../resources/sql-tests/inputs/create.sql | 12 +++-- .../sql-tests/results/create.sql.out | 52 +++++++++++++------ .../sql/execution/command/DDLSuite.scala | 13 +++-- .../sql/test/DataFrameReaderWriterSuite.scala | 40 +++++++++----- 10 files changed, 100 insertions(+), 56 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala index 0a7ca4280bd0..24fff3ad563a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala @@ -40,7 +40,6 @@ import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SubqueryAlias, import org.apache.spark.sql.catalyst.util.StringUtils import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.StructType -import org.apache.spark.sql.util.SchemaUtils object SessionCatalog { val DEFAULT_DATABASE = "default" @@ -342,10 +341,6 @@ class SessionCatalog( requireDbExists(db) requireTableExists(tableIdentifier) - SchemaUtils.checkSchemaColumnNameDuplication( - newSchema, "in the table definition of " + tableIdentifier.identifier, - conf.caseSensitiveAnalysis) - val catalogTable = externalCatalog.getTable(db, table) val oldSchema = catalogTable.schema diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala index 729bd39d821c..04b2534ca5eb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala @@ -19,8 +19,6 @@ package org.apache.spark.sql.execution.command import java.net.URI -import org.apache.hadoop.fs.Path - import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala index 8ded1060f7bf..fa50d1272241 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala @@ -20,13 +20,11 @@ package org.apache.spark.sql.execution.command import java.io.File import java.net.URI import java.nio.file.FileSystems -import java.util.Date import scala.collection.mutable.ArrayBuffer import scala.util.control.NonFatal import scala.util.Try -import org.apache.commons.lang3.StringEscapeUtils import org.apache.hadoop.fs.Path import org.apache.spark.sql.{AnalysisException, Row, SparkSession} @@ -42,6 +40,7 @@ import org.apache.spark.sql.execution.datasources.csv.CSVFileFormat import org.apache.spark.sql.execution.datasources.json.JsonFileFormat import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat import org.apache.spark.sql.types._ +import org.apache.spark.sql.util.SchemaUtils import org.apache.spark.util.Utils /** @@ -202,6 +201,11 @@ case class AlterTableAddColumnsCommand( // make sure any partition columns are at the end of the fields val reorderedSchema = catalogTable.dataSchema ++ columns ++ catalogTable.partitionSchema + + SchemaUtils.checkColumnNameDuplication( + reorderedSchema.map(_.name), "in the table definition of " + table.identifier, + conf.caseSensitiveAnalysis) + catalog.alterTableSchema( table, catalogTable.schema.copy(fields = reorderedSchema.toArray)) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala index 6c43c3eca301..abac6d6ad0ca 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala @@ -94,7 +94,13 @@ case class DataSource( dataSchema.map(_.name), "in the data schema", equality) } SchemaUtils.checkColumnNameDuplication( - partitionColumns, "in the partition schema", equality) + partitionColumns, "in the partition column(s)", equality) + bucketSpec.map { bucket => + SchemaUtils.checkColumnNameDuplication( + bucket.bucketColumnNames, "in the bucket column(s)", equality) + SchemaUtils.checkColumnNameDuplication( + bucket.sortColumnNames, "in the sort column(s)", equality) + } /** * Get the schema of the given FileFormat, if provided by `userSpecifiedSchema`, or try to infer @@ -192,7 +198,7 @@ case class DataSource( SchemaUtils.checkColumnNameDuplication( dataSchema.map(_.name), "in the data schema", equality) SchemaUtils.checkColumnNameDuplication( - partitionSchema.map(_.name), "in the partition schema", equality) + partitionSchema.map(_.name), "in the partition column(s)", equality) // We just print a waring message if the data schema and partition schema have the duplicate // columns. This is because we allow users to do so in the previous Spark releases and @@ -200,7 +206,8 @@ case class DataSource( // See SPARK-18108 and SPARK-21144 for related discussions. try { SchemaUtils.checkColumnNameDuplication( - (dataSchema ++ partitionSchema).map(_.name), "in the data schema and the partition schema", + (dataSchema ++ partitionSchema).map(_.name), + "in the data schema and the partition column(s)", equality) } catch { case e: AnalysisException => logWarning(e.getMessage) @@ -352,9 +359,6 @@ case class DataSource( "It must be specified manually") } - SchemaUtils.checkSchemaColumnNameDuplication( - dataSchema, "in the datasource", sparkSession.sessionState.conf.caseSensitiveAnalysis) - HadoopFsRelation( fileCatalog, partitionSchema = fileCatalog.partitionSchema, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala index 54cdd973420b..f8d9038c93ae 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala @@ -303,7 +303,7 @@ object PartitioningUtils { } SchemaUtils.checkColumnNameDuplication( - normalizedPartSpec.map(_._1), "in the partition specification", resolver) + normalizedPartSpec.map(_._1), "in the partition column(s)", resolver) normalizedPartSpec.toMap } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala index 85feef9e5788..f263f105489f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala @@ -254,7 +254,7 @@ case class PreprocessTableCreation(sparkSession: SparkSession) extends Rule[Logi SchemaUtils.checkColumnNameDuplication( normalizedPartitionCols, - "in the partition", + "in the partition column(s)", sparkSession.sessionState.conf.resolver) if (schema.nonEmpty && normalizedPartitionCols.length == schema.length) { @@ -288,11 +288,11 @@ case class PreprocessTableCreation(sparkSession: SparkSession) extends Rule[Logi SchemaUtils.checkColumnNameDuplication( normalizedBucketSpec.bucketColumnNames, - "in the bucket", + "in the bucket column(s)", sparkSession.sessionState.conf.resolver) SchemaUtils.checkColumnNameDuplication( normalizedBucketSpec.sortColumnNames, - "in the sort", + "in the sort column(s)", sparkSession.sessionState.conf.resolver) normalizedBucketSpec.sortColumnNames.map(schema(_)).map(_.dataType).foreach { diff --git a/sql/core/src/test/resources/sql-tests/inputs/create.sql b/sql/core/src/test/resources/sql-tests/inputs/create.sql index a716923edc91..65ff698d5166 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/create.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/create.sql @@ -3,9 +3,11 @@ SET spark.sql.caseSensitive=true; CREATE TABLE t(c0 STRING, c1 INT, c1 DOUBLE, c0 INT) USING parquet; -CREATE TABLE t(c0 INT, c1 INT) PARTITIONED BY (c1 INT) STORED AS parquet; +CREATE TABLE t(c0 INT) USING parquet PARTITIONED BY (c0, c0); -CREATE TABLE t(c0 INT) PARTITIONED BY (c1 INT, c1 INT) STORED AS parquet; +CREATE TABLE t(c0 INT) USING parquet CLUSTERED BY (c0, c0) INTO 2 BUCKETS; + +CREATE TABLE t(c0 INT, c1 INT) USING parquet CLUSTERED BY (c0) SORTED BY (c1, c1) INTO 2 BUCKETS; -- Catch case-insensitive name duplication SET spark.sql.caseSensitive=false; @@ -14,6 +16,8 @@ CREATE TABLE t(c0 STRING, c1 INT, c1 DOUBLE, c0 INT) USING parquet; CREATE TABLE t(ab STRING, cd INT, ef DOUBLE, Ab INT) USING parquet; -CREATE TABLE t(ab INT, cd INT) PARTITIONED BY (Ab INT) STORED AS parquet; +CREATE TABLE t(ab INT, cd INT) USING parquet PARTITIONED BY (Ab, aB); + +CREATE TABLE t(ab INT) USING parquet CLUSTERED BY (Ab, aB) INTO 2 BUCKETS; -CREATE TABLE t(ab INT, cd INT) PARTITIONED BY (Ab INT, aB INT) STORED AS parquet; +CREATE TABLE t(ab INT, cd INT) USING parquet CLUSTERED BY (ab) SORTED BY (cD, Cd) INTO 2 BUCKETS; diff --git a/sql/core/src/test/resources/sql-tests/results/create.sql.out b/sql/core/src/test/resources/sql-tests/results/create.sql.out index d531e320aa19..32689bf06ee8 100644 --- a/sql/core/src/test/resources/sql-tests/results/create.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/create.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 9 +-- Number of queries: 11 -- !query 0 @@ -20,51 +20,51 @@ Found duplicate column(s) in the table definition of `t`: `c1`, `c0`; -- !query 2 -CREATE TABLE t(c0 INT, c1 INT) PARTITIONED BY (c1 INT) STORED AS parquet +CREATE TABLE t(c0 INT) USING parquet PARTITIONED BY (c0, c0) -- !query 2 schema struct<> -- !query 2 output org.apache.spark.sql.AnalysisException -Found duplicate column(s) in the table definition of `t`: `c1`; +Found duplicate column(s) in the partition column(s): `c0`; -- !query 3 -CREATE TABLE t(c0 INT) PARTITIONED BY (c1 INT, c1 INT) STORED AS parquet +CREATE TABLE t(c0 INT) USING parquet CLUSTERED BY (c0, c0) INTO 2 BUCKETS -- !query 3 schema struct<> -- !query 3 output org.apache.spark.sql.AnalysisException -Found duplicate column(s) in the table definition of `t`: `c1`; +Found duplicate column(s) in the bucket column(s): `c0`; -- !query 4 -SET spark.sql.caseSensitive=false +CREATE TABLE t(c0 INT, c1 INT) USING parquet CLUSTERED BY (c0) SORTED BY (c1, c1) INTO 2 BUCKETS -- !query 4 schema -struct +struct<> -- !query 4 output -spark.sql.caseSensitive false +org.apache.spark.sql.AnalysisException +Found duplicate column(s) in the sort column(s): `c1`; -- !query 5 -CREATE TABLE t(c0 STRING, c1 INT, c1 DOUBLE, c0 INT) USING parquet +SET spark.sql.caseSensitive=false -- !query 5 schema -struct<> +struct -- !query 5 output -org.apache.spark.sql.AnalysisException -Found duplicate column(s) in the table definition of `t`: `c1`, `c0`; +spark.sql.caseSensitive false -- !query 6 -CREATE TABLE t(ab STRING, cd INT, ef DOUBLE, Ab INT) USING parquet +CREATE TABLE t(c0 STRING, c1 INT, c1 DOUBLE, c0 INT) USING parquet -- !query 6 schema struct<> -- !query 6 output org.apache.spark.sql.AnalysisException -Found duplicate column(s) in the table definition of `t`: `ab`; +Found duplicate column(s) in the table definition of `t`: `c1`, `c0`; -- !query 7 -CREATE TABLE t(ab INT, cd INT) PARTITIONED BY (Ab INT) STORED AS parquet +CREATE TABLE t(ab STRING, cd INT, ef DOUBLE, Ab INT) USING parquet -- !query 7 schema struct<> -- !query 7 output @@ -73,9 +73,27 @@ Found duplicate column(s) in the table definition of `t`: `ab`; -- !query 8 -CREATE TABLE t(ab INT, cd INT) PARTITIONED BY (Ab INT, aB INT) STORED AS parquet +CREATE TABLE t(ab INT, cd INT) USING parquet PARTITIONED BY (Ab, aB) -- !query 8 schema struct<> -- !query 8 output org.apache.spark.sql.AnalysisException -Found duplicate column(s) in the table definition of `t`: `ab`; +Found duplicate column(s) in the partition column(s): `ab`; + + +-- !query 9 +CREATE TABLE t(ab INT) USING parquet CLUSTERED BY (Ab, aB) INTO 2 BUCKETS +-- !query 9 schema +struct<> +-- !query 9 output +org.apache.spark.sql.AnalysisException +Found duplicate column(s) in the bucket column(s): `ab`; + + +-- !query 10 +CREATE TABLE t(ab INT, cd INT) USING parquet CLUSTERED BY (ab) SORTED BY (cD, Cd) INTO 2 BUCKETS +-- !query 10 schema +struct<> +-- !query 10 output +org.apache.spark.sql.AnalysisException +Found duplicate column(s) in the sort column(s): `cd`; diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala index b40eac5d5c77..079f3334b3f0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala @@ -469,14 +469,19 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils { val e = intercept[AnalysisException] { sql("CREATE TABLE tbl(a int) USING json PARTITIONED BY (a, a)") } - assert(e.message == "Found duplicate column(s) in the partition: `a`") + assert(e.message == "Found duplicate column(s) in the partition column(s): `a`") } - test("create table - column repeated in bucket columns") { - val e = intercept[AnalysisException] { + test("create table - column repeated in bucket/sort columns") { + var e = intercept[AnalysisException] { sql("CREATE TABLE tbl(a int) USING json CLUSTERED BY (a, a) INTO 4 BUCKETS") } - assert(e.message == "Found duplicate column(s) in the bucket: `a`") + assert(e.message == "Found duplicate column(s) in the bucket column(s): `a`") + + e = intercept[AnalysisException] { + sql("CREATE TABLE tbl(a int) USING json CLUSTERED BY (a) SORTED BY (a, a) INTO 4 BUCKETS") + } + assert(e.message == "Found duplicate column(s) in the sort column(s): `a`") } test("Refresh table after changing the data source table partitioning") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala index d741d125ffd5..b1fd49b98499 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala @@ -689,25 +689,41 @@ class DataFrameReaderWriterSuite extends QueryTest with SharedSQLContext with Be testRead(spark.read.schema(userSchemaString).text(Seq(dir, dir): _*), data ++ data, userSchema) } + test("SPARK-20460 Check name duplication in buckets") { + Seq((true, ("a", "a")), (false, ("aA", "Aa"))).foreach { case (caseSensitive, (c0, c1)) => + withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive.toString) { + var errorMsg = intercept[AnalysisException] { + Seq((1, 1)).toDF("col", c0).write.bucketBy(2, c0, c1).saveAsTable("t") + }.getMessage + assert(errorMsg.contains("Found duplicate column(s) in the bucket column(s)")) + + errorMsg = intercept[AnalysisException] { + Seq((1, 1)).toDF("col", c0).write.bucketBy(2, "col").sortBy(c0, c1).saveAsTable("t") + }.getMessage + assert(errorMsg.contains("Found duplicate column(s) in the sort column(s)")) + } + } + } + test("SPARK-20460 Check name duplication in schema") { def checkWriteDataColumnDuplication( format: String, colName0: String, colName1: String, tempDir: File): Unit = { - val e = intercept[AnalysisException] { + val errorMsg = intercept[AnalysisException] { Seq((1, 1)).toDF(colName0, colName1).write.format(format).mode("overwrite") .save(tempDir.getAbsolutePath) - } - assert(e.getMessage.contains("Found duplicate column(s) when inserting into")) + }.getMessage + assert(errorMsg.contains("Found duplicate column(s) when inserting into")) } def checkReadUserSpecifiedDataColumnDuplication( df: DataFrame, format: String, colName0: String, colName1: String, tempDir: File): Unit = { val testDir = Utils.createTempDir(tempDir.getAbsolutePath) df.write.format(format).mode("overwrite").save(testDir.getAbsolutePath) - val e = intercept[AnalysisException] { + val errorMsg = intercept[AnalysisException] { spark.read.format(format).schema(s"$colName0 INT, $colName1 INT") .load(testDir.getAbsolutePath) - } - assert(e.getMessage.contains("Found duplicate column(s) in the data schema:")) + }.getMessage + assert(errorMsg.contains("Found duplicate column(s) in the data schema:")) } def checkReadPartitionColumnDuplication( @@ -715,10 +731,10 @@ class DataFrameReaderWriterSuite extends QueryTest with SharedSQLContext with Be val testDir = Utils.createTempDir(tempDir.getAbsolutePath) Seq(1).toDF("col").write.format(format).mode("overwrite") .save(s"${testDir.getAbsolutePath}/$colName0=1/$colName1=1") - val e = intercept[AnalysisException] { + val errorMsg = intercept[AnalysisException] { spark.read.format(format).load(testDir.getAbsolutePath) - } - assert(e.getMessage.contains("Found duplicate column(s) in the partition schema:")) + }.getMessage + assert(errorMsg.contains("Found duplicate column(s) in the partition column(s):")) } Seq((true, ("a", "a")), (false, ("aA", "Aa"))).foreach { case (caseSensitive, (c0, c1)) => @@ -744,10 +760,10 @@ class DataFrameReaderWriterSuite extends QueryTest with SharedSQLContext with Be testDir = Utils.createTempDir(src.getAbsolutePath) Seq(s"""{"$c0":3, "$c1":5}""").toDF().write.mode("overwrite") .text(testDir.getAbsolutePath) - val e = intercept[AnalysisException] { + val errorMsg = intercept[AnalysisException] { spark.read.format("json").option("inferSchema", true).load(testDir.getAbsolutePath) - } - assert(e.getMessage.contains("Found duplicate column(s) in the data schema:")) + }.getMessage + assert(errorMsg.contains("Found duplicate column(s) in the data schema:")) checkReadPartitionColumnDuplication("json", c0, c1, src) // Check Parquet format From 052639173012bb57674e66e1e09b736a805d78d1 Mon Sep 17 00:00:00 2001 From: Takeshi Yamamuro Date: Wed, 5 Jul 2017 00:40:08 +0900 Subject: [PATCH 23/27] Revert DataSourceValidator --- .../execution/datasources/DataSource.scala | 8 +--- .../datasources/DataSourceValidator.scala | 36 --------------- .../jdbc/JdbcRelationProvider.scala | 35 ++------------- .../datasources/jdbc/JdbcUtils.scala | 44 ++++++++++++------- .../org/apache/spark/sql/jdbc/JDBCSuite.scala | 4 +- .../spark/sql/jdbc/JDBCWriteSuite.scala | 9 ++-- 6 files changed, 37 insertions(+), 99 deletions(-) delete mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceValidator.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala index abac6d6ad0ca..2c14a7edcc1a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala @@ -493,13 +493,7 @@ case class DataSource( throw new AnalysisException("Cannot save interval data type into external storage.") } - val resolvedRelation = providingClass.newInstance() match { - case relationToCheck: DataSourceValidator => - relationToCheck.validateSchema(data.schema, partitionColumns, equality, options) - relationToCheck - case r => r - } - resolvedRelation match { + providingClass.newInstance() match { case dataSource: CreatableRelationProvider => SaveIntoDataSourceCommand(data, dataSource, caseInsensitiveOptions, mode) case format: FileFormat => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceValidator.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceValidator.scala deleted file mode 100644 index 2d3ad0168461..000000000000 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceValidator.scala +++ /dev/null @@ -1,36 +0,0 @@ -/* - * 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.execution.datasources - -import org.apache.spark.sql.catalyst.analysis.Resolver -import org.apache.spark.sql.types.StructType - - -/** - * Used to validate an input schema for datasource-specific cases (e.g., schema definition - * from an datasource option and unsupported field names). `validateSchema` is called - * in [[DataSource]] for the checking. - */ -trait DataSourceValidator { - - def validateSchema( - schema: StructType, - partitionColumns: Seq[String], - resolver: Resolver, - options: Map[String, String]): Unit -} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcRelationProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcRelationProvider.scala index fd8f27a7e5fe..74dcfb06f5c2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcRelationProvider.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcRelationProvider.scala @@ -18,41 +18,14 @@ package org.apache.spark.sql.execution.datasources.jdbc import org.apache.spark.sql.{AnalysisException, DataFrame, SaveMode, SQLContext} -import org.apache.spark.sql.catalyst.analysis.Resolver -import org.apache.spark.sql.catalyst.parser.CatalystSqlParser -import org.apache.spark.sql.execution.datasources.DataSourceValidator import org.apache.spark.sql.execution.datasources.jdbc.JdbcUtils._ import org.apache.spark.sql.sources.{BaseRelation, CreatableRelationProvider, DataSourceRegister, RelationProvider} -import org.apache.spark.sql.types.StructType -import org.apache.spark.sql.util.SchemaUtils class JdbcRelationProvider extends CreatableRelationProvider - with RelationProvider with DataSourceValidator with DataSourceRegister { + with RelationProvider with DataSourceRegister { override def shortName(): String = "jdbc" - override def validateSchema( - schema: StructType, partCols: Seq[String], resolver: Resolver, options: Map[String, String]) - : Unit = { - val jdbcOptions = new JDBCOptions(options) - jdbcOptions.createTableColumnTypes.foreach { createTableColumnTypes => - val userSchema = CatalystSqlParser.parseTableSchema(createTableColumnTypes) - - // Checks duplicate columns in the user specified column types - SchemaUtils.checkColumnNameDuplication( - userSchema.map(_.name), "in the createTableColumnTypes option value", resolver) - - // Checks if user specified column names exist in the DataFrame schema - userSchema.fieldNames.foreach { col => - schema.find(f => resolver(f.name, col)).getOrElse { - throw new AnalysisException( - s"createTableColumnTypes option column $col not found in schema " + - schema.catalogString) - } - } - } - } - override def createRelation( sqlContext: SQLContext, parameters: Map[String, String]): BaseRelation = { @@ -96,8 +69,7 @@ class JdbcRelationProvider extends CreatableRelationProvider } else { // Otherwise, do not truncate the table, instead drop and recreate it dropTable(conn, options.table) - val caseSensitiveAnalysis = df.sparkSession.sessionState.conf.caseSensitiveAnalysis - createTable(conn, df.schema, caseSensitiveAnalysis, options) + createTable(conn, df, options) saveTable(df, Some(df.schema), isCaseSensitive, options) } @@ -115,8 +87,7 @@ class JdbcRelationProvider extends CreatableRelationProvider // Therefore, it is okay to do nothing here and then just return the relation below. } } else { - val caseSensitiveAnalysis = df.sparkSession.sessionState.conf.caseSensitiveAnalysis - createTable(conn, df.schema, caseSensitiveAnalysis, options) + createTable(conn, df, options) saveTable(df, Some(df.schema), isCaseSensitive, options) } } finally { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala index ccd62567ce22..bbe9024f13a4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala @@ -35,6 +35,7 @@ import org.apache.spark.sql.catalyst.parser.CatalystSqlParser import org.apache.spark.sql.catalyst.util.{CaseInsensitiveMap, DateTimeUtils, GenericArrayData} import org.apache.spark.sql.jdbc.{JdbcDialect, JdbcDialects, JdbcType} import org.apache.spark.sql.types._ +import org.apache.spark.sql.util.SchemaUtils import org.apache.spark.unsafe.types.UTF8String import org.apache.spark.util.NextIterator @@ -116,12 +117,12 @@ object JdbcUtils extends Logging { table: String, rddSchema: StructType, tableSchema: Option[StructType], - caseSensitiveAnalysis: Boolean, + isCaseSensitive: Boolean, dialect: JdbcDialect): String = { val columns = if (tableSchema.isEmpty) { rddSchema.fields.map(x => dialect.quoteIdentifier(x.name)).mkString(",") } else { - val columnNameEquality = if (caseSensitiveAnalysis) { + val columnNameEquality = if (isCaseSensitive) { org.apache.spark.sql.catalyst.analysis.caseSensitiveResolution } else { org.apache.spark.sql.catalyst.analysis.caseInsensitiveResolution @@ -709,16 +710,15 @@ object JdbcUtils extends Logging { * Compute the schema string for this RDD. */ def schemaString( - schema: StructType, - caseSensitiveAnalysis: Boolean, + df: DataFrame, url: String, createTableColumnTypes: Option[String] = None): String = { val sb = new StringBuilder() val dialect = JdbcDialects.get(url) val userSpecifiedColTypesMap = createTableColumnTypes - .map(parseUserSpecifiedCreateTableColumnTypes(schema, caseSensitiveAnalysis, _)) + .map(parseUserSpecifiedCreateTableColumnTypes(df, _)) .getOrElse(Map.empty[String, String]) - schema.fields.foreach { field => + df.schema.fields.foreach { field => val name = dialect.quoteIdentifier(field.name) val typ = userSpecifiedColTypesMap .getOrElse(field.name, getJdbcType(field.dataType, dialect).databaseTypeDefinition) @@ -734,8 +734,8 @@ object JdbcUtils extends Logging { * use in-place of the default data type. */ private def parseUserSpecifiedCreateTableColumnTypes( - schema: StructType, caseSensitiveAnalysis: Boolean, createTableColumnTypes: String) - : Map[String, String] = { + df: DataFrame, + createTableColumnTypes: String): Map[String, String] = { def typeName(f: StructField): String = { // char/varchar gets translated to string type. Real data type specified by the user // is available in the field metadata as HIVE_TYPE_STRING @@ -747,8 +747,24 @@ object JdbcUtils extends Logging { } val userSchema = CatalystSqlParser.parseTableSchema(createTableColumnTypes) + val nameEquality = df.sparkSession.sessionState.conf.resolver + + // checks duplicate columns in the user specified column types. + SchemaUtils.checkColumnNameDuplication( + userSchema.map(_.name), "in the createTableColumnTypes option value", nameEquality) + + // checks if user specified column names exist in the DataFrame schema + userSchema.fieldNames.foreach { col => + df.schema.find(f => nameEquality(f.name, col)).getOrElse { + throw new AnalysisException( + s"createTableColumnTypes option column $col not found in schema " + + df.schema.catalogString) + } + } + val userSchemaMap = userSchema.fields.map(f => f.name -> typeName(f)).toMap - if (caseSensitiveAnalysis) userSchemaMap else CaseInsensitiveMap(userSchemaMap) + val isCaseSensitive = df.sparkSession.sessionState.conf.caseSensitiveAnalysis + if (isCaseSensitive) userSchemaMap else CaseInsensitiveMap(userSchemaMap) } /** @@ -757,7 +773,7 @@ object JdbcUtils extends Logging { def saveTable( df: DataFrame, tableSchema: Option[StructType], - caseSensitiveAnalysis: Boolean, + isCaseSensitive: Boolean, options: JDBCOptions): Unit = { val url = options.url val table = options.table @@ -767,8 +783,7 @@ object JdbcUtils extends Logging { val batchSize = options.batchSize val isolationLevel = options.isolationLevel - val insertStmt = getInsertStatement( - table, rddSchema, tableSchema, caseSensitiveAnalysis, dialect) + val insertStmt = getInsertStatement(table, rddSchema, tableSchema, isCaseSensitive, dialect) val repartitionedDF = options.numPartitions match { case Some(n) if n <= 0 => throw new IllegalArgumentException( s"Invalid value `$n` for parameter `${JDBCOptions.JDBC_NUM_PARTITIONS}` in table writing " + @@ -786,11 +801,10 @@ object JdbcUtils extends Logging { */ def createTable( conn: Connection, - schema: StructType, - caseSensitiveAnalysis: Boolean, + df: DataFrame, options: JDBCOptions): Unit = { val strSchema = schemaString( - schema, caseSensitiveAnalysis, options.url, options.createTableColumnTypes) + df, options.url, options.createTableColumnTypes) val table = options.table val createTableOptions = options.createTableOptions // Create the table if the table does not exist. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala index bc4d3e9cb354..d1daf860fdff 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala @@ -883,9 +883,7 @@ class JDBCSuite extends SparkFunSuite test("SPARK-16387: Reserved SQL words are not escaped by JDBC writer") { val df = spark.createDataset(Seq("a", "b", "c")).toDF("order") - val caseSensitiveAnalysis = df.sparkSession.sessionState.conf.caseSensitiveAnalysis - val schema = JdbcUtils.schemaString( - df.schema, caseSensitiveAnalysis, "jdbc:mysql://localhost:3306/temp") + val schema = JdbcUtils.schemaString(df, "jdbc:mysql://localhost:3306/temp") assert(schema.contains("`order` TEXT")) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala index 410ae13d8306..f8840ca80c46 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.jdbc -import java.sql.{Date, DriverManager, Timestamp} +import java.sql.DriverManager import java.util.Properties import scala.collection.JavaConverters.propertiesAsScalaMapConverter @@ -377,10 +377,7 @@ class JDBCWriteSuite extends SharedSQLContext with BeforeAndAfter { val expectedSchemaStr = colTypes.map { case (col, dataType) => s""""$col" $dataType """ }.mkString(", ") - val caseSensitiveAnalysis = df.sparkSession.sessionState.conf.caseSensitiveAnalysis - val schemaStr = JdbcUtils.schemaString( - df.schema, caseSensitiveAnalysis, url1, Option(createTableColTypes)) - assert(schemaStr == expectedSchemaStr) + assert(JdbcUtils.schemaString(df, url1, Option(createTableColTypes)) == expectedSchemaStr) } testCreateTableColDataTypes(Seq("boolean")) @@ -482,7 +479,7 @@ class JDBCWriteSuite extends SharedSQLContext with BeforeAndAfter { .jdbc(url1, "TEST.USERDBTYPETEST", properties) }.getMessage() assert(msg.contains( - "Found duplicate column(s) in the createTableColumnTypes option value: `name`;")) + "Found duplicate column(s) in createTableColumnTypes option value: name, NaMe")) } } From 9e199bc1c03a22b61eca39406456790a59a0b09e Mon Sep 17 00:00:00 2001 From: Takeshi Yamamuro Date: Wed, 5 Jul 2017 00:51:31 +0900 Subject: [PATCH 24/27] Add the check for external relation providers --- .../spark/sql/execution/datasources/DataSource.scala | 8 +++++++- .../scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala | 2 +- 2 files changed, 8 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala index 2c14a7edcc1a..471e409955e8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala @@ -329,7 +329,13 @@ case class DataSource( case (dataSource: SchemaRelationProvider, Some(schema)) => dataSource.createRelation(sparkSession.sqlContext, caseInsensitiveOptions, schema) case (dataSource: RelationProvider, None) => - dataSource.createRelation(sparkSession.sqlContext, caseInsensitiveOptions) + val baseRelation = + dataSource.createRelation(sparkSession.sqlContext, caseInsensitiveOptions) + SchemaUtils.checkColumnNameDuplication( + baseRelation.schema.map(_.name), + "in the relation schema", + equality) + baseRelation case (_: SchemaRelationProvider, None) => throw new AnalysisException(s"A schema needs to be specified when using $className.") case (dataSource: RelationProvider, Some(schema)) => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala index f8840ca80c46..2334d5ae32dc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala @@ -479,7 +479,7 @@ class JDBCWriteSuite extends SharedSQLContext with BeforeAndAfter { .jdbc(url1, "TEST.USERDBTYPETEST", properties) }.getMessage() assert(msg.contains( - "Found duplicate column(s) in createTableColumnTypes option value: name, NaMe")) + "Found duplicate column(s) in the createTableColumnTypes option value: `name`")) } } From 1ae132d1ec2f54323ede16c089ec36987de1b5ec Mon Sep 17 00:00:00 2001 From: Takeshi Yamamuro Date: Wed, 5 Jul 2017 20:29:41 +0900 Subject: [PATCH 25/27] [WIP] Handle DataSource name duplication in one place --- .../execution/datasources/DataSource.scala | 31 ++++++++----------- 1 file changed, 13 insertions(+), 18 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala index 471e409955e8..459abc5b8845 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala @@ -89,12 +89,6 @@ case class DataSource( private val caseInsensitiveOptions = CaseInsensitiveMap(options) private val equality = sparkSession.sessionState.conf.resolver - userSpecifiedSchema.foreach { dataSchema => - SchemaUtils.checkColumnNameDuplication( - dataSchema.map(_.name), "in the data schema", equality) - } - SchemaUtils.checkColumnNameDuplication( - partitionColumns, "in the partition column(s)", equality) bucketSpec.map { bucket => SchemaUtils.checkColumnNameDuplication( bucket.bucketColumnNames, "in the bucket column(s)", equality) @@ -195,11 +189,6 @@ case class DataSource( s"Unable to infer schema for $format. It must be specified manually.") } - SchemaUtils.checkColumnNameDuplication( - dataSchema.map(_.name), "in the data schema", equality) - SchemaUtils.checkColumnNameDuplication( - partitionSchema.map(_.name), "in the partition column(s)", equality) - // We just print a waring message if the data schema and partition schema have the duplicate // columns. This is because we allow users to do so in the previous Spark releases and // we have the existing tests for the cases (e.g., `ParquetHadoopFsRelationSuite`). @@ -329,13 +318,7 @@ case class DataSource( case (dataSource: SchemaRelationProvider, Some(schema)) => dataSource.createRelation(sparkSession.sqlContext, caseInsensitiveOptions, schema) case (dataSource: RelationProvider, None) => - val baseRelation = - dataSource.createRelation(sparkSession.sqlContext, caseInsensitiveOptions) - SchemaUtils.checkColumnNameDuplication( - baseRelation.schema.map(_.name), - "in the relation schema", - equality) - baseRelation + dataSource.createRelation(sparkSession.sqlContext, caseInsensitiveOptions) case (_: SchemaRelationProvider, None) => throw new AnalysisException(s"A schema needs to be specified when using $className.") case (dataSource: RelationProvider, Some(schema)) => @@ -422,6 +405,18 @@ case class DataSource( s"$className is not a valid Spark SQL Data Source.") } + (relation match { + case hs: HadoopFsRelation => + Some((hs.dataSchema.map(_.name), hs.partitionSchema.map(_.name))) + case bs: BaseRelation => + Some((bs.schema.map(_.name), Seq.empty[String])) + case _ => + None + }).foreach { case (dataCols, partCols) => + SchemaUtils.checkColumnNameDuplication(dataCols, "in the data schema", equality) + SchemaUtils.checkColumnNameDuplication(partCols, "in the partition column(s)", equality) + } + relation } From 5c29a75c63acffc2965272267d53709aec256618 Mon Sep 17 00:00:00 2001 From: Takeshi Yamamuro Date: Thu, 6 Jul 2017 22:04:23 +0900 Subject: [PATCH 26/27] Fix more --- .../execution/datasources/DataSource.scala | 21 ++++++++++++------- .../InsertIntoHadoopFsRelationCommand.scala | 2 -- 2 files changed, 13 insertions(+), 10 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala index 459abc5b8845..66ed3c4bc2b2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala @@ -405,16 +405,21 @@ case class DataSource( s"$className is not a valid Spark SQL Data Source.") } - (relation match { + relation match { case hs: HadoopFsRelation => - Some((hs.dataSchema.map(_.name), hs.partitionSchema.map(_.name))) - case bs: BaseRelation => - Some((bs.schema.map(_.name), Seq.empty[String])) + SchemaUtils.checkColumnNameDuplication( + hs.dataSchema.map(_.name), + "in the data schema", + equality) + SchemaUtils.checkColumnNameDuplication( + hs.partitionSchema.map(_.name), + "in the partition column(s)", + equality) case _ => - None - }).foreach { case (dataCols, partCols) => - SchemaUtils.checkColumnNameDuplication(dataCols, "in the data schema", equality) - SchemaUtils.checkColumnNameDuplication(partCols, "in the partition column(s)", equality) + SchemaUtils.checkColumnNameDuplication( + relation.schema.map(_.name), + "in the data schema", + equality) } relation diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala index a98cd3902214..c1bcfb861078 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala @@ -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,6 @@ 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 /** From 5ed2c0d68be372304bc8a46046d28d5b7aadcb97 Mon Sep 17 00:00:00 2001 From: Takeshi Yamamuro Date: Fri, 7 Jul 2017 15:56:52 +0900 Subject: [PATCH 27/27] Move some tests to DDLSuite --- .../spark/sql/execution/command/views.scala | 2 +- .../execution/datasources/DataSource.scala | 8 +- .../datasources/PartitioningUtils.scala | 2 +- .../sql/execution/datasources/rules.scala | 6 +- .../resources/sql-tests/inputs/create.sql | 23 ----- .../sql-tests/results/create.sql.out | 99 ------------------- .../sql/execution/command/DDLSuite.scala | 57 +++++++---- .../sql/test/DataFrameReaderWriterSuite.scala | 6 +- 8 files changed, 50 insertions(+), 153 deletions(-) delete mode 100644 sql/core/src/test/resources/sql-tests/inputs/create.sql delete mode 100644 sql/core/src/test/resources/sql-tests/results/create.sql.out diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala index e167dc1b6a34..ffdfd527fa70 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala @@ -361,7 +361,7 @@ object ViewHelper { // Generate the query column names, throw an AnalysisException if there exists duplicate column // names. SchemaUtils.checkColumnNameDuplication( - queryOutput, "the view", session.sessionState.conf.resolver) + queryOutput, "in the view definition", session.sessionState.conf.resolver) // Generate the view default database name. val viewDefaultDatabase = session.sessionState.catalog.getCurrentDatabase diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala index 66ed3c4bc2b2..d36a04f1fff8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala @@ -91,9 +91,9 @@ case class DataSource( bucketSpec.map { bucket => SchemaUtils.checkColumnNameDuplication( - bucket.bucketColumnNames, "in the bucket column(s)", equality) + bucket.bucketColumnNames, "in the bucket definition", equality) SchemaUtils.checkColumnNameDuplication( - bucket.sortColumnNames, "in the sort column(s)", equality) + bucket.sortColumnNames, "in the sort definition", equality) } /** @@ -196,7 +196,7 @@ case class DataSource( try { SchemaUtils.checkColumnNameDuplication( (dataSchema ++ partitionSchema).map(_.name), - "in the data schema and the partition column(s)", + "in the data schema and the partition schema", equality) } catch { case e: AnalysisException => logWarning(e.getMessage) @@ -413,7 +413,7 @@ case class DataSource( equality) SchemaUtils.checkColumnNameDuplication( hs.partitionSchema.map(_.name), - "in the partition column(s)", + "in the partition schema", equality) case _ => SchemaUtils.checkColumnNameDuplication( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala index f8d9038c93ae..92358da6d6c6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala @@ -303,7 +303,7 @@ object PartitioningUtils { } SchemaUtils.checkColumnNameDuplication( - normalizedPartSpec.map(_._1), "in the partition column(s)", resolver) + normalizedPartSpec.map(_._1), "in the partition schema", resolver) normalizedPartSpec.toMap } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala index f263f105489f..41d40aa926fb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala @@ -254,7 +254,7 @@ case class PreprocessTableCreation(sparkSession: SparkSession) extends Rule[Logi SchemaUtils.checkColumnNameDuplication( normalizedPartitionCols, - "in the partition column(s)", + "in the partition schema", sparkSession.sessionState.conf.resolver) if (schema.nonEmpty && normalizedPartitionCols.length == schema.length) { @@ -288,11 +288,11 @@ case class PreprocessTableCreation(sparkSession: SparkSession) extends Rule[Logi SchemaUtils.checkColumnNameDuplication( normalizedBucketSpec.bucketColumnNames, - "in the bucket column(s)", + "in the bucket definition", sparkSession.sessionState.conf.resolver) SchemaUtils.checkColumnNameDuplication( normalizedBucketSpec.sortColumnNames, - "in the sort column(s)", + "in the sort definition", sparkSession.sessionState.conf.resolver) normalizedBucketSpec.sortColumnNames.map(schema(_)).map(_.dataType).foreach { diff --git a/sql/core/src/test/resources/sql-tests/inputs/create.sql b/sql/core/src/test/resources/sql-tests/inputs/create.sql deleted file mode 100644 index 65ff698d5166..000000000000 --- a/sql/core/src/test/resources/sql-tests/inputs/create.sql +++ /dev/null @@ -1,23 +0,0 @@ --- Catch case-sensitive name duplication -SET spark.sql.caseSensitive=true; - -CREATE TABLE t(c0 STRING, c1 INT, c1 DOUBLE, c0 INT) USING parquet; - -CREATE TABLE t(c0 INT) USING parquet PARTITIONED BY (c0, c0); - -CREATE TABLE t(c0 INT) USING parquet CLUSTERED BY (c0, c0) INTO 2 BUCKETS; - -CREATE TABLE t(c0 INT, c1 INT) USING parquet CLUSTERED BY (c0) SORTED BY (c1, c1) INTO 2 BUCKETS; - --- Catch case-insensitive name duplication -SET spark.sql.caseSensitive=false; - -CREATE TABLE t(c0 STRING, c1 INT, c1 DOUBLE, c0 INT) USING parquet; - -CREATE TABLE t(ab STRING, cd INT, ef DOUBLE, Ab INT) USING parquet; - -CREATE TABLE t(ab INT, cd INT) USING parquet PARTITIONED BY (Ab, aB); - -CREATE TABLE t(ab INT) USING parquet CLUSTERED BY (Ab, aB) INTO 2 BUCKETS; - -CREATE TABLE t(ab INT, cd INT) USING parquet CLUSTERED BY (ab) SORTED BY (cD, Cd) INTO 2 BUCKETS; diff --git a/sql/core/src/test/resources/sql-tests/results/create.sql.out b/sql/core/src/test/resources/sql-tests/results/create.sql.out deleted file mode 100644 index 32689bf06ee8..000000000000 --- a/sql/core/src/test/resources/sql-tests/results/create.sql.out +++ /dev/null @@ -1,99 +0,0 @@ --- Automatically generated by SQLQueryTestSuite --- Number of queries: 11 - - --- !query 0 -SET spark.sql.caseSensitive=true --- !query 0 schema -struct --- !query 0 output -spark.sql.caseSensitive true - - --- !query 1 -CREATE TABLE t(c0 STRING, c1 INT, c1 DOUBLE, c0 INT) USING parquet --- !query 1 schema -struct<> --- !query 1 output -org.apache.spark.sql.AnalysisException -Found duplicate column(s) in the table definition of `t`: `c1`, `c0`; - - --- !query 2 -CREATE TABLE t(c0 INT) USING parquet PARTITIONED BY (c0, c0) --- !query 2 schema -struct<> --- !query 2 output -org.apache.spark.sql.AnalysisException -Found duplicate column(s) in the partition column(s): `c0`; - - --- !query 3 -CREATE TABLE t(c0 INT) USING parquet CLUSTERED BY (c0, c0) INTO 2 BUCKETS --- !query 3 schema -struct<> --- !query 3 output -org.apache.spark.sql.AnalysisException -Found duplicate column(s) in the bucket column(s): `c0`; - - --- !query 4 -CREATE TABLE t(c0 INT, c1 INT) USING parquet CLUSTERED BY (c0) SORTED BY (c1, c1) INTO 2 BUCKETS --- !query 4 schema -struct<> --- !query 4 output -org.apache.spark.sql.AnalysisException -Found duplicate column(s) in the sort column(s): `c1`; - - --- !query 5 -SET spark.sql.caseSensitive=false --- !query 5 schema -struct --- !query 5 output -spark.sql.caseSensitive false - - --- !query 6 -CREATE TABLE t(c0 STRING, c1 INT, c1 DOUBLE, c0 INT) USING parquet --- !query 6 schema -struct<> --- !query 6 output -org.apache.spark.sql.AnalysisException -Found duplicate column(s) in the table definition of `t`: `c1`, `c0`; - - --- !query 7 -CREATE TABLE t(ab STRING, cd INT, ef DOUBLE, Ab INT) USING parquet --- !query 7 schema -struct<> --- !query 7 output -org.apache.spark.sql.AnalysisException -Found duplicate column(s) in the table definition of `t`: `ab`; - - --- !query 8 -CREATE TABLE t(ab INT, cd INT) USING parquet PARTITIONED BY (Ab, aB) --- !query 8 schema -struct<> --- !query 8 output -org.apache.spark.sql.AnalysisException -Found duplicate column(s) in the partition column(s): `ab`; - - --- !query 9 -CREATE TABLE t(ab INT) USING parquet CLUSTERED BY (Ab, aB) INTO 2 BUCKETS --- !query 9 schema -struct<> --- !query 9 output -org.apache.spark.sql.AnalysisException -Found duplicate column(s) in the bucket column(s): `ab`; - - --- !query 10 -CREATE TABLE t(ab INT, cd INT) USING parquet CLUSTERED BY (ab) SORTED BY (cD, Cd) INTO 2 BUCKETS --- !query 10 schema -struct<> --- !query 10 output -org.apache.spark.sql.AnalysisException -Found duplicate column(s) in the sort column(s): `cd`; diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala index 079f3334b3f0..5c0a6aa724bf 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala @@ -436,16 +436,13 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils { } test("create table - duplicate column names in the table definition") { - val e = intercept[AnalysisException] { - sql("CREATE TABLE tbl(a int, a string) USING json") - } - assert(e.message == "Found duplicate column(s) in the table definition of `tbl`: `a`") - - withSQLConf(SQLConf.CASE_SENSITIVE.key -> "false") { - val e2 = intercept[AnalysisException] { - sql("CREATE TABLE tbl(a int, A string) USING json") + Seq((true, ("a", "a")), (false, ("aA", "Aa"))).foreach { case (caseSensitive, (c0, c1)) => + withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive.toString) { + val errMsg = intercept[AnalysisException] { + sql(s"CREATE TABLE t($c0 INT, $c1 INT) USING parquet") + }.getMessage + assert(errMsg.contains("Found duplicate column(s) in the table definition of `t`")) } - assert(e2.message == "Found duplicate column(s) in the table definition of `tbl`: `a`") } } @@ -466,22 +463,33 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils { } test("create table - column repeated in partition columns") { - val e = intercept[AnalysisException] { - sql("CREATE TABLE tbl(a int) USING json PARTITIONED BY (a, a)") + Seq((true, ("a", "a")), (false, ("aA", "Aa"))).foreach { case (caseSensitive, (c0, c1)) => + withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive.toString) { + val errMsg = intercept[AnalysisException] { + sql(s"CREATE TABLE t($c0 INT) USING parquet PARTITIONED BY ($c0, $c1)") + }.getMessage + assert(errMsg.contains("Found duplicate column(s) in the partition schema")) + } } - assert(e.message == "Found duplicate column(s) in the partition column(s): `a`") } test("create table - column repeated in bucket/sort columns") { - var e = intercept[AnalysisException] { - sql("CREATE TABLE tbl(a int) USING json CLUSTERED BY (a, a) INTO 4 BUCKETS") - } - assert(e.message == "Found duplicate column(s) in the bucket column(s): `a`") + Seq((true, ("a", "a")), (false, ("aA", "Aa"))).foreach { case (caseSensitive, (c0, c1)) => + withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive.toString) { + var errMsg = intercept[AnalysisException] { + sql(s"CREATE TABLE t($c0 INT) USING parquet CLUSTERED BY ($c0, $c1) INTO 2 BUCKETS") + }.getMessage + assert(errMsg.contains("Found duplicate column(s) in the bucket definition")) - e = intercept[AnalysisException] { - sql("CREATE TABLE tbl(a int) USING json CLUSTERED BY (a) SORTED BY (a, a) INTO 4 BUCKETS") + errMsg = intercept[AnalysisException] { + sql(s""" + |CREATE TABLE t($c0 INT, col INT) USING parquet CLUSTERED BY (col) + | SORTED BY ($c0, $c1) INTO 2 BUCKETS + """.stripMargin) + }.getMessage + assert(errMsg.contains("Found duplicate column(s) in the sort definition")) + } } - assert(e.message == "Found duplicate column(s) in the sort column(s): `a`") } test("Refresh table after changing the data source table partitioning") { @@ -533,6 +541,17 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils { } } + test("create view - duplicate column names in the view definition") { + Seq((true, ("a", "a")), (false, ("aA", "Aa"))).foreach { case (caseSensitive, (c0, c1)) => + withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive.toString) { + val errMsg = intercept[AnalysisException] { + sql(s"CREATE VIEW t AS SELECT * FROM VALUES (1, 1) AS t($c0, $c1)") + }.getMessage + assert(errMsg.contains("Found duplicate column(s) in the view definition")) + } + } + } + test("Alter/Describe Database") { val catalog = spark.sessionState.catalog val databaseNames = Seq("db1", "`database`") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala index b1fd49b98499..569bac156b53 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala @@ -695,12 +695,12 @@ class DataFrameReaderWriterSuite extends QueryTest with SharedSQLContext with Be var errorMsg = intercept[AnalysisException] { Seq((1, 1)).toDF("col", c0).write.bucketBy(2, c0, c1).saveAsTable("t") }.getMessage - assert(errorMsg.contains("Found duplicate column(s) in the bucket column(s)")) + assert(errorMsg.contains("Found duplicate column(s) in the bucket definition")) errorMsg = intercept[AnalysisException] { Seq((1, 1)).toDF("col", c0).write.bucketBy(2, "col").sortBy(c0, c1).saveAsTable("t") }.getMessage - assert(errorMsg.contains("Found duplicate column(s) in the sort column(s)")) + assert(errorMsg.contains("Found duplicate column(s) in the sort definition")) } } } @@ -734,7 +734,7 @@ class DataFrameReaderWriterSuite extends QueryTest with SharedSQLContext with Be val errorMsg = intercept[AnalysisException] { spark.read.format(format).load(testDir.getAbsolutePath) }.getMessage - assert(errorMsg.contains("Found duplicate column(s) in the partition column(s):")) + assert(errorMsg.contains("Found duplicate column(s) in the partition schema:")) } Seq((true, ("a", "a")), (false, ("aA", "Aa"))).foreach { case (caseSensitive, (c0, c1)) =>