Skip to content

[SPARK-30497][SQL] migrate DESCRIBE TABLE to the new framework #27187

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

Closed
wants to merge 3 commits into from
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 @@ -218,7 +218,7 @@ statement
| (DESC | DESCRIBE) namespace EXTENDED?
multipartIdentifier #describeNamespace
| (DESC | DESCRIBE) TABLE? option=(EXTENDED | FORMATTED)?
multipartIdentifier partitionSpec? describeColName? #describeTable
multipartIdentifier partitionSpec? describeColName? #describeRelation
| (DESC | DESCRIBE) QUERY? query #describeQuery
| COMMENT ON namespace multipartIdentifier IS
comment=(STRING | NULL) #commentNamespace
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -759,6 +759,8 @@ class Analyzer(
u.failAnalysis(s"${ident.quoted} is a temp view not table.")
}
u
case u @ UnresolvedTableOrView(ident) =>
lookupTempView(ident).map(_ => ResolvedView(ident.asIdentifier)).getOrElse(u)
}

def lookupTempView(identifier: Seq[String]): Option[LogicalPlan] = {
Expand Down Expand Up @@ -803,16 +805,16 @@ class Analyzer(
.map(ResolvedTable(catalog.asTableCatalog, ident, _))
.getOrElse(u)

case u @ UnresolvedTableOrView(NonSessionCatalogAndIdentifier(catalog, ident)) =>
CatalogV2Util.loadTable(catalog, ident)
.map(ResolvedTable(catalog.asTableCatalog, ident, _))
.getOrElse(u)

case i @ InsertIntoStatement(u: UnresolvedRelation, _, _, _, _) if i.query.resolved =>
lookupV2Relation(u.multipartIdentifier)
.map(v2Relation => i.copy(table = v2Relation))
.getOrElse(i)

case desc @ DescribeTable(u: UnresolvedV2Relation, _) =>
CatalogV2Util.loadRelation(u.catalog, u.tableName)
.map(rel => desc.copy(table = rel))
.getOrElse(desc)

case alter @ AlterTable(_, _, u: UnresolvedV2Relation, _) =>
CatalogV2Util.loadRelation(u.catalog, u.tableName)
.map(rel => alter.copy(table = rel))
Expand Down Expand Up @@ -889,17 +891,28 @@ class Analyzer(
case u: UnresolvedRelation =>
lookupRelation(u.multipartIdentifier).map(resolveViews).getOrElse(u)

case u @ UnresolvedTable(identifier: Seq[String]) =>
expandRelationName(identifier) match {
case SessionCatalogAndIdentifier(catalog, ident) =>
CatalogV2Util.loadTable(catalog, ident) match {
case Some(v1Table: V1Table) if v1Table.v1Table.tableType == CatalogTableType.VIEW =>
u.failAnalysis(s"$ident is a view not table.")
case Some(table) => ResolvedTable(catalog.asTableCatalog, ident, table)
case None => u
}
case _ => u
}
case u @ UnresolvedTable(identifier) =>
lookupTableOrView(identifier).map {
case v: ResolvedView =>
u.failAnalysis(s"${v.identifier.quoted} is a view not table.")
case table => table
}.getOrElse(u)

case u @ UnresolvedTableOrView(identifier) =>
lookupTableOrView(identifier).getOrElse(u)
}

private def lookupTableOrView(identifier: Seq[String]): Option[LogicalPlan] = {
expandRelationName(identifier) match {
case SessionCatalogAndIdentifier(catalog, ident) =>
CatalogV2Util.loadTable(catalog, ident).map {
case v1Table: V1Table if v1Table.v1Table.tableType == CatalogTableType.VIEW =>
ResolvedView(ident)
case table =>
ResolvedTable(catalog.asTableCatalog, ident, table)
}
case _ => None
}
}

// Look up a relation from the session catalog with the following logic:
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -98,6 +98,9 @@ trait CheckAnalysis extends PredicateHelper {
case u: UnresolvedTable =>
u.failAnalysis(s"Table not found: ${u.multipartIdentifier.quoted}")

case u: UnresolvedTableOrView =>
u.failAnalysis(s"Table or view not found: ${u.multipartIdentifier.quoted}")

case u: UnresolvedRelation =>
u.failAnalysis(s"Table or view not found: ${u.multipartIdentifier.quoted}")

Expand All @@ -118,13 +121,6 @@ trait CheckAnalysis extends PredicateHelper {
case AlterTable(_, _, u: UnresolvedV2Relation, _) =>
failAnalysis(s"Table not found: ${u.originalNameParts.quoted}")

case DescribeTable(u: UnresolvedV2Relation, _) if isView(u.originalNameParts) =>
u.failAnalysis(
s"Invalid command: '${u.originalNameParts.quoted}' is a view not a table.")

case DescribeTable(u: UnresolvedV2Relation, _) =>
failAnalysis(s"Table not found: ${u.originalNameParts.quoted}")

case operator: LogicalPlan =>
// Check argument data types of higher-order functions downwards first.
// If the arguments of the higher-order functions are resolved but the type check fails,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -115,14 +115,6 @@ class ResolveCatalogs(val catalogManager: CatalogManager)
}
RenameTable(catalog.asTableCatalog, oldName.asIdentifier, newNameParts.asIdentifier)

case DescribeTableStatement(
nameParts @ NonSessionCatalogAndTable(catalog, tbl), partitionSpec, isExtended) =>
if (partitionSpec.nonEmpty) {
throw new AnalysisException("DESCRIBE TABLE does not support partition for v2 tables.")
}
val r = UnresolvedV2Relation(nameParts, catalog.asTableCatalog, tbl.asIdentifier)
DescribeTable(r, isExtended)

case DescribeColumnStatement(
NonSessionCatalogAndTable(catalog, tbl), colNameParts, isExtended) =>
throw new AnalysisException("Describing columns is not supported for v2 tables.")
Expand Down

This file was deleted.

This file was deleted.

Original file line number Diff line number Diff line change
@@ -0,0 +1,76 @@
/*
* 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.analysis

import org.apache.spark.sql.catalyst.expressions.Attribute
import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan}
import org.apache.spark.sql.connector.catalog.{Identifier, SupportsNamespaces, Table, TableCatalog}

/**
* Holds the name of a namespace that has yet to be looked up in a catalog. It will be resolved to
* [[ResolvedNamespace]] during analysis.
*/
case class UnresolvedNamespace(multipartIdentifier: Seq[String]) extends LeafNode {
override lazy val resolved: Boolean = false

override def output: Seq[Attribute] = Nil
}

/**
* Holds the name of a table that has yet to be looked up in a catalog. It will be resolved to
* [[ResolvedTable]] during analysis.
*/
case class UnresolvedTable(multipartIdentifier: Seq[String]) extends LeafNode {
override lazy val resolved: Boolean = false

override def output: Seq[Attribute] = Nil
}

/**
* Holds the name of a table or view that has yet to be looked up in a catalog. It will
* be resolved to [[ResolvedTable]] or [[ResolvedView]] during analysis.
*/
case class UnresolvedTableOrView(multipartIdentifier: Seq[String]) extends LeafNode {
override lazy val resolved: Boolean = false
override def output: Seq[Attribute] = Nil
}

/**
* A plan containing resolved namespace.
*/
case class ResolvedNamespace(catalog: SupportsNamespaces, namespace: Seq[String])
extends LeafNode {
override def output: Seq[Attribute] = Nil
}

/**
* A plan containing resolved table.
*/
case class ResolvedTable(catalog: TableCatalog, identifier: Identifier, table: Table)
extends LeafNode {
override def output: Seq[Attribute] = Nil
}

/**
* A plan containing resolved (temp) views.
*/
// TODO: create a generic representation for temp view, v1 view and v2 view, after we add view
// support to v2 catalog. For now we only need the identifier to fallback to v1 command.
case class ResolvedView(identifier: Identifier) extends LeafNode {
Copy link
Contributor Author

Choose a reason for hiding this comment

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

We can add UnresolvedView when we migrate ALTER VIEW command. @imback82

Copy link
Contributor

Choose a reason for hiding this comment

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

Sounds good. Thanks!

override def output: Seq[Attribute] = Nil
}
Original file line number Diff line number Diff line change
Expand Up @@ -3062,9 +3062,9 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
}

/**
* Create a [[DescribeColumnStatement]] or [[DescribeTableStatement]] commands.
* Create a [[DescribeColumnStatement]] or [[DescribeRelation]] commands.
*/
override def visitDescribeTable(ctx: DescribeTableContext): LogicalPlan = withOrigin(ctx) {
override def visitDescribeRelation(ctx: DescribeRelationContext): LogicalPlan = withOrigin(ctx) {
val isExtended = ctx.EXTENDED != null || ctx.FORMATTED != null
if (ctx.describeColName != null) {
if (ctx.partitionSpec != null) {
Expand All @@ -3086,8 +3086,8 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
} else {
Map.empty[String, String]
}
DescribeTableStatement(
visitMultipartIdentifier(ctx.multipartIdentifier()),
DescribeRelation(
UnresolvedTableOrView(visitMultipartIdentifier(ctx.multipartIdentifier())),
partitionSpec,
isExtended)
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -292,14 +292,6 @@ case class DropViewStatement(
viewName: Seq[String],
ifExists: Boolean) extends ParsedStatement

/**
* A DESCRIBE TABLE tbl_name statement, as parsed from SQL.
*/
case class DescribeTableStatement(
tableName: Seq[String],
partitionSpec: TablePartitionSpec,
isExtended: Boolean) extends ParsedStatement

/**
* A DESCRIBE TABLE tbl_name col_name statement, as parsed from SQL.
*/
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@
package org.apache.spark.sql.catalyst.plans.logical

import org.apache.spark.sql.catalyst.analysis.{NamedRelation, UnresolvedException}
import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec
import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, Expression, Unevaluable}
import org.apache.spark.sql.catalyst.plans.DescribeTableSchema
import org.apache.spark.sql.connector.catalog._
Expand Down Expand Up @@ -314,12 +315,13 @@ case class ShowNamespaces(
}

/**
* The logical plan of the DESCRIBE TABLE command that works for v2 tables.
* The logical plan of the DESCRIBE relation_name command that works for v2 tables.
*/
case class DescribeTable(table: NamedRelation, isExtended: Boolean) extends Command {

override lazy val resolved: Boolean = table.resolved

case class DescribeRelation(
relation: LogicalPlan,
partitionSpec: TablePartitionSpec,
isExtended: Boolean) extends Command {
override def children: Seq[LogicalPlan] = Seq(relation)
Copy link
Member

Choose a reason for hiding this comment

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

this def is needed for all commands resolved by the new framework, we may create a trait to reduced redundancy

override def output: Seq[Attribute] = DescribeTableSchema.describeTableAttributes()
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -98,6 +98,14 @@ private[sql] object CatalogV2Implicits {
}

def asMultipartIdentifier: Seq[String] = ident.namespace :+ ident.name

def asTableIdentifier: TableIdentifier = ident.namespace match {
case ns if ns.isEmpty => TableIdentifier(ident.name)
case Array(dbName) => TableIdentifier(ident.name, Some(dbName))
case _ =>
throw new AnalysisException(
s"$quoted is not a valid TableIdentifier as it has more than 2 name parts.")
}
}

implicit class MultipartIdentifierHelper(parts: Seq[String]) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,7 @@ package org.apache.spark.sql.catalyst.parser
import java.util.Locale

import org.apache.spark.sql.AnalysisException
import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, GlobalTempView, LocalTempView, PersistedView, UnresolvedAttribute, UnresolvedNamespace, UnresolvedRelation, UnresolvedStar, UnresolvedTable}
import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, GlobalTempView, LocalTempView, PersistedView, UnresolvedAttribute, UnresolvedNamespace, UnresolvedRelation, UnresolvedStar, UnresolvedTable, UnresolvedTableOrView}
import org.apache.spark.sql.catalyst.catalog.{ArchiveResource, BucketSpec, FileResource, FunctionResource, FunctionResourceType, JarResource}
import org.apache.spark.sql.catalyst.expressions.{EqualTo, Literal}
import org.apache.spark.sql.catalyst.plans.logical._
Expand Down Expand Up @@ -791,13 +791,13 @@ class DDLParserSuite extends AnalysisTest {

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

test("insert table: basic append") {
Expand Down
Loading