-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-17642] [SQL] support DESC EXTENDED/FORMATTED table column commands #16422
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
23dccea
9ac8c11
16c5219
21e7b37
7c901ce
4b51baf
7bc2ea2
f81cc84
e69f182
429b996
53e4b38
85cc045
0d49ee9
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -20,24 +20,22 @@ package org.apache.spark.sql.execution.command | |
| import java.io.File | ||
| import java.net.URI | ||
| import java.nio.file.FileSystems | ||
| import java.util.Date | ||
|
|
||
| import scala.collection.mutable.ArrayBuffer | ||
| import scala.util.control.NonFatal | ||
| import scala.util.Try | ||
|
|
||
| import org.apache.commons.lang3.StringEscapeUtils | ||
| import org.apache.hadoop.fs.Path | ||
|
|
||
| import org.apache.spark.sql.{AnalysisException, Row, SparkSession} | ||
| import org.apache.spark.sql.catalyst.TableIdentifier | ||
| import org.apache.spark.sql.catalyst.analysis.NoSuchPartitionException | ||
| import org.apache.spark.sql.catalyst.analysis.{NoSuchPartitionException, UnresolvedAttribute} | ||
| import org.apache.spark.sql.catalyst.catalog._ | ||
| import org.apache.spark.sql.catalyst.catalog.CatalogTableType._ | ||
| import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec | ||
| import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} | ||
| import org.apache.spark.sql.catalyst.util.quoteIdentifier | ||
| import org.apache.spark.sql.execution.datasources.{DataSource, FileFormat, PartitioningUtils} | ||
| import org.apache.spark.sql.execution.datasources.{DataSource, PartitioningUtils} | ||
| import org.apache.spark.sql.execution.datasources.csv.CSVFileFormat | ||
| import org.apache.spark.sql.execution.datasources.json.JsonFileFormat | ||
| import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat | ||
|
|
@@ -626,6 +624,74 @@ case class DescribeTableCommand( | |
| } | ||
| } | ||
|
|
||
| /** | ||
| * A command to list the info for a column, including name, data type, column stats and comment. | ||
| * This function creates a [[DescribeColumnCommand]] logical plan. | ||
| * | ||
| * The syntax of using this command in SQL is: | ||
| * {{{ | ||
| * DESCRIBE [EXTENDED|FORMATTED] table_name column_name; | ||
| * }}} | ||
| */ | ||
| case class DescribeColumnCommand( | ||
| table: TableIdentifier, | ||
| colNameParts: Seq[String], | ||
| isExtended: Boolean) | ||
| extends RunnableCommand { | ||
|
|
||
| override val output: Seq[Attribute] = { | ||
| Seq( | ||
| AttributeReference("info_name", StringType, nullable = false, | ||
| new MetadataBuilder().putString("comment", "name of the column info").build())(), | ||
| AttributeReference("info_value", StringType, nullable = false, | ||
| new MetadataBuilder().putString("comment", "value of the column info").build())() | ||
| ) | ||
| } | ||
|
|
||
| override def run(sparkSession: SparkSession): Seq[Row] = { | ||
| val catalog = sparkSession.sessionState.catalog | ||
| val resolver = sparkSession.sessionState.conf.resolver | ||
| val relation = sparkSession.table(table).queryExecution.analyzed | ||
|
|
||
| val colName = UnresolvedAttribute(colNameParts).name | ||
| val field = { | ||
| relation.resolve(colNameParts, resolver).getOrElse { | ||
| throw new AnalysisException(s"Column $colName does not exist") | ||
| } | ||
| } | ||
| if (!field.isInstanceOf[Attribute]) { | ||
| // If the field is not an attribute after `resolve`, then it's a nested field. | ||
| throw new AnalysisException( | ||
| s"DESC TABLE COLUMN command does not support nested data types: $colName") | ||
| } | ||
|
|
||
| val catalogTable = catalog.getTempViewOrPermanentTableMetadata(table) | ||
| val colStats = catalogTable.stats.map(_.colStats).getOrElse(Map.empty) | ||
| val cs = colStats.get(field.name) | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. nit: |
||
|
|
||
| val comment = if (field.metadata.contains("comment")) { | ||
| Option(field.metadata.getString("comment")) | ||
| } else { | ||
| None | ||
| } | ||
|
|
||
| val buffer = ArrayBuffer[Row]( | ||
| Row("col_name", field.name), | ||
| Row("data_type", field.dataType.catalogString), | ||
| Row("comment", comment.getOrElse("NULL")) | ||
| ) | ||
| if (isExtended) { | ||
| // Show column stats when EXTENDED or FORMATTED is specified. | ||
| buffer += Row("min", cs.flatMap(_.min.map(_.toString)).getOrElse("NULL")) | ||
| buffer += Row("max", cs.flatMap(_.max.map(_.toString)).getOrElse("NULL")) | ||
| buffer += Row("num_nulls", cs.map(_.nullCount.toString).getOrElse("NULL")) | ||
| buffer += Row("distinct_count", cs.map(_.distinctCount.toString).getOrElse("NULL")) | ||
| buffer += Row("avg_col_len", cs.map(_.avgLen.toString).getOrElse("NULL")) | ||
| buffer += Row("max_col_len", cs.map(_.maxLen.toString).getOrElse("NULL")) | ||
| } | ||
| buffer | ||
| } | ||
| } | ||
|
|
||
| /** | ||
| * A command for users to get tables in the given database. | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,35 @@ | ||
| -- Test temp table | ||
| CREATE TEMPORARY VIEW desc_col_temp_table (key int COMMENT 'column_comment') USING PARQUET; | ||
|
|
||
| DESC desc_col_temp_table key; | ||
|
|
||
| DESC EXTENDED desc_col_temp_table key; | ||
|
|
||
| DESC FORMATTED desc_col_temp_table key; | ||
|
|
||
| -- Describe a column with qualified name | ||
| DESC FORMATTED desc_col_temp_table desc_col_temp_table.key; | ||
|
|
||
| -- Describe a non-existent column | ||
| DESC desc_col_temp_table key1; | ||
|
|
||
| -- Test persistent table | ||
| CREATE TABLE desc_col_table (key int COMMENT 'column_comment') USING PARQUET; | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. shall we drop these testing tables at the end?
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. yes we should. I'll drop them in the followup pr. |
||
|
|
||
| ANALYZE TABLE desc_col_table COMPUTE STATISTICS FOR COLUMNS key; | ||
|
|
||
| DESC desc_col_table key; | ||
|
|
||
| DESC EXTENDED desc_col_table key; | ||
|
|
||
| DESC FORMATTED desc_col_table key; | ||
|
|
||
| -- Test complex columns | ||
| CREATE TABLE desc_col_complex_table (`a.b` int, col struct<x:int, y:string>) USING PARQUET; | ||
|
|
||
| DESC FORMATTED desc_col_complex_table `a.b`; | ||
|
|
||
| DESC FORMATTED desc_col_complex_table col; | ||
|
|
||
| -- Describe a nested column | ||
| DESC FORMATTED desc_col_complex_table col.x; | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,184 @@ | ||
| -- Automatically generated by SQLQueryTestSuite | ||
| -- Number of queries: 15 | ||
|
|
||
|
|
||
| -- !query 0 | ||
| CREATE TEMPORARY VIEW desc_col_temp_table (key int COMMENT 'column_comment') USING PARQUET | ||
| -- !query 0 schema | ||
| struct<> | ||
| -- !query 0 output | ||
|
|
||
|
|
||
|
|
||
| -- !query 1 | ||
| DESC desc_col_temp_table key | ||
| -- !query 1 schema | ||
| struct<info_name:string,info_value:string> | ||
| -- !query 1 output | ||
| col_name key | ||
| data_type int | ||
| comment column_comment | ||
|
|
||
|
|
||
| -- !query 2 | ||
| DESC EXTENDED desc_col_temp_table key | ||
| -- !query 2 schema | ||
| struct<info_name:string,info_value:string> | ||
| -- !query 2 output | ||
| col_name key | ||
| data_type int | ||
| comment column_comment | ||
| min NULL | ||
| max NULL | ||
| num_nulls NULL | ||
| distinct_count NULL | ||
| avg_col_len NULL | ||
| max_col_len NULL | ||
|
|
||
|
|
||
| -- !query 3 | ||
| DESC FORMATTED desc_col_temp_table key | ||
| -- !query 3 schema | ||
| struct<info_name:string,info_value:string> | ||
| -- !query 3 output | ||
| col_name key | ||
| data_type int | ||
| comment column_comment | ||
| min NULL | ||
| max NULL | ||
| num_nulls NULL | ||
| distinct_count NULL | ||
| avg_col_len NULL | ||
| max_col_len NULL | ||
|
|
||
|
|
||
| -- !query 4 | ||
| DESC FORMATTED desc_col_temp_table desc_col_temp_table.key | ||
| -- !query 4 schema | ||
| struct<info_name:string,info_value:string> | ||
| -- !query 4 output | ||
| col_name key | ||
| data_type int | ||
| comment column_comment | ||
| min NULL | ||
| max NULL | ||
| num_nulls NULL | ||
| distinct_count NULL | ||
| avg_col_len NULL | ||
| max_col_len NULL | ||
|
|
||
|
|
||
| -- !query 5 | ||
| DESC desc_col_temp_table key1 | ||
| -- !query 5 schema | ||
| struct<> | ||
| -- !query 5 output | ||
| org.apache.spark.sql.AnalysisException | ||
| Column key1 does not exist; | ||
|
|
||
|
|
||
| -- !query 6 | ||
| CREATE TABLE desc_col_table (key int COMMENT 'column_comment') USING PARQUET | ||
| -- !query 6 schema | ||
| struct<> | ||
| -- !query 6 output | ||
|
|
||
|
|
||
|
|
||
| -- !query 7 | ||
| ANALYZE TABLE desc_col_table COMPUTE STATISTICS FOR COLUMNS key | ||
| -- !query 7 schema | ||
| struct<> | ||
| -- !query 7 output | ||
|
|
||
|
|
||
|
|
||
| -- !query 8 | ||
| DESC desc_col_table key | ||
| -- !query 8 schema | ||
| struct<info_name:string,info_value:string> | ||
| -- !query 8 output | ||
| col_name key | ||
| data_type int | ||
| comment column_comment | ||
|
|
||
|
|
||
| -- !query 9 | ||
| DESC EXTENDED desc_col_table key | ||
| -- !query 9 schema | ||
| struct<info_name:string,info_value:string> | ||
| -- !query 9 output | ||
| col_name key | ||
| data_type int | ||
| comment column_comment | ||
| min NULL | ||
| max NULL | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. why min max is NULL?
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. because the table is empty |
||
| num_nulls 0 | ||
| distinct_count 0 | ||
| avg_col_len 4 | ||
| max_col_len 4 | ||
|
|
||
|
|
||
| -- !query 10 | ||
| DESC FORMATTED desc_col_table key | ||
| -- !query 10 schema | ||
| struct<info_name:string,info_value:string> | ||
| -- !query 10 output | ||
| col_name key | ||
| data_type int | ||
| comment column_comment | ||
| min NULL | ||
| max NULL | ||
| num_nulls 0 | ||
| distinct_count 0 | ||
| avg_col_len 4 | ||
| max_col_len 4 | ||
|
|
||
|
|
||
| -- !query 11 | ||
| CREATE TABLE desc_col_complex_table (`a.b` int, col struct<x:int, y:string>) USING PARQUET | ||
| -- !query 11 schema | ||
| struct<> | ||
| -- !query 11 output | ||
|
|
||
|
|
||
|
|
||
| -- !query 12 | ||
| DESC FORMATTED desc_col_complex_table `a.b` | ||
| -- !query 12 schema | ||
| struct<info_name:string,info_value:string> | ||
| -- !query 12 output | ||
| col_name a.b | ||
| data_type int | ||
| comment NULL | ||
| min NULL | ||
| max NULL | ||
| num_nulls NULL | ||
| distinct_count NULL | ||
| avg_col_len NULL | ||
| max_col_len NULL | ||
|
|
||
|
|
||
| -- !query 13 | ||
| DESC FORMATTED desc_col_complex_table col | ||
| -- !query 13 schema | ||
| struct<info_name:string,info_value:string> | ||
| -- !query 13 output | ||
| col_name col | ||
| data_type struct<x:int,y:string> | ||
| comment NULL | ||
| min NULL | ||
| max NULL | ||
| num_nulls NULL | ||
| distinct_count NULL | ||
| avg_col_len NULL | ||
| max_col_len NULL | ||
|
|
||
|
|
||
| -- !query 14 | ||
| DESC FORMATTED desc_col_complex_table col.x | ||
| -- !query 14 schema | ||
| struct<> | ||
| -- !query 14 output | ||
| org.apache.spark.sql.AnalysisException | ||
| DESC TABLE COLUMN command does not support nested data types: col.x; | ||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This comment line seems not needed.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
There are other two similar comments (
ShowPartitionsCommand,ShowColumnsCommand) in this file, shall I remove them all?There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
A followup PR to improve the comments is sent: #19213