Skip to content

Commit

Permalink
[SPARK-46073][SQL] Remove the special resolution of UnresolvedNamespa…
Browse files Browse the repository at this point in the history
…ce for certain commands

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

It's a bit hacky that we have to special-cases some commands in `ResolveCatalogs` when resolving `UnresolvedNamespace`. This PR adds a new plan node `CurrentNamespace`, so that caller side can use it to get current namespace, instead of using `UnresolvedNamespace(Nil)` and special-case it in `ResolveCatalogs`.

### Why are the changes needed?

code cleanup

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

no

### How was this patch tested?

existing tests

### Was this patch authored or co-authored using generative AI tooling?

No

Closes apache#43980 from cloud-fan/minor.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
  • Loading branch information
cloud-fan authored and dongjoon-hyun committed Nov 24, 2023
1 parent 10b9c61 commit 1131a56
Show file tree
Hide file tree
Showing 9 changed files with 38 additions and 37 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -47,21 +47,9 @@ class ResolveCatalogs(val catalogManager: CatalogManager)
val CatalogAndIdentifier(catalog, identifier) = nameParts
ResolvedIdentifier(catalog, identifier)
}
case s @ ShowTables(UnresolvedNamespace(Seq()), _, _) =>
s.copy(namespace = ResolvedNamespace(currentCatalog,
catalogManager.currentNamespace.toImmutableArraySeq))
case s @ ShowTablesExtended(UnresolvedNamespace(Seq()), _, _) =>
s.copy(namespace = ResolvedNamespace(currentCatalog,
catalogManager.currentNamespace.toImmutableArraySeq))
case s @ ShowViews(UnresolvedNamespace(Seq()), _, _) =>
s.copy(namespace = ResolvedNamespace(currentCatalog,
catalogManager.currentNamespace.toImmutableArraySeq))
case s @ ShowFunctions(UnresolvedNamespace(Seq()), _, _, _, _) =>
s.copy(namespace = ResolvedNamespace(currentCatalog,
catalogManager.currentNamespace.toImmutableArraySeq))
case a @ AnalyzeTables(UnresolvedNamespace(Seq()), _) =>
a.copy(namespace = ResolvedNamespace(currentCatalog,
catalogManager.currentNamespace.toImmutableArraySeq))

case CurrentNamespace =>
ResolvedNamespace(currentCatalog, catalogManager.currentNamespace.toImmutableArraySeq)
case UnresolvedNamespace(Seq()) =>
ResolvedNamespace(currentCatalog, Seq.empty[String])
case UnresolvedNamespace(CatalogAndNamespace(catalog, ns)) =>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -39,6 +39,12 @@ import org.apache.spark.util.ArrayImplicits._
*/
case class UnresolvedNamespace(multipartIdentifier: Seq[String]) extends UnresolvedLeafNode

/**
* A variant of [[UnresolvedNamespace]] that should be resolved to [[ResolvedNamespace]]
* representing the current namespace of the current catalog.
*/
case object CurrentNamespace extends UnresolvedLeafNode

