Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -198,7 +198,7 @@ statement
| (DESC | DESCRIBE) FUNCTION EXTENDED? describeFuncName #describeFunction
| (DESC | DESCRIBE) database EXTENDED? db=errorCapturingIdentifier #describeDatabase
| (DESC | DESCRIBE) TABLE? option=(EXTENDED | FORMATTED)?
tableIdentifier partitionSpec? describeColName? #describeTable
multipartIdentifier partitionSpec? describeColName? #describeTable
| (DESC | DESCRIBE) QUERY? query #describeQuery
| REFRESH TABLE tableIdentifier #refreshTable
| REFRESH (STRING | .*?) #refreshResource
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -36,7 +36,7 @@ import org.apache.spark.sql.catalyst.expressions.aggregate._
import org.apache.spark.sql.catalyst.expressions.objects._
import org.apache.spark.sql.catalyst.plans._
import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.sql.catalyst.plans.logical.sql.{AlterTableAddColumnsStatement, AlterTableAlterColumnStatement, AlterTableDropColumnsStatement, AlterTableRenameColumnStatement, AlterTableSetLocationStatement, AlterTableSetPropertiesStatement, AlterTableUnsetPropertiesStatement, InsertIntoStatement}
import org.apache.spark.sql.catalyst.plans.logical.sql.{AlterTableAddColumnsStatement, AlterTableAlterColumnStatement, AlterTableDropColumnsStatement, AlterTableRenameColumnStatement, AlterTableSetLocationStatement, AlterTableSetPropertiesStatement, AlterTableUnsetPropertiesStatement, DescribeTableStatement, InsertIntoStatement}
import org.apache.spark.sql.catalyst.rules._
import org.apache.spark.sql.catalyst.trees.TreeNodeRef
import org.apache.spark.sql.catalyst.util.toPrettySQL
Expand Down Expand Up @@ -171,6 +171,7 @@ class Analyzer(
Batch("Resolution", fixedPoint,
ResolveTableValuedFunctions ::
ResolveAlterTable ::
ResolveDescribeTable ::
ResolveInsertInto ::
ResolveTables ::
ResolveRelations ::
Expand Down Expand Up @@ -972,6 +973,21 @@ class Analyzer(
Seq(TableChange.setProperty("location", newLoc)))
}
}
/**
* Resolve DESCRIBE TABLE statements that use a DSv2 catalog.
*
* This rule converts unresolved DESCRIBE TABLE statements to v2 when a v2 catalog is responsible
* for the table identifier. A v2 catalog is responsible for an identifier when the identifier
* has a catalog specified, like prod_catalog.db.table, or when a default v2 catalog is set and
* the table identifier does not include a catalog.
*/
object ResolveDescribeTable extends Rule[LogicalPlan] {
override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
case describe @ DescribeTableStatement(
CatalogObjectIdentifier(Some(v2Catalog), ident), _, isExtended) =>
DescribeTable(UnresolvedRelation(describe.tableName), isExtended)
}
}

