You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by GitBox <gi...@apache.org> on 2020/07/29 07:30:55 UTC

[GitHub] [spark] cloud-fan commented on a change in pull request #29234: [SPARK-32431][SQL] Check duplicate nested columns in read from in-built datasources

cloud-fan commented on a change in pull request #29234:
URL: https://github.com/apache/spark/pull/29234#discussion_r461445888



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/util/SchemaUtils.scala
##########
@@ -42,7 +44,27 @@ private[spark] object SchemaUtils {
    */
   def checkSchemaColumnNameDuplication(
       schema: StructType, colType: String, caseSensitiveAnalysis: Boolean = false): Unit = {
-    checkColumnNameDuplication(schema.map(_.name), colType, caseSensitiveAnalysis)
+    val queue = new Queue[StructType]()
+    queue.enqueue(schema)
+    do {
+      val struct = queue.dequeue()
+      checkColumnNameDuplication(struct.map(_.name), colType, caseSensitiveAnalysis)
+      val nestedStructs = struct.map(_.dataType).collect { case st: StructType => st }
+      queue.enqueue(nestedStructs: _*)
+    } while (queue.nonEmpty)

Review comment:
       +1

##########
File path: external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala
##########
@@ -1800,6 +1800,44 @@ abstract class AvroSuite extends QueryTest with SharedSparkSession {
       assert(version === SPARK_VERSION_SHORT)
     }
   }
+
+  test("SPARK-32431: consistent error for nested and top-level duplicate columns") {

Review comment:
       is it an existing problem that avro needs to duplicate the file source tests?

##########
File path: sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala
##########
@@ -43,10 +43,56 @@ import org.apache.spark.sql.internal.SQLConf
 import org.apache.spark.sql.test.SharedSparkSession
 import org.apache.spark.sql.types._
 
+trait NestedDataSourceSuiteBase extends QueryTest with SharedSparkSession{
+  protected val nestedDataSources: Seq[String]
+
+  test("SPARK-32431: consistent error for nested and top-level duplicate columns") {
+    Seq(
+      Seq("id AS lowercase", "id + 1 AS camelCase") ->
+        new StructType()
+          .add("LowerCase", LongType)
+          .add("camelcase", LongType)
+          .add("CamelCase", LongType),
+      Seq("NAMED_STRUCT('lowercase', id, 'camelCase', id + 1) AS StructColumn") ->
+        new StructType().add("StructColumn",
+          new StructType()
+            .add("LowerCase", LongType)
+            .add("camelcase", LongType)
+            .add("CamelCase", LongType))
+    ).foreach { case (selectExpr: Seq[String], caseInsensitiveSchema: StructType) =>
+      withSQLConf(SQLConf.CASE_SENSITIVE.key -> "false") {

Review comment:
       shall we test both v1 and v2?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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