[GitHub] spark pull request #17758: [SPARK-20460][SQL] Make it more consistent to han...

2017-07-10 Thread asfgit
Github user asfgit closed the pull request at:

https://github.com/apache/spark/pull/17758


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17758: [SPARK-20460][SQL] Make it more consistent to han...

2017-07-10 Thread maropu
Github user maropu commented on a diff in the pull request:

https://github.com/apache/spark/pull/17758#discussion_r126354324
  
--- Diff: 
sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala
 ---
@@ -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] {
--- End diff --

If we have streaming metadata logs, we have [the different code 
path](https://github.com/maropu/spark/blob/5ed2c0d68be372304bc8a46046d28d5b7aadcb97/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala#L332)
 from non-streaming cases in `Dataset`. So, the original motivation of this 
tests is to check the duplication even in this path.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17758: [SPARK-20460][SQL] Make it more consistent to han...

2017-07-10 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/17758#discussion_r126352411
  
--- Diff: 
sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala
 ---
@@ -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] {
--- End diff --

why is this test related to streaming?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17758: [SPARK-20460][SQL] Make it more consistent to han...

2017-07-07 Thread maropu
Github user maropu commented on a diff in the pull request:

https://github.com/apache/spark/pull/17758#discussion_r126083104
  
--- Diff: sql/core/src/test/resources/sql-tests/inputs/create.sql ---
@@ -0,0 +1,23 @@
+-- Catch case-sensitive name duplication
+SET spark.sql.caseSensitive=true;
+
+CREATE TABLE t(c0 STRING, c1 INT, c1 DOUBLE, c0 INT) USING parquet;
--- End diff --

I moved some tests to `DDLSuite` and removed this file.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17758: [SPARK-20460][SQL] Make it more consistent to han...

2017-07-06 Thread maropu
Github user maropu commented on a diff in the pull request:

https://github.com/apache/spark/pull/17758#discussion_r126074994
  
--- Diff: sql/core/src/test/resources/sql-tests/inputs/create.sql ---
@@ -0,0 +1,23 @@
+-- Catch case-sensitive name duplication
+SET spark.sql.caseSensitive=true;
+
+CREATE TABLE t(c0 STRING, c1 INT, c1 DOUBLE, c0 INT) USING parquet;
--- End diff --

ok, will update


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17758: [SPARK-20460][SQL] Make it more consistent to han...

2017-07-06 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/17758#discussion_r126074450
  
--- Diff: sql/core/src/test/resources/sql-tests/inputs/create.sql ---
@@ -0,0 +1,23 @@
+-- Catch case-sensitive name duplication
+SET spark.sql.caseSensitive=true;
+
+CREATE TABLE t(c0 STRING, c1 INT, c1 DOUBLE, c0 INT) USING parquet;
--- End diff --

We should keep them in one place. For now I think we still need to put them 
in `DDLSuite` because we need to run it with and without hive support. Can we 
pick some typical test cases here and move them to `DDLSuite`?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17758: [SPARK-20460][SQL] Make it more consistent to han...

2017-07-06 Thread maropu
Github user maropu commented on a diff in the pull request:

https://github.com/apache/spark/pull/17758#discussion_r126074238
  
--- Diff: sql/core/src/test/resources/sql-tests/inputs/create.sql ---
@@ -0,0 +1,23 @@
+-- Catch case-sensitive name duplication
+SET spark.sql.caseSensitive=true;
+
+CREATE TABLE t(c0 STRING, c1 INT, c1 DOUBLE, c0 INT) USING parquet;
--- End diff --

In `DDLSuite`, we already have simple tests for duplicate columns. we 
better moving these tests there?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17758: [SPARK-20460][SQL] Make it more consistent to han...

2017-07-06 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/17758#discussion_r126073404
  
--- Diff: sql/core/src/test/resources/sql-tests/inputs/create.sql ---
@@ -0,0 +1,23 @@
+-- Catch case-sensitive name duplication
+SET spark.sql.caseSensitive=true;
+
+CREATE TABLE t(c0 STRING, c1 INT, c1 DOUBLE, c0 INT) USING parquet;
--- End diff --

We didn't have test cases for create table before?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17758: [SPARK-20460][SQL] Make it more consistent to han...

2017-07-06 Thread maropu
Github user maropu commented on a diff in the pull request:

https://github.com/apache/spark/pull/17758#discussion_r125891743
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala
 ---
@@ -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)
--- End diff --

ok


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17758: [SPARK-20460][SQL] Make it more consistent to han...

2017-07-06 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/17758#discussion_r125889801
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala
 ---
@@ -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)
--- End diff --

can we just inline these 2 checks in the above 2 cases?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17758: [SPARK-20460][SQL] Make it more consistent to han...

2017-07-06 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/17758#discussion_r125889675
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala
 ---
@@ -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 _ =>
--- End diff --

this case is not needed, `relation` is always a `BaseRelation`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17758: [SPARK-20460][SQL] Make it more consistent to han...

2017-07-04 Thread maropu
Github user maropu commented on a diff in the pull request:

https://github.com/apache/spark/pull/17758#discussion_r125564439
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala
 ---
@@ -304,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
--- End diff --

ok, I'll check we can do so.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17758: [SPARK-20460][SQL] Make it more consistent to han...

2017-07-04 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/17758#discussion_r125564070
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala
 ---
@@ -304,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
--- End diff --

an idea to unify the validation for both user-specified schema and inferred 
schema: we can put the validation at the end of `resolveRelation`:
```
relation match {
  case h: HadoopFsRelation =>
// validate data schema
// validate partition schema
// validate bucket
  case _ => // validate data schema
}
relation
```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17758: [SPARK-20460][SQL] Make it more consistent to han...

2017-07-04 Thread maropu
Github user maropu commented on a diff in the pull request:

https://github.com/apache/spark/pull/17758#discussion_r125498171
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala
 ---
@@ -468,7 +489,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 =>
--- End diff --

ok, I'll add the check there as you suggested. Thanks!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17758: [SPARK-20460][SQL] Make it more consistent to han...

2017-07-04 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/17758#discussion_r125489566
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala
 ---
@@ -468,7 +489,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 =>
--- End diff --

ok this is a special case, let's leave the checking logic in `JDBCUtils`. 
But I think my proposal is still valid for external data source which can infer 
schema.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17758: [SPARK-20460][SQL] Make it more consistent to han...

