diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4
index 23e925e4de63c94bea1f1ce0431ca944f131aa37..4c15f9cec65733b9f21da15b96f57ce6f0a95378 100644
--- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4
+++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4
@@ -106,7 +106,8 @@ statement
     | SHOW COLUMNS (FROM | IN) tableIdentifier
         ((FROM | IN) db=identifier)?                                   #showColumns
     | SHOW PARTITIONS tableIdentifier partitionSpec?                   #showPartitions
-    | SHOW FUNCTIONS (LIKE? (qualifiedName | pattern=STRING))?         #showFunctions
+    | SHOW identifier? FUNCTIONS
+        (LIKE? (qualifiedName | pattern=STRING))?                      #showFunctions
     | SHOW CREATE TABLE tableIdentifier                                #showCreateTable
     | (DESC | DESCRIBE) FUNCTION EXTENDED? describeFuncName            #describeFunction
     | (DESC | DESCRIBE) DATABASE EXTENDED? identifier                  #describeDatabase
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 2880087b581c7bd577d67c74b32dbed370f6da09..8c620d36e5679f74576e1f95b741d2bd0e849906 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
@@ -841,21 +841,29 @@ class SessionCatalog(
   }
 
   /**
-   * List all functions in the specified database, including temporary functions.
+   * List all functions in the specified database, including temporary functions. This
+   * returns the function identifier and the scope in which it was defined (system or user
+   * defined).
    */
-  def listFunctions(db: String): Seq[FunctionIdentifier] = listFunctions(db, "*")
+  def listFunctions(db: String): Seq[(FunctionIdentifier, String)] = listFunctions(db, "*")
 
   /**
-   * List all matching functions in the specified database, including temporary functions.
+   * List all matching functions in the specified database, including temporary functions. This
+   * returns the function identifier and the scope in which it was defined (system or user
+   * defined).
    */
-  def listFunctions(db: String, pattern: String): Seq[FunctionIdentifier] = {
+  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) }
-    dbFunctions ++ loadedFunctions
+    val functions = dbFunctions ++ loadedFunctions
+    functions.map {
+      case f if FunctionRegistry.functionSet.contains(f.funcName) => (f, "SYSTEM")
+      case f => (f, "USER")
+    }
   }
 
 
