This is an automated email from the ASF dual-hosted git repository.

maxgekk pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
     new 9cbc2d138a1 [SPARK-45710][SQL] Assign names to error 
_LEGACY_ERROR_TEMP_21[59,60,61,62]
9cbc2d138a1 is described below

commit 9cbc2d138a1de2d3ff399d224d817554ba0b9e18
Author: dengziming <dengziming1...@gmail.com>
AuthorDate: Sun Nov 5 20:07:07 2023 +0300

    [SPARK-45710][SQL] Assign names to error _LEGACY_ERROR_TEMP_21[59,60,61,62]
    
    ### What changes were proposed in this pull request?
    This PR are removing `_LEGACY_ERROR_TEMP_21[59,60,61,62]` and 
`TOO_MANY_ARRAY_ELEMENTS`:
    1. `_LEGACY_ERROR_TEMP_2159` is used in concat/array_insert;
    2. `_LEGACY_ERROR_TEMP_2160` is only used in flatten;
    3. `_LEGACY_ERROR_TEMP_2161` is used in 
array_repeat/array_insert/array_distinct/array_union/array_intersect/array_remove;
    4. `_LEGACY_ERROR_TEMP_2162` is used in array_union/array_distinct;
    5. There is another similar error class `TOO_MANY_ARRAY_ELEMENTS` which are 
used in `UnsafeArrayWriter.java`.
    
    I removed these 5 similar error classes and create a new error class 
`COLLECTION_SIZE_LIMIT_EXCEEDED` with 3 sub-classes:
    1. `PARAMETER` is used when the parameter exceed size limit, such as 
`array_repeat` with count too large;
    6. `FUNCTION` is used when trying to create an array exceeding size limit 
in a function, for example, flatten 2 arrays to a larger array;
    7. `INITIALIZE` is used in `UnsafeArrayWriter.java` when trying to 
initialize an array exceeding size limit.
    
    ### Why are the changes needed?
    To assign proper name as a part of activity in SPARK-37935.
    
    ### Does this PR introduce _any_ user-facing change?
    Yes, the error message will include the error class name.
    
    ### How was this patch tested?
    1. `COLLECTION_SIZE_LIMIT_EXCEEDED.PARAMETER` can be tested from use code;
    2.  `COLLECTION_SIZE_LIMIT_EXCEEDED.FUNCTION` is tested using a 
`ColumnarArray` in `concat/flatten`, but can't be tested in 
`array_insert/array_distinct/array_union/array_intersect/array_remove` since we 
need to deduplicate the data and create an array which will cause OOM.
    3. `INITIALIZE` is already tested in a existing case.
    
    ### Was this patch authored or co-authored using generative AI tooling?
    No.
    
    Closes #43567 from dengziming/SPARK-45710.
    
    Authored-by: dengziming <dengziming1...@gmail.com>
    Signed-off-by: Max Gekk <max.g...@gmail.com>
---
 .../src/main/resources/error/error-classes.json    | 49 ++++++-----
 docs/_data/menu-sql.yaml                           |  2 +
 ...s-collection-size-limit-exceeded-error-class.md | 38 +++++++++
 docs/sql-error-conditions.md                       | 14 ++--
 .../expressions/codegen/UnsafeArrayWriter.java     |  3 +-
 .../expressions/collectionOperations.scala         | 95 ++++++++--------------
 .../spark/sql/errors/QueryExecutionErrors.scala    | 48 +++++------
 .../codegen/UnsafeArrayWriterSuite.scala           |  6 +-
 .../sql/errors/QueryExecutionErrorsSuite.scala     | 59 +++++++++++++-
 9 files changed, 184 insertions(+), 130 deletions(-)

