diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
index d08c2d1cfe030be0a3902079d6e60e7febde38e7..aa0fc3e359a9b52dac628573012965133c2ecac6 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
@@ -794,7 +794,7 @@ class SQLContext(@transient val sparkContext: SparkContext)
    * indicating if a table is a temporary one or not).
    */
   def tables(): DataFrame = {
-    createDataFrame(catalog.getTables(None)).toDF("tableName", "isTemporary")
+    DataFrame(this, ShowTablesCommand(None))
   }
 
   /**
@@ -803,7 +803,7 @@ class SQLContext(@transient val sparkContext: SparkContext)
    * indicating if a table is a temporary one or not).
    */
   def tables(databaseName: String): DataFrame = {
-    createDataFrame(catalog.getTables(Some(databaseName))).toDF("tableName", "isTemporary")
+    DataFrame(this, ShowTablesCommand(Some(databaseName)))
   }
 
   /**
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 f1a4053b79113e18639868c3ca08e61177841d4d..00e19da4374a80fe6efd696dc48d20340642ce9e 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
@@ -23,7 +23,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.execution.{UncacheTableCommand, CacheTableCommand, SetCommand}
+import org.apache.spark.sql.execution.{ShowTablesCommand, UncacheTableCommand, CacheTableCommand, SetCommand}
 import org.apache.spark.sql.types.StringType
 
 
@@ -57,12 +57,15 @@ private[sql] class SparkSQLParser(fallback: String => LogicalPlan) extends Abstr
 
   protected val AS      = Keyword("AS")
   protected val CACHE   = Keyword("CACHE")
+  protected val IN      = Keyword("IN")
   protected val LAZY    = Keyword("LAZY")
   protected val SET     = Keyword("SET")
+  protected val SHOW    = Keyword("SHOW")
   protected val TABLE   = Keyword("TABLE")
+  protected val TABLES  = Keyword("TABLES")
   protected val UNCACHE = Keyword("UNCACHE")
 
-  override protected lazy val start: Parser[LogicalPlan] = cache | uncache | set | others
+  override protected lazy val start: Parser[LogicalPlan] = cache | uncache | set | show | others
 
   private lazy val cache: Parser[LogicalPlan] =
     CACHE ~> LAZY.? ~ (TABLE ~> ident) ~ (AS ~> restInput).? ^^ {
@@ -80,6 +83,11 @@ private[sql] class SparkSQLParser(fallback: String => LogicalPlan) extends Abstr
       case input => SetCommandParser(input)
     }
 
+  private lazy val show: Parser[LogicalPlan] =
+    SHOW ~> TABLES ~ (IN ~> ident).? ^^ {
+      case _ ~ dbName => ShowTablesCommand(dbName)
+    }
+
   private lazy val others: Parser[LogicalPlan] =
     wholeInput ^^ {
       case input => fallback(input)
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 2b1726ad4e89f2b0a0162b4c48b638c9bb4682b7..c6cd6eb6a22b6d2b4a39383449b8899a8c9fb0b8 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
@@ -20,7 +20,7 @@ package org.apache.spark.sql.execution
 import org.apache.spark.Logging
 import org.apache.spark.annotation.DeveloperApi
 import org.apache.spark.rdd.RDD
-import org.apache.spark.sql.types.StringType
+import org.apache.spark.sql.types.{BooleanType, StructField, StructType, StringType}
 import org.apache.spark.sql.{DataFrame, SQLConf, SQLContext}
 import org.apache.spark.sql.catalyst.errors.TreeNodeException
 import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Row, Attribute}
@@ -191,3 +191,35 @@ case class DescribeCommand(
     }
   }
 }
+
+/**
+ * A command for users to get tables in the given database.
+ * If a databaseName is not given, the current database will be used.
+ * The syntax of using this command in SQL is:
+ * {{{
+ *    SHOW TABLES [IN databaseName]
+ * }}}
+ * :: DeveloperApi ::
+ */
+@DeveloperApi
+case class ShowTablesCommand(databaseName: Option[String]) extends RunnableCommand {
+
+  // The result of SHOW TABLES has two columns, tableName and isTemporary.
+  override val output = {
+    val schema = StructType(
+      StructField("tableName", StringType, false) ::
+      StructField("isTemporary", BooleanType, false) :: Nil)
+
+    schema.toAttributes
+  }
+
+  override def run(sqlContext: SQLContext) = {
+    // Since we need to return a Seq of rows, we will call getTables directly
+    // instead of calling tables in sqlContext.
+    val rows = sqlContext.catalog.getTables(databaseName).map {
+      case (tableName, isTemporary) => Row(tableName, isTemporary)
+    }
+
+    rows
+  }
+}
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ListTablesSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ListTablesSuite.scala
index 282b98a987dd4a53665852082d1f256f8c7e8db5..f9f41eb358bd533cc42d87dc2c086eeee806c8c7 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/ListTablesSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/ListTablesSuite.scala
@@ -43,6 +43,10 @@ class ListTablesSuite extends QueryTest with BeforeAndAfter {
       tables().filter("tableName = 'ListTablesSuiteTable'"),
       Row("ListTablesSuiteTable", true))
 
