You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by ma...@apache.org on 2014/09/16 20:42:30 UTC

git commit: [SPARK-2890][SQL] Allow reading of data when case insensitive resolution could cause possible ambiguity.

Repository: spark
Updated Branches:
  refs/heads/master 758369987 -> 30f288ae3


[SPARK-2890][SQL] Allow reading of data when case insensitive resolution could cause possible ambiguity.

Throwing an error in the constructor makes it possible to run queries, even when there is no actual ambiguity.  Remove this check in favor of throwing an error in analysis when they query is actually is ambiguous.

Also took the opportunity to add test cases that would have caught a subtle bug in my first attempt at fixing this and refactor some other test code.

Author: Michael Armbrust <mi...@databricks.com>

Closes #2209 from marmbrus/sameNameStruct and squashes the following commits:

729cca4 [Michael Armbrust] Better tests.
a003aeb [Michael Armbrust] Remove error (it'll be caught in analysis).


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/30f288ae
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/30f288ae
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/30f288ae

Branch: refs/heads/master
Commit: 30f288ae34a67307aa45b7aecbd0d02a0a14fe69
Parents: 7583699
Author: Michael Armbrust <mi...@databricks.com>
Authored: Tue Sep 16 11:42:26 2014 -0700
Committer: Michael Armbrust <mi...@databricks.com>
Committed: Tue Sep 16 11:42:26 2014 -0700

----------------------------------------------------------------------
 .../spark/sql/catalyst/types/dataTypes.scala    |  4 --
 .../spark/sql/hive/execution/HiveUdfSuite.scala | 67 +++++++++++++-------
 2 files changed, 44 insertions(+), 27 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/spark/blob/30f288ae/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala
index 70c6d06..49520b7 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala
@@ -308,13 +308,9 @@ case class StructField(name: String, dataType: DataType, nullable: Boolean) {
 object StructType {
   protected[sql] def fromAttributes(attributes: Seq[Attribute]): StructType =
     StructType(attributes.map(a => StructField(a.name, a.dataType, a.nullable)))
-
-  private def validateFields(fields: Seq[StructField]): Boolean =
-    fields.map(field => field.name).distinct.size == fields.size
 }
 
 case class StructType(fields: Seq[StructField]) extends DataType {
-  require(StructType.validateFields(fields), "Found fields with the same name.")
 
   /**
    * Returns all field names in a [[Seq]].

http://git-wip-us.apache.org/repos/asf/spark/blob/30f288ae/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUdfSuite.scala
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUdfSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUdfSuite.scala
index b6b8592..cc125d5 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUdfSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUdfSuite.scala
@@ -17,47 +17,68 @@
 
 package org.apache.spark.sql.hive.execution
 
-import org.apache.spark.sql.hive.test.TestHive
-import org.apache.hadoop.conf.Configuration
-import org.apache.spark.SparkContext._
+import java.io.{DataOutput, DataInput}
 import java.util
-import org.apache.hadoop.fs.{FileSystem, Path}
+import java.util.Properties
+
+import org.apache.spark.util.Utils
+
+import scala.collection.JavaConversions._
+
+import org.apache.hadoop.conf.Configuration
 import org.apache.hadoop.hive.serde2.{SerDeStats, AbstractSerDe}
-import org.apache.hadoop.io.{NullWritable, Writable}
+import org.apache.hadoop.io.Writable
 import org.apache.hadoop.hive.serde2.objectinspector.{ObjectInspectorFactory, ObjectInspector}
-import java.util.Properties
+
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory
-import scala.collection.JavaConversions._
-import java.io.{DataOutput, DataInput}
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDF
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDF.DeferredObject
 
+import org.apache.spark.sql.Row
+import org.apache.spark.sql.hive.test.TestHive
+import org.apache.spark.sql.hive.test.TestHive._
+
+case class Fields(f1: Int, f2: Int, f3: Int, f4: Int, f5: Int)
+
 /**
  * A test suite for Hive custom UDFs.
  */
 class HiveUdfSuite extends HiveComparisonTest {
 
-  TestHive.sql(
-    """
+  test("spark sql udf test that returns a struct") {
+    registerFunction("getStruct", (_: Int) => Fields(1, 2, 3, 4, 5))
+    assert(sql(
+      """
+        |SELECT getStruct(1).f1,
+        |       getStruct(1).f2,
+        |       getStruct(1).f3,
+        |       getStruct(1).f4,
+        |       getStruct(1).f5 FROM src LIMIT 1
+      """.stripMargin).first() === Row(1, 2, 3, 4, 5))
+  }
+
+  test("hive struct udf") {
+    sql(
+      """
       |CREATE EXTERNAL TABLE hiveUdfTestTable (
       |   pair STRUCT<id: INT, value: INT>
       |)
       |PARTITIONED BY (partition STRING)
       |ROW FORMAT SERDE '%s'
       |STORED AS SEQUENCEFILE
-    """.stripMargin.format(classOf[PairSerDe].getName)
-  )
-
-  TestHive.sql(
-    "ALTER TABLE hiveUdfTestTable ADD IF NOT EXISTS PARTITION(partition='testUdf') LOCATION '%s'"
-      .format(this.getClass.getClassLoader.getResource("data/files/testUdf").getFile)
-  )
-
-  TestHive.sql("CREATE TEMPORARY FUNCTION testUdf AS '%s'".format(classOf[PairUdf].getName))
-
-  TestHive.sql("SELECT testUdf(pair) FROM hiveUdfTestTable")
-
-  TestHive.sql("DROP TEMPORARY FUNCTION IF EXISTS testUdf")
+    """.
+        stripMargin.format(classOf[PairSerDe].getName))
+
+    val location = Utils.getSparkClassLoader.getResource("data/files/testUdf").getFile
+    sql(s"""
+      ALTER TABLE hiveUdfTestTable
+      ADD IF NOT EXISTS PARTITION(partition='testUdf')
+      LOCATION '$location'""")
+
+    sql(s"CREATE TEMPORARY FUNCTION testUdf AS '${classOf[PairUdf].getName}'")
+    sql("SELECT testUdf(pair) FROM hiveUdfTestTable")
+    sql("DROP TEMPORARY FUNCTION IF EXISTS testUdf")
+  }
 }
 
 class TestPair(x: Int, y: Int) extends Writable with Serializable {


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