Skip to content

Commit 64fa13b

Browse files
committed
[SPARK-47060][SQL][TESTS] Check SparkIllegalArgumentException instead of IllegalArgumentException in catalyst
### What changes were proposed in this pull request? In the PR, I propose to use `checkError()` in tests of `catalyst` to check `SparkIllegalArgumentException`, and its fields. ### Why are the changes needed? By checking `SparkIllegalArgumentException` and its fields like error class and message parameters prevents replacing `SparkIllegalArgumentException` back to `IllegalArgumentException`. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? By running the modified test suites. ### Was this patch authored or co-authored using generative AI tooling? No. Closes apache#45118 from MaxGekk/migrate-IllegalArgumentException-catalyst-tests. Authored-by: Max Gekk <[email protected]> Signed-off-by: Max Gekk <[email protected]>
1 parent 61e25e1 commit 64fa13b

File tree

17 files changed

+238
-174
lines changed

17 files changed

+238
-174
lines changed

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -22,7 +22,7 @@ import java.util.concurrent.TimeUnit
2222

2323
import scala.util.control.NonFatal
2424

25-
import org.apache.spark.SparkIllegalArgumentException
25+
import org.apache.spark.{SparkIllegalArgumentException, SparkThrowable}
2626
import org.apache.spark.sql.catalyst.expressions.Literal
2727
import org.apache.spark.sql.catalyst.parser.CatalystSqlParser
2828
import org.apache.spark.sql.catalyst.util.DateTimeConstants._
@@ -202,6 +202,7 @@ object IntervalUtils extends SparkIntervalUtils {
202202
try {
203203
f
204204
} catch {
205+
case e: SparkThrowable => throw e
205206
case NonFatal(e) =>
206207
throw new SparkIllegalArgumentException(
207208
errorClass = "_LEGACY_ERROR_TEMP_3213",

sql/catalyst/src/test/scala/org/apache/spark/sql/RowJsonSuite.scala

Lines changed: 13 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -21,7 +21,7 @@ import java.time.LocalDate
2121

2222
import org.json4s.JsonAST.{JArray, JBool, JDecimal, JDouble, JLong, JNull, JObject, JString, JValue}
2323

24-
import org.apache.spark.SparkFunSuite
24+
import org.apache.spark.{SparkFunSuite, SparkIllegalArgumentException}
2525
import org.apache.spark.sql.catalyst.encoders.{ExamplePoint, ExamplePointUDT}
2626
import org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema
2727
import org.apache.spark.sql.types._
@@ -128,12 +128,17 @@ class RowJsonSuite extends SparkFunSuite {
128128
}
129129

130130
test("unsupported type") {
131-
val e = intercept[IllegalArgumentException] {
132-
val row = new GenericRowWithSchema(
133-
Array((1, 2)),
134-
new StructType().add("a", ObjectType(classOf[(Int, Int)])))
135-
row.jsonValue
136-
}
137-
assert(e.getMessage.contains("Failed to convert value"))
131+
checkError(
132+
exception = intercept[SparkIllegalArgumentException] {
133+
val row = new GenericRowWithSchema(
134+
Array((1, 2)),
135+
new StructType().add("a", ObjectType(classOf[(Int, Int)])))
136+
row.jsonValue
137+
},
138+
errorClass = "_LEGACY_ERROR_TEMP_3249",
139+
parameters = Map(
140+
"value" -> "(1,2)",
141+
"valueClass" -> "class scala.Tuple2$mcII$sp",
142+
"dataType" -> "ObjectType(class scala.Tuple2)"))
138143
}
139144
}

sql/catalyst/src/test/scala/org/apache/spark/sql/RowTest.scala

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -24,7 +24,7 @@ import org.scalatest.funspec.AnyFunSpec
2424
import org.scalatest.matchers.must.Matchers
2525
import org.scalatest.matchers.should.Matchers._
2626

27-
import org.apache.spark.{SparkException, SparkUnsupportedOperationException}
27+
import org.apache.spark.{SparkException, SparkIllegalArgumentException, SparkUnsupportedOperationException}
2828
import org.apache.spark.sql.catalyst.InternalRow
2929
import org.apache.spark.sql.catalyst.expressions.{GenericRow, GenericRowWithSchema}
3030
import org.apache.spark.sql.types._
@@ -66,7 +66,7 @@ class RowTest extends AnyFunSpec with Matchers {
6666
}
6767

6868
it("Accessing non existent field throws an exception") {
69-
intercept[IllegalArgumentException] {
69+
intercept[SparkIllegalArgumentException] {
7070
sampleRow.getAs[String]("non_existent")
7171
}
7272
}

sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/CSVExprUtilsSuite.scala

Lines changed: 5 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -76,7 +76,7 @@ class CSVExprUtilsSuite extends SparkFunSuite {
7676
// backslash, then tab
7777
("""\\t""", Some("""\t"""), None),
7878
// invalid special character (dot)
79-
("""\.""", None, Some("Unsupported special character for delimiter")),
79+
("""\.""", None, Some("_LEGACY_ERROR_TEMP_3236")),
8080
// backslash, then dot
8181
("""\\.""", Some("""\."""), None),
8282
// nothing special, just straight conversion
@@ -90,17 +90,16 @@ class CSVExprUtilsSuite extends SparkFunSuite {
9090
)
9191

9292
test("should correctly produce separator strings, or exceptions, from input") {
93-
forAll(testCases) { (input, separatorStr, expectedErrorMsg) =>
93+
forAll(testCases) { (input, separatorStr, expectedErrorClass) =>
9494
try {
9595
val separator = CSVExprUtils.toDelimiterStr(input)
9696
assert(separatorStr.isDefined)
97-
assert(expectedErrorMsg.isEmpty)
97+
assert(expectedErrorClass.isEmpty)
9898
assert(separator.equals(separatorStr.get))
9999
} catch {
100-
case e: IllegalArgumentException =>
100+
case e: SparkIllegalArgumentException =>
101101
assert(separatorStr.isEmpty)
102-
assert(expectedErrorMsg.isDefined)
103-
assert(e.getMessage.contains(expectedErrorMsg.get))
102+
assert(e.getErrorClass === expectedErrorClass.get)
104103
}
105104
}
106105
}

sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/UnivocityParserSuite.scala

Lines changed: 24 additions & 18 deletions
Original file line numberDiff line numberDiff line change
@@ -23,7 +23,7 @@ import java.util.{Locale, TimeZone}
2323

2424
import org.apache.commons.lang3.time.FastDateFormat
2525

26-
import org.apache.spark.SparkFunSuite
26+
import org.apache.spark.{SparkFunSuite, SparkIllegalArgumentException}
2727
import org.apache.spark.sql.catalyst.InternalRow
2828
import org.apache.spark.sql.catalyst.plans.SQLHelper
2929
import org.apache.spark.sql.catalyst.util.DateTimeConstants._
@@ -304,19 +304,23 @@ class UnivocityParserSuite extends SparkFunSuite with SQLHelper {
304304
filters = Seq(EqualTo("d", 3.14)),
305305
expected = Some(InternalRow(1, 3.14)))
306306

307-
val errMsg = intercept[IllegalArgumentException] {
308-
check(filters = Seq(EqualTo("invalid attr", 1)), expected = None)
309-
}.getMessage
310-
assert(errMsg.contains("invalid attr does not exist"))
311-
312-
val errMsg2 = intercept[IllegalArgumentException] {
313-
check(
314-
dataSchema = new StructType(),
315-
requiredSchema = new StructType(),
316-
filters = Seq(EqualTo("i", 1)),
317-
expected = Some(InternalRow.empty))
318-
}.getMessage
319-
assert(errMsg2.contains("i does not exist"))
307+
checkError(
308+
exception = intercept[SparkIllegalArgumentException] {
309+
check(filters = Seq(EqualTo("invalid attr", 1)), expected = None)
310+
},
311+
errorClass = "_LEGACY_ERROR_TEMP_3252",
312+
parameters = Map("name" -> "invalid attr", "fieldNames" -> "i"))
313+
314+
checkError(
315+
exception = intercept[SparkIllegalArgumentException] {
316+
check(
317+
dataSchema = new StructType(),
318+
requiredSchema = new StructType(),
319+
filters = Seq(EqualTo("i", 1)),
320+
expected = Some(InternalRow.empty))
321+
},
322+
errorClass = "_LEGACY_ERROR_TEMP_3252",
323+
parameters = Map("name" -> "i", "fieldNames" -> ""))
320324
}
321325

322326
test("SPARK-30960: parse date/timestamp string with legacy format") {
@@ -366,9 +370,11 @@ class UnivocityParserSuite extends SparkFunSuite with SQLHelper {
366370
check(new UnivocityParser(StructType(Seq.empty), optionsWithPattern(true)))
367371

368372
// With legacy parser disabled, parsing results in error.
369-
val err = intercept[IllegalArgumentException] {
370-
check(new UnivocityParser(StructType(Seq.empty), optionsWithPattern(false)))
371-
}
372-
assert(err.getMessage.contains("Illegal pattern character: n"))
373+
checkError(
374+
exception = intercept[SparkIllegalArgumentException] {
375+
check(new UnivocityParser(StructType(Seq.empty), optionsWithPattern(false)))
376+
},
377+
errorClass = "_LEGACY_ERROR_TEMP_3258",
378+
parameters = Map("c" -> "n"))
373379
}
374380
}

sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuiteBase.scala

Lines changed: 42 additions & 31 deletions
Original file line numberDiff line numberDiff line change
@@ -22,7 +22,7 @@ import java.time.{Duration, LocalDate, LocalDateTime, Period}
2222
import java.time.temporal.ChronoUnit
2323
import java.util.{Calendar, Locale, TimeZone}
2424

25-
import org.apache.spark.SparkFunSuite
25+
import org.apache.spark.{SparkFunSuite, SparkIllegalArgumentException}
2626
import org.apache.spark.sql.Row
2727
import org.apache.spark.sql.catalyst.InternalRow
2828
import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.DataTypeMismatch
@@ -1105,9 +1105,10 @@ abstract class CastSuiteBase extends SparkFunSuite with ExpressionEvalHelper {
11051105

11061106
Seq("INTERVAL '-178956970-9' YEAR TO MONTH", "INTERVAL '178956970-8' YEAR TO MONTH")
11071107
.foreach { interval =>
1108-
checkExceptionInExpression[IllegalArgumentException](
1108+
checkErrorInExpression[SparkIllegalArgumentException](
11091109
cast(Literal.create(interval), YearMonthIntervalType()),
1110-
"Error parsing interval year-month string: integer overflow")
1110+
"_LEGACY_ERROR_TEMP_3213",
1111+
Map("interval" -> "year-month", "msg" -> "integer overflow"))
11111112
}
11121113

11131114
Seq(Byte.MaxValue, Short.MaxValue, Int.MaxValue, Int.MinValue + 1, Int.MinValue)
@@ -1173,13 +1174,15 @@ abstract class CastSuiteBase extends SparkFunSuite with ExpressionEvalHelper {
11731174

11741175
Seq("INTERVAL '1-1' YEAR", "INTERVAL '1-1' MONTH").foreach { interval =>
11751176
val dataType = YearMonthIntervalType()
1176-
val expectedMsg = s"Interval string does not match year-month format of " +
1177-
s"${IntervalUtils.supportedFormat((dataType.startField, dataType.endField))
1178-
.map(format => s"`$format`").mkString(", ")} " +
1179-
s"when cast to ${dataType.typeName}: $interval"
1180-
checkExceptionInExpression[IllegalArgumentException](
1177+
checkErrorInExpression[SparkIllegalArgumentException](
11811178
cast(Literal.create(interval), dataType),
1182-
expectedMsg
1179+
"_LEGACY_ERROR_TEMP_3214",
1180+
Map(
1181+
"fallBackNotice" -> "",
1182+
"typeName" -> "interval year to month",
1183+
"intervalStr" -> "year-month",
1184+
"supportedFormat" -> "`[+|-]d h`, `INTERVAL [+|-]'[+|-]d h' DAY TO HOUR`",
1185+
"input" -> interval)
11831186
)
11841187
}
11851188
Seq(("1", YearMonthIntervalType(YEAR, MONTH)),
@@ -1193,13 +1196,17 @@ abstract class CastSuiteBase extends SparkFunSuite with ExpressionEvalHelper {
11931196
("INTERVAL '1' MONTH", YearMonthIntervalType(YEAR)),
11941197
("INTERVAL '1' MONTH", YearMonthIntervalType(YEAR, MONTH)))
11951198
.foreach { case (interval, dataType) =>
1196-
val expectedMsg = s"Interval string does not match year-month format of " +
1197-
s"${IntervalUtils.supportedFormat((dataType.startField, dataType.endField))
1198-
.map(format => s"`$format`").mkString(", ")} " +
1199-
s"when cast to ${dataType.typeName}: $interval"
1200-
checkExceptionInExpression[IllegalArgumentException](
1199+
checkErrorInExpression[SparkIllegalArgumentException](
12011200
cast(Literal.create(interval), dataType),
1202-
expectedMsg)
1201+
"_LEGACY_ERROR_TEMP_3214",
1202+
Map(
1203+
"fallBackNotice" -> "",
1204+
"typeName" -> dataType.typeName,
1205+
"intervalStr" -> "year-month",
1206+
"supportedFormat" ->
1207+
IntervalUtils.supportedFormat((dataType.startField, dataType.endField))
1208+
.map(format => s"`$format`").mkString(", "),
1209+
"input" -> interval))
12031210
}
12041211
}
12051212

@@ -1313,15 +1320,17 @@ abstract class CastSuiteBase extends SparkFunSuite with ExpressionEvalHelper {
13131320
("1.23", DayTimeIntervalType(MINUTE)),
13141321
("1.23", DayTimeIntervalType(MINUTE)))
13151322
.foreach { case (interval, dataType) =>
1316-
val expectedMsg = s"Interval string does not match day-time format of " +
1317-
s"${IntervalUtils.supportedFormat((dataType.startField, dataType.endField))
1318-
.map(format => s"`$format`").mkString(", ")} " +
1319-
s"when cast to ${dataType.typeName}: $interval, " +
1320-
s"set ${SQLConf.LEGACY_FROM_DAYTIME_STRING.key} to true " +
1321-
"to restore the behavior before Spark 3.0."
1322-
checkExceptionInExpression[IllegalArgumentException](
1323+
checkErrorInExpression[SparkIllegalArgumentException](
13231324
cast(Literal.create(interval), dataType),
1324-
expectedMsg
1325+
"_LEGACY_ERROR_TEMP_3214",
1326+
Map("fallBackNotice" -> (", set spark.sql.legacy.fromDayTimeString.enabled" +
1327+
" to true to restore the behavior before Spark 3.0."),
1328+
"intervalStr" -> "day-time",
1329+
"typeName" -> dataType.typeName,
1330+
"input" -> interval,
1331+
"supportedFormat" ->
1332+
IntervalUtils.supportedFormat((dataType.startField, dataType.endField))
1333+
.map(format => s"`$format`").mkString(", "))
13251334
)
13261335
}
13271336

@@ -1337,15 +1346,17 @@ abstract class CastSuiteBase extends SparkFunSuite with ExpressionEvalHelper {
13371346
("INTERVAL '1537228672801:54.7757' MINUTE TO SECOND", DayTimeIntervalType(MINUTE, SECOND)),
13381347
("INTERVAL '92233720368541.775807' SECOND", DayTimeIntervalType(SECOND)))
13391348
.foreach { case (interval, dataType) =>
1340-
val expectedMsg = "Interval string does not match day-time format of " +
1341-
s"${IntervalUtils.supportedFormat((dataType.startField, dataType.endField))
1342-
.map(format => s"`$format`").mkString(", ")} " +
1343-
s"when cast to ${dataType.typeName}: $interval, " +
1344-
s"set ${SQLConf.LEGACY_FROM_DAYTIME_STRING.key} to true " +
1345-
"to restore the behavior before Spark 3.0."
1346-
checkExceptionInExpression[IllegalArgumentException](
1349+
checkErrorInExpression[SparkIllegalArgumentException](
13471350
cast(Literal.create(interval), dataType),
1348-
expectedMsg)
1351+
"_LEGACY_ERROR_TEMP_3214",
1352+
Map("fallBackNotice" -> (", set spark.sql.legacy.fromDayTimeString.enabled" +
1353+
" to true to restore the behavior before Spark 3.0."),
1354+
"intervalStr" -> "day-time",
1355+
"typeName" -> dataType.typeName,
1356+
"input" -> interval,
1357+
"supportedFormat" ->
1358+
IntervalUtils.supportedFormat((dataType.startField, dataType.endField))
1359+
.map(format => s"`$format`").mkString(", ")))
13491360
}
13501361
}
13511362

sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala

Lines changed: 7 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -28,7 +28,7 @@ import scala.language.postfixOps
2828
import scala.reflect.ClassTag
2929
import scala.util.Random
3030

31-
import org.apache.spark.{SparkArithmeticException, SparkDateTimeException, SparkException, SparkFunSuite, SparkUpgradeException}
31+
import org.apache.spark.{SparkArithmeticException, SparkDateTimeException, SparkException, SparkFunSuite, SparkIllegalArgumentException, SparkUpgradeException}
3232
import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow}
3333
import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection
3434
import org.apache.spark.sql.catalyst.util.{DateTimeUtils, IntervalUtils, TimestampFormatter}
@@ -434,9 +434,12 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
434434
}
435435

436436
withSQLConf((SQLConf.ANSI_ENABLED.key, "true")) {
437-
checkExceptionInExpression[IllegalArgumentException](
437+
checkErrorInExpression[SparkIllegalArgumentException](
438438
DateAddInterval(Literal(d), Literal(new CalendarInterval(1, 1, 25 * MICROS_PER_HOUR))),
439-
"Cannot add hours, minutes or seconds, milliseconds, microseconds to a date")
439+
"_LEGACY_ERROR_TEMP_2000",
440+
Map("message" ->
441+
"Cannot add hours, minutes or seconds, milliseconds, microseconds to a date",
442+
"ansiConfig" -> "\"spark.sql.ansi.enabled\""))
440443
}
441444

442445
withSQLConf((SQLConf.ANSI_ENABLED.key, "false")) {
@@ -1499,7 +1502,7 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
14991502
}
15001503

15011504
Seq('q', 'Q', 'e', 'c', 'A', 'n', 'N', 'p').foreach { l =>
1502-
checkException[IllegalArgumentException](l.toString)
1505+
checkException[SparkIllegalArgumentException](l.toString)
15031506
}
15041507
}
15051508

sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala

Lines changed: 7 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -154,6 +154,13 @@ trait ExpressionEvalHelper extends ScalaCheckDrivenPropertyChecks with PlanTestB
154154
checkErrorInExpression[T](expression, InternalRow.empty, errorClass, parameters)
155155
}
156156

157+
protected def checkErrorInExpression[T <: SparkThrowable : ClassTag](
158+
expression: => Expression,
159+
inputRow: InternalRow,
160+
errorClass: String): Unit = {
161+
checkErrorInExpression[T](expression, inputRow, errorClass, Map.empty[String, String])
162+
}
163+
157164
protected def checkErrorInExpression[T <: SparkThrowable : ClassTag](
158165
expression: => Expression,
159166
inputRow: InternalRow,

sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/TimeWindowSuite.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -188,7 +188,7 @@ class TimeWindowSuite extends SparkFunSuite with ExpressionEvalHelper with Priva
188188
"interval 1 year 2 month",
189189
"interval '1' year '2' month",
190190
"\tinterval '1-2' year to month").foreach { interval =>
191-
intercept[IllegalArgumentException] {
191+
intercept[SparkIllegalArgumentException] {
192192
TimeWindow(Literal(10L, TimestampType), interval, interval, interval)
193193
}
194194
}

sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/TryCastSuite.scala

Lines changed: 9 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.expressions
1919

2020
import scala.reflect.ClassTag
2121

22-
import org.apache.spark.SparkFunSuite
22+
import org.apache.spark.{SparkFunSuite, SparkThrowable}
2323
import org.apache.spark.sql.catalyst.InternalRow
2424
import org.apache.spark.sql.catalyst.util.DateTimeTestUtils.UTC_OPT
2525
import org.apache.spark.sql.types._
@@ -44,6 +44,14 @@ class TryCastSuite extends CastWithAnsiOnSuite {
4444
checkEvaluation(expression, null, inputRow)
4545
}
4646

47+
override def checkErrorInExpression[T <: SparkThrowable : ClassTag](
48+
expression: => Expression,
49+
inputRow: InternalRow,
50+
errorClass: String,
51+
parameters: Map[String, String]): Unit = {
52+
checkEvaluation(expression, null, inputRow)
53+
}
54+
4755
override def checkCastToBooleanError(l: Literal, to: DataType, tryCastResult: Any): Unit = {
4856
checkEvaluation(cast(l, to), tryCastResult, InternalRow(l.value))
4957
}

0 commit comments

Comments
 (0)