You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by GitBox <gi...@apache.org> on 2022/10/07 09:47:05 UTC

[GitHub] [spark] MaxGekk commented on a diff in pull request #37887: [SPARK-40360] ALREADY_EXISTS and NOT_FOUND exceptions

MaxGekk commented on code in PR #37887:
URL: https://github.com/apache/spark/pull/37887#discussion_r989885531


##########
core/src/main/resources/error/error-classes.json:
##########
@@ -536,12 +563,71 @@
       "Failed to set original permission <permission> back to the created path: <path>. Exception: <message>"
     ]
   },
+  "ROUTINE_ALREADY_EXISTS" : {
+    "message" : [
+      "Cannot create the function <routineName> because it already exists.",
+      "Choose a different name, drop or replace the existing function, or add the IF NOT EXISTS clause to tolerate a pre-existing function."
+    ],
+    "sqlState" : "42000"
+  },
+  "ROUTINE_NOT_FOUND" : {

Review Comment:
   Why do you name it `ROUTINE...` if SQL key word is `FUNCTION...`. The last one is even used in the error message. Could you rename to `FUNCTION...` for consistency, otherwise, please, explain the difference.



##########
core/src/main/resources/error/error-classes.json:
##########
@@ -346,6 +346,18 @@
       }
     }
   },
+  "INDEX_ALREADY_EXISTS" : {

Review Comment:
   Where is it used?



##########
core/src/main/resources/error/error-classes.json:
##########
@@ -536,12 +563,71 @@
       "Failed to set original permission <permission> back to the created path: <path>. Exception: <message>"
     ]
   },
+  "ROUTINE_ALREADY_EXISTS" : {
+    "message" : [
+      "Cannot create the function <routineName> because it already exists.",
+      "Choose a different name, drop or replace the existing function, or add the IF NOT EXISTS clause to tolerate a pre-existing function."
+    ],
+    "sqlState" : "42000"
+  },
+  "ROUTINE_NOT_FOUND" : {
+    "message" : [
+      "The function <routineName> cannot be found. Verify the spelling and correctness of the schema and catalog.",
+      "If you did not qualify the name with a schema and catalog, verify the current_schema() output, or qualify the name with the correct schema and catalog.",
+      "To tolerate the error on drop use DROP FUNCTION IF EXISTS."
+    ],
+    "sqlState" : "42000"
+  },
+  "SCHEMA_ALREADY_EXISTS" : {
+    "message" : [
+      "Cannot create schema <schemaName> because it already exists.",
+      "Choose a different name, drop the existing schema, or add the IF NOT EXISTS clause to tolerate pre-existing schema."
+    ],
+    "sqlState" : "42000"
+  },
+  "SCHEMA_NOT_EMPTY" : {
+    "message" : [
+      "Cannot drop a schema <schemaName> because it contains objects.",
+      "Use DROP SCHEMA ... CASCADE to drop the schema and all its objects."
+    ],
+    "sqlState" : "42000"
+  },
+  "SCHEMA_NOT_FOUND" : {
+    "message" : [
+      "The schema <schemaName> cannot be found. Verify the spelling and correctness of the schema and catalog.",
+      "If you did not qualify the name with a catalog, verify the current_schema() output, or qualify the name with the correct catalog.",
+      "To tolerate the error on drop use DROP SCHEMA IF EXISTS."
+    ],
+    "sqlState" : "42000"
+  },
   "SECOND_FUNCTION_ARGUMENT_NOT_INTEGER" : {
     "message" : [
       "The second argument of <functionName> function needs to be an integer."
     ],
     "sqlState" : "22023"
   },