/**
* Replaces [[UnresolvedAttribute]]s with concrete [[AttributeReference]]s from
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -38,7 +38,7 @@ import org.apache.spark.sql.catalyst.expressions.aggregate.{First, Last}
import org.apache.spark.sql.catalyst.parser.SqlBaseParser._
import org.apache.spark.sql.catalyst.plans._
import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.sql.catalyst.plans.logical.sql.{AlterTableAddColumnsStatement, AlterTableAlterColumnStatement, AlterTableDropColumnsStatement, AlterTableRenameColumnStatement, AlterTableSetLocationStatement, AlterTableSetPropertiesStatement, AlterTableUnsetPropertiesStatement, AlterViewSetPropertiesStatement, AlterViewUnsetPropertiesStatement, CreateTableAsSelectStatement, CreateTableStatement, DropTableStatement, DropViewStatement, InsertIntoStatement, QualifiedColType, ReplaceTableAsSelectStatement, ReplaceTableStatement}
import org.apache.spark.sql.catalyst.plans.logical.sql.{AlterTableAddColumnsStatement, AlterTableAlterColumnStatement, AlterTableDropColumnsStatement, AlterTableRenameColumnStatement, AlterTableSetLocationStatement, AlterTableSetPropertiesStatement, AlterTableUnsetPropertiesStatement, AlterViewSetPropertiesStatement, AlterViewUnsetPropertiesStatement, CreateTableAsSelectStatement, CreateTableStatement, DescribeColumnStatement, DescribeTableStatement, DropTableStatement, DropViewStatement, InsertIntoStatement, QualifiedColType, ReplaceTableAsSelectStatement, ReplaceTableStatement}
import org.apache.spark.sql.catalyst.util.DateTimeUtils.{getZoneId, stringToDate, stringToTimestamp}
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types._
Expand Down Expand Up @@ -2561,4 +2561,36 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
visitMultipartIdentifier(ctx.multipartIdentifier),
visitLocationSpec(ctx.locationSpec))
}

/**
* Create a [[DescribeColumnStatement]] or [[DescribeTableStatement]] 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 {
DescribeColumnStatement(
visitMultipartIdentifier(ctx.multipartIdentifier()),
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]
}
DescribeTableStatement(
visitMultipartIdentifier(ctx.multipartIdentifier()),
partitionSpec,
isExtended)
}
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,31 @@
/*
* 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 DescribeTableSchema {
def describeTableAttributes(): Seq[AttributeReference] = 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())())
}
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@ 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._
Expand Down Expand Up @@ -541,6 +542,13 @@ object OverwritePartitionsDynamic {
}
}

case class DescribeTable(table: NamedRelation, isExtended: Boolean) extends Command {

override def children: Seq[LogicalPlan] = Seq(table)

override val output = DescribeTableSchema.describeTableAttributes()
}

/**
* Drop a table.
*/
Expand Down
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.DescribeTableSchema

case class DescribeTableStatement(
tableName: Seq[String],
partitionSpec: TablePartitionSpec,
isExtended: Boolean) extends ParsedStatement {
override def output: Seq[Attribute] = DescribeTableSchema.describeTableAttributes()
}
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,7 @@ import org.apache.spark.sql.catalog.v2.expressions.{ApplyTransform, BucketTransf
import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, UnresolvedRelation, UnresolvedStar}
import org.apache.spark.sql.catalyst.catalog.BucketSpec
import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project}
import org.apache.spark.sql.catalyst.plans.logical.sql.{AlterTableAddColumnsStatement, AlterTableAlterColumnStatement, AlterTableDropColumnsStatement, AlterTableRenameColumnStatement, AlterTableSetLocationStatement, AlterTableSetPropertiesStatement, AlterTableUnsetPropertiesStatement, AlterViewSetPropertiesStatement, AlterViewUnsetPropertiesStatement, CreateTableAsSelectStatement, CreateTableStatement, DropTableStatement, DropViewStatement, InsertIntoStatement, QualifiedColType, ReplaceTableAsSelectStatement, ReplaceTableStatement}
import org.apache.spark.sql.catalyst.plans.logical.sql.{AlterTableAddColumnsStatement, AlterTableAlterColumnStatement, AlterTableDropColumnsStatement, AlterTableRenameColumnStatement, AlterTableSetLocationStatement, AlterTableSetPropertiesStatement, AlterTableUnsetPropertiesStatement, AlterViewSetPropertiesStatement, AlterViewUnsetPropertiesStatement, CreateTableAsSelectStatement, CreateTableStatement, DescribeColumnStatement, DescribeTableStatement, DropTableStatement, DropViewStatement, InsertIntoStatement, QualifiedColType, ReplaceTableAsSelectStatement, ReplaceTableStatement}
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

