You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by ma...@apache.org on 2023/06/27 16:31:48 UTC

[spark] branch master updated: [SPARK-44171][SQL] Assign names to the error class _LEGACY_ERROR_TEMP_[2279-2282] & delete some unused error classes

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

maxgekk pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
     new be8b07a1534 [SPARK-44171][SQL] Assign names to the error class _LEGACY_ERROR_TEMP_[2279-2282] & delete some unused error classes
be8b07a1534 is described below

commit be8b07a15348d8fea15c33d35a75969ca1693ff6
Author: panbingkun <pb...@gmail.com>
AuthorDate: Tue Jun 27 19:31:30 2023 +0300

    [SPARK-44171][SQL] Assign names to the error class _LEGACY_ERROR_TEMP_[2279-2282] & delete some unused error classes
    
    ### What changes were proposed in this pull request?
    The pr aims to assign names to the error class _LEGACY_ERROR_TEMP_[2279-2282] and delete some unused error classes,  details as follows:
    _LEGACY_ERROR_TEMP_0036 -> `Delete`
    _LEGACY_ERROR_TEMP_1341 -> `Delete`
    _LEGACY_ERROR_TEMP_1342 -> `Delete`
    _LEGACY_ERROR_TEMP_1304 -> `Delete`
    _LEGACY_ERROR_TEMP_2072 -> `Delete`
    _LEGACY_ERROR_TEMP_2279 -> `Delete`
    _LEGACY_ERROR_TEMP_2280 -> UNSUPPORTED_FEATURE.COMMENT_NAMESPACE
    _LEGACY_ERROR_TEMP_2281 -> UNSUPPORTED_FEATURE.REMOVE_NAMESPACE_COMMENT
    _LEGACY_ERROR_TEMP_2282 -> UNSUPPORTED_FEATURE.DROP_NAMESPACE_RESTRICT
    
    ### Why are the changes needed?
    The changes improve the error framework.
    
    ### Does this PR introduce _any_ user-facing change?
    No.
    
    ### How was this patch tested?
    Pass GA.
    
    Closes #41721 from panbingkun/SPARK-44171.
    
    Lead-authored-by: panbingkun <pb...@gmail.com>
    Co-authored-by: panbingkun <84...@qq.com>
    Signed-off-by: Max Gekk <ma...@gmail.com>
---
 .../spark/sql/jdbc/v2/MySQLNamespaceSuite.scala    | 19 +++++--
 core/src/main/resources/error/error-classes.json   | 60 ++++++----------------
 .../spark/sql/errors/QueryCompilationErrors.scala  | 16 ------
 .../spark/sql/errors/QueryExecutionErrors.scala    | 30 +++++------
 .../org/apache/spark/sql/jdbc/MySQLDialect.scala   |  6 +--
 5 files changed, 47 insertions(+), 84 deletions(-)

diff --git a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/MySQLNamespaceSuite.scala b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/MySQLNamespaceSuite.scala
index a7ef8d4e104..d58146fecdf 100644
--- a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/MySQLNamespaceSuite.scala
+++ b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/MySQLNamespaceSuite.scala
@@ -73,7 +73,8 @@ class MySQLNamespaceSuite extends DockerJDBCIntegrationSuite with V2JDBCNamespac
       exception = intercept[SparkSQLFeatureNotSupportedException] {
         catalog.createNamespace(Array("foo"), Map("comment" -> "test comment").asJava)
       },
-      errorClass = "_LEGACY_ERROR_TEMP_2280"
+      errorClass = "UNSUPPORTED_FEATURE.COMMENT_NAMESPACE",
+      parameters = Map("namespace" -> "`foo`")
     )
     assert(catalog.namespaceExists(Array("foo")) === false)
     catalog.createNamespace(Array("foo"), Map.empty[String, String].asJava)
