Skip to content

Commit dde2dd6

Browse files
imback82cloud-fan
authored andcommitted
[SPARK-30282][SQL][FOLLOWUP] SHOW TBLPROPERTIES should support views
### What changes were proposed in this pull request? This PR addresses two things: - `SHOW TBLPROPERTIES` should supports view (a regression introduced by #26921) - `SHOW TBLPROPERTIES` on a temporary view should return empty result (2.4 behavior instead of throwing `AnalysisException`. ### Why are the changes needed? It's a bug. ### Does this PR introduce any user-facing change? Yes, now `SHOW TBLPROPERTIES` works on views: ``` scala> sql("CREATE VIEW view TBLPROPERTIES('p1'='v1', 'p2'='v2') AS SELECT 1 AS c1") scala> sql("SHOW TBLPROPERTIES view").show(truncate=false) +---------------------------------+-------------+ |key |value | +---------------------------------+-------------+ |view.catalogAndNamespace.numParts|2 | |view.query.out.col.0 |c1 | |view.query.out.numCols |1 | |p2 |v2 | |view.catalogAndNamespace.part.0 |spark_catalog| |p1 |v1 | |view.catalogAndNamespace.part.1 |default | +---------------------------------+-------------+ ``` And for a temporary view: ``` scala> sql("CREATE TEMPORARY VIEW tview TBLPROPERTIES('p1'='v1', 'p2'='v2') AS SELECT 1 AS c1") scala> sql("SHOW TBLPROPERTIES tview").show(truncate=false) +---+-----+ |key|value| +---+-----+ +---+-----+ ``` ### How was this patch tested? Added tests. Closes #28375 from imback82/show_tblproperties_followup. Authored-by: Terry Kim <yuminkim@gmail.com> Signed-off-by: Wenchen Fan <wenchen@databricks.com> (cherry picked from commit 3680303) Signed-off-by: Wenchen Fan <wenchen@databricks.com>
1 parent d771d79 commit dde2dd6

File tree

8 files changed

+162
-29
lines changed

8 files changed

+162
-29
lines changed

docs/sql-migration-guide.md

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -53,7 +53,7 @@ license: |
5353

5454
- In Spark 3.0, you can use `ADD FILE` to add file directories as well. Earlier you could add only single files using this command. To restore the behavior of earlier versions, set `spark.sql.legacy.addSingleFileInAddFile` to `true`.
5555

56-
- In Spark 3.0, `SHOW TBLPROPERTIES` throws `AnalysisException` if the table does not exist. In Spark version 2.4 and below, this scenario caused `NoSuchTableException`. Also, `SHOW TBLPROPERTIES` on a temporary view causes `AnalysisException`. In Spark version 2.4 and below, it returned an empty result.
56+
- In Spark 3.0, `SHOW TBLPROPERTIES` throws `AnalysisException` if the table does not exist. In Spark version 2.4 and below, this scenario caused `NoSuchTableException`.
5757

5858
- In Spark 3.0, `SHOW CREATE TABLE` always returns Spark DDL, even when the given table is a Hive SerDe table. For generating Hive DDL, use `SHOW CREATE TABLE AS SERDE` command instead.
5959

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

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -3537,7 +3537,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
35373537
override def visitShowTblProperties(
35383538
ctx: ShowTblPropertiesContext): LogicalPlan = withOrigin(ctx) {
35393539
ShowTableProperties(
3540-
UnresolvedTable(visitMultipartIdentifier(ctx.table)),
3540+
UnresolvedTableOrView(visitMultipartIdentifier(ctx.table)),
35413541
Option(ctx.key).map(visitTablePropertyKey))
35423542
}
35433543

sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -1964,11 +1964,11 @@ class DDLParserSuite extends AnalysisTest {
19641964
test("SHOW TBLPROPERTIES table") {
19651965
comparePlans(
19661966
parsePlan("SHOW TBLPROPERTIES a.b.c"),
1967-
ShowTableProperties(UnresolvedTable(Seq("a", "b", "c")), None))
1967+
ShowTableProperties(UnresolvedTableOrView(Seq("a", "b", "c")), None))
19681968

19691969
comparePlans(
19701970
parsePlan("SHOW TBLPROPERTIES a.b.c('propKey1')"),
1971-
ShowTableProperties(UnresolvedTable(Seq("a", "b", "c")), Some("propKey1")))
1971+
ShowTableProperties(UnresolvedTableOrView(Seq("a", "b", "c")), Some("propKey1")))
19721972
}
19731973