+    checkAnswer(
+      sql("SHOW tables").filter("tableName = 'ListTablesSuiteTable'"),
+      Row("ListTablesSuiteTable", true))
+
     catalog.unregisterTable(Seq("ListTablesSuiteTable"))
     assert(tables().filter("tableName = 'ListTablesSuiteTable'").count() === 0)
   }
@@ -52,25 +56,32 @@ class ListTablesSuite extends QueryTest with BeforeAndAfter {
       tables("DB").filter("tableName = 'ListTablesSuiteTable'"),
       Row("ListTablesSuiteTable", true))
 
+    checkAnswer(
+      sql("show TABLES in DB").filter("tableName = 'ListTablesSuiteTable'"),
+      Row("ListTablesSuiteTable", true))
+
     catalog.unregisterTable(Seq("ListTablesSuiteTable"))
     assert(tables().filter("tableName = 'ListTablesSuiteTable'").count() === 0)
   }
 
   test("query the returned DataFrame of tables") {
-    val tableDF = tables()
-    val schema = StructType(
-      StructField("tableName", StringType, true) ::
+    val expectedSchema = StructType(
+      StructField("tableName", StringType, false) ::
       StructField("isTemporary", BooleanType, false) :: Nil)
-    assert(schema === tableDF.schema)
 
-    tableDF.registerTempTable("tables")
-    checkAnswer(
-      sql("SELECT isTemporary, tableName from tables WHERE tableName = 'ListTablesSuiteTable'"),
-      Row(true, "ListTablesSuiteTable")
-    )
-    checkAnswer(
-      tables().filter("tableName = 'tables'").select("tableName", "isTemporary"),
-      Row("tables", true))
-    dropTempTable("tables")
+    Seq(tables(), sql("SHOW TABLes")).foreach {
+      case tableDF =>
+        assert(expectedSchema === tableDF.schema)
+
+        tableDF.registerTempTable("tables")
+        checkAnswer(
+          sql("SELECT isTemporary, tableName from tables WHERE tableName = 'ListTablesSuiteTable'"),
+          Row(true, "ListTablesSuiteTable")
+        )
+        checkAnswer(
+          tables().filter("tableName = 'tables'").select("tableName", "isTemporary"),
+          Row("tables", true))
+        dropTempTable("tables")
+    }
   }
 }
diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala
index 60953576d0e37ece5da38f6189bb1473b2bdf60d..8bca4b33b3ad19edd94856de1760cb477cab6080 100644
--- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala
+++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala
@@ -121,6 +121,6 @@ class CliSuite extends FunSuite with BeforeAndAfterAll with Logging {
   }
 
   test("Single command with -e") {
-    runCliWithin(1.minute, Seq("-e", "SHOW TABLES;"))("" -> "OK")
+    runCliWithin(1.minute, Seq("-e", "SHOW DATABASES;"))("" -> "OK")
   }
 }
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala
index 87bc9fe4fe9add1ff6b0e228c6e4ff619b8f168e..0e43faa8afdaf79f78646e58ff1f9b34018527d6 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala
@@ -220,8 +220,14 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with
   }
 
   override def getTables(databaseName: Option[String]): Seq[(String, Boolean)] = {
-    val dbName = databaseName.getOrElse(hive.sessionState.getCurrentDatabase)
-    client.getAllTables(dbName).map(tableName => (tableName, false))
+    val dbName = if (!caseSensitive) {
+      if (databaseName.isDefined) Some(databaseName.get.toLowerCase) else None
+    } else {
+      databaseName
+    }
+    val db = dbName.getOrElse(hive.sessionState.getCurrentDatabase)
+
+    client.getAllTables(db).map(tableName => (tableName, false))
   }
 
   /**
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala
index 5a1825a87dadba29e28fb6625f1544533c136a5a..98263f602e9ec980d3571cfb9bb979b8f6ec0777 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala
@@ -64,7 +64,6 @@ private[hive] object HiveQl {
     "TOK_SHOWINDEXES",
     "TOK_SHOWINDEXES",
     "TOK_SHOWPARTITIONS",
-    "TOK_SHOWTABLES",
     "TOK_SHOW_TBLPROPERTIES",
 
     "TOK_LOCKTABLE",
@@ -129,6 +128,7 @@ private[hive] object HiveQl {
   // Commands that we do not need to explain.
   protected val noExplainCommands = Seq(
     "TOK_DESCTABLE",
+    "TOK_SHOWTABLES",
     "TOK_TRUNCATETABLE"     // truncate table" is a NativeCommand, does not need to explain.
   ) ++ nativeCommands
 
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/ListTablesSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/ListTablesSuite.scala
index 321b784a3f8427060ca2c7451f8681796c73e097..e12a6c21ccac49d015ab97b839f5aea62c25a018 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/ListTablesSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/ListTablesSuite.scala
@@ -49,29 +49,33 @@ class ListTablesSuite extends QueryTest with BeforeAndAfterAll {
   }
 
   test("get all tables of current database") {
-    val allTables = tables()
-    // We are using default DB.
-    checkAnswer(
-      allTables.filter("tableName = 'listtablessuitetable'"),
-      Row("listtablessuitetable", true))
-    assert(allTables.filter("tableName = 'indblisttablessuitetable'").count() === 0)
-    checkAnswer(
-      allTables.filter("tableName = 'hivelisttablessuitetable'"),
-      Row("hivelisttablessuitetable", false))
-    assert(allTables.filter("tableName = 'hiveindblisttablessuitetable'").count() === 0)
+    Seq(tables(), sql("SHOW TABLes")).foreach {
+      case allTables =>
+        // We are using default DB.
+        checkAnswer(
+          allTables.filter("tableName = 'listtablessuitetable'"),
+          Row("listtablessuitetable", true))
+        assert(allTables.filter("tableName = 'indblisttablessuitetable'").count() === 0)
+        checkAnswer(
+          allTables.filter("tableName = 'hivelisttablessuitetable'"),
+          Row("hivelisttablessuitetable", false))
+        assert(allTables.filter("tableName = 'hiveindblisttablessuitetable'").count() === 0)
+    }
   }
 
   test("getting all tables with a database name") {
-    val allTables = tables("ListTablesSuiteDB")
-    checkAnswer(
-      allTables.filter("tableName = 'listtablessuitetable'"),
-      Row("listtablessuitetable", true))
-    checkAnswer(
-      allTables.filter("tableName = 'indblisttablessuitetable'"),
-      Row("indblisttablessuitetable", true))
-    assert(allTables.filter("tableName = 'hivelisttablessuitetable'").count() === 0)
-    checkAnswer(
-      allTables.filter("tableName = 'hiveindblisttablessuitetable'"),
-      Row("hiveindblisttablessuitetable", false))
+    Seq(tables("listtablessuiteDb"), sql("SHOW TABLes in listTablesSuitedb")).foreach {
+      case allTables =>
+        checkAnswer(
+          allTables.filter("tableName = 'listtablessuitetable'"),
+          Row("listtablessuitetable", true))
+        checkAnswer(
+          allTables.filter("tableName = 'indblisttablessuitetable'"),
+          Row("indblisttablessuitetable", true))
+        assert(allTables.filter("tableName = 'hivelisttablessuitetable'").count() === 0)
+        checkAnswer(
+          allTables.filter("tableName = 'hiveindblisttablessuitetable'"),
+          Row("hiveindblisttablessuitetable", false))
+    }
   }
 }
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala
index d01dbf80ef66d0f26d6f9be1dc2843992c95ba64..955f3f51cfe9f52ca5a59b5af8a9ad8bc259a3ba 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala
@@ -62,7 +62,7 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter {
   test("SPARK-4908: concurrent hive native commands") {
     (1 to 100).par.map { _ =>
       sql("USE default")
-      sql("SHOW TABLES")
+      sql("SHOW DATABASES")
     }
   }
 
@@ -630,24 +630,24 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter {
   }
 
   test("Query Hive native command execution result") {
-    val tableName = "test_native_commands"
+    val databaseName = "test_native_commands"
 
     assertResult(0) {
-      sql(s"DROP TABLE IF EXISTS $tableName").count()
+      sql(s"DROP DATABASE IF EXISTS $databaseName").count()
     }
 
     assertResult(0) {
-      sql(s"CREATE TABLE $tableName(key INT, value STRING)").count()
+      sql(s"CREATE DATABASE $databaseName").count()
     }
 
     assert(
-      sql("SHOW TABLES")
+      sql("SHOW DATABASES")
         .select('result)
         .collect()
         .map(_.getString(0))
-        .contains(tableName))
+        .contains(databaseName))
 
-    assert(isExplanation(sql(s"EXPLAIN SELECT key, COUNT(*) FROM $tableName GROUP BY key")))
+    assert(isExplanation(sql(s"EXPLAIN SELECT key, COUNT(*) FROM src GROUP BY key")))
 
     TestHive.reset()
   }