aboutsummaryrefslogtreecommitdiff
path: root/sql/hive
diff options
context:
space:
mode:
authorgatorsmile <gatorsmile@gmail.com>2016-05-06 11:43:07 -0700
committerYin Huai <yhuai@databricks.com>2016-05-06 11:43:07 -0700
commit5c8fad7b9bfd6677111a8e27e2574f82b04ec479 (patch)
tree5beee2fc0016634be8461e12d1617c7edfb7b3f9 /sql/hive
parent76ad04d9a0a7d4dfb762318d9c7be0d7720f4e1a (diff)
downloadspark-5c8fad7b9bfd6677111a8e27e2574f82b04ec479.tar.gz
spark-5c8fad7b9bfd6677111a8e27e2574f82b04ec479.tar.bz2
spark-5c8fad7b9bfd6677111a8e27e2574f82b04ec479.zip
[SPARK-15108][SQL] Describe Permanent UDTF
#### What changes were proposed in this pull request? When Describe a UDTF, the command returns a wrong result. The command is unable to find the function, which has been created and cataloged in the catalog but not in the functionRegistry. This PR is to correct it. If the function is not in the functionRegistry, we will check the catalog for collecting the information of the UDTF function. #### How was this patch tested? Added test cases to verify the results Author: gatorsmile <gatorsmile@gmail.com> Closes #12885 from gatorsmile/showFunction.
Diffstat (limited to 'sql/hive')
-rw-r--r--sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClient.scala2
-rw-r--r--sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala54
2 files changed, 52 insertions, 4 deletions
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClient.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClient.scala
index b224664050..befdda1621 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClient.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClient.scala
@@ -229,7 +229,7 @@ private[hive] trait HiveClient {
/** Return an existing function in the database, assuming it exists. */
final def getFunction(db: String, name: String): CatalogFunction = {
- getFunctionOption(db, name).getOrElse(throw new NoSuchFunctionException(db, name))
+ getFunctionOption(db, name).getOrElse(throw new NoSuchPermanentFunctionException(db, name))
}
/** Return an existing function in the database, or None if it doesn't exist. */
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala
index f7da9e73f6..4845da7c85 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala
@@ -29,7 +29,6 @@ import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRela
import org.apache.spark.sql.functions._
import org.apache.spark.sql.hive.{HiveUtils, MetastoreRelation}
import org.apache.spark.sql.hive.test.TestHiveSingleton
-import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.test.SQLTestUtils
import org.apache.spark.sql.types._
import org.apache.spark.unsafe.types.CalendarInterval
@@ -208,8 +207,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton {
Row("sha") :: Row("sha1") :: Row("sha2") :: Row("weekofyear") :: Nil)
}
- test("describe functions") {
- // The Spark SQL built-in functions
+ test("describe functions - built-in functions") {
checkKeywordsExist(sql("describe function extended upper"),
"Function: upper",
"Class: org.apache.spark.sql.catalyst.expressions.Upper",
@@ -253,6 +251,56 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton {
"When a = b, returns c; when a = d, return e; else return f")
}
+ test("describe functions - user defined functions") {
+ withUserDefinedFunction("udtf_count" -> false) {
+ sql(
+ s"""
+ |CREATE FUNCTION udtf_count
+ |AS 'org.apache.spark.sql.hive.execution.GenericUDTFCount2'
+ |USING JAR '${hiveContext.getHiveFile("TestUDTF.jar").getCanonicalPath()}'
+ """.stripMargin)
+
+ checkKeywordsExist(sql("describe function udtf_count"),
+ "Function: default.udtf_count",
+ "Class: org.apache.spark.sql.hive.execution.GenericUDTFCount2",
+ "Usage: N/A")
+
+ checkAnswer(
+ sql("SELECT udtf_count(a) FROM (SELECT 1 AS a FROM src LIMIT 3) t"),
+ Row(3) :: Row(3) :: Nil)
+
+ checkKeywordsExist(sql("describe function udtf_count"),
+ "Function: default.udtf_count",
+ "Class: org.apache.spark.sql.hive.execution.GenericUDTFCount2",
+ "Usage: N/A")
+ }
+ }
+
+ test("describe functions - temporary user defined functions") {
+ withUserDefinedFunction("udtf_count_temp" -> true) {
+ sql(
+ s"""
+ |CREATE TEMPORARY FUNCTION udtf_count_temp
+ |AS 'org.apache.spark.sql.hive.execution.GenericUDTFCount2'
+ |USING JAR '${hiveContext.getHiveFile("TestUDTF.jar").getCanonicalPath()}'
+ """.stripMargin)
+
+ checkKeywordsExist(sql("describe function udtf_count_temp"),
+ "Function: udtf_count_temp",
+ "Class: org.apache.spark.sql.hive.execution.GenericUDTFCount2",
+ "Usage: N/A")
+
+ checkAnswer(
+ sql("SELECT udtf_count_temp(a) FROM (SELECT 1 AS a FROM src LIMIT 3) t"),
+ Row(3) :: Row(3) :: Nil)
+
+ checkKeywordsExist(sql("describe function udtf_count_temp"),
+ "Function: udtf_count_temp",
+ "Class: org.apache.spark.sql.hive.execution.GenericUDTFCount2",
+ "Usage: N/A")
+ }
+ }
+
test("SPARK-5371: union with null and sum") {
val df = Seq((1, 1)).toDF("c1", "c2")
df.registerTempTable("table1")