diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/ExpressionDescription.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/ExpressionDescription.java
new file mode 100644
index 0000000000000000000000000000000000000000..9e10f27d59d55c8bef39b6f514e8c13ea15394e1
--- /dev/null
+++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/ExpressionDescription.java
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.expressions;
+
+import org.apache.spark.annotation.DeveloperApi;
+
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+
+/**
+ * ::DeveloperApi::
+
+ * A function description type which can be recognized by FunctionRegistry, and will be used to
+ * show the usage of the function in human language.
+ *
+ * `usage()` will be used for the function usage in brief way.
+ * `extended()` will be used for the function usage in verbose way, suppose
+ *              an example will be provided.
+ *
+ *  And we can refer the function name by `_FUNC_`, in `usage` and `extended`, as it's
+ *  registered in `FunctionRegistry`.
+ */
+@DeveloperApi
+@Retention(RetentionPolicy.RUNTIME)
+public @interface ExpressionDescription {
+    String usage() default "_FUNC_ is undocumented";
+    String extended() default "No example for _FUNC_.";
+}
diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/ExpressionInfo.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/ExpressionInfo.java
new file mode 100644
index 0000000000000000000000000000000000000000..ba8e9cb4be28b6e86334646eb85519d5b8ba16bb
--- /dev/null
+++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/ExpressionInfo.java
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.expressions;
+
+/**
+ * Expression information, will be used to describe a expression.
+ */
+public class ExpressionInfo {
+    private String className;
+    private String usage;
+    private String name;
+    private String extended;
+
+    public String getClassName() {
+        return className;
+    }
+
+    public String getUsage() {
+        return usage;
+    }
+
+    public String getName() {
+        return name;
+    }
+
+    public String getExtended() {
+        return extended;
+    }
+
+    public ExpressionInfo(String className, String name, String usage, String extended) {
+        this.className = className;
+        this.name = name;
+        this.usage = usage;
+        this.extended = extended;
+    }
+
+    public ExpressionInfo(String className, String name) {
+        this(className, name, null, null);
+    }
+}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala
index 9c349838c28a14cc34d94083a3acec525ecb8e78..aa05f448d12bc28e1b13542fc8f034593226256d 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala
@@ -30,26 +30,44 @@ import org.apache.spark.sql.catalyst.util.StringKeyHashMap
 /** A catalog for looking up user defined functions, used by an [[Analyzer]]. */
 trait FunctionRegistry {
 
-  def registerFunction(name: String, builder: FunctionBuilder): Unit
+  final def registerFunction(name: String, builder: FunctionBuilder): Unit = {
+    registerFunction(name, new ExpressionInfo(builder.getClass.getCanonicalName, name), builder)
+  }
+
+  def registerFunction(name: String, info: ExpressionInfo, builder: FunctionBuilder): Unit
 
   @throws[AnalysisException]("If function does not exist")
   def lookupFunction(name: String, children: Seq[Expression]): Expression
+
+  /* List all of the registered function names. */
+  def listFunction(): Seq[String]
+
+  /* Get the class of the registered function by specified name. */
+  def lookupFunction(name: String): Option[ExpressionInfo]
 }
 
 class SimpleFunctionRegistry extends FunctionRegistry {
 
-  private val functionBuilders = StringKeyHashMap[FunctionBuilder](caseSensitive = false)
+  private val functionBuilders =
+    StringKeyHashMap[(ExpressionInfo, FunctionBuilder)](caseSensitive = false)
 
-  override def registerFunction(name: String, builder: FunctionBuilder): Unit = {
-    functionBuilders.put(name, builder)
+  override def registerFunction(name: String, info: ExpressionInfo, builder: FunctionBuilder)
+  : Unit = {
+    functionBuilders.put(name, (info, builder))
   }
 
   override def lookupFunction(name: String, children: Seq[Expression]): Expression = {
-    val func = functionBuilders.get(name).getOrElse {
+    val func = functionBuilders.get(name).map(_._2).getOrElse {
       throw new AnalysisException(s"undefined function $name")
     }
     func(children)
   }
+
+  override def listFunction(): Seq[String] = functionBuilders.iterator.map(_._1).toList.sorted
+
+  override def lookupFunction(name: String): Option[ExpressionInfo] = {
+    functionBuilders.get(name).map(_._1)
+  }
 }
 
 /**
@@ -57,13 +75,22 @@ class SimpleFunctionRegistry extends FunctionRegistry {
  * functions are already filled in and the analyzer needs only to resolve attribute references.
  */
 object EmptyFunctionRegistry extends FunctionRegistry {
-  override def registerFunction(name: String, builder: FunctionBuilder): Unit = {
+  override def registerFunction(name: String, info: ExpressionInfo, builder: FunctionBuilder)
+  : Unit = {
     throw new UnsupportedOperationException
   }
 
   override def lookupFunction(name: String, children: Seq[Expression]): Expression = {
     throw new UnsupportedOperationException
   }
+
+  override def listFunction(): Seq[String] = {
+    throw new UnsupportedOperationException
+  }
+
+  override def lookupFunction(name: String): Option[ExpressionInfo] = {
+    throw new UnsupportedOperationException
+  }
 }
 
 
@@ -71,7 +98,7 @@ object FunctionRegistry {
 
   type FunctionBuilder = Seq[Expression] => Expression
 
-  val expressions: Map[String, FunctionBuilder] = Map(
+  val expressions: Map[String, (ExpressionInfo, FunctionBuilder)] = Map(
     // misc non-aggregate functions
     expression[Abs]("abs"),
     expression[CreateArray]("array"),
@@ -205,13 +232,13 @@ object FunctionRegistry {
 
   val builtin: FunctionRegistry = {
     val fr = new SimpleFunctionRegistry
-    expressions.foreach { case (name, builder) => fr.registerFunction(name, builder) }
+    expressions.foreach { case (name, (info, builder)) => fr.registerFunction(name, info, builder) }
     fr
   }
 
   /** See usage above. */
   private def expression[T <: Expression](name: String)
-      (implicit tag: ClassTag[T]): (String, FunctionBuilder) = {
+      (implicit tag: ClassTag[T]): (String, (ExpressionInfo, FunctionBuilder)) = {
 
     // See if we can find a constructor that accepts Seq[Expression]
     val varargCtor = Try(tag.runtimeClass.getDeclaredConstructor(classOf[Seq[_]])).toOption
@@ -237,6 +264,15 @@ object FunctionRegistry {
         }
       }
     }
-    (name, builder)
+
+    val clazz = tag.runtimeClass
+    val df = clazz.getAnnotation(classOf[ExpressionDescription])
+    if (df != null) {
+      (name,
+        (new ExpressionInfo(clazz.getCanonicalName, name, df.usage(), df.extended()),
+        builder))
+    } else {
+      (name, (new ExpressionInfo(clazz.getCanonicalName, name), builder))
+    }
   }
 }
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala
index 7c254a8750a9f09ae92bdc4aee3ffaec76d64293..b37f530ec6814a721d5502e6ba6643cf9f17cd5f 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala
@@ -65,6 +65,9 @@ case class UnaryPositive(child: Expression) extends UnaryExpression with Expects
 /**
  * A function that get the absolute value of the numeric value.
  */
+@ExpressionDescription(
+  usage = "_FUNC_(expr) - Returns the absolute value of the numeric value",
+  extended = "> SELECT _FUNC_('-1');\n1")
 case class Abs(child: Expression)
   extends UnaryExpression with ExpectsInputTypes with CodegenFallback {
 
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala
index cf187ad5a0a9f3c1f605858e8913bad350c96121..38b0fb37dee3bc97076bcf0b9639126eb40f75cd 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala
@@ -214,6 +214,9 @@ trait String2StringExpression extends ImplicitCastInputTypes {
 /**
  * A function that converts the characters of a string to uppercase.
  */
+@ExpressionDescription(
+  usage = "_FUNC_(str) - Returns str with all characters changed to uppercase",
+  extended = "> SELECT _FUNC_('SparkSql');\n 'SPARKSQL'")
 case class Upper(child: Expression)
   extends UnaryExpression with String2StringExpression {
 
@@ -227,6 +230,9 @@ case class Upper(child: Expression)
 /**
  * A function that converts the characters of a string to lowercase.
  */
+@ExpressionDescription(
+  usage = "_FUNC_(str) - Returns str with all characters changed to lowercase",
+  extended = "> SELECT _FUNC_('SparkSql');\n'sparksql'")
 case class Lower(child: Expression) extends UnaryExpression with String2StringExpression {
 
   override def convert(v: UTF8String): UTF8String = v.toLowerCase
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/commands.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/commands.scala
index 246f4d7e34d3d0bee9e711d7a551b4ed34aff761..e6621e0f50a9ebde7bfe1ae1df92941c7497679d 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/commands.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/commands.scala
@@ -17,7 +17,8 @@
 
 package org.apache.spark.sql.catalyst.plans.logical
 
-import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Attribute}
+import org.apache.spark.sql.types.StringType
 
 /**
  * A logical node that represents a non-query command to be executed by the system.  For example,
@@ -25,3 +26,28 @@ import org.apache.spark.sql.catalyst.expressions.Attribute
  * eagerly executed.
  */
 trait Command
+
+/**
+ * Returned for the "DESCRIBE [EXTENDED] FUNCTION functionName" command.
+ * @param functionName The function to be described.
+ * @param isExtended True if "DESCRIBE EXTENDED" is used. Otherwise, false.
+ */
+private[sql] case class DescribeFunction(
+    functionName: String,
+    isExtended: Boolean) extends LogicalPlan with Command {
+
+  override def children: Seq[LogicalPlan] = Seq.empty
+  override val output: Seq[Attribute] = Seq(
+    AttributeReference("function_desc", StringType, nullable = false)())
+}
+
+/**
+ * Returned for the "SHOW FUNCTIONS" command, which will list all of the
+ * registered function list.
+ */
+private[sql] case class ShowFunctions(
+    db: Option[String], pattern: Option[String]) extends LogicalPlan with Command {
+  override def children: Seq[LogicalPlan] = Seq.empty
+  override val output: Seq[Attribute] = Seq(
+    AttributeReference("function", StringType, nullable = false)())
+}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SparkSQLParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/SparkSQLParser.scala
index e59fa6e1629005b71003d5bea161c1ce4bf1302a..ea8fce6ca9cf2b475903dd29acad97d85e0ca16d 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/SparkSQLParser.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSQLParser.scala
@@ -21,7 +21,7 @@ import scala.util.parsing.combinator.RegexParsers
 
 import org.apache.spark.sql.catalyst.AbstractSparkSQLParser
 import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference}
-import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.plans.logical.{DescribeFunction, LogicalPlan, ShowFunctions}
 import org.apache.spark.sql.execution._
 import org.apache.spark.sql.types.StringType
 
@@ -57,6 +57,10 @@ private[sql] class SparkSQLParser(fallback: String => LogicalPlan) extends Abstr
   protected val AS = Keyword("AS")
   protected val CACHE = Keyword("CACHE")
   protected val CLEAR = Keyword("CLEAR")
+  protected val DESCRIBE = Keyword("DESCRIBE")
+  protected val EXTENDED = Keyword("EXTENDED")
+  protected val FUNCTION = Keyword("FUNCTION")
+  protected val FUNCTIONS = Keyword("FUNCTIONS")
   protected val IN = Keyword("IN")
   protected val LAZY = Keyword("LAZY")
   protected val SET = Keyword("SET")
@@ -65,7 +69,8 @@ private[sql] class SparkSQLParser(fallback: String => LogicalPlan) extends Abstr
   protected val TABLES = Keyword("TABLES")
   protected val UNCACHE = Keyword("UNCACHE")
 
-  override protected lazy val start: Parser[LogicalPlan] = cache | uncache | set | show | others
+  override protected lazy val start: Parser[LogicalPlan] =
+    cache | uncache | set | show | desc | others
 
   private lazy val cache: Parser[LogicalPlan] =
     CACHE ~> LAZY.? ~ (TABLE ~> ident) ~ (AS ~> restInput).? ^^ {
@@ -85,9 +90,24 @@ private[sql] class SparkSQLParser(fallback: String => LogicalPlan) extends Abstr
       case input => SetCommandParser(input)
     }
 
+  // It can be the following patterns:
+  // SHOW FUNCTIONS;
+  // SHOW FUNCTIONS mydb.func1;
+  // SHOW FUNCTIONS func1;
+  // SHOW FUNCTIONS `mydb.a`.`func1.aa`;
   private lazy val show: Parser[LogicalPlan] =
-    SHOW ~> TABLES ~ (IN ~> ident).? ^^ {
-      case _ ~ dbName => ShowTablesCommand(dbName)
+    ( SHOW ~> TABLES ~ (IN ~> ident).? ^^ {
+        case _ ~ dbName => ShowTablesCommand(dbName)
+      }
+    | SHOW ~ FUNCTIONS ~> ((ident <~ ".").? ~ (ident | stringLit)).? ^^ {
+        case Some(f) => ShowFunctions(f._1, Some(f._2))
+        case None => ShowFunctions(None, None)
+      }
+    )
+
+  private lazy val desc: Parser[LogicalPlan] =
+    DESCRIBE ~ FUNCTION ~> EXTENDED.? ~ (ident | stringLit) ^^ {
+      case isExtended ~ functionName => DescribeFunction(functionName, isExtended.isDefined)
     }
 
   private lazy val others: Parser[LogicalPlan] =
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala
index e2c7e8006f3b11670fd7028ef03c804604728212..deeea3900c241c7422a96f67176f3e218db04e9c 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala
@@ -428,6 +428,11 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] {
         ExecutedCommand(
           RunnableDescribeCommand(resultPlan, describe.output, isExtended)) :: Nil
 
+      case logical.ShowFunctions(db, pattern) => ExecutedCommand(ShowFunctions(db, pattern)) :: Nil
+
+      case logical.DescribeFunction(function, extended) =>
+        ExecutedCommand(DescribeFunction(function, extended)) :: Nil
+
       case _ => Nil
     }
   }
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala
index bace3f8a9c8d4704c25407bde4e0741685853508..6b83025d5a153caf3ebb66840b679dd62c2d68f0 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala
@@ -24,7 +24,7 @@ import org.apache.spark.annotation.DeveloperApi
 import org.apache.spark.rdd.RDD
 import org.apache.spark.sql.catalyst.{InternalRow, CatalystTypeConverters}
 import org.apache.spark.sql.catalyst.errors.TreeNodeException
-import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference}
+import org.apache.spark.sql.catalyst.expressions.{ExpressionDescription, Expression, Attribute, AttributeReference}
 import org.apache.spark.sql.catalyst.plans.logical
 import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
 import org.apache.spark.sql.types._
@@ -298,3 +298,78 @@ case class ShowTablesCommand(databaseName: Option[String]) extends RunnableComma
     rows
   }
 }
+
+/**
+ * A command for users to list all of the registered functions.
+ * The syntax of using this command in SQL is:
+ * {{{
+ *    SHOW FUNCTIONS
+ * }}}
+ * TODO currently we are simply ignore the db
+ */
+case class ShowFunctions(db: Option[String], pattern: Option[String]) extends RunnableCommand {
+  override val output: Seq[Attribute] = {
+    val schema = StructType(
+      StructField("function", StringType, nullable = false) :: Nil)
+
+    schema.toAttributes
+  }
+
+  override def run(sqlContext: SQLContext): Seq[Row] = pattern match {
+    case Some(p) =>
+      try {
+        val regex = java.util.regex.Pattern.compile(p)
+        sqlContext.functionRegistry.listFunction().filter(regex.matcher(_).matches()).map(Row(_))
+      } catch {
+        // probably will failed in the regex that user provided, then returns empty row.
+        case _: Throwable => Seq.empty[Row]
+      }
+    case None =>
+      sqlContext.functionRegistry.listFunction().map(Row(_))
+  }
+}
+
+/**
+ * A command for users to get the usage of a registered function.
+ * The syntax of using this command in SQL is
+ * {{{
+ *   DESCRIBE FUNCTION [EXTENDED] upper;
+ * }}}
+ */
+case class DescribeFunction(
+    functionName: String,
+    isExtended: Boolean) extends RunnableCommand {
+
+  override val output: Seq[Attribute] = {
+    val schema = StructType(
+      StructField("function_desc", StringType, nullable = false) :: Nil)
+
+    schema.toAttributes
+  }
+
+  private def replaceFunctionName(usage: String, functionName: String): String = {
+    if (usage == null) {
+      "To be added."
+    } else {
+      usage.replaceAll("_FUNC_", functionName)
+    }
+  }
+
+  override def run(sqlContext: SQLContext): Seq[Row] = {
+    sqlContext.functionRegistry.lookupFunction(functionName) match {
+      case Some(info) =>
+        val result =
+          Row(s"Function: ${info.getName}") ::
+          Row(s"Class: ${info.getClassName}") ::
+          Row(s"Usage: ${replaceFunctionName(info.getUsage(), info.getName)}") :: Nil
+
+        if (isExtended) {
+          result :+ Row(s"Extended Usage:\n${replaceFunctionName(info.getExtended, info.getName)}")
+        } else {
+          result
+        }
+
+      case None => Seq(Row(s"Function: $functionName is not found."))
+    }
+  }
+}
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 cd386b7a3ecf9234eef59c25f022e5afc99499ac..8cef0b39f87dc98b414667bb6082429b0125f590 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
@@ -17,6 +17,7 @@
 
 package org.apache.spark.sql
 
+import org.apache.spark.sql.catalyst.analysis.FunctionRegistry
 import org.scalatest.BeforeAndAfterAll
 
 import java.sql.Timestamp
@@ -58,6 +59,31 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll with SQLTestUtils {
     checkAnswer(queryCoalesce, Row("1") :: Nil)
   }
 
+  test("show functions") {
+    checkAnswer(sql("SHOW functions"), FunctionRegistry.builtin.listFunction().sorted.map(Row(_)))
+  }
+
+  test("describe functions") {
+    checkExistence(sql("describe function extended upper"), true,
+      "Function: upper",
+      "Class: org.apache.spark.sql.catalyst.expressions.Upper",
+      "Usage: upper(str) - Returns str with all characters changed to uppercase",
+      "Extended Usage:",
+      "> SELECT upper('SparkSql');",
+      "'SPARKSQL'")
+
+    checkExistence(sql("describe functioN Upper"), true,
+      "Function: upper",
+      "Class: org.apache.spark.sql.catalyst.expressions.Upper",
+      "Usage: upper(str) - Returns str with all characters changed to uppercase")
+
+    checkExistence(sql("describe functioN Upper"), false,
+      "Extended Usage")
+
+    checkExistence(sql("describe functioN abcadf"), true,
+      "Function: abcadf is not found.")
+  }
+
   test("SPARK-6743: no columns from cache") {
     Seq(
       (83, 0, 38),
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUDFs.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUDFs.scala
index 54bf6bd67ff8481e8eb0cac6196ee2d5584def2a..8732e9abf8d3126b7a5ccaabb0b2f5374b6d8c5d 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUDFs.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUDFs.scala
@@ -76,8 +76,32 @@ private[hive] class HiveFunctionRegistry(underlying: analysis.FunctionRegistry)
     }
   }
 
-  override def registerFunction(name: String, builder: FunctionBuilder): Unit =
-    underlying.registerFunction(name, builder)
+  override def registerFunction(name: String, info: ExpressionInfo, builder: FunctionBuilder)
+  : Unit = underlying.registerFunction(name, info, builder)
+
+  /* List all of the registered function names. */
+  override def listFunction(): Seq[String] = {
+    val a = FunctionRegistry.getFunctionNames ++ underlying.listFunction()
+    a.toList.sorted
+  }
+
+  /* Get the class of the registered function by specified name. */
+  override def lookupFunction(name: String): Option[ExpressionInfo] = {
+    underlying.lookupFunction(name).orElse(
+    Try {
+      val info = FunctionRegistry.getFunctionInfo(name)
+      val annotation = info.getFunctionClass.getAnnotation(classOf[Description])
+      if (annotation != null) {
+        Some(new ExpressionInfo(
+          info.getFunctionClass.getCanonicalName,
+          annotation.name(),
+          annotation.value(),
+          annotation.extended()))
+      } else {
+        None
+      }
+    }.getOrElse(None))
+  }
 }
 
 private[hive] case class HiveSimpleUDF(funcWrapper: HiveFunctionWrapper, children: Seq[Expression])
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala
index efb04bf3d5097392c2261b60277dc8cddc23ea69..638b9c810372a745ad37195d65239db4ec0fc12a 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala
@@ -370,7 +370,11 @@ abstract class HiveComparisonTest
             // Check that the results match unless its an EXPLAIN query.
             val preparedHive = prepareAnswer(hiveQuery, hive)
 
-            if ((!hiveQuery.logical.isInstanceOf[ExplainCommand]) && preparedHive != catalyst) {
+            // We will ignore the ExplainCommand, ShowFunctions, DescribeFunction
+            if ((!hiveQuery.logical.isInstanceOf[ExplainCommand]) &&
+                (!hiveQuery.logical.isInstanceOf[ShowFunctions]) &&
+                (!hiveQuery.logical.isInstanceOf[DescribeFunction]) &&
+                preparedHive != catalyst) {
 
               val hivePrintOut = s"== HIVE - ${preparedHive.size} row(s) ==" +: preparedHive
               val catalystPrintOut = s"== CATALYST - ${catalyst.size} row(s) ==" +: catalyst
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala
index ff42fdefaa62a0fba7ea4cbfe7619786bdb5ce2f..013936377b24cb31237175bd6c2cd4d5326e1c50 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala
@@ -19,9 +19,11 @@ package org.apache.spark.sql.hive.execution
 
 import java.sql.{Date, Timestamp}
 
+import scala.collection.JavaConversions._
+
 import org.apache.spark.sql._
 import org.apache.spark.sql.catalyst.DefaultParserDialect
-import org.apache.spark.sql.catalyst.analysis.EliminateSubQueries
+import org.apache.spark.sql.catalyst.analysis.{FunctionRegistry, EliminateSubQueries}
 import org.apache.spark.sql.catalyst.errors.DialectException
 import org.apache.spark.sql.execution.datasources.LogicalRelation
 import org.apache.spark.sql.hive.test.TestHive
@@ -138,6 +140,50 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils {
       (1 to 6).map(_ => Row("CA", 20151)))
   }
 
+  test("show functions") {
+    val allFunctions =
+      (FunctionRegistry.builtin.listFunction().toSet[String] ++
+        org.apache.hadoop.hive.ql.exec.FunctionRegistry.getFunctionNames).toList.sorted
+    checkAnswer(sql("SHOW functions"), allFunctions.map(Row(_)))
+    checkAnswer(sql("SHOW functions abs"), Row("abs"))
+    checkAnswer(sql("SHOW functions 'abs'"), Row("abs"))
+    checkAnswer(sql("SHOW functions abc.abs"), Row("abs"))
+    checkAnswer(sql("SHOW functions `abc`.`abs`"), Row("abs"))
+    checkAnswer(sql("SHOW functions `abc`.`abs`"), Row("abs"))
+    checkAnswer(sql("SHOW functions `~`"), Row("~"))
+    checkAnswer(sql("SHOW functions `a function doens't exist`"), Nil)
+    checkAnswer(sql("SHOW functions `weekofyea.*`"), Row("weekofyear"))
+    // this probably will failed if we add more function with `sha` prefixing.
+    checkAnswer(sql("SHOW functions `sha.*`"), Row("sha") :: Row("sha1") :: Row("sha2") :: Nil)
+  }
+
+  test("describe functions") {
+    // The Spark SQL built-in functions
+    checkExistence(sql("describe function extended upper"), true,
+      "Function: upper",
+      "Class: org.apache.spark.sql.catalyst.expressions.Upper",
+      "Usage: upper(str) - Returns str with all characters changed to uppercase",
+      "Extended Usage:",
+      "> SELECT upper('SparkSql')",
+      "'SPARKSQL'")
+
+    checkExistence(sql("describe functioN Upper"), true,
+      "Function: upper",
+      "Class: org.apache.spark.sql.catalyst.expressions.Upper",
+      "Usage: upper(str) - Returns str with all characters changed to uppercase")
+
+    checkExistence(sql("describe functioN Upper"), false,
+      "Extended Usage")
+
+    checkExistence(sql("describe functioN abcadf"), true,
+      "Function: abcadf is not found.")
+
+    checkExistence(sql("describe functioN  `~`"), true,
+      "Function: ~",
+      "Class: org.apache.hadoop.hive.ql.udf.UDFOPBitNot",
+      "Usage: ~ n - Bitwise not")
+  }
+
   test("SPARK-5371: union with null and sum") {
     val df = Seq((1, 1)).toDF("c1", "c2")
     df.registerTempTable("table1")