/**
* Holds the name of a table that has yet to be looked up in a catalog. It will be resolved to
* [[ResolvedTable]] during analysis.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -4100,7 +4100,7 @@ class AstBuilder extends DataTypeAstBuilder with SQLConfHelper with Logging {
val ns = if (ctx.identifierReference() != null) {
withIdentClause(ctx.identifierReference, UnresolvedNamespace(_))
} else {
UnresolvedNamespace(Seq.empty[String])
CurrentNamespace
}
ShowTables(ns, Option(ctx.pattern).map(x => string(visitStringLit(x))))
}
Expand All @@ -4127,7 +4127,7 @@ class AstBuilder extends DataTypeAstBuilder with SQLConfHelper with Logging {
val ns = if (ctx.identifierReference() != null) {
withIdentClause(ctx.identifierReference, UnresolvedNamespace)
} else {
UnresolvedNamespace(Seq.empty[String])
CurrentNamespace
}
ShowTablesExtended(ns, string(visitStringLit(ctx.pattern)))
}
Expand All @@ -4140,7 +4140,7 @@ class AstBuilder extends DataTypeAstBuilder with SQLConfHelper with Logging {
val ns = if (ctx.identifierReference() != null) {
withIdentClause(ctx.identifierReference, UnresolvedNamespace(_))
} else {
UnresolvedNamespace(Seq.empty[String])
CurrentNamespace
}
ShowViews(ns, Option(ctx.pattern).map(x => string(visitStringLit(x))))
}
Expand Down Expand Up @@ -4605,7 +4605,7 @@ class AstBuilder extends DataTypeAstBuilder with SQLConfHelper with Logging {
val ns = if (ctx.identifierReference() != null) {
withIdentClause(ctx.identifierReference, UnresolvedNamespace(_))
} else {
UnresolvedNamespace(Seq.empty[String])
CurrentNamespace
}
AnalyzeTables(ns, noScan = ctx.identifier != null)
}
Expand Down Expand Up @@ -5003,9 +5003,14 @@ class AstBuilder extends DataTypeAstBuilder with SQLConfHelper with Logging {
withIdentClause(ctx.ns, UnresolvedNamespace(_)),
userScope, systemScope, pattern)
} else if (legacy.isDefined) {
ShowFunctions(UnresolvedNamespace(legacy.get.dropRight(1)), userScope, systemScope, pattern)
val ns = if (legacy.get.length > 1) {
UnresolvedNamespace(legacy.get.dropRight(1))
} else {
CurrentNamespace
}
ShowFunctions(ns, userScope, systemScope, pattern)
} else {
ShowFunctions(UnresolvedNamespace(Nil), userScope, systemScope, pattern)
ShowFunctions(CurrentNamespace, userScope, systemScope, pattern)
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -2172,13 +2172,13 @@ class DDLParserSuite extends AnalysisTest {
test("show views") {
comparePlans(
parsePlan("SHOW VIEWS"),
ShowViews(UnresolvedNamespace(Seq.empty[String]), None))
ShowViews(CurrentNamespace, None))
comparePlans(
parsePlan("SHOW VIEWS '*test*'"),
ShowViews(UnresolvedNamespace(Seq.empty[String]), Some("*test*")))
ShowViews(CurrentNamespace, Some("*test*")))
comparePlans(
parsePlan("SHOW VIEWS LIKE '*test*'"),
ShowViews(UnresolvedNamespace(Seq.empty[String]), Some("*test*")))
ShowViews(CurrentNamespace, Some("*test*")))
comparePlans(
parsePlan("SHOW VIEWS FROM testcat.ns1.ns2.tbl"),
ShowViews(UnresolvedNamespace(Seq("testcat", "ns1", "ns2", "tbl")), None))
Expand Down Expand Up @@ -2273,6 +2273,8 @@ class DDLParserSuite extends AnalysisTest {
test("SPARK-33687: analyze tables statistics") {
comparePlans(parsePlan("ANALYZE TABLES IN a.b.c COMPUTE STATISTICS"),
AnalyzeTables(UnresolvedNamespace(Seq("a", "b", "c")), noScan = false))
comparePlans(parsePlan("ANALYZE TABLES COMPUTE STATISTICS"),
AnalyzeTables(CurrentNamespace, noScan = false))
comparePlans(parsePlan("ANALYZE TABLES FROM a COMPUTE STATISTICS NOSCAN"),
AnalyzeTables(UnresolvedNamespace(Seq("a")), noScan = true))

Expand Down
4 changes: 2 additions & 2 deletions sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
Original file line number Diff line number Diff line change
Expand Up @@ -29,7 +29,7 @@ import org.apache.spark.internal.Logging
import org.apache.spark.internal.config.ConfigEntry
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.catalyst._
import org.apache.spark.sql.catalyst.analysis.UnresolvedNamespace
import org.apache.spark.sql.catalyst.analysis.{CurrentNamespace, UnresolvedNamespace}
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.plans.logical.ShowTables
import org.apache.spark.sql.internal.{SessionState, SharedState, SQLConf}
Expand Down Expand Up @@ -669,7 +669,7 @@ class SQLContext private[sql](val sparkSession: SparkSession)
* @since 1.3.0
*/
def tables(): DataFrame = {
Dataset.ofRows(sparkSession, ShowTables(UnresolvedNamespace(Nil), None))
Dataset.ofRows(sparkSession, ShowTables(CurrentNamespace, None))
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -271,7 +271,7 @@ class CatalogImpl(sparkSession: SparkSession) extends Catalog {

// List built-in functions. We don't need to specify the namespace here as SHOW FUNCTIONS with
// only system scope does not need to know the catalog and namespace.
val plan0 = ShowFunctions(UnresolvedNamespace(Nil), false, true, pattern)
val plan0 = ShowFunctions(CurrentNamespace, false, true, pattern)
sparkSession.sessionState.executePlan(plan0).toRdd.collect().foreach { row =>
// Built-in functions do not belong to any catalog or namespace. We can only look it up with
// a single part name.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,7 @@ import scala.io.Source

import org.apache.spark.sql.{AnalysisException, Dataset, FastOperator}
import org.apache.spark.sql.catalyst.{QueryPlanningTracker, QueryPlanningTrackerCallback}
import org.apache.spark.sql.catalyst.analysis.UnresolvedNamespace
import org.apache.spark.sql.catalyst.analysis.CurrentNamespace
import org.apache.spark.sql.catalyst.expressions.UnsafeRow
import org.apache.spark.sql.catalyst.plans.QueryPlan
import org.apache.spark.sql.catalyst.plans.logical.{CommandResult, LogicalPlan, OneRowRelation, Project, ShowTables, SubqueryAlias}
Expand Down Expand Up @@ -250,7 +250,7 @@ class QueryExecutionSuite extends SharedSparkSession {
def qe(logicalPlan: LogicalPlan, callback: QueryPlanningTrackerCallback): QueryExecution =
new QueryExecution(spark, logicalPlan, new QueryPlanningTracker(Some(callback)))

val showTables = ShowTables(UnresolvedNamespace(Seq.empty[String]), None)
val showTables = ShowTables(CurrentNamespace, None)
val showTablesQe = qe(showTables, mockCallback1)
showTablesQe.assertAnalyzed()
mockCallback1.assertAnalyzed()
Expand Down Expand Up @@ -278,7 +278,7 @@ class QueryExecutionSuite extends SharedSparkSession {
test("SPARK-44145: non eagerly executed command setReadyForExecution") {
val mockCallback = MockCallback()

val showTables = ShowTables(UnresolvedNamespace(Seq.empty[String]), None)
val showTables = ShowTables(CurrentNamespace, None)
val showTablesQe = new QueryExecution(
spark,
showTables,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,12 +17,12 @@

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

import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, UnresolvedNamespace}
import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, CurrentNamespace, UnresolvedNamespace}
import org.apache.spark.sql.catalyst.parser.CatalystSqlParser.parsePlan
import org.apache.spark.sql.catalyst.plans.logical.ShowFunctions

class ShowFunctionsParserSuite extends AnalysisTest {
val nsPlan = UnresolvedNamespace(Nil)
val nsPlan = CurrentNamespace

test("show functions in the scope") {
comparePlans(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,7 @@

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

import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, UnresolvedNamespace, UnresolvedPartitionSpec, UnresolvedTable}
import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, CurrentNamespace, UnresolvedNamespace, UnresolvedPartitionSpec, UnresolvedTable}
import org.apache.spark.sql.catalyst.parser.CatalystSqlParser.parsePlan
import org.apache.spark.sql.catalyst.plans.logical.{ShowTablePartition, ShowTables, ShowTablesExtended}
import org.apache.spark.sql.test.SharedSparkSession
Expand All @@ -28,13 +28,13 @@ class ShowTablesParserSuite extends AnalysisTest with SharedSparkSession {
test("show tables") {
comparePlans(
parsePlan("SHOW TABLES"),
ShowTables(UnresolvedNamespace(Seq.empty[String]), None))
ShowTables(CurrentNamespace, None))
comparePlans(
parsePlan("SHOW TABLES '*test*'"),
ShowTables(UnresolvedNamespace(Seq.empty[String]), Some("*test*")))
ShowTables(CurrentNamespace, Some("*test*")))
comparePlans(
parsePlan("SHOW TABLES LIKE '*test*'"),
ShowTables(UnresolvedNamespace(Seq.empty[String]), Some("*test*")))
ShowTables(CurrentNamespace, Some("*test*")))
comparePlans(
parsePlan(s"SHOW TABLES FROM $catalog.ns1.ns2.tbl"),
ShowTables(UnresolvedNamespace(Seq(catalog, "ns1", "ns2", "tbl")), None))
Expand All @@ -52,7 +52,7 @@ class ShowTablesParserSuite extends AnalysisTest with SharedSparkSession {
test("show table extended") {
comparePlans(
parsePlan("SHOW TABLE EXTENDED LIKE '*test*'"),
ShowTablesExtended(UnresolvedNamespace(Seq.empty[String]), "*test*"))
ShowTablesExtended(CurrentNamespace, "*test*"))
comparePlans(
parsePlan(s"SHOW TABLE EXTENDED FROM $catalog.ns1.ns2 LIKE '*test*'"),
ShowTablesExtended(UnresolvedNamespace(Seq(catalog, "ns1", "ns2")), "*test*"))
Expand Down

0 comments on commit 1131a56

Please sign in to comment.