+  "TABLE_OR_VIEW_ALREADY_EXISTS" : {
+    "message" : [
+      "Cannot create table or view <relationName> because it already exists.",
+      "Choose a different name, drop or replace the existing object, or add the IF NOT EXISTS clause to tolerate pre-existing objects."
+    ],
+    "sqlState" : "42000"
+  },
+  "TABLE_OR_VIEW_NOT_FOUND" : {
+    "message" : [
+      "The table or view <relationName> cannot be found. Verify the spelling and correctness of the schema and catalog.",
+      "If you did not qualify the name with a schema, verify the current_schema() output, or qualify the name with the correct schema and catalog.",
+      "To tolerate the error on drop use DROP VIEW IF EXISTS or DROP TABLE IF EXISTS."
+    ],
+    "sqlState" : "42000"
+  },
+  "TEMP_TABLE_OR_VIEW_ALREADY_EXISTS" : {
+    "message" : [
+      "Cannot create the temporary view <relationName> because it already exists.",

Review Comment:
   The name is about temp **table** or view, but the error message says only about view.



##########
core/src/test/scala/org/apache/spark/SparkFunSuite.scala:
##########
@@ -374,6 +382,19 @@ abstract class SparkFunSuite
     checkError(exception, errorClass, sqlState, parameters,
       matchPVals = true, Array(context))
 
+  protected def checkErrorTableNotFound(

Review Comment:
   Here is the core, right? which is not aware of any tables. Please, move it from here.



##########
core/src/main/resources/error/error-classes.json:
##########
@@ -536,12 +563,71 @@
       "Failed to set original permission <permission> back to the created path: <path>. Exception: <message>"
     ]
   },
+  "ROUTINE_ALREADY_EXISTS" : {
+    "message" : [
+      "Cannot create the function <routineName> because it already exists.",
+      "Choose a different name, drop or replace the existing function, or add the IF NOT EXISTS clause to tolerate a pre-existing function."
+    ],
+    "sqlState" : "42000"
+  },
+  "ROUTINE_NOT_FOUND" : {
+    "message" : [
+      "The function <routineName> cannot be found. Verify the spelling and correctness of the schema and catalog.",
+      "If you did not qualify the name with a schema and catalog, verify the current_schema() output, or qualify the name with the correct schema and catalog.",
+      "To tolerate the error on drop use DROP FUNCTION IF EXISTS."
+    ],
+    "sqlState" : "42000"
+  },
+  "SCHEMA_ALREADY_EXISTS" : {
+    "message" : [
+      "Cannot create schema <schemaName> because it already exists.",
+      "Choose a different name, drop the existing schema, or add the IF NOT EXISTS clause to tolerate pre-existing schema."
+    ],
+    "sqlState" : "42000"
+  },
+  "SCHEMA_NOT_EMPTY" : {
+    "message" : [
+      "Cannot drop a schema <schemaName> because it contains objects.",
+      "Use DROP SCHEMA ... CASCADE to drop the schema and all its objects."
+    ],
+    "sqlState" : "42000"
+  },
+  "SCHEMA_NOT_FOUND" : {
+    "message" : [
+      "The schema <schemaName> cannot be found. Verify the spelling and correctness of the schema and catalog.",
+      "If you did not qualify the name with a catalog, verify the current_schema() output, or qualify the name with the correct catalog.",
+      "To tolerate the error on drop use DROP SCHEMA IF EXISTS."
+    ],
+    "sqlState" : "42000"
+  },
   "SECOND_FUNCTION_ARGUMENT_NOT_INTEGER" : {
     "message" : [
       "The second argument of <functionName> function needs to be an integer."
     ],
     "sqlState" : "22023"
   },
+  "TABLE_OR_VIEW_ALREADY_EXISTS" : {
+    "message" : [
+      "Cannot create table or view <relationName> because it already exists.",

Review Comment:
   `relationName` is new name, right? What's the old one, and why do you ignore it in exception.



##########
core/src/main/resources/error/error-classes.json:
##########
@@ -346,6 +346,18 @@
       }
     }
   },
