Repository: spark
Updated Branches:
  refs/heads/branch-2.1 efa11a42f -> 7aad057b0


[SPARK-20349][SQL] ListFunctions returns duplicate functions after using 
persistent functions

### What changes were proposed in this pull request?
The session catalog caches some persistent functions in the `FunctionRegistry`, 
so there can be duplicates. Our Catalog API `listFunctions` does not handle it.

It would be better if `SessionCatalog` API can de-duplciate the records, 
instead of doing it by each API caller. In `FunctionRegistry`, our functions 
are identified by the unquoted string. Thus, this PR is try to parse it using 
our parser interface and then de-duplicate the names.

### How was this patch tested?
Added test cases.

Author: Xiao Li <gatorsm...@gmail.com>

Closes #17646 from gatorsmile/showFunctions.

(cherry picked from commit 01ff0350a85b179715946c3bd4f003db7c5e3641)
Signed-off-by: Xiao Li <gatorsm...@gmail.com>


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

Branch: refs/heads/branch-2.1
Commit: 7aad057b00db240515692d5c07e67ee58f6b95d3
Parents: efa11a4
Author: Xiao Li <gatorsm...@gmail.com>
Authored: Mon Apr 17 09:50:20 2017 -0700
Committer: Xiao Li <gatorsm...@gmail.com>
Committed: Mon Apr 17 09:50:32 2017 -0700

----------------------------------------------------------------------
 .../sql/catalyst/catalog/SessionCatalog.scala   | 21 +++++++++++++++-----
 .../spark/sql/execution/command/functions.scala |  4 +---
 .../spark/sql/hive/execution/HiveUDFSuite.scala | 17 ++++++++++++++++
 3 files changed, 34 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/spark/blob/7aad057b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala
----------------------------------------------------------------------
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala
index a5cf719..6f302d3 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala
@@ -20,6 +20,7 @@ package org.apache.spark.sql.catalyst.catalog
 import javax.annotation.concurrent.GuardedBy
 
 import scala.collection.mutable
+import scala.util.{Failure, Success, Try}
 
 import org.apache.hadoop.conf.Configuration
 import org.apache.hadoop.fs.Path
@@ -1098,15 +1099,25 @@ class SessionCatalog(
   def listFunctions(db: String, pattern: String): Seq[(FunctionIdentifier, 
String)] = {
     val dbName = formatDatabaseName(db)
     requireDbExists(dbName)
-    val dbFunctions = externalCatalog.listFunctions(dbName, pattern)
-      .map { f => FunctionIdentifier(f, Some(dbName)) }
-    val loadedFunctions = 
StringUtils.filterPattern(functionRegistry.listFunction(), pattern)
-      .map { f => FunctionIdentifier(f) }
+    val dbFunctions = externalCatalog.listFunctions(dbName, pattern).map { f =>
+      FunctionIdentifier(f, Some(dbName)) }
+    val loadedFunctions =
+      StringUtils.filterPattern(functionRegistry.listFunction(), pattern).map 
{ f =>
+        // In functionRegistry, function names are stored as an unquoted 
format.
+        Try(parser.parseFunctionIdentifier(f)) match {
+          case Success(e) => e
+          case Failure(_) =>
+            // The names of some built-in functions are not parsable by our 
parser, e.g., %
+            FunctionIdentifier(f)
+        }
+      }
     val functions = dbFunctions ++ loadedFunctions
+    // The session catalog caches some persistent functions in the 
FunctionRegistry
+    // so there can be duplicates.
     functions.map {
       case f if FunctionRegistry.functionSet.contains(f.funcName) => (f, 
"SYSTEM")
       case f => (f, "USER")
-    }
+    }.distinct
   }
 
 

http://git-wip-us.apache.org/repos/asf/spark/blob/7aad057b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/functions.scala
----------------------------------------------------------------------
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/functions.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/functions.scala
index ea53987..75272d2 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/functions.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/functions.scala
@@ -208,8 +208,6 @@ case class ShowFunctionsCommand(
           case (f, "USER") if showUserFunctions => f.unquotedString
           case (f, "SYSTEM") if showSystemFunctions => f.unquotedString
         }
-    // The session catalog caches some persistent functions in the 
FunctionRegistry
-    // so there can be duplicates.
-    functionNames.distinct.sorted.map(Row(_))
+    functionNames.sorted.map(Row(_))
   }
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/7aad057b/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 78c80da..9368d0b 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
@@ -539,6 +539,23 @@ class HiveUDFSuite extends QueryTest with 
TestHiveSingleton with SQLTestUtils {
       checkAnswer(testData.selectExpr("statelessUDF() as s").agg(max($"s")), 
Row(1))
     }
   }
+
+  test("Show persistent functions") {
+    val testData = spark.sparkContext.parallelize(StringCaseClass("") :: 
Nil).toDF()
+    withTempView("inputTable") {
+      testData.createOrReplaceTempView("inputTable")
+      withUserDefinedFunction("testUDFToListInt" -> false) {
+        val numFunc = spark.catalog.listFunctions().count()
+        sql(s"CREATE FUNCTION testUDFToListInt AS 
'${classOf[UDFToListInt].getName}'")
+        assert(spark.catalog.listFunctions().count() == numFunc + 1)
+        checkAnswer(
+          sql("SELECT testUDFToListInt(s) FROM inputTable"),
+          Seq(Row(Seq(1, 2, 3))))
+        assert(sql("show functions").count() == numFunc + 1)
+        assert(spark.catalog.listFunctions().count() == numFunc + 1)
+      }
+    }
+  }
 }
 
 class TestPair(x: Int, y: Int) extends Writable with Serializable {


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

Reply via email to