Skip to content

Commit

Permalink
[SPARK-45710][SQL] Assign name to the error _LEGACY_ERROR_TEMP_21[59,…
Browse files Browse the repository at this point in the history
…60,61,62]
  • Loading branch information
dengziming committed Oct 27, 2023
1 parent 23c4cce commit 29af051
Show file tree
Hide file tree
Showing 6 changed files with 123 additions and 40 deletions.
43 changes: 23 additions & 20 deletions common/utils/src/main/resources/error/error-classes.json
Original file line number Diff line number Diff line change
Expand Up @@ -437,6 +437,12 @@
],
"sqlState" : "22004"
},
"CONCAT_ARRAYS_WITH_ELEMENTS_EXCEED_LIMIT" : {
"message" : [
"Unsuccessful try to concat arrays with <numberOfElements> elements due to exceeding the array size limit <maxRoundedArrayLength>."
],
"sqlState" : "54000"
},
"CONCURRENT_QUERY" : {
"message" : [
"Another instance of this query was just started by a concurrent session."
Expand Down Expand Up @@ -1051,6 +1057,12 @@
],
"sqlState" : "42704"
},
"FLATTEN_ARRAYS_WITH_ELEMENTS_EXCEED_LIMIT" : {
"message" : [
"Unsuccessful try to flatten an array of arrays with <numberOfElements> elements due to exceeding the array size limit <maxRoundedArrayLength>."
],
"sqlState" : "54000"
},
"FORBIDDEN_OPERATION" : {
"message" : [
"The operation <statement> is not allowed on the <objectType>: <objectName>."
Expand Down Expand Up @@ -1969,6 +1981,11 @@
"expects an integer value in [0, <upper>), but got <invalidValue>."
]
},
"COUNT" : {
"message" : [
"expects an integer value less than or equal to <maxRoundedArrayLength>, but got <count>."
]
},
"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 <invalidValue>."
Expand Down Expand Up @@ -3074,6 +3091,12 @@
],
"sqlState" : "42846"
},
"UNION_ARRAY_WITH_ELEMENTS_EXCEED_LIMIT" : {
"message" : [
"Unsuccessful try to union arrays with <length> elements due to exceeding the array size limit <maxRoundedArrayLength>."
],
"sqlState" : "54000"
},
"UNKNOWN_PROTOBUF_MESSAGE_TYPE" : {
"message" : [
"Attempting to treat <descriptorName> as a Message, but it was <containingType>."
Expand Down Expand Up @@ -5759,26 +5782,6 @@
"<userClass> is not annotated with SQLUserDefinedType nor registered with UDTRegistration.}"
]
},
"_LEGACY_ERROR_TEMP_2159" : {
"message" : [
"Unsuccessful try to concat arrays with <numberOfElements> elements due to exceeding the array size limit <maxRoundedArrayLength>."
]
},
"_LEGACY_ERROR_TEMP_2160" : {
"message" : [
"Unsuccessful try to flatten an array of arrays with <numberOfElements> elements due to exceeding the array size limit <maxRoundedArrayLength>."
]
},
"_LEGACY_ERROR_TEMP_2161" : {
"message" : [
"Unsuccessful try to create array with <count> elements due to exceeding the array size limit <maxRoundedArrayLength>."
]
},
"_LEGACY_ERROR_TEMP_2162" : {
"message" : [
"Unsuccessful try to union arrays with <length> elements due to exceeding the array size limit <maxRoundedArrayLength>."
]
},
"_LEGACY_ERROR_TEMP_2163" : {
"message" : [
"Initial type <dataType> must be a <target>."
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -45,6 +45,10 @@ expects one of binary formats 'base64', 'hex', 'utf-8', but got `<invalidFormat>

expects an integer value in [0, `<upper>`), but got `<invalidValue>`.

## COUNT

expects an integer value less than or equal to `<maxRoundedArrayLength>`, but got `<count>`.

## 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 `<invalidValue>`.
Expand Down
18 changes: 18 additions & 0 deletions docs/sql-error-conditions.md
Original file line number Diff line number Diff line change
Expand Up @@ -370,6 +370,12 @@ The comparator has returned a NULL for a comparison between `<firstValue>` 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 `<numberOfElements>` elements due to exceeding the array size limit `<maxRoundedArrayLength>`.

### CONCURRENT_QUERY

[SQLSTATE: 0A000](sql-error-conditions-sqlstates.html#class-0A-feature-not-supported)
Expand Down Expand Up @@ -640,6 +646,12 @@ Cannot `<op>` column, because `<fieldNames>` already exists in `<struct>`.

No such struct field `<fieldName>` in `<fields>`.

### 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 `<numberOfElements>` elements due to exceeding the array size limit `<maxRoundedArrayLength>`.

### FORBIDDEN_OPERATION

[SQLSTATE: 42809](sql-error-conditions-sqlstates.html#class-42-syntax-error-or-access-rule-violation)
Expand Down Expand Up @@ -1983,6 +1995,12 @@ Cannot invoke function `<functionName>` because it contains positional argument(

The class `<className>` has an unexpected expression serializer. Expects "STRUCT" or "IF" which returns "STRUCT" but found `<expr>`.

### UNION_ARRAY_WITH_ELEMENTS_EXCEED_LIMIT

[SQLSTATE: 54000](sql-error-conditions-sqlstates.html#class-54-program-limit-exceeded)

Unsuccessful try to union arrays with `<length>` elements due to exceeding the array size limit `<maxRoundedArrayLength>`.

### UNKNOWN_PROTOBUF_MESSAGE_TYPE

[SQLSTATE: 42K0G](sql-error-conditions-sqlstates.html#class-42-syntax-error-or-access-rule-violation)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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))
Expand Down Expand Up @@ -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)) {
Expand Down Expand Up @@ -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 {
Expand Down Expand Up @@ -4049,7 +4050,7 @@ trait ArrayBinaryLike

object ArrayBinaryLike {
def throwUnionLengthOverflowException(length: Int): Unit = {
throw QueryExecutionErrors.unionArrayWithElementsExceedLimitError(length)
throw QueryExecutionErrors.unionArraysWithElementsExceedLimitError(length)
}
}

Expand Down Expand Up @@ -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 {
Expand Down Expand Up @@ -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 {
Expand Down Expand Up @@ -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 {
Expand Down Expand Up @@ -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
Expand Down Expand Up @@ -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
Expand All @@ -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
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1402,31 +1402,34 @@ 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()))
}

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()))
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand All @@ -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
Expand Down Expand Up @@ -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 {
Expand Down

0 comments on commit 29af051

Please sign in to comment.