You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by we...@apache.org on 2020/04/29 07:08:39 UTC
[spark] branch branch-3.0 updated: [SPARK-30282][SQL][FOLLOWUP]
SHOW TBLPROPERTIES should support views
This is an automated email from the ASF dual-hosted git repository.
wenchen pushed a commit to branch branch-3.0
in repository https://gitbox.apache.org/repos/asf/spark.git
The following commit(s) were added to refs/heads/branch-3.0 by this push:
new dde2dd6 [SPARK-30282][SQL][FOLLOWUP] SHOW TBLPROPERTIES should support views
dde2dd6 is described below
commit dde2dd60414e1a1ccf0dd4fcd228fbed57b37b8f
Author: Terry Kim <yu...@gmail.com>
AuthorDate: Wed Apr 29 07:06:45 2020 +0000
[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 <yu...@gmail.com>
Signed-off-by: Wenchen Fan <we...@databricks.com>
(cherry picked from commit 36803031e850b08d689df90d15c75e1a1eeb28a8)
Signed-off-by: Wenchen Fan <we...@databricks.com>
---
docs/sql-migration-guide.md | 2 +-
.../spark/sql/catalyst/parser/AstBuilder.scala | 2 +-
.../spark/sql/catalyst/parser/DDLParserSuite.scala | 4 +-
.../catalyst/analysis/ResolveSessionCatalog.scala | 3 +
.../spark/sql/execution/command/tables.scala | 23 +++--
.../sql-tests/inputs/show-tblproperties.sql | 26 +++++
.../sql-tests/results/show-tblproperties.sql.out | 114 +++++++++++++++++++++
.../sql/hive/execution/HiveCommandSuite.scala | 17 +--
8 files changed, 162 insertions(+), 29 deletions(-)
diff --git a/docs/sql-migration-guide.md b/docs/sql-migration-guide.md
index 39619f6..e68198a 100644
--- a/docs/sql-migration-guide.md
+++ b/docs/sql-migration-guide.md
@@ -53,7 +53,7 @@ license: |
- 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`.
- - 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.
+ - 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`.
- 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.
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala
index 2a28e4b..024d7a4 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala
@@ -3537,7 +3537,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
override def visitShowTblProperties(
ctx: ShowTblPropertiesContext): LogicalPlan = withOrigin(ctx) {
ShowTableProperties(
- UnresolvedTable(visitMultipartIdentifier(ctx.table)),
+ UnresolvedTableOrView(visitMultipartIdentifier(ctx.table)),
Option(ctx.key).map(visitTablePropertyKey))
}
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala
index 1bef638..e05a51e 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala
@@ -1964,11 +1964,11 @@ class DDLParserSuite extends AnalysisTest {
test("SHOW TBLPROPERTIES table") {
comparePlans(
parsePlan("SHOW TBLPROPERTIES a.b.c"),
- ShowTableProperties(UnresolvedTable(Seq("a", "b", "c")), None))
+ ShowTableProperties(UnresolvedTableOrView(Seq("a", "b", "c")), None))
comparePlans(
parsePlan("SHOW TBLPROPERTIES a.b.c('propKey1')"),
- ShowTableProperties(UnresolvedTable(Seq("a", "b", "c")), Some("propKey1")))
+ ShowTableProperties(UnresolvedTableOrView(Seq("a", "b", "c")), Some("propKey1")))
}
test("DESCRIBE FUNCTION") {
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala
index 58d74e0..bb31c13 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala
@@ -538,6 +538,9 @@ class ResolveSessionCatalog(
case ShowTableProperties(r: ResolvedTable, propertyKey) if isSessionCatalog(r.catalog) =>
ShowTablePropertiesCommand(r.identifier.asTableIdentifier, propertyKey)
+ case ShowTableProperties(r: ResolvedView, propertyKey) =>
+ ShowTablePropertiesCommand(r.identifier.asTableIdentifier, propertyKey)
+
case DescribeFunctionStatement(nameParts, extended) =>
val functionIdent =
parseSessionCatalogFunctionIdentifier(nameParts, "DESCRIBE FUNCTION")
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala
index 2c9ca36..fc8cc11 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala
@@ -918,15 +918,20 @@ case class ShowTablePropertiesCommand(table: TableIdentifier, propertyKey: Optio
}
override def run(sparkSession: SparkSession): Seq[Row] = {
- val catalogTable = sparkSession.sessionState.catalog.getTableMetadata(table)
- propertyKey match {
- case Some(p) =>
- val propValue = catalogTable
- .properties
- .getOrElse(p, s"Table ${catalogTable.qualifiedName} does not have property: $p")
- Seq(Row(propValue))
- case None =>
- catalogTable.properties.map(p => Row(p._1, p._2)).toSeq
+ val catalog = sparkSession.sessionState.catalog
+ if (catalog.isTemporaryTable(table)) {
+ Seq.empty[Row]
+ } else {
+ val catalogTable = catalog.getTableMetadata(table)
+ propertyKey match {
+ case Some(p) =>
+ val propValue = catalogTable
+ .properties
+ .getOrElse(p, s"Table ${catalogTable.qualifiedName} does not have property: $p")
+ Seq(Row(propValue))
+ case None =>
+ catalogTable.properties.map(p => Row(p._1, p._2)).toSeq
+ }
}
}
}
diff --git a/sql/core/src/test/resources/sql-tests/inputs/show-tblproperties.sql b/sql/core/src/test/resources/sql-tests/inputs/show-tblproperties.sql
new file mode 100644
index 0000000..2861b2b
--- /dev/null
+++ b/sql/core/src/test/resources/sql-tests/inputs/show-tblproperties.sql
@@ -0,0 +1,26 @@
+-- create a table with properties
+CREATE TABLE tbl (a INT, b STRING, c INT) USING parquet
+TBLPROPERTIES('p1'='v1', 'p2'='v2');
+
+SHOW TBLPROPERTIES tbl;
+SHOW TBLPROPERTIES tbl("p1");
+SHOW TBLPROPERTIES tbl("p3");
+
+DROP TABLE tbl;
+
+-- create a view with properties
+CREATE VIEW view TBLPROPERTIES('p1'='v1', 'p2'='v2') AS SELECT 1 AS c1;
+
+SHOW TBLPROPERTIES view;
+SHOW TBLPROPERTIES view("p1");
+SHOW TBLPROPERTIES view("p3");
+
+DROP VIEW view;
+
+-- create a temporary view with properties
+CREATE TEMPORARY VIEW tv TBLPROPERTIES('p1'='v1') AS SELECT 1 AS c1;
+
+-- Properties for a temporary view should be empty
+SHOW TBLPROPERTIES tv;
+
+DROP VIEW tv;
diff --git a/sql/core/src/test/resources/sql-tests/results/show-tblproperties.sql.out b/sql/core/src/test/resources/sql-tests/results/show-tblproperties.sql.out
new file mode 100644
index 0000000..6984b34
--- /dev/null
+++ b/sql/core/src/test/resources/sql-tests/results/show-tblproperties.sql.out
@@ -0,0 +1,114 @@
+-- Automatically generated by SQLQueryTestSuite
+-- Number of queries: 13
+
+
+-- !query
+CREATE TABLE tbl (a INT, b STRING, c INT) USING parquet
+TBLPROPERTIES('p1'='v1', 'p2'='v2')
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SHOW TBLPROPERTIES tbl
+-- !query schema
+struct<key:string,value:string>
+-- !query output
+p1 v1
+p2 v2
+
+
+-- !query
+SHOW TBLPROPERTIES tbl("p1")
+-- !query schema
+struct<value:string>
+-- !query output
+v1
+
+
+-- !query
+SHOW TBLPROPERTIES tbl("p3")
+-- !query schema
+struct<value:string>
+-- !query output
+Table default.tbl does not have property: p3
+
+
+-- !query
+DROP TABLE tbl
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+CREATE VIEW view TBLPROPERTIES('p1'='v1', 'p2'='v2') AS SELECT 1 AS c1
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SHOW TBLPROPERTIES view
+-- !query schema
+struct<key:string,value:string>
+-- !query output
+p1 v1
+p2 v2
+view.catalogAndNamespace.numParts 2
+view.catalogAndNamespace.part.0 spark_catalog
+view.catalogAndNamespace.part.1 default
+view.query.out.col.0 c1
+view.query.out.numCols 1
+
+
+-- !query
+SHOW TBLPROPERTIES view("p1")
+-- !query schema
+struct<value:string>
+-- !query output
+v1
+
+
+-- !query
+SHOW TBLPROPERTIES view("p3")
+-- !query schema
+struct<value:string>
+-- !query output
+Table default.view does not have property: p3
+
+
+-- !query
+DROP VIEW view
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+CREATE TEMPORARY VIEW tv TBLPROPERTIES('p1'='v1') AS SELECT 1 AS c1
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SHOW TBLPROPERTIES tv
+-- !query schema
+struct<key:string,value:string>
+-- !query output
+
+
+
+-- !query
+DROP VIEW tv
+-- !query schema
+struct<>
+-- !query output
+
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCommandSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCommandSuite.scala
index bd85ad4..dcec8bf 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCommandSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCommandSuite.scala
@@ -179,7 +179,7 @@ class HiveCommandSuite extends QueryTest with SQLTestUtils with TestHiveSingleto
val message = intercept[AnalysisException] {
sql("SHOW TBLPROPERTIES badtable")
}.getMessage
- assert(message.contains("Table not found: badtable"))
+ assert(message.contains("Table or view not found: badtable"))
// When key is not found, a row containing the error is returned.
checkAnswer(
@@ -193,21 +193,6 @@ class HiveCommandSuite extends QueryTest with SQLTestUtils with TestHiveSingleto
checkAnswer(sql("SHOW TBLPROPERTIES parquet_tab2('`prop2Key`')"), Row("prop2Val"))
}
- test("show tblproperties for spark temporary table - AnalysisException is thrown") {
- withTempView("parquet_temp") {
- sql(
- """
- |CREATE TEMPORARY VIEW parquet_temp (c1 INT, c2 STRING)
- |USING org.apache.spark.sql.parquet.DefaultSource
- """.stripMargin)
-
- val message = intercept[AnalysisException] {
- sql("SHOW TBLPROPERTIES parquet_temp")
- }.getMessage
- assert(message.contains("parquet_temp is a temp view not table"))
- }
- }
-
Seq(true, false).foreach { local =>
val loadQuery = if (local) "LOAD DATA LOCAL" else "LOAD DATA"
test(loadQuery) {
---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@spark.apache.org
For additional commands, e-mail: commits-help@spark.apache.org