Skip to content

Commit

Permalink
Simplify SHOW CURRENT NAMESPACE command
Browse files Browse the repository at this point in the history
  • Loading branch information
cloud-fan committed Sep 28, 2021
1 parent 8c51fd8 commit 9222d6e
Show file tree
Hide file tree
Showing 7 changed files with 28 additions and 38 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -3572,14 +3572,6 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with SQLConfHelper with Logg
UseStatement(ctx.NAMESPACE != null, nameParts)
}

/**
* Create a [[ShowCurrentNamespace]].
*/
override def visitShowCurrentNamespace(
ctx: ShowCurrentNamespaceContext) : LogicalPlan = withOrigin(ctx) {
ShowCurrentNamespace()
}

/**
* Create a [[ShowTables]] command.
*/
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -636,15 +636,6 @@ case class RefreshTable(child: LogicalPlan) extends UnaryCommand {
copy(child = newChild)
}

/**
* The logical plan of the SHOW CURRENT NAMESPACE command.
*/
case class ShowCurrentNamespace() extends LeafCommand {
override val output: Seq[Attribute] = Seq(
AttributeReference("catalog", StringType, nullable = false)(),
AttributeReference("namespace", StringType, nullable = false)())
}

/**
* The logical plan of the SHOW TBLPROPERTIES command.
*/
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -2101,12 +2101,6 @@ class DDLParserSuite extends AnalysisTest {
""".stripMargin)
}

test("show current namespace") {
comparePlans(
parsePlan("SHOW CURRENT NAMESPACE"),
ShowCurrentNamespace())
}

test("alter table: SerDe properties") {
val sql1 = "ALTER TABLE table_name SET SERDE 'org.apache.class'"
val hint = Some("Please use ALTER VIEW instead.")
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -231,6 +231,14 @@ class SparkSqlAstBuilder extends AstBuilder {
DescribeQueryCommand(source(ctx.query), visitQuery(ctx.query))
}

/**
* Create a [[ShowCurrentNamespaceCommand]] logical command.
*/
override def visitShowCurrentNamespace(
ctx: ShowCurrentNamespaceContext) : LogicalPlan = withOrigin(ctx) {
ShowCurrentNamespaceCommand()
}

/**
* Converts a multi-part identifier to a TableIdentifier.
*
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -15,21 +15,23 @@
* limitations under the License.
*/

package org.apache.spark.sql.execution.datasources.v2
package org.apache.spark.sql.execution.command

import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.Attribute
import org.apache.spark.sql.connector.catalog.CatalogManager
import org.apache.spark.sql.{Row, SparkSession}
import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference}
import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.NamespaceHelper
import org.apache.spark.sql.types.StringType

/**
* Physical plan node for showing current catalog/namespace.
* The command for `SHOW CURRENT NAMESPACE`.
*/
case class ShowCurrentNamespaceExec(
output: Seq[Attribute],
catalogManager: CatalogManager)
extends LeafV2CommandExec {
override protected def run(): Seq[InternalRow] = {
Seq(toCatalystRow(catalogManager.currentCatalog.name, catalogManager.currentNamespace.quoted))
case class ShowCurrentNamespaceCommand() extends LeafRunnableCommand {
override val output: Seq[Attribute] = Seq(
AttributeReference("catalog", StringType, nullable = false)(),
AttributeReference("namespace", StringType, nullable = false)())

override def run(sparkSession: SparkSession): Seq[Row] = {
val catalogManager = sparkSession.sessionState.catalogManager
Seq(Row(catalogManager.currentCatalog.name, catalogManager.currentNamespace.quoted))
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -335,10 +335,7 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat
case SetCatalogAndNamespace(catalogManager, catalogName, ns) =>
SetCatalogAndNamespaceExec(catalogManager, catalogName, ns) :: Nil

case r: ShowCurrentNamespace =>
ShowCurrentNamespaceExec(r.output, session.sessionState.catalogManager) :: Nil

case r @ ShowTableProperties(rt: ResolvedTable, propertyKey, output) =>
case ShowTableProperties(rt: ResolvedTable, propertyKey, output) =>
ShowTablePropertiesExec(output, rt.table, propertyKey) :: Nil

case AnalyzeTable(_: ResolvedTable, _, _) | AnalyzeColumn(_: ResolvedTable, _, _) =>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -48,6 +48,12 @@ class DDLParserSuite extends AnalysisTest with SharedSparkSession {
comparePlans(plan, expected, checkAnalysis = false)
}

test("show current namespace") {
comparePlans(
parser.parsePlan("SHOW CURRENT NAMESPACE"),
ShowCurrentNamespaceCommand())
}

test("alter database - property values must be set") {
assertUnsupported(
sql = "ALTER DATABASE my_db SET DBPROPERTIES('key_without_value', 'key_with_value'='x')",
Expand Down

0 comments on commit 9222d6e

Please sign in to comment.