Skip to content

Commit 5d8422d

Browse files
committed
initial checkin
1 parent e75d9af commit 5d8422d

File tree

2 files changed

+79
-10
lines changed

2 files changed

+79
-10
lines changed

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

Lines changed: 19 additions & 10 deletions
Original file line numberDiff line numberDiff line change
@@ -748,6 +748,8 @@ class Analyzer(
748748
* [[ResolveRelations]] still resolves v1 tables.
749749
*/
750750
object ResolveTables extends Rule[LogicalPlan] {
751+
import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._
752+
751753
def apply(plan: LogicalPlan): LogicalPlan = ResolveTempViews(plan).resolveOperatorsUp {
752754
case u: UnresolvedRelation =>
753755
lookupV2Relation(u.multipartIdentifier)
@@ -759,22 +761,29 @@ class Analyzer(
759761
.getOrElse(i)
760762

761763
case desc @ DescribeTable(u: UnresolvedV2Relation, _) =>
762-
CatalogV2Util.loadRelation(u.catalog, u.tableName)
763-
.map(rel => desc.copy(table = rel))
764-
.getOrElse(desc)
764+
resolveV2Relation(u).map(rel => desc.copy(table = rel)).getOrElse(desc)
765765

766766
case alter @ AlterTable(_, _, u: UnresolvedV2Relation, _) =>
767-
CatalogV2Util.loadRelation(u.catalog, u.tableName)
768-
.map(rel => alter.copy(table = rel))
769-
.getOrElse(alter)
767+
resolveV2Relation(u).map(rel => alter.copy(table = rel)).getOrElse(alter)
770768

771769
case show @ ShowTableProperties(u: UnresolvedV2Relation, _) =>
772-
CatalogV2Util.loadRelation(u.catalog, u.tableName)
773-
.map(rel => show.copy(table = rel))
774-
.getOrElse(show)
770+
resolveV2Relation(u).map(rel => show.copy(table = rel)).getOrElse(show)
775771

776772
case u: UnresolvedV2Relation =>
777-
CatalogV2Util.loadRelation(u.catalog, u.tableName).getOrElse(u)
773+
resolveV2Relation(u).getOrElse(u)
774+
}
775+
776+
private def resolveV2Relation(unresolved: UnresolvedV2Relation) : Option[NamedRelation] = {
777+
val maybeTempView = unresolved.originalNameParts match {
778+
case Seq(part) => v1SessionCatalog.lookupTempView(part)
779+
case _ => None
780+
}
781+
if (maybeTempView.isDefined) {
782+
throw new AnalysisException(
783+
s"A temp view '${unresolved.originalNameParts.quoted}' cannot be handled by V2 commands.")
784+
}
785+
786+
CatalogV2Util.loadRelation(unresolved.catalog, unresolved.tableName)
778787
}
779788

780789
/**
Lines changed: 60 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,60 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one or more
3+
* contributor license agreements. See the NOTICE file distributed with
4+
* this work for additional information regarding copyright ownership.
5+
* The ASF licenses this file to You under the Apache License, Version 2.0
6+
* (the "License"); you may not use this file except in compliance with
7+
* the License. You may obtain a copy of the License at
8+
*
9+
* http://www.apache.org/licenses/LICENSE-2.0
10+
*
11+
* Unless required by applicable law or agreed to in writing, software
12+
* distributed under the License is distributed on an "AS IS" BASIS,
13+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14+
* See the License for the specific language governing permissions and
15+
* limitations under the License.
16+
*/
17+
18+
package org.apache.spark.sql.connector
19+
20+
import org.scalatest.BeforeAndAfter
21+
22+
import org.apache.spark.sql.{AnalysisException, QueryTest}
23+
import org.apache.spark.sql.test.SharedSparkSession
24+
25+
class TableResolutionSuite extends QueryTest with SharedSparkSession with BeforeAndAfter{
26+
27+
before {
28+
spark.conf.set("spark.sql.catalog.testcat", classOf[InMemoryTableCatalog].getName)
29+
}
30+
31+
after {
32+
spark.sessionState.catalog.reset()
33+
spark.sessionState.catalogManager.reset()
34+
spark.sessionState.conf.clear()
35+
}
36+
37+
test("V2 commands should look up temp view first") {
38+
val tbl = "t"
39+
val commands = Seq(
40+
s"DESCRIBE $tbl",
41+
s"SHOW TBLPROPERTIES $tbl",
42+
s"ALTER TABLE $tbl ADD COLUMN data string"
43+
)
44+
45+
withTempView(s"$tbl") {
46+
withTable(s"testcat.ns.$tbl") {
47+
sql(s"CREATE TEMPORARY VIEW $tbl AS SELECT 1 AS i")
48+
sql(s"CREATE TABLE testcat.ns.$tbl USING csv AS SELECT 2 AS i")
49+
sql("USE testcat.ns")
50+
51+
commands.foreach { command =>
52+
val ex = intercept[AnalysisException] {
53+
sql(command)
54+
}
55+
assert(ex.getMessage.contains("A temp view 't' cannot be handled"))
56+
}
57+
}
58+
}
59+
}
60+
}

0 commit comments

Comments
 (0)