Skip to content
Closed
Show file tree
Hide file tree
Changes from 3 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 @@ -730,6 +730,11 @@ class Analyzer(
case class ResolveNamespace(catalogManager: CatalogManager)
extends Rule[LogicalPlan] with LookupCatalog {
def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
case s @ ShowTablesStatement(UnresolvedNamespace(Seq()), _) =>
s.copy(namespace =
ResolvedNamespace(currentCatalog.asNamespaceCatalog, catalogManager.currentNamespace))
case UnresolvedNamespace(Seq()) =>
Copy link
Contributor

Choose a reason for hiding this comment

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

We can move this logic to CatalogAndNamespace, but I'm fine with what it is now.

ResolvedNamespace(currentCatalog.asNamespaceCatalog, Seq.empty[String])
case UnresolvedNamespace(CatalogAndNamespace(catalog, ns)) =>
ResolvedNamespace(catalog.asNamespaceCatalog, ns)
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -108,11 +108,11 @@ class ResolveCatalogs(val catalogManager: CatalogManager)

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

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

case RenameTableStatement(NonSessionCatalogAndTable(catalog, oldName), newNameParts, isView) =>
Expand Down Expand Up @@ -191,27 +191,20 @@ class ResolveCatalogs(val catalogManager: CatalogManager)
s"because view support in catalog has not been implemented yet")

case c @ CreateNamespaceStatement(NonSessionCatalogAndNamespace(catalog, ns), _, _) =>
CreateNamespace(catalog.asNamespaceCatalog, ns, c.ifNotExists, c.properties)
CreateNamespace(catalog, 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.asNamespaceCatalog, ns, extended)
DescribeNamespace(catalog, ns, extended)

case ShowNamespacesStatement(Some(CatalogAndNamespace(catalog, ns)), pattern) =>
val namespace = if (ns.nonEmpty) Some(ns) else None
ShowNamespaces(catalog.asNamespaceCatalog, namespace, pattern)
case ShowNamespacesStatement(NonSessionCatalogAndNamespace(catalog, ns), pattern) =>
ShowNamespaces(catalog, ns, pattern)

case ShowNamespacesStatement(None, pattern) =>
ShowNamespaces(currentCatalog.asNamespaceCatalog, None, pattern)

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

case ShowTablesStatement(None, pattern) if !isSessionCatalog(currentCatalog) =>
ShowTables(currentCatalog.asTableCatalog, catalogManager.currentNamespace, pattern)

case UseStatement(isNamespaceSet, nameParts) =>
if (isNamespaceSet) {
SetCatalogAndNamespace(catalogManager, None, Some(nameParts))
Expand Down Expand Up @@ -239,10 +232,11 @@ class ResolveCatalogs(val catalogManager: CatalogManager)
}

object NonSessionCatalogAndNamespace {
def unapply(nameParts: Seq[String]): Option[(CatalogPlugin, Seq[String])] = nameParts match {
case CatalogAndNamespace(catalog, ns) if !isSessionCatalog(catalog) =>
Some(catalog -> ns)
case _ => None
}
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 @@ -2533,7 +2533,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
}

CreateNamespaceStatement(
visitMultipartIdentifier(ctx.multipartIdentifier),
UnresolvedNamespace(visitMultipartIdentifier(ctx.multipartIdentifier)),
ctx.EXISTS != null,
properties)
}
Expand All @@ -2548,7 +2548,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
*/
override def visitDropNamespace(ctx: DropNamespaceContext): LogicalPlan = withOrigin(ctx) {
DropNamespaceStatement(
visitMultipartIdentifier(ctx.multipartIdentifier),
UnresolvedNamespace(visitMultipartIdentifier(ctx.multipartIdentifier)),
ctx.EXISTS != null,
ctx.CASCADE != null)
}
Expand All @@ -2565,7 +2565,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
override def visitSetNamespaceProperties(ctx: SetNamespacePropertiesContext): LogicalPlan = {
withOrigin(ctx) {
AlterNamespaceSetPropertiesStatement(
visitMultipartIdentifier(ctx.multipartIdentifier),
UnresolvedNamespace(visitMultipartIdentifier(ctx.multipartIdentifier)),
visitPropertyKeyValues(ctx.tablePropertyList))
}
}
Expand All @@ -2581,7 +2581,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
override def visitSetNamespaceLocation(ctx: SetNamespaceLocationContext): LogicalPlan = {
withOrigin(ctx) {
AlterNamespaceSetLocationStatement(
visitMultipartIdentifier(ctx.multipartIdentifier),
UnresolvedNamespace(visitMultipartIdentifier(ctx.multipartIdentifier)),
visitLocationSpec(ctx.locationSpec))
}
}
Expand All @@ -2594,8 +2594,9 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
throw new ParseException(s"FROM/IN operator is not allowed in SHOW DATABASES", ctx)
}

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

