Skip to content

Commit abe370f

Browse files
jzhugecloud-fan
authored andcommitted
[SPARK-27322][SQL] DataSourceV2 table relation
## What changes were proposed in this pull request? Support multi-catalog in the following SELECT code paths: - SELECT * FROM catalog.db.tbl - TABLE catalog.db.tbl - JOIN or UNION tables from different catalogs - SparkSession.table("catalog.db.tbl") - CTE relation - View text ## How was this patch tested? New unit tests. All existing unit tests in catalyst and sql core. Closes #24741 from jzhuge/SPARK-27322-pr. Authored-by: John Zhuge <jzhuge@apache.org> Signed-off-by: Wenchen Fan <wenchen@databricks.com>
1 parent ddf4a50 commit abe370f

File tree

24 files changed

+213
-55
lines changed

24 files changed

+213
-55
lines changed

sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -407,7 +407,7 @@ queryTerm
407407
queryPrimary
408408
: querySpecification #queryPrimaryDefault
409409
| fromStatement #fromStmt
410-
| TABLE tableIdentifier #table
410+
| TABLE multipartIdentifier #table
411411
| inlineTable #inlineTableDefault1
412412
| '(' queryNoWith ')' #subquery
413413
;
@@ -579,7 +579,7 @@ identifierComment
579579
;
580580

581581
relationPrimary
582-
: tableIdentifier sample? tableAlias #tableName
582+
: multipartIdentifier sample? tableAlias #tableName
583583
| '(' queryNoWith ')' sample? tableAlias #aliasedQuery
584584
| '(' relation ')' sample? tableAlias #aliasedRelation
585585
| inlineTable #inlineTableDefault2

sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/utils/CatalogV2Util.scala

Lines changed: 12 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -22,11 +22,15 @@ import java.util.Collections
2222

2323
import scala.collection.JavaConverters._
2424

25-
import org.apache.spark.sql.catalog.v2.TableChange
25+
import org.apache.spark.sql.catalog.v2.{CatalogPlugin, Identifier, TableChange}
2626
import org.apache.spark.sql.catalog.v2.TableChange.{AddColumn, DeleteColumn, RemoveProperty, RenameColumn, SetProperty, UpdateColumnComment, UpdateColumnType}
27+
import org.apache.spark.sql.catalyst.analysis.NoSuchTableException
28+
import org.apache.spark.sql.sources.v2.Table
2729
import org.apache.spark.sql.types.{StructField, StructType}
2830

2931
object CatalogV2Util {
32+
import org.apache.spark.sql.catalog.v2.CatalogV2Implicits._
33+
3034
/**
3135
* Apply properties changes to a map and return the result.
3236
*/
@@ -149,4 +153,11 @@ object CatalogV2Util {
149153

150154
new StructType(newFields)
151155
}
156+
157+
def loadTable(catalog: CatalogPlugin, ident: Identifier): Option[Table] =
158+
try {
159+
Option(catalog.asTableCatalog.loadTable(ident))
160+
} catch {
161+
case _: NoSuchTableException => None
162+
}
152163
}

sql/catalyst/src/main/java/org/apache/spark/sql/util/CaseInsensitiveStringMap.java

Lines changed: 18 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -26,6 +26,7 @@
2626
import java.util.HashMap;
2727
import java.util.Locale;
2828
import java.util.Map;
29+
import java.util.Objects;
2930
import java.util.Set;
3031