@@ -877,7 +885,7 @@ class SessionCatalog(
     listTables(default).foreach { table =>
       dropTable(table, ignoreIfNotExists = false)
     }
-    listFunctions(default).foreach { func =>
+    listFunctions(default).map(_._1).foreach { func =>
       if (func.database.isDefined) {
         dropFunction(func, ignoreIfNotExists = false)
       } else {
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala
index a4dc03cd8b2605ad3a4e421c53756b092654eb93..c8e7c5103b6eb2343c006692ef4361c4eb0efd9f 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala
@@ -950,16 +950,16 @@ class SessionCatalogSuite extends SparkFunSuite {
     catalog.createFunction(newFunc("not_me", Some("db2")), ignoreIfExists = false)
     catalog.createTempFunction("func1", info1, tempFunc1, ignoreIfExists = false)
     catalog.createTempFunction("yes_me", info2, tempFunc2, ignoreIfExists = false)
-    assert(catalog.listFunctions("db1", "*").toSet ==
+    assert(catalog.listFunctions("db1", "*").map(_._1).toSet ==
       Set(FunctionIdentifier("func1"),
         FunctionIdentifier("yes_me")))
-    assert(catalog.listFunctions("db2", "*").toSet ==
+    assert(catalog.listFunctions("db2", "*").map(_._1).toSet ==
       Set(FunctionIdentifier("func1"),
         FunctionIdentifier("yes_me"),
         FunctionIdentifier("func1", Some("db2")),
         FunctionIdentifier("func2", Some("db2")),
         FunctionIdentifier("not_me", Some("db2"))))
-    assert(catalog.listFunctions("db2", "func*").toSet ==
+    assert(catalog.listFunctions("db2", "func*").map(_._1).toSet ==
       Set(FunctionIdentifier("func1"),
         FunctionIdentifier("func1", Some("db2")),
         FunctionIdentifier("func2", Some("db2"))))
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala
index 066ff57721a3d8f5565d64c3db2a1b9b27844234..42ec210baa2d6fd6fd87433964f8bef0918e2437 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala
@@ -538,14 +538,23 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder {
    */
   override def visitShowFunctions(ctx: ShowFunctionsContext): LogicalPlan = withOrigin(ctx) {
     import ctx._
-    if (qualifiedName != null) {
+    val (user, system) = Option(ctx.identifier).map(_.getText.toLowerCase) match {
+      case None | Some("all") => (true, true)
+      case Some("system") => (false, true)
+      case Some("user") => (true, false)
+      case Some(x) => throw new ParseException(s"SHOW $x FUNCTIONS not supported", ctx)
+    }
+
+    val (db, pat) = if (qualifiedName != null) {
       val name = visitFunctionName(qualifiedName)
-      ShowFunctionsCommand(name.database, Some(name.funcName))
+      (name.database, Some(name.funcName))
     } else if (pattern != null) {
-      ShowFunctionsCommand(None, Some(string(pattern)))
+      (None, Some(string(pattern)))
     } else {
-      ShowFunctionsCommand(None, None)
+      (None, None)
     }
+
+    ShowFunctionsCommand(db, pat, user, system)
   }
 
   /**
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 d2d8e3ddeae26780a1b7804ab59664f868a31734..26593d2918a6e7d38e44a26392658f74769b248d 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
@@ -181,8 +181,11 @@ case class DropFunctionCommand(
  * '|' is for alternation.
  * For example, "show functions like 'yea*|windo*'" will return "window" and "year".
  */
-case class ShowFunctionsCommand(db: Option[String], pattern: Option[String])
-  extends RunnableCommand {
+case class ShowFunctionsCommand(
+    db: Option[String],
+    pattern: Option[String],
+    showUserFunctions: Boolean,
+    showSystemFunctions: Boolean) extends RunnableCommand {
 
   override val output: Seq[Attribute] = {
     val schema = StructType(StructField("function", StringType, nullable = false) :: Nil)
@@ -196,7 +199,10 @@ case class ShowFunctionsCommand(db: Option[String], pattern: Option[String])
     val functionNames =
       sparkSession.sessionState.catalog
         .listFunctions(dbName, pattern.getOrElse("*"))
-        .map(_.unquotedString)
+        .collect {
+          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(_))
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala
index 601334b97add39c5ff90b39fa471b1629dd6059a..44babcc93a1deaf9a64b4268e209d1381dd3f5a0 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala
@@ -121,7 +121,7 @@ class CatalogImpl(sparkSession: SparkSession) extends Catalog {
   @throws[AnalysisException]("database does not exist")
   override def listFunctions(dbName: String): Dataset[Function] = {
     requireDatabaseExists(dbName)
-    val functions = sessionCatalog.listFunctions(dbName).map { funcIdent =>
+    val functions = sessionCatalog.listFunctions(dbName).map { case (funcIdent, _) =>
       val metadata = sessionCatalog.lookupFunctionInfo(funcIdent)
       new Function(
         name = funcIdent.identifier,
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala
index 59b25cb8d0030b42bf03b61882e5137e4e90a36b..b1dbf21d4b80f98f50fcaafd1c00c30cb7f63c75 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala
@@ -61,7 +61,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext {
   test("show functions") {
     def getFunctions(pattern: String): Seq[Row] = {
       StringUtils.filterPattern(
-        spark.sessionState.catalog.listFunctions("default").map(_.funcName), pattern)
+        spark.sessionState.catalog.listFunctions("default").map(_._1.funcName), pattern)
         .map(Row(_))
     }
 
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala
index e2858bb4754017902ef5bfda6b16cf3a40ad70c4..8161c08b2cb482a89d2d8e50f96c37f70830f6ad 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala
@@ -46,10 +46,17 @@ class SparkSqlParserSuite extends PlanTest {
   }
 
   test("show functions") {
-    assertEqual("show functions", ShowFunctionsCommand(None, None))
-    assertEqual("show functions foo", ShowFunctionsCommand(None, Some("foo")))
-    assertEqual("show functions foo.bar", ShowFunctionsCommand(Some("foo"), Some("bar")))
-    assertEqual("show functions 'foo\\\\.*'", ShowFunctionsCommand(None, Some("foo\\.*")))
+    assertEqual("show functions", ShowFunctionsCommand(None, None, true, true))
+    assertEqual("show all functions", ShowFunctionsCommand(None, None, true, true))
+    assertEqual("show user functions", ShowFunctionsCommand(None, None, true, false))
+    assertEqual("show system functions", ShowFunctionsCommand(None, None, false, true))
+    intercept("show special functions", "SHOW special FUNCTIONS")
+    assertEqual("show functions foo",
+      ShowFunctionsCommand(None, Some("foo"), true, true))
+    assertEqual("show functions foo.bar",
+      ShowFunctionsCommand(Some("foo"), Some("bar"), true, true))
+    assertEqual("show functions 'foo\\\\.*'",
+      ShowFunctionsCommand(None, Some("foo\\.*"), true, true))
     intercept("show functions foo.bar.baz", "Unsupported function name")
   }
 
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala
index 47d8a28f49927d21154b571e13aaf18cbd02c3d7..0ee8d179d79eb47ce8e3f5820c018ddc93ec4083 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala
@@ -25,7 +25,7 @@ import org.scalatest.BeforeAndAfterEach
 import org.apache.spark.internal.config._
 import org.apache.spark.sql.{AnalysisException, QueryTest, Row}
 import org.apache.spark.sql.catalyst.TableIdentifier
-import org.apache.spark.sql.catalyst.analysis.{DatabaseAlreadyExistsException, NoSuchPartitionException, NoSuchTableException, TempTableAlreadyExistsException}
+import org.apache.spark.sql.catalyst.analysis.{DatabaseAlreadyExistsException, FunctionRegistry, NoSuchPartitionException, NoSuchTableException, TempTableAlreadyExistsException}
 import org.apache.spark.sql.catalyst.catalog.{CatalogDatabase, CatalogStorageFormat}
 import org.apache.spark.sql.catalyst.catalog.{CatalogColumn, CatalogTable, CatalogTableType}
 import org.apache.spark.sql.catalyst.catalog.{CatalogTablePartition, SessionCatalog}
@@ -1370,4 +1370,19 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach {
       )
     }
   }
+
+  test("show functions") {
+    withUserDefinedFunction("add_one" -> true) {
+      val numFunctions = FunctionRegistry.functionSet.size.toLong
+      assert(sql("show functions").count() === numFunctions)
+      assert(sql("show system functions").count() === numFunctions)
+      assert(sql("show all functions").count() === numFunctions)
+      assert(sql("show user functions").count() === 0L)
+      spark.udf.register("add_one", (x: Long) => x + 1)
+      assert(sql("show functions").count() === numFunctions + 1L)
+      assert(sql("show system functions").count() === numFunctions)
+      assert(sql("show all functions").count() === numFunctions + 1L)
+      assert(sql("show user functions").count() === 1L)
+    }
+  }
 }