These are getting really really long, and in particular the merge conflicts are a bit tedious to resolve. I'm normally very averse to wildcard imports, but there might come a point where we'll have to do that. Or I wonder if we could have a helper object that bundles all of these, or factory methods for these, or matchers... somehow.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

+1 for wildcard here.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This package may make sense for a wildcard import because it has no sub-packages and is unlikely to in the future. Still, because Scala will import sub-packages, I think it's probably best to keep avoiding wildcard imports, even here.

import org.apache.spark.sql.types.{IntegerType, LongType, StringType, StructType, TimestampType}
import org.apache.spark.unsafe.types.UTF8String

Expand Down Expand Up @@ -617,6 +617,47 @@ class DDLParserSuite extends AnalysisTest {
}
}

test("describe table column") {
comparePlans(parsePlan("DESCRIBE t col"),
DescribeColumnStatement(
Seq("t"), Seq("col"), isExtended = false))
comparePlans(parsePlan("DESCRIBE t `abc.xyz`"),
DescribeColumnStatement(
Seq("t"), Seq("abc.xyz"), isExtended = false))
comparePlans(parsePlan("DESCRIBE t abc.xyz"),
DescribeColumnStatement(
Seq("t"), Seq("abc", "xyz"), isExtended = false))
comparePlans(parsePlan("DESCRIBE t `a.b`.`x.y`"),
DescribeColumnStatement(
Seq("t"), Seq("a.b", "x.y"), isExtended = false))

comparePlans(parsePlan("DESCRIBE TABLE t col"),
DescribeColumnStatement(
Seq("t"), Seq("col"), isExtended = false))
comparePlans(parsePlan("DESCRIBE TABLE EXTENDED t col"),
DescribeColumnStatement(
Seq("t"), Seq("col"), isExtended = true))
comparePlans(parsePlan("DESCRIBE TABLE FORMATTED t col"),
DescribeColumnStatement(
Seq("t"), Seq("col"), isExtended = true))

val caught = intercept[AnalysisException](
parsePlan("DESCRIBE TABLE t PARTITION (ds='1970-01-01') col"))
assert(caught.getMessage.contains(
"DESC TABLE COLUMN for a specific partition is not supported"))
}

test("SPARK-17328 Fix NPE with EXPLAIN DESCRIBE TABLE") {
comparePlans(parsePlan("describe t"),
DescribeTableStatement(Seq("t"), Map.empty, isExtended = false))
comparePlans(parsePlan("describe table t"),
DescribeTableStatement(Seq("t"), Map.empty, isExtended = false))
comparePlans(parsePlan("describe table extended t"),
DescribeTableStatement(Seq("t"), Map.empty, isExtended = true))
comparePlans(parsePlan("describe table formatted t"),
DescribeTableStatement(Seq("t"), Map.empty, isExtended = true))
}

test("insert table: basic append") {
Seq(
"INSERT INTO TABLE testcat.ns1.ns2.tbl SELECT * FROM source",
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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),
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Now that these rules create DescribeColumnStatement and DescribeTableStatement, they should be moved into Catalyst. There isn't anything specific to the implementation any more.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Do you mean that DescribeColumnCommand and DescribeTableCommand should be moved to Catalyst? The V1 commands depend on a bunch of stuff that's in core, such as SparkSession and DDLUtils.

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.
*/
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -34,6 +34,7 @@ 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.plans.DescribeTableSchema
import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.sql.catalyst.util.{escapeSingleQuotedString, quoteIdentifier}
import org.apache.spark.sql.execution.datasources.{DataSource, PartitioningUtils}
Expand Down Expand Up @@ -499,15 +500,7 @@ case class TruncateTableCommand(
}

abstract class DescribeCommandBase extends RunnableCommand {
override val output: Seq[Attribute] = Seq(
// Column names are based on Hive.
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())()
)
override val output = DescribeTableSchema.describeTableAttributes()

protected def describeSchema(
schema: StructType,
Expand Down
Loading