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 Nov 4, 2023
1 parent e6b4fa8 commit 9827199
Show file tree
Hide file tree
Showing 8 changed files with 182 additions and 130 deletions.
49 changes: 23 additions & 26 deletions common/utils/src/main/resources/error/error-classes.json
Original file line number Diff line number Diff line change
Expand Up @@ -405,6 +405,29 @@
],
"sqlState" : "42704"
},
"COLLECTION_SIZE_LIMIT_EXCEEDED" : {
"message" : [
"Can't create array with <numberOfElements> elements which exceeding the array size limit <maxRoundedArrayLength>,"
],
"subClass" : {
"FUNCTION" : {
"message" : [
"unsuccessful try to create arrays in function <functionName>."
]
},
"INITIALIZE" : {
"message" : [
"cannot initialize array with specified parameters."
]
},
"PARAMETER" : {
"message" : [
"the value of parameter(s) <parameter> in <functionName> is invalid."
]
}
},
"sqlState" : "54000"
},
"COLUMN_ALIASES_IS_NOT_ALLOWED" : {
"message" : [
"Columns aliases are not allowed in <op>."
Expand Down Expand Up @@ -3012,12 +3035,6 @@
],
"sqlState" : "428EK"
},
"TOO_MANY_ARRAY_ELEMENTS" : {
"message" : [
"Cannot initialize array with <numElements> elements of size <size>."
],
"sqlState" : "54000"
},
"UDTF_ALIAS_NUMBER_MISMATCH" : {
"message" : [
"The number of aliases supplied in the AS clause does not match the number of columns output by the UDTF.",
Expand Down Expand Up @@ -5765,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
39 changes: 39 additions & 0 deletions docs/sql-error-collection-size-exceed-limit-error-class.md
Original file line number Diff line number Diff line change
@@ -0,0 +1,39 @@
---
layout: global
title: COLLECTION_SIZE_LIMIT_EXCEEDED error class
displayTitle: COLLECTION_SIZE_LIMIT_EXCEEDED error class
license: |
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements. See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to You under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License. You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
---

SQLSTATE: 54000

Generic Spark Connect error.

Can't create array with <numberOfElements> elements which exceeding the array size limit <maxRoundedArrayLength>,

## FUNCTION

unsuccessful try to create arrays in function `<functionName>`.

## INITIALIZE

cannot initialize array with specified parameters.

## PARAMETER

the value of parameter(s) `<parameter>` in `<functionName>` is invalid.

14 changes: 8 additions & 6 deletions docs/sql-error-conditions.md
Original file line number Diff line number Diff line change
Expand Up @@ -338,6 +338,14 @@ The codec `<codecName>` is not available. Consider to set the config `<configKey

Cannot find a short name for the codec `<codecName>`.

### [COLLECTION_SIZE_LIMIT_EXCEEDED](sql-error-collection-size-exceed-limit-error-class.html)

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

Can't create array with `<numberOfElements>` elements which exceeding the array size limit `<maxRoundedArrayLength>`,

For more details see [COLLECTION_SIZE_LIMIT_EXCEEDED](sql-error-collection-size-exceed-limit-error-class.html)

### COLUMN_ALIASES_IS_NOT_ALLOWED

[SQLSTATE: 42601](sql-error-conditions-sqlstates.html#class-42-syntax-error-or-access-rule-violation)
Expand Down Expand Up @@ -1921,12 +1929,6 @@ Choose a different name, drop or replace the existing view, or add the IF NOT E

CREATE TEMPORARY VIEW or the corresponding Dataset APIs only accept single-part view names, but got: `<actualName>`.

### TOO_MANY_ARRAY_ELEMENTS

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

Cannot initialize array with `<numElements>` elements of size `<size>`.

### UDTF_ALIAS_NUMBER_MISMATCH

[SQLSTATE: 42802](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 @@ -62,7 +62,7 @@ public void initialize(int numElements) {
long totalInitialSize = headerInBytes + fixedPartInBytesLong;

if (totalInitialSize > Integer.MAX_VALUE) {
throw QueryExecutionErrors.tooManyArrayElementsError(numElements, elementSize);
throw QueryExecutionErrors.tooManyArrayElementsError(fixedPartInBytesLong, Integer.MAX_VALUE);
}

// it's now safe to cast fixedPartInBytesLong and totalInitialSize to int
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -2658,7 +2658,8 @@ case class Concat(children: Seq[Expression]) extends ComplexTypeMergingExpressio
val arrayData = inputs.map(_.asInstanceOf[ArrayData])
val numberOfElements = arrayData.foldLeft(0L)((sum, ad) => sum + ad.numElements())
if (numberOfElements > ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH) {
throw QueryExecutionErrors.concatArraysWithElementsExceedLimitError(numberOfElements)
throw QueryExecutionErrors.arrayFunctionWithElementsExceedLimitError(
prettyName, numberOfElements)
}
val finalData = new Array[AnyRef](numberOfElements.toInt)
var position = 0
Expand Down Expand Up @@ -2839,7 +2840,8 @@ case class Flatten(child: Expression) extends UnaryExpression with NullIntoleran
val arrayData = elements.map(_.asInstanceOf[ArrayData])
val numberOfElements = arrayData.foldLeft(0L)((sum, e) => sum + e.numElements())
if (numberOfElements > ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH) {
throw QueryExecutionErrors.flattenArraysWithElementsExceedLimitError(numberOfElements)
throw QueryExecutionErrors.arrayFunctionWithElementsExceedLimitError(
prettyName, numberOfElements)
}
val flattenedData = new Array(numberOfElements.toInt)
var position = 0
Expand Down Expand Up @@ -3552,7 +3554,8 @@ 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 @@ -3842,10 +3845,12 @@ trait ArraySetLike {
builder: String,
value : String,
size : String,
nullElementIndex : String): String = withResultArrayNullCheck(
nullElementIndex : String,
functionName: String): String = withResultArrayNullCheck(
s"""
|if ($size > ${ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH}) {
| throw QueryExecutionErrors.createArrayWithElementsExceedLimitError($size);
| throw QueryExecutionErrors.arrayFunctionWithElementsExceedLimitError(
| "$functionName", $size);
|}
|
|if (!UnsafeArrayData.shouldUseGenericArrayData(${et.defaultSize}, $size)) {
Expand Down Expand Up @@ -3903,7 +3908,8 @@ case class ArrayDistinct(child: Expression)
(value: Any) =>
if (!hs.contains(value)) {
if (arrayBuffer.size > ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH) {
ArrayBinaryLike.throwUnionLengthOverflowException(arrayBuffer.size)
throw QueryExecutionErrors.arrayFunctionWithElementsExceedLimitError(
prettyName, arrayBuffer.size)
}
arrayBuffer += value
hs.add(value)
Expand Down Expand Up @@ -4013,7 +4019,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 @@ -4048,13 +4054,6 @@ trait ArrayBinaryLike
}
}

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


/**
* Returns an array of the elements in the union of x and y, without duplicates
*/
Expand Down Expand Up @@ -4082,7 +4081,8 @@ case class ArrayUnion(left: Expression, right: Expression) extends ArrayBinaryLi
(value: Any) =>
if (!hs.contains(value)) {
if (arrayBuffer.size > ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH) {
ArrayBinaryLike.throwUnionLengthOverflowException(arrayBuffer.size)
throw QueryExecutionErrors.arrayFunctionWithElementsExceedLimitError(
prettyName, arrayBuffer.size)
}
arrayBuffer += value
hs.add(value)
Expand Down Expand Up @@ -4125,7 +4125,8 @@ case class ArrayUnion(left: Expression, right: Expression) extends ArrayBinaryLi
}
if (!found) {
if (arrayBuffer.length > ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH) {
ArrayBinaryLike.throwUnionLengthOverflowException(arrayBuffer.length)
throw QueryExecutionErrors.arrayFunctionWithElementsExceedLimitError(
prettyName, arrayBuffer.length)
}
arrayBuffer += elem
}
Expand Down Expand Up @@ -4213,7 +4214,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 All @@ -4230,44 +4231,6 @@ case class ArrayUnion(left: Expression, right: Expression) extends ArrayBinaryLi
newLeft: Expression, newRight: Expression): ArrayUnion = copy(left = newLeft, right = newRight)
}

object ArrayUnion {
def unionOrdering(
array1: ArrayData,
array2: ArrayData,
elementType: DataType,
ordering: Ordering[Any]): ArrayData = {
val arrayBuffer = new scala.collection.mutable.ArrayBuffer[Any]
var alreadyIncludeNull = false
Seq(array1, array2).foreach(_.foreach(elementType, (_, elem) => {
var found = false
if (elem == null) {
if (alreadyIncludeNull) {
found = true
} else {
alreadyIncludeNull = true
}
} else {
// check elem is already stored in arrayBuffer or not?
var j = 0
while (!found && j < arrayBuffer.size) {
val va = arrayBuffer(j)
if (va != null && ordering.equiv(va, elem)) {
found = true
}
j = j + 1
}
}
if (!found) {
if (arrayBuffer.length > ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH) {
ArrayBinaryLike.throwUnionLengthOverflowException(arrayBuffer.length)
}
arrayBuffer += elem
}
}))
new GenericArrayData(arrayBuffer)
}
}

/**
* Returns an array of the elements in the intersect of x and y, without duplicates
*/
Expand Down Expand Up @@ -4482,7 +4445,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 @@ -4693,7 +4656,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 @@ -4808,7 +4771,8 @@ 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.arrayFunctionWithElementsExceedLimitError(
prettyName, newArrayLength)
}

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

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

val newArray = new Array[Any](newArrayLength)
Expand All @@ -4866,7 +4831,8 @@ 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.arrayFunctionWithElementsExceedLimitError(
prettyName, newArrayLength)
}

val newArray = new Array[Any](newArrayLength)
Expand Down Expand Up @@ -4912,7 +4878,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.arrayFunctionWithElementsExceedLimitError(
| "$prettyName", $resLength);
|}
|
|$allocation
Expand Down Expand Up @@ -4949,7 +4916,8 @@ case class ArrayInsert(
|
| $resLength = java.lang.Math.abs($pos) + $baseOffset;
| if ($resLength > ${ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH}) {
| throw QueryExecutionErrors.createArrayWithElementsExceedLimitError($resLength);
| throw QueryExecutionErrors.arrayFunctionWithElementsExceedLimitError(
| "$prettyName", $resLength);
| }
|
| $allocation
Expand All @@ -4976,7 +4944,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
Loading

0 comments on commit 9827199

Please sign in to comment.