You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by li...@apache.org on 2017/04/04 06:30:16 UTC

spark git commit: [SPARK-20067][SQL] Unify and Clean Up Desc Commands Using Catalog Interface

Repository: spark
Updated Branches:
  refs/heads/master 3bfb639cb -> 51d3c854c


[SPARK-20067][SQL] Unify and Clean Up Desc Commands Using Catalog Interface

### What changes were proposed in this pull request?

This PR is to unify and clean up the outputs of `DESC EXTENDED/FORMATTED` and `SHOW TABLE EXTENDED` by moving the logics into the Catalog interface. The output formats are improved. We also add the missing attributes. It impacts the DDL commands like `SHOW TABLE EXTENDED`, `DESC EXTENDED` and `DESC FORMATTED`.

In addition, by following what we did in Dataset API `printSchema`, we can use `treeString` to show the schema in the more readable way.

Below is the current way:
```
Schema: STRUCT<`a`: STRING (nullable = true), `b`: INT (nullable = true), `c`: STRING (nullable = true), `d`: STRING (nullable = true)>
```
After the change, it should look like
```
Schema: root
 |-- a: string (nullable = true)
 |-- b: integer (nullable = true)
 |-- c: string (nullable = true)
 |-- d: string (nullable = true)
```

### How was this patch tested?
`describe.sql` and `show-tables.sql`

Author: Xiao Li <ga...@gmail.com>

Closes #17394 from gatorsmile/descFollowUp.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/51d3c854
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/51d3c854
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/51d3c854

Branch: refs/heads/master
Commit: 51d3c854c54369aec1bfd55cefcd080dcd178d5f
Parents: 3bfb639
Author: Xiao Li <ga...@gmail.com>
Authored: Mon Apr 3 23:30:12 2017 -0700
Committer: Xiao Li <ga...@gmail.com>
Committed: Mon Apr 3 23:30:12 2017 -0700

----------------------------------------------------------------------
 .../spark/sql/catalyst/catalog/interface.scala  | 136 +++---
 .../spark/sql/execution/SparkSqlParser.scala    |   3 +-
 .../spark/sql/execution/command/tables.scala    | 124 ++----
 .../resources/sql-tests/inputs/describe.sql     |  53 ++-
 .../sql-tests/results/change-column.sql.out     |   9 +
 .../sql-tests/results/describe.sql.out          | 422 +++++++++++++++----
 .../sql-tests/results/show-tables.sql.out       |  67 +--
 .../apache/spark/sql/SQLQueryTestSuite.scala    |  19 +-
 .../sql/execution/SparkSqlParserSuite.scala     |   6 +-
 .../spark/sql/execution/command/DDLSuite.scala  |  12 -
 .../org/apache/spark/sql/jdbc/JDBCSuite.scala   |   4 -
 .../apache/spark/sql/sources/DDLTestSuite.scala | 123 ------
 .../spark/sql/sources/DataSourceTest.scala      |  56 +++
 .../sql/hive/MetastoreDataSourcesSuite.scala    |   8 +-
 .../sql/hive/execution/HiveComparisonTest.scala |   4 +-
 .../spark/sql/hive/execution/HiveDDLSuite.scala |  93 +---
 .../execution/HiveOperatorQueryableSuite.scala  |  53 ---
 .../sql/hive/execution/HiveQuerySuite.scala     |  56 ---
 .../sql/hive/execution/SQLQuerySuite.scala      | 131 ++----
 19 files changed, 642 insertions(+), 737 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/spark/blob/51d3c854/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala
index 70ed44e..3f25f9e 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala
@@ -20,6 +20,8 @@ package org.apache.spark.sql.catalyst.catalog
 import java.net.URI
 import java.util.Date
 
+import scala.collection.mutable
+
 import com.google.common.base.Objects
 
 import org.apache.spark.sql.AnalysisException
@@ -57,20 +59,25 @@ case class CatalogStorageFormat(
     properties: Map[String, String]) {
 
   override def toString: String = {
-    val serdePropsToString = CatalogUtils.maskCredentials(properties) match {
-      case props if props.isEmpty => ""
-      case props => "Properties: " + props.map(p => p._1 + "=" + p._2).mkString("[", ", ", "]")
-    }
-    val output =
-      Seq(locationUri.map("Location: " + _).getOrElse(""),
-        inputFormat.map("InputFormat: " + _).getOrElse(""),
-        outputFormat.map("OutputFormat: " + _).getOrElse(""),
-        if (compressed) "Compressed" else "",
-        serde.map("Serde: " + _).getOrElse(""),
-        serdePropsToString)
-    output.filter(_.nonEmpty).mkString("Storage(", ", ", ")")
+    toLinkedHashMap.map { case ((key, value)) =>
+      if (value.isEmpty) key else s"$key: $value"
+    }.mkString("Storage(", ", ", ")")
   }
 
+  def toLinkedHashMap: mutable.LinkedHashMap[String, String] = {
+    val map = new mutable.LinkedHashMap[String, String]()
+    locationUri.foreach(l => map.put("Location", l.toString))
+    serde.foreach(map.put("Serde Library", _))
+    inputFormat.foreach(map.put("InputFormat", _))
+    outputFormat.foreach(map.put("OutputFormat", _))
+    if (compressed) map.put("Compressed", "")
+    CatalogUtils.maskCredentials(properties) match {
+      case props if props.isEmpty => // No-op
+      case props =>
+        map.put("Properties", props.map(p => p._1 + "=" + p._2).mkString("[", ", ", "]"))
+    }
+    map
+  }
 }
 
 object CatalogStorageFormat {
@@ -91,15 +98,28 @@ case class CatalogTablePartition(
     storage: CatalogStorageFormat,
     parameters: Map[String, String] = Map.empty) {
 
-  override def toString: String = {
+  def toLinkedHashMap: mutable.LinkedHashMap[String, String] = {
+    val map = new mutable.LinkedHashMap[String, String]()
     val specString = spec.map { case (k, v) => s"$k=$v" }.mkString(", ")
-    val output =
-      Seq(
-        s"Partition Values: [$specString]",
-        s"$storage",
-        s"Partition Parameters:{${parameters.map(p => p._1 + "=" + p._2).mkString(", ")}}")
+    map.put("Partition Values", s"[$specString]")
+    map ++= storage.toLinkedHashMap
+    if (parameters.nonEmpty) {
+      map.put("Partition Parameters", s"{${parameters.map(p => p._1 + "=" + p._2).mkString(", ")}}")
+    }
+    map
+  }
 
-    output.filter(_.nonEmpty).mkString("CatalogPartition(\n\t", "\n\t", ")")
+  override def toString: String = {
+    toLinkedHashMap.map { case ((key, value)) =>
+      if (value.isEmpty) key else s"$key: $value"
+    }.mkString("CatalogPartition(\n\t", "\n\t", ")")
+  }
+
+  /** Readable string representation for the CatalogTablePartition. */
+  def simpleString: String = {
+    toLinkedHashMap.map { case ((key, value)) =>
+      if (value.isEmpty) key else s"$key: $value"
+    }.mkString("", "\n", "")
   }
 
   /** Return the partition location, assuming it is specified. */
@@ -154,6 +174,14 @@ case class BucketSpec(
     }
     s"$numBuckets buckets, $bucketString$sortString"
   }
+
+  def toLinkedHashMap: mutable.LinkedHashMap[String, String] = {
+    mutable.LinkedHashMap[String, String](
+      "Num Buckets" -> numBuckets.toString,
+      "Bucket Columns" -> bucketColumnNames.map(quoteIdentifier).mkString("[", ", ", "]"),
+      "Sort Columns" -> sortColumnNames.map(quoteIdentifier).mkString("[", ", ", "]")
+    )
+  }
 }
 
 /**
@@ -261,40 +289,50 @@ case class CatalogTable(
       locationUri, inputFormat, outputFormat, serde, compressed, properties))
   }
 
-  override def toString: String = {
+
+  def toLinkedHashMap: mutable.LinkedHashMap[String, String] = {
+    val map = new mutable.LinkedHashMap[String, String]()
     val tableProperties = properties.map(p => p._1 + "=" + p._2).mkString("[", ", ", "]")
     val partitionColumns = partitionColumnNames.map(quoteIdentifier).mkString("[", ", ", "]")
-    val bucketStrings = bucketSpec match {
-      case Some(BucketSpec(numBuckets, bucketColumnNames, sortColumnNames)) =>
-        val bucketColumnsString = bucketColumnNames.map(quoteIdentifier).mkString("[", ", ", "]")
-        val sortColumnsString = sortColumnNames.map(quoteIdentifier).mkString("[", ", ", "]")
-        Seq(
-          s"Num Buckets: $numBuckets",
-          if (bucketColumnNames.nonEmpty) s"Bucket Columns: $bucketColumnsString" else "",
-          if (sortColumnNames.nonEmpty) s"Sort Columns: $sortColumnsString" else ""
-        )
-
-      case _ => Nil
+
+    identifier.database.foreach(map.put("Database", _))
+    map.put("Table", identifier.table)
+    if (owner.nonEmpty) map.put("Owner", owner)
+    map.put("Created", new Date(createTime).toString)
+    map.put("Last Access", new Date(lastAccessTime).toString)
+    map.put("Type", tableType.name)
+    provider.foreach(map.put("Provider", _))
+    bucketSpec.foreach(map ++= _.toLinkedHashMap)
+    comment.foreach(map.put("Comment", _))
+    if (tableType == CatalogTableType.VIEW) {
+      viewText.foreach(map.put("View Text", _))
+      viewDefaultDatabase.foreach(map.put("View Default Database", _))
+      if (viewQueryColumnNames.nonEmpty) {
+        map.put("View Query Output Columns", viewQueryColumnNames.mkString("[", ", ", "]"))
+      }
     }
 
-    val output =
-      Seq(s"Table: ${identifier.quotedString}",
-        if (owner.nonEmpty) s"Owner: $owner" else "",
-        s"Created: ${new Date(createTime).toString}",
-        s"Last Access: ${new Date(lastAccessTime).toString}",
-        s"Type: ${tableType.name}",
-        if (schema.nonEmpty) s"Schema: ${schema.mkString("[", ", ", "]")}" else "",
-        if (provider.isDefined) s"Provider: ${provider.get}" else "",
-        if (partitionColumnNames.nonEmpty) s"Partition Columns: $partitionColumns" else ""
-      ) ++ bucketStrings ++ Seq(
-        viewText.map("View: " + _).getOrElse(""),
-        comment.map("Comment: " + _).getOrElse(""),
-        if (properties.nonEmpty) s"Properties: $tableProperties" else "",
-        if (stats.isDefined) s"Statistics: ${stats.get.simpleString}" else "",
-        s"$storage",
-        if (tracksPartitionsInCatalog) "Partition Provider: Catalog" else "")
-
-    output.filter(_.nonEmpty).mkString("CatalogTable(\n\t", "\n\t", ")")
+    if (properties.nonEmpty) map.put("Properties", tableProperties)
+    stats.foreach(s => map.put("Statistics", s.simpleString))
+    map ++= storage.toLinkedHashMap
+    if (tracksPartitionsInCatalog) map.put("Partition Provider", "Catalog")
+    if (partitionColumnNames.nonEmpty) map.put("Partition Columns", partitionColumns)
+    if (schema.nonEmpty) map.put("Schema", schema.treeString)
+
+    map
+  }
+
+  override def toString: String = {
+    toLinkedHashMap.map { case ((key, value)) =>
+      if (value.isEmpty) key else s"$key: $value"
+    }.mkString("CatalogTable(\n", "\n", ")")
+  }
+
+  /** Readable string representation for the CatalogTable. */
+  def simpleString: String = {
+    toLinkedHashMap.map { case ((key, value)) =>
+      if (value.isEmpty) key else s"$key: $value"
+    }.mkString("", "\n", "")
   }
 }
 

