Skip to content
Snippets Groups Projects
Commit 7343a094 authored by Xiao Li's avatar Xiao Li Committed by Wenchen Fan
Browse files

[SPARK-20023][SQL] Output table comment for DESC FORMATTED

### What changes were proposed in this pull request?
Currently, `DESC FORMATTED` did not output the table comment, unlike what `DESC EXTENDED` does. This PR is to fix it.

Also correct the following displayed names in `DESC FORMATTED`, for being consistent with `DESC EXTENDED`
- `"Create Time:"` -> `"Created:"`
- `"Last Access Time:"` -> `"Last Access:"`

### How was this patch tested?
Added test cases in `describe.sql`

Author: Xiao Li <gatorsmile@gmail.com>

Closes #17381 from gatorsmile/descFormattedTableComment.
parent 478fbc86
No related branches found
No related tags found
No related merge requests found
......@@ -568,11 +568,12 @@ case class DescribeTableCommand(
append(buffer, "# Detailed Table Information", "", "")
append(buffer, "Database:", table.database, "")
append(buffer, "Owner:", table.owner, "")
append(buffer, "Create Time:", new Date(table.createTime).toString, "")
append(buffer, "Last Access Time:", new Date(table.lastAccessTime).toString, "")
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:", "", "")
......
CREATE TABLE t (a STRING, b INT, c STRING, d STRING) USING parquet PARTITIONED BY (c, d);
CREATE TABLE t (a STRING, b INT, c STRING, d STRING) USING parquet PARTITIONED BY (c, d) COMMENT 'table_comment';
ALTER TABLE t ADD PARTITION (c='Us', d=1);
......@@ -8,15 +8,15 @@ DESC t;
DESC TABLE t;
-- Ignore these because there exist timestamp results, e.g., `Create Table`.
-- DESC EXTENDED t;
-- DESC FORMATTED t;
DESC FORMATTED t;
DESC EXTENDED t;
DESC t PARTITION (c='Us', d=1);
-- Ignore these because there exist timestamp results, e.g., transient_lastDdlTime.
-- DESC EXTENDED t PARTITION (c='Us', d=1);
-- DESC FORMATTED t PARTITION (c='Us', d=1);
DESC EXTENDED t PARTITION (c='Us', d=1);
DESC FORMATTED t PARTITION (c='Us', d=1);
-- NoSuchPartitionException: Partition not found in table
DESC t PARTITION (c='Us', d=2);
......
-- Automatically generated by SQLQueryTestSuite
-- Number of queries: 10
-- Number of queries: 14
-- !query 0
CREATE TABLE t (a STRING, b INT, c STRING, d STRING) USING parquet PARTITIONED BY (c, d)
CREATE TABLE t (a STRING, b INT, c STRING, d STRING) USING parquet PARTITIONED BY (c, d) COMMENT 'table_comment'
-- !query 0 schema
struct<>
-- !query 0 output
......@@ -64,12 +64,25 @@ d string
-- !query 5
DESC t PARTITION (c='Us', d=1)
DESC FORMATTED t
-- !query 5 schema
struct<col_name:string,data_type:string,comment:string>
-- !query 5 output
# Detailed Table Information
# Partition Information
# Storage Information
# 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
......@@ -79,30 +92,114 @@ d string
-- !query 6
DESC t PARTITION (c='Us', d=2)
DESC EXTENDED t
-- !query 6 schema
struct<>
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
# col_name data_type comment
a string
b int
c string
c string
d string
d string
-- !query 7
DESC t PARTITION (c='Us', d=1)
-- !query 7 schema
struct<col_name:string,data_type:string,comment:string>
-- !query 7 output
# Partition Information
# col_name data_type comment
a string
b int
c string
c string
d string
d string
-- !query 8
DESC EXTENDED t PARTITION (c='Us', d=1)
-- !query 8 schema
struct<col_name:string,data_type:string,comment:string>
-- !query 8 output
# Partition Information
# 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
c string
d string
d string
-- !query 9
DESC FORMATTED t PARTITION (c='Us', d=1)
-- !query 9 schema
struct<col_name:string,data_type:string,comment:string>
-- !query 9 output
# Detailed Partition Information
# Partition Information
# Storage Information
# 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
c string
d string
d string
-- !query 10
DESC t PARTITION (c='Us', d=2)
-- !query 10 schema
struct<>
-- !query 10 output
org.apache.spark.sql.catalyst.analysis.NoSuchPartitionException
Partition not found in table 't' database 'default':
c -> Us
d -> 2;
-- !query 7
-- !query 11
DESC t PARTITION (c='Us')
-- !query 7 schema
-- !query 11 schema
struct<>
-- !query 7 output
-- !query 11 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 8
-- !query 12
DESC t PARTITION (c='Us', d)
-- !query 8 schema
-- !query 12 schema
struct<>
-- !query 8 output
-- !query 12 output
org.apache.spark.sql.catalyst.parser.ParseException
PARTITION specification is incomplete: `d`(line 1, pos 0)
......@@ -112,9 +209,9 @@ DESC t PARTITION (c='Us', d)
^^^
-- !query 9
-- !query 13
DROP TABLE t
-- !query 9 schema
-- !query 13 schema
struct<>
-- !query 9 output
-- !query 13 output
......@@ -223,9 +223,9 @@ class SQLQueryTestSuite extends QueryTest with SharedSQLContext {
val schema = df.schema
// 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: .*\n", "Created: \n")
.replaceAll("Last Access: .*\n", "Last Access: \n"))
.replaceAll("Location:.*/sql/core/", "Location: sql/core/")
.replaceAll("Created: .*", "Created: ")
.replaceAll("Last Access: .*", "Last Access: "))
// If the output is not pre-sorted, sort it.
if (isSorted(df.queryExecution.analyzed)) (schema, answer) else (schema, answer.sorted)
......
0% Loading or .
You are about to add 0 people to the discussion. Proceed with caution.
Finish editing this message first!
Please register or to comment