Skip to content
2 changes: 1 addition & 1 deletion core/src/main/resources/error/error-classes.json
Original file line number Diff line number Diff line change
Expand Up @@ -38,7 +38,7 @@
"sqlState" : "22012"
},
"DUPLICATE_KEY" : {
"message" : [ "Found duplicate keys '%s'" ],
"message" : [ "Found duplicate keys %s" ],
"sqlState" : "23000"
},
"FAILED_EXECUTE_UDF" : {
Expand Down
2 changes: 1 addition & 1 deletion python/pyspark/sql/tests/test_pandas_udf_grouped_agg.py
Original file line number Diff line number Diff line change
Expand Up @@ -487,7 +487,7 @@ def test_invalid_args(self):
with QuietTest(self.sc):
with self.assertRaisesRegex(
AnalysisException,
"The group aggregate pandas UDF 'avg' cannot be invoked together with as other, "
"The group aggregate pandas UDF `avg` cannot be invoked together with as other, "
"non-pandas aggregate functions.",
):
df.groupby(df.id).agg(mean_udf(df.v), mean(df.v)).collect()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -46,7 +46,7 @@ import org.apache.spark.sql.types._
* As commands are executed eagerly, this also includes errors thrown during the execution of
* commands, which users can see immediately.
*/
object QueryCompilationErrors {
object QueryCompilationErrors extends QueryErrorsBase {

def groupingIDMismatchError(groupingID: GroupingID, groupByExprs: Seq[Expression]): Throwable = {
new AnalysisException(
Expand Down Expand Up @@ -94,13 +94,14 @@ object QueryCompilationErrors {
def unsupportedIfNotExistsError(tableName: String): Throwable = {
new AnalysisException(
errorClass = "UNSUPPORTED_FEATURE",
messageParameters = Array(s"IF NOT EXISTS for the table '$tableName' by INSERT INTO."))
messageParameters = Array(
s"IF NOT EXISTS for the table ${toSQLId(tableName)} by INSERT INTO."))
}

def nonPartitionColError(partitionName: String): Throwable = {
new AnalysisException(
errorClass = "NON_PARTITION_COLUMN",
messageParameters = Array(partitionName))
messageParameters = Array(toSQLId(partitionName)))
}

def missingStaticPartitionColumn(staticName: String): Throwable = {
Expand Down Expand Up @@ -1346,7 +1347,7 @@ object QueryCompilationErrors {
groupAggPandasUDFNames: Seq[String]): Throwable = {
new AnalysisException(
errorClass = "INVALID_PANDAS_UDF_PLACEMENT",
messageParameters = Array(groupAggPandasUDFNames.map(name => s"'$name'").mkString(", ")))
messageParameters = Array(groupAggPandasUDFNames.map(toSQLId).mkString(", ")))
}

def ambiguousAttributesInSelfJoinError(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@
package org.apache.spark.sql.errors

import org.apache.spark.sql.catalyst.expressions.Literal
import org.apache.spark.sql.catalyst.util.quoteIdentifier
import org.apache.spark.sql.types.{DataType, DoubleType, FloatType}

trait QueryErrorsBase {
Expand All @@ -44,4 +45,12 @@ trait QueryErrorsBase {
def toSQLValue(v: Any, t: DataType): String = {
litToErrorValue(Literal.create(v, t))
}

def toSQLId(parts: Seq[String]): String = {
parts.map(quoteIdentifier).mkString(".")
}

def toSQLId(parts: String): String = {
toSQLId(parts.split("\\."))
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -1888,28 +1888,30 @@ object QueryExecutionErrors extends QueryErrorsBase {
messageParameters = Array("Pivot not after a groupBy."))
}

private val aesFuncName = toSQLId("aes_encrypt") + "/" + toSQLId("aes_decrypt")

def invalidAesKeyLengthError(actualLength: Int): RuntimeException = {
new SparkRuntimeException(
errorClass = "INVALID_PARAMETER_VALUE",
messageParameters = Array(
"key",
"the aes_encrypt/aes_decrypt function",
s"the $aesFuncName function",
s"expects a binary value with 16, 24 or 32 bytes, but got ${actualLength.toString} bytes."))
}

def aesModeUnsupportedError(mode: String, padding: String): RuntimeException = {
new SparkRuntimeException(
errorClass = "UNSUPPORTED_FEATURE",
messageParameters = Array(
s"AES-$mode with the padding $padding by the aes_encrypt/aes_decrypt function."))
s"AES-$mode with the padding $padding by the $aesFuncName function."))
}

def aesCryptoError(detailMessage: String): RuntimeException = {
new SparkRuntimeException(
errorClass = "INVALID_PARAMETER_VALUE",
messageParameters = Array(
"expr, key",
"the aes_encrypt/aes_decrypt function",
s"the $aesFuncName function",
s"Detail message: $detailMessage"))
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,13 +22,12 @@ import org.antlr.v4.runtime.ParserRuleContext
import org.apache.spark.sql.catalyst.parser.ParseException
import org.apache.spark.sql.catalyst.parser.SqlBaseParser._
import org.apache.spark.sql.catalyst.trees.Origin
import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._

/**
* Object for grouping all error messages of the query parsing.
* Currently it includes all ParseException.
*/
object QueryParsingErrors {
object QueryParsingErrors extends QueryErrorsBase {

def invalidInsertIntoError(ctx: InsertIntoContext): Throwable = {
new ParseException("Invalid InsertIntoContext", ctx)
Expand Down Expand Up @@ -121,17 +120,17 @@ object QueryParsingErrors {

def repetitiveWindowDefinitionError(name: String, ctx: WindowClauseContext): Throwable = {
new ParseException("INVALID_SQL_SYNTAX",
Array(s"The definition of window '$name' is repetitive."), ctx)
Array(s"The definition of window ${toSQLId(name)} is repetitive."), ctx)
}

def invalidWindowReferenceError(name: String, ctx: WindowClauseContext): Throwable = {
new ParseException("INVALID_SQL_SYNTAX",
Array(s"Window reference '$name' is not a window specification."), ctx)
Array(s"Window reference ${toSQLId(name)} is not a window specification."), ctx)
}

def cannotResolveWindowReferenceError(name: String, ctx: WindowClauseContext): Throwable = {
new ParseException("INVALID_SQL_SYNTAX",
Array(s"Cannot resolve window reference '$name'."), ctx)
Array(s"Cannot resolve window reference ${toSQLId(name)}."), ctx)
}

def naturalCrossJoinUnsupportedError(ctx: RelationContext): Throwable = {
Expand Down Expand Up @@ -162,7 +161,7 @@ object QueryParsingErrors {

def functionNameUnsupportedError(functionName: String, ctx: ParserRuleContext): Throwable = {
new ParseException("INVALID_SQL_SYNTAX",
Array(s"Unsupported function name '$functionName'"), ctx)
Array(s"Unsupported function name ${toSQLId(functionName)}"), ctx)
}

def cannotParseValueTypeError(
Expand Down Expand Up @@ -231,7 +230,10 @@ object QueryParsingErrors {
}

def tooManyArgumentsForTransformError(name: String, ctx: ApplyTransformContext): Throwable = {
new ParseException("INVALID_SQL_SYNTAX", Array(s"Too many arguments for transform $name"), ctx)
new ParseException(
errorClass = "INVALID_SQL_SYNTAX",
messageParameters = Array(s"Too many arguments for transform ${toSQLId(name)}"),
ctx)
}

def invalidBucketsNumberError(describe: String, ctx: ApplyTransformContext): Throwable = {
Expand Down Expand Up @@ -299,12 +301,13 @@ object QueryParsingErrors {

def showFunctionsUnsupportedError(identifier: String, ctx: IdentifierContext): Throwable = {
new ParseException("INVALID_SQL_SYNTAX",
Array(s"SHOW $identifier FUNCTIONS not supported"), ctx)
Array(s"SHOW ${toSQLId(identifier)} FUNCTIONS not supported"), ctx)
}

def showFunctionsInvalidPatternError(pattern: String, ctx: ParserRuleContext): Throwable = {
new ParseException("INVALID_SQL_SYNTAX",
Array(s"Invalid pattern in SHOW FUNCTIONS: $pattern. It must be a string literal."), ctx)
Array(s"Invalid pattern in SHOW FUNCTIONS: ${toSQLId(pattern)}. " +
"It must be a string literal."), ctx)
}

def duplicateCteDefinitionNamesError(duplicateNames: String, ctx: CtesContext): Throwable = {
Expand All @@ -326,7 +329,7 @@ object QueryParsingErrors {

def duplicateKeysError(key: String, ctx: ParserRuleContext): Throwable = {
// Found duplicate keys '$key'
new ParseException(errorClass = "DUPLICATE_KEY", messageParameters = Array(key), ctx)
new ParseException(errorClass = "DUPLICATE_KEY", messageParameters = Array(toSQLId(key)), ctx)
}

def unexpectedFomatForSetConfigurationError(ctx: ParserRuleContext): Throwable = {
Expand Down Expand Up @@ -418,17 +421,18 @@ object QueryParsingErrors {
Array("It is not allowed to define a TEMPORARY function with IF NOT EXISTS."), ctx)
}

def unsupportedFunctionNameError(quoted: String, ctx: CreateFunctionContext): Throwable = {
def unsupportedFunctionNameError(funcName: Seq[String], ctx: CreateFunctionContext): Throwable = {
new ParseException("INVALID_SQL_SYNTAX",
Array(s"Unsupported function name '$quoted'"), ctx)
Array(s"Unsupported function name ${toSQLId(funcName)}"), ctx)
}

def specifyingDBInCreateTempFuncError(
databaseName: String,
ctx: CreateFunctionContext): Throwable = {
new ParseException(
"INVALID_SQL_SYNTAX",
Array(s"Specifying a database in CREATE TEMPORARY FUNCTION is not allowed: '$databaseName'"),
Array("Specifying a database in CREATE TEMPORARY FUNCTION is not allowed: " +
toSQLId(databaseName)),
ctx)
}

Expand All @@ -442,7 +446,7 @@ object QueryParsingErrors {

def invalidNameForDropTempFunc(name: Seq[String], ctx: ParserRuleContext): Throwable = {
new ParseException("INVALID_SQL_SYNTAX",
Array(s"DROP TEMPORARY FUNCTION requires a single part name but got: ${name.quoted}"), ctx)
Array(s"DROP TEMPORARY FUNCTION requires a single part name but got: ${toSQLId(name)}"), ctx)
}

def defaultColumnNotImplementedYetError(ctx: ParserRuleContext): Throwable = {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -2049,12 +2049,6 @@ class DDLParserSuite extends AnalysisTest {
comparePlans(
parsePlan("SHOW FUNCTIONS IN db LIKE 'funct*'"),
ShowFunctions(UnresolvedNamespace(Seq("db")), true, true, Some("funct*")))
val sql = "SHOW other FUNCTIONS"
intercept(sql, s"$sql not supported")
intercept("SHOW FUNCTIONS IN db f1",
"Invalid pattern in SHOW FUNCTIONS: f1")
intercept("SHOW FUNCTIONS IN db LIKE f1",
"Invalid pattern in SHOW FUNCTIONS: f1")

// The legacy syntax.
comparePlans(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -437,18 +437,6 @@ class PlanParserSuite extends AnalysisTest {
| w2 as w1,
| w3 as w1""".stripMargin,
WithWindowDefinition(ws1, plan))

// Fail with no reference.
intercept(s"$sql window w2 as w1", "Cannot resolve window reference 'w1'")

// Fail when resolved reference is not a window spec.
intercept(
s"""$sql
|window w1 as (partition by a, b order by c rows between 1 preceding and 1 following),
| w2 as w1,
| w3 as w2""".stripMargin,
"Window reference 'w2' is not a window specification"
)
}

test("lateral view") {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -563,7 +563,7 @@ class SparkSqlAstBuilder extends AstBuilder {
}

if (functionIdentifier.length > 2) {
throw QueryParsingErrors.unsupportedFunctionNameError(functionIdentifier.quoted, ctx)
throw QueryParsingErrors.unsupportedFunctionNameError(functionIdentifier, ctx)
} else if (functionIdentifier.length == 2) {
// Temporary function names should not contain database prefix like "database.function"
throw QueryParsingErrors.specifyingDBInCreateTempFuncError(functionIdentifier.head, ctx)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -898,7 +898,7 @@ struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException

Invalid SQL syntax: The definition of window 'w' is repetitive.(line 8, pos 0)
Invalid SQL syntax: The definition of window `w` is repetitive.(line 8, pos 0)

== SQL ==
SELECT
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -276,7 +276,7 @@ trait SQLInsertTestSuite extends QueryTest with SQLTestUtils {
val e = intercept[AnalysisException] {
sql("INSERT OVERWRITE t PARTITION (c='2', C='3') VALUES (1)")
}
assert(e.getMessage.contains("Found duplicate keys 'c'"))
assert(e.getMessage.contains("Found duplicate keys `c`"))
}
// The following code is skipped for Hive because columns stored in Hive Metastore is always
// case insensitive and we cannot create such table in Hive Metastore.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -52,7 +52,7 @@ class QueryCompilationErrorsDSv2Suite

checkAnswer(spark.table(tbl), spark.emptyDataFrame)
assert(e.getMessage === "The feature is not supported: " +
s"IF NOT EXISTS for the table '$tbl' by INSERT INTO.")
s"IF NOT EXISTS for the table `testcat`.`ns1`.`ns2`.`tbl` by INSERT INTO.")
assert(e.getErrorClass === "UNSUPPORTED_FEATURE")
assert(e.getSqlState === "0A000")
}
Expand All @@ -69,7 +69,7 @@ class QueryCompilationErrorsDSv2Suite
}

verifyTable(t1, spark.emptyDataFrame)
assert(e.getMessage === "PARTITION clause cannot contain a non-partition column name: id")
assert(e.getMessage === "PARTITION clause cannot contain a non-partition column name: `id`")
assert(e.getErrorClass === "NON_PARTITION_COLUMN")
}
}
Expand All @@ -84,7 +84,7 @@ class QueryCompilationErrorsDSv2Suite
}

verifyTable(t1, spark.emptyDataFrame)
assert(e.getMessage === "PARTITION clause cannot contain a non-partition column name: data")
assert(e.getMessage === "PARTITION clause cannot contain a non-partition column name: `data`")
assert(e.getErrorClass === "NON_PARTITION_COLUMN")
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -126,7 +126,7 @@ class QueryCompilationErrorsSuite extends QueryTest with SharedSparkSession {

assert(e.errorClass === Some("INVALID_PANDAS_UDF_PLACEMENT"))
assert(e.message ===
"The group aggregate pandas UDF 'pandas_udf_1', 'pandas_udf_2' cannot be invoked " +
"The group aggregate pandas UDF `pandas_udf_1`, `pandas_udf_2` cannot be invoked " +
"together with as other, non-pandas aggregate functions.")
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -66,8 +66,8 @@ class QueryExecutionErrorsSuite extends QueryTest
assert(e.getErrorClass === "INVALID_PARAMETER_VALUE")
assert(e.getSqlState === "22023")
assert(e.getMessage.matches(
"The value of parameter\\(s\\) 'key' in the aes_encrypt/aes_decrypt function is invalid: " +
"expects a binary value with 16, 24 or 32 bytes, but got \\d+ bytes."))
"The value of parameter\\(s\\) 'key' in the `aes_encrypt`/`aes_decrypt` function " +
"is invalid: expects a binary value with 16, 24 or 32 bytes, but got \\d+ bytes."))
}

// Encryption failure - invalid key length
Expand Down Expand Up @@ -100,7 +100,7 @@ class QueryExecutionErrorsSuite extends QueryTest
assert(e.getErrorClass === "INVALID_PARAMETER_VALUE")
assert(e.getSqlState === "22023")
assert(e.getMessage ===
"The value of parameter(s) 'expr, key' in the aes_encrypt/aes_decrypt function " +
"The value of parameter(s) 'expr, key' in the `aes_encrypt`/`aes_decrypt` function " +
"is invalid: Detail message: " +
"Given final block not properly padded. " +
"Such issues can arise if a bad key is used during decryption.")
Expand All @@ -118,7 +118,7 @@ class QueryExecutionErrorsSuite extends QueryTest
assert(e.getErrorClass === "UNSUPPORTED_FEATURE")
assert(e.getSqlState === "0A000")
assert(e.getMessage.matches("""The feature is not supported: AES-\w+ with the padding \w+""" +
" by the aes_encrypt/aes_decrypt function."))
" by the `aes_encrypt`/`aes_decrypt` function."))
}

// Unsupported AES mode and padding in encrypt
Expand Down
Loading