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 @@ -22,6 +22,7 @@ import java.util.Locale
import javax.annotation.concurrent.GuardedBy

import scala.collection.mutable
import scala.util.{Failure, Success, Try}

import com.google.common.cache.{Cache, CacheBuilder}
import org.apache.hadoop.conf.Configuration
Expand Down Expand Up @@ -1202,15 +1203,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.
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

shall we use FunctionIdentifier as the key in functionRegistry?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes. Will do it using this way in the following refactoring.

We can first fix the issue and then backport it to the previous branches.

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
Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The name of persistent function exists in both dbFunctions and loadedFunctions.

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

dbFunctions

// 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 @@ -207,8 +207,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(_))
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -558,6 +558,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 {
Expand Down