19741974
test("DESCRIBE FUNCTION") {

sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala

Lines changed: 3 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -538,6 +538,9 @@ class ResolveSessionCatalog(
538538
case ShowTableProperties(r: ResolvedTable, propertyKey) if isSessionCatalog(r.catalog) =>
539539
ShowTablePropertiesCommand(r.identifier.asTableIdentifier, propertyKey)
540540

541+
case ShowTableProperties(r: ResolvedView, propertyKey) =>
542+
ShowTablePropertiesCommand(r.identifier.asTableIdentifier, propertyKey)
543+
541544
case DescribeFunctionStatement(nameParts, extended) =>
542545
val functionIdent =
543546
parseSessionCatalogFunctionIdentifier(nameParts, "DESCRIBE FUNCTION")

sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala

Lines changed: 14 additions & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -918,15 +918,20 @@ case class ShowTablePropertiesCommand(table: TableIdentifier, propertyKey: Optio
918918
}
919919

920920
override def run(sparkSession: SparkSession): Seq[Row] = {
921-
val catalogTable = sparkSession.sessionState.catalog.getTableMetadata(table)
922-
propertyKey match {
923-
case Some(p) =>
924-
val propValue = catalogTable
925-
.properties
926-
.getOrElse(p, s"Table ${catalogTable.qualifiedName} does not have property: $p")
927-
Seq(Row(propValue))
928-
case None =>
929-
catalogTable.properties.map(p => Row(p._1, p._2)).toSeq
921+
val catalog = sparkSession.sessionState.catalog
922+
if (catalog.isTemporaryTable(table)) {
923+
Seq.empty[Row]
924+
} else {
925+
val catalogTable = catalog.getTableMetadata(table)
926+
propertyKey match {
927+
case Some(p) =>
928+
val propValue = catalogTable
929+
.properties
930+
.getOrElse(p, s"Table ${catalogTable.qualifiedName} does not have property: $p")
931+
Seq(Row(propValue))
932+
case None =>
933+
catalogTable.properties.map(p => Row(p._1, p._2)).toSeq
934+
}
930935
}
931936
}
932937
}
Lines changed: 26 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,26 @@
1+
-- create a table with properties
2+
CREATE TABLE tbl (a INT, b STRING, c INT) USING parquet
3+
TBLPROPERTIES('p1'='v1', 'p2'='v2');
4+
5+
SHOW TBLPROPERTIES tbl;
6+
SHOW TBLPROPERTIES tbl("p1");
7+
SHOW TBLPROPERTIES tbl("p3");
8+
9+
DROP TABLE tbl;
10+
11+
-- create a view with properties
12+
CREATE VIEW view TBLPROPERTIES('p1'='v1', 'p2'='v2') AS SELECT 1 AS c1;
13+
14+
SHOW TBLPROPERTIES view;
15+
SHOW TBLPROPERTIES view("p1");
16+
SHOW TBLPROPERTIES view("p3");
17+
18+
DROP VIEW view;
19+
20+
-- create a temporary view with properties
21+
CREATE TEMPORARY VIEW tv TBLPROPERTIES('p1'='v1') AS SELECT 1 AS c1;
22+
23+
-- Properties for a temporary view should be empty
24+
SHOW TBLPROPERTIES tv;
25+
26+
DROP VIEW tv;
Lines changed: 114 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,114 @@
1+
-- Automatically generated by SQLQueryTestSuite
2+
-- Number of queries: 13
3+
4+
5+
-- !query
6+
CREATE TABLE tbl (a INT, b STRING, c INT) USING parquet
7+
TBLPROPERTIES('p1'='v1', 'p2'='v2')
8+
-- !query schema
9+
struct<>
10+
-- !query output
11+
12+
13+
14+
-- !query
15+
SHOW TBLPROPERTIES tbl
16+
-- !query schema
17+
struct<key:string,value:string>
18+
-- !query output
19+
p1 v1
20+
p2 v2
21+
22+
23+
-- !query
24+
SHOW TBLPROPERTIES tbl("p1")
25+
-- !query schema
26+
struct<value:string>
27+
-- !query output
28+
v1
29+
30+
31+
-- !query
32+
SHOW TBLPROPERTIES tbl("p3")
33+
-- !query schema
34+
struct<value:string>
35+
-- !query output
36+
Table default.tbl does not have property: p3
37+
38+
39+
-- !query
40+
DROP TABLE tbl
41+
-- !query schema
42+
struct<>
43+
-- !query output
44+
45+
46+
47+
-- !query
48+
CREATE VIEW view TBLPROPERTIES('p1'='v1', 'p2'='v2') AS SELECT 1 AS c1
49+
-- !query schema
50+
struct<>
51+
-- !query output
52+
53+
54+
55+
-- !query
56+
SHOW TBLPROPERTIES view
57+
-- !query schema
58+
struct<key:string,value:string>
59+
-- !query output
60+
p1 v1
61+
p2 v2
62+
view.catalogAndNamespace.numParts 2
63+
view.catalogAndNamespace.part.0 spark_catalog
64+
view.catalogAndNamespace.part.1 default
65+
view.query.out.col.0 c1
66+
view.query.out.numCols 1
67+
68+
69+
-- !query
70+
SHOW TBLPROPERTIES view("p1")
71+
-- !query schema
72+
struct<value:string>
73+
-- !query output
74+
v1
75+
76+
77+
-- !query
78+
SHOW TBLPROPERTIES view("p3")
79+
-- !query schema
80+
struct<value:string>
81+
-- !query output
82+
Table default.view does not have property: p3
83+
84+
85+
-- !query
86+
DROP VIEW view
87+
-- !query schema
88+
struct<>
89+
-- !query output
90+
91+
92+
93+
-- !query
94+
CREATE TEMPORARY VIEW tv TBLPROPERTIES('p1'='v1') AS SELECT 1 AS c1
95+
-- !query schema
96+
struct<>
97+
-- !query output
98+
99+
100+
101+
-- !query
102+
SHOW TBLPROPERTIES tv
103+
-- !query schema
104+
struct<key:string,value:string>
105+
-- !query output
106+
107+
108+
109+
-- !query
110+
DROP VIEW tv
111+
-- !query schema
112+
struct<>
113+
-- !query output
114+

sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCommandSuite.scala

Lines changed: 1 addition & 16 deletions
Original file line numberDiff line numberDiff line change
@@ -179,7 +179,7 @@ class HiveCommandSuite extends QueryTest with SQLTestUtils with TestHiveSingleto
179179
val message = intercept[AnalysisException] {
180180
sql("SHOW TBLPROPERTIES badtable")
181181
}.getMessage
182-
assert(message.contains("Table not found: badtable"))
182+
assert(message.contains("Table or view not found: badtable"))
183183

184184
// When key is not found, a row containing the error is returned.
185185
checkAnswer(
@@ -193,21 +193,6 @@ class HiveCommandSuite extends QueryTest with SQLTestUtils with TestHiveSingleto
193193
checkAnswer(sql("SHOW TBLPROPERTIES parquet_tab2('`prop2Key`')"), Row("prop2Val"))
194194
}
195195

196-
test("show tblproperties for spark temporary table - AnalysisException is thrown") {
197-
withTempView("parquet_temp") {
198-
sql(
199-
"""
200-
|CREATE TEMPORARY VIEW parquet_temp (c1 INT, c2 STRING)
201-
|USING org.apache.spark.sql.parquet.DefaultSource
202-
""".stripMargin)
203-
204-
val message = intercept[AnalysisException] {
205-
sql("SHOW TBLPROPERTIES parquet_temp")
206-
}.getMessage
207-
assert(message.contains("parquet_temp is a temp view not table"))
208-
}
209-
}
210-
211196
Seq(true, false).foreach { local =>
212197
val loadQuery = if (local) "LOAD DATA LOCAL" else "LOAD DATA"
213198
test(loadQuery) {

0 commit comments

Comments
 (0)