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 2022/06/23 05:25:41 UTC

[spark] branch branch-3.3 updated: [SPARK-39547][SQL] V2SessionCatalog should not throw NoSuchDatabaseException in loadNamspaceMetadata

This is an automated email from the ASF dual-hosted git repository.

wenchen pushed a commit to branch branch-3.3
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/branch-3.3 by this push:
     new 86e3514ceea [SPARK-39547][SQL] V2SessionCatalog should not throw NoSuchDatabaseException in loadNamspaceMetadata
86e3514ceea is described below

commit 86e3514ceeac0b494ec58cbd413ae957f4d4bc8d
Author: Prashant Singh <ps...@amazon.com>
AuthorDate: Thu Jun 23 13:25:08 2022 +0800

    [SPARK-39547][SQL] V2SessionCatalog should not throw NoSuchDatabaseException in loadNamspaceMetadata
    
    ### What changes were proposed in this pull request?
    
    This change attempts to make V2SessionCatalog return NoSuchNameSpaceException rather than NoSuchDataseException
    
    ### Why are the changes needed?
    
    if a catalog doesn't overrides `namespaceExists` it by default uses `loadNamespaceMetadata` and in case a `db` not exists loadNamespaceMetadata throws a `NoSuchDatabaseException` which is not catched and we see failures even with `if exists` clause. One such use case we observed was in iceberg table a post test clean up was failing with `NoSuchDatabaseException` now. Also queries such as `DROP TABLE IF EXISTS {}` fails with no such db exception.
    
    ### Does this PR introduce _any_ user-facing change?
    
    No
    
    ### How was this patch tested?
    
    Modified the UT to match the proposed behviour
    
    Closes #36948 from singhpk234/fix/loadNamespaceMetadata.
    
    Authored-by: Prashant Singh <ps...@amazon.com>
    Signed-off-by: Wenchen Fan <we...@databricks.com>
    (cherry picked from commit 95133932a661742bf0dd1343bc7eda08f2cf752f)
    Signed-off-by: Wenchen Fan <we...@databricks.com>
---
 .../spark/sql/execution/datasources/v2/V2SessionCatalog.scala    | 9 +++++++--
 .../spark/sql/execution/command/v1/DescribeNamespaceSuite.scala  | 1 +
 .../sql/execution/datasources/v2/V2SessionCatalogSuite.scala     | 2 +-
 .../sql/hive/execution/command/DescribeNamespaceSuite.scala      | 1 +
 4 files changed, 10 insertions(+), 3 deletions(-)

diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala
index b9a4e0e6ba3..eb7e3d79832 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala
@@ -24,7 +24,7 @@ import scala.collection.JavaConverters._
 import scala.collection.mutable
 
 import org.apache.spark.sql.catalyst.{FunctionIdentifier, SQLConfHelper, TableIdentifier}
-import org.apache.spark.sql.catalyst.analysis.{NoSuchTableException, TableAlreadyExistsException}
+import org.apache.spark.sql.catalyst.analysis.{NoSuchDatabaseException, NoSuchTableException, TableAlreadyExistsException}
 import org.apache.spark.sql.catalyst.catalog.{CatalogDatabase, CatalogTable, CatalogTableType, CatalogUtils, SessionCatalog}
 import org.apache.spark.sql.connector.catalog.{CatalogManager, CatalogV2Util, FunctionCatalog, Identifier, NamespaceChange, SupportsNamespaces, Table, TableCatalog, TableChange, V1Table}
 import org.apache.spark.sql.connector.catalog.NamespaceChange.RemoveProperty
@@ -244,7 +244,12 @@ class V2SessionCatalog(catalog: SessionCatalog)
   override def loadNamespaceMetadata(namespace: Array[String]): util.Map[String, String] = {
     namespace match {
       case Array(db) =>
-        catalog.getDatabaseMetadata(db).toMetadata
+        try {
+          catalog.getDatabaseMetadata(db).toMetadata
+        } catch {
+          case _: NoSuchDatabaseException =>
+            throw QueryCompilationErrors.noSuchNamespaceError(namespace)
+        }
 
       case _ =>
         throw QueryCompilationErrors.noSuchNamespaceError(namespace)
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/DescribeNamespaceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/DescribeNamespaceSuite.scala
index aa4547db1e6..e71b311d241 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/DescribeNamespaceSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/DescribeNamespaceSuite.scala
@@ -59,5 +59,6 @@ trait DescribeNamespaceSuiteBase extends command.DescribeNamespaceSuiteBase
  * table catalog.
  */
 class DescribeNamespaceSuite extends DescribeNamespaceSuiteBase with CommandSuiteBase {
+  override def notFoundMsgPrefix: String = if (conf.useV1Command) "Database" else "Namespace"
   override def commandVersion: String = super[DescribeNamespaceSuiteBase].commandVersion
 }
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalogSuite.scala
index af0eafbc805..d37d5a96c65 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalogSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalogSuite.scala
@@ -850,7 +850,7 @@ class V2SessionCatalogNamespaceSuite extends V2SessionCatalogBaseSuite {
   test("loadNamespaceMetadata: fail missing namespace") {
     val catalog = newCatalog()
 
-    val exc = intercept[NoSuchDatabaseException] {
+    val exc = intercept[NoSuchNamespaceException] {
       catalog.loadNamespaceMetadata(testNs)
     }
 
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/DescribeNamespaceSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/DescribeNamespaceSuite.scala
index be8423fca0b..f730cad03ba 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/DescribeNamespaceSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/DescribeNamespaceSuite.scala
@@ -24,5 +24,6 @@ import org.apache.spark.sql.execution.command.v1
  * table catalog.
  */
 class DescribeNamespaceSuite extends v1.DescribeNamespaceSuiteBase with CommandSuiteBase {
+  override def notFoundMsgPrefix: String = if (conf.useV1Command) "Database" else "Namespace"
   override def commandVersion: String = super[DescribeNamespaceSuiteBase].commandVersion
 }


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