From 29af051bbd77950b09f04e459f51ac50484f9680 Mon Sep 17 00:00:00 2001 From: dengziming Date: Fri, 27 Oct 2023 20:22:28 +0800 Subject: [PATCH] [SPARK-45710][SQL] Assign name to the error _LEGACY_ERROR_TEMP_21[59,60,61,62] --- .../main/resources/error/error-classes.json | 43 ++++++++------- ...ons-invalid-parameter-value-error-class.md | 4 ++ docs/sql-error-conditions.md | 18 ++++++ .../expressions/collectionOperations.scala | 26 +++++---- .../sql/errors/QueryExecutionErrors.scala | 17 +++--- .../errors/QueryExecutionErrorsSuite.scala | 55 ++++++++++++++++++- 6 files changed, 123 insertions(+), 40 deletions(-) diff --git a/common/utils/src/main/resources/error/error-classes.json b/common/utils/src/main/resources/error/error-classes.json index f9cc0a8652161..1ad42921777e9 100644 --- a/common/utils/src/main/resources/error/error-classes.json +++ b/common/utils/src/main/resources/error/error-classes.json @@ -437,6 +437,12 @@ ], "sqlState" : "22004" }, + "CONCAT_ARRAYS_WITH_ELEMENTS_EXCEED_LIMIT" : { + "message" : [ + "Unsuccessful try to concat arrays with elements due to exceeding the array size limit ." + ], + "sqlState" : "54000" + }, "CONCURRENT_QUERY" : { "message" : [ "Another instance of this query was just started by a concurrent session." @@ -1051,6 +1057,12 @@ ], "sqlState" : "42704" }, + "FLATTEN_ARRAYS_WITH_ELEMENTS_EXCEED_LIMIT" : { + "message" : [ + "Unsuccessful try to flatten an array of arrays with elements due to exceeding the array size limit ." + ], + "sqlState" : "54000" + }, "FORBIDDEN_OPERATION" : { "message" : [ "The operation is not allowed on the : ." @@ -1969,6 +1981,11 @@ "expects an integer value in [0, ), but got ." ] }, + "COUNT" : { + "message" : [ + "expects an integer value less than or equal to , but got ." + ] + }, "DATETIME_UNIT" : { "message" : [ "expects one of the units without quotes YEAR, QUARTER, MONTH, WEEK, DAY, DAYOFYEAR, HOUR, MINUTE, SECOND, MILLISECOND, MICROSECOND, but got the string literal ." @@ -3074,6 +3091,12 @@ ], "sqlState" : "42846" }, + "UNION_ARRAY_WITH_ELEMENTS_EXCEED_LIMIT" : { + "message" : [ + "Unsuccessful try to union arrays with elements due to exceeding the array size limit ." + ], + "sqlState" : "54000" + }, "UNKNOWN_PROTOBUF_MESSAGE_TYPE" : { "message" : [ "Attempting to treat as a Message, but it was ." @@ -5759,26 +5782,6 @@ " is not annotated with SQLUserDefinedType nor registered with UDTRegistration.}" ] }, - "_LEGACY_ERROR_TEMP_2159" : { - "message" : [ - "Unsuccessful try to concat arrays with elements due to exceeding the array size limit ." - ] - }, - "_LEGACY_ERROR_TEMP_2160" : { - "message" : [ - "Unsuccessful try to flatten an array of arrays with elements due to exceeding the array size limit ." - ] - }, - "_LEGACY_ERROR_TEMP_2161" : { - "message" : [ - "Unsuccessful try to create array with elements due to exceeding the array size limit ." - ] - }, - "_LEGACY_ERROR_TEMP_2162" : { - "message" : [ - "Unsuccessful try to union arrays with elements due to exceeding the array size limit ." - ] - }, "_LEGACY_ERROR_TEMP_2163" : { "message" : [ "Initial type must be a ." diff --git a/docs/sql-error-conditions-invalid-parameter-value-error-class.md b/docs/sql-error-conditions-invalid-parameter-value-error-class.md index d58d4fc2f5992..eb4d91303887a 100644 --- a/docs/sql-error-conditions-invalid-parameter-value-error-class.md +++ b/docs/sql-error-conditions-invalid-parameter-value-error-class.md @@ -45,6 +45,10 @@ expects one of binary formats 'base64', 'hex', 'utf-8', but got ` expects an integer value in [0, ``), but got ``. +## COUNT + +expects an integer value less than or equal to ``, but got ``. + ## DATETIME_UNIT expects one of the units without quotes YEAR, QUARTER, MONTH, WEEK, DAY, DAYOFYEAR, HOUR, MINUTE, SECOND, MILLISECOND, MICROSECOND, but got the string literal ``. diff --git a/docs/sql-error-conditions.md b/docs/sql-error-conditions.md index 7c537f6fe20e5..83e4cb4f42384 100644 --- a/docs/sql-error-conditions.md +++ b/docs/sql-error-conditions.md @@ -370,6 +370,12 @@ The comparator has returned a NULL for a comparison between `` and ` It should return a positive integer for "greater than", 0 for "equal" and a negative integer for "less than". To revert to deprecated behavior where NULL is treated as 0 (equal), you must set "spark.sql.legacy.allowNullComparisonResultInArraySort" to "true". +### CONCAT_ARRAYS_WITH_ELEMENTS_EXCEED_LIMIT + +[SQLSTATE: 54000](sql-error-conditions-sqlstates.html#class-54-program-limit-exceeded) + +Unsuccessful try to concat arrays with `` elements due to exceeding the array size limit ``. + ### CONCURRENT_QUERY [SQLSTATE: 0A000](sql-error-conditions-sqlstates.html#class-0A-feature-not-supported) @@ -640,6 +646,12 @@ Cannot `` column, because `` already exists in ``. No such struct field `` in ``. +### FLATTEN_ARRAYS_WITH_ELEMENTS_EXCEED_LIMIT + +[SQLSTATE: 54000](sql-error-conditions-sqlstates.html#class-54-program-limit-exceeded) + +Unsuccessful try to flatten an array of arrays with `` elements due to exceeding the array size limit ``. + ### FORBIDDEN_OPERATION [SQLSTATE: 42809](sql-error-conditions-sqlstates.html#class-42-syntax-error-or-access-rule-violation) @@ -1983,6 +1995,12 @@ Cannot invoke function `` because it contains positional argument( The class `` has an unexpected expression serializer. Expects "STRUCT" or "IF" which returns "STRUCT" but found ``. +### UNION_ARRAY_WITH_ELEMENTS_EXCEED_LIMIT + +[SQLSTATE: 54000](sql-error-conditions-sqlstates.html#class-54-program-limit-exceeded) + +Unsuccessful try to union arrays with `` elements due to exceeding the array size limit ``. + ### UNKNOWN_PROTOBUF_MESSAGE_TYPE [SQLSTATE: 42K0G](sql-error-conditions-sqlstates.html#class-42-syntax-error-or-access-rule-violation) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala index 0a080423b10fe..deaf46bec18e4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala @@ -3551,7 +3551,7 @@ case class ArrayRepeat(left: Expression, right: Expression) null } else { if (count.asInstanceOf[Int] > ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH) { - throw QueryExecutionErrors.createArrayWithElementsExceedLimitError(count) + throw QueryExecutionErrors.createArrayWithElementsExceedLimitError(prettyName, count) } val element = left.eval(input) new GenericArrayData(Array.fill(count.asInstanceOf[Int])(element)) @@ -3841,10 +3841,11 @@ trait ArraySetLike { builder: String, value : String, size : String, - nullElementIndex : String): String = withResultArrayNullCheck( + nullElementIndex : String, + prettyName: String = ""): String = withResultArrayNullCheck( s""" |if ($size > ${ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH}) { - | throw QueryExecutionErrors.createArrayWithElementsExceedLimitError($size); + | throw QueryExecutionErrors.createArrayWithElementsExceedLimitError("$prettyName", $size); |} | |if (!UnsafeArrayData.shouldUseGenericArrayData(${et.defaultSize}, $size)) { @@ -4012,7 +4013,7 @@ case class ArrayDistinct(child: Expression) |for (int $i = 0; $i < $array.numElements(); $i++) { | $processArray |} - |${buildResultArray(builder, ev.value, size, nullElementIndex)} + |${buildResultArray(builder, ev.value, size, nullElementIndex, prettyName)} """.stripMargin }) } else { @@ -4049,7 +4050,7 @@ trait ArrayBinaryLike object ArrayBinaryLike { def throwUnionLengthOverflowException(length: Int): Unit = { - throw QueryExecutionErrors.unionArrayWithElementsExceedLimitError(length) + throw QueryExecutionErrors.unionArraysWithElementsExceedLimitError(length) } } @@ -4212,7 +4213,7 @@ case class ArrayUnion(left: Expression, right: Expression) extends ArrayBinaryLi | $processArray | } |} - |${buildResultArray(builder, ev.value, size, nullElementIndex)} + |${buildResultArray(builder, ev.value, size, nullElementIndex, prettyName)} """.stripMargin }) } else { @@ -4481,7 +4482,7 @@ case class ArrayIntersect(left: Expression, right: Expression) extends ArrayBina |for (int $i = 0; $i < $array1.numElements(); $i++) { | $processArray1 |} - |${buildResultArray(builder, ev.value, size, nullElementIndex)} + |${buildResultArray(builder, ev.value, size, nullElementIndex, prettyName)} """.stripMargin }) } else { @@ -4692,7 +4693,7 @@ case class ArrayExcept(left: Expression, right: Expression) extends ArrayBinaryL |for (int $i = 0; $i < $array1.numElements(); $i++) { | $processArray1 |} - |${buildResultArray(builder, ev.value, size, nullElementIndex)} + |${buildResultArray(builder, ev.value, size, nullElementIndex, prettyName)} """.stripMargin }) } else { @@ -4911,7 +4912,8 @@ case class ArrayInsert( | |final int $resLength = java.lang.Math.max($arr.numElements() + 1, ${positivePos.get}); |if ($resLength > ${ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH}) { - | throw QueryExecutionErrors.createArrayWithElementsExceedLimitError($resLength); + | throw QueryExecutionErrors.createArrayWithElementsExceedLimitError( + | "$prettyName", $resLength); |} | |$allocation @@ -4948,7 +4950,8 @@ case class ArrayInsert( | | $resLength = java.lang.Math.abs($pos) + $baseOffset; | if ($resLength > ${ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH}) { - | throw QueryExecutionErrors.createArrayWithElementsExceedLimitError($resLength); + | throw QueryExecutionErrors.createArrayWithElementsExceedLimitError( + | "$prettyName", $resLength); | } | | $allocation @@ -4975,7 +4978,8 @@ case class ArrayInsert( | | $resLength = java.lang.Math.max($arr.numElements() + 1, $itemInsertionIndex + 1); | if ($resLength > ${ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH}) { - | throw QueryExecutionErrors.createArrayWithElementsExceedLimitError($resLength); + | throw QueryExecutionErrors.createArrayWithElementsExceedLimitError( + | "$prettyName", $resLength); | } | | $allocation 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 afc244509c41d..f49801c386449 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 @@ -1402,7 +1402,7 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase with ExecutionE def concatArraysWithElementsExceedLimitError(numberOfElements: Long): SparkRuntimeException = { new SparkRuntimeException( - errorClass = "_LEGACY_ERROR_TEMP_2159", + errorClass = "CONCAT_ARRAYS_WITH_ELEMENTS_EXCEED_LIMIT", messageParameters = Map( "numberOfElements" -> numberOfElements.toString(), "maxRoundedArrayLength" -> ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH.toString())) @@ -1410,23 +1410,26 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase with ExecutionE def flattenArraysWithElementsExceedLimitError(numberOfElements: Long): SparkRuntimeException = { new SparkRuntimeException( - errorClass = "_LEGACY_ERROR_TEMP_2160", + errorClass = "FLATTEN_ARRAYS_WITH_ELEMENTS_EXCEED_LIMIT", messageParameters = Map( "numberOfElements" -> numberOfElements.toString(), "maxRoundedArrayLength" -> ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH.toString())) } - def createArrayWithElementsExceedLimitError(count: Any): SparkRuntimeException = { + def createArrayWithElementsExceedLimitError( + prettyName: String, count: Any): SparkRuntimeException = { new SparkRuntimeException( - errorClass = "_LEGACY_ERROR_TEMP_2161", + errorClass = "INVALID_PARAMETER_VALUE.COUNT", messageParameters = Map( - "count" -> count.toString(), + "parameter" -> toSQLId("count"), + "count" -> count.toString, + "functionName" -> toSQLId(prettyName), "maxRoundedArrayLength" -> ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH.toString())) } - def unionArrayWithElementsExceedLimitError(length: Int): SparkRuntimeException = { + def unionArraysWithElementsExceedLimitError(length: Int): SparkRuntimeException = { new SparkRuntimeException( - errorClass = "_LEGACY_ERROR_TEMP_2162", + errorClass = "UNION_ARRAY_WITH_ELEMENTS_EXCEED_LIMIT", messageParameters = Map( "length" -> length.toString(), "maxRoundedArrayLength" -> ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH.toString())) 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 945dd782da0e0..96b81a4c26add 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 @@ -31,7 +31,7 @@ import org.apache.spark._ import org.apache.spark.sql.{AnalysisException, DataFrame, Dataset, QueryTest, Row, SaveMode} import org.apache.spark.sql.catalyst.FunctionIdentifier import org.apache.spark.sql.catalyst.analysis.{NamedParameter, UnresolvedGenerator} -import org.apache.spark.sql.catalyst.expressions.{Grouping, Literal, RowNumber} +import org.apache.spark.sql.catalyst.expressions.{Concat, CreateArray, EmptyRow, Flatten, Grouping, Literal, RowNumber} import org.apache.spark.sql.catalyst.expressions.CodegenObjectFactoryMode._ import org.apache.spark.sql.catalyst.expressions.codegen.CodegenContext import org.apache.spark.sql.catalyst.expressions.objects.InitializeJavaBean @@ -44,16 +44,20 @@ import org.apache.spark.sql.execution.datasources.orc.OrcTest import org.apache.spark.sql.execution.datasources.parquet.ParquetTest import org.apache.spark.sql.execution.datasources.v2.jdbc.JDBCTableCatalog import org.apache.spark.sql.execution.streaming.FileSystemBasedCheckpointFileManager +import org.apache.spark.sql.execution.vectorized.ConstantColumnVector import org.apache.spark.sql.functions.{lit, lower, struct, sum, udf} import org.apache.spark.sql.internal.LegacyBehaviorPolicy.EXCEPTION import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.jdbc.{JdbcDialect, JdbcDialects} import org.apache.spark.sql.streaming.StreamingQueryException import org.apache.spark.sql.test.SharedSparkSession -import org.apache.spark.sql.types.{DataType, DecimalType, LongType, MetadataBuilder, StructType} +import org.apache.spark.sql.types.{ArrayType, BooleanType, DataType, DecimalType, LongType, MetadataBuilder, StructType} +import org.apache.spark.sql.vectorized.ColumnarArray +import org.apache.spark.unsafe.array.ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH import org.apache.spark.util.ThreadUtils import org.apache.spark.util.Utils + class QueryExecutionErrorsSuite extends QueryTest with ParquetTest @@ -1093,6 +1097,53 @@ class QueryExecutionErrorsSuite ) ) } + + test("Elements exceed limit for concat()") { + val array = new ColumnarArray( + new ConstantColumnVector(Int.MaxValue, BooleanType), 0, Int.MaxValue) + + checkError( + exception = intercept[SparkRuntimeException] { + Concat(Seq(Literal.create(array, ArrayType(BooleanType)))).eval(EmptyRow) + }, + errorClass = "CONCAT_ARRAYS_WITH_ELEMENTS_EXCEED_LIMIT", + parameters = Map( + "numberOfElements" -> Int.MaxValue.toString, + "maxRoundedArrayLength" -> MAX_ROUNDED_ARRAY_LENGTH.toString + ) + ) + } + + test("Elements exceed limit for flatten()") { + val array = new ColumnarArray( + new ConstantColumnVector(Int.MaxValue, BooleanType), 0, Int.MaxValue) + + checkError( + exception = intercept[SparkRuntimeException] { + Flatten(CreateArray(Seq(Literal.create(array, ArrayType(BooleanType))))).eval(EmptyRow) + }, + errorClass = "FLATTEN_ARRAYS_WITH_ELEMENTS_EXCEED_LIMIT", + parameters = Map( + "numberOfElements" -> Int.MaxValue.toString, + "maxRoundedArrayLength" -> MAX_ROUNDED_ARRAY_LENGTH.toString + ) + ) + } + + test("Elements exceed limit for array_repeat()") { + checkError( + exception = intercept[SparkRuntimeException] { + sql("select array_repeat(1, 2147483647)").collect() + }, + errorClass = "INVALID_PARAMETER_VALUE.COUNT", + parameters = Map( + "count" -> "2147483647", + "parameter" -> toSQLId("count"), + "functionName" -> toSQLId("array_repeat"), + "maxRoundedArrayLength" -> MAX_ROUNDED_ARRAY_LENGTH.toString + ) + ) + } } class FakeFileSystemSetPermission extends LocalFileSystem {