3132
/**
@@ -178,4 +179,21 @@ public double getDouble(String key, double defaultValue) {
178179
public Map<String, String> asCaseSensitiveMap() {
179180
return Collections.unmodifiableMap(original);
180181
}
182+
183+
@Override
184+
public boolean equals(Object o) {
185+
if (this == o) {
186+
return true;
187+
}
188+
if (o == null || getClass() != o.getClass()) {
189+
return false;
190+
}
191+
CaseInsensitiveStringMap that = (CaseInsensitiveStringMap) o;
192+
return delegate.equals(that.delegate);
193+
}
194+
195+
@Override
196+
public int hashCode() {
197+
return Objects.hash(delegate);
198+
}
181199
}

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala

Lines changed: 33 additions & 15 deletions
Original file line numberDiff line numberDiff line change
@@ -37,6 +37,7 @@ import org.apache.spark.sql.catalyst.plans.logical._
3737
import org.apache.spark.sql.catalyst.rules._
3838
import org.apache.spark.sql.catalyst.trees.TreeNodeRef
3939
import org.apache.spark.sql.catalyst.util.toPrettySQL
40+
import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
4041
import org.apache.spark.sql.internal.SQLConf
4142
import org.apache.spark.sql.types._
4243

@@ -162,6 +163,7 @@ class Analyzer(
162163
new SubstituteUnresolvedOrdinals(conf)),
163164
Batch("Resolution", fixedPoint,
164165
ResolveTableValuedFunctions ::
166+
ResolveTables ::
165167
ResolveRelations ::
166168
ResolveReferences ::
167169
ResolveCreateNamedStruct ::
@@ -226,7 +228,7 @@ class Analyzer(
226228

227229
def substituteCTE(plan: LogicalPlan, cteName: String, ctePlan: LogicalPlan): LogicalPlan = {
228230
plan resolveOperatorsUp {
229-
case UnresolvedRelation(TableIdentifier(table, None)) if resolver(cteName, table) =>
231+
case UnresolvedRelation(Seq(table)) if resolver(cteName, table) =>
230232
ctePlan
231233
case u: UnresolvedRelation =>
232234
u
@@ -657,6 +659,20 @@ class Analyzer(
657659
}
658660
}
659661

662+
/**
663+
* Resolve table relations with concrete relations from v2 catalog.
664+
*
665+
* [[ResolveRelations]] still resolves v1 tables.
666+
*/
667+
object ResolveTables extends Rule[LogicalPlan] {
668+
import org.apache.spark.sql.catalog.v2.utils.CatalogV2Util._
669+
670+
def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsUp {
671+
case u @ UnresolvedRelation(CatalogObjectIdentifier(Some(catalogPlugin), ident)) =>
672+
loadTable(catalogPlugin, ident).map(DataSourceV2Relation.create).getOrElse(u)
673+
}
674+
}
675+
660676
/**
661677
* Replaces [[UnresolvedRelation]]s with concrete relations from the catalog.
662678
*/
@@ -689,10 +705,15 @@ class Analyzer(
689705
// Note this is compatible with the views defined by older versions of Spark(before 2.2), which
690706
// have empty defaultDatabase and all the relations in viewText have database part defined.
691707
def resolveRelation(plan: LogicalPlan): LogicalPlan = plan match {
692-
case u: UnresolvedRelation if !isRunningDirectlyOnFiles(u.tableIdentifier) =>
708+
case u @ UnresolvedRelation(AsTableIdentifier(ident)) if !isRunningDirectlyOnFiles(ident) =>
693709
val defaultDatabase = AnalysisContext.get.defaultDatabase
694-
val foundRelation = lookupTableFromCatalog(u, defaultDatabase)
695-
resolveRelation(foundRelation)
710+
val foundRelation = lookupTableFromCatalog(ident, u, defaultDatabase)
711+
if (foundRelation != u) {
712+
resolveRelation(foundRelation)
713+
} else {
714+
u
715+
}
716+
696717
// The view's child should be a logical plan parsed from the `desc.viewText`, the variable
697718
// `viewText` should be defined, or else we throw an error on the generation of the View
698719
// operator.
@@ -715,8 +736,9 @@ class Analyzer(
715736
}
716737

717738
def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsUp {
718-
case i @ InsertIntoTable(u: UnresolvedRelation, parts, child, _, _) if child.resolved =>
719-
EliminateSubqueryAliases(lookupTableFromCatalog(u)) match {
739+
case i @ InsertIntoTable(u @ UnresolvedRelation(AsTableIdentifier(ident)), _, child, _, _)
740+
if child.resolved =>
741+
EliminateSubqueryAliases(lookupTableFromCatalog(ident, u)) match {
720742
case v: View =>
721743
u.failAnalysis(s"Inserting into a view is not allowed. View: ${v.desc.identifier}.")
722744
case other => i.copy(table = other)
@@ -731,20 +753,16 @@ class Analyzer(
731753
// and the default database is only used to look up a view);
732754
// 3. Use the currentDb of the SessionCatalog.
733755
private def lookupTableFromCatalog(
756+
tableIdentifier: TableIdentifier,
734757
u: UnresolvedRelation,
735758
defaultDatabase: Option[String] = None): LogicalPlan = {
736-
val tableIdentWithDb = u.tableIdentifier.copy(
737-
database = u.tableIdentifier.database.orElse(defaultDatabase))
759+
val tableIdentWithDb = tableIdentifier.copy(
760+
database = tableIdentifier.database.orElse(defaultDatabase))
738761
try {
739762
catalog.lookupRelation(tableIdentWithDb)
740763
} catch {
741-
case e: NoSuchTableException =>
742-
u.failAnalysis(s"Table or view not found: ${tableIdentWithDb.unquotedString}", e)
743-
// If the database is defined and that database is not found, throw an AnalysisException.
744-
// Note that if the database is not defined, it is possible we are looking up a temp view.
745-
case e: NoSuchDatabaseException =>
746-
u.failAnalysis(s"Table or view not found: ${tableIdentWithDb.unquotedString}, the " +
747-
s"database ${e.db} doesn't exist.", e)
764+
case _: NoSuchTableException | _: NoSuchDatabaseException =>
765+
u
748766
}
749767
}
750768

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -90,7 +90,7 @@ trait CheckAnalysis extends PredicateHelper {
9090
case p if p.analyzed => // Skip already analyzed sub-plans
9191

9292
case u: UnresolvedRelation =>
93-
u.failAnalysis(s"Table or view not found: ${u.tableIdentifier}")
93+
u.failAnalysis(s"Table or view not found: ${u.multipartIdentifier.quoted}")
9494

9595
case operator: LogicalPlan =>
9696
// Check argument data types of higher-order functions downwards first.

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveHints.scala

Lines changed: 7 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -71,18 +71,20 @@ object ResolveHints {
7171

7272
val newNode = CurrentOrigin.withOrigin(plan.origin) {
7373
plan match {
74-
case ResolvedHint(u: UnresolvedRelation, hint)
75-
if relations.exists(resolver(_, u.tableIdentifier.table)) =>
76-
relations.remove(u.tableIdentifier.table)
74+
case ResolvedHint(u @ UnresolvedRelation(ident), hint)
75+
if relations.exists(resolver(_, ident.last)) =>
76+
relations.remove(ident.last)
7777
ResolvedHint(u, createHintInfo(hintName).merge(hint, handleOverriddenHintInfo))
78+
7879
case ResolvedHint(r: SubqueryAlias, hint)
7980
if relations.exists(resolver(_, r.alias)) =>
8081
relations.remove(r.alias)
8182
ResolvedHint(r, createHintInfo(hintName).merge(hint, handleOverriddenHintInfo))
8283

83-
case u: UnresolvedRelation if relations.exists(resolver(_, u.tableIdentifier.table)) =>
84-
relations.remove(u.tableIdentifier.table)
84+
case u @ UnresolvedRelation(ident) if relations.exists(resolver(_, ident.last)) =>
85+
relations.remove(ident.last)
8586
ResolvedHint(plan, createHintInfo(hintName))
87+
8688
case r: SubqueryAlias if relations.exists(resolver(_, r.alias)) =>
8789
relations.remove(r.alias)
8890
ResolvedHint(plan, createHintInfo(hintName))

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala

Lines changed: 9 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -38,19 +38,24 @@ class UnresolvedException[TreeType <: TreeNode[_]](tree: TreeType, function: Str
3838
/**
3939
* Holds the name of a relation that has yet to be looked up in a catalog.
4040
*
41-
* @param tableIdentifier table name
41+
* @param multipartIdentifier table name
4242
*/
43-
case class UnresolvedRelation(tableIdentifier: TableIdentifier)
44-
extends LeafNode {
43+
case class UnresolvedRelation(multipartIdentifier: Seq[String]) extends LeafNode {
44+
import org.apache.spark.sql.catalog.v2.CatalogV2Implicits._
4545

4646
/** Returns a `.` separated name for this relation. */
47-
def tableName: String = tableIdentifier.unquotedString
47+
def tableName: String = multipartIdentifier.quoted
4848

4949
override def output: Seq[Attribute] = Nil
5050

5151
override lazy val resolved = false
5252
}
5353

54+
object UnresolvedRelation {
55+
def apply(tableIdentifier: TableIdentifier): UnresolvedRelation =
56+
UnresolvedRelation(tableIdentifier.database.toSeq :+ tableIdentifier.table)
57+
}
58+
5459
/**
5560
* An inline table that has not been resolved yet. Once resolved, it is turned by the analyzer into
5661
* a [[org.apache.spark.sql.catalyst.plans.logical.LocalRelation]].

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala

Lines changed: 1 addition & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -295,10 +295,7 @@ package object dsl {
295295
object expressions extends ExpressionConversions // scalastyle:ignore
296296

297297
object plans { // scalastyle:ignore
298-
def table(ref: String): LogicalPlan = UnresolvedRelation(TableIdentifier(ref))
299-
300-
def table(db: String, ref: String): LogicalPlan =
301-
UnresolvedRelation(TableIdentifier(ref, Option(db)))
298+
def table(parts: String*): LogicalPlan = UnresolvedRelation(parts)
302299

303300
implicit class DslLogicalPlan(val logicalPlan: LogicalPlan) {
304301
def select(exprs: Expression*): LogicalPlan = {

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -898,14 +898,14 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
898898
* }}}
899899
*/
900900
override def visitTable(ctx: TableContext): LogicalPlan = withOrigin(ctx) {
901-
UnresolvedRelation(visitTableIdentifier(ctx.tableIdentifier))
901+
UnresolvedRelation(visitMultipartIdentifier(ctx.multipartIdentifier))
902902
}
903903

904904
/**
905905
* Create an aliased table reference. This is typically used in FROM clauses.
906906
*/
907907
override def visitTableName(ctx: TableNameContext): LogicalPlan = withOrigin(ctx) {
908-
val tableId = visitTableIdentifier(ctx.tableIdentifier)
908+
val tableId = visitMultipartIdentifier(ctx.multipartIdentifier)
909909
val table = mayApplyAliasPlan(ctx.tableAlias, UnresolvedRelation(tableId))
910910
table.optionalMap(ctx.sample)(withSample)
911911
}

0 commit comments

Comments
 (0)