From 7ecad3cff4f8a62b97b729d529ddd68f7fe8ad65 Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Tue, 26 Apr 2022 12:33:05 +0300 Subject: [PATCH 1/4] Upper case SQL statements --- .../scala/org/apache/spark/sql/errors/QueryErrorsBase.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryErrorsBase.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryErrorsBase.scala index 1f8fa1e1b4c86..caf24156c8f3f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryErrorsBase.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryErrorsBase.scala @@ -52,9 +52,8 @@ trait QueryErrorsBase { "\"" + elem + "\"" } - // Quote sql statements in error messages. def toSQLStmt(text: String): String = { - quoteByDefault(text.toUpperCase(Locale.ROOT)) + text.toUpperCase(Locale.ROOT) } def toSQLId(parts: Seq[String]): String = { From 48c6ab63443f1dc7014673e0787a9af01fb1f11c Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Tue, 26 Apr 2022 13:32:25 +0300 Subject: [PATCH 2/4] Fix tests --- ...tractPythonUDFFromJoinConditionSuite.scala | 2 +- .../sql/catalyst/parser/PlanParserSuite.scala | 2 +- .../sql-tests/results/describe.sql.out | 4 +-- .../sql-tests/results/join-lateral.sql.out | 4 +-- .../sql-tests/results/transform.sql.out | 4 +-- .../QueryCompilationErrorsDSv2Suite.scala | 2 +- .../errors/QueryCompilationErrorsSuite.scala | 6 ++-- .../errors/QueryExecutionErrorsSuite.scala | 4 +-- .../sql/errors/QueryParsingErrorsSuite.scala | 28 +++++++++---------- 9 files changed, 28 insertions(+), 28 deletions(-) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ExtractPythonUDFFromJoinConditionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ExtractPythonUDFFromJoinConditionSuite.scala index f69f782a0a7de..c3edc5e932910 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ExtractPythonUDFFromJoinConditionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ExtractPythonUDFFromJoinConditionSuite.scala @@ -189,7 +189,7 @@ class ExtractPythonUDFFromJoinConditionSuite extends PlanTest { } assert(e.message == "[UNSUPPORTED_FEATURE] The feature is not supported: " + - s"""Using PythonUDF in join condition of join type "${joinType.sql}" is not supported.""") + s"""Using PythonUDF in join condition of join type ${joinType.sql} is not supported.""") val query2 = testRelationLeft.join( testRelationRight, diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala index 77fa99b14509b..15880548fda86 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala @@ -1253,7 +1253,7 @@ class PlanParserSuite extends AnalysisTest { | "escapeChar" = "\\") |FROM testData """.stripMargin, - "\"TRANSFORM\" with serde is only supported in hive mode") + "TRANSFORM with serde is only supported in hive mode") } 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 8de02a0c4f9a0..9d1dba8780de4 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 @@ -462,7 +462,7 @@ DESC temp_v PARTITION (c='Us', d=1) struct<> -- !query output org.apache.spark.sql.AnalysisException -[FORBIDDEN_OPERATION] The operation "DESC PARTITION" is not allowed on the temporary view: `temp_v` +[FORBIDDEN_OPERATION] The operation DESC PARTITION is not allowed on the temporary view: `temp_v` -- !query @@ -539,7 +539,7 @@ DESC v PARTITION (c='Us', d=1) struct<> -- !query output org.apache.spark.sql.AnalysisException -[FORBIDDEN_OPERATION] The operation "DESC PARTITION" is not allowed on the view: `v` +[FORBIDDEN_OPERATION] The operation DESC PARTITION is not allowed on the view: `v` -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/join-lateral.sql.out b/sql/core/src/test/resources/sql-tests/results/join-lateral.sql.out index 7b00ab1cb92ce..5b36a7959aefb 100644 --- a/sql/core/src/test/resources/sql-tests/results/join-lateral.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/join-lateral.sql.out @@ -153,7 +153,7 @@ struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException -[UNSUPPORTED_FEATURE] The feature is not supported: "LATERAL" join with "NATURAL" join.(line 1, pos 14) +[UNSUPPORTED_FEATURE] The feature is not supported: LATERAL join with NATURAL join.(line 1, pos 14) == SQL == SELECT * FROM t1 NATURAL JOIN LATERAL (SELECT c1 + c2 AS c2) @@ -167,7 +167,7 @@ struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException -[UNSUPPORTED_FEATURE] The feature is not supported: "LATERAL" join with "USING" join.(line 1, pos 14) +[UNSUPPORTED_FEATURE] The feature is not supported: LATERAL join with USING join.(line 1, pos 14) == SQL == SELECT * FROM t1 JOIN LATERAL (SELECT c1 + c2 AS c2) USING (c2) diff --git a/sql/core/src/test/resources/sql-tests/results/transform.sql.out b/sql/core/src/test/resources/sql-tests/results/transform.sql.out index 3d0fdc81b3614..6d00f1b68f248 100644 --- a/sql/core/src/test/resources/sql-tests/results/transform.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/transform.sql.out @@ -719,7 +719,7 @@ struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException -[UNSUPPORTED_FEATURE] The feature is not supported: "TRANSFORM" does not support "DISTINCT"/"ALL" in inputs(line 1, pos 17) +[UNSUPPORTED_FEATURE] The feature is not supported: TRANSFORM does not support DISTINCT/ALL in inputs(line 1, pos 17) == SQL == SELECT TRANSFORM(DISTINCT b, a, c) @@ -739,7 +739,7 @@ struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException -[UNSUPPORTED_FEATURE] The feature is not supported: "TRANSFORM" does not support "DISTINCT"/"ALL" in inputs(line 1, pos 17) +[UNSUPPORTED_FEATURE] The feature is not supported: TRANSFORM does not support DISTINCT/ALL in inputs(line 1, pos 17) == SQL == SELECT TRANSFORM(ALL b, a, c) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryCompilationErrorsDSv2Suite.scala b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryCompilationErrorsDSv2Suite.scala index 9fedf14a2a1dc..27b7dd3ddb308 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryCompilationErrorsDSv2Suite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryCompilationErrorsDSv2Suite.scala @@ -54,7 +54,7 @@ class QueryCompilationErrorsDSv2Suite exception = e, errorClass = "UNSUPPORTED_FEATURE", msg = "The feature is not supported: " + - s""""IF NOT EXISTS" for the table `testcat`.`ns1`.`ns2`.`tbl` by "INSERT INTO".""", + s"""IF NOT EXISTS for the table `testcat`.`ns1`.`ns2`.`tbl` by INSERT INTO.""", sqlState = Some("0A000")) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryCompilationErrorsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryCompilationErrorsSuite.scala index f1325a683666a..27e4b059dec7a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryCompilationErrorsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryCompilationErrorsSuite.scala @@ -166,7 +166,7 @@ class QueryCompilationErrorsSuite exception = e, errorClass = "UNSUPPORTED_FEATURE", msg = "The feature is not supported: " + - "Using PythonUDF in join condition of join type \"LEFT OUTER\" is not supported.", + "Using PythonUDF in join condition of join type LEFT OUTER is not supported.", sqlState = Some("0A000")) } @@ -349,7 +349,7 @@ class QueryCompilationErrorsSuite sql(s"DESC TABLE $tempViewName PARTITION (c='Us', d=1)") }, errorClass = "FORBIDDEN_OPERATION", - msg = s"""The operation "DESC PARTITION" is not allowed """ + + msg = s"""The operation DESC PARTITION is not allowed """ + s"on the temporary view: `$tempViewName`") } } @@ -375,7 +375,7 @@ class QueryCompilationErrorsSuite sql(s"DESC TABLE $viewName PARTITION (c='Us', d=1)") }, errorClass = "FORBIDDEN_OPERATION", - msg = s"""The operation "DESC PARTITION" is not allowed """ + + msg = s"""The operation DESC PARTITION is not allowed """ + s"on the view: `$viewName`") } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala index f84f159f6f0da..cfc0b6fa30c47 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala @@ -177,7 +177,7 @@ class QueryExecutionErrorsSuite checkErrorClass( exception = e1, errorClass = "UNSUPPORTED_FEATURE", - msg = """The feature is not supported: Repeated "PIVOT"s.""", + msg = """The feature is not supported: Repeated PIVOTs.""", sqlState = Some("0A000")) val e2 = intercept[SparkUnsupportedOperationException] { @@ -190,7 +190,7 @@ class QueryExecutionErrorsSuite checkErrorClass( exception = e2, errorClass = "UNSUPPORTED_FEATURE", - msg = """The feature is not supported: "PIVOT" not after a "GROUP BY".""", + msg = """The feature is not supported: PIVOT not after a GROUP BY.""", sqlState = Some("0A000")) } 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 6ff737ec64753..ad884c123c42e 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 @@ -29,7 +29,7 @@ class QueryParsingErrorsSuite extends QueryTest with QueryErrorsSuiteBase { errorClass = "UNSUPPORTED_FEATURE", sqlState = "0A000", message = - """The feature is not supported: "LATERAL" join with "NATURAL" join.(line 1, pos 14) + """The feature is not supported: LATERAL join with NATURAL join.(line 1, pos 14) | |== SQL == |SELECT * FROM t1 NATURAL JOIN LATERAL (SELECT c1 + c2 AS c2) @@ -43,7 +43,7 @@ class QueryParsingErrorsSuite extends QueryTest with QueryErrorsSuiteBase { errorClass = "UNSUPPORTED_FEATURE", sqlState = "0A000", message = - """The feature is not supported: "LATERAL" join with "USING" join.(line 1, pos 14) + """The feature is not supported: LATERAL join with USING join.(line 1, pos 14) | |== SQL == |SELECT * FROM t1 JOIN LATERAL (SELECT c1 + c2 AS c2) USING (c2) @@ -58,7 +58,7 @@ class QueryParsingErrorsSuite extends QueryTest with QueryErrorsSuiteBase { errorClass = "UNSUPPORTED_FEATURE", sqlState = "0A000", message = - s"""The feature is not supported: "LATERAL" join type "$joinType".(line 1, pos 14) + s"""The feature is not supported: LATERAL join type $joinType.(line 1, pos 14) | |== SQL == |SELECT * FROM t1 $joinType JOIN LATERAL (SELECT c1 + c2 AS c3) ON c2 = c3 @@ -81,7 +81,7 @@ class QueryParsingErrorsSuite extends QueryTest with QueryErrorsSuiteBase { errorClass = "INVALID_SQL_SYNTAX", sqlState = "42000", message = - s"""Invalid SQL syntax: "LATERAL" can only be used with subquery.(line 1, pos $pos) + s"""Invalid SQL syntax: LATERAL can only be used with subquery.(line 1, pos $pos) | |== SQL == |$sqlText @@ -96,7 +96,7 @@ class QueryParsingErrorsSuite extends QueryTest with QueryErrorsSuiteBase { errorClass = "UNSUPPORTED_FEATURE", sqlState = "0A000", message = - """The feature is not supported: "NATURAL CROSS JOIN".(line 1, pos 14) + """The feature is not supported: NATURAL CROSS JOIN.(line 1, pos 14) | |== SQL == |SELECT * FROM a NATURAL CROSS JOIN b @@ -152,7 +152,7 @@ class QueryParsingErrorsSuite extends QueryTest with QueryErrorsSuiteBase { errorClass = "UNSUPPORTED_FEATURE", sqlState = "0A000", message = - """The feature is not supported: "TRANSFORM" does not support "DISTINCT"/"ALL" in inputs(line 1, pos 17) + """The feature is not supported: TRANSFORM does not support DISTINCT/ALL in inputs(line 1, pos 17) | |== SQL == |SELECT TRANSFORM(DISTINCT a) USING 'a' FROM t @@ -167,7 +167,7 @@ class QueryParsingErrorsSuite extends QueryTest with QueryErrorsSuiteBase { errorClass = "UNSUPPORTED_FEATURE", sqlState = "0A000", message = - """The feature is not supported: "TRANSFORM" with serde is only supported in hive mode(line 1, pos 0) + """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 @@ -222,7 +222,7 @@ class QueryParsingErrorsSuite extends QueryTest with QueryErrorsSuiteBase { errorClass = "INVALID_SQL_SYNTAX", sqlState = "42000", message = - """Invalid SQL syntax: "SHOW" `sys` "FUNCTIONS" not supported(line 1, pos 5) + """Invalid SQL syntax: SHOW `sys` FUNCTIONS not supported(line 1, pos 5) | |== SQL == |SHOW sys FUNCTIONS @@ -236,7 +236,7 @@ class QueryParsingErrorsSuite extends QueryTest with QueryErrorsSuiteBase { 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) + """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 @@ -247,7 +247,7 @@ class QueryParsingErrorsSuite extends QueryTest with QueryErrorsSuiteBase { 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) + """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 @@ -263,7 +263,7 @@ class QueryParsingErrorsSuite extends QueryTest with QueryErrorsSuiteBase { |JAR '/path/to/jar2' |""".stripMargin val errorDesc = - """"CREATE FUNCTION" with both "IF NOT EXISTS" and "REPLACE" is not allowed.(line 2, pos 0)""" + """CREATE FUNCTION with both IF NOT EXISTS and REPLACE is not allowed.(line 2, pos 0)""" validateParsingError( sqlText = sqlText, @@ -289,7 +289,7 @@ class QueryParsingErrorsSuite extends QueryTest with QueryErrorsSuiteBase { |JAR '/path/to/jar2' |""".stripMargin val errorDesc = - """It is not allowed to define a "TEMPORARY FUNCTION" with "IF NOT EXISTS".(line 2, pos 0)""" + """It is not allowed to define a TEMPORARY FUNCTION with IF NOT EXISTS.(line 2, pos 0)""" validateParsingError( sqlText = sqlText, @@ -339,7 +339,7 @@ class QueryParsingErrorsSuite extends QueryTest with QueryErrorsSuiteBase { |JAR '/path/to/jar2' |""".stripMargin val errorDesc = - """Specifying a database in "CREATE TEMPORARY FUNCTION" is not allowed: `db`(line 2, pos 0)""" + """Specifying a database in CREATE TEMPORARY FUNCTION is not allowed: `db`(line 2, pos 0)""" validateParsingError( sqlText = sqlText, @@ -359,7 +359,7 @@ class QueryParsingErrorsSuite extends QueryTest with QueryErrorsSuiteBase { 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)" + "DROP TEMPORARY FUNCTION requires a single part name but got: `db`.`func`(line 1, pos 0)" validateParsingError( sqlText = "DROP TEMPORARY FUNCTION db.func", From 485785864632c52e88f252a6e57da70f375d0958 Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Tue, 26 Apr 2022 18:34:16 +0300 Subject: [PATCH 3/4] Fix test failures --- python/pyspark/sql/tests/test_udf.py | 2 +- .../spark/sql/execution/SparkScriptTransformationSuite.scala | 2 +- .../apache/spark/sql/execution/command/DDLParserSuite.scala | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/python/pyspark/sql/tests/test_udf.py b/python/pyspark/sql/tests/test_udf.py index e40c3ba0d6479..40deac992c4bd 100644 --- a/python/pyspark/sql/tests/test_udf.py +++ b/python/pyspark/sql/tests/test_udf.py @@ -258,7 +258,7 @@ def test_udf_not_supported_in_join_condition(self): def runWithJoinType(join_type, type_string): with self.assertRaisesRegex( AnalysisException, - """Using PythonUDF in join condition of join type "%s" is not supported""" + """Using PythonUDF in join condition of join type %s is not supported""" % type_string, ): left.join(right, [f("a", "b"), left.a1 == right.b1], join_type).collect() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkScriptTransformationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkScriptTransformationSuite.scala index 1f431e173b3c7..5638743b7633d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkScriptTransformationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkScriptTransformationSuite.scala @@ -56,7 +56,7 @@ class SparkScriptTransformationSuite extends BaseScriptTransformationSuite with |FROM v """.stripMargin) }.getMessage - assert(e.contains("\"TRANSFORM\" with serde is only supported in hive mode")) + assert(e.contains("TRANSFORM with serde is only supported in hive mode")) } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala index 44694e10c6fcf..05378e3229661 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala @@ -469,9 +469,9 @@ class DDLParserSuite extends AnalysisTest with SharedSparkSession { DropFunctionCommand(Seq("a").asFunctionIdentifier, true, true)) intercept("DROP TEMPORARY FUNCTION a.b", - "\"DROP TEMPORARY FUNCTION\" requires a single part name") + "DROP TEMPORARY FUNCTION requires a single part name") intercept("DROP TEMPORARY FUNCTION IF EXISTS a.b", - "\"DROP TEMPORARY FUNCTION\" requires a single part name") + "DROP TEMPORARY FUNCTION requires a single part name") } test("SPARK-32374: create temporary view with properties not allowed") { From 0e35ddd65477e1806ba836e464b0e44630b3e861 Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Tue, 26 Apr 2022 21:13:01 +0300 Subject: [PATCH 4/4] Trigger build