@@ -84,13 +85,25 @@ class MySQLNamespaceSuite extends DockerJDBCIntegrationSuite with V2JDBCNamespac
           Array("foo"),
           NamespaceChange.setProperty("comment", "comment for foo"))
       },
-      errorClass = "_LEGACY_ERROR_TEMP_2280")
+      errorClass = "UNSUPPORTED_FEATURE.COMMENT_NAMESPACE",
+      parameters = Map("namespace" -> "`foo`")
+    )
 
     checkError(
       exception = intercept[SparkSQLFeatureNotSupportedException] {
         catalog.alterNamespace(Array("foo"), NamespaceChange.removeProperty("comment"))
       },
-      errorClass = "_LEGACY_ERROR_TEMP_2281")
+      errorClass = "UNSUPPORTED_FEATURE.REMOVE_NAMESPACE_COMMENT",
+      parameters = Map("namespace" -> "`foo`")
+    )
+
+    checkError(
+      exception = intercept[SparkSQLFeatureNotSupportedException] {
+        catalog.dropNamespace(Array("foo"), cascade = false)
+      },
+      errorClass = "UNSUPPORTED_FEATURE.DROP_NAMESPACE",
+      parameters = Map("namespace" -> "`foo`")
+    )
     catalog.dropNamespace(Array("foo"), cascade = true)
     assert(catalog.namespaceExists(Array("foo")) === false)
   }
diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json
index 78b54d5230d..342af0ffa6c 100644
--- a/core/src/main/resources/error/error-classes.json
+++ b/core/src/main/resources/error/error-classes.json
@@ -2383,11 +2383,21 @@
           "Combination of ORDER BY/SORT BY/DISTRIBUTE BY/CLUSTER BY."
         ]
       },
+      "COMMENT_NAMESPACE" : {
+        "message" : [
+          "Attach a comment to the namespace <namespace>."
+        ]
+      },
       "DESC_TABLE_COLUMN_PARTITION" : {
         "message" : [
           "DESC TABLE COLUMN for a specific partition."
         ]
       },
+      "DROP_NAMESPACE" : {
+        "message" : [
+          "Drop the namespace <namespace>."
+        ]
+      },
       "INSERT_PARTITION_SPEC_IF_NOT_EXISTS" : {
         "message" : [
           "INSERT INTO <tableName> with IF NOT EXISTS in the PARTITION spec."
@@ -2468,6 +2478,11 @@
           "Python UDF in the ON clause of a <joinType> JOIN. In case of an INNNER JOIN consider rewriting to a CROSS JOIN with a WHERE clause."
         ]
       },
+      "REMOVE_NAMESPACE_COMMENT" : {
+        "message" : [
+          "Remove a comment from the namespace <namespace>."
+        ]
+      },
       "SET_NAMESPACE_PROPERTY" : {
         "message" : [
           "<property> is a reserved namespace property, <msg>."
@@ -2862,11 +2877,6 @@
       "Operation not allowed: <message>."
     ]
   },
-  "_LEGACY_ERROR_TEMP_0036" : {
-    "message" : [
-      "Expected `NOSCAN` instead of `<ctx>`."
-    ]
-  },
   "_LEGACY_ERROR_TEMP_0037" : {
     "message" : [
       "It is not allowed to add catalog/namespace prefix <quoted> to the table name in CACHE TABLE AS SELECT."
@@ -4111,11 +4121,6 @@
       "Unable to find the column `<colName>` given [<actualColumns>]."
     ]
   },
-  "_LEGACY_ERROR_TEMP_1304" : {
-    "message" : [
-      "Unexpected type <className> of the relation <tableName>."
-    ]
-  },
   "_LEGACY_ERROR_TEMP_1305" : {
     "message" : [
       "Unsupported TableChange <change> in JDBC catalog."
@@ -4247,16 +4252,6 @@
       "Failed to execute UPDATE command because the SET list contains a DEFAULT column reference as part of another expression; this is not allowed."
     ]
   },
