Skip to content

Commit 64c252e

Browse files
committed
revert
1 parent 622d7a8 commit 64c252e

File tree

3 files changed

+8
-34
lines changed

3 files changed

+8
-34
lines changed

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala

Lines changed: 5 additions & 16 deletions
Original file line numberDiff line numberDiff line change
@@ -20,7 +20,6 @@ package org.apache.spark.sql.catalyst.catalog
2020
import javax.annotation.concurrent.GuardedBy
2121

2222
import scala.collection.mutable
23-
import scala.util.{Failure, Success, Try}
2423

2524
import org.apache.hadoop.conf.Configuration
2625
import org.apache.hadoop.fs.Path
@@ -1099,25 +1098,15 @@ class SessionCatalog(
10991098
def listFunctions(db: String, pattern: String): Seq[(FunctionIdentifier, String)] = {
11001099
val dbName = formatDatabaseName(db)
11011100
requireDbExists(dbName)
1102-
val dbFunctions = externalCatalog.listFunctions(dbName, pattern).map { f =>
1103-
FunctionIdentifier(f, Some(dbName)) }
1104-
val loadedFunctions =
1105-
StringUtils.filterPattern(functionRegistry.listFunction(), pattern).map { f =>
1106-
// In functionRegistry, function names are stored as an unquoted format.
1107-
Try(parser.parseFunctionIdentifier(f)) match {
1108-
case Success(e) => e
1109-
case Failure(_) =>
1110-
// The names of some built-in functions are not parsable by our parser, e.g., %
1111-
FunctionIdentifier(f)
1112-
}
1113-
}
1101+
val dbFunctions = externalCatalog.listFunctions(dbName, pattern)
1102+
.map { f => FunctionIdentifier(f, Some(dbName)) }
1103+
val loadedFunctions = StringUtils.filterPattern(functionRegistry.listFunction(), pattern)
1104+
.map { f => FunctionIdentifier(f) }
11141105
val functions = dbFunctions ++ loadedFunctions
1115-
// The session catalog caches some persistent functions in the FunctionRegistry
1116-
// so there can be duplicates.
11171106
functions.map {
11181107
case f if FunctionRegistry.functionSet.contains(f.funcName) => (f, "SYSTEM")
11191108
case f => (f, "USER")
1120-
}.distinct
1109+
}
11211110
}
11221111

11231112

sql/core/src/main/scala/org/apache/spark/sql/execution/command/functions.scala

Lines changed: 3 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -208,6 +208,8 @@ case class ShowFunctionsCommand(
208208
case (f, "USER") if showUserFunctions => f.unquotedString
209209
case (f, "SYSTEM") if showSystemFunctions => f.unquotedString
210210
}
211-
functionNames.sorted.map(Row(_))
211+
// The session catalog caches some persistent functions in the FunctionRegistry
212+
// so there can be duplicates.
213+
functionNames.distinct.sorted.map(Row(_))
212214
}
213215
}

sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDFSuite.scala

Lines changed: 0 additions & 17 deletions
Original file line numberDiff line numberDiff line change
@@ -539,23 +539,6 @@ class HiveUDFSuite extends QueryTest with TestHiveSingleton with SQLTestUtils {
539539
checkAnswer(testData.selectExpr("statelessUDF() as s").agg(max($"s")), Row(1))
540540
}
541541
}
542-
543-
test("Show persistent functions") {
544-
val testData = spark.sparkContext.parallelize(StringCaseClass("") :: Nil).toDF()
545-
withTempView("inputTable") {
546-
testData.createOrReplaceTempView("inputTable")
547-
withUserDefinedFunction("testUDFToListInt" -> false) {
548-
val numFunc = spark.catalog.listFunctions().count()
549-
sql(s"CREATE FUNCTION testUDFToListInt AS '${classOf[UDFToListInt].getName}'")
550-
assert(spark.catalog.listFunctions().count() == numFunc + 1)
551-
checkAnswer(
552-
sql("SELECT testUDFToListInt(s) FROM inputTable"),
553-
Seq(Row(Seq(1, 2, 3))))
554-
assert(sql("show functions").count() == numFunc + 1)
555-
assert(spark.catalog.listFunctions().count() == numFunc + 1)
556-
}
557-
}
558-
}
559542
}
560543

561544
class TestPair(x: Int, y: Int) extends Writable with Serializable {

0 commit comments

Comments
 (0)