From bcd53d292b4a3eaa091fe984b058bc769d3533c0 Mon Sep 17 00:00:00 2001 From: lvshaokang Date: Fri, 22 Apr 2022 10:18:45 +0800 Subject: [PATCH 01/13] [SPARK-38692][SQL] Use error classes in the compilation errors of function args --- .../main/resources/error/error-classes.json | 4 ++ .../sql/errors/QueryCompilationErrors.scala | 22 ++++++++--- .../errors/QueryCompilationErrorsSuite.scala | 39 +++++++++++++++++++ 3 files changed, 59 insertions(+), 6 deletions(-) diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json index 6d070c35c74cb..6f263534cbc55 100644 --- a/core/src/main/resources/error/error-classes.json +++ b/core/src/main/resources/error/error-classes.json @@ -88,6 +88,10 @@ "message" : [ "Index %s must be between 0 and the length of the ArrayData." ], "sqlState" : "22023" }, + "INVALID_FUNCTION_ARGUMENTS": { + "message": [ "Invalid number of arguments for function %s. %s" ], + "sqlState" : "22023" + }, "INTERNAL_ERROR" : { "message" : [ "%s" ] }, diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala index dac6a6e731f39..67bd2e3fc3965 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala @@ -427,14 +427,17 @@ object QueryCompilationErrors extends QueryErrorsBase { def invalidFunctionArgumentsError( name: String, expectedInfo: String, actualNumber: Int): Throwable = { - new AnalysisException(s"Invalid number of arguments for function $name. " + - s"Expected: $expectedInfo; Found: $actualNumber") + new AnalysisException( + errorClass = "INVALID_FUNCTION_ARGUMENTS", + messageParameters = Array(name, s"Expected: $expectedInfo; Found: $actualNumber")) } def invalidFunctionArgumentNumberError( validParametersCount: Seq[Int], name: String, actualNumber: Int): Throwable = { if (validParametersCount.length == 0) { - new AnalysisException(s"Invalid arguments for function $name") + new AnalysisException( + errorClass = "INVALID_FUNCTION_ARGUMENTS", + messageParameters = Array(name, "")) } else { val expectedNumberOfParameters = if (validParametersCount.length == 1) { validParametersCount.head.toString @@ -447,7 +450,9 @@ object QueryCompilationErrors extends QueryErrorsBase { } def functionAcceptsOnlyOneArgumentError(name: String): Throwable = { - new AnalysisException(s"Function $name accepts only one argument") + new AnalysisException( + errorClass = "INVALID_FUNCTION_ARGUMENTS", + messageParameters = Array(name, s"It accepts only one argument")) } def alterV2TableSetLocationWithPartitionNotSupportedError(): Throwable = { @@ -751,7 +756,11 @@ object QueryCompilationErrors extends QueryErrorsBase { } def secondArgumentNotDoubleLiteralError(): Throwable = { - new AnalysisException("The second argument should be a double literal.") + new AnalysisException( + errorClass = "INVALID_FUNCTION_ARGUMENTS", + messageParameters = Array("approx_count_distinct", + "The second argument should be a double literal.") + ) } def dataTypeUnsupportedByExtractValueError( @@ -1464,7 +1473,8 @@ object QueryCompilationErrors extends QueryErrorsBase { unbound: UnboundFunction, arguments: Seq[Expression], unsupported: UnsupportedOperationException): Throwable = { - new AnalysisException(s"Function '${unbound.name}' cannot process " + + new AnalysisException( + s"Function '${unbound.name}' cannot process " + s"input: (${arguments.map(_.dataType.simpleString).mkString(", ")}): " + unsupported.getMessage, cause = Some(unsupported)) } 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 e4d3d589b5e34..75a8ec049d6cc 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 @@ -364,6 +364,45 @@ class QueryCompilationErrorsSuite extends QueryTest with SharedSparkSession { } } } + + test("INVALID_FUNCTION_ARGUMENTS: invalid function arguments") { + spark.udf.register("testFunc", (n: Int) => n.toString) + val e = intercept[AnalysisException]( + sql(s"SELECT testFunc(123, 123) as value") + ) + assert(e.getErrorClass === "INVALID_FUNCTION_ARGUMENTS") + assert(e.getSqlState === "22023") + assert(e.getMessage === "Invalid number of arguments for function testFunc. " + + "Expected: 1; Found: 2; line 1 pos 7") + } + + test("INVALID_FUNCTION_ARGUMENTS: invalid function arguments number") { + val e = intercept[AnalysisException]( + sql(s"SELECT to_timestamp_ntz()") + ) + assert(e.getErrorClass === "INVALID_FUNCTION_ARGUMENTS") + assert(e.getSqlState === "22023") + assert(e.getMessage === "Invalid number of arguments for function to_timestamp_ntz. " + + "Expected: one of 1 and 2; Found: 0; line 1 pos 7") + } + + test("INVALID_FUNCTION_ARGUMENTS: only accept one argument") { + val e = intercept[AnalysisException]( + sql(s"SELECT int('1', '2')") + ) + assert(e.getErrorClass === "INVALID_FUNCTION_ARGUMENTS") + assert(e.getSqlState === "22023") + assert(e.getMessage === "Invalid number of arguments for function int. " + + "It accepts only one argument; line 1 pos 7") + } + + test("INVALID_FUNCTION_ARGUMENTS: only one argument") { + val e = intercept[AnalysisException]( + sql(s"SELECT approx_count_distinct(1,1) FROM VALUES (1), (1), (2), (2)").show(10) + ) + assert(e.getErrorClass === "INVALID_FUNCTION_ARGUMENTS") + assert(e.getSqlState === "22023") + } } class MyCastToString extends SparkUserDefinedFunction( From 9602765d01e3d00ab68d08d58248b568f0786250 Mon Sep 17 00:00:00 2001 From: lvshaokang Date: Sun, 24 Apr 2022 23:46:17 +0800 Subject: [PATCH 02/13] [SPARK-38692][SQL] Use error classes in the compilation errors of function args --- .../main/resources/error/error-classes.json | 2 +- .../catalyst/analysis/FunctionRegistry.scala | 3 + .../ApproxCountDistinctForIntervals.scala | 3 +- .../aggregate/HyperLogLogPlusPlus.scala | 6 +- .../sql/errors/QueryCompilationErrors.scala | 34 +++++--- .../errors/QueryCompilationErrorsSuite.scala | 78 +++++++++++++++++-- 6 files changed, 102 insertions(+), 24 deletions(-) diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json index 6f263534cbc55..e978214a11d33 100644 --- a/core/src/main/resources/error/error-classes.json +++ b/core/src/main/resources/error/error-classes.json @@ -89,7 +89,7 @@ "sqlState" : "22023" }, "INVALID_FUNCTION_ARGUMENTS": { - "message": [ "Invalid number of arguments for function %s. %s" ], + "message": [ "Invalid %s of arguments for function %s. %s" ], "sqlState" : "22023" }, "INTERNAL_ERROR" : { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala index f7af5b35a3b97..6768b2c0f0b01 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala @@ -153,6 +153,9 @@ object FunctionRegistryBase { } catch { // the exception is an invocation exception. To get a meaningful message, we need the // cause. + case e: Exception if e.getCause.isInstanceOf[AnalysisException] => + if (e.getCause.asInstanceOf[AnalysisException].errorClass.isDefined) throw e.getCause + else throw new AnalysisException(e.getCause.getMessage) case e: Exception => throw new AnalysisException(e.getCause.getMessage) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/ApproxCountDistinctForIntervals.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/ApproxCountDistinctForIntervals.scala index f3bf251ba0b5a..e110a2795369d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/ApproxCountDistinctForIntervals.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/ApproxCountDistinctForIntervals.scala @@ -55,7 +55,8 @@ case class ApproxCountDistinctForIntervals( this( child = child, endpointsExpression = endpointsExpression, - relativeSD = HyperLogLogPlusPlus.validateDoubleLiteral(relativeSD), + relativeSD = HyperLogLogPlusPlus + .validateDoubleLiteral("ApproxCountDistinctForIntervals", relativeSD), mutableAggBufferOffset = 0, inputAggBufferOffset = 0) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/HyperLogLogPlusPlus.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/HyperLogLogPlusPlus.scala index 9b0493f3e68a4..e5ddba40c66ec 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/HyperLogLogPlusPlus.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/HyperLogLogPlusPlus.scala @@ -70,7 +70,7 @@ case class HyperLogLogPlusPlus( def this(child: Expression, relativeSD: Expression) = { this( child = child, - relativeSD = HyperLogLogPlusPlus.validateDoubleLiteral(relativeSD), + relativeSD = HyperLogLogPlusPlus.validateDoubleLiteral("approx_count_distinct", relativeSD), mutableAggBufferOffset = 0, inputAggBufferOffset = 0) } @@ -144,10 +144,10 @@ case class HyperLogLogPlusPlus( } object HyperLogLogPlusPlus { - def validateDoubleLiteral(exp: Expression): Double = exp match { + def validateDoubleLiteral(name: String, exp: Expression): Double = exp match { case Literal(d: Double, DoubleType) => d case Literal(dec: Decimal, _) => dec.toDouble case _ => - throw QueryCompilationErrors.secondArgumentNotDoubleLiteralError + throw QueryCompilationErrors.secondArgumentNotDoubleLiteralError(name) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala index 67bd2e3fc3965..4061f364c4deb 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala @@ -429,7 +429,7 @@ object QueryCompilationErrors extends QueryErrorsBase { name: String, expectedInfo: String, actualNumber: Int): Throwable = { new AnalysisException( errorClass = "INVALID_FUNCTION_ARGUMENTS", - messageParameters = Array(name, s"Expected: $expectedInfo; Found: $actualNumber")) + messageParameters = Array("number", name, s"Expected: $expectedInfo; Found: $actualNumber")) } def invalidFunctionArgumentNumberError( @@ -437,7 +437,7 @@ object QueryCompilationErrors extends QueryErrorsBase { if (validParametersCount.length == 0) { new AnalysisException( errorClass = "INVALID_FUNCTION_ARGUMENTS", - messageParameters = Array(name, "")) + messageParameters = Array("number", name, "")) } else { val expectedNumberOfParameters = if (validParametersCount.length == 1) { validParametersCount.head.toString @@ -452,7 +452,7 @@ object QueryCompilationErrors extends QueryErrorsBase { def functionAcceptsOnlyOneArgumentError(name: String): Throwable = { new AnalysisException( errorClass = "INVALID_FUNCTION_ARGUMENTS", - messageParameters = Array(name, s"It accepts only one argument")) + messageParameters = Array("number", name, s"It accepts only one argument")) } def alterV2TableSetLocationWithPartitionNotSupportedError(): Throwable = { @@ -755,11 +755,10 @@ object QueryCompilationErrors extends QueryErrorsBase { new AnalysisException(s"Unsupported component type $clz in arrays") } - def secondArgumentNotDoubleLiteralError(): Throwable = { + def secondArgumentNotDoubleLiteralError(name: String): Throwable = { new AnalysisException( errorClass = "INVALID_FUNCTION_ARGUMENTS", - messageParameters = Array("approx_count_distinct", - "The second argument should be a double literal.") + messageParameters = Array("type", name, "The second argument should be a double literal") ) } @@ -1474,16 +1473,23 @@ object QueryCompilationErrors extends QueryErrorsBase { arguments: Seq[Expression], unsupported: UnsupportedOperationException): Throwable = { new AnalysisException( - s"Function '${unbound.name}' cannot process " + - s"input: (${arguments.map(_.dataType.simpleString).mkString(", ")}): " + - unsupported.getMessage, cause = Some(unsupported)) + errorClass = "INVALID_FUNCTION_ARGUMENTS", + messageParameters = Array("operation", unbound.name, + s"It cannot process " + + s"input: (${arguments.map(_.dataType.simpleString).mkString(", ")}): " + + unsupported.getMessage), + cause = Some(unsupported)) } def v2FunctionInvalidInputTypeLengthError( bound: BoundFunction, args: Seq[Expression]): Throwable = { - new AnalysisException(s"Invalid bound function '${bound.name()}: there are ${args.length} " + - s"arguments but ${bound.inputTypes().length} parameters returned from 'inputTypes()'") + new AnalysisException( + errorClass = "INVALID_FUNCTION_ARGUMENTS", + messageParameters = Array("number", bound.name(), + s"There are ${args.length} arguments " + + s"but ${bound.inputTypes().length} parameters returned from 'inputTypes()'") + ) } def ambiguousRelationAliasNameInNestedCTEError(name: String): Throwable = { @@ -1563,7 +1569,11 @@ object QueryCompilationErrors extends QueryErrorsBase { } def secondArgumentInFunctionIsNotBooleanLiteralError(funcName: String): Throwable = { - new AnalysisException(s"The second argument in $funcName should be a boolean literal.") + new AnalysisException( + errorClass = "INVALID_FUNCTION_ARGUMENTS", + messageParameters = Array("type", funcName, + "The second argument should be a boolean literal") + ) } def joinConditionMissingOrTrivialError( 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 75a8ec049d6cc..433292363a86a 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 @@ -17,13 +17,15 @@ package org.apache.spark.sql.errors -import org.apache.spark.sql.{AnalysisException, IntegratedUDFTestUtils, QueryTest, Row} +import org.apache.spark.sql.{AnalysisException, IntegratedUDFTestUtils, Row} import org.apache.spark.sql.api.java.{UDF1, UDF2, UDF23Test} +import org.apache.spark.sql.connector.DatasourceV2SQLBase +import org.apache.spark.sql.connector.catalog.{Identifier, InMemoryCatalog} +import org.apache.spark.sql.connector.catalog.functions.{BoundFunction, ScalarFunction, UnboundFunction} import org.apache.spark.sql.expressions.SparkUserDefinedFunction import org.apache.spark.sql.functions.{grouping, grouping_id, sum, udf} import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.test.SharedSparkSession -import org.apache.spark.sql.types.{IntegerType, StringType} +import org.apache.spark.sql.types.{DataType, DataTypes, IntegerType, StringType, StructType} case class StringLongClass(a: String, b: Long) @@ -31,7 +33,7 @@ case class StringIntClass(a: String, b: Int) case class ComplexClass(a: Long, b: StringLongClass) -class QueryCompilationErrorsSuite extends QueryTest with SharedSparkSession { +class QueryCompilationErrorsSuite extends DatasourceV2SQLBase { import testImplicits._ test("CANNOT_UP_CAST_DATATYPE: invalid upcast data type") { @@ -365,7 +367,7 @@ class QueryCompilationErrorsSuite extends QueryTest with SharedSparkSession { } } - test("INVALID_FUNCTION_ARGUMENTS: invalid function arguments") { + test("INVALID_FUNCTION_ARGUMENTS: invalid number of the function arguments") { spark.udf.register("testFunc", (n: Int) => n.toString) val e = intercept[AnalysisException]( sql(s"SELECT testFunc(123, 123) as value") @@ -396,12 +398,53 @@ class QueryCompilationErrorsSuite extends QueryTest with SharedSparkSession { "It accepts only one argument; line 1 pos 7") } - test("INVALID_FUNCTION_ARGUMENTS: only one argument") { + test("INVALID_FUNCTION_ARGUMENTS: invalid the second argument of the int type") { val e = intercept[AnalysisException]( - sql(s"SELECT approx_count_distinct(1,1) FROM VALUES (1), (1), (2), (2)").show(10) + sql(s"SELECT approx_count_distinct(1,1)") ) assert(e.getErrorClass === "INVALID_FUNCTION_ARGUMENTS") assert(e.getSqlState === "22023") + assert(e.getMessage === "Invalid type of arguments for function approx_count_distinct. " + + "The second argument should be a double literal; line 1 pos 7") + } + + test("INVALID_FUNCTION_ARGUMENTS: cannot process function input of the map type") { + withSQLConf(SQLConf.DEFAULT_CATALOG.key -> "testcat") { + catalog("testcat").asInstanceOf[InMemoryCatalog] + .createFunction(Identifier.of(Array("ns"), "strlen"), new StrLen) + val e = intercept[AnalysisException]( + sql("SELECT testcat.ns.strlen(map('abc', 'abc'))") + ) + assert(e.getErrorClass === "INVALID_FUNCTION_ARGUMENTS") + assert(e.getSqlState === "22023") + assert(e.getMessage === "Invalid operation of arguments for function strlen. " + + "It cannot process input: (map): Expect StringType, " + + "but found MapType(StringType,StringType,false); line 1 pos 7") + } + } + + test("INVALID_FUNCTION_ARGUMENTS: invalid input type length of the v2Function") { + withSQLConf(SQLConf.DEFAULT_CATALOG.key -> "testcat") { + catalog("testcat").asInstanceOf[InMemoryCatalog] + .createFunction(Identifier.of(Array("ns"), "strlen"), new StrLen) + val e = intercept[AnalysisException]( + sql("SELECT testcat.ns.strlen('abc','abc')") + ) + assert(e.getErrorClass === "INVALID_FUNCTION_ARGUMENTS") + assert(e.getSqlState === "22023") + assert(e.getMessage === "Invalid number of arguments for function strlen. " + + "There are 2 arguments but 1 parameters returned from 'inputTypes()'; line 1 pos 7") + } + } + + test("INVALID_FUNCTION_ARGUMENTS: invalid the second argument of the string type") { + val e = intercept[AnalysisException]( + sql(s"SELECT first(1, '1')") + ) + assert(e.getErrorClass === "INVALID_FUNCTION_ARGUMENTS") + assert(e.getSqlState === "22023") + assert(e.getMessage === "Invalid type of arguments for function first. " + + "The second argument should be a boolean literal; line 1 pos 7") } } @@ -433,3 +476,24 @@ class MultiIntSum extends } // scalastyle:on argcount } + +class StrLen extends UnboundFunction { + override def bind(inputType: StructType): BoundFunction = { + for (field <- inputType.fields) { + if (!field.dataType.isInstanceOf[StringType]) { + throw new UnsupportedOperationException( + "Expect StringType, but found " + field.dataType) + } + } + new ScalarFunction[Int] { + override def inputTypes(): Array[DataType] = Array[DataType](StringType) + override def resultType(): DataType = DataTypes.IntegerType + override def name(): String = "strlen" + def invoke(str: String): Int = str.length + } + } + + override def description(): String = "strlen(string) -> int" + + override def name(): String = "strlen" +} From bea3fe5868780def3637219ed9922262d0379c04 Mon Sep 17 00:00:00 2001 From: lvshaokang Date: Mon, 25 Apr 2022 00:58:28 +0800 Subject: [PATCH 03/13] [SPARK-38692][SQL] Use error classes in the compilation errors of function args --- .../sql/errors/QueryCompilationErrors.scala | 5 +- .../errors/QueryCompilationErrorsSuite.scala | 111 ++++++++++-------- .../sql/errors/QueryErrorsSuiteBase.scala | 13 +- 3 files changed, 78 insertions(+), 51 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala index 848693d306770..7126a2613bd03 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala @@ -773,7 +773,8 @@ object QueryCompilationErrors extends QueryErrorsBase { def secondArgumentNotDoubleLiteralError(name: String): Throwable = { new AnalysisException( errorClass = "INVALID_FUNCTION_ARGUMENTS", - messageParameters = Array("type", name, "The second argument should be a double literal") + messageParameters = Array("type", name, + "The second argument should be a double literal") ) } @@ -1491,7 +1492,7 @@ object QueryCompilationErrors extends QueryErrorsBase { errorClass = "INVALID_FUNCTION_ARGUMENTS", messageParameters = Array("operation", unbound.name, s"It cannot process " + - s"input: (${arguments.map(_.dataType.simpleString).mkString(", ")}): " + + s"input: (${arguments.map(x => toSQLType(x.dataType)).mkString(", ")}): " + unsupported.getMessage), cause = Some(unsupported)) } 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 a1a707b6d6545..05908f2695d67 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 @@ -17,9 +17,8 @@ package org.apache.spark.sql.errors -import org.apache.spark.sql.{AnalysisException, IntegratedUDFTestUtils, Row} +import org.apache.spark.sql.{AnalysisException, IntegratedUDFTestUtils, QueryTest, Row} import org.apache.spark.sql.api.java.{UDF1, UDF2, UDF23Test} -import org.apache.spark.sql.connector.DatasourceV2SQLBase import org.apache.spark.sql.connector.catalog.{Identifier, InMemoryCatalog} import org.apache.spark.sql.connector.catalog.functions.{BoundFunction, ScalarFunction, UnboundFunction} import org.apache.spark.sql.expressions.SparkUserDefinedFunction @@ -34,7 +33,7 @@ case class StringIntClass(a: String, b: Int) case class ComplexClass(a: Long, b: StringLongClass) class QueryCompilationErrorsSuite - extends DatasourceV2SQLBase + extends QueryTest with QueryErrorsSuiteBase { import testImplicits._ @@ -415,82 +414,98 @@ class QueryCompilationErrorsSuite test("INVALID_FUNCTION_ARGUMENTS: invalid number of the function arguments") { spark.udf.register("testFunc", (n: Int) => n.toString) - val e = intercept[AnalysisException]( - sql(s"SELECT testFunc(123, 123) as value") + checkErrorClass( + exception = intercept[AnalysisException]( + sql(s"SELECT testFunc(123, 123) as value") + ), + errorClass = "INVALID_FUNCTION_ARGUMENTS", + msg = "Invalid number of arguments for function testFunc. " + + "Expected: 1; Found: 2; line 1 pos 7", + sqlState = Some("22023") ) - assert(e.getErrorClass === "INVALID_FUNCTION_ARGUMENTS") - assert(e.getSqlState === "22023") - assert(e.getMessage === "Invalid number of arguments for function testFunc. " + - "Expected: 1; Found: 2; line 1 pos 7") } test("INVALID_FUNCTION_ARGUMENTS: invalid function arguments number") { - val e = intercept[AnalysisException]( - sql(s"SELECT to_timestamp_ntz()") + checkErrorClass( + exception = intercept[AnalysisException]( + sql(s"SELECT to_timestamp_ntz()") + ), + errorClass = "INVALID_FUNCTION_ARGUMENTS", + msg = "Invalid number of arguments for function to_timestamp_ntz. " + + "Expected: one of 1 and 2; Found: 0; line 1 pos 7", + sqlState = Some("22023") ) - assert(e.getErrorClass === "INVALID_FUNCTION_ARGUMENTS") - assert(e.getSqlState === "22023") - assert(e.getMessage === "Invalid number of arguments for function to_timestamp_ntz. " + - "Expected: one of 1 and 2; Found: 0; line 1 pos 7") } test("INVALID_FUNCTION_ARGUMENTS: only accept one argument") { - val e = intercept[AnalysisException]( - sql(s"SELECT int('1', '2')") + checkErrorClass( + exception = intercept[AnalysisException]( + sql(s"SELECT int('1', '2')") + ), + errorClass = "INVALID_FUNCTION_ARGUMENTS", + msg = "Invalid number of arguments for function int. " + + "It accepts only one argument; line 1 pos 7", + sqlState = Some("22023") ) - assert(e.getErrorClass === "INVALID_FUNCTION_ARGUMENTS") - assert(e.getSqlState === "22023") - assert(e.getMessage === "Invalid number of arguments for function int. " + - "It accepts only one argument; line 1 pos 7") } test("INVALID_FUNCTION_ARGUMENTS: invalid the second argument of the int type") { - val e = intercept[AnalysisException]( - sql(s"SELECT approx_count_distinct(1,1)") + checkErrorClass( + exception = intercept[AnalysisException]( + sql(s"SELECT approx_count_distinct(1,1)") + ), + errorClass = "INVALID_FUNCTION_ARGUMENTS", + msg = "Invalid type of arguments for function approx_count_distinct. " + + "The second argument should be a double literal; line 1 pos 7", + sqlState = Some("22023") ) - assert(e.getErrorClass === "INVALID_FUNCTION_ARGUMENTS") - assert(e.getSqlState === "22023") - assert(e.getMessage === "Invalid type of arguments for function approx_count_distinct. " + - "The second argument should be a double literal; line 1 pos 7") } test("INVALID_FUNCTION_ARGUMENTS: cannot process function input of the map type") { withSQLConf(SQLConf.DEFAULT_CATALOG.key -> "testcat") { - catalog("testcat").asInstanceOf[InMemoryCatalog] + spark.sessionState.catalogManager.catalog("testcat") + .asInstanceOf[InMemoryCatalog] .createFunction(Identifier.of(Array("ns"), "strlen"), new StrLen) - val e = intercept[AnalysisException]( - sql("SELECT testcat.ns.strlen(map('abc', 'abc'))") + checkErrorClass( + exception = intercept[AnalysisException]( + sql("SELECT testcat.ns.strlen(map('abc', 'abc'))") + ), + errorClass = "INVALID_FUNCTION_ARGUMENTS", + msg = "Invalid operation of arguments for function strlen. " + + "It cannot process input: (\"MAP\"): Expect StringType, " + + "but found MapType(StringType,StringType,false); line 1 pos 7", + sqlState = Some("22023") ) - assert(e.getErrorClass === "INVALID_FUNCTION_ARGUMENTS") - assert(e.getSqlState === "22023") - assert(e.getMessage === "Invalid operation of arguments for function strlen. " + - "It cannot process input: (map): Expect StringType, " + - "but found MapType(StringType,StringType,false); line 1 pos 7") } } test("INVALID_FUNCTION_ARGUMENTS: invalid input type length of the v2Function") { withSQLConf(SQLConf.DEFAULT_CATALOG.key -> "testcat") { - catalog("testcat").asInstanceOf[InMemoryCatalog] + spark.sessionState.catalogManager.catalog("testcat") + .asInstanceOf[InMemoryCatalog] .createFunction(Identifier.of(Array("ns"), "strlen"), new StrLen) - val e = intercept[AnalysisException]( - sql("SELECT testcat.ns.strlen('abc','abc')") + checkErrorClass( + exception = intercept[AnalysisException]( + sql("SELECT testcat.ns.strlen('abc','abc')") + ), + errorClass = "INVALID_FUNCTION_ARGUMENTS", + msg = "Invalid number of arguments for function strlen. " + + "There are 2 arguments but 1 parameters returned from 'inputTypes()'; line 1 pos 7", + sqlState = Some("22023") ) - assert(e.getErrorClass === "INVALID_FUNCTION_ARGUMENTS") - assert(e.getSqlState === "22023") - assert(e.getMessage === "Invalid number of arguments for function strlen. " + - "There are 2 arguments but 1 parameters returned from 'inputTypes()'; line 1 pos 7") } } test("INVALID_FUNCTION_ARGUMENTS: invalid the second argument of the string type") { - val e = intercept[AnalysisException]( - sql(s"SELECT first(1, '1')") + checkErrorClass( + exception = intercept[AnalysisException]( + sql(s"SELECT first(1, '1')") + ), + errorClass = "INVALID_FUNCTION_ARGUMENTS", + msg = "Invalid type of arguments for function first. " + + "The second argument should be a boolean literal; line 1 pos 7", + sqlState = Some("22023") ) - assert(e.getErrorClass === "INVALID_FUNCTION_ARGUMENTS") - assert(e.getSqlState === "22023") - assert(e.getMessage === "Invalid type of arguments for function first. " + - "The second argument should be a boolean literal; line 1 pos 7") } } 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 2c2fe188e245a..9a355795284fa 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,11 +17,14 @@ package org.apache.spark.sql.errors +import org.scalatest.BeforeAndAfter + import org.apache.spark.SparkThrowable import org.apache.spark.sql.catalyst.parser.ParseException +import org.apache.spark.sql.connector.catalog.InMemoryCatalog import org.apache.spark.sql.test.SharedSparkSession -trait QueryErrorsSuiteBase extends SharedSparkSession { +trait QueryErrorsSuiteBase extends SharedSparkSession with BeforeAndAfter { def checkErrorClass( exception: Exception with SparkThrowable, errorClass: String, @@ -49,4 +52,12 @@ trait QueryErrorsSuiteBase extends SharedSparkSession { assert(e.getSqlState === sqlState) assert(e.getMessage === s"""\n[$errorClass] """ + message) } + + before { + spark.conf.set("spark.sql.catalog.testcat", classOf[InMemoryCatalog].getName) + } + + after { + spark.sessionState.conf.unsetConf("spark.sql.catalog.testcat") + } } From efa8dacd6f93205678e3b3b63a6e2f3b221686ee Mon Sep 17 00:00:00 2001 From: lvshaokang Date: Mon, 25 Apr 2022 01:03:53 +0800 Subject: [PATCH 04/13] [SPARK-38692][SQL] Use error classes in the compilation errors of function args --- core/src/main/resources/error/error-classes.json | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json index a962b66bec69b..8399f4cd223e3 100644 --- a/core/src/main/resources/error/error-classes.json +++ b/core/src/main/resources/error/error-classes.json @@ -89,7 +89,7 @@ "sqlState" : "22023" }, "INVALID_FUNCTION_ARGUMENTS": { - "message": [ "Invalid %s of arguments for function %s. %s" ], + "message": [ "Invalid of arguments for function . " ], "sqlState" : "22023" }, "INTERNAL_ERROR" : { From 85e4c96d94b07a7b764f5fa5563d8d576437498b Mon Sep 17 00:00:00 2001 From: lvshaokang Date: Mon, 25 Apr 2022 14:57:55 +0800 Subject: [PATCH 05/13] [SPARK-38692][SQL] Use error classes in the compilation errors of function args --- core/src/main/resources/error/error-classes.json | 8 ++++---- .../expressions/aggregate/FirstLastTestSuite.scala | 4 ++-- .../sql-tests/results/ansi/string-functions.sql.out | 4 ++-- .../results/ceil-floor-with-scale-param.sql.out | 4 ++-- .../sql-tests/results/csv-functions.sql.out | 2 +- .../sql-tests/results/json-functions.sql.out | 8 ++++---- .../results/sql-compatibility-functions.sql.out | 2 +- .../sql-tests/results/string-functions.sql.out | 4 ++-- .../results/table-valued-functions.sql.out | 2 +- .../sql-tests/results/timestamp-ntz.sql.out | 2 +- .../test/resources/sql-tests/results/udaf.sql.out | 2 +- .../sql-tests/results/udf/udf-udaf.sql.out | 2 +- .../test/scala/org/apache/spark/sql/UDFSuite.scala | 2 +- .../sql/errors/QueryCompilationErrorsSuite.scala | 8 ++++++-- .../spark/sql/errors/QueryErrorsSuiteBase.scala | 13 +------------ 15 files changed, 30 insertions(+), 37 deletions(-) diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json index 8399f4cd223e3..f01faa0c936ed 100644 --- a/core/src/main/resources/error/error-classes.json +++ b/core/src/main/resources/error/error-classes.json @@ -88,10 +88,6 @@ "message" : [ "Index must be between 0 and the length of the ArrayData." ], "sqlState" : "22023" }, - "INVALID_FUNCTION_ARGUMENTS": { - "message": [ "Invalid of arguments for function . " ], - "sqlState" : "22023" - }, "INTERNAL_ERROR" : { "message" : [ "" ] }, @@ -109,6 +105,10 @@ "message" : [ "The fraction of sec must be zero. Valid range is [0, 60]. If necessary set to false to bypass this error. " ], "sqlState" : "22023" }, + "INVALID_FUNCTION_ARGUMENTS" : { + "message" : [ "Invalid of arguments for function . " ], + "sqlState" : "22023" + }, "INVALID_JSON_SCHEMA_MAPTYPE" : { "message" : [ "Input schema can only contain StringType as a key type for a MapType." ] }, diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/FirstLastTestSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/FirstLastTestSuite.scala index bb6672e1046da..4dc4ac8ee799f 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/FirstLastTestSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/FirstLastTestSuite.scala @@ -112,10 +112,10 @@ class FirstLastTestSuite extends SparkFunSuite { val msg1 = intercept[AnalysisException] { new First(input, Literal(1, IntegerType)) }.getMessage - assert(msg1.contains("The second argument in first should be a boolean literal")) + assert(msg1.contains("The second argument should be a boolean literal")) val msg2 = intercept[AnalysisException] { new Last(input, Literal(1, IntegerType)) }.getMessage - assert(msg2.contains("The second argument in last should be a boolean literal")) + assert(msg2.contains("The second argument should be a boolean literal")) } } diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/string-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/string-functions.sql.out index 4cc594d4d4a6b..a0edb22976d16 100644 --- a/sql/core/src/test/resources/sql-tests/results/ansi/string-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/ansi/string-functions.sql.out @@ -667,7 +667,7 @@ select decode() struct<> -- !query output org.apache.spark.sql.AnalysisException -Invalid number of arguments for function decode. Expected: 2; Found: 0; line 1 pos 7 +[INVALID_FUNCTION_ARGUMENTS] Invalid number of arguments for function decode. Expected: 2; Found: 0; line 1 pos 7 -- !query @@ -676,7 +676,7 @@ select decode(encode('abc', 'utf-8')) struct<> -- !query output org.apache.spark.sql.AnalysisException -Invalid number of arguments for function decode. Expected: 2; Found: 1; line 1 pos 7 +[INVALID_FUNCTION_ARGUMENTS] Invalid number of arguments for function decode. Expected: 2; Found: 1; line 1 pos 7 -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/ceil-floor-with-scale-param.sql.out b/sql/core/src/test/resources/sql-tests/results/ceil-floor-with-scale-param.sql.out index 132bd96350fb1..97b27f1037ac3 100644 --- a/sql/core/src/test/resources/sql-tests/results/ceil-floor-with-scale-param.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/ceil-floor-with-scale-param.sql.out @@ -98,7 +98,7 @@ SELECT CEIL(2.5, 0, 0) struct<> -- !query output org.apache.spark.sql.AnalysisException -Invalid number of arguments for function ceil. Expected: 2; Found: 3; line 1 pos 7 +[INVALID_FUNCTION_ARGUMENTS] Invalid number of arguments for function ceil. Expected: 2; Found: 3; line 1 pos 7 -- !query @@ -197,4 +197,4 @@ SELECT FLOOR(2.5, 0, 0) struct<> -- !query output org.apache.spark.sql.AnalysisException -Invalid number of arguments for function floor. Expected: 2; Found: 3; line 1 pos 7 +[INVALID_FUNCTION_ARGUMENTS] Invalid number of arguments for function floor. Expected: 2; Found: 3; line 1 pos 7 diff --git a/sql/core/src/test/resources/sql-tests/results/csv-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/csv-functions.sql.out index d6a3de30f63ef..0a854a25cb7b6 100644 --- a/sql/core/src/test/resources/sql-tests/results/csv-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/csv-functions.sql.out @@ -73,7 +73,7 @@ select from_csv() struct<> -- !query output org.apache.spark.sql.AnalysisException -Invalid number of arguments for function from_csv. Expected: one of 2 and 3; Found: 0; line 1 pos 7 +[INVALID_FUNCTION_ARGUMENTS] Invalid number of arguments for function from_csv. Expected: one of 2 and 3; Found: 0; line 1 pos 7 -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/json-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/json-functions.sql.out index 6f1a5c3bd5bb7..0090190fae219 100644 --- a/sql/core/src/test/resources/sql-tests/results/json-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/json-functions.sql.out @@ -90,7 +90,7 @@ select to_json() struct<> -- !query output org.apache.spark.sql.AnalysisException -Invalid number of arguments for function to_json. Expected: one of 1 and 2; Found: 0; line 1 pos 7 +[INVALID_FUNCTION_ARGUMENTS] Invalid number of arguments for function to_json. Expected: one of 1 and 2; Found: 0; line 1 pos 7 -- !query @@ -164,7 +164,7 @@ select from_json() struct<> -- !query output org.apache.spark.sql.AnalysisException -Invalid number of arguments for function from_json. Expected: one of 2 and 3; Found: 0; line 1 pos 7 +[INVALID_FUNCTION_ARGUMENTS] Invalid number of arguments for function from_json. Expected: one of 2 and 3; Found: 0; line 1 pos 7 -- !query @@ -435,7 +435,7 @@ select json_array_length() struct<> -- !query output org.apache.spark.sql.AnalysisException -Invalid number of arguments for function json_array_length. Expected: 1; Found: 0; line 1 pos 7 +[INVALID_FUNCTION_ARGUMENTS] Invalid number of arguments for function json_array_length. Expected: 1; Found: 0; line 1 pos 7 -- !query @@ -508,7 +508,7 @@ select json_object_keys() struct<> -- !query output org.apache.spark.sql.AnalysisException -Invalid number of arguments for function json_object_keys. Expected: 1; Found: 0; line 1 pos 7 +[INVALID_FUNCTION_ARGUMENTS] Invalid number of arguments for function json_object_keys. Expected: 1; Found: 0; line 1 pos 7 -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/sql-compatibility-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/sql-compatibility-functions.sql.out index 065424dfd7ada..5799b78df35a7 100644 --- a/sql/core/src/test/resources/sql-tests/results/sql-compatibility-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/sql-compatibility-functions.sql.out @@ -96,7 +96,7 @@ SELECT string(1, 2) struct<> -- !query output org.apache.spark.sql.AnalysisException -Function string accepts only one argument; line 1 pos 7 +[INVALID_FUNCTION_ARGUMENTS] Invalid number of arguments for function string. It accepts only one argument; line 1 pos 7 -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/string-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/string-functions.sql.out index dc72dfe137d7e..2361fc1e52d96 100644 --- a/sql/core/src/test/resources/sql-tests/results/string-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/string-functions.sql.out @@ -651,7 +651,7 @@ select decode() struct<> -- !query output org.apache.spark.sql.AnalysisException -Invalid number of arguments for function decode. Expected: 2; Found: 0; line 1 pos 7 +[INVALID_FUNCTION_ARGUMENTS] Invalid number of arguments for function decode. Expected: 2; Found: 0; line 1 pos 7 -- !query @@ -660,7 +660,7 @@ select decode(encode('abc', 'utf-8')) struct<> -- !query output org.apache.spark.sql.AnalysisException -Invalid number of arguments for function decode. Expected: 2; Found: 1; line 1 pos 7 +[INVALID_FUNCTION_ARGUMENTS] Invalid number of arguments for function decode. Expected: 2; Found: 1; line 1 pos 7 -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/table-valued-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/table-valued-functions.sql.out index cd85308299842..584172f65c0f1 100644 --- a/sql/core/src/test/resources/sql-tests/results/table-valued-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/table-valued-functions.sql.out @@ -75,7 +75,7 @@ Table-valued function range with alternatives: range(start: long, end: long, step: long) range(start: long, end: long) range(end: long) -cannot be applied to (integer, integer, integer, integer, integer): Invalid number of arguments for function range. Expected: one of 1, 2, 3 and 4; Found: 5; line 1 pos 14 +cannot be applied to (integer, integer, integer, integer, integer): [INVALID_FUNCTION_ARGUMENTS] Invalid number of arguments for function range. Expected: one of 1, 2, 3 and 4; Found: 5; line 1 pos 14 -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/timestamp-ntz.sql.out b/sql/core/src/test/resources/sql-tests/results/timestamp-ntz.sql.out index 1388f640d5b43..4585d0afaf1ba 100644 --- a/sql/core/src/test/resources/sql-tests/results/timestamp-ntz.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/timestamp-ntz.sql.out @@ -48,7 +48,7 @@ SELECT make_timestamp_ntz(2021, 07, 11, 6, 30, 45.678, 'CET') struct<> -- !query output org.apache.spark.sql.AnalysisException -Invalid number of arguments for function make_timestamp_ntz. Expected: 6; Found: 7; line 1 pos 7 +[INVALID_FUNCTION_ARGUMENTS] Invalid number of arguments for function make_timestamp_ntz. Expected: 6; Found: 7; line 1 pos 7 -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/udaf.sql.out b/sql/core/src/test/resources/sql-tests/results/udaf.sql.out index 9f4229a11b65d..abcd261bafb02 100644 --- a/sql/core/src/test/resources/sql-tests/results/udaf.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udaf.sql.out @@ -34,7 +34,7 @@ SELECT default.myDoubleAvg(int_col1, 3) as my_avg from t1 struct<> -- !query output org.apache.spark.sql.AnalysisException -Invalid number of arguments for function default.myDoubleAvg. Expected: 1; Found: 2; line 1 pos 7 +[INVALID_FUNCTION_ARGUMENTS] Invalid number of arguments for function default.myDoubleAvg. Expected: 1; Found: 2; line 1 pos 7 -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-udaf.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-udaf.sql.out index 19221947b4a88..3430fd0d67ba7 100644 --- a/sql/core/src/test/resources/sql-tests/results/udf/udf-udaf.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-udaf.sql.out @@ -34,7 +34,7 @@ SELECT default.myDoubleAvg(udf(int_col1), udf(3)) as my_avg from t1 struct<> -- !query output org.apache.spark.sql.AnalysisException -Invalid number of arguments for function default.myDoubleAvg. Expected: 1; Found: 2; line 1 pos 7 +[INVALID_FUNCTION_ARGUMENTS] Invalid number of arguments for function default.myDoubleAvg. Expected: 1; Found: 2; line 1 pos 7 -- !query diff --git a/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala index 8fd7aa3e0cd11..8dde2f9d7b7f6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala @@ -618,7 +618,7 @@ class UDFSuite extends QueryTest with SharedSparkSession { val e = intercept[AnalysisException] { spark.sql("SELECT CAST(1)") } - assert(e.getMessage.contains("Invalid arguments for function cast")) + assert(e.getMessage.contains("Invalid number of arguments for function cast")) } test("only one case class parameter") { 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 05908f2695d67..17e7d8d4297ee 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 @@ -462,7 +462,8 @@ class QueryCompilationErrorsSuite } test("INVALID_FUNCTION_ARGUMENTS: cannot process function input of the map type") { - withSQLConf(SQLConf.DEFAULT_CATALOG.key -> "testcat") { + withSQLConf(SQLConf.DEFAULT_CATALOG.key -> "testcat", + "spark.sql.catalog.testcat" -> classOf[InMemoryCatalog].getName) { spark.sessionState.catalogManager.catalog("testcat") .asInstanceOf[InMemoryCatalog] .createFunction(Identifier.of(Array("ns"), "strlen"), new StrLen) @@ -476,11 +477,13 @@ class QueryCompilationErrorsSuite "but found MapType(StringType,StringType,false); line 1 pos 7", sqlState = Some("22023") ) + spark.sessionState.conf.unsetConf("spark.sql.catalog.testcat") } } test("INVALID_FUNCTION_ARGUMENTS: invalid input type length of the v2Function") { - withSQLConf(SQLConf.DEFAULT_CATALOG.key -> "testcat") { + withSQLConf(SQLConf.DEFAULT_CATALOG.key -> "testcat", + "spark.sql.catalog.testcat" -> classOf[InMemoryCatalog].getName) { spark.sessionState.catalogManager.catalog("testcat") .asInstanceOf[InMemoryCatalog] .createFunction(Identifier.of(Array("ns"), "strlen"), new StrLen) @@ -493,6 +496,7 @@ class QueryCompilationErrorsSuite "There are 2 arguments but 1 parameters returned from 'inputTypes()'; line 1 pos 7", sqlState = Some("22023") ) + spark.sessionState.conf.unsetConf("spark.sql.catalog.testcat") } } 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 9a355795284fa..2c2fe188e245a 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,14 +17,11 @@ package org.apache.spark.sql.errors -import org.scalatest.BeforeAndAfter - import org.apache.spark.SparkThrowable import org.apache.spark.sql.catalyst.parser.ParseException -import org.apache.spark.sql.connector.catalog.InMemoryCatalog import org.apache.spark.sql.test.SharedSparkSession -trait QueryErrorsSuiteBase extends SharedSparkSession with BeforeAndAfter { +trait QueryErrorsSuiteBase extends SharedSparkSession { def checkErrorClass( exception: Exception with SparkThrowable, errorClass: String, @@ -52,12 +49,4 @@ trait QueryErrorsSuiteBase extends SharedSparkSession with BeforeAndAfter { assert(e.getSqlState === sqlState) assert(e.getMessage === s"""\n[$errorClass] """ + message) } - - before { - spark.conf.set("spark.sql.catalog.testcat", classOf[InMemoryCatalog].getName) - } - - after { - spark.sessionState.conf.unsetConf("spark.sql.catalog.testcat") - } } From e7c6a68bcbb88e9bc82813196221f4f79f8c309a Mon Sep 17 00:00:00 2001 From: lvshaokang Date: Tue, 26 Apr 2022 10:55:05 +0800 Subject: [PATCH 06/13] style(): code style format --- .../spark/sql/catalyst/analysis/FunctionRegistry.scala | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala index 6768b2c0f0b01..7393c1891c40f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala @@ -153,10 +153,11 @@ object FunctionRegistryBase { } catch { // the exception is an invocation exception. To get a meaningful message, we need the // cause. - case e: Exception if e.getCause.isInstanceOf[AnalysisException] => - if (e.getCause.asInstanceOf[AnalysisException].errorClass.isDefined) throw e.getCause - else throw new AnalysisException(e.getCause.getMessage) - case e: Exception => throw new AnalysisException(e.getCause.getMessage) + case e: Exception => + e.getCause match { + case ae: AnalysisException if ae.errorClass.isDefined => throw ae; + case other => throw new AnalysisException(other.getMessage) + } } } } From 7c92a855adfdbade1ae72e6f0a9d8ad5693af6e6 Mon Sep 17 00:00:00 2001 From: lvshaokang Date: Sun, 1 May 2022 23:01:26 +0800 Subject: [PATCH 07/13] [SPARK-38692][SQL] Use error classes in the compilation errors of function args --- .../main/resources/error/error-classes.json | 25 +++++++- .../sql/errors/QueryCompilationErrors.scala | 28 ++++----- .../aggregate/FirstLastTestSuite.scala | 4 +- .../results/ansi/string-functions.sql.out | 4 +- .../ceil-floor-with-scale-param.sql.out | 4 +- .../sql-tests/results/csv-functions.sql.out | 2 +- .../sql-tests/results/json-functions.sql.out | 8 +-- .../sql-compatibility-functions.sql.out | 2 +- .../results/string-functions.sql.out | 4 +- .../results/table-valued-functions.sql.out | 2 +- .../sql-tests/results/timestamp-ntz.sql.out | 2 +- .../resources/sql-tests/results/udaf.sql.out | 2 +- .../sql-tests/results/udf/udf-udaf.sql.out | 2 +- .../errors/QueryCompilationErrorsSuite.scala | 61 +++++++++++++------ 14 files changed, 96 insertions(+), 54 deletions(-) diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json index 8071af5f2fe9b..96edc1690f4d7 100644 --- a/core/src/main/resources/error/error-classes.json +++ b/core/src/main/resources/error/error-classes.json @@ -106,7 +106,30 @@ "sqlState" : "22023" }, "INVALID_FUNCTION_ARGUMENTS" : { - "message" : [ "Invalid of arguments for function . " ], + "message" : [ "The function arguments invalid: " ], + "subClass" : { + "INVALID_NUMBER_OF_ARGUMENTS" : { + "message" : [ "Invalid number of arguments for function . Expected: ; Found: " ] + }, + "EMPTY_NUMBER_OF_ARGUMENTS" : { + "message" : [ "Invalid number of arguments for function " ] + }, + "INVALID_OPERATION_FOR_V2FUNCTION" : { + "message" : [ "V2Function cannot process input: (): " ] + }, + "INVALID_NUMBER_OF_ARGUMENTS_FOR_V2FUNCTION" : { + "message" : [ "There are arguments in V2Function , but parameters returned from 'inputTypes()'" ] + }, + "CAST_ALIAS" : { + "message" : [ "Function accepts only one argument" ] + }, + "APPROX_COUNT_DISTINCT" : { + "message" : [ "The second argument in should be a double literal" ] + }, + "FIRST_LAST" : { + "message" : [ "The second argument in should be a boolean literal" ] + } + }, "sqlState" : "22023" }, "INVALID_JSON_SCHEMA_MAPTYPE" : { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala index 2e05d067dc036..f3ad35c94652b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala @@ -443,15 +443,16 @@ object QueryCompilationErrors extends QueryErrorsBase { name: String, expectedInfo: String, actualNumber: Int): Throwable = { new AnalysisException( errorClass = "INVALID_FUNCTION_ARGUMENTS", - messageParameters = Array("number", name, s"Expected: $expectedInfo; Found: $actualNumber")) + messageParameters = Array("INVALID_NUMBER_OF_ARGUMENTS", + name, expectedInfo, s"$actualNumber")) } def invalidFunctionArgumentNumberError( validParametersCount: Seq[Int], name: String, actualNumber: Int): Throwable = { - if (validParametersCount.length == 0) { + if (validParametersCount.isEmpty) { new AnalysisException( errorClass = "INVALID_FUNCTION_ARGUMENTS", - messageParameters = Array("number", name, "")) + messageParameters = Array("EMPTY_NUMBER_OF_ARGUMENTS", name)) } else { val expectedNumberOfParameters = if (validParametersCount.length == 1) { validParametersCount.head.toString @@ -466,7 +467,7 @@ object QueryCompilationErrors extends QueryErrorsBase { def functionAcceptsOnlyOneArgumentError(name: String): Throwable = { new AnalysisException( errorClass = "INVALID_FUNCTION_ARGUMENTS", - messageParameters = Array("number", name, s"It accepts only one argument")) + messageParameters = Array("CAST_ALIAS", name)) } def alterV2TableSetLocationWithPartitionNotSupportedError(): Throwable = { @@ -772,8 +773,7 @@ object QueryCompilationErrors extends QueryErrorsBase { def secondArgumentNotDoubleLiteralError(name: String): Throwable = { new AnalysisException( errorClass = "INVALID_FUNCTION_ARGUMENTS", - messageParameters = Array("type", name, - "The second argument should be a double literal") + messageParameters = Array("APPROX_COUNT_DISTINCT", name) ) } @@ -1489,10 +1489,9 @@ object QueryCompilationErrors extends QueryErrorsBase { unsupported: UnsupportedOperationException): Throwable = { new AnalysisException( errorClass = "INVALID_FUNCTION_ARGUMENTS", - messageParameters = Array("operation", unbound.name, - s"It cannot process " + - s"input: (${arguments.map(x => toSQLType(x.dataType)).mkString(", ")}): " + - unsupported.getMessage), + messageParameters = Array("INVALID_OPERATION_FOR_V2FUNCTION", + unbound.name, arguments.map(x => toSQLType(x.dataType)).mkString(", "), + unsupported.getMessage), cause = Some(unsupported)) } @@ -1501,9 +1500,9 @@ object QueryCompilationErrors extends QueryErrorsBase { args: Seq[Expression]): Throwable = { new AnalysisException( errorClass = "INVALID_FUNCTION_ARGUMENTS", - messageParameters = Array("number", bound.name(), - s"There are ${args.length} arguments " + - s"but ${bound.inputTypes().length} parameters returned from 'inputTypes()'") + messageParameters = Array("INVALID_NUMBER_OF_ARGUMENTS_FOR_V2FUNCTION", + s"${args.length}", bound.name(), + s"${bound.inputTypes().length}") ) } @@ -1586,8 +1585,7 @@ object QueryCompilationErrors extends QueryErrorsBase { def secondArgumentInFunctionIsNotBooleanLiteralError(funcName: String): Throwable = { new AnalysisException( errorClass = "INVALID_FUNCTION_ARGUMENTS", - messageParameters = Array("type", funcName, - "The second argument should be a boolean literal") + messageParameters = Array("FIRST_LAST", funcName) ) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/FirstLastTestSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/FirstLastTestSuite.scala index 4dc4ac8ee799f..bb6672e1046da 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/FirstLastTestSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/FirstLastTestSuite.scala @@ -112,10 +112,10 @@ class FirstLastTestSuite extends SparkFunSuite { val msg1 = intercept[AnalysisException] { new First(input, Literal(1, IntegerType)) }.getMessage - assert(msg1.contains("The second argument should be a boolean literal")) + assert(msg1.contains("The second argument in first should be a boolean literal")) val msg2 = intercept[AnalysisException] { new Last(input, Literal(1, IntegerType)) }.getMessage - assert(msg2.contains("The second argument should be a boolean literal")) + assert(msg2.contains("The second argument in last should be a boolean literal")) } } diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/string-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/string-functions.sql.out index a0edb22976d16..81bf81a45a8f0 100644 --- a/sql/core/src/test/resources/sql-tests/results/ansi/string-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/ansi/string-functions.sql.out @@ -667,7 +667,7 @@ select decode() struct<> -- !query output org.apache.spark.sql.AnalysisException -[INVALID_FUNCTION_ARGUMENTS] Invalid number of arguments for function decode. Expected: 2; Found: 0; line 1 pos 7 +[INVALID_FUNCTION_ARGUMENTS.INVALID_NUMBER_OF_ARGUMENTS] The function arguments invalid: Invalid number of arguments for function decode. Expected: 2; Found: 0; line 1 pos 7 -- !query @@ -676,7 +676,7 @@ select decode(encode('abc', 'utf-8')) struct<> -- !query output org.apache.spark.sql.AnalysisException -[INVALID_FUNCTION_ARGUMENTS] Invalid number of arguments for function decode. Expected: 2; Found: 1; line 1 pos 7 +[INVALID_FUNCTION_ARGUMENTS.INVALID_NUMBER_OF_ARGUMENTS] The function arguments invalid: Invalid number of arguments for function decode. Expected: 2; Found: 1; line 1 pos 7 -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/ceil-floor-with-scale-param.sql.out b/sql/core/src/test/resources/sql-tests/results/ceil-floor-with-scale-param.sql.out index 97b27f1037ac3..28587dd42d1de 100644 --- a/sql/core/src/test/resources/sql-tests/results/ceil-floor-with-scale-param.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/ceil-floor-with-scale-param.sql.out @@ -98,7 +98,7 @@ SELECT CEIL(2.5, 0, 0) struct<> -- !query output org.apache.spark.sql.AnalysisException -[INVALID_FUNCTION_ARGUMENTS] Invalid number of arguments for function ceil. Expected: 2; Found: 3; line 1 pos 7 +[INVALID_FUNCTION_ARGUMENTS.INVALID_NUMBER_OF_ARGUMENTS] The function arguments invalid: Invalid number of arguments for function ceil. Expected: 2; Found: 3; line 1 pos 7 -- !query @@ -197,4 +197,4 @@ SELECT FLOOR(2.5, 0, 0) struct<> -- !query output org.apache.spark.sql.AnalysisException -[INVALID_FUNCTION_ARGUMENTS] Invalid number of arguments for function floor. Expected: 2; Found: 3; line 1 pos 7 +[INVALID_FUNCTION_ARGUMENTS.INVALID_NUMBER_OF_ARGUMENTS] The function arguments invalid: Invalid number of arguments for function floor. Expected: 2; Found: 3; line 1 pos 7 diff --git a/sql/core/src/test/resources/sql-tests/results/csv-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/csv-functions.sql.out index 0a854a25cb7b6..2c44592a7e37a 100644 --- a/sql/core/src/test/resources/sql-tests/results/csv-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/csv-functions.sql.out @@ -73,7 +73,7 @@ select from_csv() struct<> -- !query output org.apache.spark.sql.AnalysisException -[INVALID_FUNCTION_ARGUMENTS] Invalid number of arguments for function from_csv. Expected: one of 2 and 3; Found: 0; line 1 pos 7 +[INVALID_FUNCTION_ARGUMENTS.INVALID_NUMBER_OF_ARGUMENTS] The function arguments invalid: Invalid number of arguments for function from_csv. Expected: one of 2 and 3; Found: 0; line 1 pos 7 -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/json-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/json-functions.sql.out index 0090190fae219..0a454c1df73ca 100644 --- a/sql/core/src/test/resources/sql-tests/results/json-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/json-functions.sql.out @@ -90,7 +90,7 @@ select to_json() struct<> -- !query output org.apache.spark.sql.AnalysisException -[INVALID_FUNCTION_ARGUMENTS] Invalid number of arguments for function to_json. Expected: one of 1 and 2; Found: 0; line 1 pos 7 +[INVALID_FUNCTION_ARGUMENTS.INVALID_NUMBER_OF_ARGUMENTS] The function arguments invalid: Invalid number of arguments for function to_json. Expected: one of 1 and 2; Found: 0; line 1 pos 7 -- !query @@ -164,7 +164,7 @@ select from_json() struct<> -- !query output org.apache.spark.sql.AnalysisException -[INVALID_FUNCTION_ARGUMENTS] Invalid number of arguments for function from_json. Expected: one of 2 and 3; Found: 0; line 1 pos 7 +[INVALID_FUNCTION_ARGUMENTS.INVALID_NUMBER_OF_ARGUMENTS] The function arguments invalid: Invalid number of arguments for function from_json. Expected: one of 2 and 3; Found: 0; line 1 pos 7 -- !query @@ -435,7 +435,7 @@ select json_array_length() struct<> -- !query output org.apache.spark.sql.AnalysisException -[INVALID_FUNCTION_ARGUMENTS] Invalid number of arguments for function json_array_length. Expected: 1; Found: 0; line 1 pos 7 +[INVALID_FUNCTION_ARGUMENTSINVALID_NUMBER_OF_ARGUMENTS] The function arguments invalid: Invalid number of arguments for function json_array_length. Expected: 1; Found: 0; line 1 pos 7 -- !query @@ -508,7 +508,7 @@ select json_object_keys() struct<> -- !query output org.apache.spark.sql.AnalysisException -[INVALID_FUNCTION_ARGUMENTS] Invalid number of arguments for function json_object_keys. Expected: 1; Found: 0; line 1 pos 7 +[INVALID_FUNCTION_ARGUMENTS.INVALID_NUMBER_OF_ARGUMENTS] The function arguments invalid: Invalid number of arguments for function json_object_keys. Expected: 1; Found: 0; line 1 pos 7 -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/sql-compatibility-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/sql-compatibility-functions.sql.out index 5799b78df35a7..ccf8c99752f68 100644 --- a/sql/core/src/test/resources/sql-tests/results/sql-compatibility-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/sql-compatibility-functions.sql.out @@ -96,7 +96,7 @@ SELECT string(1, 2) struct<> -- !query output org.apache.spark.sql.AnalysisException -[INVALID_FUNCTION_ARGUMENTS] Invalid number of arguments for function string. It accepts only one argument; line 1 pos 7 +[INVALID_FUNCTION_ARGUMENTS.CAST_ALIAS] The function arguments invalid: Function string accepts only one argument; line 1 pos 7 -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/string-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/string-functions.sql.out index 2361fc1e52d96..86f957b628ac0 100644 --- a/sql/core/src/test/resources/sql-tests/results/string-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/string-functions.sql.out @@ -651,7 +651,7 @@ select decode() struct<> -- !query output org.apache.spark.sql.AnalysisException -[INVALID_FUNCTION_ARGUMENTS] Invalid number of arguments for function decode. Expected: 2; Found: 0; line 1 pos 7 +[INVALID_FUNCTION_ARGUMENTS.INVALID_NUMBER_OF_ARGUMENTS] The function arguments invalid: Invalid number of arguments for function decode. Expected: 2; Found: 0; line 1 pos 7 -- !query @@ -660,7 +660,7 @@ select decode(encode('abc', 'utf-8')) struct<> -- !query output org.apache.spark.sql.AnalysisException -[INVALID_FUNCTION_ARGUMENTS] Invalid number of arguments for function decode. Expected: 2; Found: 1; line 1 pos 7 +[INVALID_FUNCTION_ARGUMENTS.INVALID_NUMBER_OF_ARGUMENTS] The function arguments invalid: Invalid number of arguments for function decode. Expected: 2; Found: 1; line 1 pos 7 -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/table-valued-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/table-valued-functions.sql.out index 584172f65c0f1..11c6d1772bc17 100644 --- a/sql/core/src/test/resources/sql-tests/results/table-valued-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/table-valued-functions.sql.out @@ -75,7 +75,7 @@ Table-valued function range with alternatives: range(start: long, end: long, step: long) range(start: long, end: long) range(end: long) -cannot be applied to (integer, integer, integer, integer, integer): [INVALID_FUNCTION_ARGUMENTS] Invalid number of arguments for function range. Expected: one of 1, 2, 3 and 4; Found: 5; line 1 pos 14 +cannot be applied to (integer, integer, integer, integer, integer): [INVALID_FUNCTION_ARGUMENTS.INVALID_NUMBER_OF_ARGUMENTS] The function arguments invalid: Invalid number of arguments for function range. Expected: one of 1, 2, 3 and 4; Found: 5; line 1 pos 14 -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/timestamp-ntz.sql.out b/sql/core/src/test/resources/sql-tests/results/timestamp-ntz.sql.out index 4585d0afaf1ba..0ecd3d4ce3393 100644 --- a/sql/core/src/test/resources/sql-tests/results/timestamp-ntz.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/timestamp-ntz.sql.out @@ -48,7 +48,7 @@ SELECT make_timestamp_ntz(2021, 07, 11, 6, 30, 45.678, 'CET') struct<> -- !query output org.apache.spark.sql.AnalysisException -[INVALID_FUNCTION_ARGUMENTS] Invalid number of arguments for function make_timestamp_ntz. Expected: 6; Found: 7; line 1 pos 7 +[INVALID_FUNCTION_ARGUMENTS.INVALID_NUMBER_OF_ARGUMENTS] The function arguments invalid: Invalid number of arguments for function make_timestamp_ntz. Expected: 6; Found: 7; line 1 pos 7 -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/udaf.sql.out b/sql/core/src/test/resources/sql-tests/results/udaf.sql.out index abcd261bafb02..6634242449bde 100644 --- a/sql/core/src/test/resources/sql-tests/results/udaf.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udaf.sql.out @@ -34,7 +34,7 @@ SELECT default.myDoubleAvg(int_col1, 3) as my_avg from t1 struct<> -- !query output org.apache.spark.sql.AnalysisException -[INVALID_FUNCTION_ARGUMENTS] Invalid number of arguments for function default.myDoubleAvg. Expected: 1; Found: 2; line 1 pos 7 +[INVALID_FUNCTION_ARGUMENTS.INVALID_NUMBER_OF_ARGUMENTS] The function arguments invalid: Invalid number of arguments for function default.myDoubleAvg. Expected: 1; Found: 2; line 1 pos 7 -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-udaf.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-udaf.sql.out index 3430fd0d67ba7..9414b45f7cb3c 100644 --- a/sql/core/src/test/resources/sql-tests/results/udf/udf-udaf.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-udaf.sql.out @@ -34,7 +34,7 @@ SELECT default.myDoubleAvg(udf(int_col1), udf(3)) as my_avg from t1 struct<> -- !query output org.apache.spark.sql.AnalysisException -[INVALID_FUNCTION_ARGUMENTS] Invalid number of arguments for function default.myDoubleAvg. Expected: 1; Found: 2; line 1 pos 7 +[INVALID_FUNCTION_ARGUMENTS.INVALID_NUMBER_OF_ARGUMENTS] The function arguments invalid: Invalid number of arguments for function default.myDoubleAvg. Expected: 1; Found: 2; line 1 pos 7 -- !query 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 24bfbb19ea9ae..d87b906656fc2 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 @@ -478,39 +478,56 @@ class QueryCompilationErrorsSuite } } - test("INVALID_FUNCTION_ARGUMENTS: invalid number of the function arguments") { + test("INVALID_FUNCTION_ARGUMENTS: invalid number of the udf function arguments") { spark.udf.register("testFunc", (n: Int) => n.toString) checkErrorClass( exception = intercept[AnalysisException]( sql(s"SELECT testFunc(123, 123) as value") ), errorClass = "INVALID_FUNCTION_ARGUMENTS", - msg = "Invalid number of arguments for function testFunc. " + - "Expected: 1; Found: 2; line 1 pos 7", + errorSubClass = Some("INVALID_NUMBER_OF_ARGUMENTS"), + msg = "The function arguments invalid: " + + "Invalid number of arguments for function testFunc. Expected: 1; Found: 2; line 1 pos 7", + sqlState = Some("22023") + ) + } + + test("INVALID_FUNCTION_ARGUMENTS: empty number of the function arguments") { + checkErrorClass( + exception = intercept[AnalysisException]( + sql("SELECT CAST()") + ), + errorClass = "INVALID_FUNCTION_ARGUMENTS", + errorSubClass = Some("EMPTY_NUMBER_OF_ARGUMENTS"), + msg = "The function arguments invalid: " + + "Invalid number of arguments for function cast; line 1 pos 7", sqlState = Some("22023") ) } - test("INVALID_FUNCTION_ARGUMENTS: invalid function arguments number") { + test("INVALID_FUNCTION_ARGUMENTS: invalid number of the function arguments") { checkErrorClass( exception = intercept[AnalysisException]( - sql(s"SELECT to_timestamp_ntz()") + sql("SELECT to_timestamp_ntz()") ), errorClass = "INVALID_FUNCTION_ARGUMENTS", - msg = "Invalid number of arguments for function to_timestamp_ntz. " + + errorSubClass = Some("INVALID_NUMBER_OF_ARGUMENTS"), + msg = "The function arguments invalid: " + + "Invalid number of arguments for function to_timestamp_ntz. " + "Expected: one of 1 and 2; Found: 0; line 1 pos 7", sqlState = Some("22023") ) } - test("INVALID_FUNCTION_ARGUMENTS: only accept one argument") { + test("INVALID_FUNCTION_ARGUMENTS: the function use two arguments") { checkErrorClass( exception = intercept[AnalysisException]( sql(s"SELECT int('1', '2')") ), errorClass = "INVALID_FUNCTION_ARGUMENTS", - msg = "Invalid number of arguments for function int. " + - "It accepts only one argument; line 1 pos 7", + errorSubClass = Some("CAST_ALIAS"), + msg = "The function arguments invalid: " + + "Function int accepts only one argument; line 1 pos 7", sqlState = Some("22023") ) } @@ -521,8 +538,9 @@ class QueryCompilationErrorsSuite sql(s"SELECT approx_count_distinct(1,1)") ), errorClass = "INVALID_FUNCTION_ARGUMENTS", - msg = "Invalid type of arguments for function approx_count_distinct. " + - "The second argument should be a double literal; line 1 pos 7", + errorSubClass = Some("APPROX_COUNT_DISTINCT"), + msg = "The function arguments invalid: " + + "The second argument in approx_count_distinct should be a double literal; line 1 pos 7", sqlState = Some("22023") ) } @@ -538,8 +556,9 @@ class QueryCompilationErrorsSuite sql("SELECT testcat.ns.strlen(map('abc', 'abc'))") ), errorClass = "INVALID_FUNCTION_ARGUMENTS", - msg = "Invalid operation of arguments for function strlen. " + - "It cannot process input: (\"MAP\"): Expect StringType, " + + errorSubClass = Some("INVALID_OPERATION_FOR_V2FUNCTION"), + msg = "The function arguments invalid: " + + "V2Function strlen cannot process input: (\"MAP\"): Expect StringType, " + "but found MapType(StringType,StringType,false); line 1 pos 7", sqlState = Some("22023") ) @@ -547,7 +566,7 @@ class QueryCompilationErrorsSuite } } - test("INVALID_FUNCTION_ARGUMENTS: invalid input type length of the v2Function") { + test("INVALID_FUNCTION_ARGUMENTS: invalid input type length of the V2Function") { withSQLConf(SQLConf.DEFAULT_CATALOG.key -> "testcat", "spark.sql.catalog.testcat" -> classOf[InMemoryCatalog].getName) { spark.sessionState.catalogManager.catalog("testcat") @@ -558,8 +577,10 @@ class QueryCompilationErrorsSuite sql("SELECT testcat.ns.strlen('abc','abc')") ), errorClass = "INVALID_FUNCTION_ARGUMENTS", - msg = "Invalid number of arguments for function strlen. " + - "There are 2 arguments but 1 parameters returned from 'inputTypes()'; line 1 pos 7", + errorSubClass = Some("INVALID_NUMBER_OF_ARGUMENTS_FOR_V2FUNCTION"), + msg = "The function arguments invalid: " + + "There are 2 arguments in V2Function strlen, " + + "but 1 parameters returned from 'inputTypes()'; line 1 pos 7", sqlState = Some("22023") ) spark.sessionState.conf.unsetConf("spark.sql.catalog.testcat") @@ -572,8 +593,9 @@ class QueryCompilationErrorsSuite sql(s"SELECT first(1, '1')") ), errorClass = "INVALID_FUNCTION_ARGUMENTS", - msg = "Invalid type of arguments for function first. " + - "The second argument should be a boolean literal; line 1 pos 7", + errorSubClass = Some("FIRST_LAST"), + msg = "The function arguments invalid: " + + "The second argument in first should be a boolean literal; line 1 pos 7", sqlState = Some("22023") ) } @@ -612,8 +634,7 @@ class StrLen extends UnboundFunction { override def bind(inputType: StructType): BoundFunction = { for (field <- inputType.fields) { if (!field.dataType.isInstanceOf[StringType]) { - throw new UnsupportedOperationException( - "Expect StringType, but found " + field.dataType) + throw new UnsupportedOperationException("Expect StringType, but found " + field.dataType) } } new ScalarFunction[Int] { From e21b47a32cc9eb4dc166760b18cbce1b3a39ef11 Mon Sep 17 00:00:00 2001 From: lvshaokang Date: Thu, 5 May 2022 21:18:51 +0800 Subject: [PATCH 08/13] style(): code style format --- .../main/resources/error/error-classes.json | 42 +++++++++---------- .../sql-tests/results/json-functions.sql.out | 2 +- 2 files changed, 22 insertions(+), 22 deletions(-) diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json index 67bd5d7d62403..6a009f02d3d09 100644 --- a/core/src/main/resources/error/error-classes.json +++ b/core/src/main/resources/error/error-classes.json @@ -105,27 +105,27 @@ "INVALID_FUNCTION_ARGUMENTS" : { "message" : [ "The function arguments invalid: " ], "subClass" : { - "INVALID_NUMBER_OF_ARGUMENTS" : { - "message" : [ "Invalid number of arguments for function . Expected: ; Found: " ] - }, - "EMPTY_NUMBER_OF_ARGUMENTS" : { - "message" : [ "Invalid number of arguments for function " ] - }, - "INVALID_OPERATION_FOR_V2FUNCTION" : { - "message" : [ "V2Function cannot process input: (): " ] - }, - "INVALID_NUMBER_OF_ARGUMENTS_FOR_V2FUNCTION" : { - "message" : [ "There are arguments in V2Function , but parameters returned from 'inputTypes()'" ] - }, - "CAST_ALIAS" : { - "message" : [ "Function accepts only one argument" ] - }, - "APPROX_COUNT_DISTINCT" : { - "message" : [ "The second argument in should be a double literal" ] - }, - "FIRST_LAST" : { - "message" : [ "The second argument in should be a boolean literal" ] - } + "APPROX_COUNT_DISTINCT" : { + "message" : [ "The second argument in should be a double literal" ] + }, + "CAST_ALIAS" : { + "message" : [ "Function accepts only one argument" ] + }, + "EMPTY_NUMBER_OF_ARGUMENTS" : { + "message" : [ "Invalid number of arguments for function " ] + }, + "FIRST_LAST" : { + "message" : [ "The second argument in should be a boolean literal" ] + }, + "INVALID_NUMBER_OF_ARGUMENTS" : { + "message" : [ "Invalid number of arguments for function . Expected: ; Found: " ] + }, + "INVALID_NUMBER_OF_ARGUMENTS_FOR_V2FUNCTION" : { + "message" : [ "There are arguments in V2Function , but parameters returned from 'inputTypes()'" ] + }, + "INVALID_OPERATION_FOR_V2FUNCTION" : { + "message" : [ "V2Function cannot process input: (): " ] + } }, "sqlState" : "22023" }, diff --git a/sql/core/src/test/resources/sql-tests/results/json-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/json-functions.sql.out index 0a454c1df73ca..638a1d19f1fd6 100644 --- a/sql/core/src/test/resources/sql-tests/results/json-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/json-functions.sql.out @@ -435,7 +435,7 @@ select json_array_length() struct<> -- !query output org.apache.spark.sql.AnalysisException -[INVALID_FUNCTION_ARGUMENTSINVALID_NUMBER_OF_ARGUMENTS] The function arguments invalid: Invalid number of arguments for function json_array_length. Expected: 1; Found: 0; line 1 pos 7 +[INVALID_FUNCTION_ARGUMENTS.INVALID_NUMBER_OF_ARGUMENTS] The function arguments invalid: Invalid number of arguments for function json_array_length. Expected: 1; Found: 0; line 1 pos 7 -- !query From 11069617fd33f7968323b40ac9c7e7d71834b5e2 Mon Sep 17 00:00:00 2001 From: lvshaokang Date: Wed, 17 Aug 2022 16:24:35 +0800 Subject: [PATCH 09/13] fix(): unify `INVALID_PARAMETER_VALUE` class and format message --- .../main/resources/error/error-classes.json | 29 +++++++----- .../sql/errors/QueryCompilationErrors.scala | 8 ++-- .../sql/errors/QueryExecutionErrors.scala | 13 +++--- .../aggregate/FirstLastTestSuite.scala | 4 +- .../results/ansi/string-functions.sql.out | 4 +- .../ceil-floor-with-scale-param.sql.out | 4 +- .../sql-tests/results/csv-functions.sql.out | 2 +- .../sql-tests/results/json-functions.sql.out | 8 ++-- .../sql-tests/results/postgreSQL/text.sql.out | 2 +- .../sql-compatibility-functions.sql.out | 2 +- .../results/string-functions.sql.out | 4 +- .../results/table-valued-functions.sql.out | 2 +- .../sql-tests/results/timestamp-ntz.sql.out | 2 +- .../resources/sql-tests/results/udaf.sql.out | 2 +- .../sql-tests/results/udf/udf-udaf.sql.out | 2 +- .../scala/org/apache/spark/sql/UDFSuite.scala | 6 +-- .../errors/QueryCompilationErrorsSuite.scala | 44 ++++++++++--------- .../errors/QueryExecutionErrorsSuite.scala | 18 +++++--- 18 files changed, 86 insertions(+), 70 deletions(-) diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json index 2e84b81bc82e7..341ed1853711d 100644 --- a/core/src/main/resources/error/error-classes.json +++ b/core/src/main/resources/error/error-classes.json @@ -120,28 +120,37 @@ "sqlState" : "22023" }, "INVALID_FUNCTION_ARGUMENTS" : { - "message" : [ "The function arguments invalid: " ], + "message" : [ "Arguments of the function are invalid: " ], "subClass" : { "APPROX_COUNT_DISTINCT" : { - "message" : [ "The second argument in should be a double literal" ] + "message" : [ "The second argument should be a double literal" ] }, "CAST_ALIAS" : { - "message" : [ "Function accepts only one argument" ] + "message" : [ "Function accepts only one argument" ] }, "EMPTY_NUMBER_OF_ARGUMENTS" : { - "message" : [ "Invalid number of arguments for function " ] + "message" : [ "Empty number of arguments" ] }, "FIRST_LAST" : { - "message" : [ "The second argument in should be a boolean literal" ] + "message" : [ "The second argument should be a boolean literal" ] + }, + "INVALID_ARGUMENT_INDEX" : { + "message" : [ "The value of parameter(s) '' has invalid index, expects , but got ." ] + }, + "INVALID_ARGUMENT_LENGTH" : { + "message" : [ "The value of parameter(s) '' has invalid length, expects , but got ." ] + }, + "INVALID_ARGUMENT_VALUE" : { + "message" : [ "The value of parameter(s) '' is invalid, " ] }, "INVALID_NUMBER_OF_ARGUMENTS" : { - "message" : [ "Invalid number of arguments for function . Expected: ; Found: " ] + "message" : [ "Invalid number of arguments. Expected: ; Found: " ] }, "INVALID_NUMBER_OF_ARGUMENTS_FOR_V2FUNCTION" : { - "message" : [ "There are arguments in V2Function , but parameters returned from 'inputTypes()'" ] + "message" : [ "There are arguments in V2Function, but parameters returned from 'inputTypes()'" ] }, "INVALID_OPERATION_FOR_V2FUNCTION" : { - "message" : [ "V2Function cannot process input: (): " ] + "message" : [ "V2Function cannot process input: (): " ] } }, "sqlState" : "22023" @@ -152,10 +161,6 @@ "INVALID_PANDAS_UDF_PLACEMENT" : { "message" : [ "The group aggregate pandas UDF cannot be invoked together with as other, non-pandas aggregate functions." ] }, - "INVALID_PARAMETER_VALUE" : { - "message" : [ "The value of parameter(s) '' in is invalid: " ], - "sqlState" : "22023" - }, "INVALID_SQL_SYNTAX" : { "message" : [ "Invalid SQL syntax: " ], "sqlState" : "42000" diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala index e7b29a1029ea3..75d4cd526bba0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala @@ -68,9 +68,9 @@ object QueryCompilationErrors extends QueryErrorsBase { def zeroArgumentIndexError(): Throwable = { new AnalysisException( - errorClass = "INVALID_PARAMETER_VALUE", - messageParameters = Array( - "strfmt", toSQLId("format_string"), "expects %1$, %2$ and so on, but got %0$.")) + errorClass = "INVALID_FUNCTION_ARGUMENTS", + messageParameters = Array("INVALID_ARGUMENT_INDEX", + toSQLId("format_string"), "strfmt", "%1$, %2$ and so on", "%0$")) } def unorderablePivotColError(pivotCol: Expression): Throwable = { @@ -1502,7 +1502,7 @@ object QueryCompilationErrors extends QueryErrorsBase { new AnalysisException( errorClass = "INVALID_FUNCTION_ARGUMENTS", messageParameters = Array("INVALID_NUMBER_OF_ARGUMENTS_FOR_V2FUNCTION", - s"${args.length}", bound.name(), + bound.name(), s"${args.length}", s"${bound.inputTypes().length}") ) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala index 53d32927cee32..8e27164107abd 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala @@ -1902,11 +1902,13 @@ object QueryExecutionErrors extends QueryErrorsBase { def invalidAesKeyLengthError(actualLength: Int): RuntimeException = { new SparkRuntimeException( - errorClass = "INVALID_PARAMETER_VALUE", + errorClass = "INVALID_FUNCTION_ARGUMENTS", messageParameters = Array( + "INVALID_ARGUMENT_LENGTH", + s"$aesFuncName", "key", - s"the $aesFuncName function", - s"expects a binary value with 16, 24 or 32 bytes, but got ${actualLength.toString} bytes.")) + "a binary value with 16, 24 or 32 bytes", + s"${actualLength.toString} bytes")) } def aesModeUnsupportedError(mode: String, padding: String): RuntimeException = { @@ -1917,10 +1919,11 @@ object QueryExecutionErrors extends QueryErrorsBase { def aesCryptoError(detailMessage: String): RuntimeException = { new SparkRuntimeException( - errorClass = "INVALID_PARAMETER_VALUE", + errorClass = "INVALID_FUNCTION_ARGUMENTS", messageParameters = Array( + "INVALID_ARGUMENT_VALUE", + aesFuncName, "expr, key", - s"the $aesFuncName function", s"Detail message: $detailMessage")) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/FirstLastTestSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/FirstLastTestSuite.scala index bb6672e1046da..4dc4ac8ee799f 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/FirstLastTestSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/FirstLastTestSuite.scala @@ -112,10 +112,10 @@ class FirstLastTestSuite extends SparkFunSuite { val msg1 = intercept[AnalysisException] { new First(input, Literal(1, IntegerType)) }.getMessage - assert(msg1.contains("The second argument in first should be a boolean literal")) + assert(msg1.contains("The second argument should be a boolean literal")) val msg2 = intercept[AnalysisException] { new Last(input, Literal(1, IntegerType)) }.getMessage - assert(msg2.contains("The second argument in last should be a boolean literal")) + assert(msg2.contains("The second argument should be a boolean literal")) } } diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/string-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/string-functions.sql.out index 81bf81a45a8f0..25050b96a0c90 100644 --- a/sql/core/src/test/resources/sql-tests/results/ansi/string-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/ansi/string-functions.sql.out @@ -667,7 +667,7 @@ select decode() struct<> -- !query output org.apache.spark.sql.AnalysisException -[INVALID_FUNCTION_ARGUMENTS.INVALID_NUMBER_OF_ARGUMENTS] The function arguments invalid: Invalid number of arguments for function decode. Expected: 2; Found: 0; line 1 pos 7 +[INVALID_FUNCTION_ARGUMENTS.INVALID_NUMBER_OF_ARGUMENTS] Arguments of the decode function are invalid: Invalid number of arguments. Expected: 2; Found: 0; line 1 pos 7 -- !query @@ -676,7 +676,7 @@ select decode(encode('abc', 'utf-8')) struct<> -- !query output org.apache.spark.sql.AnalysisException -[INVALID_FUNCTION_ARGUMENTS.INVALID_NUMBER_OF_ARGUMENTS] The function arguments invalid: Invalid number of arguments for function decode. Expected: 2; Found: 1; line 1 pos 7 +[INVALID_FUNCTION_ARGUMENTS.INVALID_NUMBER_OF_ARGUMENTS] Arguments of the decode function are invalid: Invalid number of arguments. Expected: 2; Found: 1; line 1 pos 7 -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/ceil-floor-with-scale-param.sql.out b/sql/core/src/test/resources/sql-tests/results/ceil-floor-with-scale-param.sql.out index 28587dd42d1de..6f1d7ffaf01d5 100644 --- a/sql/core/src/test/resources/sql-tests/results/ceil-floor-with-scale-param.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/ceil-floor-with-scale-param.sql.out @@ -98,7 +98,7 @@ SELECT CEIL(2.5, 0, 0) struct<> -- !query output org.apache.spark.sql.AnalysisException -[INVALID_FUNCTION_ARGUMENTS.INVALID_NUMBER_OF_ARGUMENTS] The function arguments invalid: Invalid number of arguments for function ceil. Expected: 2; Found: 3; line 1 pos 7 +[INVALID_FUNCTION_ARGUMENTS.INVALID_NUMBER_OF_ARGUMENTS] Arguments of the ceil function are invalid: Invalid number of arguments. Expected: 2; Found: 3; line 1 pos 7 -- !query @@ -197,4 +197,4 @@ SELECT FLOOR(2.5, 0, 0) struct<> -- !query output org.apache.spark.sql.AnalysisException -[INVALID_FUNCTION_ARGUMENTS.INVALID_NUMBER_OF_ARGUMENTS] The function arguments invalid: Invalid number of arguments for function floor. Expected: 2; Found: 3; line 1 pos 7 +[INVALID_FUNCTION_ARGUMENTS.INVALID_NUMBER_OF_ARGUMENTS] Arguments of the floor function are invalid: Invalid number of arguments. Expected: 2; Found: 3; line 1 pos 7 diff --git a/sql/core/src/test/resources/sql-tests/results/csv-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/csv-functions.sql.out index 2c44592a7e37a..9fac974c576aa 100644 --- a/sql/core/src/test/resources/sql-tests/results/csv-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/csv-functions.sql.out @@ -73,7 +73,7 @@ select from_csv() struct<> -- !query output org.apache.spark.sql.AnalysisException -[INVALID_FUNCTION_ARGUMENTS.INVALID_NUMBER_OF_ARGUMENTS] The function arguments invalid: Invalid number of arguments for function from_csv. Expected: one of 2 and 3; Found: 0; line 1 pos 7 +[INVALID_FUNCTION_ARGUMENTS.INVALID_NUMBER_OF_ARGUMENTS] Arguments of the from_csv function are invalid: Invalid number of arguments. Expected: one of 2 and 3; Found: 0; line 1 pos 7 -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/json-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/json-functions.sql.out index e30c3d73a911d..61fdfce9bca92 100644 --- a/sql/core/src/test/resources/sql-tests/results/json-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/json-functions.sql.out @@ -90,7 +90,7 @@ select to_json() struct<> -- !query output org.apache.spark.sql.AnalysisException -[INVALID_FUNCTION_ARGUMENTS.INVALID_NUMBER_OF_ARGUMENTS] The function arguments invalid: Invalid number of arguments for function to_json. Expected: one of 1 and 2; Found: 0; line 1 pos 7 +[INVALID_FUNCTION_ARGUMENTS.INVALID_NUMBER_OF_ARGUMENTS] Arguments of the to_json function are invalid: Invalid number of arguments. Expected: one of 1 and 2; Found: 0; line 1 pos 7 -- !query @@ -164,7 +164,7 @@ select from_json() struct<> -- !query output org.apache.spark.sql.AnalysisException -[INVALID_FUNCTION_ARGUMENTS.INVALID_NUMBER_OF_ARGUMENTS] The function arguments invalid: Invalid number of arguments for function from_json. Expected: one of 2 and 3; Found: 0; line 1 pos 7 +[INVALID_FUNCTION_ARGUMENTS.INVALID_NUMBER_OF_ARGUMENTS] Arguments of the from_json function are invalid: Invalid number of arguments. Expected: one of 2 and 3; Found: 0; line 1 pos 7 -- !query @@ -437,7 +437,7 @@ select json_array_length() struct<> -- !query output org.apache.spark.sql.AnalysisException -[INVALID_FUNCTION_ARGUMENTS.INVALID_NUMBER_OF_ARGUMENTS] The function arguments invalid: Invalid number of arguments for function json_array_length. Expected: 1; Found: 0; line 1 pos 7 +[INVALID_FUNCTION_ARGUMENTS.INVALID_NUMBER_OF_ARGUMENTS] Arguments of the json_array_length function are invalid: Invalid number of arguments. Expected: 1; Found: 0; line 1 pos 7 -- !query @@ -510,7 +510,7 @@ select json_object_keys() struct<> -- !query output org.apache.spark.sql.AnalysisException -[INVALID_FUNCTION_ARGUMENTS.INVALID_NUMBER_OF_ARGUMENTS] The function arguments invalid: Invalid number of arguments for function json_object_keys. Expected: 1; Found: 0; line 1 pos 7 +[INVALID_FUNCTION_ARGUMENTS.INVALID_NUMBER_OF_ARGUMENTS] Arguments of the json_object_keys function are invalid: Invalid number of arguments. Expected: 1; Found: 0; line 1 pos 7 -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/text.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/text.sql.out index 769b2b92d80c4..a32dabe2069ac 100755 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/text.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/text.sql.out @@ -285,7 +285,7 @@ select format_string('%0$s', 'Hello') struct<> -- !query output org.apache.spark.sql.AnalysisException -[INVALID_PARAMETER_VALUE] The value of parameter(s) 'strfmt' in `format_string` is invalid: expects %1$, %2$ and so on, but got %0$.; line 1 pos 7 +[INVALID_FUNCTION_ARGUMENTS.INVALID_ARGUMENT_INDEX] Arguments of the `format_string` function are invalid: The value of parameter(s) 'strfmt' has invalid index, expects %1$, %2$ and so on, but got %0$.; line 1 pos 7 -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/sql-compatibility-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/sql-compatibility-functions.sql.out index ccf8c99752f68..df2a98742ce2a 100644 --- a/sql/core/src/test/resources/sql-tests/results/sql-compatibility-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/sql-compatibility-functions.sql.out @@ -96,7 +96,7 @@ SELECT string(1, 2) struct<> -- !query output org.apache.spark.sql.AnalysisException -[INVALID_FUNCTION_ARGUMENTS.CAST_ALIAS] The function arguments invalid: Function string accepts only one argument; line 1 pos 7 +[INVALID_FUNCTION_ARGUMENTS.CAST_ALIAS] Arguments of the string function are invalid: Function accepts only one argument; line 1 pos 7 -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/string-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/string-functions.sql.out index 86f957b628ac0..0ac8d2bd18a18 100644 --- a/sql/core/src/test/resources/sql-tests/results/string-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/string-functions.sql.out @@ -651,7 +651,7 @@ select decode() struct<> -- !query output org.apache.spark.sql.AnalysisException -[INVALID_FUNCTION_ARGUMENTS.INVALID_NUMBER_OF_ARGUMENTS] The function arguments invalid: Invalid number of arguments for function decode. Expected: 2; Found: 0; line 1 pos 7 +[INVALID_FUNCTION_ARGUMENTS.INVALID_NUMBER_OF_ARGUMENTS] Arguments of the decode function are invalid: Invalid number of arguments. Expected: 2; Found: 0; line 1 pos 7 -- !query @@ -660,7 +660,7 @@ select decode(encode('abc', 'utf-8')) struct<> -- !query output org.apache.spark.sql.AnalysisException -[INVALID_FUNCTION_ARGUMENTS.INVALID_NUMBER_OF_ARGUMENTS] The function arguments invalid: Invalid number of arguments for function decode. Expected: 2; Found: 1; line 1 pos 7 +[INVALID_FUNCTION_ARGUMENTS.INVALID_NUMBER_OF_ARGUMENTS] Arguments of the decode function are invalid: Invalid number of arguments. Expected: 2; Found: 1; line 1 pos 7 -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/table-valued-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/table-valued-functions.sql.out index 11c6d1772bc17..b36661f6084a9 100644 --- a/sql/core/src/test/resources/sql-tests/results/table-valued-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/table-valued-functions.sql.out @@ -75,7 +75,7 @@ Table-valued function range with alternatives: range(start: long, end: long, step: long) range(start: long, end: long) range(end: long) -cannot be applied to (integer, integer, integer, integer, integer): [INVALID_FUNCTION_ARGUMENTS.INVALID_NUMBER_OF_ARGUMENTS] The function arguments invalid: Invalid number of arguments for function range. Expected: one of 1, 2, 3 and 4; Found: 5; line 1 pos 14 +cannot be applied to (integer, integer, integer, integer, integer): [INVALID_FUNCTION_ARGUMENTS.INVALID_NUMBER_OF_ARGUMENTS] Arguments of the range function are invalid: Invalid number of arguments. Expected: one of 1, 2, 3 and 4; Found: 5; line 1 pos 14 -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/timestamp-ntz.sql.out b/sql/core/src/test/resources/sql-tests/results/timestamp-ntz.sql.out index 0ecd3d4ce3393..56bd344abe8e4 100644 --- a/sql/core/src/test/resources/sql-tests/results/timestamp-ntz.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/timestamp-ntz.sql.out @@ -48,7 +48,7 @@ SELECT make_timestamp_ntz(2021, 07, 11, 6, 30, 45.678, 'CET') struct<> -- !query output org.apache.spark.sql.AnalysisException -[INVALID_FUNCTION_ARGUMENTS.INVALID_NUMBER_OF_ARGUMENTS] The function arguments invalid: Invalid number of arguments for function make_timestamp_ntz. Expected: 6; Found: 7; line 1 pos 7 +[INVALID_FUNCTION_ARGUMENTS.INVALID_NUMBER_OF_ARGUMENTS] Arguments of the make_timestamp_ntz function are invalid: Invalid number of arguments. Expected: 6; Found: 7; line 1 pos 7 -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/udaf.sql.out b/sql/core/src/test/resources/sql-tests/results/udaf.sql.out index 6634242449bde..bd9aad4d73eec 100644 --- a/sql/core/src/test/resources/sql-tests/results/udaf.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udaf.sql.out @@ -34,7 +34,7 @@ SELECT default.myDoubleAvg(int_col1, 3) as my_avg from t1 struct<> -- !query output org.apache.spark.sql.AnalysisException -[INVALID_FUNCTION_ARGUMENTS.INVALID_NUMBER_OF_ARGUMENTS] The function arguments invalid: Invalid number of arguments for function default.myDoubleAvg. Expected: 1; Found: 2; line 1 pos 7 +[INVALID_FUNCTION_ARGUMENTS.INVALID_NUMBER_OF_ARGUMENTS] Arguments of the default.myDoubleAvg function are invalid: Invalid number of arguments. Expected: 1; Found: 2; line 1 pos 7 -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-udaf.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-udaf.sql.out index 9414b45f7cb3c..667538d6d697f 100644 --- a/sql/core/src/test/resources/sql-tests/results/udf/udf-udaf.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-udaf.sql.out @@ -34,7 +34,7 @@ SELECT default.myDoubleAvg(udf(int_col1), udf(3)) as my_avg from t1 struct<> -- !query output org.apache.spark.sql.AnalysisException -[INVALID_FUNCTION_ARGUMENTS.INVALID_NUMBER_OF_ARGUMENTS] The function arguments invalid: Invalid number of arguments for function default.myDoubleAvg. Expected: 1; Found: 2; line 1 pos 7 +[INVALID_FUNCTION_ARGUMENTS.INVALID_NUMBER_OF_ARGUMENTS] Arguments of the default.myDoubleAvg function are invalid: Invalid number of arguments. Expected: 1; Found: 2; line 1 pos 7 -- !query diff --git a/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala index 8dde2f9d7b7f6..6c877c61bfec6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala @@ -103,7 +103,7 @@ class UDFSuite extends QueryTest with SharedSparkSession { val e = intercept[AnalysisException] { df.selectExpr("substr('abcd', 2, 3, 4)") } - assert(e.getMessage.contains("Invalid number of arguments for function substr. Expected:")) + assert(e.getMessage.contains("Invalid number of arguments. Expected:")) } test("error reporting for incorrect number of arguments - udf") { @@ -112,7 +112,7 @@ class UDFSuite extends QueryTest with SharedSparkSession { spark.udf.register("foo", (_: String).length) df.selectExpr("foo(2, 3, 4)") } - assert(e.getMessage.contains("Invalid number of arguments for function foo. Expected:")) + assert(e.getMessage.contains("Invalid number of arguments. Expected:")) } test("error reporting for undefined functions") { @@ -618,7 +618,7 @@ class UDFSuite extends QueryTest with SharedSparkSession { val e = intercept[AnalysisException] { spark.sql("SELECT CAST(1)") } - assert(e.getMessage.contains("Invalid number of arguments for function cast")) + assert(e.getMessage.contains("Empty number of arguments")) } test("only one case class parameter") { 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 5e2dbdaa8dbbf..be90c410023bf 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 @@ -108,16 +108,20 @@ class QueryCompilationErrorsSuite } } - test("INVALID_PARAMETER_VALUE: the argument_index of string format is invalid") { + test("INVALID_FUNCTION_ARGUMENTS: the argument_index of string format is invalid") { withSQLConf(SQLConf.ALLOW_ZERO_INDEX_IN_FORMAT_STRING.key -> "false") { val e = intercept[AnalysisException] { sql("select format_string('%0$s', 'Hello')") } checkErrorClass( exception = e, - errorClass = "INVALID_PARAMETER_VALUE", - msg = "The value of parameter(s) 'strfmt' in `format_string` is invalid: " + - "expects %1$, %2$ and so on, but got %0$.; line 1 pos 7") + errorClass = "INVALID_FUNCTION_ARGUMENTS", + errorSubClass = Some("INVALID_ARGUMENT_INDEX"), + msg = "Arguments of the `format_string` function are invalid: " + + "The value of parameter(s) 'strfmt' has invalid index, " + + "expects %1$, %2$ and so on, but got %0$.; line 1 pos 7", + sqlState = Some("22023") + ) } } @@ -556,8 +560,8 @@ class QueryCompilationErrorsSuite ), errorClass = "INVALID_FUNCTION_ARGUMENTS", errorSubClass = Some("INVALID_NUMBER_OF_ARGUMENTS"), - msg = "The function arguments invalid: " + - "Invalid number of arguments for function testFunc. Expected: 1; Found: 2; line 1 pos 7", + msg = "Arguments of the testFunc function are invalid: " + + "Invalid number of arguments. Expected: 1; Found: 2; line 1 pos 7", sqlState = Some("22023") ) } @@ -569,8 +573,8 @@ class QueryCompilationErrorsSuite ), errorClass = "INVALID_FUNCTION_ARGUMENTS", errorSubClass = Some("EMPTY_NUMBER_OF_ARGUMENTS"), - msg = "The function arguments invalid: " + - "Invalid number of arguments for function cast; line 1 pos 7", + msg = "Arguments of the cast function are invalid: " + + "Empty number of arguments; line 1 pos 7", sqlState = Some("22023") ) } @@ -582,8 +586,8 @@ class QueryCompilationErrorsSuite ), errorClass = "INVALID_FUNCTION_ARGUMENTS", errorSubClass = Some("INVALID_NUMBER_OF_ARGUMENTS"), - msg = "The function arguments invalid: " + - "Invalid number of arguments for function to_timestamp_ntz. " + + msg = "Arguments of the to_timestamp_ntz function are invalid: " + + "Invalid number of arguments. " + "Expected: one of 1 and 2; Found: 0; line 1 pos 7", sqlState = Some("22023") ) @@ -596,8 +600,8 @@ class QueryCompilationErrorsSuite ), errorClass = "INVALID_FUNCTION_ARGUMENTS", errorSubClass = Some("CAST_ALIAS"), - msg = "The function arguments invalid: " + - "Function int accepts only one argument; line 1 pos 7", + msg = "Arguments of the int function are invalid: " + + "Function accepts only one argument; line 1 pos 7", sqlState = Some("22023") ) } @@ -609,8 +613,8 @@ class QueryCompilationErrorsSuite ), errorClass = "INVALID_FUNCTION_ARGUMENTS", errorSubClass = Some("APPROX_COUNT_DISTINCT"), - msg = "The function arguments invalid: " + - "The second argument in approx_count_distinct should be a double literal; line 1 pos 7", + msg = "Arguments of the approx_count_distinct function are invalid: " + + "The second argument should be a double literal; line 1 pos 7", sqlState = Some("22023") ) } @@ -627,8 +631,8 @@ class QueryCompilationErrorsSuite ), errorClass = "INVALID_FUNCTION_ARGUMENTS", errorSubClass = Some("INVALID_OPERATION_FOR_V2FUNCTION"), - msg = "The function arguments invalid: " + - "V2Function strlen cannot process input: (\"MAP\"): Expect StringType, " + + msg = "Arguments of the strlen function are invalid: " + + "V2Function cannot process input: (\"MAP\"): Expect StringType, " + "but found MapType(StringType,StringType,false); line 1 pos 7", sqlState = Some("22023") ) @@ -648,8 +652,8 @@ class QueryCompilationErrorsSuite ), errorClass = "INVALID_FUNCTION_ARGUMENTS", errorSubClass = Some("INVALID_NUMBER_OF_ARGUMENTS_FOR_V2FUNCTION"), - msg = "The function arguments invalid: " + - "There are 2 arguments in V2Function strlen, " + + msg = "Arguments of the strlen function are invalid: " + + "There are 2 arguments in V2Function, " + "but 1 parameters returned from 'inputTypes()'; line 1 pos 7", sqlState = Some("22023") ) @@ -664,8 +668,8 @@ class QueryCompilationErrorsSuite ), errorClass = "INVALID_FUNCTION_ARGUMENTS", errorSubClass = Some("FIRST_LAST"), - msg = "The function arguments invalid: " + - "The second argument in first should be a boolean literal; line 1 pos 7", + msg = "Arguments of the first function are invalid: " + + "The second argument should be a boolean literal; line 1 pos 7", sqlState = Some("22023") ) } 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 7a5592c148a90..0d6f8d54ed648 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 @@ -64,16 +64,18 @@ class QueryExecutionErrorsSuite (df1, df2) } - test("INVALID_PARAMETER_VALUE: invalid key lengths in AES functions") { + test("INVALID_FUNCTION_ARGUMENTS: invalid key lengths in AES functions") { val (df1, df2) = getAesInputs() def checkInvalidKeyLength(df: => DataFrame): Unit = { checkErrorClass( exception = intercept[SparkException] { df.collect }.getCause.asInstanceOf[SparkRuntimeException], - errorClass = "INVALID_PARAMETER_VALUE", - msg = "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.", + errorClass = "INVALID_FUNCTION_ARGUMENTS", + errorSubClass = Some("INVALID_ARGUMENT_LENGTH"), + msg = "Arguments of the `aes_encrypt`\\/`aes_decrypt` function are invalid: " + + "The value of parameter\\(s\\) 'key' has invalid length, " + + "expects a binary value with 16, 24 or 32 bytes, but got \\d+ bytes.", sqlState = Some("22023"), matchMsg = true) } @@ -106,10 +108,12 @@ class QueryExecutionErrorsSuite exception = intercept[SparkException] { df2.selectExpr(s"aes_decrypt(unbase64($colName), binary('$key'), 'ECB')").collect }.getCause.asInstanceOf[SparkRuntimeException], - errorClass = "INVALID_PARAMETER_VALUE", + errorClass = "INVALID_FUNCTION_ARGUMENTS", + errorSubClass = Some("INVALID_ARGUMENT_VALUE"), msg = - "The value of parameter(s) 'expr, key' in the `aes_encrypt`/`aes_decrypt` function " + - "is invalid: Detail message: " + + "Arguments of the `aes_encrypt`/`aes_decrypt` function are invalid: " + + "The value of parameter(s) 'expr, key' is invalid, " + + "Detail message: " + "Given final block not properly padded. " + "Such issues can arise if a bad key is used during decryption.", sqlState = Some("22023")) From c522bb821e41979014d6d269e4395825715f7620 Mon Sep 17 00:00:00 2001 From: lvshaokang Date: Thu, 18 Aug 2022 10:07:07 +0800 Subject: [PATCH 10/13] fix(): format message --- .../main/resources/error/error-classes.json | 4 +- .../apache/spark/sql/AnalysisException.scala | 12 +++ .../sql/errors/QueryCompilationErrors.scala | 28 ++++--- .../sql/errors/QueryExecutionErrors.scala | 7 +- .../expressions/RegexpExpressionsSuite.scala | 10 ++- .../results/regexp-functions.sql.out | 2 +- .../errors/QueryCompilationErrorsSuite.scala | 76 +++++++++++-------- .../errors/QueryExecutionErrorsSuite.scala | 15 ++-- 8 files changed, 94 insertions(+), 60 deletions(-) diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json index f622bd4047728..655ef3b9c86ca 100644 --- a/core/src/main/resources/error/error-classes.json +++ b/core/src/main/resources/error/error-classes.json @@ -256,7 +256,7 @@ "sqlState" : "22023" }, "INVALID_FUNCTION_ARGUMENTS" : { - "message" : [ "Arguments of the function are invalid: " ], + "message" : [ "Arguments of the function are invalid:" ], "subClass" : { "APPROX_COUNT_DISTINCT" : { "message" : [ "The second argument should be a double literal" ] @@ -277,7 +277,7 @@ "message" : [ "The value of parameter(s) '' has invalid length, expects , but got ." ] }, "INVALID_ARGUMENT_VALUE" : { - "message" : [ "The value of parameter(s) '' is invalid, " ] + "message" : [ "The value of parameter(s) '' is invalid: " ] }, "INVALID_NUMBER_OF_ARGUMENTS" : { "message" : [ "Invalid number of arguments. Expected: ; Found: " ] diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/AnalysisException.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/AnalysisException.scala index 9ab0b223e1172..df3b5a93eaf0e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/AnalysisException.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/AnalysisException.scala @@ -90,6 +90,18 @@ class AnalysisException protected[sql] ( errorSubClass = Some(errorSubClass), messageParameters = messageParameters) + def this( + errorClass: String, + errorSubClass: String, + messageParameters: Array[String], + cause: Option[Throwable]) = + this( + SparkThrowableHelper.getMessage(errorClass, errorSubClass, messageParameters), + errorClass = Some(errorClass), + errorSubClass = Some(errorSubClass), + messageParameters = messageParameters, + cause = cause) + def this( errorClass: String, errorSubClass: String, diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala index 8fcc05c537ce5..aeb07327ea1af 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala @@ -70,7 +70,8 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase { def zeroArgumentIndexError(): Throwable = { new AnalysisException( errorClass = "INVALID_FUNCTION_ARGUMENTS", - messageParameters = Array("INVALID_ARGUMENT_INDEX", + errorSubClass = "INVALID_ARGUMENT_INDEX", + messageParameters = Array( toSQLId("format_string"), "strfmt", "%1$, %2$ and so on", "%0$")) } @@ -498,8 +499,8 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase { name: String, expectedInfo: String, actualNumber: Int): Throwable = { new AnalysisException( errorClass = "INVALID_FUNCTION_ARGUMENTS", - messageParameters = Array("INVALID_NUMBER_OF_ARGUMENTS", - name, expectedInfo, s"$actualNumber")) + errorSubClass = "INVALID_NUMBER_OF_ARGUMENTS", + messageParameters = Array(name, expectedInfo, s"$actualNumber")) } def invalidFunctionArgumentNumberError( @@ -507,7 +508,8 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase { if (validParametersCount.isEmpty) { new AnalysisException( errorClass = "INVALID_FUNCTION_ARGUMENTS", - messageParameters = Array("EMPTY_NUMBER_OF_ARGUMENTS", name)) + errorSubClass = "EMPTY_NUMBER_OF_ARGUMENTS", + messageParameters = Array(name)) } else { val expectedNumberOfParameters = if (validParametersCount.length == 1) { validParametersCount.head.toString @@ -522,7 +524,8 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase { def functionAcceptsOnlyOneArgumentError(name: String): Throwable = { new AnalysisException( errorClass = "INVALID_FUNCTION_ARGUMENTS", - messageParameters = Array("CAST_ALIAS", name)) + errorSubClass = "CAST_ALIAS", + messageParameters = Array(name)) } def alterV2TableSetLocationWithPartitionNotSupportedError(): Throwable = { @@ -828,7 +831,8 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase { def secondArgumentNotDoubleLiteralError(name: String): Throwable = { new AnalysisException( errorClass = "INVALID_FUNCTION_ARGUMENTS", - messageParameters = Array("APPROX_COUNT_DISTINCT", name) + errorSubClass = "APPROX_COUNT_DISTINCT", + messageParameters = Array(name) ) } @@ -1551,8 +1555,10 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase { unsupported: UnsupportedOperationException): Throwable = { new AnalysisException( errorClass = "INVALID_FUNCTION_ARGUMENTS", - messageParameters = Array("INVALID_OPERATION_FOR_V2FUNCTION", - unbound.name, arguments.map(x => toSQLType(x.dataType)).mkString(", "), + errorSubClass = "INVALID_OPERATION_FOR_V2FUNCTION", + messageParameters = Array( + unbound.name, + arguments.map(x => toSQLType(x.dataType)).mkString(", "), unsupported.getMessage), cause = Some(unsupported)) } @@ -1562,7 +1568,8 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase { args: Seq[Expression]): Throwable = { new AnalysisException( errorClass = "INVALID_FUNCTION_ARGUMENTS", - messageParameters = Array("INVALID_NUMBER_OF_ARGUMENTS_FOR_V2FUNCTION", + errorSubClass = "INVALID_NUMBER_OF_ARGUMENTS_FOR_V2FUNCTION", + messageParameters = Array( bound.name(), s"${args.length}", s"${bound.inputTypes().length}") ) @@ -1647,7 +1654,8 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase { def secondArgumentInFunctionIsNotBooleanLiteralError(funcName: String): Throwable = { new AnalysisException( errorClass = "INVALID_FUNCTION_ARGUMENTS", - messageParameters = Array("FIRST_LAST", funcName) + errorSubClass = "FIRST_LAST", + messageParameters = Array(funcName) ) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala index ed85ecdde5f2a..f1aa9f4a8f3ee 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala @@ -1987,7 +1987,7 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase { aesFuncName, "key", "a binary value with 16, 24 or 32 bytes", - s"${actualLength.toString} bytes.")) + s"${actualLength.toString} bytes")) } def aesModeUnsupportedError(mode: String, padding: String): RuntimeException = { @@ -2086,10 +2086,11 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase { def invalidPatternError(funcName: String, pattern: String): RuntimeException = { new SparkRuntimeException( - errorClass = "INVALID_PARAMETER_VALUE", + errorClass = "INVALID_FUNCTION_ARGUMENTS", + errorSubClass = "INVALID_ARGUMENT_VALUE", messageParameters = Array( - "regexp", toSQLId(funcName), + "regexp", pattern)) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/RegexpExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/RegexpExpressionsSuite.scala index 91b0f0e1039f6..a40674f6a6bf3 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/RegexpExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/RegexpExpressionsSuite.scala @@ -502,18 +502,20 @@ class RegexpExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { val s = $"s".string.at(0) val p = $"p".string.at(1) val r = $"r".int.at(2) - val prefix = "[INVALID_PARAMETER_VALUE] The value of parameter(s) 'regexp' in" + val prefix = "[INVALID_FUNCTION_ARGUMENTS.INVALID_ARGUMENT_VALUE] " + + "Arguments of the %s function are invalid: " + + "The value of parameter(s) 'regexp'" checkExceptionInExpression[SparkRuntimeException]( RegExpExtract(s, p, r), create_row("1a 2b 14m", "(?l)", 0), - s"$prefix `regexp_extract` is invalid: (?l)") + s"${prefix.format("`regexp_extract`")} is invalid: (?l)") checkExceptionInExpression[SparkRuntimeException]( RegExpExtractAll(s, p, r), create_row("abc", "] [", 0), - s"$prefix `regexp_extract_all` is invalid: ] [") + s"${prefix.format("`regexp_extract_all`")} is invalid: ] [") checkExceptionInExpression[SparkRuntimeException]( RegExpInStr(s, p, r), create_row("abc", ", (", 0), - s"$prefix `regexp_instr` is invalid: , (") + s"${prefix.format("`regexp_instr`")} is invalid: , (") } } diff --git a/sql/core/src/test/resources/sql-tests/results/regexp-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/regexp-functions.sql.out index 6a07df854beb5..2064fd51497eb 100644 --- a/sql/core/src/test/resources/sql-tests/results/regexp-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/regexp-functions.sql.out @@ -131,7 +131,7 @@ SELECT regexp_extract('1a 2b 14m', '(?l)') struct<> -- !query output org.apache.spark.SparkRuntimeException -[INVALID_PARAMETER_VALUE] The value of parameter(s) 'regexp' in `regexp_extract` is invalid: (?l) +[INVALID_FUNCTION_ARGUMENTS.INVALID_ARGUMENT_VALUE] Arguments of the `regexp_extract` function are invalid: The value of parameter(s) 'regexp' is invalid: (?l) -- !query 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 d53c9dbf5ba5b..48b52067276f8 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 @@ -120,13 +120,10 @@ class QueryCompilationErrorsSuite errorClass = "INVALID_FUNCTION_ARGUMENTS", errorSubClass = Some("INVALID_ARGUMENT_INDEX"), parameters = Map( + "funcName" -> "`format_string`", "parameter" -> "strfmt", - "functionName" -> "`format_string`", - "expected" -> "expects %1$, %2$ and so on, but got %0$.")) -// msg = "Arguments of the `format_string` function are invalid: " + -// "The value of parameter(s) 'strfmt' has invalid index, " + -// "expects %1$, %2$ and so on, but got %0$.; line 1 pos 7" - , + "expected" -> "%1$, %2$ and so on", + "found" -> "%0$")) } } @@ -626,67 +623,75 @@ class QueryCompilationErrorsSuite test("INVALID_FUNCTION_ARGUMENTS: invalid number of the udf function arguments") { spark.udf.register("testFunc", (n: Int) => n.toString) - checkErrorClass( + checkError( exception = intercept[AnalysisException]( sql(s"SELECT testFunc(123, 123) as value") ), errorClass = "INVALID_FUNCTION_ARGUMENTS", errorSubClass = Some("INVALID_NUMBER_OF_ARGUMENTS"), - msg = "Arguments of the testFunc function are invalid: " + - "Invalid number of arguments. Expected: 1; Found: 2; line 1 pos 7", + parameters = Map( + "funcName" -> "testFunc", + "expected" -> "1", + "found" -> "2" + ), sqlState = Some("22023") ) } test("INVALID_FUNCTION_ARGUMENTS: empty number of the function arguments") { - checkErrorClass( + checkError( exception = intercept[AnalysisException]( sql("SELECT CAST()") ), errorClass = "INVALID_FUNCTION_ARGUMENTS", errorSubClass = Some("EMPTY_NUMBER_OF_ARGUMENTS"), - msg = "Arguments of the cast function are invalid: " + - "Empty number of arguments; line 1 pos 7", + parameters = Map( + "funcName" -> "cast" + ), sqlState = Some("22023") ) } test("INVALID_FUNCTION_ARGUMENTS: invalid number of the function arguments") { - checkErrorClass( + checkError( exception = intercept[AnalysisException]( sql("SELECT to_timestamp_ntz()") ), errorClass = "INVALID_FUNCTION_ARGUMENTS", errorSubClass = Some("INVALID_NUMBER_OF_ARGUMENTS"), - msg = "Arguments of the to_timestamp_ntz function are invalid: " + - "Invalid number of arguments. " + - "Expected: one of 1 and 2; Found: 0; line 1 pos 7", + parameters = Map( + "funcName" -> "to_timestamp_ntz", + "expected" -> "one of 1 and 2", + "found" -> "0" + ), sqlState = Some("22023") ) } test("INVALID_FUNCTION_ARGUMENTS: the function use two arguments") { - checkErrorClass( + checkError( exception = intercept[AnalysisException]( sql(s"SELECT int('1', '2')") ), errorClass = "INVALID_FUNCTION_ARGUMENTS", errorSubClass = Some("CAST_ALIAS"), - msg = "Arguments of the int function are invalid: " + - "Function accepts only one argument; line 1 pos 7", + parameters = Map( + "funcName" -> "int" + ), sqlState = Some("22023") ) } test("INVALID_FUNCTION_ARGUMENTS: invalid the second argument of the int type") { - checkErrorClass( + checkError( exception = intercept[AnalysisException]( sql(s"SELECT approx_count_distinct(1,1)") ), errorClass = "INVALID_FUNCTION_ARGUMENTS", errorSubClass = Some("APPROX_COUNT_DISTINCT"), - msg = "Arguments of the approx_count_distinct function are invalid: " + - "The second argument should be a double literal; line 1 pos 7", + parameters = Map( + "funcName" -> "approx_count_distinct" + ), sqlState = Some("22023") ) } @@ -697,15 +702,17 @@ class QueryCompilationErrorsSuite spark.sessionState.catalogManager.catalog("testcat") .asInstanceOf[InMemoryCatalog] .createFunction(Identifier.of(Array("ns"), "strlen"), new StrLen) - checkErrorClass( + checkError( exception = intercept[AnalysisException]( sql("SELECT testcat.ns.strlen(map('abc', 'abc'))") ), errorClass = "INVALID_FUNCTION_ARGUMENTS", errorSubClass = Some("INVALID_OPERATION_FOR_V2FUNCTION"), - msg = "Arguments of the strlen function are invalid: " + - "V2Function cannot process input: (\"MAP\"): Expect StringType, " + - "but found MapType(StringType,StringType,false); line 1 pos 7", + parameters = Map( + "funcName" -> "strlen", + "type" -> "\"MAP\"", + "message" -> "Expect StringType, but found MapType(StringType,StringType,false)" + ), sqlState = Some("22023") ) spark.sessionState.conf.unsetConf("spark.sql.catalog.testcat") @@ -718,15 +725,17 @@ class QueryCompilationErrorsSuite spark.sessionState.catalogManager.catalog("testcat") .asInstanceOf[InMemoryCatalog] .createFunction(Identifier.of(Array("ns"), "strlen"), new StrLen) - checkErrorClass( + checkError( exception = intercept[AnalysisException]( sql("SELECT testcat.ns.strlen('abc','abc')") ), errorClass = "INVALID_FUNCTION_ARGUMENTS", errorSubClass = Some("INVALID_NUMBER_OF_ARGUMENTS_FOR_V2FUNCTION"), - msg = "Arguments of the strlen function are invalid: " + - "There are 2 arguments in V2Function, " + - "but 1 parameters returned from 'inputTypes()'; line 1 pos 7", + parameters = Map( + "funcName" -> "strlen", + "expected" -> "2", + "found" -> "1" + ), sqlState = Some("22023") ) spark.sessionState.conf.unsetConf("spark.sql.catalog.testcat") @@ -734,14 +743,15 @@ class QueryCompilationErrorsSuite } test("INVALID_FUNCTION_ARGUMENTS: invalid the second argument of the string type") { - checkErrorClass( + checkError( exception = intercept[AnalysisException]( sql(s"SELECT first(1, '1')") ), errorClass = "INVALID_FUNCTION_ARGUMENTS", errorSubClass = Some("FIRST_LAST"), - msg = "Arguments of the first function are invalid: " + - "The second argument should be a boolean literal; line 1 pos 7", + parameters = Map( + "funcName" -> "first" + ), sqlState = Some("22023") ) } 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 4f0b5397c1c28..f65245022effa 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 @@ -76,10 +76,10 @@ class QueryExecutionErrorsSuite }.getCause.asInstanceOf[SparkRuntimeException], errorClass = "INVALID_FUNCTION_ARGUMENTS", errorSubClass = "INVALID_ARGUMENT_LENGTH", - parameters = Map("parameter" -> "key", - "functionName" -> "`aes_encrypt`/`aes_decrypt`", - "expected" -> ("expects a binary value with 16, 24 or 32 bytes, but got " + - inputBytes.toString + " bytes.")), + parameters = Map("funcName" -> "`aes_encrypt`/`aes_decrypt`", + "parameter" -> "key", + "expected" -> "a binary value with 16, 24 or 32 bytes", + "found" -> s"${inputBytes.toString} bytes"), sqlState = "22023") } @@ -114,9 +114,10 @@ class QueryExecutionErrorsSuite }.getCause.asInstanceOf[SparkRuntimeException], errorClass = "INVALID_FUNCTION_ARGUMENTS", errorSubClass = "INVALID_ARGUMENT_VALUE", - parameters = Map("parameter" -> "expr, key", - "functionName" -> "`aes_encrypt`/`aes_decrypt`", - "expected" -> ("Detail message: " + + parameters = Map( + "funcName" -> "`aes_encrypt`/`aes_decrypt`", + "parameter" -> "expr, key", + "message" -> ("Detail message: " + "Given final block not properly padded. " + "Such issues can arise if a bad key is used during decryption.")), sqlState = "22023") From 82a6fd7a9d1618f8984c8d50ba00362c81d5eb64 Mon Sep 17 00:00:00 2001 From: lvshaokang Date: Thu, 18 Aug 2022 15:19:03 +0800 Subject: [PATCH 11/13] fix(): format message --- .../main/resources/error/error-classes.json | 44 ++++++++++++++----- .../spark/sql/DataFrameFunctionsSuite.scala | 2 +- .../spark/sql/StringFunctionsSuite.scala | 2 +- .../sql/hive/execution/HiveUDAFSuite.scala | 2 +- 4 files changed, 36 insertions(+), 14 deletions(-) diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json index 7b95b3a5de867..07b30c0b49211 100644 --- a/core/src/main/resources/error/error-classes.json +++ b/core/src/main/resources/error/error-classes.json @@ -256,37 +256,59 @@ "sqlState" : "22023" }, "INVALID_FUNCTION_ARGUMENTS" : { - "message" : [ "Arguments of the function are invalid:" ], + "message" : [ + "Arguments of the function are invalid:" + ], "subClass" : { "APPROX_COUNT_DISTINCT" : { - "message" : [ "The second argument should be a double literal" ] + "message" : [ + "The second argument should be a double literal" + ] }, "CAST_ALIAS" : { - "message" : [ "Function accepts only one argument" ] + "message" : [ + "Function accepts only one argument" + ] }, "EMPTY_NUMBER_OF_ARGUMENTS" : { - "message" : [ "Empty number of arguments" ] + "message" : [ + "Empty number of arguments" + ] }, "FIRST_LAST" : { - "message" : [ "The second argument should be a boolean literal" ] + "message" : [ + "The second argument should be a boolean literal" + ] }, "INVALID_ARGUMENT_INDEX" : { - "message" : [ "The value of parameter(s) '' has invalid index, expects , but got ." ] + "message" : [ + "The value of parameter(s) '' has invalid index, expects , but got ." + ] }, "INVALID_ARGUMENT_LENGTH" : { - "message" : [ "The value of parameter(s) '' has invalid length, expects , but got ." ] + "message" : [ + "The value of parameter(s) '' has invalid length, expects , but got ." + ] }, "INVALID_ARGUMENT_VALUE" : { - "message" : [ "The value of parameter(s) '' is invalid: " ] + "message" : [ + "The value of parameter(s) '' is invalid: " + ] }, "INVALID_NUMBER_OF_ARGUMENTS" : { - "message" : [ "Invalid number of arguments. Expected: ; Found: " ] + "message" : [ + "Invalid number of arguments. Expected: ; Found: " + ] }, "INVALID_NUMBER_OF_ARGUMENTS_FOR_V2FUNCTION" : { - "message" : [ "There are arguments in V2Function, but parameters returned from 'inputTypes()'" ] + "message" : [ + "There are arguments in V2Function, but parameters returned from 'inputTypes()'" + ] }, "INVALID_OPERATION_FOR_V2FUNCTION" : { - "message" : [ "V2Function cannot process input: (): " ] + "message" : [ + "V2Function cannot process input: (): " + ] } }, "sqlState" : "22023" diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala index 9924fbfbf626c..c08ce7fb32377 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala @@ -3675,7 +3675,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { 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")) + assert(ex2.getMessage.contains("Invalid number of arguments")) val ex3 = intercept[AnalysisException] { df.selectExpr("zip_with(i, a2, (acc, x) -> x)") } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/StringFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/StringFunctionsSuite.scala index d07be9c19714e..d0638db04a6c4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/StringFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/StringFunctionsSuite.scala @@ -582,7 +582,7 @@ class StringFunctionsSuite extends QueryTest with SharedSparkSession { val m = intercept[AnalysisException] { df.selectExpr("sentences()") }.getMessage - assert(m.contains("Invalid number of arguments for function sentences")) + assert(m.contains("Invalid number of arguments")) } test("str_to_map function") { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDAFSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDAFSuite.scala index 3fcc8612b89ee..08e61d5122261 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDAFSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDAFSuite.scala @@ -174,7 +174,7 @@ class HiveUDAFSuite extends QueryTest sql(s"SELECT $functionName(100)") }.getMessage assert(e.contains( - s"Invalid number of arguments for function $functionName. Expected: 2; Found: 1;")) + s"Invalid number of arguments. Expected: 2; Found: 1;")) } } } From acad3220cdc3d36fb027d4e4d6b2a5679f4de0b3 Mon Sep 17 00:00:00 2001 From: lvshaokang Date: Thu, 18 Aug 2022 18:01:49 +0800 Subject: [PATCH 12/13] fix(): format message --- .../test/resources/sql-tests/results/regexp-functions.sql.out | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sql/core/src/test/resources/sql-tests/results/regexp-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/regexp-functions.sql.out index 2064fd51497eb..4ea8c980fc32c 100644 --- a/sql/core/src/test/resources/sql-tests/results/regexp-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/regexp-functions.sql.out @@ -266,7 +266,7 @@ SELECT regexp_extract_all('abc', col0, 1) FROM VALUES('], [') AS t(col0) struct<> -- !query output org.apache.spark.SparkRuntimeException -[INVALID_PARAMETER_VALUE] The value of parameter(s) 'regexp' in `regexp_extract_all` is invalid: ], [ +[INVALID_FUNCTION_ARGUMENTS.INVALID_ARGUMENT_VALUE] Arguments of the `regexp_extract_all` function are invalid: The value of parameter(s) 'regexp' is invalid: ], [ -- !query @@ -565,4 +565,4 @@ SELECT regexp_instr('abc', col0, 1) FROM VALUES(') ?') AS t(col0) struct<> -- !query output org.apache.spark.SparkRuntimeException -[INVALID_PARAMETER_VALUE] The value of parameter(s) 'regexp' in `regexp_instr` is invalid: ) ? +[INVALID_FUNCTION_ARGUMENTS.INVALID_ARGUMENT_VALUE] Arguments of the `regexp_instr` function are invalid: The value of parameter(s) 'regexp' is invalid: ) ? From 43ac9a0f5a2a4e9e48c5d45e9a86840ba4eeba5d Mon Sep 17 00:00:00 2001 From: lvshaokang Date: Sat, 20 Aug 2022 01:59:45 +0800 Subject: [PATCH 13/13] format message with json --- .../results/ansi/string-functions.sql.out | 24 ++++++++-- .../ceil-floor-with-scale-param.sql.out | 22 ++++++++- .../sql-tests/results/csv-functions.sql.out | 12 ++++- .../sql-tests/results/json-functions.sql.out | 48 +++++++++++++++---- .../sql-tests/results/postgreSQL/text.sql.out | 9 ++-- .../results/regexp-functions.sql.out | 24 +++++----- .../sql-compatibility-functions.sql.out | 9 +++- .../results/string-functions.sql.out | 23 +++++++-- .../sql-tests/results/timestamp-ntz.sql.out | 12 ++++- .../resources/sql-tests/results/udaf.sql.out | 11 ++++- .../sql-tests/results/udf/udf-udaf.sql.out | 12 ++++- 11 files changed, 165 insertions(+), 41 deletions(-) diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/string-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/string-functions.sql.out index d0f53a77981f5..5bec56af82b67 100644 --- a/sql/core/src/test/resources/sql-tests/results/ansi/string-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/ansi/string-functions.sql.out @@ -719,8 +719,16 @@ select decode() struct<> -- !query output org.apache.spark.sql.AnalysisException -[INVALID_FUNCTION_ARGUMENTS.INVALID_NUMBER_OF_ARGUMENTS] Arguments of the decode function are invalid: Invalid number of arguments. Expected: 2; Found: 0; line 1 pos 7 - +{ + "errorClass" : "INVALID_FUNCTION_ARGUMENTS", + "errorSubClass" : "INVALID_NUMBER_OF_ARGUMENTS", + "sqlState" : "22023", + "messageParameters" : { + "funcName" : "decode", + "expected" : "2", + "found" : "0" + } +} -- !query select decode(encode('abc', 'utf-8')) @@ -728,8 +736,16 @@ select decode(encode('abc', 'utf-8')) struct<> -- !query output org.apache.spark.sql.AnalysisException -[INVALID_FUNCTION_ARGUMENTS.INVALID_NUMBER_OF_ARGUMENTS] Arguments of the decode function are invalid: Invalid number of arguments. Expected: 2; Found: 1; line 1 pos 7 - +{ + "errorClass" : "INVALID_FUNCTION_ARGUMENTS", + "errorSubClass" : "INVALID_NUMBER_OF_ARGUMENTS", + "sqlState" : "22023", + "messageParameters" : { + "funcName" : "decode", + "expected" : "2", + "found" : "1" + } +} -- !query select decode(encode('abc', 'utf-8'), 'utf-8') diff --git a/sql/core/src/test/resources/sql-tests/results/ceil-floor-with-scale-param.sql.out b/sql/core/src/test/resources/sql-tests/results/ceil-floor-with-scale-param.sql.out index 32885eb941048..e5b2d62cbd2ff 100644 --- a/sql/core/src/test/resources/sql-tests/results/ceil-floor-with-scale-param.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/ceil-floor-with-scale-param.sql.out @@ -111,7 +111,16 @@ SELECT CEIL(2.5, 0, 0) struct<> -- !query output org.apache.spark.sql.AnalysisException -[INVALID_FUNCTION_ARGUMENTS.INVALID_NUMBER_OF_ARGUMENTS] Arguments of the ceil function are invalid: Invalid number of arguments. Expected: 2; Found: 3; line 1 pos 7 +{ + "errorClass" : "INVALID_FUNCTION_ARGUMENTS", + "errorSubClass" : "INVALID_NUMBER_OF_ARGUMENTS", + "sqlState" : "22023", + "messageParameters" : { + "funcName" : "ceil", + "expected" : "2", + "found" : "3" + } +} -- !query @@ -226,4 +235,13 @@ SELECT FLOOR(2.5, 0, 0) struct<> -- !query output org.apache.spark.sql.AnalysisException -[INVALID_FUNCTION_ARGUMENTS.INVALID_NUMBER_OF_ARGUMENTS] Arguments of the floor function are invalid: Invalid number of arguments. Expected: 2; Found: 3; line 1 pos 7 +{ + "errorClass" : "INVALID_FUNCTION_ARGUMENTS", + "errorSubClass" : "INVALID_NUMBER_OF_ARGUMENTS", + "sqlState" : "22023", + "messageParameters" : { + "funcName" : "floor", + "expected" : "2", + "found" : "3" + } +} diff --git a/sql/core/src/test/resources/sql-tests/results/csv-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/csv-functions.sql.out index d8b1b4f250414..030405a08f472 100644 --- a/sql/core/src/test/resources/sql-tests/results/csv-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/csv-functions.sql.out @@ -70,8 +70,16 @@ select from_csv() struct<> -- !query output org.apache.spark.sql.AnalysisException -[INVALID_FUNCTION_ARGUMENTS.INVALID_NUMBER_OF_ARGUMENTS] Arguments of the from_csv function are invalid: Invalid number of arguments. Expected: one of 2 and 3; Found: 0; line 1 pos 7 - +{ + "errorClass" : "INVALID_FUNCTION_ARGUMENTS", + "errorSubClass" : "INVALID_NUMBER_OF_ARGUMENTS", + "sqlState" : "22023", + "messageParameters" : { + "funcName" : "from_csv", + "expected" : "one of 2 and 3", + "found" : "0" + } +} -- !query select from_csv('1,abc', schema_of_csv('1,abc')) diff --git a/sql/core/src/test/resources/sql-tests/results/json-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/json-functions.sql.out index a17cf6aa79b90..966e6e9c97420 100644 --- a/sql/core/src/test/resources/sql-tests/results/json-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/json-functions.sql.out @@ -87,8 +87,16 @@ select to_json() struct<> -- !query output org.apache.spark.sql.AnalysisException -[INVALID_FUNCTION_ARGUMENTS.INVALID_NUMBER_OF_ARGUMENTS] Arguments of the to_json function are invalid: Invalid number of arguments. Expected: one of 1 and 2; Found: 0; line 1 pos 7 - +{ + "errorClass" : "INVALID_FUNCTION_ARGUMENTS", + "errorSubClass" : "INVALID_NUMBER_OF_ARGUMENTS", + "sqlState" : "22023", + "messageParameters" : { + "funcName" : "to_json", + "expected" : "one of 1 and 2", + "found" : "0" + } +} -- !query select from_json('{"a":1}', 'a INT') @@ -161,8 +169,16 @@ select from_json() struct<> -- !query output org.apache.spark.sql.AnalysisException -[INVALID_FUNCTION_ARGUMENTS.INVALID_NUMBER_OF_ARGUMENTS] Arguments of the from_json function are invalid: Invalid number of arguments. Expected: one of 2 and 3; Found: 0; line 1 pos 7 - +{ + "errorClass" : "INVALID_FUNCTION_ARGUMENTS", + "errorSubClass" : "INVALID_NUMBER_OF_ARGUMENTS", + "sqlState" : "22023", + "messageParameters" : { + "funcName" : "from_json", + "expected" : "one of 2 and 3", + "found" : "0" + } +} -- !query SELECT json_tuple('{"a" : 1, "b" : 2}', CAST(NULL AS STRING), 'b', CAST(NULL AS STRING), 'a') @@ -446,8 +462,16 @@ select json_array_length() struct<> -- !query output org.apache.spark.sql.AnalysisException -[INVALID_FUNCTION_ARGUMENTS.INVALID_NUMBER_OF_ARGUMENTS] Arguments of the json_array_length function are invalid: Invalid number of arguments. Expected: 1; Found: 0; line 1 pos 7 - +{ + "errorClass" : "INVALID_FUNCTION_ARGUMENTS", + "errorSubClass" : "INVALID_NUMBER_OF_ARGUMENTS", + "sqlState" : "22023", + "messageParameters" : { + "funcName" : "json_array_length", + "expected" : "1", + "found" : "0" + } +} -- !query select json_array_length('') @@ -519,8 +543,16 @@ select json_object_keys() struct<> -- !query output org.apache.spark.sql.AnalysisException -[INVALID_FUNCTION_ARGUMENTS.INVALID_NUMBER_OF_ARGUMENTS] Arguments of the json_object_keys function are invalid: Invalid number of arguments. Expected: 1; Found: 0; line 1 pos 7 - +{ + "errorClass" : "INVALID_FUNCTION_ARGUMENTS", + "errorSubClass" : "INVALID_NUMBER_OF_ARGUMENTS", + "sqlState" : "22023", + "messageParameters" : { + "funcName" : "json_object_keys", + "expected" : "1", + "found" : "0" + } +} -- !query select json_object_keys(null) diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/text.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/text.sql.out index a00c203ed6c44..b9a354bc4ddc6 100755 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/text.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/text.sql.out @@ -308,14 +308,15 @@ select format_string('%0$s', 'Hello') struct<> -- !query output org.apache.spark.sql.AnalysisException -#[INVALID_FUNCTION_ARGUMENTS.INVALID_ARGUMENT_INDEX] Arguments of the `format_string` function are invalid: The value of parameter(s) 'strfmt' has invalid index, expects %1$, %2$ and so on, but got %0$.; line 1 pos 7 { - "errorClass" : "INVALID_PARAMETER_VALUE", + "errorClass" : "INVALID_FUNCTION_ARGUMENTS", + "errorSubClass" : "INVALID_ARGUMENT_INDEX", "sqlState" : "22023", "messageParameters" : { + "funcName" : "`format_string`", "parameter" : "strfmt", - "functionName" : "`format_string`", - "expected" : "expects %1$, %2$ and so on, but got %0$." + "expected" : "%1$, %2$ and so on", + "found" : "%0$" } } diff --git a/sql/core/src/test/resources/sql-tests/results/regexp-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/regexp-functions.sql.out index f7524f09177e2..2617b98a09468 100644 --- a/sql/core/src/test/resources/sql-tests/results/regexp-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/regexp-functions.sql.out @@ -131,14 +131,14 @@ SELECT regexp_extract('1a 2b 14m', '(?l)') struct<> -- !query output org.apache.spark.SparkRuntimeException -#[INVALID_FUNCTION_ARGUMENTS.INVALID_ARGUMENT_VALUE] Arguments of the `regexp_extract` function are invalid: The value of parameter(s) 'regexp' is invalid: (?l) { - "errorClass" : "INVALID_PARAMETER_VALUE", + "errorClass" : "INVALID_FUNCTION_ARGUMENTS", + "errorSubClass" : "INVALID_ARGUMENT_VALUE", "sqlState" : "22023", "messageParameters" : { + "funcName" : "`regexp_extract`", "parameter" : "regexp", - "functionName" : "`regexp_extract`", - "expected" : "(?l)" + "message" : "(?l)" } } @@ -275,14 +275,14 @@ SELECT regexp_extract_all('abc', col0, 1) FROM VALUES('], [') AS t(col0) struct<> -- !query output org.apache.spark.SparkRuntimeException -#[INVALID_FUNCTION_ARGUMENTS.INVALID_ARGUMENT_VALUE] Arguments of the `regexp_extract_all` function are invalid: The value of parameter(s) 'regexp' is invalid: ], [ { - "errorClass" : "INVALID_PARAMETER_VALUE", + "errorClass" : "INVALID_FUNCTION_ARGUMENTS", + "errorSubClass" : "INVALID_ARGUMENT_VALUE", "sqlState" : "22023", "messageParameters" : { + "funcName" : "`regexp_extract_all`", "parameter" : "regexp", - "functionName" : "`regexp_extract_all`", - "expected" : "], [" + "message" : "], [" } } @@ -582,13 +582,13 @@ SELECT regexp_instr('abc', col0, 1) FROM VALUES(') ?') AS t(col0) struct<> -- !query output org.apache.spark.SparkRuntimeException -#[INVALID_FUNCTION_ARGUMENTS.INVALID_ARGUMENT_VALUE] Arguments of the `regexp_instr` function are invalid: The value of parameter(s) 'regexp' is invalid: ) ? { - "errorClass" : "INVALID_PARAMETER_VALUE", + "errorClass" : "INVALID_FUNCTION_ARGUMENTS", + "errorSubClass" : "INVALID_ARGUMENT_VALUE", "sqlState" : "22023", "messageParameters" : { + "funcName" : "`regexp_instr`", "parameter" : "regexp", - "functionName" : "`regexp_instr`", - "expected" : ") ?" + "message" : ") ?" } } diff --git a/sql/core/src/test/resources/sql-tests/results/sql-compatibility-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/sql-compatibility-functions.sql.out index 9c794ac0f7132..051515eb86091 100644 --- a/sql/core/src/test/resources/sql-tests/results/sql-compatibility-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/sql-compatibility-functions.sql.out @@ -93,7 +93,14 @@ SELECT string(1, 2) struct<> -- !query output org.apache.spark.sql.AnalysisException -[INVALID_FUNCTION_ARGUMENTS.CAST_ALIAS] Arguments of the string function are invalid: Function accepts only one argument; line 1 pos 7 +{ + "errorClass" : "INVALID_FUNCTION_ARGUMENTS", + "errorSubClass" : "CAST_ALIAS", + "sqlState" : "22023", + "messageParameters" : { + "funcName" : "string" + } +} -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/string-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/string-functions.sql.out index c69c7ef931fd0..5c14729451414 100644 --- a/sql/core/src/test/resources/sql-tests/results/string-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/string-functions.sql.out @@ -651,7 +651,16 @@ select decode() struct<> -- !query output org.apache.spark.sql.AnalysisException -[INVALID_FUNCTION_ARGUMENTS.INVALID_NUMBER_OF_ARGUMENTS] Arguments of the decode function are invalid: Invalid number of arguments. Expected: 2; Found: 0; line 1 pos 7 +{ + "errorClass" : "INVALID_FUNCTION_ARGUMENTS", + "errorSubClass" : "INVALID_NUMBER_OF_ARGUMENTS", + "sqlState" : "22023", + "messageParameters" : { + "funcName" : "decode", + "expected" : "2", + "found" : "0" + } +} -- !query @@ -660,8 +669,16 @@ select decode(encode('abc', 'utf-8')) struct<> -- !query output org.apache.spark.sql.AnalysisException -[INVALID_FUNCTION_ARGUMENTS.INVALID_NUMBER_OF_ARGUMENTS] Arguments of the decode function are invalid: Invalid number of arguments. Expected: 2; Found: 1; line 1 pos 7 - +{ + "errorClass" : "INVALID_FUNCTION_ARGUMENTS", + "errorSubClass" : "INVALID_NUMBER_OF_ARGUMENTS", + "sqlState" : "22023", + "messageParameters" : { + "funcName" : "decode", + "expected" : "2", + "found" : "1" + } +} -- !query select decode(encode('abc', 'utf-8'), 'utf-8') diff --git a/sql/core/src/test/resources/sql-tests/results/timestamp-ntz.sql.out b/sql/core/src/test/resources/sql-tests/results/timestamp-ntz.sql.out index 335d1aa9e89e3..d4e9d07ac5e7d 100644 --- a/sql/core/src/test/resources/sql-tests/results/timestamp-ntz.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/timestamp-ntz.sql.out @@ -45,8 +45,16 @@ SELECT make_timestamp_ntz(2021, 07, 11, 6, 30, 45.678, 'CET') struct<> -- !query output org.apache.spark.sql.AnalysisException -[INVALID_FUNCTION_ARGUMENTS.INVALID_NUMBER_OF_ARGUMENTS] Arguments of the make_timestamp_ntz function are invalid: Invalid number of arguments. Expected: 6; Found: 7; line 1 pos 7 - +{ + "errorClass" : "INVALID_FUNCTION_ARGUMENTS", + "errorSubClass" : "INVALID_NUMBER_OF_ARGUMENTS", + "sqlState" : "22023", + "messageParameters" : { + "funcName" : "make_timestamp_ntz", + "expected" : "6", + "found" : "7" + } +} -- !query SELECT make_timestamp_ntz(2021, 07, 11, 6, 30, 60.007) diff --git a/sql/core/src/test/resources/sql-tests/results/udaf.sql.out b/sql/core/src/test/resources/sql-tests/results/udaf.sql.out index 82af04099e714..24d8e1db697d5 100644 --- a/sql/core/src/test/resources/sql-tests/results/udaf.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udaf.sql.out @@ -31,7 +31,16 @@ SELECT default.myDoubleAvg(int_col1, 3) as my_avg from t1 struct<> -- !query output org.apache.spark.sql.AnalysisException -[INVALID_FUNCTION_ARGUMENTS.INVALID_NUMBER_OF_ARGUMENTS] Arguments of the spark_catalog.default.mydoubleavg function are invalid: Invalid number of arguments. Expected: 1; Found: 2; line 1 pos 7 +{ + "errorClass" : "INVALID_FUNCTION_ARGUMENTS", + "errorSubClass" : "INVALID_NUMBER_OF_ARGUMENTS", + "sqlState" : "22023", + "messageParameters" : { + "funcName" : "spark_catalog.default.mydoubleavg", + "expected" : "1", + "found" : "2" + } +} -- !query CREATE FUNCTION udaf1 AS 'test.non.existent.udaf' diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-udaf.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-udaf.sql.out index de33f6ae75a2b..873b7530ecd43 100644 --- a/sql/core/src/test/resources/sql-tests/results/udf/udf-udaf.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-udaf.sql.out @@ -31,8 +31,16 @@ SELECT default.myDoubleAvg(udf(int_col1), udf(3)) as my_avg from t1 struct<> -- !query output org.apache.spark.sql.AnalysisException -[INVALID_FUNCTION_ARGUMENTS.INVALID_NUMBER_OF_ARGUMENTS] Arguments of the spark_catalog.default.mydoubleavg function are invalid: Invalid number of arguments. Expected: 1; Found: 2; line 1 pos 7 - +{ + "errorClass" : "INVALID_FUNCTION_ARGUMENTS", + "errorSubClass" : "INVALID_NUMBER_OF_ARGUMENTS", + "sqlState" : "22023", + "messageParameters" : { + "funcName" : "spark_catalog.default.mydoubleavg", + "expected" : "1", + "found" : "2" + } +} -- !query CREATE FUNCTION udaf1 AS 'test.non.existent.udaf'