Expand All @@ -2610,7 +2611,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
override def visitDescribeNamespace(ctx: DescribeNamespaceContext): LogicalPlan =
withOrigin(ctx) {
DescribeNamespaceStatement(
visitMultipartIdentifier(ctx.multipartIdentifier()),
UnresolvedNamespace(visitMultipartIdentifier(ctx.multipartIdentifier())),
ctx.EXTENDED != null)
}

Expand Down Expand Up @@ -2780,8 +2781,9 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
* Create a [[ShowTablesStatement]] command.
*/
override def visitShowTables(ctx: ShowTablesContext): LogicalPlan = withOrigin(ctx) {
val multiPart = Option(ctx.multipartIdentifier).map(visitMultipartIdentifier)
ShowTablesStatement(
Option(ctx.multipartIdentifier).map(visitMultipartIdentifier),
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 @@ -302,8 +302,10 @@ case class DescribeTableStatement(
* A DESCRIBE NAMESPACE statement, as parsed from SQL.
*/
case class DescribeNamespaceStatement(
namespace: Seq[String],
extended: Boolean) extends ParsedStatement
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 @@ -346,8 +348,11 @@ case class InsertIntoStatement(
/**
* A SHOW TABLES statement, as parsed from SQL.
*/
case class ShowTablesStatement(namespace: Option[Seq[String]], pattern: Option[String])
extends ParsedStatement
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 @@ -362,37 +367,48 @@ case class ShowTableStatement(
* A CREATE NAMESPACE statement, as parsed from SQL.
*/
case class CreateNamespaceStatement(
namespace: Seq[String],
namespace: LogicalPlan,
Copy link
Contributor

@cloud-fan cloud-fan Jan 7, 2020

Choose a reason for hiding this comment

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

I'm not very sure about this. For CREATE commands, they only need to know the catalog, but no need to lookup table or namespace. I think we can keep using Seq[String], still handle the commands in ResolveCatalogs/ResolveSessionCatalogs and resolve the Seq[String] directly.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

yea you are right, and in this case, it will be consistent for tables as well.

ifNotExists: Boolean,
properties: Map[String, String]) extends ParsedStatement
properties: Map[String, String]) extends ParsedStatement {
override def children: Seq[LogicalPlan] = Seq(namespace)
}

/**
* A DROP NAMESPACE statement, as parsed from SQL.
*/
case class DropNamespaceStatement(
namespace: Seq[String],
namespace: LogicalPlan,
ifExists: Boolean,
cascade: Boolean) extends ParsedStatement
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: Seq[String],
properties: Map[String, String]) extends ParsedStatement
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: Seq[String],
location: String) extends ParsedStatement
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: Option[Seq[String]], pattern: Option[String])
extends ParsedStatement
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 @@ -284,7 +284,7 @@ case class AlterNamespaceSetProperties(
*/
case class ShowNamespaces(
catalog: SupportsNamespaces,
namespace: Option[Seq[String]],
namespace: Seq[String],
pattern: Option[String]) extends Command {
override val output: Seq[Attribute] = Seq(
AttributeReference("namespace", StringType, nullable = false)())
Expand Down
Loading