-  "_LEGACY_ERROR_TEMP_1341" : {
-    "message" : [
-      "Failed to execute UPDATE command because the WHERE clause contains a DEFAULT column reference; this is not allowed."
-    ]
-  },
-  "_LEGACY_ERROR_TEMP_1342" : {
-    "message" : [
-      "Failed to execute MERGE command because the WHERE clause contains a DEFAULT column reference; this is not allowed."
-    ]
-  },
   "_LEGACY_ERROR_TEMP_1343" : {
     "message" : [
       "Failed to execute MERGE INTO command because one of its INSERT or UPDATE assignments contains a DEFAULT column reference as part of another expression; this is not allowed."
@@ -4561,11 +4556,6 @@
       "Commit denied for partition <partId> (task <taskId>, attempt <attemptId>, stage <stageId>.<stageAttempt>)."
     ]
   },
-  "_LEGACY_ERROR_TEMP_2072" : {
-    "message" : [
-      "Table implementation does not support writes: <ident>."
-    ]
-  },
   "_LEGACY_ERROR_TEMP_2073" : {
     "message" : [
       "Cannot create JDBC table with partition."
@@ -5502,26 +5492,6 @@
       "The input <valueType> '<input>' does not match the given number format: '<format>'."
     ]
   },
-  "_LEGACY_ERROR_TEMP_2279" : {
-    "message" : [
-      "Multiple bucket transforms are not supported."
-    ]
-  },
-  "_LEGACY_ERROR_TEMP_2280" : {
-    "message" : [
-      "Create namespace comment is not supported."
-    ]
-  },
-  "_LEGACY_ERROR_TEMP_2281" : {
-    "message" : [
-      "Remove namespace comment is not supported."
-    ]
-  },
-  "_LEGACY_ERROR_TEMP_2282" : {
-    "message" : [
-      "Drop namespace restrict is not supported."
-    ]
-  },
   "_LEGACY_ERROR_TEMP_2300" : {
     "message" : [
       "The number of lambda function arguments '<namesSize>' does not match the number of arguments expected by the higher order function '<argInfoSize>'."
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala
index ecae9778fd0..17319eb46ad 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala
@@ -3164,22 +3164,6 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase {
       messageParameters = Map.empty)
   }
 
-  // Return a more descriptive error message if the user tries to use a DEFAULT column reference
-  // inside an UPDATE command's WHERE clause; this is not allowed.
-  def defaultReferencesNotAllowedInUpdateWhereClause(): Throwable = {
-    new AnalysisException(
-      errorClass = "_LEGACY_ERROR_TEMP_1341",
-      messageParameters = Map.empty)
-  }
-
-  // Return a more descriptive error message if the user tries to use a DEFAULT column reference
-  // inside an UPDATE command's WHERE clause; this is not allowed.
-  def defaultReferencesNotAllowedInMergeCondition(): Throwable = {
-    new AnalysisException(
-      errorClass = "_LEGACY_ERROR_TEMP_1342",
-      messageParameters = Map.empty)
-  }
-
   def defaultReferencesNotAllowedInComplexExpressionsInMergeInsertsOrUpdates(): Throwable = {
     new AnalysisException(
       errorClass = "_LEGACY_ERROR_TEMP_1343",
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala
index df2116df8f2..cfaf39cb7ef 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala
@@ -45,7 +45,7 @@ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
 import org.apache.spark.sql.catalyst.plans.logical.statsEstimation.ValueInterval
 import org.apache.spark.sql.catalyst.trees.{SQLQueryContext, TreeNode}
 import org.apache.spark.sql.catalyst.util.{sideBySide, BadRecordException, DateTimeUtils, FailFastMode}
-import org.apache.spark.sql.connector.catalog.{CatalogNotFoundException, Identifier, Table, TableProvider}
+import org.apache.spark.sql.connector.catalog.{CatalogNotFoundException, Table, TableProvider}
 import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._
 import org.apache.spark.sql.connector.expressions.Transform
 import org.apache.spark.sql.internal.SQLConf
@@ -929,13 +929,6 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase {
       cause = null)
   }
 
-  def unsupportedTableWritesError(ident: Identifier): Throwable = {
-    new SparkException(
-      errorClass = "_LEGACY_ERROR_TEMP_2072",
-      messageParameters = Map("idnt" -> ident.quoted),
-      cause = null)
-  }
-
   def cannotCreateJDBCTableWithPartitionsError(): SparkUnsupportedOperationException = {
     new SparkUnsupportedOperationException(
       errorClass = "_LEGACY_ERROR_TEMP_2073",
@@ -2742,22 +2735,25 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase {
       messageParameters = Map.empty)
   }
 
-  def unsupportedCreateNamespaceCommentError(): SparkSQLFeatureNotSupportedException = {
+  def unsupportedCommentNamespaceError(
+      namespace: String): SparkSQLFeatureNotSupportedException = {
     new SparkSQLFeatureNotSupportedException(
-      errorClass = "_LEGACY_ERROR_TEMP_2280",
-      messageParameters = Map.empty)
+      errorClass = "UNSUPPORTED_FEATURE.COMMENT_NAMESPACE",
+      messageParameters = Map("namespace" -> toSQLId(namespace)))
   }
 
-  def unsupportedRemoveNamespaceCommentError(): SparkSQLFeatureNotSupportedException = {
+  def unsupportedRemoveNamespaceCommentError(
+      namespace: String): SparkSQLFeatureNotSupportedException = {
     new SparkSQLFeatureNotSupportedException(
-      errorClass = "_LEGACY_ERROR_TEMP_2281",
-      messageParameters = Map.empty)
+      errorClass = "UNSUPPORTED_FEATURE.REMOVE_NAMESPACE_COMMENT",
+      messageParameters = Map("namespace" -> toSQLId(namespace)))
   }
 
-  def unsupportedDropNamespaceRestrictError(): SparkSQLFeatureNotSupportedException = {
+  def unsupportedDropNamespaceError(
+      namespace: String): SparkSQLFeatureNotSupportedException = {
     new SparkSQLFeatureNotSupportedException(
-      errorClass = "_LEGACY_ERROR_TEMP_2282",
-      messageParameters = Map.empty)
+      errorClass = "UNSUPPORTED_FEATURE.DROP_NAMESPACE",
+      messageParameters = Map("namespace" -> toSQLId(namespace)))
   }
 
   def timestampAddOverflowError(micros: Long, amount: Int, unit: String): ArithmeticException = {
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/MySQLDialect.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/MySQLDialect.scala
index d6edb67e57e..a08c89318b6 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/MySQLDialect.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/MySQLDialect.scala
@@ -188,11 +188,11 @@ private case object MySQLDialect extends JdbcDialect with SQLConfHelper {
   }
 
   override def getSchemaCommentQuery(schema: String, comment: String): String = {
-    throw QueryExecutionErrors.unsupportedCreateNamespaceCommentError()
+    throw QueryExecutionErrors.unsupportedCommentNamespaceError(schema)
   }
 
   override def removeSchemaCommentQuery(schema: String): String = {
-    throw QueryExecutionErrors.unsupportedRemoveNamespaceCommentError()
+    throw QueryExecutionErrors.unsupportedRemoveNamespaceCommentError(schema)
   }
 
   // CREATE INDEX syntax
@@ -296,7 +296,7 @@ private case object MySQLDialect extends JdbcDialect with SQLConfHelper {
     if (cascade) {
       s"DROP SCHEMA ${quoteIdentifier(schema)}"
     } else {
-      throw QueryExecutionErrors.unsupportedDropNamespaceRestrictError()
+      throw QueryExecutionErrors.unsupportedDropNamespaceError(schema)
     }
   }
 


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