Skip to content
10 changes: 5 additions & 5 deletions core/src/main/resources/error/error-classes.json
Original file line number Diff line number Diff line change
Expand Up @@ -1468,6 +1468,11 @@
],
"sqlState" : "42000"
},
"WRONG_NUM_ARGS" : {
"message" : [
"The <functionName> requires <expectedNum> parameters but the actual number is <actualNum>."
]
},
"_LEGACY_ERROR_TEMP_0001" : {
"message" : [
"Invalid InsertIntoContext"
Expand Down Expand Up @@ -1950,11 +1955,6 @@
"Undefined function <name>."
]
},
"_LEGACY_ERROR_TEMP_1042" : {
"message" : [
"Invalid number of arguments for function <name>. Expected: <expectedInfo>; Found: <actualNumber>."
]
},
"_LEGACY_ERROR_TEMP_1043" : {
"message" : [
"Invalid arguments for function <name>."
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -637,13 +637,13 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase {
}

def invalidFunctionArgumentsError(
name: String, expectedInfo: String, actualNumber: Int): Throwable = {
name: String, expectedNum: String, actualNum: Int): Throwable = {
new AnalysisException(
errorClass = "_LEGACY_ERROR_TEMP_1042",
errorClass = "WRONG_NUM_ARGS",
messageParameters = Map(
"name" -> name,
"expectedInfo" -> expectedInfo,
"actualNumber" -> actualNumber.toString))
"functionName" -> toSQLId(name),
"expectedNum" -> expectedNum,
"actualNum" -> actualNum.toString))
}

