From 7aad057b00db240515692d5c07e67ee58f6b95d3 Mon Sep 17 00:00:00 2001
From: Xiao Li <gatorsmile@gmail.com>
Date: Mon, 17 Apr 2017 09:50:20 -0700
Subject: [PATCH] [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 <gatorsmile@gmail.com>

Closes #17646 from gatorsmile/showFunctions.

(cherry picked from commit 01ff0350a85b179715946c3bd4f003db7c5e3641)
Signed-off-by: Xiao Li <gatorsmile@gmail.com>
---
 .../sql/catalyst/catalog/SessionCatalog.scala | 21 ++++++++++++++-----
 .../sql/execution/command/functions.scala     |  4 +---
 .../sql/hive/execution/HiveUDFSuite.scala     | 17 +++++++++++++++
 3 files changed, 34 insertions(+), 8 deletions(-)

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 a5cf7196b2..6f302d3d02 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
   }
 
 
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 ea5398761c..75272d295b 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(_))
   }
 }
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 78c80dacb9..9368d0ba8e 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 {
-- 
GitLab