Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down Expand Up @@ -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
}
}


Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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(_))
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -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 {
Expand Down