Skip to content

Commit

Permalink
[SPARK-30214][SQL][FOLLOWUP] Remove statement logical plans for names…
Browse files Browse the repository at this point in the history
…pace commands

### What changes were proposed in this pull request?

This is a follow-up to address the following comment: #27095 (comment)

Currently, a SQL command string is parsed to a "statement" logical plan, converted to a logical plan with catalog/namespace, then finally converted to a physical plan. With the new resolution framework, there is no need to create a "statement" logical plan; a logical plan can contain `UnresolvedNamespace` which will be resolved to a `ResolvedNamespace`. This should simply the code base and make it a bit easier to add a new command.

### Why are the changes needed?

Clean up codebase.

### Does this PR introduce any user-facing change?

No

### How was this patch tested?

Existing tests should cover the changes.

Closes #27125 from imback82/SPARK-30214-followup.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
  • Loading branch information
imback82 authored and cloud-fan committed Jan 8, 2020
1 parent 0d589f4 commit b2ed6d0
Show file tree
Hide file tree
Showing 8 changed files with 91 additions and 159 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -730,7 +730,7 @@ class Analyzer(
case class ResolveNamespace(catalogManager: CatalogManager)
extends Rule[LogicalPlan] with LookupCatalog {
def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
case s @ ShowTablesStatement(UnresolvedNamespace(Seq()), _) =>
case s @ ShowTables(UnresolvedNamespace(Seq()), _) =>
s.copy(namespace =
ResolvedNamespace(currentCatalog.asNamespaceCatalog, catalogManager.currentNamespace))
case UnresolvedNamespace(Seq()) =>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -106,15 +106,6 @@ class ResolveCatalogs(val catalogManager: CatalogManager)
s"Can not specify catalog `${catalog.name}` for view ${tbl.quoted} " +
s"because view support in catalog has not been implemented yet")

case AlterNamespaceSetPropertiesStatement(
NonSessionCatalogAndNamespace(catalog, ns), properties) =>
AlterNamespaceSetProperties(catalog, ns, properties)

case AlterNamespaceSetLocationStatement(
NonSessionCatalogAndNamespace(catalog, ns), location) =>
AlterNamespaceSetProperties(catalog, ns,
Map(SupportsNamespaces.PROP_LOCATION -> location))

case RenameTableStatement(NonSessionCatalogAndTable(catalog, oldName), newNameParts, isView) =>
if (isView) {
throw new AnalysisException("Renaming view is not supported in v2 catalogs.")
Expand Down Expand Up @@ -194,18 +185,6 @@ class ResolveCatalogs(val catalogManager: CatalogManager)
if !isSessionCatalog(catalog) =>
CreateNamespace(catalog.asNamespaceCatalog, ns, c.ifNotExists, c.properties)

case DropNamespaceStatement(NonSessionCatalogAndNamespace(catalog, ns), ifExists, cascade) =>
DropNamespace(catalog, ns, ifExists, cascade)

case DescribeNamespaceStatement(NonSessionCatalogAndNamespace(catalog, ns), extended) =>
DescribeNamespace(catalog, ns, extended)

case ShowNamespacesStatement(NonSessionCatalogAndNamespace(catalog, ns), pattern) =>
ShowNamespaces(catalog, ns, pattern)

case ShowTablesStatement(NonSessionCatalogAndNamespace(catalog, ns), pattern) =>
ShowTables(catalog.asTableCatalog, ns, pattern)

case UseStatement(isNamespaceSet, nameParts) =>
if (isNamespaceSet) {
SetCatalogAndNamespace(catalogManager, None, Some(nameParts))
Expand All @@ -231,13 +210,4 @@ class ResolveCatalogs(val catalogManager: CatalogManager)
case _ => None
}
}

object NonSessionCatalogAndNamespace {
def unapply(resolved: ResolvedNamespace): Option[(SupportsNamespaces, Seq[String])] =
if (!isSessionCatalog(resolved.catalog)) {
Some(resolved.catalog -> resolved.namespace)
} else {
None
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -2563,22 +2563,22 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
}

/**
* Create a [[DropNamespaceStatement]] command.
* Create a [[DropNamespace]] command.
*
* For example:
* {{{
* DROP (DATABASE|SCHEMA|NAMESPACE) [IF EXISTS] ns1.ns2 [RESTRICT|CASCADE];
* }}}
*/
override def visitDropNamespace(ctx: DropNamespaceContext): LogicalPlan = withOrigin(ctx) {
DropNamespaceStatement(
DropNamespace(
UnresolvedNamespace(visitMultipartIdentifier(ctx.multipartIdentifier)),
ctx.EXISTS != null,
ctx.CASCADE != null)
}

/**
* Create an [[AlterNamespaceSetPropertiesStatement]] logical plan.
* Create an [[AlterNamespaceSetProperties]] logical plan.
*
* For example:
* {{{
Expand All @@ -2588,14 +2588,14 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
*/
override def visitSetNamespaceProperties(ctx: SetNamespacePropertiesContext): LogicalPlan = {
withOrigin(ctx) {
AlterNamespaceSetPropertiesStatement(
AlterNamespaceSetProperties(
UnresolvedNamespace(visitMultipartIdentifier(ctx.multipartIdentifier)),
visitPropertyKeyValues(ctx.tablePropertyList))
}
}

/**
* Create an [[AlterNamespaceSetLocationStatement]] logical plan.
* Create an [[AlterNamespaceSetLocation]] logical plan.
*
* For example:
* {{{
Expand All @@ -2604,28 +2604,28 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
*/
override def visitSetNamespaceLocation(ctx: SetNamespaceLocationContext): LogicalPlan = {
withOrigin(ctx) {
AlterNamespaceSetLocationStatement(
AlterNamespaceSetLocation(
UnresolvedNamespace(visitMultipartIdentifier(ctx.multipartIdentifier)),
visitLocationSpec(ctx.locationSpec))
}
}

/**
* Create a [[ShowNamespacesStatement]] command.
* Create a [[ShowNamespaces]] command.
*/
override def visitShowNamespaces(ctx: ShowNamespacesContext): LogicalPlan = withOrigin(ctx) {
if (ctx.DATABASES != null && ctx.multipartIdentifier != null) {
throw new ParseException(s"FROM/IN operator is not allowed in SHOW DATABASES", ctx)
}

val multiPart = Option(ctx.multipartIdentifier).map(visitMultipartIdentifier)
ShowNamespacesStatement(
ShowNamespaces(
UnresolvedNamespace(multiPart.getOrElse(Seq.empty[String])),
Option(ctx.pattern).map(string))
}

/**
* Create a [[DescribeNamespaceStatement]].
* Create a [[DescribeNamespace]].
*
* For example:
* {{{
Expand All @@ -2634,7 +2634,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
*/
override def visitDescribeNamespace(ctx: DescribeNamespaceContext): LogicalPlan =
withOrigin(ctx) {
DescribeNamespaceStatement(
DescribeNamespace(
UnresolvedNamespace(visitMultipartIdentifier(ctx.multipartIdentifier())),
ctx.EXTENDED != null)
}
Expand Down Expand Up @@ -2802,11 +2802,11 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
}

/**
* Create a [[ShowTablesStatement]] command.
* Create a [[ShowTables]] command.
*/
override def visitShowTables(ctx: ShowTablesContext): LogicalPlan = withOrigin(ctx) {
val multiPart = Option(ctx.multipartIdentifier).map(visitMultipartIdentifier)
ShowTablesStatement(
ShowTables(
UnresolvedNamespace(multiPart.getOrElse(Seq.empty[String])),
Option(ctx.pattern).map(string))
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -298,15 +298,6 @@ case class DescribeTableStatement(
partitionSpec: TablePartitionSpec,
isExtended: Boolean) extends ParsedStatement

/**
* A DESCRIBE NAMESPACE statement, as parsed from SQL.
*/
case class DescribeNamespaceStatement(
namespace: LogicalPlan,
extended: Boolean) extends ParsedStatement {
override def children: Seq[LogicalPlan] = Seq(namespace)
}

/**
* A DESCRIBE TABLE tbl_name col_name statement, as parsed from SQL.
*/
Expand Down Expand Up @@ -345,15 +336,6 @@ case class InsertIntoStatement(
override def children: Seq[LogicalPlan] = query :: Nil
}

/**
* A SHOW TABLES statement, as parsed from SQL.
*/
case class ShowTablesStatement(
namespace: LogicalPlan,
pattern: Option[String]) extends ParsedStatement {
override def children: Seq[LogicalPlan] = Seq(namespace)
}

/**
* A SHOW TABLE EXTENDED statement, as parsed from SQL.
*/
Expand All @@ -371,43 +353,6 @@ case class CreateNamespaceStatement(
ifNotExists: Boolean,
properties: Map[String, String]) extends ParsedStatement

/**
* A DROP NAMESPACE statement, as parsed from SQL.
*/
case class DropNamespaceStatement(
namespace: LogicalPlan,
ifExists: Boolean,
cascade: Boolean) extends ParsedStatement {
override def children: Seq[LogicalPlan] = Seq(namespace)
}

/**
* ALTER (DATABASE|SCHEMA|NAMESPACE) ... SET (DBPROPERTIES|PROPERTIES) command, as parsed from SQL.
*/
case class AlterNamespaceSetPropertiesStatement(
namespace: LogicalPlan,
properties: Map[String, String]) extends ParsedStatement {
override def children: Seq[LogicalPlan] = Seq(namespace)
}

/**
* ALTER (DATABASE|SCHEMA|NAMESPACE) ... SET LOCATION command, as parsed from SQL.
*/
case class AlterNamespaceSetLocationStatement(
namespace: LogicalPlan,
location: String) extends ParsedStatement {
override def children: Seq[LogicalPlan] = Seq(namespace)
}

/**
* A SHOW NAMESPACES statement, as parsed from SQL.
*/
case class ShowNamespacesStatement(
namespace: LogicalPlan,
pattern: Option[String]) extends ParsedStatement {
override def children: Seq[LogicalPlan] = Seq(namespace)
}

/**
* A USE statement, as parsed from SQL.
*/
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -250,18 +250,19 @@ case class CreateNamespace(
* The logical plan of the DROP NAMESPACE command that works for v2 catalogs.
*/
case class DropNamespace(
catalog: CatalogPlugin,
namespace: Seq[String],
namespace: LogicalPlan,
ifExists: Boolean,
cascade: Boolean) extends Command
cascade: Boolean) extends Command {
override def children: Seq[LogicalPlan] = Seq(namespace)
}

/**
* The logical plan of the DESCRIBE NAMESPACE command that works for v2 catalogs.
*/
case class DescribeNamespace(
catalog: SupportsNamespaces,
namespace: Seq[String],
namespace: LogicalPlan,
extended: Boolean) extends Command {
override def children: Seq[LogicalPlan] = Seq(namespace)

override def output: Seq[Attribute] = Seq(
AttributeReference("name", StringType, nullable = false,
Expand All @@ -275,17 +276,29 @@ case class DescribeNamespace(
* command that works for v2 catalogs.
*/
case class AlterNamespaceSetProperties(
catalog: SupportsNamespaces,
namespace: Seq[String],
properties: Map[String, String]) extends Command
namespace: LogicalPlan,
properties: Map[String, String]) extends Command {
override def children: Seq[LogicalPlan] = Seq(namespace)
}

/**
* The logical plan of the ALTER (DATABASE|SCHEMA|NAMESPACE) ... SET LOCATION
* command that works for v2 catalogs.
*/
case class AlterNamespaceSetLocation(
namespace: LogicalPlan,
location: String) extends Command {
override def children: Seq[LogicalPlan] = Seq(namespace)
}

/**
* The logical plan of the SHOW NAMESPACES command that works for v2 catalogs.
*/
case class ShowNamespaces(
catalog: SupportsNamespaces,
namespace: Seq[String],
namespace: LogicalPlan,
pattern: Option[String]) extends Command {
override def children: Seq[LogicalPlan] = Seq(namespace)

override val output: Seq[Attribute] = Seq(
AttributeReference("namespace", StringType, nullable = false)())
}
Expand Down Expand Up @@ -412,9 +425,10 @@ case class RenameTable(
* The logical plan of the SHOW TABLE command that works for v2 catalogs.
*/
case class ShowTables(
catalog: TableCatalog,
namespace: Seq[String],
namespace: LogicalPlan,
pattern: Option[String]) extends Command {
override def children: Seq[LogicalPlan] = Seq(namespace)

override val output: Seq[Attribute] = Seq(
AttributeReference("namespace", StringType, nullable = false)(),
AttributeReference("tableName", StringType, nullable = false)())
Expand Down
Loading

0 comments on commit b2ed6d0

Please sign in to comment.