Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
2 changes: 1 addition & 1 deletion python/pyspark/sql/tests/test_udf.py
Original file line number Diff line number Diff line change
Expand Up @@ -258,7 +258,7 @@ def test_udf_not_supported_in_join_condition(self):
def runWithJoinType(join_type, type_string):
with self.assertRaisesRegex(
AnalysisException,
"""Python UDF in the ON clause of a "%s" JOIN.""" % type_string,
"""Python UDF in the ON clause of a %s JOIN.""" % type_string,
):
left.join(right, [f("a", "b"), left.a1 == right.b1], join_type).collect()

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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 = {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -189,7 +189,7 @@ class ExtractPythonUDFFromJoinConditionSuite extends PlanTest {
}
assert(e.message ==
"[UNSUPPORTED_FEATURE.PYTHON_UDF_IN_ON_CLAUSE] The feature is not supported: " +
s"""Python UDF in the ON clause of a "${joinType.sql}" JOIN.""")
s"""Python UDF in the ON clause of a ${joinType.sql} JOIN.""")

val query2 = testRelationLeft.join(
testRelationRight,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down Expand Up @@ -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
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -167,7 +167,7 @@ class QueryCompilationErrorsSuite
errorClass = "UNSUPPORTED_FEATURE",
errorSubClass = Some("PYTHON_UDF_IN_ON_CLAUSE"),
msg = "The feature is not supported: " +
"Python UDF in the ON clause of a \"LEFT OUTER\" JOIN.",
"Python UDF in the ON clause of a LEFT OUTER JOIN.",
sqlState = Some("0A000"))
}

Expand Down Expand Up @@ -352,7 +352,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`")
}
}
Expand All @@ -378,7 +378,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`")
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -61,7 +61,7 @@ class QueryParsingErrorsSuite extends QueryTest with QueryErrorsSuiteBase {
errorSubClass = Some("LATERAL_JOIN_OF_TYPE"),
sqlState = "0A000",
message =
s"""The feature is not supported: "$joinType" JOIN with LATERAL correlation.(line 1, pos 14)
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
Expand All @@ -84,7 +84,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
Expand Down Expand Up @@ -173,7 +173,7 @@ class QueryParsingErrorsSuite extends QueryTest with QueryErrorsSuiteBase {
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)
"""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
Expand Down Expand Up @@ -228,7 +228,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
Expand All @@ -242,7 +242,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
Expand All @@ -253,7 +253,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
Expand All @@ -269,7 +269,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,
Expand All @@ -295,7 +295,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,
Expand Down Expand Up @@ -345,7 +345,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,
Expand All @@ -365,7 +365,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",
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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") {
Expand Down