Skip to content

Commit

Permalink
revert INVALID_PARAMETER_VALUE.COUNT
Browse files Browse the repository at this point in the history
  • Loading branch information
dengziming committed Oct 28, 2023
1 parent 29af051 commit 921b9f1
Show file tree
Hide file tree
Showing 6 changed files with 21 additions and 19 deletions.
11 changes: 6 additions & 5 deletions common/utils/src/main/resources/error/error-classes.json
Original file line number Diff line number Diff line change
Expand Up @@ -495,6 +495,12 @@
],
"sqlState" : "22018"
},
"CREATE_ARRAYS_WITH_ELEMENTS_EXCEED_LIMIT" : {
"message" : [
"Unsuccessful try to create array with <count> elements in <functionName> due to exceeding the array size limit <maxRoundedArrayLength>."
],
"sqlState" : "54000"
},
"CREATE_PERMANENT_VIEW_WITHOUT_ALIAS" : {
"message" : [
"Not allowed to create the permanent view <name> without explicitly assigning an alias for the expression <attr>."
Expand Down Expand Up @@ -1981,11 +1987,6 @@
"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
Original file line number Diff line number Diff line change
Expand Up @@ -45,10 +45,6 @@ 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
6 changes: 6 additions & 0 deletions docs/sql-error-conditions.md
Original file line number Diff line number Diff line change
Expand Up @@ -403,6 +403,12 @@ For more details see [CONNECT](sql-error-conditions-connect-error-class.html)

The value `<str>` (`<fmt>`) cannot be converted to `<targetType>` because it is malformed. Correct the value as per the syntax, or change its format. Use `<suggestion>` to tolerate malformed input and return NULL instead.

### CREATE_ARRAYS_WITH_ELEMENTS_EXCEED_LIMIT

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

Unsuccessful try to create array with `<count>` elements in `<functionName>` due to exceeding the array size limit `<maxRoundedArrayLength>`.

### CREATE_PERMANENT_VIEW_WITHOUT_ALIAS

[SQLSTATE: 0A000](sql-error-conditions-sqlstates.html#class-0A-feature-not-supported)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -3842,10 +3842,11 @@ trait ArraySetLike {
value : String,
size : String,
nullElementIndex : String,
prettyName: String = ""): String = withResultArrayNullCheck(
functionName: String): String = withResultArrayNullCheck(
s"""
|if ($size > ${ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH}) {
| throw QueryExecutionErrors.createArrayWithElementsExceedLimitError("$prettyName", $size);
| throw QueryExecutionErrors.createArrayWithElementsExceedLimitError(
| "$functionName", $size);
|}
|
|if (!UnsafeArrayData.shouldUseGenericArrayData(${et.defaultSize}, $size)) {
Expand Down Expand Up @@ -3903,7 +3904,7 @@ case class ArrayDistinct(child: Expression)
(value: Any) =>
if (!hs.contains(value)) {
if (arrayBuffer.size > ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH) {
ArrayBinaryLike.throwUnionLengthOverflowException(arrayBuffer.size)
QueryExecutionErrors.createArrayWithElementsExceedLimitError(prettyName, arrayBuffer.size)
}
arrayBuffer += value
hs.add(value)
Expand Down Expand Up @@ -4808,7 +4809,7 @@ case class ArrayInsert(
val newArrayLength = math.max(baseArr.numElements() + 1, positivePos.get)

if (newArrayLength > ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH) {
throw QueryExecutionErrors.concatArraysWithElementsExceedLimitError(newArrayLength)
throw QueryExecutionErrors.createArrayWithElementsExceedLimitError(prettyName, newArrayLength)
}

val newArray = new Array[Any](newArrayLength)
Expand Down Expand Up @@ -4842,7 +4843,7 @@ case class ArrayInsert(
val newArrayLength = -posInt + baseOffset

if (newArrayLength > ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH) {
throw QueryExecutionErrors.concatArraysWithElementsExceedLimitError(newArrayLength)
throw QueryExecutionErrors.createArrayWithElementsExceedLimitError(prettyName, newArrayLength)
}

val newArray = new Array[Any](newArrayLength)
Expand All @@ -4866,7 +4867,7 @@ case class ArrayInsert(
val newArrayLength = math.max(baseArr.numElements() + 1, posInt + 1)

if (newArrayLength > ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH) {
throw QueryExecutionErrors.concatArraysWithElementsExceedLimitError(newArrayLength)
throw QueryExecutionErrors.createArrayWithElementsExceedLimitError(prettyName, newArrayLength)
}

val newArray = new Array[Any](newArrayLength)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1419,9 +1419,8 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase with ExecutionE
def createArrayWithElementsExceedLimitError(
prettyName: String, count: Any): SparkRuntimeException = {
new SparkRuntimeException(
errorClass = "INVALID_PARAMETER_VALUE.COUNT",
errorClass = "CREATE_ARRAYS_WITH_ELEMENTS_EXCEED_LIMIT",
messageParameters = Map(
"parameter" -> toSQLId("count"),
"count" -> count.toString,
"functionName" -> toSQLId(prettyName),
"maxRoundedArrayLength" -> ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH.toString()))
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1135,10 +1135,9 @@ class QueryExecutionErrorsSuite
exception = intercept[SparkRuntimeException] {
sql("select array_repeat(1, 2147483647)").collect()
},
errorClass = "INVALID_PARAMETER_VALUE.COUNT",
errorClass = "CREATE_ARRAYS_WITH_ELEMENTS_EXCEED_LIMIT",
parameters = Map(
"count" -> "2147483647",
"parameter" -> toSQLId("count"),
"functionName" -> toSQLId("array_repeat"),
"maxRoundedArrayLength" -> MAX_ROUNDED_ARRAY_LENGTH.toString
)
Expand Down

0 comments on commit 921b9f1

Please sign in to comment.