-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-28238][SQL] Implement DESCRIBE TABLE for Data Source V2 Tables. #25040
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 6 commits
d998c96
596832b
bdae301
527cbc3
6e5f0ea
eb5c843
25ea40b
4d32701
a947006
da202b1
c396d32
ee31d8a
18d688f
cf51488
2b77c80
f701daa
9e04c6e
d0a4533
cff78a1
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 |
|---|---|---|
| @@ -0,0 +1,35 @@ | ||
| /* | ||
| * Licensed to the Apache Software Foundation (ASF) under one or more | ||
| * contributor license agreements. See the NOTICE file distributed with | ||
| * this work for additional information regarding copyright ownership. | ||
| * The ASF licenses this file to You under the Apache License, Version 2.0 | ||
| * (the "License"); you may not use this file except in compliance with | ||
| * the License. You may obtain a copy of the License at | ||
| * | ||
| * http://www.apache.org/licenses/LICENSE-2.0 | ||
| * | ||
| * Unless required by applicable law or agreed to in writing, software | ||
| * distributed under the License is distributed on an "AS IS" BASIS, | ||
| * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
| * See the License for the specific language governing permissions and | ||
| * limitations under the License. | ||
| */ | ||
|
|
||
| package org.apache.spark.sql.catalyst.plans | ||
|
|
||
| import org.apache.spark.sql.catalyst.expressions.AttributeReference | ||
| import org.apache.spark.sql.types.{MetadataBuilder, StringType, StructField, StructType} | ||
|
|
||
| private[sql] object DescribeTableSchemas { | ||
| val DESCRIBE_TABLE_ATTRIBUTES = Seq( | ||
|
||
| AttributeReference("col_name", StringType, nullable = false, | ||
| new MetadataBuilder().putString("comment", "name of the column").build())(), | ||
| AttributeReference("data_type", StringType, nullable = false, | ||
| new MetadataBuilder().putString("comment", "data type of the column").build())(), | ||
| AttributeReference("comment", StringType, nullable = true, | ||
| new MetadataBuilder().putString("comment", "comment of the column").build())() | ||
| ) | ||
|
|
||
| val DESCRIBE_TABLE_SCHEMA = StructType( | ||
| DESCRIBE_TABLE_ATTRIBUTES.map(attr => StructField(attr.name, attr.dataType, attr.nullable))) | ||
|
||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -22,9 +22,11 @@ import org.apache.spark.sql.catalog.v2.expressions.Transform | |
| import org.apache.spark.sql.catalyst.AliasIdentifier | ||
| import org.apache.spark.sql.catalyst.analysis.{MultiInstanceRelation, NamedRelation} | ||
| import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable} | ||
| import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec | ||
| import org.apache.spark.sql.catalyst.expressions._ | ||
| import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, AggregateFunction} | ||
| import org.apache.spark.sql.catalyst.plans._ | ||
| import org.apache.spark.sql.catalyst.plans.logical.sql.DescribeTableStatement | ||
| import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning, RangePartitioning, RoundRobinPartitioning} | ||
| import org.apache.spark.sql.catalyst.util.truncatedString | ||
| import org.apache.spark.sql.types._ | ||
|
|
@@ -499,6 +501,14 @@ object OverwritePartitionsDynamic { | |
| } | ||
| } | ||
|
|
||
| case class DescribeTable( | ||
| catalog: TableCatalog, | ||
| ident: Identifier, | ||
| isExtended: Boolean) extends Command { | ||
| override lazy val output = DescribeTableSchemas.DESCRIBE_TABLE_ATTRIBUTES | ||
|
||
| override lazy val schema = DescribeTableSchemas.DESCRIBE_TABLE_SCHEMA | ||
|
||
| } | ||
|
|
||
| /** | ||
| * Drop a table. | ||
| */ | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,35 @@ | ||
| /* | ||
| * Licensed to the Apache Software Foundation (ASF) under one or more | ||
| * contributor license agreements. See the NOTICE file distributed with | ||
| * this work for additional information regarding copyright ownership. | ||
| * The ASF licenses this file to You under the Apache License, Version 2.0 | ||
| * (the "License"); you may not use this file except in compliance with | ||
| * the License. You may obtain a copy of the License at | ||
| * | ||
| * http://www.apache.org/licenses/LICENSE-2.0 | ||
| * | ||
| * Unless required by applicable law or agreed to in writing, software | ||
| * distributed under the License is distributed on an "AS IS" BASIS, | ||
| * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
| * See the License for the specific language governing permissions and | ||
| * limitations under the License. | ||
| */ | ||
|
|
||
| package org.apache.spark.sql.catalyst.plans.logical.sql | ||
|
|
||
| import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} | ||
| import org.apache.spark.sql.types.{MetadataBuilder, StringType} | ||
|
|
||
| case class DescribeColumnStatement( | ||
| tableName: Seq[String], | ||
| colNameParts: Seq[String], | ||
| isExtended: Boolean) extends ParsedStatement { | ||
| override def 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())() | ||
| ) | ||
| } | ||
| } |
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,29 @@ | ||
| /* | ||
| * Licensed to the Apache Software Foundation (ASF) under one or more | ||
| * contributor license agreements. See the NOTICE file distributed with | ||
| * this work for additional information regarding copyright ownership. | ||
| * The ASF licenses this file to You under the Apache License, Version 2.0 | ||
| * (the "License"); you may not use this file except in compliance with | ||
| * the License. You may obtain a copy of the License at | ||
| * | ||
| * http://www.apache.org/licenses/LICENSE-2.0 | ||
| * | ||
| * Unless required by applicable law or agreed to in writing, software | ||
| * distributed under the License is distributed on an "AS IS" BASIS, | ||
| * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
| * See the License for the specific language governing permissions and | ||
| * limitations under the License. | ||
| */ | ||
|
|
||
| package org.apache.spark.sql.catalyst.plans.logical.sql | ||
|
|
||
| import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec | ||
| import org.apache.spark.sql.catalyst.expressions.Attribute | ||
| import org.apache.spark.sql.catalyst.plans.DescribeTableSchemas | ||
|
|
||
| case class DescribeTableStatement( | ||
| tableName: Seq[String], | ||
| partitionSpec: TablePartitionSpec, | ||
| isExtended: Boolean) extends ParsedStatement { | ||
| override def output: Seq[Attribute] = DescribeTableSchemas.DESCRIBE_TABLE_ATTRIBUTES | ||
| } |
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -327,38 +327,6 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { | |
| } | ||
| } | ||
|
|
||
| /** | ||
| * Create a [[DescribeColumnCommand]] or [[DescribeTableCommand]] logical commands. | ||
| */ | ||
| override def visitDescribeTable(ctx: DescribeTableContext): LogicalPlan = withOrigin(ctx) { | ||
| val isExtended = ctx.EXTENDED != null || ctx.FORMATTED != null | ||
| if (ctx.describeColName != null) { | ||
| if (ctx.partitionSpec != null) { | ||
| throw new ParseException("DESC TABLE COLUMN for a specific partition is not supported", ctx) | ||
| } else { | ||
| DescribeColumnCommand( | ||
| visitTableIdentifier(ctx.tableIdentifier), | ||
|
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. Now that these rules create
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. Do you mean that |
||
| ctx.describeColName.nameParts.asScala.map(_.getText), | ||
| isExtended) | ||
| } | ||
| } else { | ||
| val partitionSpec = if (ctx.partitionSpec != null) { | ||
| // According to the syntax, visitPartitionSpec returns `Map[String, Option[String]]`. | ||
| visitPartitionSpec(ctx.partitionSpec).map { | ||
| case (key, Some(value)) => key -> value | ||
| case (key, _) => | ||
| throw new ParseException(s"PARTITION specification is incomplete: `$key`", ctx) | ||
| } | ||
| } else { | ||
| Map.empty[String, String] | ||
| } | ||
| DescribeTableCommand( | ||
| visitTableIdentifier(ctx.tableIdentifier), | ||
| partitionSpec, | ||
| isExtended) | ||
| } | ||
| } | ||
|
|
||
| /** | ||
| * Create a [[DescribeQueryCommand]] logical command. | ||
| */ | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -27,10 +27,10 @@ import org.apache.spark.sql.catalog.v2.expressions.Transform | |
| import org.apache.spark.sql.catalyst.TableIdentifier | ||
| import org.apache.spark.sql.catalyst.analysis.CastSupport | ||
| import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogTable, CatalogTableType, CatalogUtils} | ||
| import org.apache.spark.sql.catalyst.plans.logical.{CreateTableAsSelect, CreateV2Table, DropTable, LogicalPlan} | ||
| import org.apache.spark.sql.catalyst.plans.logical.sql.{AlterTableAddColumnsStatement, AlterTableSetLocationStatement, AlterTableSetPropertiesStatement, AlterTableUnsetPropertiesStatement, AlterViewSetPropertiesStatement, AlterViewUnsetPropertiesStatement, CreateTableAsSelectStatement, CreateTableStatement, DropTableStatement, DropViewStatement, QualifiedColType} | ||
| import org.apache.spark.sql.catalyst.plans.logical.{CreateTableAsSelect, CreateV2Table, DescribeTable, DropTable, LogicalPlan} | ||
| import org.apache.spark.sql.catalyst.plans.logical.sql.{AlterTableAddColumnsStatement, AlterTableSetLocationStatement, AlterTableSetPropertiesStatement, AlterTableUnsetPropertiesStatement, AlterViewSetPropertiesStatement, AlterViewUnsetPropertiesStatement, CreateTableAsSelectStatement, CreateTableStatement, DescribeColumnStatement, DescribeTableStatement, DropTableStatement, DropViewStatement, QualifiedColType} | ||
| import org.apache.spark.sql.catalyst.rules.Rule | ||
| import org.apache.spark.sql.execution.command.{AlterTableAddColumnsCommand, AlterTableSetLocationCommand, AlterTableSetPropertiesCommand, AlterTableUnsetPropertiesCommand, DropTableCommand} | ||
| import org.apache.spark.sql.execution.command.{AlterTableAddColumnsCommand, AlterTableSetLocationCommand, AlterTableSetPropertiesCommand, AlterTableUnsetPropertiesCommand, DescribeColumnCommand, DescribeTableCommand, DropTableCommand} | ||
| import org.apache.spark.sql.internal.SQLConf | ||
| import org.apache.spark.sql.sources.v2.TableProvider | ||
| import org.apache.spark.sql.types.{HIVE_TYPE_STRING, HiveStringType, MetadataBuilder, StructField, StructType} | ||
|
|
@@ -85,6 +85,25 @@ case class DataSourceResolution( | |
| .asTableCatalog | ||
| convertCTAS(catalog, identifier, create) | ||
|
|
||
| case DescribeColumnStatement( | ||
| AsTableIdentifier(tableName), colName, isExtended) => | ||
| DescribeColumnCommand(tableName, colName, isExtended) | ||
|
|
||
| case DescribeColumnStatement( | ||
| CatalogObjectIdentifier(Some(catalog), ident), colName, isExtended) => | ||
| throw new AnalysisException("Describing columns is not supported for v2 tables.") | ||
|
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. Should this be supported eventually, or is it redundant if DESCRIBE TABLE is available?
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. Think we need to support it eventually if only to keep parity with V1 tables. |
||
|
|
||
| case DescribeTableStatement( | ||
| AsTableIdentifier(tableName), partitionSpec, isExtended) => | ||
| DescribeTableCommand(tableName, partitionSpec, isExtended) | ||
|
|
||
| case DescribeTableStatement( | ||
| CatalogObjectIdentifier(Some(catalog), ident), partitionSpec, isExtended) => | ||
| if (partitionSpec.nonEmpty) { | ||
| throw new AnalysisException("DESCRIBE TABLE... PARTITION is not supported for v2 tables.") | ||
| } | ||
| DescribeTable(catalog.asTableCatalog, ident, isExtended) | ||
|
|
||
| case DropTableStatement(CatalogObjectIdentifier(Some(catalog), ident), ifExists, _) => | ||
| DropTable(catalog.asTableCatalog, ident, ifExists) | ||
|
|
||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -23,8 +23,9 @@ import scala.collection.mutable | |
| import org.apache.spark.sql.{AnalysisException, Strategy} | ||
| import org.apache.spark.sql.catalyst.expressions.{And, AttributeReference, AttributeSet, Expression, PredicateHelper, SubqueryExpression} | ||
| import org.apache.spark.sql.catalyst.planning.PhysicalOperation | ||
| import org.apache.spark.sql.catalyst.plans.logical.{AppendData, CreateTableAsSelect, CreateV2Table, DropTable, LogicalPlan, OverwriteByExpression, OverwritePartitionsDynamic, Repartition} | ||
| import org.apache.spark.sql.catalyst.plans.logical.{AppendData, CreateTableAsSelect, CreateV2Table, DescribeTable, DropTable, LogicalPlan, OverwriteByExpression, OverwritePartitionsDynamic, Repartition} | ||
| import org.apache.spark.sql.execution.{FilterExec, ProjectExec, SparkPlan} | ||
| import org.apache.spark.sql.execution.command.ExecutedCommandExec | ||
| import org.apache.spark.sql.execution.datasources.DataSourceStrategy | ||
| import org.apache.spark.sql.execution.streaming.continuous.{ContinuousCoalesceExec, WriteToContinuousDataSource, WriteToContinuousDataSourceExec} | ||
| import org.apache.spark.sql.sources | ||
|
|
@@ -199,6 +200,9 @@ object DataSourceV2Strategy extends Strategy with PredicateHelper { | |
| Nil | ||
| } | ||
|
|
||
| case DescribeTable(catalog, ident, isExtended) => | ||
| DescribeTableExec(catalog, ident, isExtended) :: Nil | ||
|
||
|
|
||
| case DropTable(catalog, ident, ifExists) => | ||
| DropTableExec(catalog, ident, ifExists) :: Nil | ||
|
|
||
|
|
||
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.
Singular,
DescribeTableSchema?