http://git-wip-us.apache.org/repos/asf/spark/blob/51d3c854/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala
----------------------------------------------------------------------
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 d4f23f9..80afb59 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
@@ -322,8 +322,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder {
       DescribeTableCommand(
         visitTableIdentifier(ctx.tableIdentifier),
         partitionSpec,
-        ctx.EXTENDED != null,
-        ctx.FORMATTED != null)
+        ctx.EXTENDED != null || ctx.FORMATTED != null)
     }
   }
 

http://git-wip-us.apache.org/repos/asf/spark/blob/51d3c854/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala
index c7aeef0..ebf03e1 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala
@@ -500,8 +500,7 @@ case class TruncateTableCommand(
 case class DescribeTableCommand(
     table: TableIdentifier,
     partitionSpec: TablePartitionSpec,
-    isExtended: Boolean,
-    isFormatted: Boolean)
+    isExtended: Boolean)
   extends RunnableCommand {
 
   override val output: Seq[Attribute] = Seq(
@@ -536,14 +535,12 @@ case class DescribeTableCommand(
 
       describePartitionInfo(metadata, result)
 
-      if (partitionSpec.isEmpty) {
-        if (isExtended) {
-          describeExtendedTableInfo(metadata, result)
-        } else if (isFormatted) {
-          describeFormattedTableInfo(metadata, result)
-        }
-      } else {
+      if (partitionSpec.nonEmpty) {
+        // Outputs the partition-specific info for the DDL command:
+        // "DESCRIBE [EXTENDED|FORMATTED] table_name PARTITION (partitionVal*)"
         describeDetailedPartitionInfo(sparkSession, catalog, metadata, result)
+      } else if (isExtended) {
+        describeFormattedTableInfo(metadata, result)
       }
     }
 
@@ -553,76 +550,20 @@ case class DescribeTableCommand(
   private def describePartitionInfo(table: CatalogTable, buffer: ArrayBuffer[Row]): Unit = {
     if (table.partitionColumnNames.nonEmpty) {
       append(buffer, "# Partition Information", "", "")
-      append(buffer, s"# ${output.head.name}", output(1).name, output(2).name)
       describeSchema(table.partitionSchema, buffer)
     }
   }
 
-  private def describeExtendedTableInfo(table: CatalogTable, buffer: ArrayBuffer[Row]): Unit = {
-    append(buffer, "", "", "")
-    append(buffer, "# Detailed Table Information", table.toString, "")
-  }
-
   private def describeFormattedTableInfo(table: CatalogTable, buffer: ArrayBuffer[Row]): Unit = {
+    // The following information has been already shown in the previous outputs
+    val excludedTableInfo = Seq(
+      "Partition Columns",
+      "Schema"
+    )
     append(buffer, "", "", "")
     append(buffer, "# Detailed Table Information", "", "")
-    append(buffer, "Database:", table.database, "")
-    append(buffer, "Owner:", table.owner, "")
-    append(buffer, "Created:", new Date(table.createTime).toString, "")
-    append(buffer, "Last Access:", new Date(table.lastAccessTime).toString, "")
-    append(buffer, "Location:", table.storage.locationUri.map(CatalogUtils.URIToString(_))
-      .getOrElse(""), "")
-    append(buffer, "Table Type:", table.tableType.name, "")
-    append(buffer, "Comment:", table.comment.getOrElse(""), "")
-    table.stats.foreach(s => append(buffer, "Statistics:", s.simpleString, ""))
-
-    append(buffer, "Table Parameters:", "", "")
-    table.properties.foreach { case (key, value) =>
-      append(buffer, s"  $key", value, "")
-    }
-
-    describeStorageInfo(table, buffer)
-
-    if (table.tableType == CatalogTableType.VIEW) describeViewInfo(table, buffer)
-
-    if (DDLUtils.isDatasourceTable(table) && table.tracksPartitionsInCatalog) {
-      append(buffer, "Partition Provider:", "Catalog", "")
-    }
-  }
-
-  private def describeStorageInfo(metadata: CatalogTable, buffer: ArrayBuffer[Row]): Unit = {
-    append(buffer, "", "", "")
-    append(buffer, "# Storage Information", "", "")
-    metadata.storage.serde.foreach(serdeLib => append(buffer, "SerDe Library:", serdeLib, ""))
-    metadata.storage.inputFormat.foreach(format => append(buffer, "InputFormat:", format, ""))
-    metadata.storage.outputFormat.foreach(format => append(buffer, "OutputFormat:", format, ""))
-    append(buffer, "Compressed:", if (metadata.storage.compressed) "Yes" else "No", "")
-    describeBucketingInfo(metadata, buffer)
-
-    append(buffer, "Storage Desc Parameters:", "", "")
-    val maskedProperties = CatalogUtils.maskCredentials(metadata.storage.properties)
-    maskedProperties.foreach { case (key, value) =>
-      append(buffer, s"  $key", value, "")
-    }
-  }
-
-  private def describeViewInfo(metadata: CatalogTable, buffer: ArrayBuffer[Row]): Unit = {
-    append(buffer, "", "", "")
-    append(buffer, "# View Information", "", "")
-    append(buffer, "View Text:", metadata.viewText.getOrElse(""), "")
-    append(buffer, "View Default Database:", metadata.viewDefaultDatabase.getOrElse(""), "")
-    append(buffer, "View Query Output Columns:",
-      metadata.viewQueryColumnNames.mkString("[", ", ", "]"), "")
-  }
-
-  private def describeBucketingInfo(metadata: CatalogTable, buffer: ArrayBuffer[Row]): Unit = {
-    metadata.bucketSpec match {
-      case Some(BucketSpec(numBuckets, bucketColumnNames, sortColumnNames)) =>
-        append(buffer, "Num Buckets:", numBuckets.toString, "")
-        append(buffer, "Bucket Columns:", bucketColumnNames.mkString("[", ", ", "]"), "")
-        append(buffer, "Sort Columns:", sortColumnNames.mkString("[", ", ", "]"), "")
-
-      case _ =>
+    table.toLinkedHashMap.filterKeys(!excludedTableInfo.contains(_)).foreach {
+      s => append(buffer, s._1, s._2, "")
     }
   }
 
@@ -637,21 +578,7 @@ case class DescribeTableCommand(
     }
     DDLUtils.verifyPartitionProviderIsHive(spark, metadata, "DESC PARTITION")
     val partition = catalog.getPartition(table, partitionSpec)
-    if (isExtended) {
-      describeExtendedDetailedPartitionInfo(table, metadata, partition, result)
-    } else if (isFormatted) {
-      describeFormattedDetailedPartitionInfo(table, metadata, partition, result)
-      describeStorageInfo(metadata, result)
-    }
-  }
-
-  private def describeExtendedDetailedPartitionInfo(
-      tableIdentifier: TableIdentifier,
-      table: CatalogTable,
-      partition: CatalogTablePartition,
-      buffer: ArrayBuffer[Row]): Unit = {
-    append(buffer, "", "", "")
-    append(buffer, "Detailed Partition Information " + partition.toString, "", "")
+    if (isExtended) describeFormattedDetailedPartitionInfo(table, metadata, partition, result)
   }
 
   private def describeFormattedDetailedPartitionInfo(
@@ -661,18 +588,21 @@ case class DescribeTableCommand(
       buffer: ArrayBuffer[Row]): Unit = {
     append(buffer, "", "", "")
     append(buffer, "# Detailed Partition Information", "", "")
-    append(buffer, "Partition Value:", s"[${partition.spec.values.mkString(", ")}]", "")
-    append(buffer, "Database:", table.database, "")
-    append(buffer, "Table:", tableIdentifier.table, "")
-    append(buffer, "Location:", partition.storage.locationUri.map(CatalogUtils.URIToString(_))
-      .getOrElse(""), "")
-    append(buffer, "Partition Parameters:", "", "")
-    partition.parameters.foreach { case (key, value) =>
-      append(buffer, s"  $key", value, "")
+    append(buffer, "Database", table.database, "")
+    append(buffer, "Table", tableIdentifier.table, "")
+    partition.toLinkedHashMap.foreach(s => append(buffer, s._1, s._2, ""))
+    append(buffer, "", "", "")
+    append(buffer, "# Storage Information", "", "")
+    table.bucketSpec match {
+      case Some(spec) =>
+        spec.toLinkedHashMap.foreach(s => append(buffer, s._1, s._2, ""))
+      case _ =>
     }
+    table.storage.toLinkedHashMap.foreach(s => append(buffer, s._1, s._2, ""))
   }
 
   private def describeSchema(schema: StructType, buffer: ArrayBuffer[Row]): Unit = {
+    append(buffer, s"# ${output.head.name}", output(1).name, output(2).name)
     schema.foreach { column =>
       append(buffer, column.name, column.dataType.simpleString, column.getComment().orNull)
     }
@@ -728,7 +658,7 @@ case class ShowTablesCommand(
         val tableName = tableIdent.table
         val isTemp = catalog.isTemporaryTable(tableIdent)
         if (isExtended) {
-          val information = catalog.getTempViewOrPermanentTableMetadata(tableIdent).toString
+          val information = catalog.getTempViewOrPermanentTableMetadata(tableIdent).simpleString
           Row(database, tableName, isTemp, s"$information\n")
         } else {
           Row(database, tableName, isTemp)
@@ -745,7 +675,7 @@ case class ShowTablesCommand(
       val database = table.database.getOrElse("")
       val tableName = table.table
       val isTemp = catalog.isTemporaryTable(table)
-      val information = partition.toString
+      val information = partition.simpleString
       Seq(Row(database, tableName, isTemp, s"$information\n"))
     }
   }

http://git-wip-us.apache.org/repos/asf/spark/blob/51d3c854/sql/core/src/test/resources/sql-tests/inputs/describe.sql
----------------------------------------------------------------------
diff --git a/sql/core/src/test/resources/sql-tests/inputs/describe.sql b/sql/core/src/test/resources/sql-tests/inputs/describe.sql
index 56f3281..6de4cf0 100644
--- a/sql/core/src/test/resources/sql-tests/inputs/describe.sql
+++ b/sql/core/src/test/resources/sql-tests/inputs/describe.sql
@@ -1,10 +1,23 @@
-CREATE TABLE t (a STRING, b INT, c STRING, d STRING) USING parquet PARTITIONED BY (c, d) COMMENT 'table_comment';
+CREATE TABLE t (a STRING, b INT, c STRING, d STRING) USING parquet
+  PARTITIONED BY (c, d) CLUSTERED BY (a) SORTED BY (b ASC) INTO 2 BUCKETS
+  COMMENT 'table_comment';
+
+CREATE TEMPORARY VIEW temp_v AS SELECT * FROM t;
+
+CREATE TEMPORARY VIEW temp_Data_Source_View
+  USING org.apache.spark.sql.sources.DDLScanSource
+  OPTIONS (
+    From '1',
+    To '10',
+    Table 'test1');
+
+CREATE VIEW v AS SELECT * FROM t;
 
 ALTER TABLE t ADD PARTITION (c='Us', d=1);
 
 DESCRIBE t;
 
-DESC t;
+DESC default.t;
 
 DESC TABLE t;
 
@@ -27,5 +40,39 @@ DESC t PARTITION (c='Us');
 -- ParseException: PARTITION specification is incomplete
 DESC t PARTITION (c='Us', d);
 
--- DROP TEST TABLE
+-- DESC Temp View
+
+DESC temp_v;
+
+DESC TABLE temp_v;
+
+DESC FORMATTED temp_v;
+
+DESC EXTENDED temp_v;
+
+DESC temp_Data_Source_View;
+
+-- AnalysisException DESC PARTITION is not allowed on a temporary view
+DESC temp_v PARTITION (c='Us', d=1);
+
+-- DESC Persistent View
+
+DESC v;
+
+DESC TABLE v;
+
+DESC FORMATTED v;
+
+DESC EXTENDED v;
+
+-- AnalysisException DESC PARTITION is not allowed on a view
+DESC v PARTITION (c='Us', d=1);
+
+-- DROP TEST TABLES/VIEWS
 DROP TABLE t;
+
+DROP VIEW temp_v;
+
+DROP VIEW temp_Data_Source_View;
+
+DROP VIEW v;

http://git-wip-us.apache.org/repos/asf/spark/blob/51d3c854/sql/core/src/test/resources/sql-tests/results/change-column.sql.out
----------------------------------------------------------------------
diff --git a/sql/core/src/test/resources/sql-tests/results/change-column.sql.out b/sql/core/src/test/resources/sql-tests/results/change-column.sql.out
index ba8bc93..678a3f0 100644
--- a/sql/core/src/test/resources/sql-tests/results/change-column.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/change-column.sql.out
@@ -15,6 +15,7 @@ DESC test_change
 -- !query 1 schema
 struct<col_name:string,data_type:string,comment:string>
 -- !query 1 output
+# col_name          	data_type           	comment             
 a                   	int                 	                    
 b                   	string              	                    
 c                   	int
@@ -34,6 +35,7 @@ DESC test_change
 -- !query 3 schema
 struct<col_name:string,data_type:string,comment:string>
 -- !query 3 output
+# col_name          	data_type           	comment             
 a                   	int                 	                    
 b                   	string              	                    
 c                   	int
@@ -53,6 +55,7 @@ DESC test_change
 -- !query 5 schema
 struct<col_name:string,data_type:string,comment:string>
 -- !query 5 output
+# col_name          	data_type           	comment             
 a                   	int                 	                    
 b                   	string              	                    
 c                   	int
@@ -91,6 +94,7 @@ DESC test_change
 -- !query 8 schema
 struct<col_name:string,data_type:string,comment:string>
 -- !query 8 output
+# col_name          	data_type           	comment             
 a                   	int                 	                    
 b                   	string              	                    
 c                   	int
@@ -125,6 +129,7 @@ DESC test_change
 -- !query 12 schema
 struct<col_name:string,data_type:string,comment:string>
 -- !query 12 output
+# col_name          	data_type           	comment             
 a                   	int                 	this is column a    
 b                   	string              	#*02?`              
 c                   	int
@@ -143,6 +148,7 @@ DESC test_change
 -- !query 14 schema
 struct<col_name:string,data_type:string,comment:string>
 -- !query 14 output
+# col_name          	data_type           	comment             
 a                   	int                 	this is column a    
 b                   	string              	#*02?`              
 c                   	int
@@ -162,6 +168,7 @@ DESC test_change
 -- !query 16 schema
 struct<col_name:string,data_type:string,comment:string>
 -- !query 16 output
+# col_name          	data_type           	comment             
 a                   	int                 	this is column a    
 b                   	string              	#*02?`              
 c                   	int
@@ -186,6 +193,7 @@ DESC test_change
 -- !query 18 schema
 struct<col_name:string,data_type:string,comment:string>
 -- !query 18 output
+# col_name          	data_type           	comment             
 a                   	int                 	this is column a    
 b                   	string              	#*02?`              
 c                   	int
@@ -229,6 +237,7 @@ DESC test_change
 -- !query 23 schema
 struct<col_name:string,data_type:string,comment:string>
 -- !query 23 output
+# col_name          	data_type           	comment             
 a                   	int                 	this is column A    
 b                   	string              	#*02?`              
 c                   	int

http://git-wip-us.apache.org/repos/asf/spark/blob/51d3c854/sql/core/src/test/resources/sql-tests/results/describe.sql.out
----------------------------------------------------------------------
diff --git a/sql/core/src/test/resources/sql-tests/results/describe.sql.out b/sql/core/src/test/resources/sql-tests/results/describe.sql.out
index 422d548..de10b29 100644
--- a/sql/core/src/test/resources/sql-tests/results/describe.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/describe.sql.out
@@ -1,9 +1,11 @@
 -- Automatically generated by SQLQueryTestSuite
--- Number of queries: 14
+-- Number of queries: 31
 
 
 -- !query 0
-CREATE TABLE t (a STRING, b INT, c STRING, d STRING) USING parquet PARTITIONED BY (c, d) COMMENT 'table_comment'
+CREATE TABLE t (a STRING, b INT, c STRING, d STRING) USING parquet
+  PARTITIONED BY (c, d) CLUSTERED BY (a) SORTED BY (b ASC) INTO 2 BUCKETS
+  COMMENT 'table_comment'
 -- !query 0 schema
 struct<>
 -- !query 0 output
@@ -11,7 +13,7 @@ struct<>
 
 
 -- !query 1
-ALTER TABLE t ADD PARTITION (c='Us', d=1)
+CREATE TEMPORARY VIEW temp_v AS SELECT * FROM t
 -- !query 1 schema
 struct<>
 -- !query 1 output
@@ -19,187 +21,239 @@ struct<>
 
 
 -- !query 2
-DESCRIBE t
+CREATE TEMPORARY VIEW temp_Data_Source_View
+  USING org.apache.spark.sql.sources.DDLScanSource
+  OPTIONS (
+    From '1',
+    To '10',
+    Table 'test1')
 -- !query 2 schema
-struct<col_name:string,data_type:string,comment:string>
+struct<>
 -- !query 2 output
-# Partition Information	                    	                    
+
+
+
+-- !query 3
+CREATE VIEW v AS SELECT * FROM t
+-- !query 3 schema
+struct<>
+-- !query 3 output
+
+
+
+-- !query 4
+ALTER TABLE t ADD PARTITION (c='Us', d=1)
+-- !query 4 schema
+struct<>
+-- !query 4 output
+
+
+
+-- !query 5
+DESCRIBE t
+-- !query 5 schema
+struct<col_name:string,data_type:string,comment:string>
+-- !query 5 output
 # col_name          	data_type           	comment             
 a                   	string              	                    
 b                   	int                 	                    
 c                   	string              	                    
-c                   	string              	                    
 d                   	string              	                    
+# Partition Information	                    	                    
+# col_name          	data_type           	comment             
+c                   	string              	                    
 d                   	string
 
 
--- !query 3
-DESC t
--- !query 3 schema
+-- !query 6
+DESC default.t
+-- !query 6 schema
 struct<col_name:string,data_type:string,comment:string>
--- !query 3 output
-# Partition Information	                    	                    
+-- !query 6 output
 # col_name          	data_type           	comment             
 a                   	string              	                    
 b                   	int                 	                    
 c                   	string              	                    
-c                   	string              	                    
 d                   	string              	                    
+# Partition Information	                    	                    
+# col_name          	data_type           	comment             
+c                   	string              	                    
 d                   	string
 
 
--- !query 4
+-- !query 7
 DESC TABLE t
--- !query 4 schema
+-- !query 7 schema
 struct<col_name:string,data_type:string,comment:string>
--- !query 4 output
-# Partition Information	                    	                    
+-- !query 7 output
 # col_name          	data_type           	comment             
 a                   	string              	                    
 b                   	int                 	                    
 c                   	string              	                    
-c                   	string              	                    
 d                   	string              	                    
+# Partition Information	                    	                    
+# col_name          	data_type           	comment             
+c                   	string              	                    
 d                   	string
 
 
--- !query 5
+-- !query 8
 DESC FORMATTED t
--- !query 5 schema
+-- !query 8 schema
 struct<col_name:string,data_type:string,comment:string>
--- !query 5 output
-# Detailed Table Information	                    	                    
-# Partition Information	                    	                    
-# Storage Information	                    	                    
+-- !query 8 output
 # col_name          	data_type           	comment             
-Comment:            	table_comment       	                    
-Compressed:         	No                  	                    
-Created: 
-Database:           	default             	                    
-Last Access: 
-Location: sql/core/spark-warehouse/t	                    
-Owner:              	                    	                    
-Partition Provider: 	Catalog             	                    
-Storage Desc Parameters:	                    	                    
-Table Parameters:   	                    	                    
-Table Type:         	MANAGED             	                    
 a                   	string              	                    
 b                   	int                 	                    
 c                   	string              	                    
+d                   	string              	                    
+# Partition Information	                    	                    
+# col_name          	data_type           	comment             
 c                   	string              	                    
 d                   	string              	                    
-d                   	string
+                    	                    	                    
+# Detailed Table Information	                    	                    
+Database            	default             	                    
+Table               	t                   	                    
+Created [not included in comparison]
+Last Access [not included in comparison]
+Type                	MANAGED             	                    
+Provider            	parquet             	                    
+Num Buckets         	2                   	                    
+Bucket Columns      	[`a`]               	                    
+Sort Columns        	[`b`]               	                    
+Comment             	table_comment       	                    
+Location [not included in comparison]sql/core/spark-warehouse/t	                    
+Partition Provider  	Catalog
 
 
--- !query 6
+-- !query 9
 DESC EXTENDED t
--- !query 6 schema
+-- !query 9 schema
 struct<col_name:string,data_type:string,comment:string>
--- !query 6 output
-# Detailed Table Information	CatalogTable(
-	Table: `default`.`t`
-	Created: 
-	Last Access: 
-	Type: MANAGED
-	Schema: [StructField(a,StringType,true), StructField(b,IntegerType,true), StructField(c,StringType,true), StructField(d,StringType,true)]
-	Provider: parquet
-	Partition Columns: [`c`, `d`]
-	Comment: table_comment
-	Storage(Location: sql/core/spark-warehouse/t)
-	Partition Provider: Catalog)	                    
-# Partition Information	                    	                    
+-- !query 9 output
 # col_name          	data_type           	comment             
 a                   	string              	                    
 b                   	int                 	                    
 c                   	string              	                    
+d                   	string              	                    
+# Partition Information	                    	                    
+# col_name          	data_type           	comment             
 c                   	string              	                    
 d                   	string              	                    
-d                   	string
+                    	                    	                    
+# Detailed Table Information	                    	                    
+Database            	default             	                    
+Table               	t                   	                    
+Created [not included in comparison]
+Last Access [not included in comparison]
+Type                	MANAGED             	                    
+Provider            	parquet             	                    
+Num Buckets         	2                   	                    
+Bucket Columns      	[`a`]               	                    
+Sort Columns        	[`b`]               	                    
+Comment             	table_comment       	                    
+Location [not included in comparison]sql/core/spark-warehouse/t	                    
+Partition Provider  	Catalog
 
 
--- !query 7
+-- !query 10
 DESC t PARTITION (c='Us', d=1)
--- !query 7 schema
+-- !query 10 schema
 struct<col_name:string,data_type:string,comment:string>
--- !query 7 output
-# Partition Information	                    	                    
+-- !query 10 output
 # col_name          	data_type           	comment             
 a                   	string              	                    
 b                   	int                 	                    
 c                   	string              	                    
-c                   	string              	                    
 d                   	string              	                    
+# Partition Information	                    	                    
+# col_name          	data_type           	comment             
+c                   	string              	                    
 d                   	string
 
 
--- !query 8
+-- !query 11
 DESC EXTENDED t PARTITION (c='Us', d=1)
--- !query 8 schema
+-- !query 11 schema
 struct<col_name:string,data_type:string,comment:string>
--- !query 8 output
-# Partition Information	                    	                    
+-- !query 11 output
 # col_name          	data_type           	comment             
-Detailed Partition Information CatalogPartition(
-	Partition Values: [c=Us, d=1]
-	Storage(Location: sql/core/spark-warehouse/t/c=Us/d=1)
-	Partition Parameters:{})	                    	                    
 a                   	string              	                    
 b                   	int                 	                    
 c                   	string              	                    
+d                   	string              	                    
+# Partition Information	                    	                    
+# col_name          	data_type           	comment             
 c                   	string              	                    
 d                   	string              	                    
-d                   	string
+                    	                    	                    
+# Detailed Partition Information	                    	                    
+Database            	default             	                    
+Table               	t                   	                    
+Partition Values    	[c=Us, d=1]         	                    
+Location [not included in comparison]sql/core/spark-warehouse/t/c=Us/d=1	                    
+                    	                    	                    
+# Storage Information	                    	                    
+Num Buckets         	2                   	                    
+Bucket Columns      	[`a`]               	                    
+Sort Columns        	[`b`]               	                    
+Location [not included in comparison]sql/core/spark-warehouse/t
 
 
--- !query 9
+-- !query 12
 DESC FORMATTED t PARTITION (c='Us', d=1)
--- !query 9 schema
+-- !query 12 schema
 struct<col_name:string,data_type:string,comment:string>
--- !query 9 output
-# Detailed Partition Information	                    	                    
-# Partition Information	                    	                    
-# Storage Information	                    	                    
+-- !query 12 output
 # col_name          	data_type           	comment             
-Compressed:         	No                  	                    
-Database:           	default             	                    
-Location: sql/core/spark-warehouse/t/c=Us/d=1	                    
-Partition Parameters:	                    	                    
-Partition Value:    	[Us, 1]             	                    
-Storage Desc Parameters:	                    	                    
-Table:              	t                   	                    
 a                   	string              	                    
 b                   	int                 	                    
 c                   	string              	                    
+d                   	string              	                    
+# Partition Information	                    	                    
+# col_name          	data_type           	comment             
 c                   	string              	                    
 d                   	string              	                    
-d                   	string
+                    	                    	                    
+# Detailed Partition Information	                    	                    
+Database            	default             	                    
+Table               	t                   	                    
+Partition Values    	[c=Us, d=1]         	                    
+Location [not included in comparison]sql/core/spark-warehouse/t/c=Us/d=1	                    
+                    	                    	                    
+# Storage Information	                    	                    
+Num Buckets         	2                   	                    
+Bucket Columns      	[`a`]               	                    
+Sort Columns        	[`b`]               	                    
+Location [not included in comparison]sql/core/spark-warehouse/t
 
 
--- !query 10
+-- !query 13
 DESC t PARTITION (c='Us', d=2)
--- !query 10 schema
+-- !query 13 schema
 struct<>
--- !query 10 output
+-- !query 13 output
 org.apache.spark.sql.catalyst.analysis.NoSuchPartitionException
 Partition not found in table 't' database 'default':
 c -> Us
 d -> 2;
 
 
--- !query 11
+-- !query 14
 DESC t PARTITION (c='Us')
--- !query 11 schema
+-- !query 14 schema
 struct<>
--- !query 11 output
+-- !query 14 output
 org.apache.spark.sql.AnalysisException
 Partition spec is invalid. The spec (c) must match the partition spec (c, d) defined in table '`default`.`t`';
 
 
--- !query 12
+-- !query 15
 DESC t PARTITION (c='Us', d)
--- !query 12 schema
+-- !query 15 schema
 struct<>
--- !query 12 output
+-- !query 15 output
 org.apache.spark.sql.catalyst.parser.ParseException
 
 PARTITION specification is incomplete: `d`(line 1, pos 0)
@@ -209,9 +263,193 @@ DESC t PARTITION (c='Us', d)
 ^^^
 
 
--- !query 13
+-- !query 16
+DESC temp_v
+-- !query 16 schema
+struct<col_name:string,data_type:string,comment:string>
+-- !query 16 output
+# col_name          	data_type           	comment             
+a                   	string              	                    
+b                   	int                 	                    
+c                   	string              	                    
+d                   	string
+
+
+-- !query 17
+DESC TABLE temp_v
+-- !query 17 schema
+struct<col_name:string,data_type:string,comment:string>
+-- !query 17 output
+# col_name          	data_type           	comment             
+a                   	string              	                    
+b                   	int                 	                    
+c                   	string              	                    
+d                   	string
+
+
+-- !query 18
+DESC FORMATTED temp_v
+-- !query 18 schema
+struct<col_name:string,data_type:string,comment:string>
+-- !query 18 output
+# col_name          	data_type           	comment             
+a                   	string              	                    
+b                   	int                 	                    
+c                   	string              	                    
+d                   	string
+
+
+-- !query 19
+DESC EXTENDED temp_v
+-- !query 19 schema
+struct<col_name:string,data_type:string,comment:string>
+-- !query 19 output
+# col_name          	data_type           	comment             
+a                   	string              	                    
+b                   	int                 	                    
+c                   	string              	                    
+d                   	string
+
+
+-- !query 20
+DESC temp_Data_Source_View
+-- !query 20 schema
+struct<col_name:string,data_type:string,comment:string>
+-- !query 20 output
+# col_name          	data_type           	comment             
+intType             	int                 	test comment test1  
+stringType          	string              	                    
+dateType            	date                	                    
+timestampType       	timestamp           	                    
+doubleType          	double              	                    
+bigintType          	bigint              	                    
+tinyintType         	tinyint             	                    
+decimalType         	decimal(10,0)       	                    
+fixedDecimalType    	decimal(5,1)        	                    
+binaryType          	binary              	                    
+booleanType         	boolean             	                    
+smallIntType        	smallint            	                    
+floatType           	float               	                    
+mapType             	map<string,string>  	                    
+arrayType           	array<string>       	                    
+structType          	struct<f1:string,f2:int>
+
+
+-- !query 21
+DESC temp_v PARTITION (c='Us', d=1)
+-- !query 21 schema
+struct<>
+-- !query 21 output
+org.apache.spark.sql.AnalysisException
+DESC PARTITION is not allowed on a temporary view: temp_v;
+
+
+-- !query 22
+DESC v
+-- !query 22 schema
+struct<col_name:string,data_type:string,comment:string>
+-- !query 22 output
+# col_name          	data_type           	comment             
+a                   	string              	                    
+b                   	int                 	                    
+c                   	string              	                    
+d                   	string
+
+
+-- !query 23
+DESC TABLE v
+-- !query 23 schema
+struct<col_name:string,data_type:string,comment:string>
+-- !query 23 output
+# col_name          	data_type           	comment             
+a                   	string              	                    
+b                   	int                 	                    
+c                   	string              	                    
+d                   	string
+
+
+-- !query 24
+DESC FORMATTED v
+-- !query 24 schema
+struct<col_name:string,data_type:string,comment:string>
+-- !query 24 output
+# col_name          	data_type           	comment             
+a                   	string              	                    
+b                   	int                 	                    
+c                   	string              	                    
+d                   	string              	                    
+                    	                    	                    
+# Detailed Table Information	                    	                    
+Database            	default             	                    
+Table               	v                   	                    
+Created [not included in comparison]
+Last Access [not included in comparison]
+Type                	VIEW                	                    
+View Text           	SELECT * FROM t     	                    
+View Default Database	default             	                    
+View Query Output Columns	[a, b, c, d]        	                    
+Properties          	[view.query.out.col.3=d, view.query.out.col.0=a, view.query.out.numCols=4, view.default.database=default, view.query.out.col.1=b, view.query.out.col.2=c]
+
+
+-- !query 25
+DESC EXTENDED v
+-- !query 25 schema
+struct<col_name:string,data_type:string,comment:string>
+-- !query 25 output
+# col_name          	data_type           	comment             
+a                   	string              	                    
+b                   	int                 	                    
+c                   	string              	                    
+d                   	string              	                    
+                    	                    	                    
+# Detailed Table Information	                    	                    
+Database            	default             	                    
+Table               	v                   	                    
+Created [not included in comparison]
+Last Access [not included in comparison]
+Type                	VIEW                	                    
+View Text           	SELECT * FROM t     	                    
+View Default Database	default             	                    
+View Query Output Columns	[a, b, c, d]        	                    
+Properties          	[view.query.out.col.3=d, view.query.out.col.0=a, view.query.out.numCols=4, view.default.database=default, view.query.out.col.1=b, view.query.out.col.2=c]
+
+
+-- !query 26
+DESC v PARTITION (c='Us', d=1)
+-- !query 26 schema
+struct<>
+-- !query 26 output
+org.apache.spark.sql.AnalysisException
+DESC PARTITION is not allowed on a view: v;
+
+
+-- !query 27
 DROP TABLE t
--- !query 13 schema
+-- !query 27 schema
 struct<>
--- !query 13 output
+-- !query 27 output
+
+
+
+-- !query 28
+DROP VIEW temp_v
+-- !query 28 schema
+struct<>
+-- !query 28 output
+
+
+
+-- !query 29
+DROP VIEW temp_Data_Source_View
+-- !query 29 schema
+struct<>
+-- !query 29 output
+
+
+
+-- !query 30
+DROP VIEW v
+-- !query 30 schema
+struct<>
+-- !query 30 output
 

http://git-wip-us.apache.org/repos/asf/spark/blob/51d3c854/sql/core/src/test/resources/sql-tests/results/show-tables.sql.out
----------------------------------------------------------------------
diff --git a/sql/core/src/test/resources/sql-tests/results/show-tables.sql.out b/sql/core/src/test/resources/sql-tests/results/show-tables.sql.out
index 6d62e60..8f2a54f 100644
--- a/sql/core/src/test/resources/sql-tests/results/show-tables.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/show-tables.sql.out
@@ -118,33 +118,40 @@ SHOW TABLE EXTENDED LIKE 'show_t*'
 -- !query 12 schema
 struct<database:string,tableName:string,isTemporary:boolean,information:string>
 -- !query 12 output
-show_t3	true	CatalogTable(
-	Table: `show_t3`
-	Created: 
-	Last Access: 
-	Type: VIEW
-	Schema: [StructField(e,IntegerType,true)]
-	Storage())
-
-showdb	show_t1	false	CatalogTable(
-	Table: `showdb`.`show_t1`
-	Created: 
-	Last Access: 
-	Type: MANAGED
-	Schema: [StructField(a,StringType,true), StructField(b,IntegerType,true), StructField(c,StringType,true), StructField(d,StringType,true)]
-	Provider: parquet
-	Partition Columns: [`c`, `d`]
-	Storage(Location: sql/core/spark-warehouse/showdb.db/show_t1)
-	Partition Provider: Catalog)
-
-showdb	show_t2	false	CatalogTable(
-	Table: `showdb`.`show_t2`
-	Created: 
-	Last Access: 
-	Type: MANAGED
-	Schema: [StructField(b,StringType,true), StructField(d,IntegerType,true)]
-	Provider: parquet
-	Storage(Location: sql/core/spark-warehouse/showdb.db/show_t2))
+show_t3	true	Table: show_t3
+Created [not included in comparison]
+Last Access [not included in comparison]
+Type: VIEW
+Schema: root
+ |-- e: integer (nullable = true)
+
+
+showdb	show_t1	false	Database: showdb
+Table: show_t1
+Created [not included in comparison]
+Last Access [not included in comparison]
+Type: MANAGED
+Provider: parquet
+Location [not included in comparison]sql/core/spark-warehouse/showdb.db/show_t1
+Partition Provider: Catalog
+Partition Columns: [`c`, `d`]
+Schema: root
+ |-- a: string (nullable = true)
+ |-- b: integer (nullable = true)
+ |-- c: string (nullable = true)
+ |-- d: string (nullable = true)
+
+
+showdb	show_t2	false	Database: showdb
+Table: show_t2
+Created [not included in comparison]
+Last Access [not included in comparison]
+Type: MANAGED
+Provider: parquet
+Location [not included in comparison]sql/core/spark-warehouse/showdb.db/show_t2
+Schema: root
+ |-- b: string (nullable = true)
+ |-- d: integer (nullable = true)
 
 
 -- !query 13
@@ -166,10 +173,8 @@ SHOW TABLE EXTENDED LIKE 'show_t1' PARTITION(c='Us', d=1)
 -- !query 14 schema
 struct<database:string,tableName:string,isTemporary:boolean,information:string>
 -- !query 14 output
-showdb	show_t1	false	CatalogPartition(
-	Partition Values: [c=Us, d=1]
-	Storage(Location: sql/core/spark-warehouse/showdb.db/show_t1/c=Us/d=1)
-	Partition Parameters:{})
+showdb	show_t1	false	Partition Values: [c=Us, d=1]
+Location [not included in comparison]sql/core/spark-warehouse/showdb.db/show_t1/c=Us/d=1
 
 
 -- !query 15

http://git-wip-us.apache.org/repos/asf/spark/blob/51d3c854/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala
index 4092862..4b69baf 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala
@@ -26,6 +26,7 @@ import org.apache.spark.sql.catalyst.planning.PhysicalOperation
 import org.apache.spark.sql.catalyst.plans.logical._
 import org.apache.spark.sql.catalyst.rules.RuleExecutor
 import org.apache.spark.sql.catalyst.util.{fileToString, stringToFile}
+import org.apache.spark.sql.execution.command.DescribeTableCommand
 import org.apache.spark.sql.test.SharedSQLContext
 import org.apache.spark.sql.types.StructType
 
@@ -165,8 +166,8 @@ class SQLQueryTestSuite extends QueryTest with SharedSQLContext {
         s"-- Number of queries: ${outputs.size}\n\n\n" +
         outputs.zipWithIndex.map{case (qr, i) => qr.toString(i)}.mkString("\n\n\n") + "\n"
       }
-      val resultFile = new File(testCase.resultFile);
-      val parent = resultFile.getParentFile();
+      val resultFile = new File(testCase.resultFile)
+      val parent = resultFile.getParentFile
       if (!parent.exists()) {
         assert(parent.mkdirs(), "Could not create directory: " + parent)
       }
@@ -212,23 +213,25 @@ class SQLQueryTestSuite extends QueryTest with SharedSQLContext {
   /** Executes a query and returns the result as (schema of the output, normalized output). */
   private def getNormalizedResult(session: SparkSession, sql: String): (StructType, Seq[String]) = {
     // Returns true if the plan is supposed to be sorted.
-    def isSorted(plan: LogicalPlan): Boolean = plan match {
+    def needSort(plan: LogicalPlan): Boolean = plan match {
       case _: Join | _: Aggregate | _: Generate | _: Sample | _: Distinct => false
+      case _: DescribeTableCommand => true
       case PhysicalOperation(_, _, Sort(_, true, _)) => true
-      case _ => plan.children.iterator.exists(isSorted)
+      case _ => plan.children.iterator.exists(needSort)
     }
 
     try {
       val df = session.sql(sql)
       val schema = df.schema
+      val notIncludedMsg = "[not included in comparison]"
       // Get answer, but also get rid of the #1234 expression ids that show up in explain plans
       val answer = df.queryExecution.hiveResultString().map(_.replaceAll("#\\d+", "#x")
-        .replaceAll("Location:.*/sql/core/", "Location: sql/core/")
-        .replaceAll("Created: .*", "Created: ")
-        .replaceAll("Last Access: .*", "Last Access: "))
+        .replaceAll("Location.*/sql/core/", s"Location ${notIncludedMsg}sql/core/")
+        .replaceAll("Created.*", s"Created $notIncludedMsg")
+        .replaceAll("Last Access.*", s"Last Access $notIncludedMsg"))
 
       // If the output is not pre-sorted, sort it.
-      if (isSorted(df.queryExecution.analyzed)) (schema, answer) else (schema, answer.sorted)
+      if (needSort(df.queryExecution.analyzed)) (schema, answer) else (schema, answer.sorted)
 
     } catch {
       case a: AnalysisException =>

http://git-wip-us.apache.org/repos/asf/spark/blob/51d3c854/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala
----------------------------------------------------------------------
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 a4d012c..908b955 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
@@ -224,13 +224,13 @@ class SparkSqlParserSuite extends PlanTest {
   test("SPARK-17328 Fix NPE with EXPLAIN DESCRIBE TABLE") {
     assertEqual("describe table t",
       DescribeTableCommand(
-        TableIdentifier("t"), Map.empty, isExtended = false, isFormatted = false))
+        TableIdentifier("t"), Map.empty, isExtended = false))
     assertEqual("describe table extended t",
       DescribeTableCommand(
-        TableIdentifier("t"), Map.empty, isExtended = true, isFormatted = false))
+        TableIdentifier("t"), Map.empty, isExtended = true))
     assertEqual("describe table formatted t",
       DescribeTableCommand(
-        TableIdentifier("t"), Map.empty, isExtended = false, isFormatted = true))
+        TableIdentifier("t"), Map.empty, isExtended = true))
 
     intercept("explain describe tables x", "Unsupported SQL statement")
   }

http://git-wip-us.apache.org/repos/asf/spark/blob/51d3c854/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala
----------------------------------------------------------------------
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 648b179..9ebf2dd 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
@@ -69,18 +69,6 @@ class InMemoryCatalogedDDLSuite extends DDLSuite with SharedSQLContext with Befo
       tracksPartitionsInCatalog = true)
   }
 
-  test("desc table for parquet data source table using in-memory catalog") {
-    val tabName = "tab1"
-    withTable(tabName) {
-      sql(s"CREATE TABLE $tabName(a int comment 'test') USING parquet ")
-
-      checkAnswer(
-        sql(s"DESC $tabName").select("col_name", "data_type", "comment"),
-        Row("a", "int", "test")
-      )
-    }
-  }
-
   test("alter table: set location (datasource table)") {
     testSetLocation(isDatasourceTable = true)
   }

http://git-wip-us.apache.org/repos/asf/spark/blob/51d3c854/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala
index 4a02277..5bd36ec 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala
@@ -806,10 +806,6 @@ class JDBCSuite extends SparkFunSuite
         sql(s"DESC FORMATTED $tableName").collect().foreach { r =>
           assert(!r.toString().contains(password))
         }
-
-        sql(s"DESC EXTENDED $tableName").collect().foreach { r =>
-          assert(!r.toString().contains(password))
-        }
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/spark/blob/51d3c854/sql/core/src/test/scala/org/apache/spark/sql/sources/DDLTestSuite.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/DDLTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/DDLTestSuite.scala
deleted file mode 100644
index 674463f..0000000
--- a/sql/core/src/test/scala/org/apache/spark/sql/sources/DDLTestSuite.scala
+++ /dev/null
@@ -1,123 +0,0 @@
-/*
-* 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.sources
-
-import org.apache.spark.rdd.RDD
-import org.apache.spark.sql._
-import org.apache.spark.sql.catalyst.InternalRow
-import org.apache.spark.sql.test.SharedSQLContext
-import org.apache.spark.sql.types._
-import org.apache.spark.unsafe.types.UTF8String
-
-class DDLScanSource extends RelationProvider {
-  override def createRelation(
-      sqlContext: SQLContext,
-      parameters: Map[String, String]): BaseRelation = {
-    SimpleDDLScan(
-      parameters("from").toInt,
-      parameters("TO").toInt,
-      parameters("Table"))(sqlContext.sparkSession)
-  }
-}
-
-case class SimpleDDLScan(
-    from: Int,
-    to: Int,
-    table: String)(@transient val sparkSession: SparkSession)
-  extends BaseRelation with TableScan {
-
-  override def sqlContext: SQLContext = sparkSession.sqlContext
-
-  override def schema: StructType =
-    StructType(Seq(
-      StructField("intType", IntegerType, nullable = false).withComment(s"test comment $table"),
-      StructField("stringType", StringType, nullable = false),
-      StructField("dateType", DateType, nullable = false),
-      StructField("timestampType", TimestampType, nullable = false),
-      StructField("doubleType", DoubleType, nullable = false),
-      StructField("bigintType", LongType, nullable = false),
-      StructField("tinyintType", ByteType, nullable = false),
-      StructField("decimalType", DecimalType.USER_DEFAULT, nullable = false),
-      StructField("fixedDecimalType", DecimalType(5, 1), nullable = false),
-      StructField("binaryType", BinaryType, nullable = false),
-      StructField("booleanType", BooleanType, nullable = false),
-      StructField("smallIntType", ShortType, nullable = false),
-      StructField("floatType", FloatType, nullable = false),
-      StructField("mapType", MapType(StringType, StringType)),
-      StructField("arrayType", ArrayType(StringType)),
-      StructField("structType",
-        StructType(StructField("f1", StringType) :: StructField("f2", IntegerType) :: Nil
-        )
-      )
-    ))
-
-  override def needConversion: Boolean = false
-
-  override def buildScan(): RDD[Row] = {
-    // Rely on a type erasure hack to pass RDD[InternalRow] back as RDD[Row]
-    sparkSession.sparkContext.parallelize(from to to).map { e =>
-      InternalRow(UTF8String.fromString(s"people$e"), e * 2)
-    }.asInstanceOf[RDD[Row]]
-  }
-}
-
-class DDLTestSuite extends DataSourceTest with SharedSQLContext {
-  protected override lazy val sql = spark.sql _
-
-  override def beforeAll(): Unit = {
-    super.beforeAll()
-    sql(
-      """
-       |CREATE OR REPLACE TEMPORARY VIEW ddlPeople
-       |USING org.apache.spark.sql.sources.DDLScanSource
-       |OPTIONS (
-       |  From '1',
-       |  To '10',
-       |  Table 'test1'
-       |)
-      """.stripMargin)
-  }
-
-  sqlTest(
-      "describe ddlPeople",
-      Seq(
-        Row("intType", "int", "test comment test1"),
-        Row("stringType", "string", null),
-        Row("dateType", "date", null),
-        Row("timestampType", "timestamp", null),
-        Row("doubleType", "double", null),
-        Row("bigintType", "bigint", null),
-        Row("tinyintType", "tinyint", null),
-        Row("decimalType", "decimal(10,0)", null),
-        Row("fixedDecimalType", "decimal(5,1)", null),
-        Row("binaryType", "binary", null),
-        Row("booleanType", "boolean", null),
-        Row("smallIntType", "smallint", null),
-        Row("floatType", "float", null),
-        Row("mapType", "map<string,string>", null),
-        Row("arrayType", "array<string>", null),
-        Row("structType", "struct<f1:string,f2:int>", null)
-      ))
-
-  test("SPARK-7686 DescribeCommand should have correct physical plan output attributes") {
-    val attributes = sql("describe ddlPeople")
-      .queryExecution.executedPlan.output
-    assert(attributes.map(_.name) === Seq("col_name", "data_type", "comment"))
-    assert(attributes.map(_.dataType).toSet === Set(StringType))
-  }
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/51d3c854/sql/core/src/test/scala/org/apache/spark/sql/sources/DataSourceTest.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/DataSourceTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/DataSourceTest.scala
index cc77d3c..80868ff 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/sources/DataSourceTest.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/DataSourceTest.scala
@@ -17,7 +17,11 @@
 
 package org.apache.spark.sql.sources
 
+import org.apache.spark.rdd.RDD
 import org.apache.spark.sql._
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.types._
+import org.apache.spark.unsafe.types.UTF8String
 
 private[sql] abstract class DataSourceTest extends QueryTest {
 
@@ -28,3 +32,55 @@ private[sql] abstract class DataSourceTest extends QueryTest {
   }
 
 }
+
+class DDLScanSource extends RelationProvider {
+  override def createRelation(
+      sqlContext: SQLContext,
+      parameters: Map[String, String]): BaseRelation = {
+    SimpleDDLScan(
+      parameters("from").toInt,
+      parameters("TO").toInt,
+      parameters("Table"))(sqlContext.sparkSession)
+  }
+}
+
+case class SimpleDDLScan(
+    from: Int,
+    to: Int,
+    table: String)(@transient val sparkSession: SparkSession)
+  extends BaseRelation with TableScan {
+
+  override def sqlContext: SQLContext = sparkSession.sqlContext
+
+  override def schema: StructType =
+    StructType(Seq(
+      StructField("intType", IntegerType, nullable = false).withComment(s"test comment $table"),
+      StructField("stringType", StringType, nullable = false),
+      StructField("dateType", DateType, nullable = false),
+      StructField("timestampType", TimestampType, nullable = false),
+      StructField("doubleType", DoubleType, nullable = false),
+      StructField("bigintType", LongType, nullable = false),
+      StructField("tinyintType", ByteType, nullable = false),
+      StructField("decimalType", DecimalType.USER_DEFAULT, nullable = false),
+      StructField("fixedDecimalType", DecimalType(5, 1), nullable = false),
+      StructField("binaryType", BinaryType, nullable = false),
+      StructField("booleanType", BooleanType, nullable = false),
+      StructField("smallIntType", ShortType, nullable = false),
+      StructField("floatType", FloatType, nullable = false),
+      StructField("mapType", MapType(StringType, StringType)),
+      StructField("arrayType", ArrayType(StringType)),
+      StructField("structType",
+        StructType(StructField("f1", StringType) :: StructField("f2", IntegerType) :: Nil
+        )
+      )
+    ))
+
+  override def needConversion: Boolean = false
+
+  override def buildScan(): RDD[Row] = {
+    // Rely on a type erasure hack to pass RDD[InternalRow] back as RDD[Row]
+    sparkSession.sparkContext.parallelize(from to to).map { e =>
+      InternalRow(UTF8String.fromString(s"people$e"), e * 2)
+    }.asInstanceOf[RDD[Row]]
+  }
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/51d3c854/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala
index 55e02ac..b554694 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala
@@ -767,9 +767,6 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv
       sessionState.refreshTable(tableName)
       val actualSchema = table(tableName).schema
       assert(schema === actualSchema)
-
-      // Checks the DESCRIBE output.
-      checkAnswer(sql("DESCRIBE spark6655"), Row("int", "int", null) :: Nil)
     }
   }
 
@@ -1381,7 +1378,10 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv
 
         checkAnswer(spark.table("old"), Row(1, "a"))
 
-        checkAnswer(sql("DESC old"), Row("i", "int", null) :: Row("j", "string", null) :: Nil)
+        val expectedSchema = StructType(Seq(
+          StructField("i", IntegerType, nullable = true),
+          StructField("j", StringType, nullable = true)))
+        assert(table("old").schema === expectedSchema)
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/spark/blob/51d3c854/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala
----------------------------------------------------------------------
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 536ca8f..e45cf97 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
@@ -207,6 +207,7 @@ abstract class HiveComparisonTest
   // This list contains indicators for those lines which do not have actual results and we
   // want to ignore.
   lazy val ignoredLineIndicators = Seq(
+    "# Detailed Table Information",
     "# Partition Information",
     "# col_name"
   )
@@ -358,7 +359,7 @@ abstract class HiveComparisonTest
               stringToFile(new File(failedDirectory, testCaseName), errorMessage + consoleTestCase)
               fail(errorMessage)
           }
-        }.toSeq
+        }
 
         (queryList, hiveResults, catalystResults).zipped.foreach {
           case (query, hive, (hiveQuery, catalyst)) =>
@@ -369,6 +370,7 @@ abstract class HiveComparisonTest
             if ((!hiveQuery.logical.isInstanceOf[ExplainCommand]) &&
                 (!hiveQuery.logical.isInstanceOf[ShowFunctionsCommand]) &&
                 (!hiveQuery.logical.isInstanceOf[DescribeFunctionCommand]) &&
+                (!hiveQuery.logical.isInstanceOf[DescribeTableCommand]) &&
                 preparedHive != catalyst) {
 
               val hivePrintOut = s"== HIVE - ${preparedHive.size} row(s) ==" +: preparedHive

http://git-wip-us.apache.org/repos/asf/spark/blob/51d3c854/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala
index f0a995c..3906968 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala
@@ -708,23 +708,6 @@ class HiveDDLSuite
     }
   }
 
-  test("desc table for Hive table") {
-    withTable("tab1") {
-      val tabName = "tab1"
-      sql(s"CREATE TABLE $tabName(c1 int)")
-
-      assert(sql(s"DESC $tabName").collect().length == 1)
-
-      assert(
-        sql(s"DESC FORMATTED $tabName").collect()
-          .exists(_.getString(0) == "# Storage Information"))
-
-      assert(
-        sql(s"DESC EXTENDED $tabName").collect()
-          .exists(_.getString(0) == "# Detailed Table Information"))
-    }
-  }
-
   test("desc table for Hive table - partitioned table") {
     withTable("tbl") {
       sql("CREATE TABLE tbl(a int) PARTITIONED BY (b int)")
@@ -741,23 +724,6 @@ class HiveDDLSuite
     }
   }
 
-  test("desc formatted table for permanent view") {
-    withTable("tbl") {
-      withView("view1") {
-        sql("CREATE TABLE tbl(a int)")
-        sql("CREATE VIEW view1 AS SELECT * FROM tbl")
-        assert(sql("DESC FORMATTED view1").collect().containsSlice(
-          Seq(
-            Row("# View Information", "", ""),
-            Row("View Text:", "SELECT * FROM tbl", ""),
-            Row("View Default Database:", "default", ""),
-            Row("View Query Output Columns:", "[a]", "")
-          )
-        ))
-      }
-    }
-  }
-
   test("desc table for data source table using Hive Metastore") {
     assume(spark.sparkContext.conf.get(CATALOG_IMPLEMENTATION) == "hive")
     val tabName = "tab1"
@@ -766,7 +732,7 @@ class HiveDDLSuite
 
       checkAnswer(
         sql(s"DESC $tabName").select("col_name", "data_type", "comment"),
-        Row("a", "int", "test")
+        Row("# col_name", "data_type", "comment") :: Row("a", "int", "test") :: Nil
       )
     }
   }
@@ -1218,23 +1184,6 @@ class HiveDDLSuite
       sql(s"SELECT * FROM ${targetTable.identifier}"))
   }
 
-  test("desc table for data source table") {
-    withTable("tab1") {
-      val tabName = "tab1"
-      spark.range(1).write.format("json").saveAsTable(tabName)
-
-      assert(sql(s"DESC $tabName").collect().length == 1)
-
-      assert(
-        sql(s"DESC FORMATTED $tabName").collect()
-          .exists(_.getString(0) == "# Storage Information"))
-
-      assert(
-        sql(s"DESC EXTENDED $tabName").collect()
-          .exists(_.getString(0) == "# Detailed Table Information"))
-    }
-  }
-
   test("create table with the same name as an index table") {
     val tabName = "tab1"
     val indexName = tabName + "_index"
@@ -1320,46 +1269,6 @@ class HiveDDLSuite
     }
   }
 
-  test("desc table for data source table - partitioned bucketed table") {
-    withTable("t1") {
-      spark
-        .range(1).select('id as 'a, 'id as 'b, 'id as 'c, 'id as 'd).write
-        .bucketBy(2, "b").sortBy("c").partitionBy("d")
-        .saveAsTable("t1")
-
-      val formattedDesc = sql("DESC FORMATTED t1").collect()
-
-      assert(formattedDesc.containsSlice(
-        Seq(
-          Row("a", "bigint", null),
-          Row("b", "bigint", null),
-          Row("c", "bigint", null),
-          Row("d", "bigint", null),
-          Row("# Partition Information", "", ""),
-          Row("# col_name", "data_type", "comment"),
-          Row("d", "bigint", null),
-          Row("", "", ""),
-          Row("# Detailed Table Information", "", ""),
-          Row("Database:", "default", "")
-        )
-      ))
-
-      assert(formattedDesc.containsSlice(
-        Seq(
-          Row("Table Type:", "MANAGED", "")
-        )
-      ))
-
-      assert(formattedDesc.containsSlice(
-        Seq(
-          Row("Num Buckets:", "2", ""),
-          Row("Bucket Columns:", "[b]", ""),
-          Row("Sort Columns:", "[c]", "")
-        )
-      ))
-    }
-  }
-
   test("datasource and statistics table property keys are not allowed") {
     import org.apache.spark.sql.hive.HiveExternalCatalog.DATASOURCE_PREFIX
     import org.apache.spark.sql.hive.HiveExternalCatalog.STATISTICS_PREFIX

http://git-wip-us.apache.org/repos/asf/spark/blob/51d3c854/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveOperatorQueryableSuite.scala
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveOperatorQueryableSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveOperatorQueryableSuite.scala
deleted file mode 100644
index 0e89e99..0000000
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveOperatorQueryableSuite.scala
+++ /dev/null
@@ -1,53 +0,0 @@
-/*
- * 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.hive.execution
-
-import org.apache.spark.sql.{QueryTest, Row}
-import org.apache.spark.sql.hive.test.TestHiveSingleton
-
-/**
- * A set of tests that validates commands can also be queried by like a table
- */
-class HiveOperatorQueryableSuite extends QueryTest with TestHiveSingleton {
-  import spark._
-
-  test("SPARK-5324 query result of describe command") {
-    hiveContext.loadTestTable("src")
-
-    // Creates a temporary view with the output of a describe command
-    sql("desc src").createOrReplaceTempView("mydesc")
-    checkAnswer(
-      sql("desc mydesc"),
-      Seq(
-        Row("col_name", "string", "name of the column"),
-        Row("data_type", "string", "data type of the column"),
-        Row("comment", "string", "comment of the column")))
-
-    checkAnswer(
-      sql("select * from mydesc"),
-      Seq(
-        Row("key", "int", null),
-        Row("value", "string", null)))
-
-    checkAnswer(
-      sql("select col_name, data_type, comment from mydesc"),
-      Seq(
-        Row("key", "int", null),
-        Row("value", "string", null)))
-  }
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/51d3c854/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala
----------------------------------------------------------------------
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 dd278f6..65a902f 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
@@ -789,62 +789,6 @@ class HiveQuerySuite extends HiveComparisonTest with SQLTestUtils with BeforeAnd
     assert(Try(q0.count()).isSuccess)
   }
 
-  test("DESCRIBE commands") {
-    sql(s"CREATE TABLE test_describe_commands1 (key INT, value STRING) PARTITIONED BY (dt STRING)")
-
-    sql(
-      """FROM src INSERT OVERWRITE TABLE test_describe_commands1 PARTITION (dt='2008-06-08')
-        |SELECT key, value
-      """.stripMargin)
-
-    // Describe a table
-    assertResult(
-      Array(
-        Row("key", "int", null),
-        Row("value", "string", null),
-        Row("dt", "string", null),
-        Row("# Partition Information", "", ""),
-        Row("# col_name", "data_type", "comment"),
-        Row("dt", "string", null))
-    ) {
-      sql("DESCRIBE test_describe_commands1")
-        .select('col_name, 'data_type, 'comment)
-        .collect()
-    }
-
-    // Describe a table with a fully qualified table name
-    assertResult(
-      Array(
-        Row("key", "int", null),
-        Row("value", "string", null),
-        Row("dt", "string", null),
-        Row("# Partition Information", "", ""),
-        Row("# col_name", "data_type", "comment"),
-        Row("dt", "string", null))
-    ) {
-      sql("DESCRIBE default.test_describe_commands1")
-        .select('col_name, 'data_type, 'comment)
-        .collect()
-    }
-
-    // Describe a temporary view.
-    val testData =
-      TestHive.sparkContext.parallelize(
-        TestData(1, "str1") ::
-        TestData(1, "str2") :: Nil)
-    testData.toDF().createOrReplaceTempView("test_describe_commands2")
-
-    assertResult(
-      Array(
-        Row("a", "int", null),
-        Row("b", "string", null))
-    ) {
-      sql("DESCRIBE test_describe_commands2")
-        .select('col_name, 'data_type, 'comment)
-        .collect()
-    }
-  }
-
   test("SPARK-2263: Insert Map<K, V> values") {
     sql("CREATE TABLE m(value MAP<INT, STRING>)")
     sql("INSERT OVERWRITE TABLE m SELECT MAP(key, value) FROM src LIMIT 10")

http://git-wip-us.apache.org/repos/asf/spark/blob/51d3c854/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala
----------------------------------------------------------------------
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 55ff4bb..d012797 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
@@ -363,79 +363,6 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton {
     }
   }
 
-  test("describe partition") {
-    withTable("partitioned_table") {
-      sql("CREATE TABLE partitioned_table (a STRING, b INT) PARTITIONED BY (c STRING, d STRING)")
-      sql("ALTER TABLE partitioned_table ADD PARTITION (c='Us', d=1)")
-
-      checkKeywordsExist(sql("DESC partitioned_table PARTITION (c='Us', d=1)"),
-        "# Partition Information",
-        "# col_name")
-
-      checkKeywordsExist(sql("DESC EXTENDED partitioned_table PARTITION (c='Us', d=1)"),
-        "# Partition Information",
-        "# col_name",
-        "Detailed Partition Information CatalogPartition(",
-        "Partition Values: [c=Us, d=1]",
-        "Storage(Location:",
-        "Partition Parameters")
-
-      checkKeywordsExist(sql("DESC FORMATTED partitioned_table PARTITION (c='Us', d=1)"),
-        "# Partition Information",
-        "# col_name",
-        "# Detailed Partition Information",
-        "Partition Value:",
-        "Database:",
-        "Table:",
-        "Location:",
-        "Partition Parameters:",
-        "# Storage Information")
-    }
-  }
-
-  test("describe partition - error handling") {
-    withTable("partitioned_table", "datasource_table") {
-      sql("CREATE TABLE partitioned_table (a STRING, b INT) PARTITIONED BY (c STRING, d STRING)")
-      sql("ALTER TABLE partitioned_table ADD PARTITION (c='Us', d=1)")
-
-      val m = intercept[NoSuchPartitionException] {
-        sql("DESC partitioned_table PARTITION (c='Us', d=2)")
-      }.getMessage()
-      assert(m.contains("Partition not found in table"))
-
-      val m2 = intercept[AnalysisException] {
-        sql("DESC partitioned_table PARTITION (c='Us')")
-      }.getMessage()
-      assert(m2.contains("Partition spec is invalid"))
-
-      val m3 = intercept[ParseException] {
-        sql("DESC partitioned_table PARTITION (c='Us', d)")
-      }.getMessage()
-      assert(m3.contains("PARTITION specification is incomplete: `d`"))
-
-      spark
-        .range(1).select('id as 'a, 'id as 'b, 'id as 'c, 'id as 'd).write
-        .partitionBy("d")
-        .saveAsTable("datasource_table")
-
-      sql("DESC datasource_table PARTITION (d=0)")
-
-      val m5 = intercept[AnalysisException] {
-        spark.range(10).select('id as 'a, 'id as 'b).createTempView("view1")
-        sql("DESC view1 PARTITION (c='Us', d=1)")
-      }.getMessage()
-      assert(m5.contains("DESC PARTITION is not allowed on a temporary view"))
-
-      withView("permanent_view") {
-        val m = intercept[AnalysisException] {
-          sql("CREATE VIEW permanent_view AS SELECT * FROM partitioned_table")
-          sql("DESC permanent_view PARTITION (c='Us', d=1)")
-        }.getMessage()
-        assert(m.contains("DESC PARTITION is not allowed on a view"))
-      }
-    }
-  }
-
   test("SPARK-5371: union with null and sum") {
     val df = Seq((1, 1)).toDF("c1", "c2")
     df.createOrReplaceTempView("table1")
@@ -676,7 +603,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton {
   }
 
   test("CTAS with serde") {
-    sql("CREATE TABLE ctas1 AS SELECT key k, value FROM src ORDER BY k, value").collect()
+    sql("CREATE TABLE ctas1 AS SELECT key k, value FROM src ORDER BY k, value")
     sql(
       """CREATE TABLE ctas2
         | ROW FORMAT SERDE "org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe"
@@ -686,86 +613,76 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton {
         | AS
         |   SELECT key, value
         |   FROM src
-        |   ORDER BY key, value""".stripMargin).collect()
+        |   ORDER BY key, value""".stripMargin)
+
+    val storageCtas2 = spark.sessionState.catalog.getTableMetadata(TableIdentifier("ctas2")).storage
+    assert(storageCtas2.inputFormat == Some("org.apache.hadoop.hive.ql.io.RCFileInputFormat"))
+    assert(storageCtas2.outputFormat == Some("org.apache.hadoop.hive.ql.io.RCFileOutputFormat"))
+    assert(storageCtas2.serde == Some("org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe"))
+
     sql(
       """CREATE TABLE ctas3
         | ROW FORMAT DELIMITED FIELDS TERMINATED BY ',' LINES TERMINATED BY '\012'
         | STORED AS textfile AS
         |   SELECT key, value
         |   FROM src
-        |   ORDER BY key, value""".stripMargin).collect()
+        |   ORDER BY key, value""".stripMargin)
 
     // the table schema may like (key: integer, value: string)
     sql(
       """CREATE TABLE IF NOT EXISTS ctas4 AS
-        | SELECT 1 AS key, value FROM src LIMIT 1""".stripMargin).collect()
+        | SELECT 1 AS key, value FROM src LIMIT 1""".stripMargin)
     // do nothing cause the table ctas4 already existed.
     sql(
       """CREATE TABLE IF NOT EXISTS ctas4 AS
-        | SELECT key, value FROM src ORDER BY key, value""".stripMargin).collect()
+        | SELECT key, value FROM src ORDER BY key, value""".stripMargin)
 
     checkAnswer(
       sql("SELECT k, value FROM ctas1 ORDER BY k, value"),
-      sql("SELECT key, value FROM src ORDER BY key, value").collect().toSeq)
+      sql("SELECT key, value FROM src ORDER BY key, value"))
     checkAnswer(
       sql("SELECT key, value FROM ctas2 ORDER BY key, value"),
       sql(
         """
           SELECT key, value
           FROM src
-          ORDER BY key, value""").collect().toSeq)
+          ORDER BY key, value"""))
     checkAnswer(
       sql("SELECT key, value FROM ctas3 ORDER BY key, value"),
       sql(
         """
           SELECT key, value
           FROM src
-          ORDER BY key, value""").collect().toSeq)
+          ORDER BY key, value"""))
     intercept[AnalysisException] {
       sql(
         """CREATE TABLE ctas4 AS
-          | SELECT key, value FROM src ORDER BY key, value""".stripMargin).collect()
+          | SELECT key, value FROM src ORDER BY key, value""".stripMargin)
     }
     checkAnswer(
       sql("SELECT key, value FROM ctas4 ORDER BY key, value"),
       sql("SELECT key, value FROM ctas4 LIMIT 1").collect().toSeq)
 
-    /*
-    Disabled because our describe table does not output the serde information right now.
-    checkKeywordsExist(sql("DESC EXTENDED ctas2"),
-      "name:key", "type:string", "name:value", "ctas2",
-      "org.apache.hadoop.hive.ql.io.RCFileInputFormat",
-      "org.apache.hadoop.hive.ql.io.RCFileOutputFormat",
-      "org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe",
-      "serde_p1=p1", "serde_p2=p2", "tbl_p1=p11", "tbl_p2=p22", "MANAGED_TABLE"
-    )
-    */
-
     sql(
       """CREATE TABLE ctas5
         | STORED AS parquet AS
         |   SELECT key, value
         |   FROM src
-        |   ORDER BY key, value""".stripMargin).collect()
+        |   ORDER BY key, value""".stripMargin)
+    val storageCtas5 = spark.sessionState.catalog.getTableMetadata(TableIdentifier("ctas5")).storage
+    assert(storageCtas5.inputFormat ==
+      Some("org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat"))
+    assert(storageCtas5.outputFormat ==
+      Some("org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat"))
+    assert(storageCtas5.serde ==
+      Some("org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe"))
 
-    /*
-    Disabled because our describe table does not output the serde information right now.
-    withSQLConf(HiveUtils.CONVERT_METASTORE_PARQUET.key -> "false") {
-      checkKeywordsExist(sql("DESC EXTENDED ctas5"),
-        "name:key", "type:string", "name:value", "ctas5",
-        "org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat",
-        "org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat",
-        "org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe",
-        "MANAGED_TABLE"
-      )
-    }
-    */
 
     // use the Hive SerDe for parquet tables
     withSQLConf(HiveUtils.CONVERT_METASTORE_PARQUET.key -> "false") {
       checkAnswer(
         sql("SELECT key, value FROM ctas5 ORDER BY key, value"),
-        sql("SELECT key, value FROM src ORDER BY key, value").collect().toSeq)
+        sql("SELECT key, value FROM src ORDER BY key, value"))
     }
   }
 


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@spark.apache.org
For additional commands, e-mail: commits-help@spark.apache.org