+  "INDEX_ALREADY_EXISTS" : {
+    "message" : [
+      "Cannot create the index because it already exists. <message>."
+    ],
+    "sqlState" : "42000"
+  },
+  "INDEX_NOT_FOUND" : {
+    "message" : [
+      "Cannot find the index. <message>."

Review Comment:
   What's the `message`? Can't you create sub-classes instead?



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala:
##########
@@ -784,21 +784,17 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase {
   }
 
   def renameTempViewToExistingViewError(oldName: String, newName: String): Throwable = {
-    new AnalysisException(
-      errorClass = "_LEGACY_ERROR_TEMP_1062",

Review Comment:
   If you remove this error classes, please, remove it from `error-classes.json` too.



##########
core/src/test/scala/org/apache/spark/SparkFunSuite.scala:
##########
@@ -316,14 +316,22 @@ abstract class SparkFunSuite
     } else {
       assert(expectedParameters === parameters)
     }
-    val actualQueryContext = exception.getQueryContext()
-    assert(actualQueryContext.length === queryContext.length, "Invalid length of the query context")
-    actualQueryContext.zip(queryContext).foreach { case (actual, expected) =>
-      assert(actual.objectType() === expected.objectType(), "Invalid objectType of a query context")
-      assert(actual.objectName() === expected.objectName(), "Invalid objectName of a query context")
-      assert(actual.startIndex() === expected.startIndex(), "Invalid startIndex of a query context")
-      assert(actual.stopIndex() === expected.stopIndex(), "Invalid stopIndex of a query context")
-      assert(actual.fragment() === expected.fragment(), "Invalid fragment of a query context")
+    if (!queryContext.isEmpty) {

Review Comment:
   Please, remove this check. Test must reflect current imple of exception. If exceptions have already had query contexts, test should check them.



##########
core/src/main/resources/error/error-classes.json:
##########
@@ -536,12 +563,71 @@
       "Failed to set original permission <permission> back to the created path: <path>. Exception: <message>"
     ]
   },
+  "ROUTINE_ALREADY_EXISTS" : {
+    "message" : [
+      "Cannot create the function <routineName> because it already exists.",
+      "Choose a different name, drop or replace the existing function, or add the IF NOT EXISTS clause to tolerate a pre-existing function."
+    ],
+    "sqlState" : "42000"
+  },
+  "ROUTINE_NOT_FOUND" : {
+    "message" : [
+      "The function <routineName> cannot be found. Verify the spelling and correctness of the schema and catalog.",
+      "If you did not qualify the name with a schema and catalog, verify the current_schema() output, or qualify the name with the correct schema and catalog.",
+      "To tolerate the error on drop use DROP FUNCTION IF EXISTS."
+    ],
+    "sqlState" : "42000"
+  },
+  "SCHEMA_ALREADY_EXISTS" : {

Review Comment:
   Why not `NAMESPACE`?



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/AlreadyExistException.scala:
##########
@@ -20,72 +20,105 @@ package org.apache.spark.sql.catalyst.analysis
 import org.apache.spark.sql.AnalysisException
 import org.apache.spark.sql.catalyst.InternalRow
 import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec
-import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._
-import org.apache.spark.sql.connector.catalog.Identifier
+import org.apache.spark.sql.catalyst.util.{quoteIdentifier, quoteNameParts }
 import org.apache.spark.sql.types.StructType
 
 /**
  * Thrown by a catalog when an item already exists. The analyzer will rethrow the exception
  * as an [[org.apache.spark.sql.AnalysisException]] with the correct position information.
  */
 class DatabaseAlreadyExistsException(db: String)
-  extends NamespaceAlreadyExistsException(s"Database '$db' already exists")
+  extends NamespaceAlreadyExistsException(Array(db))
 
-class NamespaceAlreadyExistsException(message: String)
-  extends AnalysisException(
-    message,
-    errorClass = Some("_LEGACY_ERROR_TEMP_1118"),
-    messageParameters = Map("msg" -> message)) {
+
+class NamespaceAlreadyExistsException(errorClass: String, messageParameters: Map[String, String])
+  extends AnalysisException(errorClass, messageParameters) {
   def this(namespace: Array[String]) = {
-    this(s"Namespace '${namespace.quoted}' already exists")
+    this(errorClass = "SCHEMA_ALREADY_EXISTS",
+      Map("schemaName" -> quoteNameParts(namespace)))
   }
 }
 
-class TableAlreadyExistsException(message: String, cause: Option[Throwable] = None)
-  extends AnalysisException(
-    message,
-    errorClass = Some("_LEGACY_ERROR_TEMP_1116"),
-    messageParameters = Map("msg" -> message),
-    cause = cause) {
+
+class TableAlreadyExistsException(errorClass: String, messageParameters: Map[String, String],
+  cause: Option[Throwable] = None)
+  extends AnalysisException(errorClass, messageParameters, cause = cause) {
   def this(db: String, table: String) = {
-    this(s"Table or view '$table' already exists in database '$db'")
+    this(errorClass = "TABLE_OR_VIEW_ALREADY_EXISTS",
+      messageParameters = Map("relationName" ->
+        (quoteIdentifier(db) + "." + quoteIdentifier(table))))

Review Comment:
   Can't you used existing functions to form the name?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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