diff --git a/common/utils/src/main/resources/error/error-classes.json 
b/common/utils/src/main/resources/error/error-classes.json
index 8b0951a7b00..3e0743d366a 100644
--- a/common/utils/src/main/resources/error/error-classes.json
+++ b/common/utils/src/main/resources/error/error-classes.json
@@ -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 the function <functionName>."
+        ]
+      },
+      "INITIALIZE" : {
+        "message" : [
+          "cannot initialize an array with specified parameters."
+        ]
+      },
+      "PARAMETER" : {
+        "message" : [
+          "the value of parameter(s) <parameter> in the function 
<functionName> is invalid."
+        ]
+      }
+    },
+    "sqlState" : "54000"
+  },
   "COLUMN_ALIASES_IS_NOT_ALLOWED" : {
     "message" : [
       "Columns aliases are not allowed in <op>."
@@ -3017,12 +3040,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.",
@@ -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>."
diff --git a/docs/_data/menu-sql.yaml b/docs/_data/menu-sql.yaml
index 41258606422..95833849cc5 100644
--- a/docs/_data/menu-sql.yaml
+++ b/docs/_data/menu-sql.yaml
@@ -109,6 +109,8 @@
   subitems:
     - text: SQLSTATE Codes
       url: sql-error-conditions-sqlstates.html
+    - text: COLLECTION_SIZE_LIMIT_EXCEEDED error class
+      url: sql-error-conditions-collection-size-limit-exceeded-error-class.html
     - text: CONNECT error class
       url: sql-error-conditions-connect-error-class.html
     - text: DATATYPE_MISMATCH error class
diff --git 
a/docs/sql-error-conditions-collection-size-limit-exceeded-error-class.md 
b/docs/sql-error-conditions-collection-size-limit-exceeded-error-class.md
new file mode 100644
index 00000000000..78b9e438261
--- /dev/null
+++ b/docs/sql-error-conditions-collection-size-limit-exceeded-error-class.md
@@ -0,0 +1,38 @@
+---
+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](sql-error-conditions-sqlstates.html#class-54-program-limit-exceeded)
+
+Can't create array with `<numberOfElements>` elements which exceeding the 
array size limit `<maxRoundedArrayLength>`,
+
+This error class has the following derived error classes:
+
+## FUNCTION
+
+unsuccessful try to create arrays in the function `<functionName>`.
+
+## INITIALIZE
+
+cannot initialize an array with specified parameters.
+
+## PARAMETER
+
+the value of parameter(s) `<parameter>` in the function `<functionName>` is 
invalid.
diff --git a/docs/sql-error-conditions.md b/docs/sql-error-conditions.md
index 1741e49f561..a6f003647dd 100644
--- a/docs/sql-error-conditions.md
+++ b/docs/sql-error-conditions.md
@@ -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-conditions-collection-size-limit-exceeded-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-conditions-collection-size-limit-exceeded-error-class.html)
+
 ### COLUMN_ALIASES_IS_NOT_ALLOWED
 
 [SQLSTATE: 
42601](sql-error-conditions-sqlstates.html#class-42-syntax-error-or-access-rule-violation)
@@ -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)
diff --git 
a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeArrayWriter.java
 
b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeArrayWriter.java
index 65d984bcd19..3070fa3e74b 100644
--- 
a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeArrayWriter.java
+++ 
b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeArrayWriter.java
@@ -62,7 +62,8 @@ public final class UnsafeArrayWriter extends UnsafeWriter {
     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
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 25da787b887..196c4a6cdd6 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
@@ -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
@@ -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
@@ -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))
@@ -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)) {
@@ -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)
@@ -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 {
@@ -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
  */
@@ -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)
@@ -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
           }
@@ -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 {
@@ -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
  */
@@ -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 {
@@ -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 {
@@ -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)
@@ -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)
@@ -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)
@@ -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
@@ -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
@@ -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
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 3a60480cfc5..0d315f526d9 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
@@ -1400,36 +1400,27 @@ private[sql] object QueryExecutionErrors extends 
QueryErrorsBase with ExecutionE
       summary = getSummary(context))
   }
 
-  def concatArraysWithElementsExceedLimitError(numberOfElements: Long): 
SparkRuntimeException = {
+  def arrayFunctionWithElementsExceedLimitError(
+    prettyName: String, numberOfElements: Long): SparkRuntimeException = {
     new SparkRuntimeException(
-      errorClass = "_LEGACY_ERROR_TEMP_2159",
+      errorClass = "COLLECTION_SIZE_LIMIT_EXCEEDED.FUNCTION",
       messageParameters = Map(
         "numberOfElements" -> numberOfElements.toString(),
-        "maxRoundedArrayLength" -> 
ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH.toString()))
-  }
-
-  def flattenArraysWithElementsExceedLimitError(numberOfElements: Long): 
SparkRuntimeException = {
-    new SparkRuntimeException(
-      errorClass = "_LEGACY_ERROR_TEMP_2160",
-      messageParameters = Map(
-        "numberOfElements" -> numberOfElements.toString(),
-        "maxRoundedArrayLength" -> 
ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH.toString()))
-  }
-
-  def createArrayWithElementsExceedLimitError(count: Any): 
SparkRuntimeException = {
-    new SparkRuntimeException(
-      errorClass = "_LEGACY_ERROR_TEMP_2161",
-      messageParameters = Map(
-        "count" -> count.toString(),
-        "maxRoundedArrayLength" -> 
ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH.toString()))
+        "maxRoundedArrayLength" -> 
ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH.toString(),
+        "functionName" -> toSQLId(prettyName)
+      ))
   }
 
