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/05/04 12:24:10 UTC

[spark] branch branch-3.0 updated: [SPARK-31624] Fix SHOW TBLPROPERTIES for V2 tables that leverage the session catalog

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 357fdb2  [SPARK-31624] Fix SHOW TBLPROPERTIES for V2 tables that leverage the session catalog
357fdb2 is described below

commit 357fdb265e8f1e84115b8fbf6c8b08ca4f67ea53
Author: Burak Yavuz <br...@gmail.com>
AuthorDate: Mon May 4 12:22:29 2020 +0000

    [SPARK-31624] Fix SHOW TBLPROPERTIES for V2 tables that leverage the session catalog
    
    ## What changes were proposed in this pull request?
    
    SHOW TBLPROPERTIES does not get the correct table properties for tables using the Session Catalog. This PR fixes that, by explicitly falling back to the V1 implementation if the table is in fact a V1 table. We also hide the reserved table properties for V2 tables, as users do not have control over setting these table properties. Henceforth, if they cannot be set or controlled by the user, then they shouldn't be displayed as such.
    
    ### Why are the changes needed?
    
    Shows the incorrect table properties, i.e. only what exists in the Hive MetaStore for V2 tables that may have table properties outside of the MetaStore.
    
    ### Does this PR introduce _any_ user-facing change?
    
    Fixes a bug
    
    ### How was this patch tested?
    
    Regression test
    
    Closes #28434 from brkyvz/ddlCommands.
    
    Authored-by: Burak Yavuz <br...@gmail.com>
    Signed-off-by: Wenchen Fan <we...@databricks.com>
    (cherry picked from commit 02a319d7e157c195d0a2b8c2bb992d980dde7d5c)
    Signed-off-by: Wenchen Fan <we...@databricks.com>
---
 .../sql/catalyst/analysis/ResolveSessionCatalog.scala  |  3 ++-
 .../datasources/v2/ShowTablePropertiesExec.scala       |  8 ++++++--
 .../connector/DataSourceV2SQLSessionCatalogSuite.scala | 18 +++++++++++++++++-
 .../spark/sql/connector/DataSourceV2SQLSuite.scala     |  2 --
 4 files changed, 25 insertions(+), 6 deletions(-)

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 bb31c13..9a8d2f0 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
@@ -535,7 +535,8 @@ class ResolveSessionCatalog(
             "SHOW VIEWS, only SessionCatalog supports this command.")
       }
 
-    case ShowTableProperties(r: ResolvedTable, propertyKey) if isSessionCatalog(r.catalog) =>
+    case ShowTableProperties(
+        r @ ResolvedTable(_, _, _: V1Table), propertyKey) if isSessionCatalog(r.catalog) =>
       ShowTablePropertiesCommand(r.identifier.asTableIdentifier, propertyKey)
 
     case ShowTableProperties(r: ResolvedView, propertyKey) =>
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablePropertiesExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablePropertiesExec.scala
index 0bcd7ea..fef63cb 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablePropertiesExec.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablePropertiesExec.scala
@@ -19,8 +19,8 @@ package org.apache.spark.sql.execution.datasources.v2
 
 import org.apache.spark.sql.catalyst.InternalRow
 import org.apache.spark.sql.catalyst.encoders.RowEncoder
-import org.apache.spark.sql.catalyst.expressions.{Attribute, GenericRowWithSchema}
-import org.apache.spark.sql.connector.catalog.Table
+import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeSet, GenericRowWithSchema}
+import org.apache.spark.sql.connector.catalog.{CatalogV2Util, Table, TableCatalog}
 
 /**
  * Physical plan node for showing table properties.
@@ -30,11 +30,15 @@ case class ShowTablePropertiesExec(
     catalogTable: Table,
     propertyKey: Option[String]) extends V2CommandExec {
 
+  override def producedAttributes: AttributeSet = AttributeSet(output)
+
   override protected def run(): Seq[InternalRow] = {
     import scala.collection.JavaConverters._
     val toRow = RowEncoder(schema).resolveAndBind().createSerializer()
 
+    // The reservered properties are accessible through DESCRIBE
     val properties = catalogTable.properties.asScala
+      .filter { case (k, v) => !CatalogV2Util.TABLE_RESERVED_PROPERTIES.contains(k) }
     propertyKey match {
       case Some(p) =>
         val propValue = properties
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSessionCatalogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSessionCatalogSuite.scala
index 249b27c..cf00b3b 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSessionCatalogSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSessionCatalogSuite.scala
@@ -17,7 +17,7 @@
 
 package org.apache.spark.sql.connector
 
-import org.apache.spark.sql.{DataFrame, SaveMode}
+import org.apache.spark.sql.{DataFrame, Row, SaveMode}
 import org.apache.spark.sql.connector.catalog.{Identifier, Table, TableCatalog}
 
 class DataSourceV2SQLSessionCatalogSuite
@@ -63,4 +63,20 @@ class DataSourceV2SQLSessionCatalogSuite
       }
     }
   }
+
+  test("SPARK-31624: SHOW TBLPROPERTIES working with V2 tables and the session catalog") {
+    val t1 = "tbl"
+    withTable(t1) {
+      sql(s"CREATE TABLE $t1 (id bigint, data string) USING $v2Format TBLPROPERTIES " +
+        "(key='v', key2='v2')")
+
+      checkAnswer(sql(s"SHOW TBLPROPERTIES $t1"), Seq(Row("key", "v"), Row("key2", "v2")))
+
+      checkAnswer(sql(s"SHOW TBLPROPERTIES $t1('key')"), Row("key", "v"))
+
+      checkAnswer(
+        sql(s"SHOW TBLPROPERTIES $t1('keyX')"),
+        Row("keyX", s"Table default.$t1 does not have property: keyX"))
+    }
+  }
 }
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala
index 3244684..e947e15 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala
@@ -2122,8 +2122,6 @@ class DataSourceV2SQLSuite
         .add("value", StringType, nullable = false)
 
       val expected = Seq(
-        Row(TableCatalog.PROP_OWNER, defaultUser),
-        Row("provider", provider),
         Row("status", status),
         Row("user", user))
 


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@spark.apache.org
For additional commands, e-mail: commits-help@spark.apache.org