diff --git a/R/pkg/tests/fulltests/test_sparkSQL.R b/R/pkg/tests/fulltests/test_sparkSQL.R index 68fa6aac8e7ae..b40e0d1327d20 100644 --- a/R/pkg/tests/fulltests/test_sparkSQL.R +++ b/R/pkg/tests/fulltests/test_sparkSQL.R @@ -722,7 +722,7 @@ test_that("test tableExists, cache, uncache and clearCache", { clearCache() expect_error(uncacheTable("zxwtyswklpf"), - "Error in uncacheTable : analysis error - Table or view not found: zxwtyswklpf") + "[TABLE_OR_VIEW_NOT_FOUND] The table or view `zxwtyswklpf` cannot be found.") expect_true(tableExists("table1")) expect_true(tableExists("default.table1")) @@ -3367,8 +3367,8 @@ test_that("approxQuantile() on a DataFrame", { test_that("SQL error message is returned from JVM", { retError <- tryCatch(sql("select * from blah"), error = function(e) e) - expect_equal(grepl("Table or view not found", retError), TRUE) - expect_equal(grepl("blah", retError), TRUE) + expect_equal(grepl("[TABLE_OR_VIEW_NOT_FOUND]", retError), TRUE) + expect_equal(grepl("`blah`", retError), TRUE) }) irisDF <- suppressWarnings(createDataFrame(iris)) @@ -4076,8 +4076,7 @@ test_that("catalog APIs, currentDatabase, setCurrentDatabase, listDatabases, get expect_equal(currentDatabase(), "default") expect_error(setCurrentDatabase("default"), NA) expect_error(setCurrentDatabase("zxwtyswklpf"), - paste0("Error in setCurrentDatabase : no such database - Database ", - "'zxwtyswklpf' not found")) + "[SCHEMA_NOT_FOUND] The schema `zxwtyswklpf` cannot be found.") expect_true(databaseExists("default")) expect_true(databaseExists("spark_catalog.default")) @@ -4109,7 +4108,7 @@ test_that("catalog APIs, listTables, getTable, listColumns, listFunctions, funct tbs <- collect(tb) expect_true(nrow(tbs[tbs$name == "cars", ]) > 0) expect_error(listTables("bar"), - "Error in listTables : no such database - Database 'bar' not found") + "[SCHEMA_NOT_FOUND] The schema `bar` cannot be found.") c <- listColumns("cars") expect_equal(nrow(c), 2) @@ -4117,7 +4116,8 @@ test_that("catalog APIs, listTables, getTable, listColumns, listFunctions, funct c("name", "description", "dataType", "nullable", "isPartition", "isBucket")) expect_equal(collect(c)[[1]][[1]], "speed") expect_error(listColumns("zxwtyswklpf", "default"), - paste("Table or view not found: spark_catalog.default.zxwtyswklpf")) + paste("[TABLE_OR_VIEW_NOT_FOUND]", + " The table or view `spark_catalog`.`default`.`zxwtyswklpf` cannot be found.")) f <- listFunctions() expect_true(nrow(f) >= 200) # 250 @@ -4126,8 +4126,7 @@ test_that("catalog APIs, listTables, getTable, listColumns, listFunctions, funct expect_equal(take(orderBy(filter(f, "className IS NOT NULL"), "className"), 1)$className, "org.apache.spark.sql.catalyst.expressions.Abs") expect_error(listFunctions("zxwtyswklpf_db"), - paste("Error in listFunctions : no such database - Database", - "'zxwtyswklpf_db' not found")) + paste("[SCHEMA_NOT_FOUND] The schema `zxwtyswklpf_db` cannot be found.")) expect_true(functionExists("abs")) expect_false(functionExists("aabbss")) diff --git a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/V2JDBCNamespaceTest.scala b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/V2JDBCNamespaceTest.scala index bae0d7c361635..50e79e03a7a19 100644 --- a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/V2JDBCNamespaceTest.scala +++ b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/V2JDBCNamespaceTest.scala @@ -87,10 +87,12 @@ private[v2] trait V2JDBCNamespaceTest extends SharedSparkSession with DockerInte } assert(catalog.namespaceExists(Array("foo")) === false) assert(catalog.listNamespaces() === builtinNamespaces) - val msg = intercept[AnalysisException] { + val e = intercept[AnalysisException] { catalog.listNamespaces(Array("foo")) - }.getMessage - assert(msg.contains("Namespace 'foo' not found")) + } + checkError(e, + errorClass = "SCHEMA_NOT_FOUND", + parameters = Map("schemaName" -> "`foo`")) } } diff --git a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/V2JDBCTest.scala b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/V2JDBCTest.scala index 543c8465ed2b5..1c885178acf32 100644 --- a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/V2JDBCTest.scala +++ b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/V2JDBCTest.scala @@ -20,8 +20,9 @@ package org.apache.spark.sql.jdbc.v2 import org.apache.logging.log4j.Level import org.apache.spark.sql.{AnalysisException, DataFrame} -import org.apache.spark.sql.catalyst.analysis.{IndexAlreadyExistsException, NoSuchIndexException} +import org.apache.spark.sql.catalyst.analysis.{IndexAlreadyExistsException, NoSuchIndexException, UnresolvedAttribute} import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Filter, Sample} +import org.apache.spark.sql.catalyst.util.quoteIdentifier import org.apache.spark.sql.connector.catalog.{Catalogs, Identifier, TableCatalog} import org.apache.spark.sql.connector.catalog.index.SupportsIndex import org.apache.spark.sql.connector.expressions.aggregate.GeneralAggregateFunc @@ -99,10 +100,12 @@ private[v2] trait V2JDBCTest extends SharedSparkSession with DockerIntegrationFu assert(msg.contains("Cannot add column, because C3 already exists")) } // Add a column to not existing table - val msg = intercept[AnalysisException] { + val e = intercept[AnalysisException] { sql(s"ALTER TABLE $catalogName.not_existing_table ADD COLUMNS (C4 STRING)") - }.getMessage - assert(msg.contains("Table not found")) + } + checkError(e, + errorClass = "TABLE_OR_VIEW_NOT_FOUND", + parameters = Map("relation_name" -> s"`$catalogName`.`not_existing_table`")) } test("SPARK-33034: ALTER TABLE ... drop column") { @@ -120,10 +123,12 @@ private[v2] trait V2JDBCTest extends SharedSparkSession with DockerIntegrationFu assert(msg.contains(s"Missing field bad_column in table $catalogName.alt_table")) } // Drop a column from a not existing table - val msg = intercept[AnalysisException] { + val e = intercept[AnalysisException] { sql(s"ALTER TABLE $catalogName.not_existing_table DROP COLUMN C1") - }.getMessage - assert(msg.contains("Table not found")) + } + checkError(e, + errorClass = "TABLE_OR_VIEW_NOT_FOUND", + parameters = Map("relation_name" -> s"`$catalogName`.`not_existing_table`")) } test("SPARK-33034: ALTER TABLE ... update column type") { @@ -136,10 +141,12 @@ private[v2] trait V2JDBCTest extends SharedSparkSession with DockerIntegrationFu assert(msg2.contains("Missing field bad_column")) } // Update column type in not existing table - val msg = intercept[AnalysisException] { + val e = intercept[AnalysisException] { sql(s"ALTER TABLE $catalogName.not_existing_table ALTER COLUMN id TYPE DOUBLE") - }.getMessage - assert(msg.contains("Table not found")) + } + checkError(e, + errorClass = "TABLE_OR_VIEW_NOT_FOUND", + parameters = Map("relation_name" -> s"`$catalogName`.`not_existing_table`")) } test("SPARK-33034: ALTER TABLE ... rename column") { @@ -154,10 +161,14 @@ private[v2] trait V2JDBCTest extends SharedSparkSession with DockerIntegrationFu assert(msg.contains("Cannot rename column, because ID2 already exists")) } // Rename a column in a not existing table - val msg = intercept[AnalysisException] { + val e = intercept[AnalysisException] { sql(s"ALTER TABLE $catalogName.not_existing_table RENAME COLUMN ID TO C") - }.getMessage - assert(msg.contains("Table not found")) + } + checkError(e, + errorClass = "TABLE_OR_VIEW_NOT_FOUND", + parameters = Map("relation_name" -> + UnresolvedAttribute.parseAttributeName(s"$catalogName.not_existing_table") + .map(part => quoteIdentifier(part)).mkString("."))) } test("SPARK-33034: ALTER TABLE ... update column nullability") { @@ -165,10 +176,12 @@ private[v2] trait V2JDBCTest extends SharedSparkSession with DockerIntegrationFu testUpdateColumnNullability(s"$catalogName.alt_table") } // Update column nullability in not existing table - val msg = intercept[AnalysisException] { + val e = intercept[AnalysisException] { sql(s"ALTER TABLE $catalogName.not_existing_table ALTER COLUMN ID DROP NOT NULL") - }.getMessage - assert(msg.contains("Table not found")) + } + checkError(e, + errorClass = "TABLE_OR_VIEW_NOT_FOUND", + parameters = Map("relation_name" -> s"`$catalogName`.`not_existing_table`")) } test("CREATE TABLE with table comment") { @@ -187,10 +200,12 @@ private[v2] trait V2JDBCTest extends SharedSparkSession with DockerIntegrationFu test("CREATE TABLE with table property") { withTable(s"$catalogName.new_table") { - val m = intercept[AnalysisException] { + val e = intercept[AnalysisException] { sql(s"CREATE TABLE $catalogName.new_table (i INT) TBLPROPERTIES('a'='1')") - }.message - assert(m.contains("Failed table creation")) + } + checkError(e, + errorClass = "TABLE_OR_VIEW_ALREADY_EXISTS", + parameters = Map("relation_name" -> s"`$catalogName`.`new_table`")) testCreateTableWithProperty(s"$catalogName.new_table") } } diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json index 999a9b0a4aec6..0b68af2d62540 100644 --- a/core/src/main/resources/error/error-classes.json +++ b/core/src/main/resources/error/error-classes.json @@ -239,6 +239,24 @@ } } }, + "INDEX_ALREADY_EXISTS" : { + "message" : [ + "Cannot create the index because it already exists. ." + ], + "sqlState" : "42000" + }, + "INDEX_NOT_FOUND" : { + "message" : [ + "Cannot find the index. ." + ], + "sqlState" : "42000" + }, + "INDEX_OUT_OF_BOUNDS" : { + "message" : [ + "Index must be between 0 and the length of the ArrayData." + ], + "sqlState" : "22023" + }, "INTERNAL_ERROR" : { "message" : [ "" @@ -400,6 +418,21 @@ ], "sqlState" : "42000" }, + "PARTITIONS_ALREADY_EXIST" : { + "message" : [ + "Cannot ADD or RENAME TO partition(s) in table because they already exist.", + "Choose a different name, drop the existing partition, or add the IF NOT EXISTS clause to tolerate a pre-existing partition." + ], + "sqlState" : "42000" + }, + "PARTITIONS_NOT_FOUND" : { + "message" : [ + "The partition(s) cannot be found in table .", + "Verify the partition specification and table name.", + "To tolerate the error on drop use ALTER TABLE … DROP IF EXISTS PARTITION." + ], + "sqlState" : "42000" + }, "PIVOT_VALUE_DATA_TYPE_MISMATCH" : { "message" : [ "Invalid pivot value '': value data type does not match pivot column data type " @@ -417,12 +450,71 @@ "Failed to set original permission back to the created path: . Exception: " ] }, + "ROUTINE_ALREADY_EXISTS" : { + "message" : [ + "Cannot create the function 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 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 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 because it contains objects.", + "Use DROP SCHEMA ... CASCADE to drop the schema and all its objects." + ], + "sqlState" : "42000" + }, + "SCHEMA_NOT_FOUND" : { + "message" : [ + "The schema 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 function needs to be an integer." ], "sqlState" : "22023" }, + "TABLE_OR_VIEW_ALREADY_EXISTS" : { + "message" : [ + "Cannot create table or view 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 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 because it already exists.", + "Choose a different name, drop or replace the existing view, or add the IF NOT EXISTS clause to tolerate pre-existing views." + ], + "sqlState" : "42000" + }, "TOO_MANY_ARRAY_ELEMENTS" : { "message" : [ "Cannot initialize array with elements of size " diff --git a/core/src/test/scala/org/apache/spark/SparkFunSuite.scala b/core/src/test/scala/org/apache/spark/SparkFunSuite.scala index ade20dbff83ff..2399ac5473756 100644 --- a/core/src/test/scala/org/apache/spark/SparkFunSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkFunSuite.scala @@ -321,14 +321,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) { + 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") + } } } diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/SupportsAtomicPartitionManagement.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/SupportsAtomicPartitionManagement.java index e2c693f2d0a92..794d4d5c5b863 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/SupportsAtomicPartitionManagement.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/SupportsAtomicPartitionManagement.java @@ -54,7 +54,8 @@ default void createPartition( try { createPartitions(new InternalRow[]{ident}, new Map[]{properties}); } catch (PartitionsAlreadyExistException e) { - throw new PartitionAlreadyExistsException(e.getMessage()); + throw new PartitionAlreadyExistsException("PARTITIONS_ALREADY_EXIST", + e.messageParameters()); } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/AlreadyExistException.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/AlreadyExistException.scala index fb177251a7306..af73dc2647c49 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/AlreadyExistException.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/AlreadyExistException.scala @@ -20,6 +20,7 @@ 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.catalyst.util.quoteIdentifier import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ import org.apache.spark.sql.connector.catalog.Identifier import org.apache.spark.sql.types.StructType @@ -29,55 +30,103 @@ import org.apache.spark.sql.types.StructType * 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) { +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" -> (namespace.map(quoteIdentifier).mkString(".")))) } } -class TableAlreadyExistsException(message: String, cause: Option[Throwable] = None) - extends AnalysisException(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)))) + } + + def this(table: String) = { + this(errorClass = "TABLE_OR_VIEW_ALREADY_EXISTS", + messageParameters = Map("relationName" -> + (UnresolvedAttribute.parseAttributeName(table) + .map(part => quoteIdentifier(part)).mkString(".")))) } def this(tableIdent: Identifier) = { - this(s"Table ${tableIdent.quoted} already exists") + this(errorClass = "TABLE_OR_VIEW_ALREADY_EXISTS", + messageParameters = Map("relationName" -> tableIdent.quoted)) + } + + def this(table: Seq[String]) = { + this(errorClass = "TABLE_OR_VIEW_ALREADY_EXISTS", + messageParameters = Map("relationName" -> table.map(quoteIdentifier).mkString("."))) } } -class TempTableAlreadyExistsException(table: String) - extends TableAlreadyExistsException(s"Temporary view '$table' already exists") +class TempTableAlreadyExistsException(errorClass: String, messageParameters: Map[String, String], + cause: Option[Throwable] = None) + extends AnalysisException(errorClass, messageParameters, cause = cause) { + def this(table: String) = { + this(errorClass = "TEMP_TABLE_OR_VIEW_ALREADY_EXISTS", + messageParameters = Map("relationName" -> UnresolvedAttribute.parseAttributeName(table) + .map(part => quoteIdentifier(part)).mkString("."))) + } +} -class PartitionAlreadyExistsException(message: String) extends AnalysisException(message) { +class PartitionAlreadyExistsException(errorClass: String, messageParameters: Map[String, String]) + extends AnalysisException(errorClass, messageParameters) { def this(db: String, table: String, spec: TablePartitionSpec) = { - this(s"Partition already exists in table '$table' database '$db':\n" + spec.mkString("\n")) + this(errorClass = "PARTITIONS_ALREADY_EXIST", + Map("partitionList" -> ("PARTITION (" + + spec.map( kv => quoteIdentifier(kv._1) + s" = ${kv._2}").mkString(", ") + ")"), + "tableName" -> (quoteIdentifier(db) + "." + quoteIdentifier(table)))) } def this(tableName: String, partitionIdent: InternalRow, partitionSchema: StructType) = { - this(s"Partition already exists in table $tableName:" + - partitionIdent.toSeq(partitionSchema).zip(partitionSchema.map(_.name)) - .map( kv => s"${kv._1} -> ${kv._2}").mkString(",")) + this(errorClass = "PARTITIONS_ALREADY_EXIST", + Map("partitionList" -> + ("PARTITION (" + partitionIdent.toSeq(partitionSchema).zip(partitionSchema.map(_.name)) + .map( kv => quoteIdentifier(s"${kv._2}") + s" = ${kv._1}").mkString(", ") + ")"), + "tableName" -> UnresolvedAttribute.parseAttributeName(tableName) + .map(part => quoteIdentifier(part)).mkString("."))) } } -class PartitionsAlreadyExistException(message: String) extends AnalysisException(message) { +class PartitionsAlreadyExistException(errorClass: String, messageParameters: Map[String, String]) + extends AnalysisException(errorClass, messageParameters) { def this(db: String, table: String, specs: Seq[TablePartitionSpec]) = { - this(s"The following partitions already exists in table '$table' database '$db':\n" - + specs.mkString("\n===\n")) + this(errorClass = "PARTITIONS_ALREADY_EXIST", + Map("partitionList" -> + ("PARTITION (" + + specs.map(spec => spec.map(kv => quoteIdentifier(kv._1) + s" = ${kv._2}").mkString(", ")) + .mkString("), PARTITION (") + ")"), + "tableName" -> (quoteIdentifier(db) + "." + quoteIdentifier(table)))) } def this(tableName: String, partitionIdents: Seq[InternalRow], partitionSchema: StructType) = { - this(s"The following partitions already exists in table $tableName:" + - partitionIdents.map(_.toSeq(partitionSchema).zip(partitionSchema.map(_.name)) - .map( kv => s"${kv._1} -> ${kv._2}").mkString(",")).mkString("\n===\n")) + this(errorClass = "PARTITIONS_ALREADY_EXIST", + Map("partitionList" -> + ("PARTITION (" + + partitionIdents.map(_.toSeq(partitionSchema).zip(partitionSchema.map(_.name)) + .map( kv => quoteIdentifier(s"${kv._2}") + s" = ${kv._1}") + .mkString(", ")).mkString("), PARTITION (") + ")"), + "tableName" -> UnresolvedAttribute.parseAttributeName(tableName) + .map(part => quoteIdentifier(part)).mkString("."))) } } -class FunctionAlreadyExistsException(db: String, func: String) - extends AnalysisException(s"Function '$func' already exists in database '$db'") +class FunctionAlreadyExistsException(errorClass: String, messageParameters: Map[String, String]) + extends AnalysisException(errorClass, messageParameters) { + + def this(function: Seq[String]) = { + this (errorClass = "ROUTINE_ALREADY_EXISTS", + Map("routineName" -> function.map(quoteIdentifier).mkString("."))) + } +} class IndexAlreadyExistsException(message: String, cause: Option[Throwable] = None) extends AnalysisException(message, cause = cause) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 6fc9d756c998d..4a01b1382a7ed 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -80,7 +80,7 @@ object FakeV2SessionCatalog extends TableCatalog with FunctionCatalog { private def fail() = throw new UnsupportedOperationException override def listTables(namespace: Array[String]): Array[Identifier] = fail() override def loadTable(ident: Identifier): Table = { - throw new NoSuchTableException(ident.toString) + throw new NoSuchTableException(ident) } override def createTable( ident: Identifier, diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala index 68ed8991553e3..9a98a8a98d1b2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala @@ -126,10 +126,10 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog { "[BUG] logical plan should not have output of char/varchar type: " + leaf) case u: UnresolvedNamespace => - u.failAnalysis(s"Namespace not found: ${u.multipartIdentifier.quoted}") + u.schemaNotFound(u.multipartIdentifier) case u: UnresolvedTable => - u.failAnalysis(s"Table not found: ${u.multipartIdentifier.quoted}") + u.tableNotFound(u.multipartIdentifier) case u @ UnresolvedView(NonSessionCatalogAndIdentifier(catalog, ident), cmd, _, _) => u.failAnalysis( @@ -138,15 +138,13 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog { s"$cmd expects a view.") case u: UnresolvedView => - u.failAnalysis(s"View not found: ${u.multipartIdentifier.quoted}") + u.tableNotFound(u.multipartIdentifier) case u: UnresolvedTableOrView => - val viewStr = if (u.allowTempView) "view" else "permanent view" - u.failAnalysis( - s"Table or $viewStr not found: ${u.multipartIdentifier.quoted}") + u.tableNotFound(u.multipartIdentifier) case u: UnresolvedRelation => - u.failAnalysis(s"Table or view not found: ${u.multipartIdentifier.quoted}") + u.tableNotFound(u.multipartIdentifier) case u: UnresolvedFunc => throw QueryCompilationErrors.noSuchFunctionError( @@ -156,7 +154,7 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog { u.failAnalysis(s"Hint not found: ${u.name}") case InsertIntoStatement(u: UnresolvedRelation, _, _, _, _, _) => - u.failAnalysis(s"Table not found: ${u.multipartIdentifier.quoted}") + u.tableNotFound(u.multipartIdentifier) // TODO (SPARK-27484): handle streaming write commands when we have them. case write: V2WriteCommand if write.table.isInstanceOf[UnresolvedRelation] => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/NoSuchItemException.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/NoSuchItemException.scala index 805f3080c8472..8db227eadaf3c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/NoSuchItemException.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/NoSuchItemException.scala @@ -20,6 +20,7 @@ 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.catalyst.util.quoteIdentifier import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ import org.apache.spark.sql.connector.catalog.Identifier import org.apache.spark.sql.types.StructType @@ -30,83 +31,113 @@ import org.apache.spark.sql.types.StructType * as an [[org.apache.spark.sql.AnalysisException]] with the correct position information. */ case class NoSuchDatabaseException(db: String) - extends AnalysisException(s"Database '$db' not found") + extends AnalysisException(errorClass = "SCHEMA_NOT_FOUND", + messageParameters = Map("schemaName" -> quoteIdentifier(db))) -case class NoSuchNamespaceException( - override val message: String, - override val cause: Option[Throwable] = None) - extends AnalysisException(message, cause = cause) { +class NoSuchNamespaceException(errorClass: String, messageParameters: Map[String, String]) + extends AnalysisException(errorClass, messageParameters) { + def this(namespace: Seq[String]) = { + this(errorClass = "SCHEMA_NOT_FOUND", + Map("schemaName" -> namespace.map(part => quoteIdentifier(part)).mkString("."))) + } def this(namespace: Array[String]) = { - this(s"Namespace '${namespace.quoted}' not found") + this(errorClass = "SCHEMA_NOT_FOUND", + Map("schemaName" -> namespace.map(part => quoteIdentifier(part)).mkString("."))) } } -case class NoSuchTableException( - override val message: String, - override val cause: Option[Throwable] = None) - extends AnalysisException(message, cause = cause) { +class NoSuchTableException(errorClass: String, messageParameters: Map[String, String]) + extends AnalysisException(errorClass, messageParameters) { def this(db: String, table: String) = { - this(s"Table or view '$table' not found in database '$db'") + this(errorClass = "TABLE_OR_VIEW_NOT_FOUND", + messageParameters = Map("relationName" -> + (quoteIdentifier(db) + "." + quoteIdentifier(table)))) + } + + def this(name : Seq[String]) = { + this(errorClass = "TABLE_OR_VIEW_NOT_FOUND", + messageParameters = Map("relationName" -> + name.map(part => quoteIdentifier(part)).mkString("."))) } def this(tableIdent: Identifier) = { - this(s"Table ${tableIdent.quoted} not found") + this(errorClass = "TABLE_OR_VIEW_NOT_FOUND", + messageParameters = Map("relationName" -> tableIdent.quoted)) + } + + def this(table: String) = { + this(errorClass = "TABLE_OR_VIEW_NOT_FOUND", + messageParameters = Map("relationName" -> + UnresolvedAttribute.parseAttributeName(table) + .map(part => quoteIdentifier(part)).mkString("."))) } } -case class NoSuchPartitionException( - override val message: String) - extends AnalysisException(message) { +class NoSuchPartitionException(errorClass: String, messageParameters: Map[String, String]) + extends AnalysisException(errorClass, messageParameters) { def this(db: String, table: String, spec: TablePartitionSpec) = { - this(s"Partition not found in table '$table' database '$db':\n" + spec.mkString("\n")) + this(errorClass = "PARTITIONS_NOT_FOUND", + Map("partitionList" -> + ("PARTITION (" + + spec.map( kv => quoteIdentifier(kv._1) + s" = ${kv._2}").mkString(", ") + ")"), + "tableName" -> (quoteIdentifier(db) + "." + quoteIdentifier(table)))) } def this(tableName: String, partitionIdent: InternalRow, partitionSchema: StructType) = { - this(s"Partition not found in table $tableName: " - + partitionIdent.toSeq(partitionSchema).zip(partitionSchema.map(_.name)) - .map( kv => s"${kv._1} -> ${kv._2}").mkString(",")) + this(errorClass = "PARTITIONS_NOT_FOUND", + Map("partitionList" -> + ("PARTITION (" + partitionIdent.toSeq(partitionSchema).zip(partitionSchema.map(_.name)) + .map( kv => quoteIdentifier(s"${kv._2}") + s" = ${kv._1}").mkString(", ") + ")"), + "tableName" -> UnresolvedAttribute.parseAttributeName(tableName) + .map(part => quoteIdentifier(part)).mkString("."))) } } -case class NoSuchPermanentFunctionException(db: String, func: String) - extends AnalysisException(s"Function '$func' not found in database '$db'") +class NoSuchPermanentFunctionException(db: String, func: String) + extends AnalysisException(errorClass = "ROUTINE_NOT_FOUND", + Map("routineName" -> (quoteIdentifier(db) + "." + quoteIdentifier(func)))) -case class NoSuchFunctionException( - override val message: String, - override val cause: Option[Throwable]) - extends AnalysisException(message, cause = cause) { +class NoSuchFunctionException(errorClass: String, messageParameters: Map[String, String]) + extends AnalysisException(errorClass, messageParameters) { - def this(db: String, func: String, cause: Option[Throwable] = None) = { - this(s"Undefined function: '$func'. " + - s"This function is neither a registered temporary function nor " + - s"a permanent function registered in the database '$db'.", cause = cause) + def this(db: String, func: String) = { + this(errorClass = "ROUTINE_NOT_FOUND", + Map("routineName" -> (quoteIdentifier(db) + "." + quoteIdentifier(func)))) } def this(identifier: Identifier) = { - this(s"Undefined function: ${identifier.quoted}", cause = None) + this(errorClass = "ROUTINE_NOT_FOUND", Map("routineName" -> identifier.quoted)) } } -case class NoSuchPartitionsException(override val message: String) - extends AnalysisException(message) { +class NoSuchPartitionsException(errorClass: String, messageParameters: Map[String, String]) + extends AnalysisException(errorClass, messageParameters) { def this(db: String, table: String, specs: Seq[TablePartitionSpec]) = { - this(s"The following partitions not found in table '$table' database '$db':\n" - + specs.mkString("\n===\n")) + this(errorClass = "PARTITIONS_NOT_FOUND", + Map("partitionList" -> ("PARTITION (" + + specs.map(spec => spec.map(kv => quoteIdentifier(kv._1) + s" = ${kv._2}").mkString(", ")) + .mkString("), PARTITION (") + ")"), + "tableName" -> (quoteIdentifier(db) + "." + quoteIdentifier(table)))) } def this(tableName: String, partitionIdents: Seq[InternalRow], partitionSchema: StructType) = { - this(s"The following partitions not found in table $tableName: " - + partitionIdents.map(_.toSeq(partitionSchema).zip(partitionSchema.map(_.name)) - .map( kv => s"${kv._1} -> ${kv._2}").mkString(",")).mkString("\n===\n")) + this(errorClass = "PARTITIONS_NOT_FOUND", + Map("partitionList" -> ("PARTITION (" + + partitionIdents.map(_.toSeq(partitionSchema).zip(partitionSchema.map(_.name)) + .map( kv => quoteIdentifier(s"${kv._2}") + s" = ${kv._1}") + .mkString(", ")).mkString("), PARTITION (") + ")"), + "tableName" -> UnresolvedAttribute.parseAttributeName(tableName) + .map(part => quoteIdentifier(part)).mkString("."))) } } -case class NoSuchTempFunctionException(func: String) - extends AnalysisException(s"Temporary function '$func' not found") +class NoSuchTempFunctionException(func: String) + extends AnalysisException(errorClass = "ROUTINE_NOT_FOUND", Map("routineName" -> s"`$func`")) class NoSuchIndexException(message: String, cause: Option[Throwable] = None) - extends AnalysisException(message, cause = cause) + extends AnalysisException(errorClass = "INDEX_NOT_FOUND", + Map("message" -> message), cause) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/package.scala index 4e32e506c6881..01a48429fdab5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/package.scala @@ -21,6 +21,7 @@ import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.DataTypeMismatch import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.catalyst.trees.TreeNode +import org.apache.spark.sql.catalyst.util.quoteIdentifier import org.apache.spark.sql.errors.QueryErrorsBase /** @@ -64,6 +65,22 @@ package object analysis { messageParameters = mismatch.messageParameters + ("sqlExpr" -> toSQLExpr(expr)), origin = t.origin) } + + def tableNotFound(name: Seq[String]): Nothing = { + throw new AnalysisException( + errorClass = "TABLE_OR_VIEW_NOT_FOUND", + messageParameters = Map("relationName" -> + name.map(part => quoteIdentifier(part)).mkString(".")), + origin = t.origin) + } + + def schemaNotFound(name: Seq[String]): Nothing = { + throw new AnalysisException( + errorClass = "SCHEMA_NOT_FOUND", + messageParameters = Map("schemaName" -> + name.map(part => quoteIdentifier(part)).mkString(".")), + origin = t.origin) + } } /** Catches any AnalysisExceptions thrown by `f` and attaches `t`'s position if any. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalog.scala index 5643bf8b3a9b7..979613ae11266 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalog.scala @@ -57,7 +57,7 @@ trait ExternalCatalog { protected def requireFunctionNotExists(db: String, funcName: String): Unit = { if (functionExists(db, funcName)) { - throw new FunctionAlreadyExistsException(db = db, func = funcName) + throw new FunctionAlreadyExistsException(Seq(db, funcName)) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala index 5d5d8b202c533..2cd8806d485c0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala @@ -1394,7 +1394,7 @@ class SessionCatalog( if (!functionExists(qualifiedIdent)) { externalCatalog.createFunction(db, newFuncDefinition) } else if (!ignoreIfExists) { - throw new FunctionAlreadyExistsException(db = db, func = qualifiedIdent.funcName) + throw new FunctionAlreadyExistsException(Seq(db, qualifiedIdent.funcName)) } } @@ -1591,7 +1591,7 @@ class SessionCatalog( protected[sql] def failFunctionLookup( name: FunctionIdentifier, cause: Option[Throwable] = None): Nothing = { throw new NoSuchFunctionException( - db = name.database.getOrElse(getCurrentDatabase), func = name.funcName, cause) + db = name.database.getOrElse(getCurrentDatabase), func = name.funcName) } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/identifiers.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/identifiers.scala index 926514ac62d98..3be45292f62af 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/identifiers.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/identifiers.scala @@ -58,6 +58,16 @@ sealed trait CatalystIdentifier { } } + def parts: Seq[String] = { + if (catalog.isDefined && database.isDefined) { + Seq(catalog.get, database.get, identifier) + } else if (database.isDefined) { + Seq(database.get, identifier) + } else { + Seq(identifier) + } + } + override def toString: String = quotedString } 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 c1d8f0a4a8a51..f76c677accf0f 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 @@ -23,7 +23,7 @@ import org.apache.hadoop.fs.Path import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.{FunctionIdentifier, QualifiedTableName, TableIdentifier} -import org.apache.spark.sql.catalyst.analysis.{CannotReplaceMissingTableException, NamespaceAlreadyExistsException, NoSuchFunctionException, NoSuchNamespaceException, NoSuchPartitionException, NoSuchTableException, ResolvedTable, Star, TableAlreadyExistsException, UnresolvedRegex} +import org.apache.spark.sql.catalyst.analysis.{CannotReplaceMissingTableException, FunctionAlreadyExistsException, NamespaceAlreadyExistsException, NoSuchFunctionException, NoSuchNamespaceException, NoSuchPartitionException, NoSuchTableException, ResolvedTable, Star, TableAlreadyExistsException, UnresolvedRegex} import org.apache.spark.sql.catalyst.catalog.{CatalogTable, InvalidUDFClassException} import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeReference, AttributeSet, CreateMap, Expression, GroupingID, NamedExpression, SpecifiedWindowFrame, WindowFrame, WindowFunction, WindowSpecDefinition} @@ -648,13 +648,12 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase { } def renameTempViewToExistingViewError(oldName: String, newName: String): Throwable = { - new AnalysisException( - s"rename temporary view from '$oldName' to '$newName': destination view already exists") + new TableAlreadyExistsException(newName) } def cannotDropNonemptyDatabaseError(db: String): Throwable = { - new AnalysisException(s"Cannot drop a non-empty database: $db. " + - "Use CASCADE option to drop a non-empty database.") + new AnalysisException(errorClass = "SCHEMA_NOT_EMPTY", + Map("schemaName" -> toSQLId(db))) } def cannotDropNonemptyNamespaceError(namespace: Seq[String]): Throwable = { @@ -720,8 +719,7 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase { def cannotRenameTempViewToExistingTableError( oldName: TableIdentifier, newName: TableIdentifier): Throwable = { - new AnalysisException(s"RENAME TEMPORARY VIEW from '$oldName' to '$newName': " + - "destination table already exists") + new TableAlreadyExistsException(newName.parts) } def invalidPartitionSpecError(details: String): Throwable = { @@ -729,7 +727,7 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase { } def functionAlreadyExistsError(func: FunctionIdentifier): Throwable = { - new AnalysisException(s"Function $func already exists") + new FunctionAlreadyExistsException(func.parts) } def cannotLoadClassWhenRegisteringFunctionError( @@ -1714,7 +1712,7 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase { } def tableIdentifierExistsError(tableIdentifier: TableIdentifier): Throwable = { - new AnalysisException(s"$tableIdentifier already exists.") + new TableAlreadyExistsException(tableIdentifier.parts) } def tableIdentifierNotConvertedToHadoopFsRelationError( @@ -1853,8 +1851,8 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase { s"$name of data type: $dataType.") } - def tableAlreadyExistsError(table: String, guide: String = ""): Throwable = { - new AnalysisException(s"Table $table already exists." + guide) + def tableAlreadyExistsError(table: String): Throwable = { + new TableAlreadyExistsException(table) } def createTableAsSelectWithNonEmptyDirectoryError(tablePath: String): Throwable = { @@ -1865,7 +1863,7 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase { } def tableOrViewNotFoundError(table: String): Throwable = { - new AnalysisException(s"Table or view not found: $table") + new NoSuchTableException(table) } def noSuchFunctionError( @@ -2269,7 +2267,7 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase { } def tableOrViewNotFound(ident: Seq[String]): Throwable = { - new AnalysisException(s"Table or view '${ident.quoted}' not found") + new NoSuchTableException(ident.quoted) } def unexpectedTypeOfRelationError(relation: LogicalPlan, tableName: String): Throwable = { @@ -2331,7 +2329,7 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase { } def tableAlreadyExistsError(tableIdent: TableIdentifier): Throwable = { - new AnalysisException(s"Table $tableIdent already exists.") + new TableAlreadyExistsException(tableIdent.parts) } def cannotOverwriteTableThatIsBeingReadFromError(tableName: String): Throwable = { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala index 29c28f85e0ee0..409e6e452df64 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala @@ -128,7 +128,7 @@ class AnalysisErrorSuite extends AnalysisTest { messageParameters: Map[String, String]): Unit = { test(name) { assertAnalysisErrorClass(plan, errorClass, errorSubClass, messageParameters, - caseSensitive = true) + caseSensitive = true, line = -1, pos = -1) } } @@ -793,10 +793,12 @@ class AnalysisErrorSuite extends AnalysisTest { Project( Alias(Literal(1), "x")() :: Nil, UnresolvedRelation(TableIdentifier("t", Option("nonexist"))))))) - assertAnalysisError(plan, "Table or view not found:" :: Nil) + assertAnalysisErrorClass(plan, + expectedErrorClass = "TABLE_OR_VIEW_NOT_FOUND", + Map("relationName" -> "`nonexist`.`t`")) } - test("SPARK-33909: Check rand functions seed is legal at analyer side") { + test("SPARK-33909: Check rand functions seed is legal at analyzer side") { Seq(Rand("a".attr), Randn("a".attr)).foreach { r => val plan = Project(Seq(r.as("r")), testRelation) assertAnalysisError(plan, diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisExceptionPositionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisExceptionPositionSuite.scala index e50a58f8ce5fe..49caa0ff1ebcd 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisExceptionPositionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisExceptionPositionSuite.scala @@ -50,9 +50,12 @@ class AnalysisExceptionPositionSuite extends AnalysisTest { verifyTableOrViewPosition("REFRESH TABLE unknown", "unknown") verifyTableOrViewPosition("SHOW COLUMNS FROM unknown", "unknown") // Special case where namespace is prepended to the table name. - assertAnalysisError( + assertAnalysisErrorClass( parsePlan("SHOW COLUMNS FROM unknown IN db"), - Seq(s"Table or view not found: db.unknown; line 1 pos 18")) + "TABLE_OR_VIEW_NOT_FOUND", + Map("relationName" -> "`db`.`unknown`"), + line = 1, + pos = 18) verifyTableOrViewPosition("ALTER TABLE unknown RENAME TO t", "unknown") verifyTableOrViewPosition("ALTER VIEW unknown RENAME TO v", "unknown") } @@ -75,26 +78,29 @@ class AnalysisExceptionPositionSuite extends AnalysisTest { } private def verifyTablePosition(sql: String, table: String): Unit = { - verifyPosition(sql, table, "Table") + verifyPosition(sql, table) } private def verifyViewPosition(sql: String, table: String): Unit = { - verifyPosition(sql, table, "View") + verifyPosition(sql, table) } private def verifyTableOrViewPosition(sql: String, table: String): Unit = { - verifyPosition(sql, table, "Table or view") + verifyPosition(sql, table) } private def verifyTableOrPermanentViewPosition(sql: String, table: String): Unit = { - verifyPosition(sql, table, "Table or permanent view") + verifyPosition(sql, table) } - private def verifyPosition(sql: String, table: String, msgPrefix: String): Unit = { + private def verifyPosition(sql: String, table: String): Unit = { val expectedPos = sql.indexOf(table) assert(expectedPos != -1) - assertAnalysisError( + assertAnalysisErrorClass( parsePlan(sql), - Seq(s"$msgPrefix not found: $table; line 1 pos $expectedPos")) + "TABLE_OR_VIEW_NOT_FOUND", + Map("relationName" -> s"`$table`"), + line = 1, + pos = expectedPos) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala index e161a61cdf92a..7538ea4a8dcaa 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala @@ -106,7 +106,9 @@ class AnalysisSuite extends AnalysisTest with Matchers { "UNRESOLVED_COLUMN", "WITH_SUGGESTION", Map("objectName" -> "`tBl`.`a`", "proposal" -> "`TbL`.`a`"), - caseSensitive = true) + caseSensitive = true, + line = -1, + pos = -1) checkAnalysisWithoutViewWrapper( Project(Seq(UnresolvedAttribute("TbL.a")), @@ -716,7 +718,9 @@ class AnalysisSuite extends AnalysisTest with Matchers { "UNRESOLVED_COLUMN", "WITH_SUGGESTION", Map("objectName" -> "`y`", "proposal" -> "`t`.`x`"), - caseSensitive = true) + caseSensitive = true, + line = -1, + pos = -1) } test("CTE with non-matching column alias") { @@ -726,8 +730,8 @@ class AnalysisSuite extends AnalysisTest with Matchers { } test("SPARK-28251: Insert into non-existing table error message is user friendly") { - assertAnalysisError(parsePlan("INSERT INTO test VALUES (1)"), - Seq("Table not found: test")) + assertAnalysisErrorClass(parsePlan("INSERT INTO test VALUES (1)"), + "TABLE_OR_VIEW_NOT_FOUND", Map("relationName" -> "`test`")) } test("check CollectMetrics resolved") { @@ -1156,7 +1160,9 @@ class AnalysisSuite extends AnalysisTest with Matchers { "UNRESOLVED_COLUMN", "WITH_SUGGESTION", Map("objectName" -> "`c`.`y`", "proposal" -> "`x`"), - caseSensitive = true) + caseSensitive = true, + line = -1, + pos = -1) } test("SPARK-38118: Func(wrong_type) in the HAVING clause should throw data mismatch error") { @@ -1183,16 +1189,15 @@ class AnalysisSuite extends AnalysisTest with Matchers { } } - test("SPARK-39354: should be `Table or view not found`") { - assertAnalysisError(parsePlan( + test("SPARK-39354: should be [TABLE_OR_VIEW_NOT_FOUND]") { + assertAnalysisErrorClass(parsePlan( s""" |WITH t1 as (SELECT 1 user_id, CAST("2022-06-02" AS DATE) dt) |SELECT * |FROM t1 |JOIN t2 ON t1.user_id = t2.user_id |WHERE t1.dt >= DATE_SUB('2020-12-27', 90)""".stripMargin), - Seq(s"Table or view not found: t2"), - false) + "TABLE_OR_VIEW_NOT_FOUND", Map("relationName" -> "`t2`")) } test("SPARK-39144: nested subquery expressions deduplicate relations should be done bottom up") { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisTest.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisTest.scala index 8bb0588b536e6..0f4810d326d77 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisTest.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisTest.scala @@ -174,13 +174,17 @@ trait AnalysisTest extends PlanTest { inputPlan: LogicalPlan, expectedErrorClass: String, expectedMessageParameters: Map[String, String], - caseSensitive: Boolean = true): Unit = { + caseSensitive: Boolean = true, + line: Int = -1, + pos: Int = -1): Unit = { assertAnalysisErrorClass( inputPlan, expectedErrorClass, null, expectedMessageParameters, - caseSensitive) + caseSensitive, + line, + pos) } protected def assertAnalysisErrorClass( @@ -188,7 +192,9 @@ trait AnalysisTest extends PlanTest { expectedErrorClass: String, expectedErrorSubClass: String, expectedMessageParameters: Map[String, String], - caseSensitive: Boolean): Unit = { + caseSensitive: Boolean, + line: Int, + pos: Int ): Unit = { withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive.toString) { val analyzer = getAnalyzer val e = intercept[AnalysisException] { @@ -197,7 +203,9 @@ trait AnalysisTest extends PlanTest { if (e.getErrorClass != expectedErrorClass || !e.messageParameters.sameElements(expectedMessageParameters) || - e.getErrorSubClass != expectedErrorSubClass) { + e.getErrorSubClass != expectedErrorSubClass || + (line >= 0 && e.line.getOrElse(-1) != line) || + (pos >= 0) && e.startPosition.getOrElse(-1) != pos) { var failMsg = "" if (e.getErrorClass != expectedErrorClass) { failMsg += @@ -217,6 +225,12 @@ trait AnalysisTest extends PlanTest { |Actual message parameters: ${e.messageParameters.mkString("\n ")} """.stripMargin } + if (e.line.getOrElse(-1) != line || e.startPosition.getOrElse(-1) != pos) { + failMsg += + s"""Line/position should be: $line, $pos + |Actual line/position: ${e.line.getOrElse(-1)}, ${e.startPosition.getOrElse(-1)} + """.stripMargin + } fail(failMsg) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveSubquerySuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveSubquerySuite.scala index ab4d79593e7b6..3669c68dcb17b 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveSubquerySuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveSubquerySuite.scala @@ -136,7 +136,9 @@ class ResolveSubquerySuite extends AnalysisTest { "UNRESOLVED_COLUMN", "WITHOUT_SUGGESTION", Map("objectName" -> "`a`"), - caseSensitive = true) + caseSensitive = true, + line = -1, + pos = -1) } test("lateral subquery with unresolvable attributes") { @@ -146,28 +148,36 @@ class ResolveSubquerySuite extends AnalysisTest { "UNRESOLVED_COLUMN", "WITHOUT_SUGGESTION", Map("objectName" -> "`c`"), - caseSensitive = true) + caseSensitive = true, + line = -1, + pos = -1) // SELECT * FROM t1, LATERAL (SELECT a, b, c, d FROM t2) assertAnalysisErrorClass( lateralJoin(t1, t2.select($"a", $"b", $"c", $"d")), "UNRESOLVED_COLUMN", "WITH_SUGGESTION", Map("objectName" -> "`d`", "proposal" -> "`b`, `c`"), - caseSensitive = true) + caseSensitive = true, + line = -1, + pos = -1) // SELECT * FROM t1, LATERAL (SELECT * FROM t2, LATERAL (SELECT t1.a)) assertAnalysisErrorClass( lateralJoin(t1, lateralJoin(t2, t0.select($"t1.a"))), "UNRESOLVED_COLUMN", "WITHOUT_SUGGESTION", Map("objectName" -> "`t1`.`a`"), - caseSensitive = true) + caseSensitive = true, + line = -1, + pos = -1) // SELECT * FROM t1, LATERAL (SELECT * FROM t2, LATERAL (SELECT a, b)) assertAnalysisErrorClass( lateralJoin(t1, lateralJoin(t2, t0.select($"a", $"b"))), "UNRESOLVED_COLUMN", "WITHOUT_SUGGESTION", Map("objectName" -> "`a`"), - caseSensitive = true) + caseSensitive = true, + line = -1, + pos = -1) } test("lateral subquery with struct type") { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/V2WriteAnalysisSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/V2WriteAnalysisSuite.scala index d5c9660c77a80..2e0a7845325c4 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/V2WriteAnalysisSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/V2WriteAnalysisSuite.scala @@ -693,7 +693,9 @@ abstract class V2WriteAnalysisSuiteBase extends AnalysisTest { "UNRESOLVED_COLUMN", "WITH_SUGGESTION", Map("objectName" -> "`a`", "proposal" -> "`x`, `y`"), - caseSensitive = true) + caseSensitive = true, + line = -1, + pos = -1) val tableAcceptAnySchema = TestRelationAcceptAnySchema(StructType(Seq( StructField("x", DoubleType, nullable = false), @@ -707,7 +709,9 @@ abstract class V2WriteAnalysisSuiteBase extends AnalysisTest { "UNRESOLVED_COLUMN", "WITH_SUGGESTION", Map("objectName" -> "`a`", "proposal" -> "`x`, `y`"), - caseSensitive = true) + caseSensitive = true, + line = -1, + pos = -1) } test("SPARK-36498: reorder inner fields with byName mode") { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala index 62491e04831cc..5a702e2aeb18b 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala @@ -1467,9 +1467,11 @@ abstract class SessionCatalogSuite extends AnalysisTest with Eventually { val e = intercept[AnalysisException] { catalog.registerFunction( newFunc("temp1", None), overrideIfExists = false, functionBuilder = Some(tempFunc3)) - }.getMessage - assert(e.contains("Function temp1 already exists")) - // Temporary function is overridden + } + checkError(e, + errorClass = "ROUTINE_ALREADY_EXISTS", + parameters = Map("routineName" -> "`temp1`")) + // Temporary function is overridden catalog.registerFunction( newFunc("temp1", None), overrideIfExists = true, functionBuilder = Some(tempFunc3)) assert( diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/CatalogSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/CatalogSuite.scala index 54aad8b63ad55..c61b33dacdb36 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/CatalogSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/CatalogSuite.scala @@ -171,8 +171,9 @@ class CatalogSuite extends SparkFunSuite { catalog.loadTable(testIdent) } - assert(exc.message.contains(testIdent.quoted)) - assert(exc.message.contains("not found")) + checkError(exc, + errorClass = "TABLE_OR_VIEW_NOT_FOUND", + parameters = Map("relationName" -> testIdent.quoted)) } test("invalidateTable") { @@ -623,8 +624,9 @@ class CatalogSuite extends SparkFunSuite { catalog.alterTable(testIdent, TableChange.setProperty("prop", "val")) } - assert(exc.message.contains(testIdent.quoted)) - assert(exc.message.contains("not found")) + checkError(exc, + errorClass = "TABLE_OR_VIEW_NOT_FOUND", + parameters = Map("relationName" -> testIdent.quoted)) } test("dropTable") { @@ -680,8 +682,9 @@ class CatalogSuite extends SparkFunSuite { catalog.renameTable(testIdent, testIdentNew) } - assert(exc.message.contains(testIdent.quoted)) - assert(exc.message.contains("not found")) + checkError(exc, + errorClass = "TABLE_OR_VIEW_NOT_FOUND", + parameters = Map("relationName" -> testIdent.quoted)) } test("renameTable: fail if new table name already exists") { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/StagingInMemoryTableCatalog.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/StagingInMemoryTableCatalog.scala index 954650ae0eebd..013c6a87866e8 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/StagingInMemoryTableCatalog.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/StagingInMemoryTableCatalog.scala @@ -104,8 +104,7 @@ class StagingInMemoryTableCatalog extends InMemoryTableCatalog with StagingTable override def commitStagedChanges(): Unit = { val maybePreCommittedTable = tables.putIfAbsent(ident, delegateTable) if (maybePreCommittedTable != null) { - throw new TableAlreadyExistsException( - s"Table with identifier $ident and name $name was already created.") + throw new TableAlreadyExistsException(tableIdent = ident) } } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/SupportsAtomicPartitionManagementSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/SupportsAtomicPartitionManagementSuite.scala index df2fbd6d179bb..0590ca721cc8d 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/SupportsAtomicPartitionManagementSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/SupportsAtomicPartitionManagementSuite.scala @@ -163,10 +163,13 @@ class SupportsAtomicPartitionManagementSuite extends SparkFunSuite { assert(partTable.rows === InternalRow(2, "zyx", "5") :: Nil) // Truncate non-existing partition - val errMsg = intercept[NoSuchPartitionException] { + val e = intercept[NoSuchPartitionException] { partTable.truncatePartitions(Array(InternalRow("5"), InternalRow("6"))) - }.getMessage - assert(errMsg.contains("Partition not found in table test.ns.test_table: 6 -> dt")) + } + checkError(e, + errorClass = "PARTITIONS_NOT_FOUND", + parameters = Map("partitionList" -> "PARTITION (`dt` = 6)", + "tableName" -> "`test`.`ns`.`test_table`")) assert(partTable.rows === InternalRow(2, "zyx", "5") :: Nil) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/SupportsPartitionManagementSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/SupportsPartitionManagementSuite.scala index e5aeb90b841a6..fe7618c8a5675 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/SupportsPartitionManagementSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/SupportsPartitionManagementSuite.scala @@ -218,16 +218,22 @@ class SupportsPartitionManagementSuite extends SparkFunSuite { test("renamePartition") { val partTable = createMultiPartTable() - val errMsg1 = intercept[PartitionAlreadyExistsException] { + val e = intercept[PartitionAlreadyExistsException] { partTable.renamePartition(InternalRow(0, "abc"), InternalRow(1, "abc")) - }.getMessage - assert(errMsg1.contains("Partition already exists")) + } + checkError(e, + errorClass = "PARTITIONS_ALREADY_EXIST", + parameters = Map("partitionList" -> "PARTITION (`part0` = 1, `part1` = abc)", + "tableName" -> "`test`.`ns`.`test_table`")) val newPart = InternalRow(2, "xyz") - val errMsg2 = intercept[NoSuchPartitionException] { + val e2 = intercept[NoSuchPartitionException] { partTable.renamePartition(newPart, InternalRow(3, "abc")) - }.getMessage - assert(errMsg2.contains("Partition not found")) + } + checkError(e2, + errorClass = "PARTITIONS_NOT_FOUND", + parameters = Map("partitionList" -> "PARTITION (`part0` = 2, `part1` = xyz)", + "tableName" -> "`test`.`ns`.`test_table`")) assert(partTable.renamePartition(InternalRow(0, "abc"), newPart)) assert(partTable.partitionExists(newPart)) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala index 6cc356488393c..95e1a159ef84f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala @@ -173,8 +173,7 @@ case class CreateDataSourceTableAsSelectCommand( s"Expect the table $tableName has been dropped when the save mode is Overwrite") if (mode == SaveMode.ErrorIfExists) { - throw QueryCompilationErrors.tableAlreadyExistsError( - tableName, " You need to drop it first.") + throw QueryCompilationErrors.tableAlreadyExistsError(tableName) } if (mode == SaveMode.Ignore) { // Since the table already exists and the save mode is Ignore, we will just return. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTableCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTableCatalog.scala index 386164d63d5bd..47322669c3a12 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTableCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTableCatalog.scala @@ -158,7 +158,7 @@ class JDBCTableCatalog extends TableCatalog val writeOptions = new JdbcOptionsInWrite(tableOptions) val caseSensitive = SQLConf.get.caseSensitiveAnalysis JdbcUtils.withConnection(options) { conn => - JdbcUtils.classifyException(s"Failed table creation: $ident", dialect) { + JdbcUtils.classifyException(ident.quoted, dialect) { JdbcUtils.createTable(conn, getTableName(ident), schema, caseSensitive, writeOptions) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/H2Dialect.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/H2Dialect.scala index 7665bb91c6ee4..47b1d75088c27 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/H2Dialect.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/H2Dialect.scala @@ -28,7 +28,7 @@ import scala.util.control.NonFatal import org.apache.commons.lang3.StringUtils import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.catalyst.analysis.{IndexAlreadyExistsException, NoSuchIndexException, NoSuchNamespaceException, NoSuchTableException, TableAlreadyExistsException} +import org.apache.spark.sql.catalyst.analysis.{IndexAlreadyExistsException, NoSuchIndexException, NoSuchNamespaceException, NoSuchTableException, TableAlreadyExistsException, UnresolvedAttribute} import org.apache.spark.sql.connector.catalog.Identifier import org.apache.spark.sql.connector.catalog.functions.UnboundFunction import org.apache.spark.sql.connector.catalog.index.TableIndex @@ -186,13 +186,23 @@ private[sql] object H2Dialect extends JdbcDialect { exception.getErrorCode match { // TABLE_OR_VIEW_ALREADY_EXISTS_1 case 42101 => - throw new TableAlreadyExistsException(message, cause = Some(e)) + val quotedName = UnresolvedAttribute.parseAttributeName(message) + .map(part => quoteIdentifier(part)).mkString(".") + throw new TableAlreadyExistsException(errorClass = "TABLE_OR_VIEW_ALREADY_EXISTS", + messageParameters = Map("relationName" -> quotedName), + cause = Some(e)) // TABLE_OR_VIEW_NOT_FOUND_1 case 42102 => - throw NoSuchTableException(message, cause = Some(e)) + val quotedName = UnresolvedAttribute.parseAttributeName(message) + .map(part => quoteIdentifier(part)).mkString(".") + throw new NoSuchTableException(errorClass = "TABLE_OR_VIEW_NOT_FOUND", + messageParameters = Map("relationName" -> quotedName)) // SCHEMA_NOT_FOUND_1 case 90079 => - throw NoSuchNamespaceException(message, cause = Some(e)) + val quotedName = UnresolvedAttribute.parseAttributeName(message) + .map(part => quoteIdentifier(part)).mkString(".") + throw new NoSuchNamespaceException(errorClass = "SCHEMA_NOT_FOUND", + messageParameters = Map("schemaName" -> quotedName)) // INDEX_ALREADY_EXISTS_1 case 42111 => throw new IndexAlreadyExistsException(message, cause = Some(e)) diff --git a/sql/core/src/test/resources/sql-tests/results/cte-legacy.sql.out b/sql/core/src/test/resources/sql-tests/results/cte-legacy.sql.out index b72da5e739e2c..013c5f27b502f 100644 --- a/sql/core/src/test/resources/sql-tests/results/cte-legacy.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/cte-legacy.sql.out @@ -232,7 +232,20 @@ SELECT * FROM t2 struct<> -- !query output org.apache.spark.sql.AnalysisException -Table or view not found: t1; line 5 pos 20 +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42000", + "messageParameters" : { + "relationName" : "`t1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 73, + "stopIndex" : 74, + "fragment" : "t1" + } ] +} -- !query @@ -270,7 +283,20 @@ SELECT * FROM ( struct<> -- !query output org.apache.spark.sql.AnalysisException -Table or view not found: cte_outer; line 8 pos 22 +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42000", + "messageParameters" : { + "relationName" : "`cte_outer`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 146, + "stopIndex" : 154, + "fragment" : "cte_outer" + } ] +} -- !query @@ -290,7 +316,20 @@ SELECT * FROM ( struct<> -- !query output org.apache.spark.sql.AnalysisException -Table or view not found: cte_invisible_inner; line 9 pos 18 +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42000", + "messageParameters" : { + "relationName" : "`cte_invisible_inner`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 164, + "stopIndex" : 182, + "fragment" : "cte_invisible_inner" + } ] +} -- !query @@ -312,4 +351,17 @@ SELECT * FROM ( struct<> -- !query output org.apache.spark.sql.AnalysisException -Table or view not found: cte_invisible_inner; line 11 pos 18 +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42000", + "messageParameters" : { + "relationName" : "`cte_invisible_inner`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 194, + "stopIndex" : 212, + "fragment" : "cte_invisible_inner" + } ] +} diff --git a/sql/core/src/test/resources/sql-tests/results/cte-nested.sql.out b/sql/core/src/test/resources/sql-tests/results/cte-nested.sql.out index e6382d7430938..de27d5fec81ee 100644 --- a/sql/core/src/test/resources/sql-tests/results/cte-nested.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/cte-nested.sql.out @@ -296,7 +296,20 @@ SELECT * FROM ( struct<> -- !query output org.apache.spark.sql.AnalysisException -Table or view not found: cte_invisible_inner; line 9 pos 18 +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42000", + "messageParameters" : { + "relationName" : "`cte_invisible_inner`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 164, + "stopIndex" : 182, + "fragment" : "cte_invisible_inner" + } ] +} -- !query @@ -318,4 +331,17 @@ SELECT * FROM ( struct<> -- !query output org.apache.spark.sql.AnalysisException -Table or view not found: cte_invisible_inner; line 11 pos 18 +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42000", + "messageParameters" : { + "relationName" : "`cte_invisible_inner`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 194, + "stopIndex" : 212, + "fragment" : "cte_invisible_inner" + } ] +} diff --git a/sql/core/src/test/resources/sql-tests/results/cte-nonlegacy.sql.out b/sql/core/src/test/resources/sql-tests/results/cte-nonlegacy.sql.out index 26665fd33e8c2..6a48e1bec438d 100644 --- a/sql/core/src/test/resources/sql-tests/results/cte-nonlegacy.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/cte-nonlegacy.sql.out @@ -288,7 +288,20 @@ SELECT * FROM ( struct<> -- !query output org.apache.spark.sql.AnalysisException -Table or view not found: cte_invisible_inner; line 9 pos 18 +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42000", + "messageParameters" : { + "relationName" : "`cte_invisible_inner`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 164, + "stopIndex" : 182, + "fragment" : "cte_invisible_inner" + } ] +} -- !query @@ -310,4 +323,17 @@ SELECT * FROM ( struct<> -- !query output org.apache.spark.sql.AnalysisException -Table or view not found: cte_invisible_inner; line 11 pos 18 +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42000", + "messageParameters" : { + "relationName" : "`cte_invisible_inner`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 194, + "stopIndex" : 212, + "fragment" : "cte_invisible_inner" + } ] +} diff --git a/sql/core/src/test/resources/sql-tests/results/cte.sql.out b/sql/core/src/test/resources/sql-tests/results/cte.sql.out index f88fd884ab782..ffa691e636ede 100644 --- a/sql/core/src/test/resources/sql-tests/results/cte.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/cte.sql.out @@ -21,7 +21,20 @@ WITH s AS (SELECT 1 FROM s) SELECT * FROM s struct<> -- !query output org.apache.spark.sql.AnalysisException -Table or view not found: s; line 1 pos 25 +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42000", + "messageParameters" : { + "relationName" : "`s`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 26, + "stopIndex" : 26, + "fragment" : "s" + } ] +} -- !query @@ -31,7 +44,20 @@ SELECT * FROM r struct<> -- !query output org.apache.spark.sql.AnalysisException -Table or view not found: r; line 1 pos 33 +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42000", + "messageParameters" : { + "relationName" : "`r`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 34, + "stopIndex" : 34, + "fragment" : "r" + } ] +} -- !query @@ -50,7 +76,20 @@ WITH s1 AS (SELECT 1 FROM s2), s2 AS (SELECT 1 FROM s1) SELECT * FROM s1, s2 struct<> -- !query output org.apache.spark.sql.AnalysisException -Table or view not found: s2; line 1 pos 26 +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42000", + "messageParameters" : { + "relationName" : "`s2`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 27, + "stopIndex" : 28, + "fragment" : "s2" + } ] +} -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/describe.sql.out b/sql/core/src/test/resources/sql-tests/results/describe.sql.out index 5bb86a398c82a..922d37d33207d 100644 --- a/sql/core/src/test/resources/sql-tests/results/describe.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/describe.sql.out @@ -362,9 +362,14 @@ DESC t PARTITION (c='Us', d=2) struct<> -- !query output org.apache.spark.sql.catalyst.analysis.NoSuchPartitionException -Partition not found in table 't' database 'default': -c -> Us -d -> 2 +{ + "errorClass" : "PARTITIONS_NOT_FOUND", + "sqlState" : "42000", + "messageParameters" : { + "partitionList" : "PARTITION (`c` = Us, `d` = 2)", + "tableName" : "`default`.`t`" + } +} -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/create_view.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/create_view.sql.out index fcdd42551d1d3..fb487cb2d5947 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/create_view.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/create_view.sql.out @@ -589,7 +589,20 @@ CREATE VIEW v10_temp AS SELECT * FROM v7_temp struct<> -- !query output org.apache.spark.sql.AnalysisException -Table or view not found: v7_temp; line 1 pos 38 +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42000", + "messageParameters" : { + "relationName" : "`v7_temp`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 39, + "stopIndex" : 45, + "fragment" : "v7_temp" + } ] +} -- !query @@ -598,7 +611,20 @@ CREATE VIEW v11_temp AS SELECT t1.id, t2.a FROM base_table t1, v10_temp t2 struct<> -- !query output org.apache.spark.sql.AnalysisException -Table or view not found: v10_temp; line 1 pos 63 +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42000", + "messageParameters" : { + "relationName" : "`v10_temp`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 64, + "stopIndex" : 74, + "fragment" : "v10_temp t2" + } ] +} -- !query @@ -607,7 +633,20 @@ CREATE VIEW v12_temp AS SELECT true FROM v11_temp struct<> -- !query output org.apache.spark.sql.AnalysisException -Table or view not found: v11_temp; line 1 pos 41 +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42000", + "messageParameters" : { + "relationName" : "`v11_temp`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 42, + "stopIndex" : 49, + "fragment" : "v11_temp" + } ] +} -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/with.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/with.sql.out index 0a4e2f179d8c2..56929023279cf 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/with.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/with.sql.out @@ -216,7 +216,20 @@ SELECT * FROM outermost ORDER BY 1 struct<> -- !query output org.apache.spark.sql.AnalysisException -Table or view not found: outermost; line 4 pos 23 +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42000", + "messageParameters" : { + "relationName" : "`outermost`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 96, + "stopIndex" : 104, + "fragment" : "outermost" + } ] +} -- !query @@ -382,7 +395,20 @@ WITH test AS (SELECT 42) INSERT INTO test VALUES (1) struct<> -- !query output org.apache.spark.sql.AnalysisException -Table not found: test; line 1 pos 37 +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42000", + "messageParameters" : { + "relationName" : "`test`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 38, + "stopIndex" : 41, + "fragment" : "test" + } ] +} -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/show-tables.sql.out b/sql/core/src/test/resources/sql-tests/results/show-tables.sql.out index 181d5854badb4..7e0d98e60f9e5 100644 --- a/sql/core/src/test/resources/sql-tests/results/show-tables.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/show-tables.sql.out @@ -209,7 +209,13 @@ SHOW TABLE EXTENDED LIKE 'show_t*' PARTITION(c='Us', d=1) struct<> -- !query output org.apache.spark.sql.catalyst.analysis.NoSuchTableException -Table or view 'show_t*' not found in database 'showdb' +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42000", + "messageParameters" : { + "relationName" : "`showdb`.`show_t*`" + } +} -- !query @@ -236,9 +242,14 @@ SHOW TABLE EXTENDED LIKE 'show_t1' PARTITION(c='Ch', d=1) struct<> -- !query output org.apache.spark.sql.catalyst.analysis.NoSuchPartitionException -Partition not found in table 'show_t1' database 'showdb': -c -> Ch -d -> 1 +{ + "errorClass" : "PARTITIONS_NOT_FOUND", + "sqlState" : "42000", + "messageParameters" : { + "partitionList" : "PARTITION (`c` = Ch, `d` = 1)", + "tableName" : "`showdb`.`show_t1`" + } +} -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/show-views.sql.out b/sql/core/src/test/resources/sql-tests/results/show-views.sql.out index 8dc2f663ba067..81e41673adea2 100644 --- a/sql/core/src/test/resources/sql-tests/results/show-views.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/show-views.sql.out @@ -139,7 +139,13 @@ SHOW VIEWS IN wrongdb LIKE 'view_*' struct<> -- !query output org.apache.spark.sql.catalyst.analysis.NoSuchDatabaseException -Database 'wrongdb' not found +{ + "errorClass" : "SCHEMA_NOT_FOUND", + "sqlState" : "42000", + "messageParameters" : { + "schemaName" : "`wrongdb`" + } +} -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/show_columns.sql.out b/sql/core/src/test/resources/sql-tests/results/show_columns.sql.out index 8a858343b240e..4c79ab3a54c41 100644 --- a/sql/core/src/test/resources/sql-tests/results/show_columns.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/show_columns.sql.out @@ -91,7 +91,20 @@ SHOW COLUMNS IN badtable FROM showdb struct<> -- !query output org.apache.spark.sql.AnalysisException -Table or view not found: showdb.badtable; line 1 pos 16 +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42000", + "messageParameters" : { + "relationName" : "`showdb`.`badtable`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 24, + "fragment" : "badtable" + } ] +} -- !query @@ -127,7 +140,20 @@ SHOW COLUMNS IN showdb.showcolumn3 struct<> -- !query output org.apache.spark.sql.AnalysisException -Table or view not found: showdb.showcolumn3; line 1 pos 16 +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42000", + "messageParameters" : { + "relationName" : "`showdb`.`showcolumn3`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 34, + "fragment" : "showdb.showcolumn3" + } ] +} -- !query @@ -136,7 +162,20 @@ SHOW COLUMNS IN showcolumn3 FROM showdb struct<> -- !query output org.apache.spark.sql.AnalysisException -Table or view not found: showdb.showcolumn3; line 1 pos 16 +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42000", + "messageParameters" : { + "relationName" : "`showdb`.`showcolumn3`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 27, + "fragment" : "showcolumn3" + } ] +} -- !query @@ -145,7 +184,20 @@ SHOW COLUMNS IN showcolumn4 struct<> -- !query output org.apache.spark.sql.AnalysisException -Table or view not found: showcolumn4; line 1 pos 16 +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42000", + "messageParameters" : { + "relationName" : "`showcolumn4`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 27, + "fragment" : "showcolumn4" + } ] +} -- !query diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CTEInlineSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CTEInlineSuite.scala index 7801c8e644d1f..f196a338a3451 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CTEInlineSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CTEInlineSuite.scala @@ -275,7 +275,9 @@ abstract class CTEInlineSuiteBase | select * from v2 where c1 > 0 union select * from v2 where c2 > 0 |) """.stripMargin)) - assert(ex.message.contains("Table or view not found: v1")) + checkError(ex, + errorClass = "TABLE_OR_VIEW_NOT_FOUND", + parameters = Map("relationName" -> "`v`")) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala index 4de409f56d043..f444e57592ae0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala @@ -153,7 +153,9 @@ class CachedTableSuite extends QueryTest with SQLTestUtils val e = intercept[TempTableAlreadyExistsException] { sql("CACHE TABLE tempView AS SELECT 1") } - assert(e.getMessage.contains("Temporary view 'tempView' already exists")) + checkError(e, + errorClass = "TEMP_TABLE_OR_VIEW_ALREADY_EXISTS", + parameters = Map("relationName" -> "`tempView`")) } } @@ -962,7 +964,9 @@ class CachedTableSuite extends QueryTest with SQLTestUtils if (!storeAnalyzed) { // t2 should become invalid after t1 is dropped val e = intercept[AnalysisException](spark.catalog.isCached("t2")) - assert(e.message.contains(s"Table or view not found")) + checkError(e, + errorClass = "TABLE_OR_VIEW_NOT_FOUND", + parameters = Map("relationName" -> "`t1`")) } } } @@ -993,7 +997,9 @@ class CachedTableSuite extends QueryTest with SQLTestUtils if (!storeAnalyzed) { // t2 should become invalid after t1 is dropped val e = intercept[AnalysisException](spark.catalog.isCached("t2")) - assert(e.message.contains(s"Table or view not found")) + checkError(e, + errorClass = "TABLE_OR_VIEW_NOT_FOUND", + parameters = Map("relationName" -> "`t1`")) } } } @@ -1431,7 +1437,9 @@ class CachedTableSuite extends QueryTest with SQLTestUtils checkAnswer(sql("SELECT * FROM v"), Row(1) :: Nil) sql(s"DROP TABLE $t") val e = intercept[AnalysisException](sql("SELECT * FROM v")) - assert(e.message.contains(s"Table or view not found: $t")) + checkError(e, + errorClass = "TABLE_OR_VIEW_NOT_FOUND", + parameters = Map("relationName" -> s"`$t`")) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala index 1dcf0b6e4a99b..addd25051e5ba 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala @@ -2809,7 +2809,9 @@ class DataFrameSuite extends QueryTest val e = intercept[AnalysisException] { sql("WITH t AS (SELECT 1 FROM nonexist.t) SELECT * FROM t") } - assert(e.getMessage.contains("Table or view not found:")) + checkError(e, + errorClass = "TABLE_OR_VIEW_NOT_FOUND", + parameters = Map("relation_name" -> "`nonexist`.`t`")) } test("SPARK-32680: Don't analyze CTAS with unresolved query") { @@ -2817,7 +2819,9 @@ class DataFrameSuite extends QueryTest val e = intercept[AnalysisException] { sql(s"CREATE TABLE t USING $v2Source AS SELECT * from nonexist") } - assert(e.getMessage.contains("Table or view not found:")) + checkError(e, + errorClass = "TABLE_OR_VIEW_NOT_FOUND", + parameters = Map("relation_name" -> "`nonexist`")) } test("CalendarInterval reflection support") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWriterV2Suite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWriterV2Suite.scala index 17d99a00fb094..a5122125b47bc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWriterV2Suite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWriterV2Suite.scala @@ -157,7 +157,9 @@ class DataFrameWriterV2Suite extends QueryTest with SharedSparkSession with Befo spark.table("source").writeTo("testcat.table_name").append() } - assert(exc.getMessage.contains("Table or view not found: testcat.table_name")) + checkError(exc, + errorClass = "TABLE_OR_VIEW_NOT_FOUND", + parameters = Map("relationName" -> "`testcat`.`table_name`")) } test("Append: fail if it writes to a temp view that is not v2 relation") { @@ -261,7 +263,9 @@ class DataFrameWriterV2Suite extends QueryTest with SharedSparkSession with Befo spark.table("source").writeTo("testcat.table_name").overwrite(lit(true)) } - assert(exc.getMessage.contains("Table or view not found: testcat.table_name")) + checkError(exc, + errorClass = "TABLE_OR_VIEW_NOT_FOUND", + parameters = Map("relationName" -> "`testcat`.`table_name`")) } test("Overwrite: fail if it writes to a temp view that is not v2 relation") { @@ -365,7 +369,9 @@ class DataFrameWriterV2Suite extends QueryTest with SharedSparkSession with Befo spark.table("source").writeTo("testcat.table_name").overwritePartitions() } - assert(exc.getMessage.contains("Table or view not found: testcat.table_name")) + checkError(exc, + errorClass = "TABLE_OR_VIEW_NOT_FOUND", + parameters = Map("relationName" -> "`testcat`.`table_name`")) } test("OverwritePartitions: fail if it writes to a temp view that is not v2 relation") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index 215b005319e29..5b42435a7c09f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -1616,12 +1616,16 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark var e = intercept[AnalysisException] { sql("select * from in_valid_table") } - assert(e.message.contains("Table or view not found")) + checkError(e, + errorClass = "TABLE_OR_VIEW_NOT_FOUND", + parameters = Map("relationName" -> "`in_valid_table`")) e = intercept[AnalysisException] { sql("select * from no_db.no_table").show() } - assert(e.message.contains("Table or view not found")) + checkError(e, + errorClass = "TABLE_OR_VIEW_NOT_FOUND", + parameters = Map("relationName" -> "`no_db`.`no_table`")) e = intercept[AnalysisException] { sql("select * from json.invalid_file") @@ -1636,8 +1640,10 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark e = intercept[AnalysisException] { sql(s"select id from `org.apache.spark.sql.sources.HadoopFsRelationProvider`.`file_path`") } - assert(e.message.contains("Table or view not found: " + - "`org.apache.spark.sql.sources.HadoopFsRelationProvider`.file_path")) + checkError(e, + errorClass = "TABLE_OR_VIEW_NOT_FOUND", + parameters = Map("relationName" -> + "`org.apache.spark.sql.sources.HadoopFsRelationProvider`.`file_path`")) e = intercept[AnalysisException] { sql(s"select id from `Jdbc`.`file_path`") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/StatisticsCollectionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/StatisticsCollectionSuite.scala index 25ae356bf784c..64f2558b5c1f2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/StatisticsCollectionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/StatisticsCollectionSuite.scala @@ -597,11 +597,12 @@ class StatisticsCollectionSuite extends StatisticsCollectionTestBase with Shared test("analyzes column statistics in cached global temporary view") { withGlobalTempView("gTempView") { val globalTempDB = spark.sharedState.globalTempViewManager.database - val errMsg1 = intercept[AnalysisException] { + val e1 = intercept[AnalysisException] { sql(s"ANALYZE TABLE $globalTempDB.gTempView COMPUTE STATISTICS FOR COLUMNS id") - }.getMessage - assert(errMsg1.contains("Table or view not found: " + - s"$globalTempDB.gTempView")) + } + checkError(e1, + errorClass = "TABLE_OR_VIEW_NOT_FOUND", + parameters = Map("relationName" -> s"`$globalTempDB`.`gTempView`")) // Analyzes in a global temporary view sql("CREATE GLOBAL TEMP VIEW gTempView AS SELECT 1 id") val errMsg2 = intercept[AnalysisException] { @@ -793,9 +794,11 @@ class StatisticsCollectionSuite extends StatisticsCollectionTestBase with Shared } } - val errMsg = intercept[AnalysisException] { + val e = intercept[AnalysisException] { sql(s"ANALYZE TABLES IN db_not_exists COMPUTE STATISTICS") - }.getMessage - assert(errMsg.contains("Database 'db_not_exists' not found")) + } + checkError(e, + errorClass = "SCHEMA_NOT_FOUND", + parameters = Map("schemaName" -> "`db_not_exists`")) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/AlterTableTests.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/AlterTableTests.scala index ee707acf5a019..9f1d8a09e11fa 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/AlterTableTests.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/AlterTableTests.scala @@ -21,6 +21,8 @@ import scala.collection.JavaConverters._ import org.apache.spark.SparkException import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute +import org.apache.spark.sql.catalyst.util.quoteIdentifier import org.apache.spark.sql.connector.catalog.CatalogV2Util.withDefaultOwnership import org.apache.spark.sql.connector.catalog.Table import org.apache.spark.sql.internal.SQLConf @@ -45,14 +47,17 @@ trait AlterTableTests extends SharedSparkSession { test("AlterTable: table does not exist") { val t2 = s"${catalogAndNamespace}fake_table" + val quotedT2 = UnresolvedAttribute.parseAttributeName(t2) + .map(part => quoteIdentifier(part)).mkString(".") withTable(t2) { sql(s"CREATE TABLE $t2 (id int) USING $v2Format") val exc = intercept[AnalysisException] { sql(s"ALTER TABLE ${catalogAndNamespace}table_name DROP COLUMN id") } - assert(exc.getMessage.contains(s"${catalogAndNamespace}table_name")) - assert(exc.getMessage.contains("Table not found")) + checkError(exc, + errorClass = "TABLE_OR_VIEW_NOT_FOUND", + parameters = Map("relationName" -> quotedT2)) } } 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 11f4fe0649be4..00a28f406eb43 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 @@ -1206,7 +1206,9 @@ class DataSourceV2SQLSuiteV1Filter extends DataSourceV2SQLSuite with AlterTableT val exception = intercept[NoSuchDatabaseException] { sql("USE ns1") } - assert(exception.getMessage.contains("Database 'ns1' not found")) + checkError(exception, + errorClass = "SCHEMA_NOT_FOUND", + parameters = Map("schemaName" -> "`ns`")) } test("SPARK-31100: Use: v2 catalog that implements SupportsNamespaces is used " + @@ -1215,7 +1217,9 @@ class DataSourceV2SQLSuiteV1Filter extends DataSourceV2SQLSuite with AlterTableT val exception = intercept[NoSuchNamespaceException] { sql("USE testcat.ns1.ns2") } - assert(exception.getMessage.contains("Namespace 'ns1.ns2' not found")) + checkError(exception, + errorClass = "SCHEMA_NOT_FOUND", + parameters = Map("schemaName" -> "`ns1`.`ns2`")) } test("SPARK-31100: Use: v2 catalog that does not implement SupportsNameSpaces is used " + @@ -1571,9 +1575,11 @@ class DataSourceV2SQLSuiteV1Filter extends DataSourceV2SQLSuite with AlterTableT """.stripMargin) // UPDATE non-existing table - assertAnalysisError( + assertAnalysisErrorClass( "UPDATE dummy SET name='abc'", - "Table or view not found") + expectedErrorClass = "TABLE_OR_VIEW_NOT_FOUND", + expectedErrorSubClass = "", + expectedErrorMessageParameters = Map("relationName" -> "`dummy`")) // UPDATE non-existing column assertAnalysisErrorClass( @@ -1620,7 +1626,7 @@ class DataSourceV2SQLSuiteV1Filter extends DataSourceV2SQLSuite with AlterTableT """.stripMargin) // MERGE INTO non-existing table - assertAnalysisError( + assertAnalysisErrorClass( s""" |MERGE INTO testcat.ns1.ns2.dummy AS target |USING testcat.ns1.ns2.source AS source @@ -1630,10 +1636,12 @@ class DataSourceV2SQLSuiteV1Filter extends DataSourceV2SQLSuite with AlterTableT |WHEN NOT MATCHED AND (target.col2='insert') |THEN INSERT * """.stripMargin, - "Table or view not found") + expectedErrorClass = "TABLE_OR_VIEW_NOT_FOUND", + expectedErrorSubClass = "", + expectedErrorMessageParameters = Map("relationName" -> "`testcat`.`ns1`.`ns2`.`dummy`")) // USING non-existing table - assertAnalysisError( + assertAnalysisErrorClass( s""" |MERGE INTO testcat.ns1.ns2.target AS target |USING testcat.ns1.ns2.dummy AS source @@ -1643,7 +1651,10 @@ class DataSourceV2SQLSuiteV1Filter extends DataSourceV2SQLSuite with AlterTableT |WHEN NOT MATCHED AND (target.col2='insert') |THEN INSERT * """.stripMargin, - "Table or view not found") + expectedErrorClass = "TABLE_OR_VIEW_NOT_FOUND", + expectedErrorSubClass = "", + expectedErrorMessageParameters = Map("relationName" -> "`testcat`.`ns1`.`ns2`.`dummy`")) + // UPDATE non-existing column assertAnalysisError( @@ -1704,8 +1715,9 @@ class DataSourceV2SQLSuiteV1Filter extends DataSourceV2SQLSuite with AlterTableT val e = intercept[AnalysisException] { sql(s"ALTER VIEW testcat.ns.tbl RENAME TO ns.view") } - assert(e.getMessage.contains( - "Table or view not found: testcat.ns.tbl")) + checkError(e, + errorClass = "TABLE_OR_VIEW_NOT_FOUND", + parameters = Map("relationName" -> "`testcat`.`ns`.`tbl`")) } test("ANALYZE TABLE") { @@ -1762,7 +1774,9 @@ class DataSourceV2SQLSuiteV1Filter extends DataSourceV2SQLSuite with AlterTableT val e = intercept[AnalysisException] { sql(s"UNCACHE TABLE $t") } - assert(e.message.contains("Table or view not found: testcat.ns1.ns2.tbl")) + checkError(e, + errorClass = "TABLE_OR_VIEW_NOT_FOUND", + parameters = Map("relationName" -> "`testcat`.`ns1`.`ns2`.`tbl`")) // If "IF EXISTS" is set, UNCACHE TABLE will not throw an exception. sql(s"UNCACHE TABLE IF EXISTS $t") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/InsertIntoTests.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/InsertIntoTests.scala index 7493966790ccc..557be40bd7a89 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/InsertIntoTests.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/InsertIntoTests.scala @@ -234,8 +234,9 @@ trait InsertIntoSQLOnlyTests val e = intercept[AnalysisException] { sql(s"INSERT INTO $t2 VALUES (2L, 'dummy')") } - assert(e.getMessage.contains(t2)) - assert(e.getMessage.contains("Table not found")) + checkError(e, + errorClass = "TABLE_OR_VIEW_NOT_FOUND", + parameters = Map("relationName" -> "`tbl2`")) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/V1WriteFallbackSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/V1WriteFallbackSuite.scala index 992c46cc6cdb1..22ab271c99054 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/V1WriteFallbackSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/V1WriteFallbackSuite.scala @@ -24,6 +24,7 @@ import org.scalatest.BeforeAndAfter import org.apache.spark.rdd.RDD import org.apache.spark.sql.{AnalysisException, DataFrame, QueryTest, Row, SaveMode, SparkSession, SQLContext} +import org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.catalyst.trees.TreeNodeTag @@ -298,7 +299,7 @@ class InMemoryV1Provider } if (mode == SaveMode.ErrorIfExists && tableOpt.isDefined) { - throw new AnalysisException("Table already exists") + throw new TableAlreadyExistsException(table.name) } else if (mode == SaveMode.Ignore && tableOpt.isDefined) { // do nothing return getRelation diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewSuite.scala index ad1e64c69a70a..7daf31a8d85d4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewSuite.scala @@ -230,6 +230,13 @@ abstract class SQLViewSuite extends QueryTest with SQLTestUtils { assert(e.message.contains(message)) } + private def assertAnalysisErrorClass(query: String, + errorClass: String, + parameters: Map[String, String]): Unit = { + val e = intercept[AnalysisException](sql(query)) + checkError(e, errorClass = errorClass, parameters = parameters) + } + private def assertErrorForAlterTableOnTempView( sqlText: String, viewName: String, cmdName: String): Unit = { assertAnalysisError( @@ -258,10 +265,14 @@ abstract class SQLViewSuite extends QueryTest with SQLTestUtils { test("error handling: fail if the view sql itself is invalid") { // A database that does not exist - assertInvalidReference("CREATE OR REPLACE VIEW myabcdview AS SELECT * FROM db_not_exist234.jt") + assertRelationNotFound( + "CREATE OR REPLACE VIEW myabcdview AS SELECT * FROM db_not_exist234.jt", + "`db_not_exist234`.`jt`") // A table that does not exist - assertInvalidReference("CREATE OR REPLACE VIEW myabcdview AS SELECT * FROM table_not_exist345") + assertRelationNotFound( + "CREATE OR REPLACE VIEW myabcdview AS SELECT * FROM table_not_exist345", + "`table_not_exist345`") // A column that does not exist intercept[AnalysisException] { @@ -269,11 +280,13 @@ abstract class SQLViewSuite extends QueryTest with SQLTestUtils { } } - private def assertInvalidReference(query: String): Unit = { + private def assertRelationNotFound(query: String, relation: String): Unit = { val e = intercept[AnalysisException] { sql(query) - }.getMessage - assert(e.contains("Table or view not found")) + } + checkError(e, + errorClass = "TABLE_OR_VIEW_NOT_FOUND", + parameters = Map("relationName" -> relation)) } @@ -296,12 +309,15 @@ abstract class SQLViewSuite extends QueryTest with SQLTestUtils { test("error handling: fail if the temp view sql itself is invalid") { // A database that does not exist - assertInvalidReference( - "CREATE OR REPLACE TEMPORARY VIEW myabcdview AS SELECT * FROM db_not_exist234.jt") + assertAnalysisErrorClass( + "CREATE OR REPLACE TEMPORARY VIEW myabcdview AS SELECT * FROM db_not_exist234.jt", + "TABLE_OR_VIEW_NOT_FOUND", + Map("relationName" -> "`db_not_exist234`.`jt`")) // A table that does not exist - assertInvalidReference( - "CREATE OR REPLACE TEMPORARY VIEW myabcdview AS SELECT * FROM table_not_exist1345") + assertRelationNotFound( + "CREATE OR REPLACE TEMPORARY VIEW myabcdview AS SELECT * FROM table_not_exist1345", + "`table_not_exist1345`") // A column that does not exist, for temporary view intercept[AnalysisException] { @@ -368,7 +384,9 @@ abstract class SQLViewSuite extends QueryTest with SQLTestUtils { sql("CREATE TEMPORARY VIEW testView AS SELECT id FROM jt") } - assert(e.message.contains("Temporary view") && e.message.contains("already exists")) + checkError(e, + "TEMP_TABLE_OR_VIEW_ALREADY_EXISTS", + parameters = Map("relationName" -> "`testView`")) } } @@ -468,13 +486,13 @@ abstract class SQLViewSuite extends QueryTest with SQLTestUtils { } test("should not allow ALTER VIEW AS when the view does not exist") { - assertAnalysisError( + assertRelationNotFound( "ALTER VIEW testView AS SELECT 1, 2", - "View not found: testView") + "`testView`") - assertAnalysisError( + assertRelationNotFound( "ALTER VIEW default.testView AS SELECT 1, 2", - "View not found: default.testView") + "`default`.`testView`") } test("ALTER VIEW AS should try to alter temp view first if view name has no database part") { @@ -618,21 +636,21 @@ abstract class SQLViewSuite extends QueryTest with SQLTestUtils { } } } - assertInvalidReference("SELECT * FROM view1") + assertRelationNotFound("SELECT * FROM view1", "`table1`") // Fail if the referenced table is invalid. withTable("table2") { sql("CREATE TABLE table2(a int, b string) USING parquet") sql("CREATE VIEW view2 AS SELECT * FROM table2") } - assertInvalidReference("SELECT * FROM view2") + assertRelationNotFound("SELECT * FROM view2", "`table2`") // Fail if the referenced view is invalid. withView("testView") { sql("CREATE VIEW testView AS SELECT * FROM jt") sql("CREATE VIEW view3 AS SELECT * FROM testView") } - assertInvalidReference("SELECT * FROM view3") + assertRelationNotFound("SELECT * FROM view3", "`testView`") } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewTestSuite.scala index 2a45c0ab50968..f85cb93e67024 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewTestSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewTestSuite.scala @@ -288,8 +288,10 @@ abstract class SQLViewTestSuite extends QueryTest with SQLTestUtils { sql("DROP TABLE IF EXISTS default.t") val e = intercept[AnalysisException] { sql(s"SELECT * FROM $viewName").collect() - }.getMessage - assert(e.contains("Table or view not found: t")) + } + checkError(e, + errorClass = "TABLE_OR_VIEW_NOT_FOUND", + parameters = Map("relationName" -> "`t`")) } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterNamespaceSetLocationSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterNamespaceSetLocationSuiteBase.scala index 25bae01821a8c..ad4f27a8830b8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterNamespaceSetLocationSuiteBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterNamespaceSetLocationSuiteBase.scala @@ -54,10 +54,12 @@ trait AlterNamespaceSetLocationSuiteBase extends QueryTest with DDLCommandTestUt test("Namespace does not exist") { val ns = "not_exist" - val message = intercept[AnalysisException] { + val e = intercept[AnalysisException] { sql(s"ALTER DATABASE $catalog.$ns SET LOCATION 'loc'") - }.getMessage - assert(message.contains(s"$notFoundMsgPrefix '$ns' not found")) + } + checkError(e, + errorClass = "SCHEMA_NOT_FOUND", + parameters = Map("schemaName" -> "`not_exist`")) } // Hive catalog does not support "ALTER NAMESPACE ... SET LOCATION", thus diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterNamespaceSetPropertiesSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterNamespaceSetPropertiesSuiteBase.scala index 1351d09e0344c..c28c7b9db0436 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterNamespaceSetPropertiesSuiteBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterNamespaceSetPropertiesSuiteBase.scala @@ -45,10 +45,12 @@ trait AlterNamespaceSetPropertiesSuiteBase extends QueryTest with DDLCommandTest test("Namespace does not exist") { val ns = "not_exist" - val message = intercept[AnalysisException] { + val e = intercept[AnalysisException] { sql(s"ALTER DATABASE $catalog.$ns SET PROPERTIES ('d'='d')") - }.getMessage - assert(message.contains(s"$notFoundMsgPrefix '$ns' not found")) + } + checkError(e, + errorClass = "SCHEMA_NOT_FOUND", + parameters = Map("schemaName" -> s"`$ns`")) } test("basic test") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableAddPartitionSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableAddPartitionSuiteBase.scala index dee14953c0924..de098ffcc0d7c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableAddPartitionSuiteBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableAddPartitionSuiteBase.scala @@ -81,10 +81,12 @@ trait AlterTableAddPartitionSuiteBase extends QueryTest with DDLCommandTestUtils test("table to alter does not exist") { withNamespaceAndTable("ns", "does_not_exist") { t => - val errMsg = intercept[AnalysisException] { + val e = intercept[AnalysisException] { sql(s"ALTER TABLE $t ADD IF NOT EXISTS PARTITION (a='4', b='9')") - }.getMessage - assert(errMsg.contains("Table not found")) + } + checkError(e, + errorClass = "TABLE_OR_VIEW_NOT_FOUND", + parameters = Map("relationName" -> "`test_catalog`.`ns`.`does_not_exist`")) } } @@ -172,11 +174,14 @@ trait AlterTableAddPartitionSuiteBase extends QueryTest with DDLCommandTestUtils sql(s"CREATE TABLE $t (id bigint, data string) $defaultUsing PARTITIONED BY (id)") sql(s"ALTER TABLE $t ADD PARTITION (id=2) LOCATION 'loc1'") - val errMsg = intercept[PartitionsAlreadyExistException] { + val e = intercept[PartitionsAlreadyExistException] { sql(s"ALTER TABLE $t ADD PARTITION (id=1) LOCATION 'loc'" + " PARTITION (id=2) LOCATION 'loc1'") - }.getMessage - assert(errMsg.contains("The following partitions already exists")) + } + checkError(e, + errorClass = "PARTITIONS_ALREADY_EXIST", + parameters = Map("partitionList" -> "PARTITION (`id` = 1)", + "tableName" -> "`ns`.`tbl`")) sql(s"ALTER TABLE $t ADD IF NOT EXISTS PARTITION (id=1) LOCATION 'loc'" + " PARTITION (id=2) LOCATION 'loc1'") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableDropPartitionSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableDropPartitionSuiteBase.scala index 41ac62ed333aa..f48f78f8d51fc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableDropPartitionSuiteBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableDropPartitionSuiteBase.scala @@ -86,10 +86,12 @@ trait AlterTableDropPartitionSuiteBase extends QueryTest with DDLCommandTestUtil test("table to alter does not exist") { withNamespaceAndTable("ns", "does_not_exist") { t => - val errMsg = intercept[AnalysisException] { + val e = intercept[AnalysisException] { sql(s"ALTER TABLE $t DROP PARTITION (a='4', b='9')") - }.getMessage - assert(errMsg.contains("Table not found")) + } + checkError(e, + errorClass = "TABLE_OR_VIEW_NOT_FOUND", + parameters = Map("relationName" -> "`test_catalog`.`ns`.`does_not_exist`")) } } @@ -130,10 +132,13 @@ trait AlterTableDropPartitionSuiteBase extends QueryTest with DDLCommandTestUtil sql(s"CREATE TABLE $t (id bigint, data string) $defaultUsing PARTITIONED BY (id)") sql(s"ALTER TABLE $t ADD PARTITION (id=1) LOCATION 'loc'") - val errMsg = intercept[NoSuchPartitionsException] { + val e = intercept[NoSuchPartitionsException] { sql(s"ALTER TABLE $t DROP PARTITION (id=1), PARTITION (id=2)") - }.getMessage - assert(errMsg.contains("partitions not found in table")) + } + checkError(e, + errorClass = "PARTITIONS_NOT_FOUND", + parameters = Map("partitionList" -> "PARTITION (`id` = 2)", + "tableName" -> "`test_catalog`.`ns`.`tbl`")) checkPartitions(t, Map("id" -> "1")) sql(s"ALTER TABLE $t DROP IF EXISTS PARTITION (id=1), PARTITION (id=2)") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableRenamePartitionSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableRenamePartitionSuiteBase.scala index 080cd89c4a209..d1ff61bff9194 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableRenamePartitionSuiteBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableRenamePartitionSuiteBase.scala @@ -57,10 +57,12 @@ trait AlterTableRenamePartitionSuiteBase extends QueryTest with DDLCommandTestUt test("table to alter does not exist") { withNamespace(s"$catalog.ns") { sql(s"CREATE NAMESPACE $catalog.ns") - val errMsg = intercept[AnalysisException] { + val e = intercept[AnalysisException] { sql(s"ALTER TABLE $catalog.ns.no_tbl PARTITION (id=1) RENAME TO PARTITION (id=2)") - }.getMessage - assert(errMsg.contains("Table not found")) + } + checkError(e, + errorClass = "TABLE_OR_VIEW_NOT_FOUND", + parameters = Map("relationName" -> s"`$catalog`.`ns`.`no_tbl`")) } } @@ -68,10 +70,13 @@ trait AlterTableRenamePartitionSuiteBase extends QueryTest with DDLCommandTestUt withNamespaceAndTable("ns", "tbl") { t => createSinglePartTable(t) checkPartitions(t, Map("id" -> "1")) - val errMsg = intercept[NoSuchPartitionException] { + val e = intercept[NoSuchPartitionException] { sql(s"ALTER TABLE $t PARTITION (id = 3) RENAME TO PARTITION (id = 2)") - }.getMessage - assert(errMsg.contains("Partition not found in table")) + } + checkError(e, + errorClass = "PARTITIONS_NOT_FOUND", + parameters = Map("partitionList" -> "PARTITION (`id` = 3)", + "tableName" -> "`ns`.`tbl`")) } } @@ -80,10 +85,12 @@ trait AlterTableRenamePartitionSuiteBase extends QueryTest with DDLCommandTestUt createSinglePartTable(t) sql(s"INSERT INTO $t PARTITION (id = 2) SELECT 'def'") checkPartitions(t, Map("id" -> "1"), Map("id" -> "2")) - val errMsg = intercept[PartitionAlreadyExistsException] { + val e = intercept[PartitionAlreadyExistsException] { sql(s"ALTER TABLE $t PARTITION (id = 1) RENAME TO PARTITION (id = 2)") - }.getMessage - assert(errMsg.contains("Partition already exists")) + } + checkError(e, + errorClass = "PARTITIONS_ALREADY_EXIST", + parameters = Map("partitionList" -> "PARTITION (`id` = 2)", "tableName" -> "`ns`.`tbl`")) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableRenameSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableRenameSuiteBase.scala index 2942d61f7fb7f..c4f7d762b6497 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableRenameSuiteBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableRenameSuiteBase.scala @@ -47,10 +47,12 @@ trait AlterTableRenameSuiteBase extends QueryTest with DDLCommandTestUtils { } test("table to rename does not exist") { - val errMsg = intercept[AnalysisException] { + val e = intercept[AnalysisException] { sql(s"ALTER TABLE $catalog.dbx.does_not_exist RENAME TO dbx.tab2") - }.getMessage - assert(errMsg.contains("Table or view not found")) + } + checkError(e, + errorClass = "TABLE_OR_VIEW_NOT_FOUND", + parameters = Map("relationName" -> s"`$catalog`.`dbx`.`does_not_exist`")) } test("omit namespace in the destination table") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/CreateNamespaceSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/CreateNamespaceSuiteBase.scala index 7db8fba8ac366..f19d19345b5f4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/CreateNamespaceSuiteBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/CreateNamespaceSuiteBase.scala @@ -92,7 +92,9 @@ trait CreateNamespaceSuiteBase extends QueryTest with DDLCommandTestUtils { val e = intercept[NamespaceAlreadyExistsException] { sql(s"CREATE NAMESPACE $ns") } - assert(e.getMessage.contains(s"$notFoundMsgPrefix '$namespace' already exists")) + checkError(e, + errorClass = "SCHEMA_ALREADY_EXISTS", + parameters = Map("schemaName" -> s"`$namespace`")) // The following will be no-op since the namespace already exists. sql(s"CREATE NAMESPACE IF NOT EXISTS $ns") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala index c7fa365abbdeb..d20dab0aec37b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala @@ -824,8 +824,10 @@ abstract class DDLSuite extends QueryTest with DDLSuiteBase { spark.range(10).createOrReplaceTempView("tab1") sql("ALTER TABLE tab1 RENAME TO tab2") checkAnswer(spark.table("tab2"), spark.range(10).toDF()) - val e = intercept[AnalysisException](spark.table("tab1")).getMessage - assert(e.contains("Table or view not found")) + val e = intercept[AnalysisException](spark.table("tab1")) + checkError(e, + errorClass = "TABLE_OR_VIEW_NOT_FOUND", + parameters = Map("relationName" -> "`tab1`")) sql("ALTER VIEW tab2 RENAME TO tab1") checkAnswer(spark.table("tab1"), spark.range(10).toDF()) intercept[AnalysisException] { spark.table("tab2") } @@ -859,8 +861,9 @@ abstract class DDLSuite extends QueryTest with DDLSuiteBase { val e = intercept[AnalysisException] { sql("ALTER TABLE tab1 RENAME TO tab2") } - assert(e.getMessage.contains( - "RENAME TEMPORARY VIEW from '`tab1`' to '`tab2`': destination table already exists")) + checkError(e, + "TABLE_OR_VIEW_ALREADY_EXISTS", + parameters = Map("relationName" -> "`tab2`")) val catalog = spark.sessionState.catalog assert(catalog.listTables("default") == Seq(TableIdentifier("tab1"), TableIdentifier("tab2"))) @@ -894,8 +897,9 @@ abstract class DDLSuite extends QueryTest with DDLSuiteBase { val e = intercept[AnalysisException] { sql("ALTER TABLE view1 RENAME TO view2") } - assert(e.getMessage.contains( - "RENAME TEMPORARY VIEW from '`view1`' to '`view2`': destination table already exists")) + checkError(e, + errorClass = "TABLE_OR_VIEW_ALREADY_EXISTS", + parameters = Map("relation_name" -> "`view2`")) val catalog = spark.sessionState.catalog assert(catalog.listTables("default") == @@ -1215,8 +1219,10 @@ abstract class DDLSuite extends QueryTest with DDLSuiteBase { sql("CREATE TEMPORARY VIEW t_temp AS SELECT 1, 2") val e = intercept[TempTableAlreadyExistsException] { sql("CREATE TEMPORARY TABLE t_temp (c3 int, c4 string) USING JSON") - }.getMessage - assert(e.contains("Temporary view 't_temp' already exists")) + } + checkError(e, + errorClass = "TEMP_TABLE_OR_VIEW_ALREADY_EXISTS", + parameters = Map("relation_name" -> "`t_temp`")) } } @@ -1225,8 +1231,10 @@ abstract class DDLSuite extends QueryTest with DDLSuiteBase { sql("CREATE TEMPORARY VIEW t_temp AS SELECT 1, 2") val e = intercept[TempTableAlreadyExistsException] { sql("CREATE TEMPORARY VIEW t_temp (c3 int, c4 string) USING JSON") - }.getMessage - assert(e.contains("Temporary view 't_temp' already exists")) + } + checkError(e, + errorClass = "TEMP_TABLE_OR_VIEW_ALREADY_EXISTS", + parameters = Map("relation_name" -> "`t_temp`")) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DescribeNamespaceSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DescribeNamespaceSuiteBase.scala index e55f18007b23d..1309ba05b3f19 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DescribeNamespaceSuiteBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DescribeNamespaceSuiteBase.scala @@ -39,11 +39,12 @@ trait DescribeNamespaceSuiteBase extends QueryTest with DDLCommandTestUtils { test("namespace does not exists") { val ns = "db1" - val message = intercept[AnalysisException] { + val e = intercept[AnalysisException] { sql(s"DESCRIBE NAMESPACE EXTENDED $catalog.$ns") - }.getMessage - - assert(message.contains(s"$notFoundMsgPrefix '$ns' not found")) + } + checkError(e, + errorClass = "SCHEMA_NOT_FOUND", + parameters = Map("schemaName" -> "`db1`")) } test("Keep the legacy output schema") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DescribeTableSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DescribeTableSuiteBase.scala index c1b92699c6ecb..4e615fb31b397 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DescribeTableSuiteBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DescribeTableSuiteBase.scala @@ -40,7 +40,9 @@ trait DescribeTableSuiteBase extends QueryTest with DDLCommandTestUtils { val e = intercept[AnalysisException] { sql(s"DESCRIBE TABLE ${tbl}_non_existence") } - assert(e.getMessage.contains(s"Table or view not found: ${tbl}_non_existence")) + checkError(e, + errorClass = "TABLE_OR_VIEW_NOT_FOUND", + parameters = Map("relation_name" -> "`spark_catalog`.`ns`.`table_non_existence`")) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DropNamespaceSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DropNamespaceSuiteBase.scala index 376f376c32d5b..6eb4465124a69 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DropNamespaceSuiteBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DropNamespaceSuiteBase.scala @@ -60,10 +60,12 @@ trait DropNamespaceSuiteBase extends QueryTest with DDLCommandTestUtils { test("namespace does not exist") { // Namespace $catalog.unknown does not exist. - val message = intercept[AnalysisException] { + val e = intercept[AnalysisException] { sql(s"DROP NAMESPACE $catalog.unknown") - }.getMessage - assert(message.contains(s"'unknown' not found")) + } + checkError(e, + errorClass = "SCHEMA_NOT_FOUND", + parameters = Map("schemaName" -> "`unknown`")) } test("drop non-empty namespace with a non-cascading mode") { @@ -75,7 +77,9 @@ trait DropNamespaceSuiteBase extends QueryTest with DDLCommandTestUtils { val e = intercept[AnalysisException] { sql(s"DROP NAMESPACE $catalog.ns") } - assert(e.getMessage.contains(s"Cannot drop a non-empty $namespaceAlias: ns")) + checkError(e, + errorClass = "SCHEMA_NOT_EMPTY", + parameters = Map("schemaName" -> "`ns`")) sql(s"DROP TABLE $catalog.ns.table") // Now that $catalog.ns is empty, it can be dropped. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DropTableSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DropTableSuiteBase.scala index 3c9b39af8ef22..cfb9a7c8285d5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DropTableSuiteBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DropTableSuiteBase.scala @@ -54,10 +54,12 @@ trait DropTableSuiteBase extends QueryTest with DDLCommandTestUtils { sql(s"CREATE NAMESPACE $catalog.ns") checkTables("ns") // no tables - val errMsg = intercept[AnalysisException] { + val e = intercept[AnalysisException] { sql(s"DROP TABLE $catalog.ns.tbl") - }.getMessage - assert(errMsg.contains("Table or view not found")) + } + checkError(e, + errorClass = "TABLE_OR_VIEW_NOT_FOUND", + parameters = Map("relationName" -> s"`$catalog`.`ns`.`tbl`")) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala index 7e8816553499d..7d54a010a475d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala @@ -154,7 +154,7 @@ class PlanResolutionSuite extends AnalysisTest { case "defaultvalues" => defaultValues case "defaultvalues2" => defaultValues2 case "tablewithcolumnnameddefault" => tableWithColumnNamedDefault - case name => throw new NoSuchTableException(name) + case name => throw new NoSuchTableException(Seq(name)) } }) when(newCatalog.name()).thenReturn("testcat") @@ -172,7 +172,7 @@ class PlanResolutionSuite extends AnalysisTest { case "v2Table1" => table1 case "v2TableWithAcceptAnySchemaCapability" => tableWithAcceptAnySchemaCapability case "view" => createV1TableMock(ident, tableType = CatalogTableType.VIEW) - case name => throw new NoSuchTableException(name) + case name => throw new NoSuchTableException(Seq(name)) } }) when(newCatalog.name()).thenReturn(CatalogManager.SESSION_CATALOG_NAME) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowTblPropertiesSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowTblPropertiesSuiteBase.scala index e3490ad2d5e5b..84e6cf10df167 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowTblPropertiesSuiteBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowTblPropertiesSuiteBase.scala @@ -72,10 +72,12 @@ trait ShowTblPropertiesSuiteBase extends QueryTest with DDLCommandTestUtils { } test("SHOW TBLPROPERTIES WITH TABLE NOT EXIST") { - val message = intercept[AnalysisException] { + val e = intercept[AnalysisException] { sql("SHOW TBLPROPERTIES BADTABLE") - }.getMessage - assert(message.contains("Table or view not found: BADTABLE")) + } + checkError(e, + errorClass = "TABLE_OR_VIEW_NOT_FOUND", + parameters = Map("relationName" -> "`BADTABLE`")) } test("SHOW TBLPROPERTIES(KEY) KEY NOT FOUND") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/TruncateTableSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/TruncateTableSuiteBase.scala index 001ec8e250def..901464dc328ca 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/TruncateTableSuiteBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/TruncateTableSuiteBase.scala @@ -36,10 +36,12 @@ trait TruncateTableSuiteBase extends QueryTest with DDLCommandTestUtils { test("table does not exist") { withNamespaceAndTable("ns", "does_not_exist") { t => - val errMsg = intercept[AnalysisException] { + val e = intercept[AnalysisException] { sql(s"TRUNCATE TABLE $t") - }.getMessage - assert(errMsg.contains("Table not found")) + } + checkError(e, + errorClass = "TABLE_OR_VIEW_NOT_FOUND", + parameters = Map("relationName" -> "`spark_catalog`.`ns`.`does_not_exist`")) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/AlterTableRecoverPartitionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/AlterTableRecoverPartitionsSuite.scala index 16dd6a3a03f1d..e245cad381ec9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/AlterTableRecoverPartitionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/AlterTableRecoverPartitionsSuite.scala @@ -37,10 +37,12 @@ import org.apache.spark.sql.execution.command */ trait AlterTableRecoverPartitionsSuiteBase extends command.AlterTableRecoverPartitionsSuiteBase { test("table does not exist") { - val errMsg = intercept[AnalysisException] { + val e = intercept[AnalysisException] { sql("ALTER TABLE does_not_exist RECOVER PARTITIONS") - }.getMessage - assert(errMsg.contains("Table not found")) + } + checkError(e, + errorClass = "TABLE_OR_VIEW_NOT_FOUND", + parameters = Map("relationName" -> "`does_not_exist`")) } test("valid locations") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/AlterTableSetLocationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/AlterTableSetLocationSuite.scala index a8af349a89bd3..2f8e04f1c1fd9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/AlterTableSetLocationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/AlterTableSetLocationSuite.scala @@ -111,7 +111,9 @@ trait AlterTableSetLocationSuiteBase extends command.AlterTableSetLocationSuiteB val e = intercept[AnalysisException] { sql("ALTER TABLE ns.does_not_exist SET LOCATION '/mister/spark'") } - assert(e.getMessage.contains("Table not found: ns.does_not_exist")) + checkError(e, + errorClass = "TABLE_OR_VIEW_NOT_FOUND", + parameters = Map("relation_name" -> "`ns`.`does_not_exist`")) } test("partition to alter set location does not exist") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/AlterTableSetSerdeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/AlterTableSetSerdeSuite.scala index b34d7b03d4768..261609c649d98 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/AlterTableSetSerdeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/AlterTableSetSerdeSuite.scala @@ -107,7 +107,9 @@ class AlterTableSetSerdeSuite extends AlterTableSetSerdeSuiteBase with CommandSu val e3 = intercept[AnalysisException] { sql("ALTER TABLE does_not_exist SET SERDEPROPERTIES ('x' = 'y')") } - assert(e3.getMessage.contains("Table not found: does_not_exist")) + checkError(e3, + errorClass = "TABLE_OR_VIEW_NOT_FOUND", + parameters = Map("relation_name" -> "`does_not_exist`")) } } @@ -162,7 +164,9 @@ class AlterTableSetSerdeSuite extends AlterTableSetSerdeSuiteBase with CommandSu val e5 = intercept[AnalysisException] { sql("ALTER TABLE does_not_exist PARTITION (a=1, b=2) SET SERDEPROPERTIES ('x' = 'y')") } - assert(e5.getMessage.contains("Table not found: does_not_exist")) + checkError(e5, + errorClass = "TABLE_OR_VIEW_NOT_FOUND", + parameters = Map("relation_name" -> "`does_not_exist`")) } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/DescribeTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/DescribeTableSuite.scala index da4eab13afb05..fd536174020f9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/DescribeTableSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/DescribeTableSuite.scala @@ -46,7 +46,10 @@ trait DescribeTableSuiteBase extends command.DescribeTableSuiteBase val e = intercept[AnalysisException] { sql(s"DESCRIBE TABLE $tbl PARTITION (id = 1)") } - assert(e.message === "Partition not found in table 'table' database 'ns':\nid -> 1") + checkError(e, + errorClass = "PARTITIONS_NOT_FOUND", + parameters = Map("partitionList" -> "PARTITION (`id` = 1)", + "tableName" -> "`ns`.`table`")) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowNamespacesSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowNamespacesSuite.scala index b65a9acb65612..ee5ac09e00892 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowNamespacesSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowNamespacesSuite.scala @@ -45,10 +45,12 @@ trait ShowNamespacesSuiteBase extends command.ShowNamespacesSuiteBase { } test("IN namespace doesn't exist") { - val errMsg = intercept[AnalysisException] { + val e = intercept[AnalysisException] { sql("SHOW NAMESPACES in dummy") - }.getMessage - assert(errMsg.contains("Namespace 'dummy' not found")) + } + checkError(e, + errorClass = "SCHEMA_NOT_FOUND", + parameters = Map("schemaName" -> "`dummy`")) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowTablesSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowTablesSuite.scala index 68ad1c40d8270..4db42f1d7202d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowTablesSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowTablesSuite.scala @@ -130,10 +130,12 @@ trait ShowTablesSuiteBase extends command.ShowTablesSuiteBase with command.Tests } test("show table in a not existing namespace") { - val msg = intercept[NoSuchDatabaseException] { + val e = intercept[NoSuchDatabaseException] { runShowTablesSql(s"SHOW TABLES IN $catalog.unknown", Seq()) - }.getMessage - assert(msg.matches("(Database|Namespace) 'unknown' not found")) + } + checkError(e, + errorClass = "SCHEMA_NOT_FOUND", + parameters = Map("schemaName" -> "`unknown`")) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowTablesSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowTablesSuite.scala index 6dc8a050f9ca8..e7e5c71c9ef0a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowTablesSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowTablesSuite.scala @@ -92,9 +92,11 @@ class ShowTablesSuite extends command.ShowTablesSuiteBase with CommandSuiteBase } test("show table in a not existing namespace") { - val msg = intercept[NoSuchNamespaceException] { + val e = intercept[NoSuchNamespaceException] { runShowTablesSql(s"SHOW TABLES IN $catalog.unknown", Seq()) - }.getMessage - assert(msg.matches("(Database|Namespace) 'unknown' not found")) + } + checkError(e, + errorClass = "SCHEMA_NOT_FOUND", + parameters = Map("schemaName" -> "`unknown`")) } } 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 d37d5a96c656e..fe93017b2a3dc 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 @@ -232,7 +232,9 @@ class V2SessionCatalogTableSuite extends V2SessionCatalogBaseSuite { catalog.loadTable(testIdent) } - assert(exc.message.contains("Table or view 'test_table' not found in database 'db'")) + checkError(exc, + errorClass = "TABLE_OR_VIEW_NOT_FOUND", + parameters = Map("relationName" -> "`db`.`test_table`")) } test("invalidateTable") { @@ -683,8 +685,9 @@ class V2SessionCatalogTableSuite extends V2SessionCatalogBaseSuite { catalog.alterTable(testIdent, TableChange.setProperty("prop", "val")) } - assert(exc.message.contains(testIdent.quoted)) - assert(exc.message.contains("not found")) + checkError(exc, + errorClass = "TABLE_OR_VIEW_NOT_FOUND", + parameters = Map("relationName" -> testIdent.quoted)) } test("alterTable: location") { @@ -760,7 +763,9 @@ class V2SessionCatalogTableSuite extends V2SessionCatalogBaseSuite { catalog.renameTable(testIdent, testIdentNew) } - assert(exc.message.contains("Table or view 'test_table' not found in database 'db'")) + checkError(exc, + errorClass = "TABLE_OR_VIEW_NOT_FOUND", + parameters = Map("relationName" -> "`db`.`test_table`")) } test("renameTable: fail if new table name already exists") { @@ -779,8 +784,9 @@ class V2SessionCatalogTableSuite extends V2SessionCatalogBaseSuite { catalog.renameTable(testIdent, testIdentNew) } - assert(exc.message.contains(testIdentNew.quoted)) - assert(exc.message.contains("already exists")) + checkError(exc, + errorClass = "TABLE_OR_VIEW_ALREADY_EXISTS", + parameters = Map("relationName" -> testIdentNew.quoted)) } test("renameTable: fail if db does not match for old and new table names") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTableCatalogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTableCatalogSuite.scala index 7aa8adc07edd3..ef3708a8ced3b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTableCatalogSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTableCatalogSuite.scala @@ -81,15 +81,15 @@ class JDBCTableCatalogSuite extends QueryTest with SharedSparkSession { sql("DROP TABLE h2.test.to_drop") checkAnswer(sql("SHOW TABLES IN h2.test"), Seq(Row("test", "people", false))) Seq( - "h2.test.not_existing_table" -> - "Table or view not found: h2.test.not_existing_table", - "h2.bad_test.not_existing_table" -> - "Table or view not found: h2.bad_test.not_existing_table" - ).foreach { case (table, expectedMsg) => - val msg = intercept[AnalysisException] { + "h2.test.not_existing_table" -> "`h2`.`test`.`not_existing_table`", + "h2.bad_test.not_existing_table" -> "`h2`.`bad_test`.`not_existing_table`" + ).foreach { case (table, expected) => + val e = intercept[AnalysisException] { sql(s"DROP TABLE $table") - }.getMessage - assert(msg.contains(expectedMsg)) + } + checkError(e, + errorClass = "TABLE_OR_VIEW_NOT_FOUND", + parameters = Map("relationName" -> expected)) } } @@ -110,13 +110,15 @@ class JDBCTableCatalogSuite extends QueryTest with SharedSparkSession { val exp1 = intercept[AnalysisException] { sql("ALTER TABLE h2.test.not_existing_table RENAME TO test.dst_table") } - assert(exp1.getMessage.contains( - "Table or view not found: h2.test.not_existing_table")) + checkError(exp1, + errorClass = "TABLE_OR_VIEW_NOT_FOUND", + parameters = Map("relationName" -> "`h2`.`test`.`not_existing_table`")) val exp2 = intercept[AnalysisException] { sql("ALTER TABLE h2.bad_test.not_existing_table RENAME TO test.dst_table") } - assert(exp2.getMessage.contains( - "Table or view not found: h2.bad_test.not_existing_table")) + checkError(exp2, + errorClass = "TABLE_OR_VIEW_NOT_FOUND", + parameters = Map("relationName" -> "`h2`.`bad_test`.`not_existing_table`")) // Rename to an existing table withTable("h2.test.dst_table") { withConnection { conn => @@ -142,11 +144,16 @@ class JDBCTableCatalogSuite extends QueryTest with SharedSparkSession { .add("NAME", StringType, true, defaultMetadata) .add("ID", IntegerType, true, defaultMetadata) assert(t.schema === expectedSchema) - Seq("h2.test.not_existing_table", "h2.bad_test.not_existing_table").foreach { table => - val msg = intercept[AnalysisException] { + Seq( + "h2.test.not_existing_table" -> "`h2`.`test`.`not_existing_table`", + "h2.bad_test.not_existing_table" -> "`h2`.`bad_test`.`not_existing_table`" + ).foreach { case (table, expected) => + val e = intercept[AnalysisException] { spark.table(table).schema - }.getMessage - assert(msg.contains("Table or view not found")) + } + checkError(e, + errorClass = "TABLE_OR_VIEW_NOT_FOUND", + parameters = Map("relationName" -> expected)) } } @@ -159,10 +166,12 @@ class JDBCTableCatalogSuite extends QueryTest with SharedSparkSession { } withTable("h2.test.new_table") { sql("CREATE TABLE h2.test.new_table(i INT, j STRING)") - val msg = intercept[AnalysisException] { + val e = intercept[AnalysisException] { sql("CREATE TABLE h2.test.new_table(i INT, j STRING)") - }.getMessage - assert(msg.contains("Table test.new_table already exists")) + } + checkError(e, + errorClass = "TABLE_OR_VIEW_ALREADY_EXISTS", + parameters = Map("relationName" -> "`test`.`new_table`")) } val exp = intercept[NoSuchNamespaceException] { sql("CREATE TABLE h2.bad_test.new_table(i INT, j STRING)") @@ -193,11 +202,17 @@ class JDBCTableCatalogSuite extends QueryTest with SharedSparkSession { assert(msg.contains("Cannot add column, because c3 already exists")) } // Add a column to not existing table and namespace - Seq("h2.test.not_existing_table", "h2.bad_test.not_existing_table").foreach { table => - val msg = intercept[AnalysisException] { + Seq( + "h2.test.not_existing_table" -> "`h2`.`test`.`not_existing_table`", + "h2.bad_test.not_existing_table" -> "`h2`.`bad_test`.`not_existing_table`" + ).foreach { case (table, expected) => + val e = intercept[AnalysisException] { sql(s"ALTER TABLE $table ADD COLUMNS (C4 STRING)") - }.getMessage - assert(msg.contains("Table not found")) + } + checkError(e, + errorClass = "TABLE_OR_VIEW_NOT_FOUND", + parameters = Map("relationName" -> expected) + ) } } @@ -218,11 +233,16 @@ class JDBCTableCatalogSuite extends QueryTest with SharedSparkSession { assert(msg.contains("Cannot rename column, because C0 already exists")) } // Rename a column in not existing table and namespace - Seq("h2.test.not_existing_table", "h2.bad_test.not_existing_table").foreach { table => - val msg = intercept[AnalysisException] { + Seq( + "h2.test.not_existing_table" -> "`h2`.`test`.`not_existing_table`", + "h2.bad_test.not_existing_table" -> "`h2`.`bad_test`.`not_existing_table`" + ).foreach { case (table, expected) => + val e = intercept[AnalysisException] { sql(s"ALTER TABLE $table RENAME COLUMN ID TO C") - }.getMessage - assert(msg.contains("Table not found")) + } + checkError(e, + errorClass = "TABLE_OR_VIEW_NOT_FOUND", + parameters = Map("relationName" -> expected)) } } @@ -242,11 +262,16 @@ class JDBCTableCatalogSuite extends QueryTest with SharedSparkSession { assert(msg.contains("Missing field bad_column in table h2.test.alt_table")) } // Drop a column to not existing table and namespace - Seq("h2.test.not_existing_table", "h2.bad_test.not_existing_table").foreach { table => - val msg = intercept[AnalysisException] { + Seq( + "h2.test.not_existing_table" -> "`h2`.`test`.`not_existing_table`", + "h2.bad_test.not_existing_table" -> "`h2`.`bad_test`.`not_existing_table`" + ).foreach { case (table, expected) => + val e = intercept[AnalysisException] { sql(s"ALTER TABLE $table DROP COLUMN C1") - }.getMessage - assert(msg.contains("Table not found")) + } + checkError(e, + errorClass = "TABLE_OR_VIEW_NOT_FOUND", + parameters = Map("relationName" -> expected)) } } @@ -273,11 +298,16 @@ class JDBCTableCatalogSuite extends QueryTest with SharedSparkSession { assert(msg2.contains("DataType bad_type is not supported")) } // Update column type in not existing table and namespace - Seq("h2.test.not_existing_table", "h2.bad_test.not_existing_table").foreach { table => - val msg = intercept[AnalysisException] { + Seq( + "h2.test.not_existing_table" -> "`h2`.`test`.`not_existing_table`", + "h2.bad_test.not_existing_table" -> "`h2`.`bad_test`.`not_existing_table`" + ).foreach { case (table, expected) => + val e = intercept[AnalysisException] { sql(s"ALTER TABLE $table ALTER COLUMN id TYPE DOUBLE") - }.getMessage - assert(msg.contains("Table not found")) + } + checkError(e, + errorClass = "TABLE_OR_VIEW_NOT_FOUND", + parameters = Map("relationName" -> expected)) } } @@ -299,11 +329,16 @@ class JDBCTableCatalogSuite extends QueryTest with SharedSparkSession { assert(msg.contains("Missing field bad_column in table h2.test.alt_table")) } // Update column nullability in not existing table and namespace - Seq("h2.test.not_existing_table", "h2.bad_test.not_existing_table").foreach { table => - val msg = intercept[AnalysisException] { + Seq( + "h2.test.not_existing_table" -> "`h2`.`test`.`not_existing_table`", + "h2.bad_test.not_existing_table" -> "`h2`.`bad_test`.`not_existing_table`" + ).foreach { case (table, expected) => + val e = intercept[AnalysisException] { sql(s"ALTER TABLE $table ALTER COLUMN ID DROP NOT NULL") - }.getMessage - assert(msg.contains("Table not found")) + } + checkError(e, + errorClass = "TABLE_OR_VIEW_NOT_FOUND", + parameters = Map("relationName" -> expected)) } } @@ -323,11 +358,16 @@ class JDBCTableCatalogSuite extends QueryTest with SharedSparkSession { assert(msg.contains("Missing field bad_column in table h2.test.alt_table")) } // Update column comments in not existing table and namespace - Seq("h2.test.not_existing_table", "h2.bad_test.not_existing_table").foreach { table => - val msg = intercept[AnalysisException] { + Seq( + "h2.test.not_existing_table" -> "`h2`.`test`.`not_existing_table`", + "h2.bad_test.not_existing_table" -> "`h2`.`bad_test`.`not_existing_table`" + ).foreach { case (table, expected) => + val e = intercept[AnalysisException] { sql(s"ALTER TABLE $table ALTER COLUMN ID COMMENT 'test'") - }.getMessage - assert(msg.contains("Table not found")) + } + checkError(e, + errorClass = "TABLE_OR_VIEW_NOT_FOUND", + parameters = Map("relationName" -> expected)) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamTableAPISuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamTableAPISuite.scala index 60d2b8844451b..64297f947872f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamTableAPISuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamTableAPISuite.scala @@ -75,9 +75,12 @@ class DataStreamTableAPISuite extends StreamTest with BeforeAndAfter { } test("read: read non-exist table") { - intercept[AnalysisException] { + val e = intercept[AnalysisException] { spark.readStream.table("non_exist_table") - }.message.contains("Table not found") + } + checkError(e, + errorClass = "TABLE_OR_VIEW_NOT_FOUND", + parameters = Map("relationName" -> "`non-existent_table`")) } test("read: stream table API with temp view") { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala index a8598db1003d8..85892f7dadc26 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala @@ -102,18 +102,23 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with TestHiveSingleto } test("uncache of nonexistent tables") { - val expectedErrorMsg = "Table or view not found:" // make sure table doesn't exist - var e = intercept[AnalysisException](spark.table("nonexistentTable")).getMessage - assert(e.contains(s"$expectedErrorMsg nonexistentTable")) + var e = intercept[AnalysisException](spark.table("nonexistentTable")) + checkError(e, + errorClass = "TABLE_OR_VIEW_NOT_FOUND", + parameters = Map("relationName" -> "`nonexistentTable`")) e = intercept[AnalysisException] { uncacheTable("nonexistentTable") - }.getMessage - assert(e.contains(expectedErrorMsg)) + } + checkError(e, + errorClass = "TABLE_OR_VIEW_NOT_FOUND", + parameters = Map("relationName" -> "`nonexistentTable`")) e = intercept[AnalysisException] { sql("UNCACHE TABLE nonexistentTable") - }.getMessage - assert(e.contains("Table or view not found: nonexistentTable")) + } + checkError(e, + errorClass = "TABLE_OR_VIEW_NOT_FOUND", + parameters = Map("relationName" -> "`nonexistentTable`")) sql("UNCACHE TABLE IF EXISTS nonexistentTable") } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala index a7148e9c92158..09202c8234656 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala @@ -750,8 +750,10 @@ class InsertSuite extends QueryTest with TestHiveSingleton with BeforeAndAfter val e = intercept[AnalysisException] { sql(s"INSERT OVERWRITE LOCAL DIRECTORY '${path}' TABLE nonexistent") - }.getMessage - assert(e.contains("Table or view not found")) + } + checkError(e, + errorClass = "TABLE_OR_VIEW_NOT_FOUND", + parameters = Map("relationName" -> "`nonexistent`")) } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala index ffd0e6b48af6e..cb26d029de219 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala @@ -324,7 +324,7 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv """.stripMargin) // Create the table again should trigger a AnalysisException. - val message = intercept[AnalysisException] { + val e = intercept[AnalysisException] { sql( s"""CREATE TABLE ctasJsonTable |USING org.apache.spark.sql.json.DefaultSource @@ -333,11 +333,11 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv |) AS |SELECT * FROM jsonTable """.stripMargin) - }.getMessage + } - assert( - message.contains(s"Table $SESSION_CATALOG_NAME.default.ctasJsonTable already exists."), - "We should complain that ctasJsonTable already exists") + checkError(e, + errorClass = "TABLE_OR_VIEW_ALREADY_EXISTS", + parameters = Map("relationName" -> s"`$SESSION_CATALOG_NAME`.`default`.`ctasJsonTable`")) // The following statement should be fine if it has IF NOT EXISTS. // It tries to create a table ctasJsonTable with a new schema. @@ -522,11 +522,13 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv assert(table("createdJsonTable").schema === df.schema) checkAnswer(sql("SELECT * FROM createdJsonTable"), df) - assert( - intercept[AnalysisException] { + val e = intercept[AnalysisException] { sparkSession.catalog.createTable("createdJsonTable", jsonFilePath.toString) - }.getMessage.contains( - s"Table $SESSION_CATALOG_NAME.default.createdJsonTable already exists.")) + } + checkError(e, + errorClass = "TABLE_OR_VIEW_ALREADY_EXISTS", + parameters = Map("relationName" + -> s"`$SESSION_CATALOG_NAME`.`default`.`createdJsonTable`")) } // Data should not be deleted. @@ -1244,14 +1246,18 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv var e = intercept[AnalysisException] { table(tableName).write.mode(SaveMode.Overwrite).saveAsTable(tableName) - }.getMessage - assert(e.contains(s"Cannot overwrite table $SESSION_CATALOG_NAME.default.$tableName " + + } + assert(e.getMessage.contains( + s"Cannot overwrite table $SESSION_CATALOG_NAME.default.$tableName " + "that is also being read from")) e = intercept[AnalysisException] { table(tableName).write.mode(SaveMode.ErrorIfExists).saveAsTable(tableName) - }.getMessage - assert(e.contains(s"Table `$SESSION_CATALOG_NAME`.`default`.`$tableName` already exists")) + } + checkError(e, + errorClass = "TABLE_OR_VIEW_ALREADY_EXISTS", + parameters = Map("relationName" -> s"`$SESSION_CATALOG_NAME`.`default`.`$tableName`") + ) } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MultiDatabaseSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MultiDatabaseSuite.scala index 2d3e462531245..f43d5317aa71b 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MultiDatabaseSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MultiDatabaseSuite.scala @@ -268,17 +268,21 @@ class MultiDatabaseSuite extends QueryTest with SQLTestUtils with TestHiveSingle test("invalid database name and table names") { { - val message = intercept[AnalysisException] { + val e = intercept[AnalysisException] { df.write.format("parquet").saveAsTable("`d:b`.`t:a`") - }.getMessage - assert(message.contains("Database 'd:b' not found")) + } + checkError(e, + errorClass = "SCHEMA_NOT_FOUND", + parameters = Map("schemaName" -> "`d:b`")) } { - val message = intercept[AnalysisException] { + val e = intercept[AnalysisException] { df.write.format("parquet").saveAsTable("`d:b`.`table`") - }.getMessage - assert(message.contains("Database 'd:b' not found")) + } + checkError(e, + errorClass = "SCHEMA_NOT_FOUND", + parameters = Map("schemaName" -> "`d:b`")) } withTempDir { dir => @@ -298,7 +302,7 @@ class MultiDatabaseSuite extends QueryTest with SQLTestUtils with TestHiveSingle } { - val message = intercept[AnalysisException] { + val e = intercept[AnalysisException] { sql( s""" |CREATE TABLE `d:b`.`table` (a int) @@ -307,8 +311,10 @@ class MultiDatabaseSuite extends QueryTest with SQLTestUtils with TestHiveSingle | path '${dir.toURI}' |) """.stripMargin) - }.getMessage - assert(message.contains("Database 'd:b' not found")) + } + checkError(e, + errorClass = "SCHEMA_NOT_FOUND", + parameters = Map("schemaName" -> "`d:b`")) } } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/TestHiveSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/TestHiveSuite.scala index 72f8e8ff7c688..e38cca4dce6a7 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/TestHiveSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/TestHiveSuite.scala @@ -38,7 +38,9 @@ class TestHiveSuite extends TestHiveSingleton with SQLTestUtils { val err = intercept[AnalysisException] { sql("SELECT * FROM SRC").queryExecution.analyzed } - assert(err.message.contains("Table or view not found")) + checkError(err, + errorClass = "TABLE_OR_VIEW_NOT_FOUND", + parameters = Map("relationName" -> "`SRC`")) } testHiveSparkSession.reset() } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientSuite.scala index 22698b91cb62a..e86a37dac71f0 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientSuite.scala @@ -189,7 +189,9 @@ class HiveClientSuite(version: String, allVersions: Seq[String]) client.dropDatabase("temporary", ignoreIfNotExists = false, cascade = false) assert(false, "dropDatabase should throw HiveException") } - assert(ex.message.contains("Cannot drop a non-empty database: temporary.")) + checkError(ex, + errorClass = "SCHEMA_NOT_EMPTY", + parameters = Map("schemaName" -> "`temporary`")) client.dropDatabase("temporary", ignoreIfNotExists = false, cascade = true) assert(!client.databaseExists("temporary")) @@ -525,10 +527,13 @@ class HiveClientSuite(version: String, allVersions: Seq[String]) storageFormat)) try { client.createPartitions("default", "src_part", partitions, ignoreIfExists = false) - val errMsg = intercept[PartitionsAlreadyExistException] { + val e = intercept[PartitionsAlreadyExistException] { client.createPartitions("default", "src_part", partitions, ignoreIfExists = false) - }.getMessage - assert(errMsg.contains("partitions already exists")) + } + checkError(e, + errorClass = "PARTITIONS_ALREADY_EXIST", + parameters = Map("partitionList" -> "PARTITION (`key1` = 101, `key2` = 102)", + "tableName" -> "`default`.`src_part`")) } finally { client.dropPartitions( "default", diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala index f1bb8d30eed99..1a8413e9225c0 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala @@ -776,6 +776,13 @@ class HiveDDLSuite assert(e.message.contains(message)) } + private def assertAnalysisErrorClass(sqlText: String, errorClass: String, + parameters: Map[String, String]): Unit = { + val e = intercept[AnalysisException](sql(sqlText)) + checkError(e, + errorClass = errorClass, parameters = parameters) + } + private def assertErrorForAlterTableOnView(sqlText: String): Unit = { val message = intercept[AnalysisException](sql(sqlText)).getMessage assert(message.contains("Cannot alter a view with ALTER TABLE. Please use ALTER VIEW instead")) @@ -1212,9 +1219,10 @@ class HiveDDLSuite sql(s"USE default") val sqlDropDatabase = s"DROP DATABASE $dbName ${if (cascade) "CASCADE" else "RESTRICT"}" if (tableExists && !cascade) { - assertAnalysisError( + assertAnalysisErrorClass( sqlDropDatabase, - s"Cannot drop a non-empty database: $dbName.") + "SCHEMA_NOT_EMPTY", + Map("schemaName" -> s"`$dbName`")) // the database directory was not removed assert(fs.exists(new Path(expectedDBLocation))) } else { @@ -2956,11 +2964,13 @@ class HiveDDLSuite spark.sparkContext.addedJars.keys.find(_.contains(jarName)) .foreach(spark.sparkContext.addedJars.remove) assert(!spark.sparkContext.listJars().exists(_.contains(jarName))) - val msg = intercept[AnalysisException] { + val e = intercept[AnalysisException] { sql("CREATE TEMPORARY FUNCTION f1 AS " + s"'org.apache.hadoop.hive.ql.udf.UDFUUID' USING JAR '$jar'") - }.getMessage - assert(msg.contains("Function f1 already exists")) + } + checkError(e, + errorClass = "ROUTINE_ALREADY_EXISTS", + parameters = Map("routineName" -> "`f1`")) assert(!spark.sparkContext.listJars().exists(_.contains(jarName))) sql("CREATE OR REPLACE TEMPORARY FUNCTION f1 AS " + diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala index faefbfd1d6d92..f634d36e8a9d5 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala @@ -84,10 +84,12 @@ abstract class SQLQuerySuiteBase extends QueryTest with SQLTestUtils with TestHi test("non-existent global temp view") { val global_temp_db = spark.conf.get(GLOBAL_TEMP_DATABASE) - val message = intercept[AnalysisException] { + val e = intercept[AnalysisException] { spark.sql(s"select * from ${global_temp_db}.nonexistentview") - }.getMessage - assert(message.contains("Table or view not found")) + } + checkError(e, + errorClass = "TABLE_OR_VIEW_NOT_FOUND", + parameters = Map("relationName" -> s"`${global_temp_db}`.`nonexistentview`")) } test("script") { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/AlterTableSetSerdeSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/AlterTableSetSerdeSuite.scala index 48b48dbbaa4cd..96cbc6d4dab49 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/AlterTableSetSerdeSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/AlterTableSetSerdeSuite.scala @@ -64,7 +64,9 @@ class AlterTableSetSerdeSuite extends v1.AlterTableSetSerdeSuiteBase with Comman val e = intercept[AnalysisException] { sql("ALTER TABLE does_not_exist SET SERDEPROPERTIES ('x' = 'y')") } - assert(e.getMessage.contains("Table not found: does_not_exist")) + checkError(e, + errorClass = "TABLE_OR_VIEW_NOT_FOUND", + parameters = Map("relation_name" -> "`does_not_exist`")) } } @@ -111,7 +113,9 @@ class AlterTableSetSerdeSuite extends v1.AlterTableSetSerdeSuiteBase with Comman val e = intercept[AnalysisException] { sql("ALTER TABLE does_not_exist PARTITION (a=1, b=2) SET SERDEPROPERTIES ('x' = 'y')") } - assert(e.getMessage.contains("Table not found: does_not_exist")) + checkError(e, + errorClass = "TABLE_OR_VIEW_NOT_FOUND", + parameters = Map("relation_name" -> "`does_not_exist`")) } } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/HadoopFsRelationTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/HadoopFsRelationTest.scala index ce800e88218aa..51614537a4093 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/sources/HadoopFsRelationTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/HadoopFsRelationTest.scala @@ -382,10 +382,12 @@ abstract class HadoopFsRelationTest extends QueryTest with SQLTestUtils with Tes test("saveAsTable()/load() - non-partitioned table - ErrorIfExists") { withTable("t") { sql(s"CREATE TABLE t(i INT) USING $dataSourceName") - val msg = intercept[AnalysisException] { + val e = intercept[AnalysisException] { testDF.write.format(dataSourceName).mode(SaveMode.ErrorIfExists).saveAsTable("t") - }.getMessage - assert(msg.contains(s"Table `$SESSION_CATALOG_NAME`.`default`.`t` already exists")) + } + checkError(e, + errorClass = "TABLE_OR_VIEW_ALREADY_EXISTS", + parameters = Map("relationName" -> s"`$SESSION_CATALOG_NAME`.`default`.`t`")) } }