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 6f302d3d0250..a5cf7196b21e 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,7 +20,6 @@ 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 @@ -1099,25 +1098,15 @@ 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 => - // 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 dbFunctions = externalCatalog.listFunctions(dbName, pattern) + .map { f => FunctionIdentifier(f, Some(dbName)) } + val loadedFunctions = StringUtils.filterPattern(functionRegistry.listFunction(), pattern) + .map { f => 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 75272d295b16..ea5398761c46 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,6 +208,8 @@ case class ShowFunctionsCommand( case (f, "USER") if showUserFunctions => f.unquotedString case (f, "SYSTEM") if showSystemFunctions => f.unquotedString } - functionNames.sorted.map(Row(_)) + // The session catalog caches some persistent functions in the FunctionRegistry + // so there can be duplicates. + functionNames.distinct.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 9368d0ba8ef6..78c80dacb9fa 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,23 +539,6 @@ 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 {