2017-07-04 Thread maropu
Github user maropu commented on a diff in the pull request:

https://github.com/apache/spark/pull/17758#discussion_r125483095
  
--- Diff: 
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 {
--- End diff --

As explained above, I couldn't catch the name duplication for the JDBC 
case. So, I added this interface so as to catch each datasource-specific name 
duplication and other schema invalidation. But, I felt I went too far for the 
corner case...


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17758: [SPARK-20460][SQL] Make it more consistent to han...

2017-07-04 Thread maropu
Github user maropu commented on a diff in the pull request:

https://github.com/apache/spark/pull/17758#discussion_r125482229
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala
 ---
@@ -468,7 +489,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 =>
--- End diff --

I tried though, I couldn't because the check at this point couldn't pass 
[this `JDBCWrite` 
test](https://github.com/apache/spark/pull/17758/files#diff-4e7d97319461efb33f06b6d245254dcdL477).
 This test catches name duplication specified by a JDBC option 
`createTableColumnTypes`. But, if we move [this 
check](https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala#L744)
 in `JdbcUtils` into here, [the following 
logic](https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala#L764)
 eats the duplication and `r.schema` above does not have the duplication (so, I 
think this is a corner case though, IIUC we couldn't catch at this point for 
JDBC relations)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17758: [SPARK-20460][SQL] Make it more consistent to han...

2017-07-03 Thread maropu
Github user maropu commented on a diff in the pull request:

https://github.com/apache/spark/pull/17758#discussion_r125400282
  
--- Diff: 
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)
 
--- End diff --

ok, I' add test, too.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17758: [SPARK-20460][SQL] Make it more consistent to han...

2017-07-03 Thread maropu
Github user maropu commented on a diff in the pull request:

https://github.com/apache/spark/pull/17758#discussion_r125400239
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala
 ---
@@ -334,6 +352,9 @@ case class DataSource(
 "It must be specified manually")
 }
 
+SchemaUtils.checkSchemaColumnNameDuplication(
--- End diff --

yea, I'll remove (I wrongly reverted this entry, too...)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17758: [SPARK-20460][SQL] Make it more consistent to han...

2017-07-03 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/17758#discussion_r125396517
  
--- Diff: 
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 {
--- End diff --

why we have this?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17758: [SPARK-20460][SQL] Make it more consistent to han...

2017-07-03 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/17758#discussion_r125396465
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala
 ---
@@ -468,7 +489,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 =>
--- End diff --

can we do the check at 
https://github.com/apache/spark/pull/17758/files#diff-7a6cb188d2ae31eb3347b5629a679cecR325
 ?

Basically
```
val r = dataSource.createRelation(sparkSession.sqlContext, 
caseInsensitiveOptions)
SchemaUtils.checkSchemaColumnNameDuplication(r.schema)
r
```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17758: [SPARK-20460][SQL] Make it more consistent to han...

2017-07-03 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/17758#discussion_r125395801
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala
 ---
@@ -334,6 +352,9 @@ case class DataSource(
 "It must be specified manually")
 }
 
+SchemaUtils.checkSchemaColumnNameDuplication(
--- End diff --

what are we checking here? both user-specified schema and inferred schema 
should have been checked


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17758: [SPARK-20460][SQL] Make it more consistent to han...

2017-07-03 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/17758#discussion_r125395258
  
--- Diff: 
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)
 
--- End diff --

yea good idea


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17758: [SPARK-20460][SQL] Make it more consistent to han...

2017-07-03 Thread maropu
Github user maropu commented on a diff in the pull request:

https://github.com/apache/spark/pull/17758#discussion_r125270419
  
--- Diff: 
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)
 
--- End diff --

@cloud-fan We need to check the duplication for `bucketSpec` here? The 
current version also could catch the case in `bucketSpec` like;
```
scala> Seq((1, 2)).toDF("a", "b").write.bucketBy(3, "a", 
"a").saveAsTable("t")
org.apache.spark.sql.AnalysisException: Found duplicate column(s) in the 
bucket: `a`;
  at 
org.apache.spark.sql.util.SchemaUtils$.checkColumnNameDuplication(SchemaUtils.scala:85)
  at 
org.apache.spark.sql.util.SchemaUtils$.checkColumnNameDuplication(SchemaUtils.scala:67)
  at 
org.apache.spark.sql.execution.datasources.PreprocessTableCreation.normalizeBucketSpec(rules.scala:289)
  at 
org.apache.spark.sql.execution.datasources.PreprocessTableCreation.org$apache$spark$sql$execution$datasources$PreprocessTableCreation$$normalizeCatalogTable(rules.scala:232)
  at 
org.apache.spark.sql.execution.datasources.PreprocessTableCreation$$anonfun$apply$2.applyOrElse(rules.scala:197)
  at 
org.apache.spark.sql.execution.datasources.PreprocessTableCreation$$anonfun$apply$2.applyOrElse(rules.scala:76)
```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17758: [SPARK-20460][SQL] Make it more consistent to han...

2017-07-03 Thread maropu
Github user maropu commented on a diff in the pull request:

https://github.com/apache/spark/pull/17758#discussion_r125267663
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala
 ---
@@ -468,7 +489,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 =>
--- End diff --

> In other cases, we need to ad-hoc check the duplication (e.g., 
JDBCRelation)

@cloud-fan How about this? Since we couldn't pass `df.schema` into the 
check in `resolveRelation` you suggested, so I put the check here for write 
paths. Actually, IMHO we need not have this datasource-specific check for read 
paths because each datasource implementation should provide a valid schema when 
inferring it in `FileFormat.inferSchema`, `JdbcUtils.getSchema`, ...

On the other hand, in write paths, I feel other datasource-specific checks 
would be better to be done in `DataSource`. For example;
```
scala> spark.range(1).selectExpr("rand()").write.save("path")
org.apache.spark.sql.AnalysisException: Attribute name 
"rand(1595701563628455153)" contains invalid character(s) among " ,;{}()\n\t=". 
Please use alias to rename it.;
  at 
org.apache.spark.sql.execution.datasources.parquet.ParquetSchemaConverter$.checkConversionRequirement(ParquetSchemaConverter.scala:581)
  at 
org.apache.spark.sql.execution.datasources.parquet.ParquetSchemaConverter$.checkFieldName(ParquetSchemaConverter.scala:567)
  at 
org.apache.spark.sql.execution.datasources.parquet.ParquetWriteSupport$$anonfun$setSchema$2.apply(ParquetWriteSupport.scala:446)
  at 
org.apache.spark.sql.execution.datasources.parquet.ParquetWriteSupport$$anonfun$setSchema$2.apply(ParquetWriteSupport.scala:446)
  at scala.collection.immutable.List.foreach(List.scala:381)
  at 
org.apache.spark.sql.execution.datasources.parquet.ParquetWriteSupport$.setSchema(ParquetWriteSupport.scala:446)
  at 
org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat.prepareWrite(ParquetFileFormat.scala:112)
  at 
org.apache.spark.sql.execution.datasources.FileFormatWriter$.write(FileFormatWriter.scala:134)
```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17758: [SPARK-20460][SQL] Make it more consistent to han...

2017-06-20 Thread maropu
Github user maropu commented on a diff in the pull request:

https://github.com/apache/spark/pull/17758#discussion_r122898892
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala
 ---
@@ -182,6 +183,10 @@ case class DataSource(
   throw new AnalysisException(
 s"Unable to infer schema for $format. It must be specified 
manually.")
 }
+
+SchemaUtils.checkSchemaColumnNameDuplication(
+  dataSchema, "the datasource", 
sparkSession.sessionState.conf.caseSensitiveAnalysis)
--- End diff --

Fixed code and added tests: 
https://github.com/apache/spark/pull/17758/commits/f9c65ccc546f49cf4ff7797d285f803516f24303


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17758: [SPARK-20460][SQL] Make it more consistent to han...

2017-06-19 Thread maropu
Github user maropu commented on a diff in the pull request:

https://github.com/apache/spark/pull/17758#discussion_r122868692
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala
 ---
@@ -62,13 +63,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, "the query", 
sparkSession.sessionState.conf.caseSensitiveAnalysis)
--- End diff --

`df.write.save` also uses this path though, it's okay for the name? 
https://github.com/apache/spark/pull/17758/files#diff-5d2ebf4e9ca5a990136b276859769289R1126


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17758: [SPARK-20460][SQL] Make it more consistent to han...

2017-06-19 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/17758#discussion_r122867659
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala
 ---
@@ -62,13 +63,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, "the query", 
sparkSession.sessionState.conf.caseSensitiveAnalysis)
--- End diff --

`the query` -> `inserting into $outputPath`?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17758: [SPARK-20460][SQL] Make it more consistent to han...

2017-06-19 Thread maropu
Github user maropu commented on a diff in the pull request:

https://github.com/apache/spark/pull/17758#discussion_r122867252
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala
 ---
@@ -328,6 +333,9 @@ case class DataSource(
 "It must be specified manually")
 }
 
