diff --git a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryErrorsSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryErrorsSuiteBase.scala index d03c18882a4c7..525771f303842 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryErrorsSuiteBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryErrorsSuiteBase.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.errors -import org.apache.spark.{QueryContext, SparkThrowable} +import org.apache.spark.QueryContext import org.apache.spark.sql.catalyst.parser.ParseException import org.apache.spark.sql.test.SharedSparkSession @@ -28,28 +28,13 @@ trait QueryErrorsSuiteBase extends SharedSparkSession { errorClass: String, errorSubClass: Option[String] = None, sqlState: String, - message: String): Unit = { - val exception = intercept[ParseException] { - sql(sqlText) - } - checkParsingError(exception, errorClass, errorSubClass, sqlState, message) - } - - def checkParsingError( - exception: Exception with SparkThrowable, - errorClass: String, - errorSubClass: Option[String] = None, - sqlState: String, - message: String): Unit = { - val fullErrorClass = if (errorSubClass.isDefined) { - errorClass + "." + errorSubClass.get - } else { - errorClass - } - assert(exception.getErrorClass === errorClass) - assert(exception.getErrorSubClass === errorSubClass.orNull) - assert(exception.getSqlState === sqlState) - assert(exception.getMessage === s"""\n[$fullErrorClass] """ + message) + parameters: Map[String, String] = Map.empty): Unit = { + checkError( + exception = intercept[ParseException](sql(sqlText)), + errorClass = errorClass, + errorSubClass = errorSubClass, + sqlState = Some(sqlState), + parameters = parameters) } case class ExpectedContext( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryParsingErrorsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryParsingErrorsSuite.scala index 6dc40c3eadb6a..e9379b461ecb9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryParsingErrorsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryParsingErrorsSuite.scala @@ -28,14 +28,7 @@ class QueryParsingErrorsSuite extends QueryTest with QueryErrorsSuiteBase { sqlText = "SELECT * FROM t1 NATURAL JOIN LATERAL (SELECT c1 + c2 AS c2)", errorClass = "UNSUPPORTED_FEATURE", errorSubClass = Some("LATERAL_NATURAL_JOIN"), - sqlState = "0A000", - message = - """The feature is not supported: NATURAL join with LATERAL correlation.(line 1, pos 14) - | - |== SQL == - |SELECT * FROM t1 NATURAL JOIN LATERAL (SELECT c1 + c2 AS c2) - |--------------^^^ - |""".stripMargin) + sqlState = "0A000") } test("UNSUPPORTED_FEATURE: LATERAL join with USING join not supported") { @@ -43,14 +36,7 @@ class QueryParsingErrorsSuite extends QueryTest with QueryErrorsSuiteBase { sqlText = "SELECT * FROM t1 JOIN LATERAL (SELECT c1 + c2 AS c2) USING (c2)", errorClass = "UNSUPPORTED_FEATURE", errorSubClass = Some("LATERAL_JOIN_USING"), - sqlState = "0A000", - message = - """The feature is not supported: JOIN USING with LATERAL correlation.(line 1, pos 14) - | - |== SQL == - |SELECT * FROM t1 JOIN LATERAL (SELECT c1 + c2 AS c2) USING (c2) - |--------------^^^ - |""".stripMargin) + sqlState = "0A000") } test("UNSUPPORTED_FEATURE: Unsupported LATERAL join type") { @@ -60,13 +46,7 @@ class QueryParsingErrorsSuite extends QueryTest with QueryErrorsSuiteBase { errorClass = "UNSUPPORTED_FEATURE", errorSubClass = Some("LATERAL_JOIN_OF_TYPE"), sqlState = "0A000", - message = - s"""The feature is not supported: $joinType JOIN with LATERAL correlation.(line 1, pos 14) - | - |== SQL == - |SELECT * FROM t1 $joinType JOIN LATERAL (SELECT c1 + c2 AS c3) ON c2 = c3 - |--------------^^^ - |""".stripMargin) + parameters = Map("joinType" -> joinType)) } } @@ -83,13 +63,7 @@ class QueryParsingErrorsSuite extends QueryTest with QueryErrorsSuiteBase { sqlText = sqlText, errorClass = "INVALID_SQL_SYNTAX", sqlState = "42000", - message = - s"""Invalid SQL syntax: LATERAL can only be used with subquery.(line 1, pos $pos) - | - |== SQL == - |$sqlText - |${"-" * pos}^^^ - |""".stripMargin) + parameters = Map("inputString" -> "LATERAL can only be used with subquery.")) } } @@ -98,14 +72,7 @@ class QueryParsingErrorsSuite extends QueryTest with QueryErrorsSuiteBase { sqlText = "SELECT * FROM a NATURAL CROSS JOIN b", errorClass = "UNSUPPORTED_FEATURE", errorSubClass = Some("NATURAL_CROSS_JOIN"), - sqlState = "0A000", - message = - """The feature is not supported: NATURAL CROSS JOIN.(line 1, pos 14) - | - |== SQL == - |SELECT * FROM a NATURAL CROSS JOIN b - |--------------^^^ - |""".stripMargin) + sqlState = "0A000") } test("INVALID_SQL_SYNTAX: redefine window") { @@ -113,13 +80,7 @@ class QueryParsingErrorsSuite extends QueryTest with QueryErrorsSuiteBase { sqlText = "SELECT min(a) OVER win FROM t1 WINDOW win AS win, win AS win2", errorClass = "INVALID_SQL_SYNTAX", sqlState = "42000", - message = - """Invalid SQL syntax: The definition of window `win` is repetitive.(line 1, pos 31) - | - |== SQL == - |SELECT min(a) OVER win FROM t1 WINDOW win AS win, win AS win2 - |-------------------------------^^^ - |""".stripMargin) + parameters = Map("inputString" -> "The definition of window `win` is repetitive.")) } test("INVALID_SQL_SYNTAX: invalid window reference") { @@ -127,13 +88,7 @@ class QueryParsingErrorsSuite extends QueryTest with QueryErrorsSuiteBase { sqlText = "SELECT min(a) OVER win FROM t1 WINDOW win AS win", errorClass = "INVALID_SQL_SYNTAX", sqlState = "42000", - message = - """Invalid SQL syntax: Window reference `win` is not a window specification.(line 1, pos 31) - | - |== SQL == - |SELECT min(a) OVER win FROM t1 WINDOW win AS win - |-------------------------------^^^ - |""".stripMargin) + parameters = Map("inputString" -> "Window reference `win` is not a window specification.")) } test("INVALID_SQL_SYNTAX: window reference cannot be resolved") { @@ -141,13 +96,7 @@ class QueryParsingErrorsSuite extends QueryTest with QueryErrorsSuiteBase { sqlText = "SELECT min(a) OVER win FROM t1 WINDOW win AS win2", errorClass = "INVALID_SQL_SYNTAX", sqlState = "42000", - message = - """Invalid SQL syntax: Cannot resolve window reference `win2`.(line 1, pos 31) - | - |== SQL == - |SELECT min(a) OVER win FROM t1 WINDOW win AS win2 - |-------------------------------^^^ - |""".stripMargin) + parameters = Map("inputString" -> "Cannot resolve window reference `win2`.")) } test("UNSUPPORTED_FEATURE: TRANSFORM does not support DISTINCT/ALL") { @@ -155,14 +104,7 @@ class QueryParsingErrorsSuite extends QueryTest with QueryErrorsSuiteBase { sqlText = "SELECT TRANSFORM(DISTINCT a) USING 'a' FROM t", errorClass = "UNSUPPORTED_FEATURE", errorSubClass = Some("TRANSFORM_DISTINCT_ALL"), - sqlState = "0A000", - message = - """The feature is not supported: TRANSFORM with the DISTINCT/ALL clause.(line 1, pos 17) - | - |== SQL == - |SELECT TRANSFORM(DISTINCT a) USING 'a' FROM t - |-----------------^^^ - |""".stripMargin) + sqlState = "0A000") } test("UNSUPPORTED_FEATURE: In-memory mode does not support TRANSFORM with serde") { @@ -171,14 +113,7 @@ class QueryParsingErrorsSuite extends QueryTest with QueryErrorsSuiteBase { "'org.apache.hadoop.hive.serde2.OpenCSVSerde' USING 'a' FROM t", errorClass = "UNSUPPORTED_FEATURE", errorSubClass = Some("TRANSFORM_NON_HIVE"), - sqlState = "0A000", - message = - """The feature is not supported: TRANSFORM with SERDE is only supported in hive mode.(line 1, pos 0) - | - |== SQL == - |SELECT TRANSFORM(a) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.OpenCSVSerde' USING 'a' FROM t - |^^^ - |""".stripMargin) + sqlState = "0A000") } test("INVALID_SQL_SYNTAX: Too many arguments for transform") { @@ -186,13 +121,7 @@ class QueryParsingErrorsSuite extends QueryTest with QueryErrorsSuiteBase { sqlText = "CREATE TABLE table(col int) PARTITIONED BY (years(col,col))", errorClass = "INVALID_SQL_SYNTAX", sqlState = "42000", - message = - """Invalid SQL syntax: Too many arguments for transform `years`(line 1, pos 44) - | - |== SQL == - |CREATE TABLE table(col int) PARTITIONED BY (years(col,col)) - |--------------------------------------------^^^ - |""".stripMargin) + parameters = Map("inputString" -> "Too many arguments for transform `years`")) } test("INVALID_SQL_SYNTAX: Invalid table value function name") { @@ -200,26 +129,13 @@ class QueryParsingErrorsSuite extends QueryTest with QueryErrorsSuiteBase { sqlText = "SELECT * FROM db.func()", errorClass = "INVALID_SQL_SYNTAX", sqlState = "42000", - message = - """Invalid SQL syntax: table valued function cannot specify database name (line 1, pos 14) - | - |== SQL == - |SELECT * FROM db.func() - |--------------^^^ - |""".stripMargin - ) + parameters = Map("inputString" -> "table valued function cannot specify database name ")) validateParsingError( sqlText = "SELECT * FROM ns.db.func()", errorClass = "INVALID_SQL_SYNTAX", sqlState = "42000", - message = - """Invalid SQL syntax: table valued function cannot specify database name (line 1, pos 14) - | - |== SQL == - |SELECT * FROM ns.db.func() - |--------------^^^ - |""".stripMargin) + parameters = Map("inputString" -> "table valued function cannot specify database name ")) } test("INVALID_SQL_SYNTAX: Invalid scope in show functions") { @@ -227,13 +143,7 @@ class QueryParsingErrorsSuite extends QueryTest with QueryErrorsSuiteBase { sqlText = "SHOW sys FUNCTIONS", errorClass = "INVALID_SQL_SYNTAX", sqlState = "42000", - message = - """Invalid SQL syntax: SHOW `sys` FUNCTIONS not supported(line 1, pos 5) - | - |== SQL == - |SHOW sys FUNCTIONS - |-----^^^ - |""".stripMargin) + parameters = Map("inputString" -> "SHOW `sys` FUNCTIONS not supported")) } test("INVALID_SQL_SYNTAX: Invalid pattern in show functions") { @@ -241,24 +151,14 @@ class QueryParsingErrorsSuite extends QueryTest with QueryErrorsSuiteBase { sqlText = "SHOW FUNCTIONS IN db f1", errorClass = "INVALID_SQL_SYNTAX", sqlState = "42000", - message = - """Invalid SQL syntax: Invalid pattern in SHOW FUNCTIONS: `f1`. It must be a "STRING" literal.(line 1, pos 21) - | - |== SQL == - |SHOW FUNCTIONS IN db f1 - |---------------------^^^ - |""".stripMargin) + parameters = Map("inputString" -> + "Invalid pattern in SHOW FUNCTIONS: `f1`. It must be a \"STRING\" literal.")) validateParsingError( sqlText = "SHOW FUNCTIONS IN db LIKE f1", errorClass = "INVALID_SQL_SYNTAX", sqlState = "42000", - message = - """Invalid SQL syntax: Invalid pattern in SHOW FUNCTIONS: `f1`. It must be a "STRING" literal.(line 1, pos 26) - | - |== SQL == - |SHOW FUNCTIONS IN db LIKE f1 - |--------------------------^^^ - |""".stripMargin) + parameters = Map("inputString" -> + "Invalid pattern in SHOW FUNCTIONS: `f1`. It must be a \"STRING\" literal.")) } test("INVALID_SQL_SYNTAX: Create function with both if not exists and replace") { @@ -268,23 +168,13 @@ class QueryParsingErrorsSuite extends QueryTest with QueryErrorsSuiteBase { |'com.matthewrathbone.example.SimpleUDFExample' USING JAR '/path/to/jar1', |JAR '/path/to/jar2' |""".stripMargin - val errorDesc = - """CREATE FUNCTION with both IF NOT EXISTS and REPLACE is not allowed.(line 2, pos 0)""" validateParsingError( sqlText = sqlText, errorClass = "INVALID_SQL_SYNTAX", sqlState = "42000", - message = - s"""Invalid SQL syntax: $errorDesc - | - |== SQL == - | - |CREATE OR REPLACE FUNCTION IF NOT EXISTS func1 as - |^^^ - |'com.matthewrathbone.example.SimpleUDFExample' USING JAR '/path/to/jar1', - |JAR '/path/to/jar2' - |""".stripMargin) + parameters = Map("inputString" -> + "CREATE FUNCTION with both IF NOT EXISTS and REPLACE is not allowed.")) } test("INVALID_SQL_SYNTAX: Create temporary function with if not exists") { @@ -294,23 +184,13 @@ class QueryParsingErrorsSuite extends QueryTest with QueryErrorsSuiteBase { |'com.matthewrathbone.example.SimpleUDFExample' USING JAR '/path/to/jar1', |JAR '/path/to/jar2' |""".stripMargin - val errorDesc = - """It is not allowed to define a TEMPORARY FUNCTION with IF NOT EXISTS.(line 2, pos 0)""" validateParsingError( sqlText = sqlText, errorClass = "INVALID_SQL_SYNTAX", sqlState = "42000", - message = - s"""Invalid SQL syntax: $errorDesc - | - |== SQL == - | - |CREATE TEMPORARY FUNCTION IF NOT EXISTS func1 as - |^^^ - |'com.matthewrathbone.example.SimpleUDFExample' USING JAR '/path/to/jar1', - |JAR '/path/to/jar2' - |""".stripMargin) + parameters = Map("inputString" -> + "It is not allowed to define a TEMPORARY FUNCTION with IF NOT EXISTS.")) } test("INVALID_SQL_SYNTAX: Create temporary function with multi-part name") { @@ -325,16 +205,7 @@ class QueryParsingErrorsSuite extends QueryTest with QueryErrorsSuiteBase { sqlText = sqlText, errorClass = "INVALID_SQL_SYNTAX", sqlState = "42000", - message = - """Invalid SQL syntax: Unsupported function name `ns`.`db`.`func`(line 2, pos 0) - | - |== SQL == - | - |CREATE TEMPORARY FUNCTION ns.db.func as - |^^^ - |'com.matthewrathbone.example.SimpleUDFExample' USING JAR '/path/to/jar1', - |JAR '/path/to/jar2' - |""".stripMargin) + parameters = Map("inputString" -> "Unsupported function name `ns`.`db`.`func`")) } test("INVALID_SQL_SYNTAX: Specifying database while creating temporary function") { @@ -344,40 +215,22 @@ class QueryParsingErrorsSuite extends QueryTest with QueryErrorsSuiteBase { |'com.matthewrathbone.example.SimpleUDFExample' USING JAR '/path/to/jar1', |JAR '/path/to/jar2' |""".stripMargin - val errorDesc = - """Specifying a database in CREATE TEMPORARY FUNCTION is not allowed: `db`(line 2, pos 0)""" validateParsingError( sqlText = sqlText, errorClass = "INVALID_SQL_SYNTAX", sqlState = "42000", - message = - s"""Invalid SQL syntax: $errorDesc - | - |== SQL == - | - |CREATE TEMPORARY FUNCTION db.func as - |^^^ - |'com.matthewrathbone.example.SimpleUDFExample' USING JAR '/path/to/jar1', - |JAR '/path/to/jar2' - |""".stripMargin) + parameters = Map("inputString" -> + "Specifying a database in CREATE TEMPORARY FUNCTION is not allowed: `db`")) } test("INVALID_SQL_SYNTAX: Drop temporary function requires a single part name") { - val errorDesc = - "DROP TEMPORARY FUNCTION requires a single part name but got: `db`.`func`(line 1, pos 0)" - validateParsingError( sqlText = "DROP TEMPORARY FUNCTION db.func", errorClass = "INVALID_SQL_SYNTAX", sqlState = "42000", - message = - s"""Invalid SQL syntax: $errorDesc - | - |== SQL == - |DROP TEMPORARY FUNCTION db.func - |^^^ - |""".stripMargin) + parameters = Map("inputString" -> + "DROP TEMPORARY FUNCTION requires a single part name but got: `db`.`func`")) } test("DUPLICATE_KEY: Found duplicate partition keys") { @@ -385,13 +238,7 @@ class QueryParsingErrorsSuite extends QueryTest with QueryErrorsSuiteBase { sqlText = "INSERT OVERWRITE TABLE table PARTITION(p1='1', p1='1') SELECT 'col1', 'col2'", errorClass = "DUPLICATE_KEY", sqlState = "23000", - message = - """Found duplicate keys `p1`(line 1, pos 29) - | - |== SQL == - |INSERT OVERWRITE TABLE table PARTITION(p1='1', p1='1') SELECT 'col1', 'col2' - |-----------------------------^^^ - |""".stripMargin) + parameters = Map("keyColumn" -> "`p1`")) } test("DUPLICATE_KEY: in table properties") { @@ -399,51 +246,24 @@ class QueryParsingErrorsSuite extends QueryTest with QueryErrorsSuiteBase { sqlText = "ALTER TABLE dbx.tab1 SET TBLPROPERTIES ('key1' = '1', 'key1' = '2')", errorClass = "DUPLICATE_KEY", sqlState = "23000", - message = - """Found duplicate keys `key1`(line 1, pos 39) - | - |== SQL == - |ALTER TABLE dbx.tab1 SET TBLPROPERTIES ('key1' = '1', 'key1' = '2') - |---------------------------------------^^^ - |""".stripMargin) + parameters = Map("keyColumn" -> "`key1`")) } test("PARSE_EMPTY_STATEMENT: empty input") { validateParsingError( sqlText = "", errorClass = "PARSE_EMPTY_STATEMENT", - sqlState = "42000", - message = - """Syntax error, unexpected empty statement(line 1, pos 0) - | - |== SQL == - | - |^^^ - |""".stripMargin) + sqlState = "42000") validateParsingError( sqlText = " ", errorClass = "PARSE_EMPTY_STATEMENT", - sqlState = "42000", - message = - s"""Syntax error, unexpected empty statement(line 1, pos 3) - | - |== SQL == - |${" "} - |---^^^ - |""".stripMargin) + sqlState = "42000") validateParsingError( sqlText = " \n", errorClass = "PARSE_EMPTY_STATEMENT", - sqlState = "42000", - message = - s"""Syntax error, unexpected empty statement(line 2, pos 0) - | - |== SQL == - |${" "} - |^^^ - |""".stripMargin) + sqlState = "42000") } test("PARSE_SYNTAX_ERROR: no viable input") { @@ -452,13 +272,7 @@ class QueryParsingErrorsSuite extends QueryTest with QueryErrorsSuiteBase { sqlText = sqlText, errorClass = "PARSE_SYNTAX_ERROR", sqlState = "42000", - message = - s"""Syntax error at or near end of input(line 1, pos 16) - | - |== SQL == - |$sqlText - |----------------^^^ - |""".stripMargin) + parameters = Map("error" -> "end of input", "hint" -> "")) } test("PARSE_SYNTAX_ERROR: extraneous input") { @@ -466,26 +280,13 @@ class QueryParsingErrorsSuite extends QueryTest with QueryErrorsSuiteBase { sqlText = "select 1 1", errorClass = "PARSE_SYNTAX_ERROR", sqlState = "42000", - message = - """Syntax error at or near '1': extra input '1'(line 1, pos 9) - | - |== SQL == - |select 1 1 - |---------^^^ - |""".stripMargin) + parameters = Map("error" -> "'1'", "hint" -> ": extra input '1'")) validateParsingError( sqlText = "select *\nfrom r as q t", errorClass = "PARSE_SYNTAX_ERROR", sqlState = "42000", - message = - """Syntax error at or near 't': extra input 't'(line 2, pos 12) - | - |== SQL == - |select * - |from r as q t - |------------^^^ - |""".stripMargin) + parameters = Map("error" -> "'t'", "hint" -> ": extra input 't'")) } test("PARSE_SYNTAX_ERROR: mismatched input") { @@ -493,28 +294,13 @@ class QueryParsingErrorsSuite extends QueryTest with QueryErrorsSuiteBase { sqlText = "select * from r order by q from t", errorClass = "PARSE_SYNTAX_ERROR", sqlState = "42000", - message = - """Syntax error at or near 'from'(line 1, pos 27) - | - |== SQL == - |select * from r order by q from t - |---------------------------^^^ - |""".stripMargin) + parameters = Map("error" -> "'from'", "hint" -> "")) validateParsingError( sqlText = "select *\nfrom r\norder by q\nfrom t", errorClass = "PARSE_SYNTAX_ERROR", sqlState = "42000", - message = - """Syntax error at or near 'from'(line 4, pos 0) - | - |== SQL == - |select * - |from r - |order by q - |from t - |^^^ - |""".stripMargin) + parameters = Map("error" -> "'from'", "hint" -> "")) } test("PARSE_SYNTAX_ERROR: jargon token substitute to user-facing language") { @@ -523,25 +309,13 @@ class QueryParsingErrorsSuite extends QueryTest with QueryErrorsSuiteBase { sqlText = "select count(*", errorClass = "PARSE_SYNTAX_ERROR", sqlState = "42000", - message = - """Syntax error at or near end of input(line 1, pos 14) - | - |== SQL == - |select count(* - |--------------^^^ - |""".stripMargin) + parameters = Map("error" -> "end of input", "hint" -> "")) validateParsingError( sqlText = "select 1 as a from", errorClass = "PARSE_SYNTAX_ERROR", sqlState = "42000", - message = - """Syntax error at or near end of input(line 1, pos 18) - | - |== SQL == - |select 1 as a from - |------------------^^^ - |""".stripMargin) + parameters = Map("error" -> "end of input", "hint" -> "")) } test("PARSE_SYNTAX_ERROR - SPARK-21136: " + @@ -550,37 +324,19 @@ class QueryParsingErrorsSuite extends QueryTest with QueryErrorsSuiteBase { sqlText = "select * from a left join_ b on a.id = b.id", errorClass = "PARSE_SYNTAX_ERROR", sqlState = "42000", - message = - """Syntax error at or near 'join_': missing 'JOIN'(line 1, pos 21) - | - |== SQL == - |select * from a left join_ b on a.id = b.id - |---------------------^^^ - |""".stripMargin) + parameters = Map("error" -> "'join_'", "hint" -> ": missing 'JOIN'")) validateParsingError( sqlText = "select * from test where test.t is like 'test'", errorClass = "PARSE_SYNTAX_ERROR", sqlState = "42000", - message = - """Syntax error at or near 'is'(line 1, pos 32) - | - |== SQL == - |select * from test where test.t is like 'test' - |--------------------------------^^^ - |""".stripMargin) + parameters = Map("error" -> "'is'", "hint" -> "")) validateParsingError( sqlText = "SELECT * FROM test WHERE x NOT NULL", errorClass = "PARSE_SYNTAX_ERROR", sqlState = "42000", - message = - """Syntax error at or near 'NOT'(line 1, pos 27) - | - |== SQL == - |SELECT * FROM test WHERE x NOT NULL - |---------------------------^^^ - |""".stripMargin) + parameters = Map("error" -> "'NOT'", "hint" -> "")) } test("INVALID_SQL_SYNTAX: show table partition key must set value") { @@ -588,13 +344,7 @@ class QueryParsingErrorsSuite extends QueryTest with QueryErrorsSuiteBase { sqlText = "SHOW TABLE EXTENDED IN default LIKE 'employee' PARTITION (grade)", errorClass = "INVALID_SQL_SYNTAX", sqlState = "42000", - message = - """Invalid SQL syntax: Partition key `grade` must set value (can't be empty).(line 1, pos 47) - | - |== SQL == - |SHOW TABLE EXTENDED IN default LIKE 'employee' PARTITION (grade) - |-----------------------------------------------^^^ - |""".stripMargin) + parameters = Map("inputString" -> "Partition key `grade` must set value (can't be empty).")) } test("INVALID_SQL_SYNTAX: expected a column reference for transform bucket") { @@ -603,13 +353,7 @@ class QueryParsingErrorsSuite extends QueryTest with QueryErrorsSuiteBase { "CREATE TABLE my_tab(a INT, b STRING) USING parquet PARTITIONED BY (bucket(32, a, 66))", errorClass = "INVALID_SQL_SYNTAX", sqlState = "42000", - message = - """Invalid SQL syntax: Expected a column reference for transform `bucket`: 66(line 1, pos 67) - | - |== SQL == - |CREATE TABLE my_tab(a INT, b STRING) USING parquet PARTITIONED BY (bucket(32, a, 66)) - |-------------------------------------------------------------------^^^ - |""".stripMargin) + parameters = Map("inputString" -> "Expected a column reference for transform `bucket`: 66")) } test("UNSUPPORTED_FEATURE: DESC TABLE COLUMN for a specific partition") { @@ -617,16 +361,7 @@ class QueryParsingErrorsSuite extends QueryTest with QueryErrorsSuiteBase { sqlText = "DESCRIBE TABLE EXTENDED customer PARTITION (grade = 'A') customer.age", errorClass = "UNSUPPORTED_FEATURE", errorSubClass = Some("DESC_TABLE_COLUMN_PARTITION"), - sqlState = "0A000", - message = - """The feature is not supported: DESC TABLE COLUMN for a specific partition""" + - """.(line 1, pos 0)""" + - """| - | - |== SQL == - |DESCRIBE TABLE EXTENDED customer PARTITION (grade = 'A') customer.age - |^^^ - |""".stripMargin) + sqlState = "0A000") } test("INVALID_SQL_SYNTAX: PARTITION specification is incomplete") { @@ -634,13 +369,7 @@ class QueryParsingErrorsSuite extends QueryTest with QueryErrorsSuiteBase { sqlText = "DESCRIBE TABLE EXTENDED customer PARTITION (grade)", errorClass = "INVALID_SQL_SYNTAX", sqlState = "42000", - message = - """Invalid SQL syntax: PARTITION specification is incomplete: `grade`(line 1, pos 0) - | - |== SQL == - |DESCRIBE TABLE EXTENDED customer PARTITION (grade) - |^^^ - |""".stripMargin) + parameters = Map("inputString" -> "PARTITION specification is incomplete: `grade`")) } test("UNSUPPORTED_FEATURE: cannot set reserved namespace property") { @@ -650,15 +379,9 @@ class QueryParsingErrorsSuite extends QueryTest with QueryErrorsSuiteBase { errorClass = "UNSUPPORTED_FEATURE", errorSubClass = Some("SET_NAMESPACE_PROPERTY"), sqlState = "0A000", - message = - """The feature is not supported: location is a reserved namespace property, """ + - """please use the LOCATION clause to specify it.(line 1, pos 0)""" + - s""" - | - |== SQL == - |$sql - |^^^ - |""".stripMargin) + parameters = Map( + "property" -> "location", + "msg" -> "please use the LOCATION clause to specify it")) } test("UNSUPPORTED_FEATURE: cannot set reserved table property") { @@ -669,15 +392,9 @@ class QueryParsingErrorsSuite extends QueryTest with QueryErrorsSuiteBase { errorClass = "UNSUPPORTED_FEATURE", errorSubClass = Some("SET_TABLE_PROPERTY"), sqlState = "0A000", - message = - """The feature is not supported: provider is a reserved table property, """ + - """please use the USING clause to specify it.(line 1, pos 66)""" + - s""" - | - |== SQL == - |$sql - |------------------------------------------------------------------^^^ - |""".stripMargin) + parameters = Map( + "property" -> "provider", + "msg" -> "please use the USING clause to specify it")) } test("INVALID_PROPERTY_KEY: invalid property key for set quoted configuration") { @@ -686,13 +403,7 @@ class QueryParsingErrorsSuite extends QueryTest with QueryErrorsSuiteBase { sqlText = sql, errorClass = "INVALID_PROPERTY_KEY", sqlState = null, - message = - s""""" is an invalid property key, please use quotes, e.g. SET ""="value"(line 1, pos 0) - | - |== SQL == - |$sql - |^^^ - |""".stripMargin) + parameters = Map("key" -> "\"\"", "value" -> "\"value\"")) } test("INVALID_PROPERTY_VALUE: invalid property value for set quoted configuration") { @@ -701,15 +412,7 @@ class QueryParsingErrorsSuite extends QueryTest with QueryErrorsSuiteBase { sqlText = sql, errorClass = "INVALID_PROPERTY_VALUE", sqlState = null, - message = - """"1;2;;" is an invalid property value, please use quotes, """ + - """e.g. SET "key"="1;2;;"(line 1, pos 0)""" + - s""" - | - |== SQL == - |$sql - |^^^ - |""".stripMargin) + parameters = Map("value" -> "\"1;2;;\"", "key" -> "\"key\"")) } test("UNSUPPORTED_FEATURE: cannot set Properties and DbProperties at the same time") { @@ -719,15 +422,6 @@ class QueryParsingErrorsSuite extends QueryTest with QueryErrorsSuiteBase { sqlText = sql, errorClass = "UNSUPPORTED_FEATURE", errorSubClass = Some("SET_PROPERTIES_AND_DBPROPERTIES"), - sqlState = "0A000", - message = - """The feature is not supported: set PROPERTIES and DBPROPERTIES at the same time.""" + - """(line 1, pos 0)""" + - s""" - | - |== SQL == - |$sql - |^^^ - |""".stripMargin) + sqlState = "0A000") } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowPartitionsParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowPartitionsParserSuite.scala index 8f75f5df7b9eb..c9cc59114849d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowPartitionsParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowPartitionsParserSuite.scala @@ -48,18 +48,12 @@ class ShowPartitionsParserSuite extends AnalysisTest with QueryErrorsSuiteBase { } test("empty values in non-optional partition specs") { - checkParsingError( + checkError( exception = intercept[ParseException] { new SparkSqlParser().parsePlan("SHOW PARTITIONS dbx.tab1 PARTITION (a='1', b)") }, errorClass = "INVALID_SQL_SYNTAX", sqlState = "42000", - message = - """Invalid SQL syntax: Partition key `b` must set value (can't be empty).(line 1, pos 25) - | - |== SQL == - |SHOW PARTITIONS dbx.tab1 PARTITION (a='1', b) - |-------------------------^^^ - |""".stripMargin) + parameters = Map("inputString" -> "Partition key `b` must set value (can't be empty).")) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/TruncateTableParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/TruncateTableParserSuite.scala index 761e42227920e..ef45cdebbe895 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/TruncateTableParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/TruncateTableParserSuite.scala @@ -47,18 +47,12 @@ class TruncateTableParserSuite extends AnalysisTest with QueryErrorsSuiteBase { } test("empty values in non-optional partition specs") { - checkParsingError( + checkError( exception = intercept[ParseException] { parsePlan("TRUNCATE TABLE dbx.tab1 PARTITION (a='1', b)") }, errorClass = "INVALID_SQL_SYNTAX", sqlState = "42000", - message = - """Invalid SQL syntax: Partition key `b` must set value (can't be empty).(line 1, pos 24) - | - |== SQL == - |TRUNCATE TABLE dbx.tab1 PARTITION (a='1', b) - |------------------------^^^ - |""".stripMargin) + parameters = Map("inputString" -> "Partition key `b` must set value (can't be empty).")) } }