You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by "MaxGekk (via GitHub)" <gi...@apache.org> on 2023/11/04 09:36:33 UTC

Re: [PR] [SPARK-45710][SQL] Assign names to error _LEGACY_ERROR_TEMP_21[59,60,61,62] [spark]

MaxGekk commented on code in PR #43567:
URL: https://github.com/apache/spark/pull/43567#discussion_r1382362098


##########
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 function <functionName>."
+        ]
+      },
+      "INITIALIZE" : {
+        "message" : [
+          "cannot initialize array with specified parameters."

Review Comment:
   ```suggestion
             "cannot initialize an array with specified parameters."
   ```



##########
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 function <functionName>."
+        ]
+      },
+      "INITIALIZE" : {
+        "message" : [
+          "cannot initialize array with specified parameters."
+        ]
+      },
+      "PARAMETER" : {
+        "message" : [
+          "the value of parameter(s) <parameter> in <functionName> is invalid."

Review Comment:
   ```suggestion
             "the value of parameter(s) <parameter> in the function <functionName> is invalid."
   ```



##########
sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala:
##########
@@ -1095,6 +1098,55 @@ 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()") {
+    checkError(
+      exception = intercept[SparkRuntimeException] {
+        sql("select array_repeat(1, 2147483647)").collect()
+      },
+      errorClass = "COLLECTION_SIZE_LIMIT_EXCEEDED.PARAMETER",
+      parameters = Map(
+        "parameter" -> toSQLId("count"),
+        "numberOfElements" -> "2147483647",

Review Comment:
   nit: place to a val if it is the same as in `sql`



##########
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 function <functionName>."

Review Comment:
   ```suggestion
             "unsuccessful try to create arrays in the function <functionName>."
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org