From e4cccc6bc7bdf3c270ffb9107c820e7044c174f0 Mon Sep 17 00:00:00 2001 From: Deepayan Patra Date: Wed, 15 Nov 2023 14:27:34 +0800 Subject: [PATCH 1/4] Cherry-pick + resolve conflict. --- .../expressions/collectionOperations.scala | 47 +++++++++++----- .../CollectionExpressionsSuite.scala | 56 +++++++++++++++++-- 2 files changed, 83 insertions(+), 20 deletions(-) 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 c4bf65bb8abcb..13e4a52d6ba7f 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 @@ -22,6 +22,7 @@ import java.util.Comparator import scala.collection.mutable import scala.reflect.ClassTag +import org.apache.spark.SparkException.internalError import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.{TypeCheckResult, TypeCoercion, UnresolvedAttribute, UnresolvedSeed} import org.apache.spark.sql.catalyst.expressions.ArraySortLike.NullOrder @@ -38,7 +39,6 @@ import org.apache.spark.sql.types._ import org.apache.spark.sql.util.SQLOpenHashSet import org.apache.spark.unsafe.UTF8StringBuilder import org.apache.spark.unsafe.array.ByteArrayMethods -import org.apache.spark.unsafe.array.ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH import org.apache.spark.unsafe.types.{ByteArray, CalendarInterval, UTF8String} /** @@ -2837,6 +2837,34 @@ case class Sequence( } object Sequence { + private def prettyName: String = "sequence" + + def sequenceLength(start: Long, stop: Long, step: Long): Int = { + try { + val delta = Math.subtractExact(stop, start) + if (delta == Long.MinValue && step == -1L) { + // We must special-case division of Long.MinValue by -1 to catch potential unchecked + // overflow in next operation. Division does not have a builtin overflow check. We + // previously special-case div-by-zero. + throw new ArithmeticException("Long overflow (Long.MinValue / -1)") + } + val len = if (stop == start) 1L else Math.addExact(1L, (delta / step)) + if (len > ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH) { + throw QueryExecutionErrors.createArrayWithElementsExceedLimitError(prettyName, len) + } + len.toInt + } catch { + // We handle overflows in the previous try block by raising an appropriate exception. + case _: ArithmeticException => + val safeLen = + BigInt(1) + (BigInt(stop) - BigInt(start)) / BigInt(step) + if (safeLen > ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH) { + throw QueryExecutionErrors.createArrayWithElementsExceedLimitError(prettyName, safeLen) + } + throw internalError("Unreachable code reached.") + case e: Exception => throw e + } + } private type LessThanOrEqualFn = (Any, Any) => Boolean @@ -3208,13 +3236,7 @@ object Sequence { || (estimatedStep == num.zero && start == stop), s"Illegal sequence boundaries: $start to $stop by $step") - val len = if (start == stop) 1L else 1L + (stop.toLong - start.toLong) / estimatedStep.toLong - - require( - len <= MAX_ROUNDED_ARRAY_LENGTH, - s"Too long sequence: $len. Should be <= $MAX_ROUNDED_ARRAY_LENGTH") - - len.toInt + sequenceLength(start.toLong, stop.toLong, estimatedStep.toLong) } private def genSequenceLengthCode( @@ -3224,7 +3246,7 @@ object Sequence { step: String, estimatedStep: String, len: String): String = { - val longLen = ctx.freshName("longLen") + val calcFn = classOf[Sequence].getName + ".sequenceLength" s""" |if (!(($estimatedStep > 0 && $start <= $stop) || | ($estimatedStep < 0 && $start >= $stop) || @@ -3232,12 +3254,7 @@ object Sequence { | throw new IllegalArgumentException( | "Illegal sequence boundaries: " + $start + " to " + $stop + " by " + $step); |} - |long $longLen = $stop == $start ? 1L : 1L + ((long) $stop - $start) / $estimatedStep; - |if ($longLen > $MAX_ROUNDED_ARRAY_LENGTH) { - | throw new IllegalArgumentException( - | "Too long sequence: " + $longLen + ". Should be <= $MAX_ROUNDED_ARRAY_LENGTH"); - |} - |int $len = (int) $longLen; + |int $len = $calcFn((long) $start, (long) $stop, (long) $estimatedStep); """.stripMargin } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala index c5f5425978430..354f0fc23818e 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala @@ -33,7 +33,7 @@ import org.apache.spark.sql.catalyst.util.DateTimeTestUtils.{outstandingZoneIds, import org.apache.spark.sql.catalyst.util.IntervalUtils._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ -import org.apache.spark.unsafe.array.ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH +import org.apache.spark.unsafe.array.ByteArrayMethods import org.apache.spark.unsafe.types.UTF8String class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { @@ -710,10 +710,6 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper // test sequence boundaries checking - checkExceptionInExpression[IllegalArgumentException]( - new Sequence(Literal(Int.MinValue), Literal(Int.MaxValue), Literal(1)), - EmptyRow, s"Too long sequence: 4294967296. Should be <= $MAX_ROUNDED_ARRAY_LENGTH") - checkExceptionInExpression[IllegalArgumentException]( new Sequence(Literal(1), Literal(2), Literal(0)), EmptyRow, "boundaries: 1 to 2 by 0") checkExceptionInExpression[IllegalArgumentException]( @@ -723,6 +719,56 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper checkExceptionInExpression[IllegalArgumentException]( new Sequence(Literal(1), Literal(2), Literal(-1)), EmptyRow, "boundaries: 1 to 2 by -1") + // SPARK-43393: test Sequence overflow checking + checkErrorInExpression[SparkRuntimeException]( + new Sequence(Literal(Int.MinValue), Literal(Int.MaxValue), Literal(1)), + errorClass = "COLLECTION_SIZE_LIMIT_EXCEEDED.PARAMETER", + parameters = Map( + "numberOfElements" -> (BigInt(Int.MaxValue) - BigInt { Int.MinValue } + 1).toString, + "functionName" -> toSQLId("sequence"), + "maxRoundedArrayLength" -> ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH.toString(), + "parameter" -> toSQLId("count"))) + checkErrorInExpression[SparkRuntimeException]( + new Sequence(Literal(0L), Literal(Long.MaxValue), Literal(1L)), + errorClass = "COLLECTION_SIZE_LIMIT_EXCEEDED.PARAMETER", + parameters = Map( + "numberOfElements" -> (BigInt(Long.MaxValue) + 1).toString, + "functionName" -> toSQLId("sequence"), + "maxRoundedArrayLength" -> ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH.toString(), + "parameter" -> toSQLId("count"))) + checkErrorInExpression[SparkRuntimeException]( + new Sequence(Literal(0L), Literal(Long.MinValue), Literal(-1L)), + errorClass = "COLLECTION_SIZE_LIMIT_EXCEEDED.PARAMETER", + parameters = Map( + "numberOfElements" -> ((0 - BigInt(Long.MinValue)) + 1).toString(), + "functionName" -> toSQLId("sequence"), + "maxRoundedArrayLength" -> ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH.toString(), + "parameter" -> toSQLId("count"))) + checkErrorInExpression[SparkRuntimeException]( + new Sequence(Literal(Long.MinValue), Literal(Long.MaxValue), Literal(1L)), + errorClass = "COLLECTION_SIZE_LIMIT_EXCEEDED.PARAMETER", + parameters = Map( + "numberOfElements" -> (BigInt(Long.MaxValue) - BigInt { Long.MinValue } + 1).toString, + "functionName" -> toSQLId("sequence"), + "maxRoundedArrayLength" -> ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH.toString(), + "parameter" -> toSQLId("count"))) + checkErrorInExpression[SparkRuntimeException]( + new Sequence(Literal(Long.MaxValue), Literal(Long.MinValue), Literal(-1L)), + errorClass = "COLLECTION_SIZE_LIMIT_EXCEEDED.PARAMETER", + parameters = Map( + "numberOfElements" -> (BigInt(Long.MaxValue) - BigInt { Long.MinValue } + 1).toString, + "functionName" -> toSQLId("sequence"), + "maxRoundedArrayLength" -> ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH.toString(), + "parameter" -> toSQLId("count"))) + checkErrorInExpression[SparkRuntimeException]( + new Sequence(Literal(Long.MaxValue), Literal(-1L), Literal(-1L)), + errorClass = "COLLECTION_SIZE_LIMIT_EXCEEDED.PARAMETER", + parameters = Map( + "numberOfElements" -> (BigInt(Long.MaxValue) - BigInt { -1L } + 1).toString, + "functionName" -> toSQLId("sequence"), + "maxRoundedArrayLength" -> ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH.toString(), + "parameter" -> toSQLId("count"))) + // test sequence with one element (zero step or equal start and stop) checkEvaluation(new Sequence(Literal(1), Literal(1), Literal(-1)), Seq(1)) From fdf9d65b6ed7dc5be3f1f6e6c4de4e260269da52 Mon Sep 17 00:00:00 2001 From: Deepayan Patra Date: Fri, 17 Nov 2023 01:21:57 -0800 Subject: [PATCH 2/4] Fixes. --- .../expressions/collectionOperations.scala | 4 +- .../CollectionExpressionsSuite.scala | 48 +++++++------------ 2 files changed, 20 insertions(+), 32 deletions(-) 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 13e4a52d6ba7f..e2fac2ba31cef 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 @@ -2850,7 +2850,7 @@ object Sequence { } val len = if (stop == start) 1L else Math.addExact(1L, (delta / step)) if (len > ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH) { - throw QueryExecutionErrors.createArrayWithElementsExceedLimitError(prettyName, len) + throw QueryExecutionErrors.createArrayWithElementsExceedLimitError(len) } len.toInt } catch { @@ -2859,7 +2859,7 @@ object Sequence { val safeLen = BigInt(1) + (BigInt(stop) - BigInt(start)) / BigInt(step) if (safeLen > ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH) { - throw QueryExecutionErrors.createArrayWithElementsExceedLimitError(prettyName, safeLen) + throw QueryExecutionErrors.createArrayWithElementsExceedLimitError(safeLen) } throw internalError("Unreachable code reached.") case e: Exception => throw e diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala index 354f0fc23818e..8af8c4ee68487 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala @@ -722,52 +722,40 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper // SPARK-43393: test Sequence overflow checking checkErrorInExpression[SparkRuntimeException]( new Sequence(Literal(Int.MinValue), Literal(Int.MaxValue), Literal(1)), - errorClass = "COLLECTION_SIZE_LIMIT_EXCEEDED.PARAMETER", + errorClass = "_LEGACY_ERROR_TEMP_2161", parameters = Map( - "numberOfElements" -> (BigInt(Int.MaxValue) - BigInt { Int.MinValue } + 1).toString, - "functionName" -> toSQLId("sequence"), - "maxRoundedArrayLength" -> ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH.toString(), - "parameter" -> toSQLId("count"))) + "count" -> (BigInt(Int.MaxValue) - BigInt { Int.MinValue } + 1).toString, + "maxRoundedArrayLength" -> ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH.toString())) checkErrorInExpression[SparkRuntimeException]( new Sequence(Literal(0L), Literal(Long.MaxValue), Literal(1L)), - errorClass = "COLLECTION_SIZE_LIMIT_EXCEEDED.PARAMETER", + errorClass = "_LEGACY_ERROR_TEMP_2161", parameters = Map( - "numberOfElements" -> (BigInt(Long.MaxValue) + 1).toString, - "functionName" -> toSQLId("sequence"), - "maxRoundedArrayLength" -> ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH.toString(), - "parameter" -> toSQLId("count"))) + "count" -> (BigInt(Long.MaxValue) + 1).toString, + "maxRoundedArrayLength" -> ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH.toString())) checkErrorInExpression[SparkRuntimeException]( new Sequence(Literal(0L), Literal(Long.MinValue), Literal(-1L)), - errorClass = "COLLECTION_SIZE_LIMIT_EXCEEDED.PARAMETER", + errorClass = "_LEGACY_ERROR_TEMP_2161", parameters = Map( - "numberOfElements" -> ((0 - BigInt(Long.MinValue)) + 1).toString(), - "functionName" -> toSQLId("sequence"), - "maxRoundedArrayLength" -> ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH.toString(), - "parameter" -> toSQLId("count"))) + "count" -> ((0 - BigInt(Long.MinValue)) + 1).toString(), + "maxRoundedArrayLength" -> ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH.toString())) checkErrorInExpression[SparkRuntimeException]( new Sequence(Literal(Long.MinValue), Literal(Long.MaxValue), Literal(1L)), - errorClass = "COLLECTION_SIZE_LIMIT_EXCEEDED.PARAMETER", + errorClass = "_LEGACY_ERROR_TEMP_2161", parameters = Map( - "numberOfElements" -> (BigInt(Long.MaxValue) - BigInt { Long.MinValue } + 1).toString, - "functionName" -> toSQLId("sequence"), - "maxRoundedArrayLength" -> ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH.toString(), - "parameter" -> toSQLId("count"))) + "count" -> (BigInt(Long.MaxValue) - BigInt { Long.MinValue } + 1).toString, + "maxRoundedArrayLength" -> ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH.toString())) checkErrorInExpression[SparkRuntimeException]( new Sequence(Literal(Long.MaxValue), Literal(Long.MinValue), Literal(-1L)), - errorClass = "COLLECTION_SIZE_LIMIT_EXCEEDED.PARAMETER", + errorClass = "_LEGACY_ERROR_TEMP_2161", parameters = Map( - "numberOfElements" -> (BigInt(Long.MaxValue) - BigInt { Long.MinValue } + 1).toString, - "functionName" -> toSQLId("sequence"), - "maxRoundedArrayLength" -> ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH.toString(), - "parameter" -> toSQLId("count"))) + "count" -> (BigInt(Long.MaxValue) - BigInt { Long.MinValue } + 1).toString, + "maxRoundedArrayLength" -> ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH.toString())) checkErrorInExpression[SparkRuntimeException]( new Sequence(Literal(Long.MaxValue), Literal(-1L), Literal(-1L)), - errorClass = "COLLECTION_SIZE_LIMIT_EXCEEDED.PARAMETER", + errorClass = "_LEGACY_ERROR_TEMP_2161", parameters = Map( - "numberOfElements" -> (BigInt(Long.MaxValue) - BigInt { -1L } + 1).toString, - "functionName" -> toSQLId("sequence"), - "maxRoundedArrayLength" -> ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH.toString(), - "parameter" -> toSQLId("count"))) + "count" -> (BigInt(Long.MaxValue) - BigInt { -1L } + 1).toString, + "maxRoundedArrayLength" -> ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH.toString())) // test sequence with one element (zero step or equal start and stop) From 48b8214c7dd88f4c6746ac3d35895ceb3f53a1d1 Mon Sep 17 00:00:00 2001 From: Deepayan Patra Date: Fri, 17 Nov 2023 01:52:35 -0800 Subject: [PATCH 3/4] Fixes. --- .../expressions/collectionOperations.scala | 3 +- .../CollectionExpressionsSuite.scala | 72 +++++++++++-------- 2 files changed, 43 insertions(+), 32 deletions(-) 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 e2fac2ba31cef..9bd2e001ea0c3 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 @@ -22,7 +22,6 @@ import java.util.Comparator import scala.collection.mutable import scala.reflect.ClassTag -import org.apache.spark.SparkException.internalError import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.{TypeCheckResult, TypeCoercion, UnresolvedAttribute, UnresolvedSeed} import org.apache.spark.sql.catalyst.expressions.ArraySortLike.NullOrder @@ -2861,7 +2860,7 @@ object Sequence { if (safeLen > ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH) { throw QueryExecutionErrors.createArrayWithElementsExceedLimitError(safeLen) } - throw internalError("Unreachable code reached.") + throw QueryExecutionErrors.unreachableError("Unreachable code reached in `sequence`.") case e: Exception => throw e } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala index 8af8c4ee68487..1988ccea2a8f5 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala @@ -720,42 +720,54 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper new Sequence(Literal(1), Literal(2), Literal(-1)), EmptyRow, "boundaries: 1 to 2 by -1") // SPARK-43393: test Sequence overflow checking - checkErrorInExpression[SparkRuntimeException]( + checkExceptionInExpression[RuntimeException]( new Sequence(Literal(Int.MinValue), Literal(Int.MaxValue), Literal(1)), - errorClass = "_LEGACY_ERROR_TEMP_2161", - parameters = Map( - "count" -> (BigInt(Int.MaxValue) - BigInt { Int.MinValue } + 1).toString, - "maxRoundedArrayLength" -> ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH.toString())) - checkErrorInExpression[SparkRuntimeException]( + EmptyRow, + s""" + |Unsuccessful try to create array with ${BigInt(Int.MaxValue) - BigInt { Int.MinValue } + 1} + |elements due to exceeding the array size limit + |${ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH}. + """.stripMargin.replaceAll("\n", " ")) + checkExceptionInExpression[RuntimeException]( new Sequence(Literal(0L), Literal(Long.MaxValue), Literal(1L)), - errorClass = "_LEGACY_ERROR_TEMP_2161", - parameters = Map( - "count" -> (BigInt(Long.MaxValue) + 1).toString, - "maxRoundedArrayLength" -> ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH.toString())) - checkErrorInExpression[SparkRuntimeException]( + EmptyRow, + s""" + |Unsuccessful try to create array with ${BigInt(Long.MaxValue) + 1} + |elements due to exceeding the array size limit + |${ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH}. + """.stripMargin.replaceAll("\n", " ")) + checkExceptionInExpression[RuntimeException]( new Sequence(Literal(0L), Literal(Long.MinValue), Literal(-1L)), - errorClass = "_LEGACY_ERROR_TEMP_2161", - parameters = Map( - "count" -> ((0 - BigInt(Long.MinValue)) + 1).toString(), - "maxRoundedArrayLength" -> ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH.toString())) - checkErrorInExpression[SparkRuntimeException]( + EmptyRow, + s""" + |Unsuccessful try to create array with ${(0 - BigInt(Long.MinValue)) + 1} + |elements due to exceeding the array size limit + |${ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH}. + """.stripMargin.replaceAll("\n", " ")) + checkExceptionInExpression[RuntimeException]( new Sequence(Literal(Long.MinValue), Literal(Long.MaxValue), Literal(1L)), - errorClass = "_LEGACY_ERROR_TEMP_2161", - parameters = Map( - "count" -> (BigInt(Long.MaxValue) - BigInt { Long.MinValue } + 1).toString, - "maxRoundedArrayLength" -> ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH.toString())) - checkErrorInExpression[SparkRuntimeException]( + EmptyRow, + s""" + |Unsuccessful try to create array with ${BigInt(Long.MaxValue) - BigInt { Long.MinValue } + 1} + |elements due to exceeding the array size limit + |${ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH}. + """.stripMargin.replaceAll("\n", " ")) + checkExceptionInExpression[RuntimeException]( new Sequence(Literal(Long.MaxValue), Literal(Long.MinValue), Literal(-1L)), - errorClass = "_LEGACY_ERROR_TEMP_2161", - parameters = Map( - "count" -> (BigInt(Long.MaxValue) - BigInt { Long.MinValue } + 1).toString, - "maxRoundedArrayLength" -> ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH.toString())) - checkErrorInExpression[SparkRuntimeException]( + EmptyRow, + s""" + |Unsuccessful try to create array with ${BigInt(Long.MaxValue) - BigInt { Long.MinValue } + 1} + |elements due to exceeding the array size limit + |${ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH}. + """.stripMargin.replaceAll("\n", " ")) + checkExceptionInExpression[RuntimeException]( new Sequence(Literal(Long.MaxValue), Literal(-1L), Literal(-1L)), - errorClass = "_LEGACY_ERROR_TEMP_2161", - parameters = Map( - "count" -> (BigInt(Long.MaxValue) - BigInt { -1L } + 1).toString, - "maxRoundedArrayLength" -> ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH.toString())) + EmptyRow, + s""" + |Unsuccessful try to create array with ${BigInt(Long.MaxValue) - BigInt { -1L } + 1} + |elements due to exceeding the array size limit + |${ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH}. + """.stripMargin.replaceAll("\n", " ")) // test sequence with one element (zero step or equal start and stop) From 12484855de1ea95ae17161e00800882e5406896a Mon Sep 17 00:00:00 2001 From: Deepayan Patra Date: Fri, 17 Nov 2023 15:13:06 -0800 Subject: [PATCH 4/4] Fixes. --- .../expressions/CollectionExpressionsSuite.scala | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala index 1988ccea2a8f5..25025cf505879 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala @@ -724,7 +724,8 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper new Sequence(Literal(Int.MinValue), Literal(Int.MaxValue), Literal(1)), EmptyRow, s""" - |Unsuccessful try to create array with ${BigInt(Int.MaxValue) - BigInt { Int.MinValue } + 1} + |Unsuccessful try to create array with + |${BigInt(Int.MaxValue) - BigInt { Int.MinValue } + 1} |elements due to exceeding the array size limit |${ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH}. """.stripMargin.replaceAll("\n", " ")) @@ -748,7 +749,8 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper new Sequence(Literal(Long.MinValue), Literal(Long.MaxValue), Literal(1L)), EmptyRow, s""" - |Unsuccessful try to create array with ${BigInt(Long.MaxValue) - BigInt { Long.MinValue } + 1} + |Unsuccessful try to create array with + |${BigInt(Long.MaxValue) - BigInt { Long.MinValue } + 1} |elements due to exceeding the array size limit |${ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH}. """.stripMargin.replaceAll("\n", " ")) @@ -756,7 +758,8 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper new Sequence(Literal(Long.MaxValue), Literal(Long.MinValue), Literal(-1L)), EmptyRow, s""" - |Unsuccessful try to create array with ${BigInt(Long.MaxValue) - BigInt { Long.MinValue } + 1} + |Unsuccessful try to create array with + |${BigInt(Long.MaxValue) - BigInt { Long.MinValue } + 1} |elements due to exceeding the array size limit |${ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH}. """.stripMargin.replaceAll("\n", " "))