-  def unionArrayWithElementsExceedLimitError(length: Int): 
SparkRuntimeException = {
+  def createArrayWithElementsExceedLimitError(
+    prettyName: String, count: Any): SparkRuntimeException = {
     new SparkRuntimeException(
-      errorClass = "_LEGACY_ERROR_TEMP_2162",
+      errorClass = "COLLECTION_SIZE_LIMIT_EXCEEDED.PARAMETER",
       messageParameters = Map(
-        "length" -> length.toString(),
-        "maxRoundedArrayLength" -> 
ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH.toString()))
+        "numberOfElements" -> count.toString,
+        "functionName" -> toSQLId(prettyName),
+        "maxRoundedArrayLength" -> 
ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH.toString(),
+        "parameter" -> toSQLId("count")
+      ))
   }
 
   def initialTypeNotTargetDataTypeError(
@@ -2567,13 +2558,14 @@ private[sql] object QueryExecutionErrors extends 
QueryErrorsBase with ExecutionE
   }
 
   def tooManyArrayElementsError(
-      numElements: Int,
-      elementSize: Int): SparkIllegalArgumentException = {
+      numElements: Long,
+      maxRoundedArrayLength: Int): SparkIllegalArgumentException = {
     new SparkIllegalArgumentException(
-      errorClass = "TOO_MANY_ARRAY_ELEMENTS",
+      errorClass = "COLLECTION_SIZE_LIMIT_EXCEEDED.INITIALIZE",
       messageParameters = Map(
-        "numElements" -> numElements.toString,
-        "size" -> elementSize.toString))
+        "numberOfElements" -> numElements.toString,
+        "maxRoundedArrayLength" -> maxRoundedArrayLength.toString)
+    )
   }
 
   def invalidEmptyLocationError(location: String): 
SparkIllegalArgumentException = {
diff --git 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeArrayWriterSuite.scala
 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeArrayWriterSuite.scala
index f10fb0754f5..a968b1fe535 100644
--- 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeArrayWriterSuite.scala
+++ 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeArrayWriterSuite.scala
@@ -30,10 +30,10 @@ class UnsafeArrayWriterSuite extends SparkFunSuite {
       exception = intercept[SparkIllegalArgumentException] {
         arrayWriter.initialize(numElements)
       },
-      errorClass = "TOO_MANY_ARRAY_ELEMENTS",
+      errorClass = "COLLECTION_SIZE_LIMIT_EXCEEDED.INITIALIZE",
       parameters = Map(
-        "numElements" -> numElements.toString,
-        "size" -> elementSize.toString
+        "numberOfElements" -> (numElements * elementSize).toString,
+        "maxRoundedArrayLength" -> Int.MaxValue.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 dd3f3dc6004..a49352cbe50 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
@@ -33,29 +33,32 @@ 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
 import org.apache.spark.sql.catalyst.rules.RuleIdCollection
 import org.apache.spark.sql.catalyst.util.BadRecordException
-import org.apache.spark.sql.errors.DataTypeErrorsBase
 import org.apache.spark.sql.execution.datasources.jdbc.{DriverRegistry, 
JDBCOptions}
 import 
org.apache.spark.sql.execution.datasources.jdbc.connection.ConnectionProvider
 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
@@ -1095,6 +1098,56 @@ 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 = "COLLECTION_SIZE_LIMIT_EXCEEDED.FUNCTION",
+      parameters = Map(
+        "numberOfElements" -> Int.MaxValue.toString,
+        "maxRoundedArrayLength" -> MAX_ROUNDED_ARRAY_LENGTH.toString,
+        "functionName" -> toSQLId("concat")
+      )
+    )
+  }
+
+  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 = "COLLECTION_SIZE_LIMIT_EXCEEDED.FUNCTION",
+      parameters = Map(
+        "numberOfElements" -> Int.MaxValue.toString,
+        "maxRoundedArrayLength" -> MAX_ROUNDED_ARRAY_LENGTH.toString,
+        "functionName" -> toSQLId("flatten")
+      )
+    )
+  }
+
+  test("Elements exceed limit for array_repeat()") {
+    val count = 2147483647
+    checkError(
+      exception = intercept[SparkRuntimeException] {
+        sql(s"select array_repeat(1, $count)").collect()
+      },
+      errorClass = "COLLECTION_SIZE_LIMIT_EXCEEDED.PARAMETER",
+      parameters = Map(
+        "parameter" -> toSQLId("count"),
+        "numberOfElements" -> count.toString,
+        "functionName" -> toSQLId("array_repeat"),
+        "maxRoundedArrayLength" -> MAX_ROUNDED_ARRAY_LENGTH.toString
+      )
+    )
+  }
 }
 
 class FakeFileSystemSetPermission extends LocalFileSystem {


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org

Reply via email to