+SchemaUtils.checkSchemaColumnNameDuplication(
+  dataSchema, "the datasource", 
sparkSession.sessionState.conf.caseSensitiveAnalysis)
--- End diff --

Ah, good catch. I'll try to add tests for that case.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17758: [SPARK-20460][SQL] Make it more consistent to han...

2017-06-19 Thread maropu
Github user maropu commented on a diff in the pull request:

https://github.com/apache/spark/pull/17758#discussion_r122866890
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/util/SchemaUtils.scala ---
@@ -0,0 +1,74 @@
+/*
+ * 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 scala.collection.mutable
+
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.analysis._
+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 {
+
+  /**
+   * 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) {
+  caseSensitiveResolution
+} else {
+  caseInsensitiveResolution
+}
+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(
+  colNames: Seq[String], colType: String, resolver: Resolver): Unit = {
+val duplicateColumns = mutable.ArrayBuffer[String]()
+colNames.foreach { name =>
+  val sameColNames = colNames.filter(resolver(_, name))
--- End diff --

ok, I'll re-check.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17758: [SPARK-20460][SQL] Make it more consistent to han...

2017-06-19 Thread maropu
Github user maropu commented on a diff in the pull request:

https://github.com/apache/spark/pull/17758#discussion_r122866830
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala ---
@@ -355,12 +356,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, "the view", 
session.sessionState.conf.caseSensitiveAnalysis)
--- End diff --

ok


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17758: [SPARK-20460][SQL] Make it more consistent to han...

2017-06-19 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/17758#discussion_r122866332
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala
 ---
@@ -328,6 +333,9 @@ case class DataSource(
 "It must be specified manually")
 }
 
+SchemaUtils.checkSchemaColumnNameDuplication(
+  dataSchema, "the datasource", 
sparkSession.sessionState.conf.caseSensitiveAnalysis)
--- End diff --

is it possible to write out duplicated columns json files by other system?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17758: [SPARK-20460][SQL] Make it more consistent to han...

2017-06-19 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/17758#discussion_r122865863
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala ---
@@ -355,12 +356,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, "the view", 
session.sessionState.conf.caseSensitiveAnalysis)
--- End diff --

use `queryOutput` instead of `analyzedPlan.schema`?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17758: [SPARK-20460][SQL] Make it more consistent to han...

2017-06-19 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/17758#discussion_r122865721
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/util/SchemaUtils.scala ---
@@ -0,0 +1,74 @@
+/*
+ * 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 scala.collection.mutable
+
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.analysis._
+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 {
+
+  /**
+   * 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) {
+  caseSensitiveResolution
+} else {
+  caseInsensitiveResolution
+}
+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(
+  colNames: Seq[String], colType: String, resolver: Resolver): Unit = {
+val duplicateColumns = mutable.ArrayBuffer[String]()
+colNames.foreach { name =>
+  val sameColNames = colNames.filter(resolver(_, name))
--- End diff --

the time complexity here is O(n^2), shall we follow the [previous 
approach](https://github.com/apache/spark/pull/17758/files#diff-b3f9800839b9b9a1df9da9cbfc01adf8L191)
 and make it O(n)? 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17758: [SPARK-20460][SQL] Make it more consistent to han...

2017-06-19 Thread maropu
Github user maropu commented on a diff in the pull request:

https://github.com/apache/spark/pull/17758#discussion_r122863353
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala
 ---
@@ -328,6 +333,9 @@ case class DataSource(
 "It must be specified manually")
 }
 
+SchemaUtils.checkSchemaColumnNameDuplication(
+  dataSchema, "the datasource", 
sparkSession.sessionState.conf.caseSensitiveAnalysis)
--- End diff --

I though firstly I added this check for `userSpecifiedSchema`: 
https://github.com/apache/spark/pull/17758/files#diff-7a6cb188d2ae31eb3347b5629a679cecR325.
 But, does we need to get a schema from `userSpecifiedSchema` here? As you 
suggested, in this code path, ISTM it's okay to get a schema only from the 
metadata log. I checked existing test suites though, I couldn't find tests for 
using `userSpecifiedSchema` in this code path.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17758: [SPARK-20460][SQL] Make it more consistent to han...

2017-06-19 Thread maropu
Github user maropu commented on a diff in the pull request:

https://github.com/apache/spark/pull/17758#discussion_r122862830
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala ---
@@ -248,6 +249,10 @@ private[hive] class HiveMetastoreCatalog(sparkSession: 
SparkSession) extends Log
   if (inferenceMode == INFER_AND_SAVE) {
 updateCatalogSchema(relation.tableMeta.identifier, schema)
   }
+
+  SchemaUtils.checkSchemaColumnNameDuplication(
--- End diff --

I checked code again and it seems I found no column duplication happen here 
if the hive catalog has no name duplication. `mergeWithMetastoreSchema` merges 
a catalog schema with an inferred one on the catalog names, so IIUC this does 
not lead to the duplication. So, we could safely remove this check?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17758: [SPARK-20460][SQL] Make it more consistent to han...

2017-06-19 Thread maropu
Github user maropu commented on a diff in the pull request:

https://github.com/apache/spark/pull/17758#discussion_r122862386
  
--- Diff: 
sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala
 ---
@@ -687,4 +688,52 @@ 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 =>
+  // 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) =>
--- End diff --

Fixed


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17758: [SPARK-20460][SQL] Make it more consistent to han...

2017-06-19 Thread maropu
Github user maropu commented on a diff in the pull request:

https://github.com/apache/spark/pull/17758#discussion_r122853060
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala
 ---
@@ -182,6 +183,10 @@ case class DataSource(
   throw new AnalysisException(
 s"Unable to infer schema for $format. It must be specified 
manually.")
 }
+
+SchemaUtils.checkSchemaColumnNameDuplication(
+  dataSchema, "the datasource", 
sparkSession.sessionState.conf.caseSensitiveAnalysis)
--- End diff --

ok, I'll do that first.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17758: [SPARK-20460][SQL] Make it more consistent to han...

2017-06-19 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/17758#discussion_r122831948
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala
 ---
@@ -182,6 +183,10 @@ case class DataSource(
   throw new AnalysisException(
 s"Unable to infer schema for $format. It must be specified 
manually.")
 }
+
+SchemaUtils.checkSchemaColumnNameDuplication(
+  dataSchema, "the datasource", 
sparkSession.sessionState.conf.caseSensitiveAnalysis)
--- End diff --

@maropu Could you submit a new PR to 2.2 now? We should fix this issue in 
2.2 now. 

Just opened the JIRA: https://issues.apache.org/jira/browse/SPARK-21144 



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17758: [SPARK-20460][SQL] Make it more consistent to han...

2017-06-18 Thread maropu
Github user maropu commented on a diff in the pull request:

https://github.com/apache/spark/pull/17758#discussion_r122610979
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala
 ---
@@ -182,6 +183,10 @@ case class DataSource(
   throw new AnalysisException(
 s"Unable to infer schema for $format. It must be specified 
manually.")
 }
+
+SchemaUtils.checkSchemaColumnNameDuplication(
+  dataSchema, "the datasource", 
sparkSession.sessionState.conf.caseSensitiveAnalysis)
--- End diff --

ok


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17758: [SPARK-20460][SQL] Make it more consistent to han...

2017-06-18 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/17758#discussion_r122596253
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala ---
@@ -248,6 +249,10 @@ private[hive] class HiveMetastoreCatalog(sparkSession: 
SparkSession) extends Log
   if (inferenceMode == INFER_AND_SAVE) {
 updateCatalogSchema(relation.tableMeta.identifier, schema)
   }
+
+  SchemaUtils.checkSchemaColumnNameDuplication(
--- End diff --

Do we have a test case to cover this check?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17758: [SPARK-20460][SQL] Make it more consistent to han...

2017-06-18 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/17758#discussion_r122595919
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala
 ---
@@ -328,6 +333,9 @@ case class DataSource(
 "It must be specified manually")
 }
 
+SchemaUtils.checkSchemaColumnNameDuplication(
+  dataSchema, "the datasource", 
sparkSession.sessionState.conf.caseSensitiveAnalysis)
--- End diff --

This branch is to load files from the metadata log. IMO, this check is not 
needed, unless you can reproduce the issue. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17758: [SPARK-20460][SQL] Make it more consistent to han...

2017-06-18 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/17758#discussion_r122595783
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala
 ---
@@ -182,6 +183,10 @@ case class DataSource(
   throw new AnalysisException(
 s"Unable to infer schema for $format. It must be specified 
manually.")
 }
+
+SchemaUtils.checkSchemaColumnNameDuplication(
+  dataSchema, "the datasource", 
sparkSession.sessionState.conf.caseSensitiveAnalysis)
--- End diff --

`dataSchema + partitionSchema` is our full schema. When checking 
duplication, we need to check the full schema together. 

Could you also add such a test case? 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17758: [SPARK-20460][SQL] Make it more consistent to han...

2017-06-18 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/17758#discussion_r122595701
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/util/SchemaUtils.scala ---
@@ -0,0 +1,74 @@
+/*
+ * 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 scala.collection.mutable
+
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.analysis._
+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 {
+
+  /**
+   * 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) {
+  caseSensitiveResolution
+} else {
+  caseInsensitiveResolution
+}
+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(
+  colNames: Seq[String], colType: String, resolver: Resolver): Unit = {
+val duplicateColumns = mutable.ArrayBuffer[String]()
+colNames.foreach { name =>
+  val sameColNames = colNames.filter(resolver(_, name))
--- End diff --

uh, I see.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17758: [SPARK-20460][SQL] Make it more consistent to han...

2017-06-18 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/17758#discussion_r122595287
  
--- Diff: 
sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala
 ---
@@ -687,4 +688,52 @@ 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 =>
+  // 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) =>
--- End diff --

`map` -> `foreach`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17758: [SPARK-20460][SQL] Make it more consistent to han...

2017-06-18 Thread maropu
Github user maropu commented on a diff in the pull request:

https://github.com/apache/spark/pull/17758#discussion_r122584251
  
--- Diff: 
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))
--- End diff --

I think we do not need to check the duplication in each datasource because 
it will be checked in `DataSource`. 
https://github.com/apache/spark/pull/17758/files#diff-7a6cb188d2ae31eb3347b5629a679cecR187


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17758: [SPARK-20460][SQL] Make it more consistent to han...

2017-06-17 Thread wzhfy
Github user wzhfy commented on a diff in the pull request:

https://github.com/apache/spark/pull/17758#discussion_r122582071
  
--- Diff: 
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))
--- End diff --

don't need to check duplication here?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17758: [SPARK-20460][SQL] Make it more consistent to han...

2017-06-16 Thread maropu
Github user maropu commented on a diff in the pull request:

https://github.com/apache/spark/pull/17758#discussion_r122558846
  
--- Diff: 
sql/catalyst/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.catalyst.analysis._
+import org.apache.spark.sql.types.StructType
+
+class SchemaUtilsSuite extends SparkFunSuite {
+
+  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)
+}.getMessage
+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`;"""))
+
+// Check no exception thrown
+SchemaUtils.checkSchemaColumnNameDuplication(
+  StructType.fromDDL("a INT, b INT, c INT"), "SchemaUtilsSuite", 
caseSensitiveAnalysis = true)
+SchemaUtils.checkColumnNameDuplication(
+  "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", 
caseSensitiveResolution)
+  }
+
+  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`;"""))
+val msg4 = intercept[AnalysisException] {
+  SchemaUtils.checkColumnNameDuplication(
+"Aa" :: "b" :: "Aa" :: Nil, "SchemaUtilsSuite", 
caseInsensitiveResolution)
--- End diff --

Fixed. Also, I added more tests here.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17758: [SPARK-20460][SQL] Make it more consistent to han...

2017-06-16 Thread maropu
Github user maropu commented on a diff in the pull request:

https://github.com/apache/spark/pull/17758#discussion_r122558842
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala
 ---
@@ -62,13 +63,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)
--- End diff --

Checked and I think we still need this because this duplication check is 
tested here: 
https://github.com/apache/spark/pull/17758/files#diff-5d2ebf4e9ca5a990136b276859769289R1126


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17758: [SPARK-20460][SQL] Make it more consistent to han...

2017-06-16 Thread maropu
Github user maropu commented on a diff in the pull request:

https://github.com/apache/spark/pull/17758#discussion_r122557795
  
--- Diff: 
sql/catalyst/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.catalyst.analysis._
+import org.apache.spark.sql.types.StructType
+
+class SchemaUtilsSuite extends SparkFunSuite {
+
+  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)
+}.getMessage
+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`;"""))
+
+// Check no exception thrown
+SchemaUtils.checkSchemaColumnNameDuplication(
+  StructType.fromDDL("a INT, b INT, c INT"), "SchemaUtilsSuite", 
caseSensitiveAnalysis = true)
+SchemaUtils.checkColumnNameDuplication(
+  "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", 
caseSensitiveResolution)
+  }
+
+  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`;"""))
+val msg4 = intercept[AnalysisException] {
+  SchemaUtils.checkColumnNameDuplication(
+"Aa" :: "b" :: "Aa" :: Nil, "SchemaUtilsSuite", 
caseInsensitiveResolution)
--- End diff --

ok


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17758: [SPARK-20460][SQL] Make it more consistent to han...

2017-06-16 Thread maropu
Github user maropu commented on a diff in the pull request:

https://github.com/apache/spark/pull/17758#discussion_r122557773
  
--- Diff: 
sql/catalyst/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.catalyst.analysis._
+import org.apache.spark.sql.types.StructType
+
+class SchemaUtilsSuite extends SparkFunSuite {
+
+  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)
+}.getMessage
+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`;"""))
+
+// Check no exception thrown
+SchemaUtils.checkSchemaColumnNameDuplication(
+  StructType.fromDDL("a INT, b INT, c INT"), "SchemaUtilsSuite", 
caseSensitiveAnalysis = true)
+SchemaUtils.checkColumnNameDuplication(
+  "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", 
caseSensitiveResolution)
+  }
+
+  test("Check column name duplication in case-insensitive cases") {
+val msg3 = intercept[AnalysisException] {
--- End diff --

ok


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17758: [SPARK-20460][SQL] Make it more consistent to han...

2017-06-16 Thread maropu
Github user maropu commented on a diff in the pull request:

https://github.com/apache/spark/pull/17758#discussion_r122557676
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala ---
@@ -355,12 +356,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)
--- End diff --

ok


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17758: [SPARK-20460][SQL] Make it more consistent to han...

2017-06-16 Thread maropu
Github user maropu commented on a diff in the pull request:

https://github.com/apache/spark/pull/17758#discussion_r122557657
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/util/SchemaUtils.scala ---
@@ -0,0 +1,74 @@
+/*
+ * 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 scala.collection.mutable
+
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.analysis._
+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 {
+
+  /**
+   * 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) {
+  caseSensitiveResolution
+} else {
+  caseInsensitiveResolution
+}
+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(
+  colNames: Seq[String], colType: String, resolver: Resolver): Unit = {
+val duplicateColumns = mutable.ArrayBuffer[String]()
+colNames.foreach { name =>
+  val sameColNames = colNames.filter(resolver(_, name))
--- End diff --

This filter always returns more than one entry (cuz one entry is always 
matched with oneself), so I used `filter` here. I missed you suggestion?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17758: [SPARK-20460][SQL] Make it more consistent to han...

2017-06-16 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/17758#discussion_r122555690
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala
 ---
@@ -182,6 +183,10 @@ case class DataSource(
   throw new AnalysisException(
 s"Unable to infer schema for $format. It must be specified 
manually.")
 }
+
+SchemaUtils.checkSchemaColumnNameDuplication(
+  dataSchema, "datasource", 
sparkSession.sessionState.conf.caseSensitiveAnalysis)
--- End diff --

IIRC we can not persistent data with duplicated column names in schema, so 
should be fine


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17758: [SPARK-20460][SQL] Make it more consistent to han...

2017-06-16 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/17758#discussion_r122539610
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala
 ---
@@ -62,13 +63,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)
--- End diff --

Could you check whether we still need this check? 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17758: [SPARK-20460][SQL] Make it more consistent to han...

2017-06-16 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/17758#discussion_r122538611
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala
 ---
@@ -328,6 +333,9 @@ case class DataSource(
 "It must be specified manually")
 }
 
+SchemaUtils.checkSchemaColumnNameDuplication(
+  dataSchema, "datasource", 
sparkSession.sessionState.conf.caseSensitiveAnalysis)
--- End diff --

The same here


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17758: [SPARK-20460][SQL] Make it more consistent to han...

2017-06-16 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/17758#discussion_r122538571
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala
 ---
@@ -182,6 +183,10 @@ case class DataSource(
   throw new AnalysisException(
 s"Unable to infer schema for $format. It must be specified 
manually.")
 }
+
+SchemaUtils.checkSchemaColumnNameDuplication(
+  dataSchema, "datasource", 
sparkSession.sessionState.conf.caseSensitiveAnalysis)
--- End diff --

Will it introduce the behavior changes?

cc @cloud-fan 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17758: [SPARK-20460][SQL] Make it more consistent to han...

2017-06-16 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/17758#discussion_r122535046
  
--- Diff: 
sql/catalyst/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.catalyst.analysis._
+import org.apache.spark.sql.types.StructType
+
+class SchemaUtilsSuite extends SparkFunSuite {
+
+  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)
+}.getMessage
+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`;"""))
+
+// Check no exception thrown
+SchemaUtils.checkSchemaColumnNameDuplication(
+  StructType.fromDDL("a INT, b INT, c INT"), "SchemaUtilsSuite", 
caseSensitiveAnalysis = true)
+SchemaUtils.checkColumnNameDuplication(
+  "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", 
caseSensitiveResolution)
+  }
+
+  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`;"""))
+val msg4 = intercept[AnalysisException] {
+  SchemaUtils.checkColumnNameDuplication(
+"Aa" :: "b" :: "Aa" :: Nil, "SchemaUtilsSuite", 
caseInsensitiveResolution)
--- End diff --

Add a case for three duplicate columns. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17758: [SPARK-20460][SQL] Make it more consistent to han...

2017-06-16 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/17758#discussion_r122534924
  
--- Diff: 
sql/catalyst/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.catalyst.analysis._
+import org.apache.spark.sql.types.StructType
+
+class SchemaUtilsSuite extends SparkFunSuite {
+
+  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)
+}.getMessage
+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`;"""))
+
+// Check no exception thrown
+SchemaUtils.checkSchemaColumnNameDuplication(
+  StructType.fromDDL("a INT, b INT, c INT"), "SchemaUtilsSuite", 
caseSensitiveAnalysis = true)
+SchemaUtils.checkColumnNameDuplication(
+  "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", 
caseSensitiveResolution)
+  }
+
+  test("Check column name duplication in case-insensitive cases") {
+val msg3 = intercept[AnalysisException] {
--- End diff --

-> `var msg`. Use `msg` instead in the following cases.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17758: [SPARK-20460][SQL] Make it more consistent to han...

2017-06-16 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/17758#discussion_r122534244
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala ---
@@ -355,12 +356,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)
--- End diff --

`the view`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17758: [SPARK-20460][SQL] Make it more consistent to han...

2017-06-16 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/17758#discussion_r122534073
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/util/SchemaUtils.scala ---
@@ -0,0 +1,74 @@
+/*
+ * 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 scala.collection.mutable
+
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.analysis._
+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 {
+
+  /**
+   * 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) {
+  caseSensitiveResolution
+} else {
+  caseInsensitiveResolution
+}
+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(
+  colNames: Seq[String], colType: String, resolver: Resolver): Unit = {
+val duplicateColumns = mutable.ArrayBuffer[String]()
+colNames.foreach { name =>
+  val sameColNames = colNames.filter(resolver(_, name))
--- End diff --

-> `find`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17758: [SPARK-20460][SQL] Make it more consistent to han...

2017-06-16 Thread maropu
Github user maropu commented on a diff in the pull request:

https://github.com/apache/spark/pull/17758#discussion_r122407108
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala ---
@@ -248,6 +249,10 @@ private[hive] class HiveMetastoreCatalog(sparkSession: 
SparkSession) extends Log
   if (inferenceMode == INFER_AND_SAVE) {
 updateCatalogSchema(relation.tableMeta.identifier, schema)
   }
+
+  SchemaUtils.checkSchemaColumnNameDuplication(
--- End diff --

I checked in hive-v2.0.1
```
hive> CREATE TABLE t(a INT, b INT, a INT);
FAILED: SemanticException [Error 10036]: Duplicate column name: a
hive> CREATE TABLE t(a INT, b INT, A INT);
FAILED: SemanticException [Error 10036]: Duplicate column name: a
```

Also, I checked in the Hive doc: `Table names and column names are case 
insensitive but SerDe and property names are case sensitive.`

https://cwiki.apache.org/confluence/display/Hive/LanguageManual+DDL#LanguageManualDDL-CreateTable



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17758: [SPARK-20460][SQL] Make it more consistent to han...

2017-06-16 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/17758#discussion_r122378171
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/util/SchemaUtils.scala ---
@@ -0,0 +1,58 @@
+/*
+ * 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 scala.collection.mutable
+
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.analysis._
+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 = {
+val resolver = if (caseSensitiveAnalysis) {
+  caseSensitiveResolution
+} else {
+  caseInsensitiveResolution
+}
+checkColumnNameDuplication(schema.map(_.name), colType, resolver)
+  }
+
+  def checkColumnNameDuplication(
+  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.map(colName => s$colName).mkString(", "))
--- End diff --

We normally use backquotes


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17758: [SPARK-20460][SQL] Make it more consistent to han...

2017-06-15 Thread maropu
Github user maropu commented on a diff in the pull request:

https://github.com/apache/spark/pull/17758#discussion_r122375436
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala ---
@@ -248,6 +249,10 @@ private[hive] class HiveMetastoreCatalog(sparkSession: 
SparkSession) extends Log
   if (inferenceMode == INFER_AND_SAVE) {
 updateCatalogSchema(relation.tableMeta.identifier, schema)
   }
+
+  SchemaUtils.checkSchemaColumnNameDuplication(
--- End diff --

ok


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17758: [SPARK-20460][SQL] Make it more consistent to han...

2017-06-15 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/17758#discussion_r122375082
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala ---
@@ -248,6 +249,10 @@ private[hive] class HiveMetastoreCatalog(sparkSession: 
SparkSession) extends Log
   if (inferenceMode == INFER_AND_SAVE) {
 updateCatalogSchema(relation.tableMeta.identifier, schema)
   }
+
+  SchemaUtils.checkSchemaColumnNameDuplication(
--- End diff --

Could you please double check whether Hive allows duplicate column names?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17758: [SPARK-20460][SQL] Make it more consistent to han...

2017-06-15 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/17758#discussion_r122374729
  
--- Diff: 
sql/catalyst/src/test/scala/org/apache/spark/sql/util/SchemaUtilsSuite.scala ---
@@ -0,0 +1,82 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.util
+
+import org.apache.spark.SparkFunSuite
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.analysis._
+import org.apache.spark.sql.types.StructType
+
+class SchemaUtilsSuite extends SparkFunSuite {
+
+  test("Check column name duplication") {
--- End diff --

Could you split it to multiple smaller test cases with the reasonable 
names, just like the comment you added below?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17758: [SPARK-20460][SQL] Make it more consistent to han...

2017-06-15 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/17758#discussion_r122374556
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/util/SchemaUtils.scala ---
@@ -0,0 +1,58 @@
+/*
+ * 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 scala.collection.mutable
+
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.analysis._
+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 = {
+val resolver = if (caseSensitiveAnalysis) {
+  caseSensitiveResolution
+} else {
+  caseInsensitiveResolution
+}
+checkColumnNameDuplication(schema.map(_.name), colType, resolver)
+  }
+
+  def checkColumnNameDuplication(
--- End diff --

The same here.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17758: [SPARK-20460][SQL] Make it more consistent to han...

2017-06-15 Thread maropu
Github user maropu commented on a diff in the pull request:

https://github.com/apache/spark/pull/17758#discussion_r122374471
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/util/SchemaUtils.scala ---
@@ -0,0 +1,58 @@
+/*
+ * 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 scala.collection.mutable
+
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.analysis._
+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(
--- End diff --

ok


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17758: [SPARK-20460][SQL] Make it more consistent to han...

2017-06-15 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/17758#discussion_r122374344
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/util/SchemaUtils.scala ---
@@ -0,0 +1,58 @@
+/*
+ * 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 scala.collection.mutable
+
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.analysis._
+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(
--- End diff --

Could you add the function description and parameter description?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17758: [SPARK-20460][SQL] Make it more consistent to han...

2017-06-14 Thread maropu
Github user maropu commented on a diff in the pull request:

https://github.com/apache/spark/pull/17758#discussion_r122128317
  
--- Diff: sql/core/src/test/resources/sql-tests/inputs/create.sql ---
@@ -0,0 +1,9 @@
+-- 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;
+
+-- Catch case-insensitive name duplication
+SET spark.sql.caseSensitive=false;
+CREATE TABLE t (ab STRING, cd INT, ef DOUBLE, Ab INT) USING parquet;
--- End diff --

ok


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17758: [SPARK-20460][SQL] Make it more consistent to han...

2017-06-14 Thread maropu
Github user maropu commented on a diff in the pull request:

https://github.com/apache/spark/pull/17758#discussion_r122128225
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala ---
@@ -466,13 +467,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)
--- End diff --

ok, I'll fix to do so.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17758: [SPARK-20460][SQL] Make it more consistent to han...

2017-06-14 Thread maropu
Github user maropu commented on a diff in the pull request:

https://github.com/apache/spark/pull/17758#discussion_r122128029
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala
 ---
@@ -741,14 +742,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")
--- End diff --

oh, sorry, my bad. I'll fix this.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17758: [SPARK-20460][SQL] Make it more consistent to han...

2017-06-14 Thread wzhfy
Github user wzhfy commented on a diff in the pull request:

https://github.com/apache/spark/pull/17758#discussion_r122127202
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala ---
@@ -466,13 +467,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)
--- End diff --

seems to me `sparkSession.sessionState.conf.caseSensitiveAnalysis` and 
`sparkSession.sessionState.conf.resolver` are kind of redundant, can we just 
use `resolver` to detect duplication?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17758: [SPARK-20460][SQL] Make it more consistent to han...

2017-06-14 Thread wzhfy
Github user wzhfy commented on a diff in the pull request:

https://github.com/apache/spark/pull/17758#discussion_r122126852
  
--- Diff: sql/core/src/test/resources/sql-tests/inputs/create.sql ---
@@ -0,0 +1,9 @@
+-- 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;
+
+-- Catch case-insensitive name duplication
+SET spark.sql.caseSensitive=false;
+CREATE TABLE t (ab STRING, cd INT, ef DOUBLE, Ab INT) USING parquet;
--- End diff --

We only need two tests here, set case sensitive explicitly as true or false.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17758: [SPARK-20460][SQL] Make it more consistent to han...

2017-06-14 Thread wzhfy
Github user wzhfy commented on a diff in the pull request:

https://github.com/apache/spark/pull/17758#discussion_r122126261
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala
 ---
@@ -741,14 +742,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")
--- End diff --

Why? In `parseUserSpecifiedCreateTableColumnTypes`, apparently there are 
case sensitive checking. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17758: [SPARK-20460][SQL] Make it more consistent to han...

2017-06-14 Thread maropu
Github user maropu commented on a diff in the pull request:

https://github.com/apache/spark/pull/17758#discussion_r121878933
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala
 ---
@@ -741,14 +742,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")
--- End diff --

In this JDBC case, we currently ignore case-sensitivity. IMHO we might need 
to a JDBC data source option for case-sensitivity?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17758: [SPARK-20460][SQL] Make it more consistent to han...

2017-06-12 Thread maropu
Github user maropu commented on a diff in the pull request:

https://github.com/apache/spark/pull/17758#discussion_r121591640
  
--- Diff: 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;
--- End diff --

Added `SchemaUtilsSuite`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17758: [SPARK-20460][SQL] Make it more consistent to han...

2017-06-11 Thread wzhfy
Github user wzhfy commented on a diff in the pull request:

https://github.com/apache/spark/pull/17758#discussion_r121288368
  
--- Diff: 
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)
--- End diff --

Is `header` option necessary?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17758: [SPARK-20460][SQL] Make it more consistent to han...

2017-06-11 Thread wzhfy
Github user wzhfy commented on a diff in the pull request:

https://github.com/apache/spark/pull/17758#discussion_r121288462
  
--- Diff: 
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) {
--- End diff --

We also need to add test cases for case sensitivity.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17758: [SPARK-20460][SQL] Make it more consistent to han...

2017-06-11 Thread wzhfy
Github user wzhfy commented on a diff in the pull request:

https://github.com/apache/spark/pull/17758#discussion_r121287958
  
--- Diff: 
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 {
--- End diff --

Rename the file (SchemaUtil) also as `SchemaUtils`?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17758: [SPARK-20460][SQL] Make it more consistent to han...

2017-06-11 Thread wzhfy
Github user wzhfy commented on a diff in the pull request:

https://github.com/apache/spark/pull/17758#discussion_r121288362
  
--- Diff: 
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)
--- End diff --

nit: duplicate `"""`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17758: [SPARK-20460][SQL] Make it more consistent to han...

2017-06-11 Thread wzhfy
Github user wzhfy commented on a diff in the pull request:

https://github.com/apache/spark/pull/17758#discussion_r121288487
  
--- Diff: 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;
--- End diff --

Instead of end-to-end tests, maybe we can just add a suite for SchemaUtils.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17758: [SPARK-20460][SQL] Make it more consistent to han...

2017-04-25 Thread maropu
GitHub user maropu opened a pull request:

https://github.com/apache/spark/pull/17758

[SPARK-20460][SQL] Make it more consistent to handle column name duplication

## What changes were proposed in this pull request?
This pr made it more consistent to handle column name duplication. In the 
current master, error handling is different when hitting column name 
duplication:
```
// json
scala> val schema = StructType(StructField("a", IntegerType) :: 
StructField("a", IntegerType) :: Nil)
scala> Seq("""{"a":1, 
"a":1}"").toDF().coalesce(1).write.mode("overwrite").text("/tmp/data")
scala> spark.read.format("json").schema(schema).load("/tmp/data").show
org.apache.spark.sql.AnalysisException: Reference 'a' is ambiguous, could 
be: a#12, a#13.;
  at 
org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.resolve(LogicalPlan.scala:287)
  at 
org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.resolve(LogicalPlan.scala:181)
  at 
org.apache.spark.sql.catalyst.plans.logical.LogicalPlan$$anonfun$resolve$1.apply(LogicalPlan.scala:153)

scala> spark.read.format("json").load("/tmp/data").show
org.apache.spark.sql.AnalysisException: Duplicate column(s) : "a" found, 
cannot save to JSON format;
  at 
org.apache.spark.sql.execution.datasources.json.JsonDataSource.checkConstraints(JsonDataSource.scala:81)
  at 
org.apache.spark.sql.execution.datasources.json.JsonDataSource.inferSchema(JsonDataSource.scala:63)
  at 
org.apache.spark.sql.execution.datasources.json.JsonFileFormat.inferSchema(JsonFileFormat.scala:57)
  at 
org.apache.spark.sql.execution.datasources.DataSource$$anonfun$7.apply(DataSource.scala:176)
  at 
org.apache.spark.sql.execution.datasources.DataSource$$anonfun$7.apply(DataSource.scala:176)

// csv
scala> val schema = StructType(StructField("a", IntegerType) :: 
StructField("a", IntegerType) :: Nil)
scala> Seq("a,a", 
"1,1").toDF().coalesce(1).write.mode("overwrite").text("/tmp/data")
scala> spark.read.format("csv").schema(schema).option("header", 
false).load("/tmp/data").show
org.apache.spark.sql.AnalysisException: Reference 'a' is ambiguous, could 
be: a#41, a#42.;
  at 
org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.resolve(LogicalPlan.scala:287)
  at 
org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.resolve(LogicalPlan.scala:181)
  at 
org.apache.spark.sql.catalyst.plans.logical.LogicalPlan$$anonfun$resolve$1.apply(LogicalPlan.scala:153)
  at 
org.apache.spark.sql.catalyst.plans.logical.LogicalPlan$$anonfun$resolve$1.apply(LogicalPlan.scala:152)

// If `inferSchema` is true, a CSV format is duplicate-safe (See 
SPARK-16896)
scala> spark.read.format("csv").option("header", 
true).load("/tmp/data").show
+---+---+
| a0| a1|
+---+---+
|  1|  1|
+---+---+

// parquet
scala> val schema = StructType(StructField("a", IntegerType) :: 
StructField("a", IntegerType) :: Nil)
scala> Seq((1, 1)).toDF("a", 
"b").coalesce(1).write.mode("overwrite").parquet("/tmp/data")
scala> spark.read.format("parquet").schema(schema).option("header", 
false).load("/tmp/data").show
org.apache.spark.sql.AnalysisException: Reference 'a' is ambiguous, could 
be: a#110, a#111.;
  at 
org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.resolve(LogicalPlan.scala:287)
  at 
org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.resolve(LogicalPlan.scala:181)
  at 
org.apache.spark.sql.catalyst.plans.logical.LogicalPlan$$anonfun$resolve$1.apply(LogicalPlan.scala:153)
  at 
org.apache.spark.sql.catalyst.plans.logical.LogicalPlan$$anonfun$resolve$1.apply(LogicalPlan.scala:152)
  at 
scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
  at 
scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
```
When this patch applied, the results change to;
```

// json
scala> val schema = StructType(StructField("a", IntegerType) :: 
StructField("a", IntegerType) :: Nil)
scala> Seq("""{"a":1, 
"a":1}"").toDF().coalesce(1).write.mode("overwrite").text("/tmp/data")
scala> spark.read.format("json").schema(schema).load("/tmp/data").show
org.apache.spark.sql.AnalysisException: Found duplicate column(s) in 
datasource: "a";
  at 
org.apache.spark.sql.util.SchemaUtils$.checkColumnNameDuplication(SchemaUtil.scala:47)
  at 
org.apache.spark.sql.util.SchemaUtils$.checkSchemaColumnNameDuplication(SchemaUtil.scala:33)
  at 
org.apache.spark.sql.execution.datasources.DataSource.getOrInferFileFormatSchema(DataSource.scala:186)
  at 
org.apache.spark.sql.execution.datasources.DataSource.resolveRelation(DataSource.scala:368)

scala> spark.read.format("json").load("/tmp/data").show
org.apache.spark.sql.AnalysisException: Found duplicate column(s) in 
datasource: "a";
  at 
org.apache.spark.sql.util.SchemaUtils$.checkCol