diff --git a/R/pkg/tests/fulltests/test_sparkSQL.R b/R/pkg/tests/fulltests/test_sparkSQL.R index 90538c3e7767e..534ec07abac52 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]*`zxwtyswklpf`*") 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)) @@ -4077,8 +4077,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]*`zxwtyswklpf`*") expect_true(databaseExists("default")) expect_true(databaseExists("spark_catalog.default")) @@ -4110,7 +4109,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]*`bar`*") c <- listColumns("cars") expect_equal(nrow(c), 2) @@ -4118,7 +4117,7 @@ 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")) + "[TABLE_OR_VIEW_NOT_FOUND]*`spark_catalog`.`default`.`zxwtyswklpf`*") f <- listFunctions() expect_true(nrow(f) >= 200) # 250 @@ -4127,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")) + "[SCHEMA_NOT_FOUND]*`zxwtyswklpf_db`*") 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..3528540b4252b 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")) + } + checkErrorTableNotFound(e, s"`$catalogName`.`not_existing_table`", + ExpectedContext(s"$catalogName.not_existing_table", 12, + 11 + s"$catalogName.not_existing_table".length)) } 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")) + } + checkErrorTableNotFound(e, s"`$catalogName`.`not_existing_table`", + ExpectedContext(s"$catalogName.not_existing_table", 12, + 11 + s"$catalogName.not_existing_table".length)) } 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")) + } + checkErrorTableNotFound(e, s"`$catalogName`.`not_existing_table`", + ExpectedContext(s"$catalogName.not_existing_table", 12, + 11 + s"$catalogName.not_existing_table".length)) } 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")) + } + checkErrorTableNotFound(e, + UnresolvedAttribute.parseAttributeName(s"$catalogName.not_existing_table") + .map(part => quoteIdentifier(part)).mkString("."), + ExpectedContext(s"$catalogName.not_existing_table", 12, + 11 + s"$catalogName.not_existing_table".length)) } 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")) + } + checkErrorTableNotFound(e, s"`$catalogName`.`not_existing_table`", + ExpectedContext(s"$catalogName.not_existing_table", 12, + 11 + s"$catalogName.not_existing_table".length)) } test("CREATE TABLE with table comment") { diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json index 784b8c04d8972..1474d800f7252 100644 --- a/core/src/main/resources/error/error-classes.json +++ b/core/src/main/resources/error/error-classes.json @@ -396,6 +396,18 @@ } } }, + "INDEX_ALREADY_EXISTS" : { + "message" : [ + "Cannot create the index because it already exists. ." + ], + "sqlState" : "42000" + }, + "INDEX_NOT_FOUND" : { + "message" : [ + "Cannot find the index. ." + ], + "sqlState" : "42000" + }, "INTERNAL_ERROR" : { "message" : [ "" @@ -506,6 +518,28 @@ "More than one row returned by a subquery used as an expression." ] }, + "NAMESPACE_ALREADY_EXISTS" : { + "message" : [ + "Cannot create namespace because it already exists.", + "Choose a different name, drop the existing namespace, or add the IF NOT EXISTS clause to tolerate pre-existing namespace." + ], + "sqlState" : "42000" + }, + "NAMESPACE_NOT_EMPTY" : { + "message" : [ + "Cannot drop a namespace because it contains objects.", + "Use DROP NAMESPACE ... CASCADE to drop the namespace and all its objects." + ], + "sqlState" : "42000" + }, + "NAMESPACE_NOT_FOUND" : { + "message" : [ + "The namespace cannot be found. Verify the spelling and correctness of the namespace.", + "If you did not qualify the name with, verify the current_schema() output, or qualify the name with the correctly.", + "To tolerate the error on drop use DROP NAMESPACE IF EXISTS." + ], + "sqlState" : "42000" + }, "NON_LITERAL_PIVOT_VALUES" : { "message" : [ "Literal expressions required for pivot values, found ." @@ -569,6 +603,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 " @@ -586,12 +635,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 " @@ -1593,21 +1701,6 @@ "Column does not exist." ] }, - "_LEGACY_ERROR_TEMP_1062" : { - "message" : [ - "Rename temporary view from '' to '': destination view already exists." - ] - }, - "_LEGACY_ERROR_TEMP_1063" : { - "message" : [ - "Cannot drop a non-empty database: . Use CASCADE option to drop a non-empty database." - ] - }, - "_LEGACY_ERROR_TEMP_1064" : { - "message" : [ - "Cannot drop a non-empty namespace: . Use CASCADE option to drop a non-empty namespace." - ] - }, "_LEGACY_ERROR_TEMP_1065" : { "message" : [ "`` is not a valid name for tables/databases. Valid names only contain alphabet characters, numbers and _." @@ -1658,21 +1751,11 @@ "RENAME TEMPORARY VIEW from '' to '': cannot specify database name '' in the destination table." ] }, - "_LEGACY_ERROR_TEMP_1075" : { - "message" : [ - "RENAME TEMPORARY VIEW from '' to '': destination table already exists." - ] - }, "_LEGACY_ERROR_TEMP_1076" : { "message" : [ "Partition spec is invalid.
." ] }, - "_LEGACY_ERROR_TEMP_1077" : { - "message" : [ - "Function already exists." - ] - }, "_LEGACY_ERROR_TEMP_1078" : { "message" : [ "Can not load class '' when registering the function '', please make sure it is on the classpath." @@ -1843,11 +1926,6 @@ "DESCRIBE does not support partition for v2 tables." ] }, - "_LEGACY_ERROR_TEMP_1112" : { - "message" : [ - "Table cannot be replaced as it did not exist. Use CREATE OR REPLACE TABLE to create the table." - ] - }, "_LEGACY_ERROR_TEMP_1113" : { "message" : [ "Table
does not support ." @@ -1860,26 +1938,11 @@ "Sources support continuous: " ] }, - "_LEGACY_ERROR_TEMP_1115" : { - "message" : [ - "." - ] - }, - "_LEGACY_ERROR_TEMP_1116" : { - "message" : [ - "." - ] - }, "_LEGACY_ERROR_TEMP_1117" : { "message" : [ " requires a single-part namespace, but got ." ] }, - "_LEGACY_ERROR_TEMP_1118" : { - "message" : [ - "." - ] - }, "_LEGACY_ERROR_TEMP_1119" : { "message" : [ " is not supported in JDBC catalog." @@ -2395,11 +2458,6 @@ "The pattern '' is invalid, ." ] }, - "_LEGACY_ERROR_TEMP_1217" : { - "message" : [ - " already exists." - ] - }, "_LEGACY_ERROR_TEMP_1218" : { "message" : [ " should be converted to HadoopFsRelation." @@ -2501,21 +2559,11 @@ "The list of partition columns with values in partition specification for table '
' in database '' is not a prefix of the list of partition columns defined in the table schema. Expected a prefix of [], but got []." ] }, - "_LEGACY_ERROR_TEMP_1238" : { - "message" : [ - "." - ] - }, "_LEGACY_ERROR_TEMP_1239" : { "message" : [ "Analyzing column statistics is not supported for column of data type: ." ] }, - "_LEGACY_ERROR_TEMP_1240" : { - "message" : [ - "Table
already exists." - ] - }, "_LEGACY_ERROR_TEMP_1241" : { "message" : [ "CREATE-TABLE-AS-SELECT cannot create table with location to a non-empty directory . To allow overwriting the existing non-empty directory, set '' to true." @@ -2601,11 +2649,6 @@ "Cannot refresh temporary function ." ] }, - "_LEGACY_ERROR_TEMP_1258" : { - "message" : [ - "." - ] - }, "_LEGACY_ERROR_TEMP_1259" : { "message" : [ "ALTER ADD COLUMNS does not support views. You must drop and re-create the views for adding the new columns. Views:
." @@ -2838,9 +2881,9 @@ "Boundary end is not a valid integer: ." ] }, - "_LEGACY_ERROR_TEMP_1303" : { + "_LEGACY_ERROR_TEMP_1304" : { "message" : [ - "Table or view '' not found." + "Unexpected type of the relation ." ] }, "_LEGACY_ERROR_TEMP_1305" : { @@ -2888,11 +2931,6 @@ "'' does not support bucketBy and sortBy right now." ] }, - "_LEGACY_ERROR_TEMP_1314" : { - "message" : [ - "Table already exists." - ] - }, "_LEGACY_ERROR_TEMP_1315" : { "message" : [ "Cannot overwrite table that is also being read from." diff --git a/core/src/test/scala/org/apache/spark/SparkFunSuite.scala b/core/src/test/scala/org/apache/spark/SparkFunSuite.scala index a28727532ec49..46b62d879cf3f 100644 --- a/core/src/test/scala/org/apache/spark/SparkFunSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkFunSuite.scala @@ -319,11 +319,16 @@ abstract class SparkFunSuite 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") + assert(actual.objectType() === expected.objectType(), + "Invalid objectType of a query context Actual:" + actual.toString) + assert(actual.objectName() === expected.objectName(), + "Invalid objectName of a query context. Actual:" + actual.toString) + assert(actual.startIndex() === expected.startIndex(), + "Invalid startIndex of a query context. Actual:" + actual.toString) + assert(actual.stopIndex() === expected.stopIndex(), + "Invalid stopIndex of a query context. Actual:" + actual.toString) + assert(actual.fragment() === expected.fragment(), + "Invalid fragment of a query context. Actual:" + actual.toString) } } @@ -374,6 +379,28 @@ abstract class SparkFunSuite checkError(exception, errorClass, sqlState, parameters, matchPVals = true, Array(context)) + protected def checkErrorTableNotFound( + exception: SparkThrowable, + tableName: String, + queryContext: ExpectedContext): Unit = + checkError(exception = exception, + errorClass = "TABLE_OR_VIEW_NOT_FOUND", + parameters = Map("relationName" -> tableName), + queryContext = Array(queryContext)) + + protected def checkErrorTableNotFound( + exception: SparkThrowable, + tableName: String): Unit = + checkError(exception = exception, + errorClass = "TABLE_OR_VIEW_NOT_FOUND", + parameters = Map("relationName" -> tableName)) + + protected def checkErrorTableAlreadyExists(exception: SparkThrowable, + tableName: String): Unit = + checkError(exception = exception, + errorClass = "TABLE_OR_VIEW_ALREADY_EXISTS", + parameters = Map("relationName" -> tableName)) + case class ExpectedContext( objectType: String, objectName: String, 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 6c9e5ac577a7b..3eb9bf9f91349 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 PartitionsAlreadyExistException(e.getMessage()); + throw new PartitionsAlreadyExistException("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 c1dd80e3f77e2..50050d391592d 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,8 +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.connector.catalog.CatalogV2Implicits._ -import org.apache.spark.sql.connector.catalog.Identifier +import org.apache.spark.sql.catalyst.util.{quoteIdentifier, quoteNameParts } import org.apache.spark.sql.types.StructType /** @@ -29,57 +28,104 @@ 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, - errorClass = Some("_LEGACY_ERROR_TEMP_1118"), - messageParameters = Map("msg" -> message)) { + +class NamespaceAlreadyExistsException(errorClass: String, messageParameters: Map[String, String]) + extends AnalysisException(errorClass, messageParameters) { def this(namespace: Array[String]) = { - this(s"Namespace '${namespace.quoted}' already exists") + this(errorClass = "SCHEMA_ALREADY_EXISTS", + Map("schemaName" -> quoteNameParts(namespace))) } } -class TableAlreadyExistsException(message: String, cause: Option[Throwable] = None) - extends AnalysisException( - message, - errorClass = Some("_LEGACY_ERROR_TEMP_1116"), - messageParameters = Map("msg" -> message), - cause = cause) { + +class TableAlreadyExistsException(errorClass: String, messageParameters: Map[String, String], + cause: Option[Throwable] = None) + extends AnalysisException(errorClass, messageParameters, cause = cause) { def this(db: String, table: String) = { - this(s"Table or view '$table' already exists in database '$db'") + this(errorClass = "TABLE_OR_VIEW_ALREADY_EXISTS", + messageParameters = Map("relationName" -> + (quoteIdentifier(db) + "." + quoteIdentifier(table)))) + } + + def this(table: String) = { + this(errorClass = "TABLE_OR_VIEW_ALREADY_EXISTS", + messageParameters = Map("relationName" -> + quoteNameParts(UnresolvedAttribute.parseAttributeName(table)))) } - def this(tableIdent: Identifier) = { - this(s"Table ${tableIdent.quoted} already exists") + def this(table: Seq[String]) = { + this(errorClass = "TABLE_OR_VIEW_ALREADY_EXISTS", + messageParameters = Map("relationName" -> quoteNameParts(table))) + } +} + +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" + -> quoteNameParts(UnresolvedAttribute.parseAttributeName(table)))) } } -class TempTableAlreadyExistsException(table: String) - extends TableAlreadyExistsException(s"Temporary view '$table' already exists") +class PartitionAlreadyExistsException(errorClass: String, messageParameters: Map[String, String]) + extends AnalysisException(errorClass, messageParameters) { + def this(db: String, table: String, spec: TablePartitionSpec) = { + 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(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" -> quoteNameParts(UnresolvedAttribute.parseAttributeName(tableName)))) + } +} -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 exist 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(db: String, table: String, spec: TablePartitionSpec) = this(db, table, Seq(spec)) def this(tableName: String, partitionIdents: Seq[InternalRow], partitionSchema: StructType) = { - this(s"The following partitions already exist in table $tableName:" + - partitionIdents.map(id => partitionSchema.map(_.name).zip(id.toSeq(partitionSchema)) - .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" -> quoteNameParts(UnresolvedAttribute.parseAttributeName(tableName)))) } def this(tableName: String, partitionIdent: InternalRow, partitionSchema: StructType) = this(tableName, Seq(partitionIdent), partitionSchema) } -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" -> quoteNameParts(function))) + } +} class IndexAlreadyExistsException(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/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 097307bce3ded..441e696bfb883 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 @@ -84,7 +84,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.asMultipartIdentifier) } override def createTable( ident: Identifier, diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CannotReplaceMissingTableException.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CannotReplaceMissingTableException.scala index 7a52bc07d5392..910bb9d374971 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CannotReplaceMissingTableException.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CannotReplaceMissingTableException.scala @@ -19,12 +19,13 @@ package org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.util.quoteNameParts import org.apache.spark.sql.connector.catalog.Identifier class CannotReplaceMissingTableException( tableIdentifier: Identifier, cause: Option[Throwable] = None) extends AnalysisException( - errorClass = "_LEGACY_ERROR_TEMP_1112", - messageParameters = Map("table" -> tableIdentifier.toString), - cause = cause) + errorClass = "TABLE_OR_VIEW_NOT_FOUND", + messageParameters = Map("relationName" + -> quoteNameParts(tableIdentifier.namespace :+ tableIdentifier.name))) 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 11ac5ae7a3478..4346f51b613a2 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 @@ -137,21 +137,19 @@ 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 => - 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( @@ -161,12 +159,12 @@ 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] => val tblName = write.table.asInstanceOf[UnresolvedRelation].multipartIdentifier - write.table.failAnalysis(s"Table or view not found: ${tblName.quoted}") + write.table.tableNotFound(tblName) case command: V2PartitionCommand => command.table match { 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 6b4353acc96bf..474bf9915e86a 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, quoteNameParts} import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ import org.apache.spark.sql.connector.catalog.Identifier import org.apache.spark.sql.types.StructType @@ -30,95 +31,104 @@ 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" -> quoteNameParts(namespace))) + } def this(namespace: Array[String]) = { - this(s"Namespace '${namespace.quoted}' not found") + this(errorClass = "SCHEMA_NOT_FOUND", + Map("schemaName" -> quoteNameParts(namespace))) } } -case class NoSuchTableException( - override val message: String, - override val cause: Option[Throwable] = None) - extends AnalysisException( - message, - errorClass = Some("_LEGACY_ERROR_TEMP_1115"), - messageParameters = Map("msg" -> 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(tableIdent: Identifier) = { - this(s"Table ${tableIdent.quoted} not found") + def this(name : Seq[String]) = { + this(errorClass = "TABLE_OR_VIEW_NOT_FOUND", + messageParameters = Map("relationName" -> quoteNameParts(name))) } - def this(nameParts: Seq[String]) = { - this(s"Table ${nameParts.quoted} not found") + def this(table: String) = { + this(errorClass = "TABLE_OR_VIEW_NOT_FOUND", + messageParameters = Map("relationName" -> + quoteNameParts(UnresolvedAttribute.parseAttributeName(table)))) } } -case class NoSuchPartitionException( - override val message: String) - extends AnalysisException( - message, - errorClass = Some("_LEGACY_ERROR_TEMP_1238"), - messageParameters = Map("msg" -> 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" -> quoteNameParts(UnresolvedAttribute.parseAttributeName(tableName)))) } } -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) - extends AnalysisException( - message, - errorClass = Some("_LEGACY_ERROR_TEMP_1258"), - messageParameters = Map("msg" -> message)) { +class NoSuchFunctionException(errorClass: String, messageParameters: Map[String, String]) + extends AnalysisException(errorClass, messageParameters) { def this(db: String, func: String) = { - this(s"Undefined function: '$func'. " + - "This function is neither a registered temporary function nor " + - s"a permanent function registered in the database '$db'.") + this(errorClass = "ROUTINE_NOT_FOUND", + Map("routineName" -> (quoteIdentifier(db) + "." + quoteIdentifier(func)))) } def this(identifier: Identifier) = { - this(s"Undefined function: ${identifier.quoted}") + 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" -> quoteNameParts(UnresolvedAttribute.parseAttributeName(tableName)))) } } -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 f82350c16f62b..2e6f8dac59274 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._ import org.apache.spark.sql.catalyst.trees.TreeNode +import org.apache.spark.sql.catalyst.util.quoteNameParts import org.apache.spark.sql.errors.QueryErrorsBase /** @@ -67,6 +68,20 @@ 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" -> quoteNameParts(name)), + origin = t.origin) + } + + def schemaNotFound(name: Seq[String]): Nothing = { + throw new AnalysisException( + errorClass = "SCHEMA_NOT_FOUND", + messageParameters = Map("schemaName" -> quoteNameParts(name)), + 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 bc01986afdb14..bd1e6da217fef 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 @@ -1395,7 +1395,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)) } } 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..2f818fecad93a 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 nameParts: 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/catalyst/util/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala index 8d8bbb088de90..08d84c927e44e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala @@ -130,6 +130,10 @@ package object util extends Logging { "`" + name.replace("`", "``") + "`" } + def quoteNameParts(name: Seq[String]): String = { + name.map(part => quoteIdentifier(part)).mkString(".") + } + def quoteIfNeeded(part: String): String = { if (part.matches("[a-zA-Z0-9_]+") && !part.matches("\\d+")) { part 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 0c6aeedfc4acb..7f143a00b9c19 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 @@ -24,14 +24,14 @@ import org.apache.hadoop.fs.Path import org.apache.spark.SparkThrowableHelper 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, CreateStruct, Expression, GroupingID, NamedExpression, SpecifiedWindowFrame, WindowFrame, WindowFunction, WindowSpecDefinition} import org.apache.spark.sql.catalyst.plans.JoinType import org.apache.spark.sql.catalyst.plans.logical.{InsertIntoStatement, Join, LogicalPlan, SerdeInfo, Window} import org.apache.spark.sql.catalyst.trees.{Origin, TreeNode} -import org.apache.spark.sql.catalyst.util.{FailFastMode, ParseMode, PermissiveMode} +import org.apache.spark.sql.catalyst.util.{quoteIdentifier, FailFastMode, ParseMode, PermissiveMode} import org.apache.spark.sql.connector.catalog._ import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ import org.apache.spark.sql.connector.catalog.functions.{BoundFunction, UnboundFunction} @@ -807,21 +807,17 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase { } def renameTempViewToExistingViewError(oldName: String, newName: String): Throwable = { - new AnalysisException( - errorClass = "_LEGACY_ERROR_TEMP_1062", - messageParameters = Map("oldName" -> oldName, "newName" -> newName)) + new TableAlreadyExistsException(newName) } def cannotDropNonemptyDatabaseError(db: String): Throwable = { - new AnalysisException( - errorClass = "_LEGACY_ERROR_TEMP_1063", - messageParameters = Map("db" -> db)) + new AnalysisException(errorClass = "SCHEMA_NOT_EMPTY", + Map("schemaName" -> toSQLId(db))) } def cannotDropNonemptyNamespaceError(namespace: Seq[String]): Throwable = { - new AnalysisException( - errorClass = "_LEGACY_ERROR_TEMP_1064", - messageParameters = Map("namespace" -> namespace.quoted)) + new AnalysisException(errorClass = "SCHEMA_NOT_EMPTY", + Map("schemaName" -> namespace.map(part => quoteIdentifier(part)).mkString("."))) } def invalidNameForTableOrDatabaseError(name: String): Throwable = { @@ -897,11 +893,7 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase { def cannotRenameTempViewToExistingTableError( oldName: TableIdentifier, newName: TableIdentifier): Throwable = { - new AnalysisException( - errorClass = "_LEGACY_ERROR_TEMP_1075", - messageParameters = Map( - "oldName" -> oldName.toString, - "newName" -> newName.toString)) + new TableAlreadyExistsException(newName.nameParts) } def invalidPartitionSpecError(details: String): Throwable = { @@ -911,9 +903,7 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase { } def functionAlreadyExistsError(func: FunctionIdentifier): Throwable = { - new AnalysisException( - errorClass = "_LEGACY_ERROR_TEMP_1077", - messageParameters = Map("func" -> func.toString)) + new FunctionAlreadyExistsException(func.nameParts) } def cannotLoadClassWhenRegisteringFunctionError( @@ -1223,7 +1213,7 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase { } def noSuchTableError(ident: Identifier): NoSuchTableException = { - new NoSuchTableException(ident) + new NoSuchTableException(ident.asMultipartIdentifier) } def noSuchTableError(nameParts: Seq[String]): Throwable = { @@ -1235,7 +1225,7 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase { } def tableAlreadyExistsError(ident: Identifier): Throwable = { - new TableAlreadyExistsException(ident) + new TableAlreadyExistsException(ident.asMultipartIdentifier) } def requiresSinglePartNamespaceError(ns: Seq[String]): Throwable = { @@ -2148,9 +2138,7 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase { } def tableIdentifierExistsError(tableIdentifier: TableIdentifier): Throwable = { - new AnalysisException( - errorClass = "_LEGACY_ERROR_TEMP_1217", - messageParameters = Map("tableIdentifier" -> tableIdentifier.toString)) + new TableAlreadyExistsException(tableIdentifier.nameParts) } def tableIdentifierNotConvertedToHadoopFsRelationError( @@ -2342,12 +2330,8 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase { "dataType" -> dataType.toString)) } - def tableAlreadyExistsError(table: String, guide: String = ""): Throwable = { - new AnalysisException( - errorClass = "_LEGACY_ERROR_TEMP_1240", - messageParameters = Map( - "table" -> table, - "guide" -> guide)) + def tableAlreadyExistsError(table: String): Throwable = { + new TableAlreadyExistsException(table) } def createTableAsSelectWithNonEmptyDirectoryError(tablePath: String): Throwable = { @@ -2358,6 +2342,10 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase { "config" -> SQLConf.ALLOW_NON_EMPTY_LOCATION_IN_CTAS.key)) } + def tableOrViewNotFoundError(table: String): Throwable = { + new NoSuchTableException(table) + } + def noSuchFunctionError( rawName: Seq[String], t: TreeNode[_], @@ -2845,9 +2833,7 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase { } def tableOrViewNotFound(ident: Seq[String]): Throwable = { - new AnalysisException( - errorClass = "_LEGACY_ERROR_TEMP_1303", - messageParameters = Map("ident" -> ident.quoted)) + new NoSuchTableException(ident) } def unsupportedTableChangeInJDBCCatalogError(change: TableChange): Throwable = { @@ -2909,10 +2895,7 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase { } def tableAlreadyExistsError(tableIdent: TableIdentifier): Throwable = { - new AnalysisException( - errorClass = "_LEGACY_ERROR_TEMP_1314", - messageParameters = Map("tableIdent" -> tableIdent.toString)) - new AnalysisException(s"Table $tableIdent already exists.") + new TableAlreadyExistsException(tableIdent.nameParts) } 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 ac823183ce9da..a4bce2f2a6c48 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 @@ -119,7 +119,8 @@ class AnalysisErrorSuite extends AnalysisTest { messageParameters: Map[String, String], caseSensitive: Boolean = true): Unit = { test(name) { - assertAnalysisErrorClass(plan, errorClass, messageParameters, caseSensitive) + assertAnalysisErrorClass(plan, errorClass, messageParameters, + caseSensitive = true, line = -1, pos = -1) } } @@ -809,10 +810,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, @@ -832,7 +835,9 @@ class AnalysisErrorSuite extends AnalysisTest { "inputSql" -> inputSql, "inputType" -> inputType, "requiredType" -> "(\"INT\" or \"BIGINT\")"), - caseSensitive = false) + caseSensitive = false, + line = -1, + pos = -1) } } 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 785d5ae05cfff..7b720a7a0472b 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 @@ -48,9 +48,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") } @@ -73,26 +76,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 f74cdab55443a..3036742c83f8c 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 @@ -105,7 +105,9 @@ class AnalysisSuite extends AnalysisTest with Matchers { SubqueryAlias("TbL", UnresolvedRelation(TableIdentifier("TaBlE")))), "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")), @@ -714,7 +716,9 @@ class AnalysisSuite extends AnalysisTest with Matchers { assertAnalysisErrorClass(parsePlan("WITH t(x) AS (SELECT 1) SELECT * FROM t WHERE y = 1"), "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") { @@ -724,8 +728,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") { @@ -1153,7 +1157,9 @@ class AnalysisSuite extends AnalysisTest with Matchers { |""".stripMargin), "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") { @@ -1193,7 +1199,9 @@ class AnalysisSuite extends AnalysisTest with Matchers { "inputType" -> "\"BOOLEAN\"", "requiredType" -> "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\")"), - caseSensitive = false) + caseSensitive = false, + line = -1, + pos = -1) assertAnalysisErrorClass( inputPlan = parsePlan( @@ -1211,19 +1219,20 @@ class AnalysisSuite extends AnalysisTest with Matchers { "inputType" -> "\"BOOLEAN\"", "requiredType" -> "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\")"), - caseSensitive = false) + caseSensitive = false, + line = -1, + pos = -1) } - 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 1ac8d4222a22a..f2697d4ca3b04 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,7 +174,9 @@ trait AnalysisTest extends PlanTest { inputPlan: LogicalPlan, expectedErrorClass: String, expectedMessageParameters: Map[String, String], - caseSensitive: Boolean): Unit = { + caseSensitive: Boolean = true, + line: Int = -1, + pos: Int = -1): Unit = { withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive.toString) { val analyzer = getAnalyzer val e = intercept[AnalysisException] { @@ -182,7 +184,9 @@ trait AnalysisTest extends PlanTest { } if (e.getErrorClass != expectedErrorClass || - !e.messageParameters.sameElements(expectedMessageParameters)) { + !e.messageParameters.sameElements(expectedMessageParameters) || + (line >= 0 && e.line.getOrElse(-1) != line) || + (pos >= 0) && e.startPosition.getOrElse(-1) != pos) { var failMsg = "" if (e.getErrorClass != expectedErrorClass) { failMsg += @@ -196,6 +200,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 42db7c46181fb..716d7aeb60fb7 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 @@ -135,7 +135,9 @@ class ResolveSubquerySuite extends AnalysisTest { lateralJoin(t1, lateralJoin(t2, t0.select($"a", $"b", $"c"))), "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", Map("objectName" -> "`a`"), - caseSensitive = true) + caseSensitive = true, + line = -1, + pos = -1) } test("lateral subquery with unresolvable attributes") { @@ -144,25 +146,33 @@ class ResolveSubquerySuite extends AnalysisTest { lateralJoin(t1, t0.select($"a", $"c")), "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 b0952b7f4f491..b875318329700 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 @@ -692,7 +692,9 @@ abstract class V2WriteAnalysisSuiteBase extends AnalysisTest { parsedPlan, "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), @@ -705,7 +707,9 @@ abstract class V2WriteAnalysisSuiteBase extends AnalysisTest { parsedPlan2, "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 6ed14bd641653..d846162b7842c 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,8 +1467,10 @@ 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")) + } + checkError(e, + errorClass = "ROUTINE_ALREADY_EXISTS", + parameters = Map("routineName" -> "`temp1`")) // Temporary function is overridden catalog.registerFunction( newFunc("temp1", None), overrideIfExists = true, functionBuilder = Some(tempFunc3)) 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..032b04bb887de 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 @@ -26,6 +26,7 @@ import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.{NamespaceAlreadyExistsException, NoSuchFunctionException, NoSuchNamespaceException, NoSuchTableException, TableAlreadyExistsException} import org.apache.spark.sql.catalyst.parser.CatalystSqlParser +import org.apache.spark.sql.catalyst.util.quoteIdentifier import org.apache.spark.sql.connector.catalog.functions.{BoundFunction, ScalarFunction, UnboundFunction} import org.apache.spark.sql.connector.expressions.LogicalExpressions import org.apache.spark.sql.internal.SQLConf @@ -48,7 +49,12 @@ class CatalogSuite extends SparkFunSuite { private val testNs = Array("`", ".") private val testIdent = Identifier.of(testNs, "test_table") + private val testIdentQuoted = testIdent.asMultipartIdentifier + .map(part => quoteIdentifier(part)).mkString(".") + private val testIdentNew = Identifier.of(testNs, "test_table_new") + private val testIdentNewQuoted = testIdentNew.asMultipartIdentifier + .map(part => quoteIdentifier(part)).mkString(".") test("Catalogs can load the catalog") { val catalog = newCatalog() @@ -133,8 +139,7 @@ class CatalogSuite extends SparkFunSuite { catalog.createTable(testIdent, schema, Array.empty, emptyProps) } - assert(exc.message.contains(testIdent.quoted)) - assert(exc.message.contains("already exists")) + checkErrorTableAlreadyExists(exc, testIdentQuoted) assert(catalog.tableExists(testIdent)) } @@ -171,8 +176,7 @@ class CatalogSuite extends SparkFunSuite { catalog.loadTable(testIdent) } - assert(exc.message.contains(testIdent.quoted)) - assert(exc.message.contains("not found")) + checkErrorTableNotFound(exc, testIdentQuoted) } test("invalidateTable") { @@ -623,8 +627,7 @@ class CatalogSuite extends SparkFunSuite { catalog.alterTable(testIdent, TableChange.setProperty("prop", "val")) } - assert(exc.message.contains(testIdent.quoted)) - assert(exc.message.contains("not found")) + checkErrorTableNotFound(exc, testIdentQuoted) } test("dropTable") { @@ -680,8 +683,7 @@ class CatalogSuite extends SparkFunSuite { catalog.renameTable(testIdent, testIdentNew) } - assert(exc.message.contains(testIdent.quoted)) - assert(exc.message.contains("not found")) + checkErrorTableNotFound(exc, testIdentQuoted) } test("renameTable: fail if new table name already exists") { @@ -700,8 +702,7 @@ class CatalogSuite extends SparkFunSuite { catalog.renameTable(testIdent, testIdentNew) } - assert(exc.message.contains(testIdentNew.quoted)) - assert(exc.message.contains("already exists")) + checkErrorTableAlreadyExists(exc, testIdentNewQuoted) } test("listNamespaces: list namespaces from metadata") { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryPartitionTableCatalog.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryPartitionTableCatalog.scala index a24f5c9a0c463..9a45d64209837 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryPartitionTableCatalog.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryPartitionTableCatalog.scala @@ -32,7 +32,7 @@ class InMemoryPartitionTableCatalog extends InMemoryTableCatalog { partitions: Array[Transform], properties: util.Map[String, String]): Table = { if (tables.containsKey(ident)) { - throw new TableAlreadyExistsException(ident) + throw new TableAlreadyExistsException(ident.asMultipartIdentifier) } InMemoryTableCatalog.maybeSimulateFailedTableCreation(properties) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryRowLevelOperationTableCatalog.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryRowLevelOperationTableCatalog.scala index 2d9a9f04785e7..94e6947612219 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryRowLevelOperationTableCatalog.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryRowLevelOperationTableCatalog.scala @@ -32,7 +32,7 @@ class InMemoryRowLevelOperationTableCatalog extends InMemoryTableCatalog { partitions: Array[Transform], properties: util.Map[String, String]): Table = { if (tables.containsKey(ident)) { - throw new TableAlreadyExistsException(ident) + throw new TableAlreadyExistsException(ident.asMultipartIdentifier) } InMemoryTableCatalog.maybeSimulateFailedTableCreation(properties) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryTableCatalog.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryTableCatalog.scala index 3736ba4d785af..af8070652da2d 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryTableCatalog.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryTableCatalog.scala @@ -56,7 +56,7 @@ class BasicInMemoryTableCatalog extends TableCatalog { case Some(table) => table case _ => - throw new NoSuchTableException(ident) + throw new NoSuchTableException(ident.asMultipartIdentifier) } } @@ -66,7 +66,7 @@ class BasicInMemoryTableCatalog extends TableCatalog { case Some(table) => table case _ => - throw new NoSuchTableException(ident) + throw new NoSuchTableException(ident.asMultipartIdentifier) } } @@ -76,7 +76,7 @@ class BasicInMemoryTableCatalog extends TableCatalog { case Some(table) => table case _ => - throw new NoSuchTableException(ident) + throw new NoSuchTableException(ident.asMultipartIdentifier) } } @@ -103,7 +103,7 @@ class BasicInMemoryTableCatalog extends TableCatalog { requiredNumPartitions: Option[Int], distributionStrictlyRequired: Boolean = true): Table = { if (tables.containsKey(ident)) { - throw new TableAlreadyExistsException(ident) + throw new TableAlreadyExistsException(ident.asMultipartIdentifier) } InMemoryTableCatalog.maybeSimulateFailedTableCreation(properties) @@ -138,14 +138,14 @@ class BasicInMemoryTableCatalog extends TableCatalog { override def renameTable(oldIdent: Identifier, newIdent: Identifier): Unit = { if (tables.containsKey(newIdent)) { - throw new TableAlreadyExistsException(newIdent) + throw new TableAlreadyExistsException(newIdent.asMultipartIdentifier) } Option(tables.remove(oldIdent)) match { case Some(table) => tables.put(newIdent, table) case _ => - throw new NoSuchTableException(oldIdent) + throw new NoSuchTableException(oldIdent.asMultipartIdentifier) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryTableWithV2FilterCatalog.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryTableWithV2FilterCatalog.scala index 08c1f65db290c..240550fdcf99d 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryTableWithV2FilterCatalog.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryTableWithV2FilterCatalog.scala @@ -32,7 +32,7 @@ class InMemoryTableWithV2FilterCatalog extends InMemoryTableCatalog { partitions: Array[Transform], properties: util.Map[String, String]): Table = { if (tables.containsKey(ident)) { - throw new TableAlreadyExistsException(ident) + throw new TableAlreadyExistsException(ident.asMultipartIdentifier) } InMemoryTableCatalog.maybeSimulateFailedTableCreation(properties) 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..8038bb335d91c 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(ident.asMultipartIdentifier) } } } 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 7f7c529944501..ddd08185527e8 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[PartitionsAlreadyExistException] { + val e = intercept[PartitionsAlreadyExistException] { partTable.renamePartition(InternalRow(0, "abc"), InternalRow(1, "abc")) - }.getMessage - assert(errMsg1.contains("partitions already exist")) + } + 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/jdbc/H2Dialect.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/H2Dialect.scala index 8783d0da6b9dd..0b735dd0e462e 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,8 @@ 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.catalyst.util.quoteNameParts 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 +187,25 @@ private[sql] object H2Dialect extends JdbcDialect { exception.getErrorCode match { // TABLE_OR_VIEW_ALREADY_EXISTS_1 case 42101 => - throw new TableAlreadyExistsException(message, cause = Some(e)) + // The message is: Table "identifier" already exists + val regex = """"((?:[^"\\]|\\[\\"ntbrf])+)"""".r + val name = regex.findFirstMatchIn(e.getMessage).get.group(1) + val quotedName = org.apache.spark.sql.catalyst.util.quoteIdentifier(name) + 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 = quoteNameParts(UnresolvedAttribute.parseAttributeName(message)) + 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 regex = """"((?:[^"\\]|\\[\\"ntbrf])+)"""".r + val name = regex.findFirstMatchIn(e.getMessage).get.group(1) + val quotedName = org.apache.spark.sql.catalyst.util.quoteIdentifier(name) + 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/ansi/double-quoted-identifiers-disabled.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/double-quoted-identifiers-disabled.sql.out index 57fad89d57c94..f34070baa6979 100644 --- a/sql/core/src/test/resources/sql-tests/results/ansi/double-quoted-identifiers-disabled.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/ansi/double-quoted-identifiers-disabled.sql.out @@ -133,7 +133,20 @@ SELECT 1 FROM `hello` struct<> -- !query output org.apache.spark.sql.AnalysisException -Table or view not found: hello; line 1 pos 14 +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42000", + "messageParameters" : { + "relationName" : "`hello`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 21, + "fragment" : "`hello`" + } ] +} -- !query @@ -142,7 +155,13 @@ USE SCHEMA `not_exist` struct<> -- !query output org.apache.spark.sql.catalyst.analysis.NoSuchDatabaseException -Database 'not_exist' not found +{ + "errorClass" : "SCHEMA_NOT_FOUND", + "sqlState" : "42000", + "messageParameters" : { + "schemaName" : "`not_exist`" + } +} -- !query @@ -151,7 +170,20 @@ ALTER TABLE `not_exist` ADD COLUMN not_exist int struct<> -- !query output org.apache.spark.sql.AnalysisException -Table not found: not_exist; line 1 pos 12 +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42000", + "messageParameters" : { + "relationName" : "`not_exist`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 13, + "stopIndex" : 23, + "fragment" : "`not_exist`" + } ] +} -- !query @@ -160,7 +192,20 @@ ALTER TABLE not_exist ADD COLUMN `not_exist` int struct<> -- !query output org.apache.spark.sql.AnalysisException -Table not found: not_exist; line 1 pos 12 +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42000", + "messageParameters" : { + "relationName" : "`not_exist`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 13, + "stopIndex" : 21, + "fragment" : "not_exist" + } ] +} -- !query @@ -169,7 +214,20 @@ SELECT 1 AS `not_exist` FROM `not_exist` struct<> -- !query output org.apache.spark.sql.AnalysisException -Table or view not found: not_exist; line 1 pos 29 +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42000", + "messageParameters" : { + "relationName" : "`not_exist`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 30, + "stopIndex" : 40, + "fragment" : "`not_exist`" + } ] +} -- !query @@ -178,7 +236,20 @@ SELECT 1 FROM not_exist AS X(`hello`) struct<> -- !query output org.apache.spark.sql.AnalysisException -Table or view not found: not_exist; line 1 pos 14 +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42000", + "messageParameters" : { + "relationName" : "`not_exist`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 37, + "fragment" : "not_exist AS X(`hello`)" + } ] +} -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/double-quoted-identifiers-enabled.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/double-quoted-identifiers-enabled.sql.out index fb34e9a16197a..426951377f861 100644 --- a/sql/core/src/test/resources/sql-tests/results/ansi/double-quoted-identifiers-enabled.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/ansi/double-quoted-identifiers-enabled.sql.out @@ -5,7 +5,20 @@ SELECT 1 FROM "not_exist" struct<> -- !query output org.apache.spark.sql.AnalysisException -Table or view not found: not_exist; line 1 pos 14 +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42000", + "messageParameters" : { + "relationName" : "`not_exist`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 25, + "fragment" : "\"not_exist\"" + } ] +} -- !query @@ -14,7 +27,13 @@ USE SCHEMA "not_exist" struct<> -- !query output org.apache.spark.sql.catalyst.analysis.NoSuchDatabaseException -Database 'not_exist' not found +{ + "errorClass" : "SCHEMA_NOT_FOUND", + "sqlState" : "42000", + "messageParameters" : { + "schemaName" : "`not_exist`" + } +} -- !query @@ -23,7 +42,20 @@ ALTER TABLE "not_exist" ADD COLUMN not_exist int struct<> -- !query output org.apache.spark.sql.AnalysisException -Table not found: not_exist; line 1 pos 12 +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42000", + "messageParameters" : { + "relationName" : "`not_exist`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 13, + "stopIndex" : 23, + "fragment" : "\"not_exist\"" + } ] +} -- !query @@ -32,7 +64,20 @@ ALTER TABLE not_exist ADD COLUMN "not_exist" int struct<> -- !query output org.apache.spark.sql.AnalysisException -Table not found: not_exist; line 1 pos 12 +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42000", + "messageParameters" : { + "relationName" : "`not_exist`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 13, + "stopIndex" : 21, + "fragment" : "not_exist" + } ] +} -- !query @@ -41,7 +86,20 @@ SELECT 1 AS "not_exist" FROM not_exist struct<> -- !query output org.apache.spark.sql.AnalysisException -Table or view not found: not_exist; line 1 pos 29 +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42000", + "messageParameters" : { + "relationName" : "`not_exist`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 30, + "stopIndex" : 38, + "fragment" : "not_exist" + } ] +} -- !query @@ -50,7 +108,20 @@ SELECT 1 FROM not_exist AS X("hello") struct<> -- !query output org.apache.spark.sql.AnalysisException -Table or view not found: not_exist; line 1 pos 14 +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42000", + "messageParameters" : { + "relationName" : "`not_exist`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 37, + "fragment" : "not_exist AS X(\"hello\")" + } ] +} -- !query @@ -102,7 +173,20 @@ SELECT 1 FROM `hello` struct<> -- !query output org.apache.spark.sql.AnalysisException -Table or view not found: hello; line 1 pos 14 +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42000", + "messageParameters" : { + "relationName" : "`hello`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 21, + "fragment" : "`hello`" + } ] +} -- !query @@ -111,7 +195,13 @@ USE SCHEMA `not_exist` struct<> -- !query output org.apache.spark.sql.catalyst.analysis.NoSuchDatabaseException -Database 'not_exist' not found +{ + "errorClass" : "SCHEMA_NOT_FOUND", + "sqlState" : "42000", + "messageParameters" : { + "schemaName" : "`not_exist`" + } +} -- !query @@ -120,7 +210,20 @@ ALTER TABLE `not_exist` ADD COLUMN not_exist int struct<> -- !query output org.apache.spark.sql.AnalysisException -Table not found: not_exist; line 1 pos 12 +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42000", + "messageParameters" : { + "relationName" : "`not_exist`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 13, + "stopIndex" : 23, + "fragment" : "`not_exist`" + } ] +} -- !query @@ -129,7 +232,20 @@ ALTER TABLE not_exist ADD COLUMN `not_exist` int struct<> -- !query output org.apache.spark.sql.AnalysisException -Table not found: not_exist; line 1 pos 12 +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42000", + "messageParameters" : { + "relationName" : "`not_exist`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 13, + "stopIndex" : 21, + "fragment" : "not_exist" + } ] +} -- !query @@ -138,7 +254,20 @@ SELECT 1 AS `not_exist` FROM `not_exist` struct<> -- !query output org.apache.spark.sql.AnalysisException -Table or view not found: not_exist; line 1 pos 29 +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42000", + "messageParameters" : { + "relationName" : "`not_exist`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 30, + "stopIndex" : 40, + "fragment" : "`not_exist`" + } ] +} -- !query @@ -147,7 +276,20 @@ SELECT 1 FROM not_exist AS X(`hello`) struct<> -- !query output org.apache.spark.sql.AnalysisException -Table or view not found: not_exist; line 1 pos 14 +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42000", + "messageParameters" : { + "relationName" : "`not_exist`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 37, + "fragment" : "not_exist AS X(`hello`)" + } ] +} -- !query @@ -238,9 +380,10 @@ struct<> -- !query output org.apache.spark.sql.catalyst.analysis.NoSuchTableException { - "errorClass" : "_LEGACY_ERROR_TEMP_1115", + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42000", "messageParameters" : { - "msg" : "Table spark_catalog.default.v not found" + "relationName" : "`spark_catalog`.`default`.`v`" } } diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/map.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/map.sql.out index c7c6f5578e76d..a550dbbec8820 100644 --- a/sql/core/src/test/resources/sql-tests/results/ansi/map.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/ansi/map.sql.out @@ -110,4 +110,4 @@ org.apache.spark.sql.AnalysisException "stopIndex" : 49, "fragment" : "map_contains_key(map(1, 'a', 2, 'b'), '1')" } ] -} \ No newline at end of file +} 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 75eb280900679..ed6d69b233ecb 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 @@ -344,7 +344,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 @@ -366,4 +379,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 eb57b49119c95..1c9a54ec6d029 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 @@ -170,7 +209,20 @@ SELECT 2 struct<> -- !query output org.apache.spark.sql.AnalysisException -Table or view not found: non_existing_table; line 1 pos 25 +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42000", + "messageParameters" : { + "relationName" : "`non_existing_table`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 26, + "stopIndex" : 43, + "fragment" : "non_existing_table" + } ] +} -- !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 5c689cacf4d5f..518bd1c1ed458 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 @@ -363,9 +363,11 @@ struct<> -- !query output org.apache.spark.sql.catalyst.analysis.NoSuchPartitionException { - "errorClass" : "_LEGACY_ERROR_TEMP_1238", + "errorClass" : "PARTITIONS_NOT_FOUND", + "sqlState" : "42000", "messageParameters" : { - "msg" : "Partition not found in table 't' database 'default':\nc -> Us\nd -> 2" + "partitionList" : "PARTITION (`c` = Us, `d` = 2)", + "tableName" : "`default`.`t`" } } diff --git a/sql/core/src/test/resources/sql-tests/results/double-quoted-identifiers.sql.out b/sql/core/src/test/resources/sql-tests/results/double-quoted-identifiers.sql.out index 57fad89d57c94..f34070baa6979 100644 --- a/sql/core/src/test/resources/sql-tests/results/double-quoted-identifiers.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/double-quoted-identifiers.sql.out @@ -133,7 +133,20 @@ SELECT 1 FROM `hello` struct<> -- !query output org.apache.spark.sql.AnalysisException -Table or view not found: hello; line 1 pos 14 +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42000", + "messageParameters" : { + "relationName" : "`hello`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 21, + "fragment" : "`hello`" + } ] +} -- !query @@ -142,7 +155,13 @@ USE SCHEMA `not_exist` struct<> -- !query output org.apache.spark.sql.catalyst.analysis.NoSuchDatabaseException -Database 'not_exist' not found +{ + "errorClass" : "SCHEMA_NOT_FOUND", + "sqlState" : "42000", + "messageParameters" : { + "schemaName" : "`not_exist`" + } +} -- !query @@ -151,7 +170,20 @@ ALTER TABLE `not_exist` ADD COLUMN not_exist int struct<> -- !query output org.apache.spark.sql.AnalysisException -Table not found: not_exist; line 1 pos 12 +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42000", + "messageParameters" : { + "relationName" : "`not_exist`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 13, + "stopIndex" : 23, + "fragment" : "`not_exist`" + } ] +} -- !query @@ -160,7 +192,20 @@ ALTER TABLE not_exist ADD COLUMN `not_exist` int struct<> -- !query output org.apache.spark.sql.AnalysisException -Table not found: not_exist; line 1 pos 12 +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42000", + "messageParameters" : { + "relationName" : "`not_exist`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 13, + "stopIndex" : 21, + "fragment" : "not_exist" + } ] +} -- !query @@ -169,7 +214,20 @@ SELECT 1 AS `not_exist` FROM `not_exist` struct<> -- !query output org.apache.spark.sql.AnalysisException -Table or view not found: not_exist; line 1 pos 29 +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42000", + "messageParameters" : { + "relationName" : "`not_exist`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 30, + "stopIndex" : 40, + "fragment" : "`not_exist`" + } ] +} -- !query @@ -178,7 +236,20 @@ SELECT 1 FROM not_exist AS X(`hello`) struct<> -- !query output org.apache.spark.sql.AnalysisException -Table or view not found: not_exist; line 1 pos 14 +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42000", + "messageParameters" : { + "relationName" : "`not_exist`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 37, + "fragment" : "not_exist AS X(`hello`)" + } ] +} -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/map.sql.out b/sql/core/src/test/resources/sql-tests/results/map.sql.out index c7c6f5578e76d..a550dbbec8820 100644 --- a/sql/core/src/test/resources/sql-tests/results/map.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/map.sql.out @@ -110,4 +110,4 @@ org.apache.spark.sql.AnalysisException "stopIndex" : 49, "fragment" : "map_contains_key(map(1, 'a', 2, 'b'), '1')" } ] -} \ No newline at end of file +} 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 aa37c0cc7fded..1e1c5b36a6235 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 @@ -636,7 +636,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 @@ -645,7 +658,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 @@ -654,7 +680,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 f6f3fcee20bac..8cff1ab350117 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 @@ -396,7 +409,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 7170a8d608150..fa1d76fd40bf3 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 @@ -210,9 +210,10 @@ struct<> -- !query output org.apache.spark.sql.catalyst.analysis.NoSuchTableException { - "errorClass" : "_LEGACY_ERROR_TEMP_1115", + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42000", "messageParameters" : { - "msg" : "Table or view 'show_t*' not found in database 'showdb'" + "relationName" : "`showdb`.`show_t*`" } } @@ -255,9 +256,11 @@ struct<> -- !query output org.apache.spark.sql.catalyst.analysis.NoSuchPartitionException { - "errorClass" : "_LEGACY_ERROR_TEMP_1238", + "errorClass" : "PARTITIONS_NOT_FOUND", + "sqlState" : "42000", "messageParameters" : { - "msg" : "Partition not found in table 'show_t1' database 'showdb':\nc -> Ch\nd -> 1" + "partitionList" : "PARTITION (`c` = Ch, `d` = 1)", + "tableName" : "`showdb`.`show_t1`" } } 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 7624d1800f52e..dfe256a79e775 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 @@ -133,7 +146,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 @@ -142,7 +168,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 @@ -151,7 +190,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/resources/sql-tests/results/typeCoercion/native/mapconcat.sql.out b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/mapconcat.sql.out index 5c1f5d4d917f5..0e2b0cf2789ed 100644 --- a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/mapconcat.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/mapconcat.sql.out @@ -40,6 +40,7 @@ struct<> -- !query output + -- !query SELECT map_concat(boolean_map1, boolean_map2) boolean_map, @@ -204,4 +205,4 @@ org.apache.spark.sql.AnalysisException "stopIndex" : 43, "fragment" : "map_concat(int_map1, array_map2)" } ] -} \ No newline at end of file +} 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..c7c09bf7c7940 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,8 @@ 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")) + checkErrorTableNotFound(ex, "`v1`", + ExpectedContext("v1", 29, 30)) } } 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..5548108b91508 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,8 @@ 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")) + checkErrorTableNotFound(e, "`t1`", + ExpectedContext("VIEW", "t2", 14, 15, "t1")) } } } @@ -993,7 +996,8 @@ 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")) + checkErrorTableNotFound(e, "`t1`", + ExpectedContext("VIEW", "t2", 14, 15, "t1")) } } } @@ -1431,7 +1435,8 @@ 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")) + checkErrorTableNotFound(e, s"`$t`", + ExpectedContext("VIEW", "v", 14, 13 + t.length, 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 cb453902ce9e5..fabd0a4e1a951 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 @@ -2891,7 +2891,8 @@ 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:")) + checkErrorTableNotFound(e, "`nonexist`.`t`", + ExpectedContext("nonexist.t", 25, 34)) } test("SPARK-32680: Don't analyze CTAS with unresolved query") { @@ -2899,7 +2900,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:")) + checkErrorTableNotFound(e, "`nonexist`", + ExpectedContext("nonexist", s"CREATE TABLE t USING $v2Source AS SELECT * from ".length, + s"CREATE TABLE t USING $v2Source AS SELECT * from nonexist".length - 1)) } 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..913a77cedb7fb 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,7 @@ 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")) + checkErrorTableNotFound(exc, "`testcat`.`table_name`") } test("Append: fail if it writes to a temp view that is not v2 relation") { @@ -261,7 +261,7 @@ 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")) + checkErrorTableNotFound(exc, "`testcat`.`table_name`") } test("Overwrite: fail if it writes to a temp view that is not v2 relation") { @@ -365,8 +365,8 @@ 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")) - } + checkErrorTableNotFound(exc, "`testcat`.`table_name`") + } test("OverwritePartitions: fail if it writes to a temp view that is not v2 relation") { spark.range(10).createOrReplaceTempView("temp_view") @@ -634,7 +634,7 @@ class DataFrameWriterV2Suite extends QueryTest with SharedSparkSession with Befo spark.table("source").writeTo("testcat.table_name").replace() } - assert(exc.getMessage.contains("table_name")) + checkErrorTableNotFound(exc, "`table_name`") } test("CreateOrReplace: table does not exist") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala index 591d2b9435906..88ad54dc7b4e7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala @@ -1146,7 +1146,9 @@ class DatasetSuite extends QueryTest val e = intercept[AnalysisException]( dataset.createTempView("tempView")) intercept[AnalysisException](dataset.createTempView("tempView")) - assert(e.message.contains("already exists")) + checkError(e, + errorClass = "TEMP_TABLE_OR_VIEW_ALREADY_EXISTS", + parameters = Map("relationName" -> "`tempView`")) dataset.sparkSession.catalog.dropTempView("tempView") } 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 6a6f1ba989f92..030e68d227aae 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,14 @@ 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")) + checkErrorTableNotFound(e, "`in_valid_table`", + ExpectedContext("in_valid_table", 14, 13 + "in_valid_table".length)) e = intercept[AnalysisException] { sql("select * from no_db.no_table").show() } - assert(e.message.contains("Table or view not found")) + checkErrorTableNotFound(e, "`no_db`.`no_table`", + ExpectedContext("no_db.no_table", 14, 13 + "no_db.no_table".length)) e = intercept[AnalysisException] { sql("select * from json.invalid_file") @@ -1636,8 +1638,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")) + checkErrorTableNotFound(e, + "`org.apache.spark.sql.sources.HadoopFsRelationProvider`.`file_path`", + ExpectedContext("`org.apache.spark.sql.sources.HadoopFsRelationProvider`.`file_path`", 15, + 14 + "`org.apache.spark.sql.sources.HadoopFsRelationProvider`.`file_path`".length)) 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..95cc5f2972086 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,11 @@ 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")) + } + checkErrorTableNotFound(e1, s"`$globalTempDB`.`gTempView`", + ExpectedContext(s"$globalTempDB.gTempView", 14, 13 + s"$globalTempDB.gTempView".length)) // Analyzes in a global temporary view sql("CREATE GLOBAL TEMP VIEW gTempView AS SELECT 1 id") val errMsg2 = intercept[AnalysisException] { @@ -793,9 +793,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..fc078e997449a 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 quoted = UnresolvedAttribute.parseAttributeName(s"${catalogAndNamespace}table_name") + .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")) + checkErrorTableNotFound(exc, quoted, + ExpectedContext(s"${catalogAndNamespace}table_name", 12, + 11 + s"${catalogAndNamespace}table_name".length)) } } 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 8f5d436aacfe8..41a1563aebca5 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 @@ -1204,7 +1204,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" -> "`ns1`")) } test("SPARK-31100: Use: v2 catalog that implements SupportsNamespaces is used " + @@ -1213,7 +1215,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 " + @@ -1568,9 +1572,10 @@ 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", + expectedErrorMessageParameters = Map("relationName" -> "`dummy`")) // UPDATE non-existing column assertAnalysisErrorClass( @@ -1615,7 +1620,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 @@ -1625,10 +1630,11 @@ 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", + 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 @@ -1638,7 +1644,9 @@ 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", + expectedErrorMessageParameters = Map("relationName" -> "`testcat`.`ns1`.`ns2`.`dummy`")) + // UPDATE non-existing column assertAnalysisError( @@ -1699,8 +1707,8 @@ 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")) + checkErrorTableNotFound(e, "`testcat`.`ns`.`tbl`", + ExpectedContext("testcat.ns.tbl", 11, 10 + "testcat.ns.tbl".length)) } test("ANALYZE TABLE") { @@ -1757,7 +1765,8 @@ 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")) + checkErrorTableNotFound(e, "`testcat`.`ns1`.`ns2`.`tbl`", + ExpectedContext(t, 14, 13 + t.length)) // If "IF EXISTS" is set, UNCACHE TABLE will not throw an exception. sql(s"UNCACHE TABLE IF EXISTS $t") @@ -2377,7 +2386,7 @@ class DataSourceV2SQLSuiteV1Filter extends DataSourceV2SQLSuite with AlterTableT ) assert(e2.getMessage.contains(errMsg)) } - checkSubqueryError("SELECT 1 FROM non_exist", "Table or view not found: non_exist") + checkSubqueryError("SELECT 1 FROM non_exist", "TABLE_OR_VIEW_NOT_FOUND") checkSubqueryError("SELECT col", "UNRESOLVED_COLUMN") checkSubqueryError("SELECT 1, 2", "Scalar subquery must return only one column") checkSubqueryError("SELECT * FROM VALUES (1), (2)", "MULTI_VALUE_SUBQUERY_ERROR") 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..2ff9981c8d97d 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 @@ -20,6 +20,8 @@ package org.apache.spark.sql.connector import org.scalatest.BeforeAndAfter import org.apache.spark.sql._ +import org.apache.spark.sql.catalyst.parser.CatalystSqlParser +import org.apache.spark.sql.catalyst.util.quoteIdentifier import org.apache.spark.sql.internal.SQLConf.{PARTITION_OVERWRITE_MODE, PartitionOverwriteMode} import org.apache.spark.sql.test.SharedSparkSession @@ -231,11 +233,13 @@ trait InsertIntoSQLOnlyTests val t2 = s"${catalogAndNamespace}tbl2" withTableAndData(t1) { _ => sql(s"CREATE TABLE $t1 (id bigint, data string) USING $v2Format") + val parsed = CatalystSqlParser.parseMultipartIdentifier(t2) + .map(part => quoteIdentifier(part)).mkString(".") val e = intercept[AnalysisException] { sql(s"INSERT INTO $t2 VALUES (2L, 'dummy')") } - assert(e.getMessage.contains(t2)) - assert(e.getMessage.contains("Table not found")) + checkErrorTableNotFound(e, parsed, + ExpectedContext(t2, 12, 11 + t2.length)) } } 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..c5cccb2db78fc 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,9 +24,11 @@ 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 +import org.apache.spark.sql.catalyst.util.quoteIdentifier import org.apache.spark.sql.connector.catalog.{Identifier, InMemoryTable, SupportsRead, SupportsWrite, Table, TableCapability} import org.apache.spark.sql.connector.expressions.{FieldReference, IdentityTransform, Transform} import org.apache.spark.sql.connector.read.{Scan, ScanBuilder, V1Scan} @@ -99,7 +101,7 @@ class V1WriteFallbackSuite extends QueryTest with SharedSparkSession with Before val e = intercept[AnalysisException] { df.write.option("name", "t1").format(format).partitionBy("a").save() } - assert(e.getMessage.contains("already exists")) + checkErrorTableAlreadyExists(e, "`t1`") } test("save: Ignore mode") { @@ -298,7 +300,7 @@ class InMemoryV1Provider } if (mode == SaveMode.ErrorIfExists && tableOpt.isDefined) { - throw new AnalysisException("Table already exists") + throw new TableAlreadyExistsException(quoteIdentifier(tableName)) } else if (mode == SaveMode.Ignore && tableOpt.isDefined) { // do nothing return getRelation diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/GlobalTempViewSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/GlobalTempViewSuite.scala index 1a9baecfa747a..6e2200380d6cc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/GlobalTempViewSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/GlobalTempViewSuite.scala @@ -43,20 +43,20 @@ class GlobalTempViewSuite extends QueryTest with SharedSparkSession { // If there is no database in table name, we should try local temp view first, if not found, // try table/view in current database, which is "default" in this case. So we expect // NoSuchTableException here. - var e = intercept[AnalysisException](spark.table("src")).getMessage - assert(e.contains(expectedErrorMsg)) + var e = intercept[AnalysisException](spark.table("src")) + checkErrorTableNotFound(e, "`src`") // Use qualified name to refer to the global temp view explicitly. checkAnswer(spark.table(s"$globalTempDB.src"), Row(1, "a")) // Table name without database will never refer to a global temp view. - e = intercept[AnalysisException](sql("DROP VIEW src")).getMessage - assert(e.contains(expectedErrorMsg)) + e = intercept[AnalysisException](sql("DROP VIEW src")) + checkErrorTableNotFound(e, "`spark_catalog`.`default`.`src`") sql(s"DROP VIEW $globalTempDB.src") // The global temp view should be dropped successfully. - e = intercept[AnalysisException](spark.table(s"$globalTempDB.src")).getMessage - assert(e.contains(expectedErrorMsg)) + e = intercept[AnalysisException](spark.table(s"$globalTempDB.src")) + checkErrorTableNotFound(e, "`global_temp`.`src`") // We can also use Dataset API to create global temp view Seq(1 -> "a").toDF("i", "j").createGlobalTempView("src") @@ -64,8 +64,8 @@ class GlobalTempViewSuite extends QueryTest with SharedSparkSession { // Use qualified name to rename a global temp view. sql(s"ALTER VIEW $globalTempDB.src RENAME TO src2") - e = intercept[AnalysisException](spark.table(s"$globalTempDB.src")).getMessage - assert(e.contains(expectedErrorMsg)) + e = intercept[AnalysisException](spark.table(s"$globalTempDB.src")) + checkErrorTableNotFound(e, "`global_temp`.`src`") checkAnswer(spark.table(s"$globalTempDB.src2"), Row(1, "a")) // Use qualified name to alter a global temp view. @@ -74,8 +74,8 @@ class GlobalTempViewSuite extends QueryTest with SharedSparkSession { // We can also use Catalog API to drop global temp view spark.catalog.dropGlobalTempView("src2") - e = intercept[AnalysisException](spark.table(s"$globalTempDB.src2")).getMessage - assert(e.contains(expectedErrorMsg)) + e = intercept[AnalysisException](spark.table(s"$globalTempDB.src2")) + checkErrorTableNotFound(e, "`global_temp`.`src2`") // We can also use Dataset API to replace global temp view Seq(2 -> "b").toDF("i", "j").createOrReplaceGlobalTempView("src") 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 4cdd9fc208c13..231be20f60bf6 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,21 @@ 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 assertAnalysisErrorClass(query: String, + errorClass: String, + parameters: Map[String, String], + context: ExpectedContext): Unit = { + val e = intercept[AnalysisException](sql(query)) + checkError(e, errorClass = errorClass, parameters = parameters, context = context) + } + private def assertErrorForAlterTableOnTempView( sqlText: String, viewName: String, cmdName: String): Unit = { assertAnalysisError( @@ -258,10 +273,16 @@ 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`", + ExpectedContext("db_not_exist234.jt", 51, 50 + "db_not_exist234.jt".length)) // 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`", + ExpectedContext("table_not_exist345", 51, 50 + "table_not_exist345".length)) // A column that does not exist intercept[AnalysisException] { @@ -269,11 +290,19 @@ 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) + } + checkErrorTableNotFound(e, relation) + } + + private def assertRelationNotFound(query: String, relation: String, context: ExpectedContext): + Unit = { val e = intercept[AnalysisException] { sql(query) - }.getMessage - assert(e.contains("Table or view not found")) + } + checkErrorTableNotFound(e, relation, context) } @@ -296,12 +325,17 @@ 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`"), + ExpectedContext("db_not_exist234.jt", 61, 60 + "db_not_exist234.jt".length)) // 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`", + ExpectedContext("table_not_exist1345", 61, 60 + "table_not_exist1345".length)) // A column that does not exist, for temporary view intercept[AnalysisException] { @@ -368,7 +402,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 +504,15 @@ 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`", + ExpectedContext("testView", 11, 10 + "testView".length)) - assertAnalysisError( + assertRelationNotFound( "ALTER VIEW default.testView AS SELECT 1, 2", - "View not found: default.testView") + "`default`.`testView`", + ExpectedContext("default.testView", 11, 10 + "default.testView".length)) } test("ALTER VIEW AS should try to alter temp view first if view name has no database part") { @@ -618,21 +656,25 @@ abstract class SQLViewSuite extends QueryTest with SQLTestUtils { } } } - assertInvalidReference("SELECT * FROM view1") + assertRelationNotFound("SELECT * FROM view1", "`table1`", + ExpectedContext("VIEW", "spark_catalog.default.view1", 14, 13 + "table1".length, "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`", + ExpectedContext("VIEW", "spark_catalog.default.view2", 14, 13 + "table2".length, "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`", + ExpectedContext("VIEW", "spark_catalog.default.view3", 14, + 13 + "testView".length, "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..577d2551c6fc5 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 @@ -40,6 +40,7 @@ abstract class SQLViewTestSuite extends QueryTest with SQLTestUtils { protected def viewTypeString: String protected def formattedViewName(viewName: String): String + protected def fullyQualifiedViewName(viewName: String): String protected def tableIdentifier(viewName: String): TableIdentifier def createView( @@ -278,7 +279,7 @@ abstract class SQLViewTestSuite extends QueryTest with SQLTestUtils { test("SPARK-34490 - query should fail if the view refers a dropped table") { withTable("t") { Seq(2, 3, 1).toDF("c1").write.format("parquet").saveAsTable("t") - val viewName = createView("testView", "SELECT * FROM t") + val viewName = createView("testview", "SELECT * FROM t") withView(viewName) { // Always create a temp view in this case, not use `createView` on purpose sql("CREATE TEMP VIEW t AS SELECT 1 AS c1") @@ -288,8 +289,9 @@ 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")) + } + checkErrorTableNotFound(e, "`t`", + ExpectedContext("VIEW", fullyQualifiedViewName("testview"), 14, 14, "t")) } } } @@ -468,6 +470,7 @@ abstract class TempViewTestSuite extends SQLViewTestSuite { class LocalTempViewTestSuite extends TempViewTestSuite with SharedSparkSession { override protected def viewTypeString: String = "TEMPORARY VIEW" override protected def formattedViewName(viewName: String): String = viewName + override protected def fullyQualifiedViewName(viewName: String): String = viewName override protected def tableIdentifier(viewName: String): TableIdentifier = { TableIdentifier(viewName) } @@ -482,6 +485,9 @@ class GlobalTempViewTestSuite extends TempViewTestSuite with SharedSparkSession override protected def formattedViewName(viewName: String): String = { s"$db.$viewName" } + override protected def fullyQualifiedViewName(viewName: String): String = { + s"$db.$viewName" + } override protected def tableIdentifier(viewName: String): TableIdentifier = { TableIdentifier(viewName, Some(db)) } @@ -508,6 +514,8 @@ class PersistedViewTestSuite extends SQLViewTestSuite with SharedSparkSession { private def db: String = "default" override protected def viewTypeString: String = "VIEW" override protected def formattedViewName(viewName: String): String = s"$db.$viewName" + override protected def fullyQualifiedViewName(viewName: String): String = + s"spark_catalog.$db.$viewName" override protected def tableIdentifier(viewName: String): TableIdentifier = { TableIdentifier(viewName, Some(db), Some(SESSION_CATALOG_NAME)) } 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 8472a581fa793..e113499ec685e 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 @@ -20,6 +20,8 @@ package org.apache.spark.sql.execution.command import java.time.{Duration, Period} import org.apache.spark.sql.{AnalysisException, QueryTest, Row} +import org.apache.spark.sql.catalyst.parser.CatalystSqlParser +import org.apache.spark.sql.catalyst.util.quoteIdentifier import org.apache.spark.sql.internal.SQLConf /** @@ -80,10 +82,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 parsed = CatalystSqlParser.parseMultipartIdentifier(t) + .map(part => quoteIdentifier(part)).mkString(".") + 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")) + } + checkErrorTableNotFound(e, parsed, ExpectedContext(t, 12, 11 + t.length)) } } 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..b38f5b2dfeb57 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 @@ -19,6 +19,8 @@ package org.apache.spark.sql.execution.command import org.apache.spark.sql.{AnalysisException, QueryTest, Row} import org.apache.spark.sql.catalyst.analysis.NoSuchPartitionsException +import org.apache.spark.sql.catalyst.parser.CatalystSqlParser +import org.apache.spark.sql.catalyst.util.quoteIdentifier import org.apache.spark.sql.internal.SQLConf /** @@ -86,10 +88,13 @@ trait AlterTableDropPartitionSuiteBase extends QueryTest with DDLCommandTestUtil test("table to alter does not exist") { withNamespaceAndTable("ns", "does_not_exist") { t => - val errMsg = intercept[AnalysisException] { + val parsed = CatalystSqlParser.parseMultipartIdentifier(t) + .map(part => quoteIdentifier(part)).mkString(".") + val e = intercept[AnalysisException] { sql(s"ALTER TABLE $t DROP PARTITION (a='4', b='9')") - }.getMessage - assert(errMsg.contains("Table not found")) + } + checkErrorTableNotFound(e, parsed, + ExpectedContext(t, 12, 11 + t.length)) } } @@ -119,9 +124,10 @@ trait AlterTableDropPartitionSuiteBase extends QueryTest with DDLCommandTestUtil |$defaultUsing |PARTITIONED BY (part0, part1)""".stripMargin) val errMsg = intercept[AnalysisException] { - sql(s"ALTER TABLE $t DROP PARTITION (part0 = 1)") + sql(s"ALTER TABLE $t ADD PARTITION (part0 = 1)") }.getMessage - assert(errMsg.contains(notFullPartitionSpecErr)) + assert(errMsg.contains("Partition spec is invalid. " + + "The spec (part0) must match the partition spec (part0, part1)")) } } @@ -130,10 +136,18 @@ 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")) + } + val expectedTableName = if (commandVersion == DDLCommandTestUtils.V1_COMMAND_VERSION) { + "`ns`.`tbl`" + } else { + "`test_catalog`.`ns`.`tbl`" + } + checkError(e, + errorClass = "PARTITIONS_NOT_FOUND", + parameters = Map("partitionList" -> "PARTITION (`id` = 2)", + "tableName" -> expectedTableName)) 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 6e67946a557ad..f24cebbf13875 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 @@ -19,6 +19,8 @@ package org.apache.spark.sql.execution.command import org.apache.spark.sql.{AnalysisException, QueryTest, Row} import org.apache.spark.sql.catalyst.analysis.{NoSuchPartitionException, PartitionsAlreadyExistException} +import org.apache.spark.sql.catalyst.parser.CatalystSqlParser +import org.apache.spark.sql.catalyst.util.quoteIdentifier import org.apache.spark.sql.internal.SQLConf /** @@ -57,10 +59,11 @@ 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")) + } + checkErrorTableNotFound(e, s"`$catalog`.`ns`.`no_tbl`", + ExpectedContext(s"$catalog.ns.no_tbl", 12, 11 + s"$catalog.ns.no_tbl".length)) } } @@ -68,10 +71,19 @@ trait AlterTableRenamePartitionSuiteBase extends QueryTest with DDLCommandTestUt withNamespaceAndTable("ns", "tbl") { t => createSinglePartTable(t) checkPartitions(t, Map("id" -> "1")) - val errMsg = intercept[NoSuchPartitionException] { + val parsed = if (commandVersion == DDLCommandTestUtils.V1_COMMAND_VERSION) { + "`ns`.`tbl`" + } else { + CatalystSqlParser.parseMultipartIdentifier(t) + .map(part => quoteIdentifier(part)).mkString(".") + } + 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" -> parsed)) } } @@ -80,10 +92,19 @@ 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[PartitionsAlreadyExistException] { + val parsed = if (commandVersion == DDLCommandTestUtils.V1_COMMAND_VERSION) { + "`ns`.`tbl`" + } else { + CatalystSqlParser.parseMultipartIdentifier(t) + .map(part => quoteIdentifier(part)).mkString(".") + } + + val e = intercept[PartitionsAlreadyExistException] { sql(s"ALTER TABLE $t PARTITION (id = 1) RENAME TO PARTITION (id = 2)") - }.getMessage - assert(errMsg.contains("partitions already exist")) + } + checkError(e, + errorClass = "PARTITIONS_ALREADY_EXIST", + parameters = Map("partitionList" -> "PARTITION (`id` = 2)", "tableName" -> parsed)) } } 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..506b44741ab4b 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")) + } + checkErrorTableNotFound(e, s"`$catalog`.`dbx`.`does_not_exist`", + ExpectedContext(s"$catalog.dbx.does_not_exist", 12, + 11 + s"$catalog.dbx.does_not_exist".length)) } 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..ecd2f570aeb69 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 @@ -23,7 +23,8 @@ import org.apache.hadoop.fs.Path import org.apache.spark.sql.QueryTest import org.apache.spark.sql.catalyst.analysis.NamespaceAlreadyExistsException -import org.apache.spark.sql.catalyst.parser.ParseException +import org.apache.spark.sql.catalyst.parser.{CatalystSqlParser, ParseException} +import org.apache.spark.sql.catalyst.util.quoteIdentifier import org.apache.spark.sql.connector.catalog.{CatalogPlugin, CatalogV2Util, SupportsNamespaces} import org.apache.spark.sql.execution.command.DDLCommandTestUtils.V1_COMMAND_VERSION import org.apache.spark.sql.internal.SQLConf @@ -86,13 +87,19 @@ trait CreateNamespaceSuiteBase extends QueryTest with DDLCommandTestUtils { test("Namespace already exists") { val ns = s"$catalog.$namespace" + withNamespace(ns) { sql(s"CREATE NAMESPACE $ns") + val parsed = CatalystSqlParser.parseMultipartIdentifier(namespace) + .map(part => quoteIdentifier(part)).mkString(".") + 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" -> parsed)) // 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 3db7b2878c5ec..883672cc112b6 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 @@ -833,8 +833,8 @@ 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")) + checkErrorTableNotFound(e, "`tab1`") sql("ALTER VIEW tab2 RENAME TO tab1") checkAnswer(spark.table("tab1"), spark.range(10).toDF()) intercept[AnalysisException] { spark.table("tab2") } @@ -868,8 +868,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"))) @@ -903,8 +904,7 @@ 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")) + checkErrorTableAlreadyExists(e, "`view2`") val catalog = spark.sessionState.catalog assert(catalog.listTables("default") == @@ -1224,8 +1224,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("relationName" -> "`t_temp`")) } } @@ -1234,8 +1236,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("relationName" -> "`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 3b99738c78bde..4a7d5551fe52f 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 @@ -18,6 +18,8 @@ package org.apache.spark.sql.execution.command import org.apache.spark.sql.{AnalysisException, QueryTest, Row} +import org.apache.spark.sql.catalyst.parser.CatalystSqlParser +import org.apache.spark.sql.catalyst.util.quoteIdentifier import org.apache.spark.sql.types.{BooleanType, MetadataBuilder, StringType, StructType} /** @@ -37,10 +39,13 @@ trait DescribeTableSuiteBase extends QueryTest with DDLCommandTestUtils { test("DESCRIBE TABLE in a catalog when table does not exist") { withNamespaceAndTable("ns", "table") { tbl => + val parsed = CatalystSqlParser.parseMultipartIdentifier(s"${tbl}_non_existence") + .map(part => quoteIdentifier(part)).mkString(".") val e = intercept[AnalysisException] { sql(s"DESCRIBE TABLE ${tbl}_non_existence") } - assert(e.getMessage.contains(s"Table or view not found: ${tbl}_non_existence")) + checkErrorTableNotFound(e, parsed, + ExpectedContext(s"${tbl}_non_existence", 15, 14 + s"${tbl}_non_existence".length)) } } 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 c26022addf0c2..3df9843f8a435 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,10 @@ 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(s"Table $catalog.ns.tbl not found")) + } + checkErrorTableNotFound(e, 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 9c7687c72ae18..6276b1a3b60f8 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..53f4b831482b4 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,11 @@ 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")) + } + checkErrorTableNotFound(e, "`BADTABLE`", + ExpectedContext("BADTABLE", 19, 18 + "BADTABLE".length)) } 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..fbf0e45872a20 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 @@ -19,6 +19,8 @@ package org.apache.spark.sql.execution.command import org.apache.spark.sql.{AnalysisException, QueryTest, Row} import org.apache.spark.sql.catalyst.analysis.NoSuchPartitionException +import org.apache.spark.sql.catalyst.parser.CatalystSqlParser +import org.apache.spark.sql.catalyst.util.quoteIdentifier import org.apache.spark.sql.internal.SQLConf /** @@ -36,10 +38,12 @@ trait TruncateTableSuiteBase extends QueryTest with DDLCommandTestUtils { test("table does not exist") { withNamespaceAndTable("ns", "does_not_exist") { t => - val errMsg = intercept[AnalysisException] { + val parsed = CatalystSqlParser.parseMultipartIdentifier(t) + .map(part => quoteIdentifier(part)).mkString(".") + val e = intercept[AnalysisException] { sql(s"TRUNCATE TABLE $t") - }.getMessage - assert(errMsg.contains("Table not found")) + } + checkErrorTableNotFound(e, parsed, ExpectedContext(t, 15, 14 + t.length)) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/AlterTableAddPartitionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/AlterTableAddPartitionSuite.scala index 54287cc6a47bf..11df5ede8bbf4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/AlterTableAddPartitionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/AlterTableAddPartitionSuite.scala @@ -143,13 +143,14 @@ trait AlterTableAddPartitionSuiteBase extends command.AlterTableAddPartitionSuit 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 === - """The following partitions already exist in table 'tbl' database 'ns': - |Map(id -> 2)""".stripMargin) + } + checkError(e, + errorClass = "PARTITIONS_ALREADY_EXIST", + parameters = Map("partitionList" -> "PARTITION (`id` = 2)", + "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/v1/AlterTableRecoverPartitionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/AlterTableRecoverPartitionsSuite.scala index 16dd6a3a03f1d..32963f5fc2deb 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,11 @@ 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")) + } + checkErrorTableNotFound(e, "`does_not_exist`", + ExpectedContext("does_not_exist", 12, 11 + "does_not_exist".length)) } 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 d6eea8ae8ed9b..d0f1a83594284 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,8 @@ 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")) + checkErrorTableNotFound(e, "`ns`.`does_not_exist`", + ExpectedContext("ns.does_not_exist", 12, 11 + "ns.does_not_exist".length)) } 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 708c7f7a61880..6e4d6a8a0c8f0 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 @@ -109,7 +109,8 @@ 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")) + checkErrorTableNotFound(e3, "`does_not_exist`", + ExpectedContext("does_not_exist", 12, 11 + "does_not_exist".length)) } } @@ -164,7 +165,8 @@ 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")) + checkErrorTableNotFound(e5, "`does_not_exist`", + ExpectedContext("does_not_exist", 12, 11 + "does_not_exist".length)) } } } 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 fe017866f25a5..9bee8d38c0bfb 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/AlterTableAddPartitionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/AlterTableAddPartitionSuite.scala index dc6e5a2909da0..835be8573fdc4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/AlterTableAddPartitionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/AlterTableAddPartitionSuite.scala @@ -107,11 +107,14 @@ class AlterTableAddPartitionSuite 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 === s"The following partitions already exist in table $t:id -> 2") + } + checkError(e, + errorClass = "PARTITIONS_ALREADY_EXIST", + parameters = Map("partitionList" -> "PARTITION (`id` = 2)", + "tableName" -> "`test_catalog`.`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/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..3dfe60638772c 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 @@ -29,6 +29,7 @@ import org.scalatest.BeforeAndAfter import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.analysis.{NamespaceAlreadyExistsException, NoSuchDatabaseException, NoSuchNamespaceException, NoSuchTableException, TableAlreadyExistsException} import org.apache.spark.sql.catalyst.parser.CatalystSqlParser +import org.apache.spark.sql.catalyst.util.quoteIdentifier import org.apache.spark.sql.connector.catalog.{CatalogV2Util, Identifier, NamespaceChange, SupportsNamespaces, TableCatalog, TableChange, V1Table} import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.{DoubleType, IntegerType, LongType, StringType, StructField, StructType, TimestampType} @@ -44,6 +45,8 @@ abstract class V2SessionCatalogBaseSuite extends SharedSparkSession with BeforeA val testNs: Array[String] = Array("db") val defaultNs: Array[String] = Array("default") val testIdent: Identifier = Identifier.of(testNs, "test_table") + val testIdentQuoted: String = (testIdent.namespace :+ testIdent.name) + .map(part => quoteIdentifier(part)).mkString(".") def newCatalog(): V2SessionCatalog = { val newCatalog = new V2SessionCatalog(spark.sessionState.catalog) @@ -81,6 +84,8 @@ class V2SessionCatalogTableSuite extends V2SessionCatalogBaseSuite { } private val testIdentNew = Identifier.of(testNs, "test_table_new") + private val testIdentNewQuoted = (testIdentNew.namespace :+ testIdentNew.name) + .map(part => quoteIdentifier(part)).mkString(".") test("listTables") { val catalog = newCatalog() @@ -153,12 +158,14 @@ class V2SessionCatalogTableSuite extends V2SessionCatalogBaseSuite { val table = catalog.createTable(testIdent, schema, Array.empty, emptyProps) + val parsed = CatalystSqlParser.parseMultipartIdentifier(table.name) + .map(part => quoteIdentifier(part)).mkString(".") + val exc = intercept[TableAlreadyExistsException] { catalog.createTable(testIdent, schema, Array.empty, emptyProps) } - assert(exc.message.contains(table.name())) - assert(exc.message.contains("already exists")) + checkErrorTableAlreadyExists(exc, parsed) assert(catalog.tableExists(testIdent)) } @@ -232,7 +239,7 @@ class V2SessionCatalogTableSuite extends V2SessionCatalogBaseSuite { catalog.loadTable(testIdent) } - assert(exc.message.contains("Table or view 'test_table' not found in database 'db'")) + checkErrorTableNotFound(exc, testIdentQuoted) } test("invalidateTable") { @@ -683,8 +690,7 @@ class V2SessionCatalogTableSuite extends V2SessionCatalogBaseSuite { catalog.alterTable(testIdent, TableChange.setProperty("prop", "val")) } - assert(exc.message.contains(testIdent.quoted)) - assert(exc.message.contains("not found")) + checkErrorTableNotFound(exc, testIdentQuoted) } test("alterTable: location") { @@ -760,7 +766,7 @@ class V2SessionCatalogTableSuite extends V2SessionCatalogBaseSuite { catalog.renameTable(testIdent, testIdentNew) } - assert(exc.message.contains("Table or view 'test_table' not found in database 'db'")) + checkErrorTableNotFound(exc, testIdentQuoted) } test("renameTable: fail if new table name already exists") { @@ -779,8 +785,7 @@ class V2SessionCatalogTableSuite extends V2SessionCatalogBaseSuite { catalog.renameTable(testIdent, testIdentNew) } - assert(exc.message.contains(testIdentNew.quoted)) - assert(exc.message.contains("already exists")) + checkErrorTableAlreadyExists(exc, testIdentNewQuoted) } 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 7371b6cf0bc5f..af6f1b35bb846 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,13 @@ 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 h2.test.not_existing_table not found", - "h2.bad_test.not_existing_table" -> - "Table h2.bad_test.not_existing_table not found" - ).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)) + } + checkErrorTableNotFound(e, expected) } } @@ -110,13 +108,14 @@ 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")) + checkErrorTableNotFound(exp1, "`h2`.`test`.`not_existing_table`", + ExpectedContext("h2.test.not_existing_table", 12, 11 + "h2.test.not_existing_table".length)) 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")) + checkErrorTableNotFound(exp2, "`h2`.`bad_test`.`not_existing_table`", + ExpectedContext("h2.bad_test.not_existing_table", 12, + 11 + "h2.bad_test.not_existing_table".length)) // Rename to an existing table withTable("h2.test.dst_table") { withConnection { conn => @@ -129,9 +128,7 @@ class JDBCTableCatalogSuite extends QueryTest with SharedSparkSession { val exp = intercept[TableAlreadyExistsException] { sql("ALTER TABLE h2.test.src_table RENAME TO test.dst_table") } - assert(exp.getMessage.contains( - "Failed table renaming from test.src_table to test.dst_table")) - assert(exp.cause.get.getMessage.contains("Table \"dst_table\" already exists")) + checkErrorTableAlreadyExists(exp, "`dst_table`") } } } @@ -142,11 +139,14 @@ 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")) + } + checkErrorTableNotFound(e, expected) } } @@ -159,16 +159,17 @@ 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")) + } + checkErrorTableAlreadyExists(e, "`test`.`new_table`") } val exp = intercept[NoSuchNamespaceException] { sql("CREATE TABLE h2.bad_test.new_table(i INT, j STRING)") } - assert(exp.getMessage.contains("Failed table creation: bad_test.new_table")) - assert(exp.cause.get.getMessage.contains("Schema \"bad_test\" not found")) + checkError(exp, + errorClass = "SCHEMA_NOT_FOUND", + parameters = Map("schemaName" -> "`bad_test`")) } test("ALTER TABLE ... add column") { @@ -193,11 +194,15 @@ 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")) + } + checkErrorTableNotFound(e, expected, + ExpectedContext(table, 12, 11 + table.length)) } } @@ -218,11 +223,15 @@ 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")) + } + checkErrorTableNotFound(e, expected, + ExpectedContext(table, 12, 11 + table.length)) } } @@ -242,11 +251,15 @@ 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")) + } + checkErrorTableNotFound(e, expected, + ExpectedContext(table, 12, 11 + table.length)) } } @@ -273,11 +286,15 @@ 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")) + } + checkErrorTableNotFound(e, expected, + ExpectedContext(table, 12, 11 + table.length)) } } @@ -299,11 +316,15 @@ 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")) + } + checkErrorTableNotFound(e, expected, + ExpectedContext(table, 12, 11 + table.length)) } } @@ -323,11 +344,15 @@ 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")) + } + checkErrorTableNotFound(e, expected, + ExpectedContext(table, 12, 11 + table.length)) } } 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..0d1242fbb190a 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,10 @@ 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") + } + checkErrorTableNotFound(e, "`non_exist_table`") } test("read: stream table API with temp view") { @@ -605,7 +606,7 @@ class InMemoryStreamTableCatalog extends InMemoryTableCatalog { partitions: Array[Transform], properties: util.Map[String, String]): Table = { if (tables.containsKey(ident)) { - throw new TableAlreadyExistsException(ident) + throw new TableAlreadyExistsException(ident.asMultipartIdentifier) } val table = if (ident.name() == DataStreamTableAPISuite.V1FallbackTestTableName) { diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala index fb5beb60c5cad..98a257e60d732 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala @@ -388,7 +388,7 @@ class CliSuite extends SparkFunSuite { test("SPARK-11188 Analysis error reporting") { runCliWithin(timeout = 2.minute, errorResponses = Seq("AnalysisException"))( - "select * from nonexistent_table;" -> "Table or view not found: nonexistent_table;" + "select * from nonexistent_table;" -> "nonexistent_table" ) } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala index f6dc35131b74c..f6b06b08cbc71 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala @@ -636,11 +636,11 @@ private[hive] class HiveClientImpl( ignoreIfExists: Boolean): Unit = withHiveState { def replaceExistException(e: Throwable): Unit = e match { case _: HiveException if e.getCause.isInstanceOf[AlreadyExistsException] => - val t = shim.getTable(client, db, table) - val exists = parts.filter { part => - shim.getPartition(client, t, part.spec.asJava, forceCreate = false) != null + val hiveTable = client.getTable(db, table) + val existingParts = parts.filter { p => + shim.getPartitions(client, hiveTable, p.spec.asJava).nonEmpty } - throw new PartitionsAlreadyExistException(db, table, exists.map(_.spec)) + throw new PartitionsAlreadyExistException(db, table, existingParts.map(_.spec)) case _ => throw e } try { 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..258aff1f20623 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,18 @@ 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")) + checkErrorTableNotFound(e, "`nonexistentTable`") e = intercept[AnalysisException] { uncacheTable("nonexistentTable") - }.getMessage - assert(e.contains(expectedErrorMsg)) - e = intercept[AnalysisException] { + } + checkErrorTableNotFound(e, "`nonexistentTable`") + e = intercept[AnalysisException] { sql("UNCACHE TABLE nonexistentTable") - }.getMessage - assert(e.contains("Table or view not found: nonexistentTable")) + } + checkErrorTableNotFound(e, "`nonexistentTable`", + ExpectedContext("nonexistentTable", 14, 13 + "nonexistentTable".length)) 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..3de6d375149b8 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 @@ -747,11 +747,13 @@ class InsertSuite extends QueryTest with TestHiveSingleton with BeforeAndAfter test("insert overwrite to dir from non-existent table") { withTempDir { dir => val path = dir.toURI.getPath - + val stmt = s"INSERT OVERWRITE LOCAL DIRECTORY '${path}' TABLE nonexistent" val e = intercept[AnalysisException] { - sql(s"INSERT OVERWRITE LOCAL DIRECTORY '${path}' TABLE nonexistent") - }.getMessage - assert(e.contains("Table or view not found")) + sql(stmt) + } + checkErrorTableNotFound(e, "`nonexistent`", + ExpectedContext("TABLE nonexistent", stmt.length - "TABLE nonexistent".length, + stmt.length - 1)) } } 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 7b2e772fa4908..e76ef4725f762 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,9 @@ 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") + checkErrorTableAlreadyExists(e, 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 +520,10 @@ 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.")) + } + checkErrorTableAlreadyExists(e, s"`$SESSION_CATALOG_NAME`.`default`.`createdJsonTable`") } // Data should not be deleted. @@ -1251,14 +1248,15 @@ 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")) + } + checkErrorTableAlreadyExists(e, 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..b2914855ade2d 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,8 @@ 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")) + checkErrorTableNotFound(err, "`SRC`", + ExpectedContext("SRC", 14, 13 + "SRC".length)) } 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 e6abc7b96c17d..6f799bbe7d3f5 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 exist")) + } + 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 ffb6993ccf1d1..afd28cd4c9dff 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 62a48a660c87e..7eeff8116490e 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")) + } + checkErrorTableNotFound(e, s"`${global_temp_db}`.`nonexistentview`", + ExpectedContext(s"${global_temp_db}.nonexistentview", 14, + 13 + s"${global_temp_db}.nonexistentview".length)) } test("script") { @@ -439,10 +441,10 @@ abstract class SQLQuerySuiteBase extends QueryTest with SQLTestUtils with TestHi withTable("ctas1") { sql("CREATE TABLE ctas1 AS SELECT key k, value FROM src ORDER BY k, value") sql("CREATE TABLE IF NOT EXISTS ctas1 AS SELECT key k, value FROM src ORDER BY k, value") - val message = intercept[AnalysisException] { + val e = intercept[AnalysisException] { sql("CREATE TABLE ctas1 AS SELECT key k, value FROM src ORDER BY k, value") - }.getMessage - assert(message.contains("already exists")) + } + checkErrorTableAlreadyExists(e, "`spark_catalog`.`default`.`ctas1`") checkRelation("ctas1", isDataSourceTable = true, defaultDataSource) } 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..d0516198b7120 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,8 @@ 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")) + checkErrorTableNotFound(e, "`does_not_exist`", + ExpectedContext("does_not_exist", 12, 11 + "does_not_exist".length)) } } @@ -111,7 +112,8 @@ 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")) + checkErrorTableNotFound(e, "`does_not_exist`", + ExpectedContext("does_not_exist", 12, 11 + "does_not_exist".length)) } } } 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..b7f06d3dcc3c2 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,10 @@ 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")) + } + checkErrorTableAlreadyExists(e, s"`$SESSION_CATALOG_NAME`.`default`.`t`") } }