def invalidFunctionArgumentNumberError(
Expand All @@ -656,8 +656,7 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase {
val expectedNumberOfParameters = if (validParametersCount.length == 1) {
validParametersCount.head.toString
} else {
validParametersCount.init.mkString("one of ", ", ", " and ") +
validParametersCount.last
validParametersCount.mkString("[", ", ", "]")
}
invalidFunctionArgumentsError(name, expectedNumberOfParameters, actualNumber)
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -821,11 +821,11 @@ struct<>
-- !query output
org.apache.spark.sql.AnalysisException
{
"errorClass" : "_LEGACY_ERROR_TEMP_1042",
"errorClass" : "WRONG_NUM_ARGS",
"messageParameters" : {
"actualNumber" : "0",
"expectedInfo" : "2",
"name" : "decode"
"actualNum" : "0",
"expectedNum" : "2",
"functionName" : "`decode`"
},
"queryContext" : [ {
"objectType" : "",
Expand All @@ -844,11 +844,11 @@ struct<>
-- !query output
org.apache.spark.sql.AnalysisException
{
"errorClass" : "_LEGACY_ERROR_TEMP_1042",
"errorClass" : "WRONG_NUM_ARGS",
"messageParameters" : {
"actualNumber" : "1",
"expectedInfo" : "2",
"name" : "decode"
"actualNum" : "1",
"expectedNum" : "2",
"functionName" : "`decode`"
},
"queryContext" : [ {
"objectType" : "",
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -140,11 +140,11 @@ struct<>
-- !query output
org.apache.spark.sql.AnalysisException
{
"errorClass" : "_LEGACY_ERROR_TEMP_1042",
"errorClass" : "WRONG_NUM_ARGS",
"messageParameters" : {
"actualNumber" : "3",
"expectedInfo" : "2",
"name" : "ceil"
"actualNum" : "3",
"expectedNum" : "2",
"functionName" : "`ceil`"
},
"queryContext" : [ {
"objectType" : "",
Expand Down Expand Up @@ -297,11 +297,11 @@ struct<>
-- !query output
org.apache.spark.sql.AnalysisException
{
"errorClass" : "_LEGACY_ERROR_TEMP_1042",
"errorClass" : "WRONG_NUM_ARGS",
"messageParameters" : {
"actualNumber" : "3",
"expectedInfo" : "2",
"name" : "floor"
"actualNum" : "3",
"expectedNum" : "2",
"functionName" : "`floor`"
},
"queryContext" : [ {
"objectType" : "",
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -105,11 +105,11 @@ struct<>
-- !query output
org.apache.spark.sql.AnalysisException
{
"errorClass" : "_LEGACY_ERROR_TEMP_1042",
"errorClass" : "WRONG_NUM_ARGS",
"messageParameters" : {
"actualNumber" : "0",
"expectedInfo" : "one of 2 and 3",
"name" : "from_csv"
"actualNum" : "0",
"expectedNum" : "[2, 3]",
"functionName" : "`from_csv`"
},
"queryContext" : [ {
"objectType" : "",
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -109,11 +109,11 @@ struct<>
-- !query output
org.apache.spark.sql.AnalysisException
{
"errorClass" : "_LEGACY_ERROR_TEMP_1042",
"errorClass" : "WRONG_NUM_ARGS",
"messageParameters" : {
"actualNumber" : "0",
"expectedInfo" : "one of 1 and 2",
"name" : "to_json"
"actualNum" : "0",
"expectedNum" : "[1, 2]",
"functionName" : "`to_json`"
},
"queryContext" : [ {
"objectType" : "",
Expand Down Expand Up @@ -231,11 +231,11 @@ struct<>
-- !query output
org.apache.spark.sql.AnalysisException
{
"errorClass" : "_LEGACY_ERROR_TEMP_1042",
"errorClass" : "WRONG_NUM_ARGS",
"messageParameters" : {
"actualNumber" : "0",
"expectedInfo" : "one of 2 and 3",
"name" : "from_json"
"actualNum" : "0",
"expectedNum" : "[2, 3]",
"functionName" : "`from_json`"
},
"queryContext" : [ {
"objectType" : "",
Expand Down Expand Up @@ -572,11 +572,11 @@ struct<>
-- !query output
org.apache.spark.sql.AnalysisException
{
"errorClass" : "_LEGACY_ERROR_TEMP_1042",
"errorClass" : "WRONG_NUM_ARGS",
"messageParameters" : {
"actualNumber" : "0",
"expectedInfo" : "1",
"name" : "json_array_length"
"actualNum" : "0",
"expectedNum" : "1",
"functionName" : "`json_array_length`"
},
"queryContext" : [ {
"objectType" : "",
Expand Down Expand Up @@ -659,11 +659,11 @@ struct<>
-- !query output
org.apache.spark.sql.AnalysisException
{
"errorClass" : "_LEGACY_ERROR_TEMP_1042",
"errorClass" : "WRONG_NUM_ARGS",
"messageParameters" : {
"actualNumber" : "0",
"expectedInfo" : "1",
"name" : "json_object_keys"
"actualNum" : "0",
"expectedNum" : "1",
"functionName" : "`json_object_keys`"
},
"queryContext" : [ {
"objectType" : "",
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -753,11 +753,11 @@ struct<>
-- !query output
org.apache.spark.sql.AnalysisException
{
"errorClass" : "_LEGACY_ERROR_TEMP_1042",
"errorClass" : "WRONG_NUM_ARGS",
"messageParameters" : {
"actualNumber" : "0",
"expectedInfo" : "2",
"name" : "decode"
"actualNum" : "0",
"expectedNum" : "2",
"functionName" : "`decode`"
},
"queryContext" : [ {
"objectType" : "",
Expand All @@ -776,11 +776,11 @@ struct<>
-- !query output
org.apache.spark.sql.AnalysisException
{
"errorClass" : "_LEGACY_ERROR_TEMP_1042",
"errorClass" : "WRONG_NUM_ARGS",
"messageParameters" : {
"actualNumber" : "1",
"expectedInfo" : "2",
"name" : "decode"
"actualNum" : "1",
"expectedNum" : "2",
"functionName" : "`decode`"
},
"queryContext" : [ {
"objectType" : "",
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -83,7 +83,7 @@ org.apache.spark.sql.AnalysisException
"errorClass" : "_LEGACY_ERROR_TEMP_1179",
"messageParameters" : {
"arguments" : "integer, integer, integer, integer, integer",
"details" : "Invalid number of arguments for function range. Expected: one of 1, 2, 3 and 4; Found: 5.",
"details" : "[WRONG_NUM_ARGS] The `range` requires [1, 2, 3, 4] parameters but the actual number is 5.",
"name" : "range",
"usage" : "\n range(start: long, end: long, step: long, numSlices: integer)\n range(start: long, end: long, step: long)\n range(start: long, end: long)\n range(end: long)"
},
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -46,11 +46,11 @@ struct<>
-- !query output
org.apache.spark.sql.AnalysisException
{
"errorClass" : "_LEGACY_ERROR_TEMP_1042",
"errorClass" : "WRONG_NUM_ARGS",
"messageParameters" : {
"actualNumber" : "7",
"expectedInfo" : "6",
"name" : "make_timestamp_ntz"
"actualNum" : "7",
"expectedNum" : "6",
"functionName" : "`make_timestamp_ntz`"
},
"queryContext" : [ {
"objectType" : "",
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -32,11 +32,11 @@ struct<>
-- !query output
org.apache.spark.sql.AnalysisException
{
"errorClass" : "_LEGACY_ERROR_TEMP_1042",
"errorClass" : "WRONG_NUM_ARGS",
"messageParameters" : {
"actualNumber" : "2",
"expectedInfo" : "1",
"name" : "spark_catalog.default.mydoubleavg"
"actualNum" : "2",
"expectedNum" : "1",
"functionName" : "`spark_catalog`.`default`.`mydoubleavg`"
},
"queryContext" : [ {
"objectType" : "",
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -32,11 +32,11 @@ struct<>
-- !query output
org.apache.spark.sql.AnalysisException
{
"errorClass" : "_LEGACY_ERROR_TEMP_1042",
"errorClass" : "WRONG_NUM_ARGS",
"messageParameters" : {
"actualNumber" : "2",
"expectedInfo" : "1",
"name" : "spark_catalog.default.mydoubleavg"
"actualNum" : "2",
"expectedNum" : "1",
"functionName" : "`spark_catalog`.`default`.`mydoubleavg`"
},
"queryContext" : [ {
"objectType" : "",
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -28,6 +28,7 @@ import org.apache.spark.SparkException
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.analysis.{FunctionRegistry, UnresolvedAttribute}
import org.apache.spark.sql.catalyst.expressions.{Alias, ArraysZip, AttributeReference, Expression, NamedExpression, UnaryExpression}
import org.apache.spark.sql.catalyst.expressions.Cast._
import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback
import org.apache.spark.sql.catalyst.plans.logical.OneRowRelation
import org.apache.spark.sql.catalyst.util.DateTimeTestUtils.{withDefaultTimeZone, UTC}
Expand Down Expand Up @@ -4847,14 +4848,34 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession {
(Seq.empty, Seq("x", "z"), 3),
(null, Seq("x", "z"), 4)
).toDF("a1", "a2", "i")
val ex1 = intercept[AnalysisException] {
df.selectExpr("zip_with(a1, a2, x -> x)")
}
assert(ex1.getMessage.contains("The number of lambda function arguments '1' does not match"))
val ex2 = intercept[AnalysisException] {
df.selectExpr("zip_with(a1, a2, (acc, x) -> x, (acc, x) -> x)")
}
assert(ex2.getMessage.contains("Invalid number of arguments for function zip_with"))
checkError(
exception = intercept[AnalysisException] {
df.selectExpr("zip_with(a1, a2, x -> x)")
},
errorClass = "_LEGACY_ERROR_TEMP_2300",
parameters = Map(
"namesSize" -> "1",
"argInfoSize" -> "2"),
context = ExpectedContext(
fragment = "x -> x",
start = 17,
stop = 22)
)

checkError(
exception = intercept[AnalysisException] {
df.selectExpr("zip_with(a1, a2, (acc, x) -> x, (acc, x) -> x)")
},
errorClass = "WRONG_NUM_ARGS",
parameters = Map(
"functionName" -> toSQLId("zip_with"),
"expectedNum" -> "3",
"actualNum" -> "4"),
context = ExpectedContext(
fragment = "zip_with(a1, a2, (acc, x) -> x, (acc, x) -> x)",
start = 0,
stop = 45)
)

checkError(
exception = intercept[AnalysisException] {
Expand Down
18 changes: 16 additions & 2 deletions sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala
Original file line number Diff line number Diff line change
Expand Up @@ -2630,8 +2630,22 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark
}

test("RuntimeReplaceable functions should not take extra parameters") {
val e = intercept[AnalysisException](sql("SELECT nvl(1, 2, 3)"))
assert(e.message.contains("Invalid number of arguments"))
checkError(
exception = intercept[AnalysisException] {
sql("SELECT nvl(1, 2, 3)")
},
errorClass = "WRONG_NUM_ARGS",
parameters = Map(
"functionName" -> toSQLId("nvl"),
"expectedNum" -> "2",
"actualNum" -> "3"
),
context = ExpectedContext(
start = 7,
stop = 18,
fragment = "nvl(1, 2, 3)"
)
)
}

test("SPARK-21228: InSet incorrect handling of structs") {
Expand Down
Loading