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 64fa13b5744f [SPARK-47060][SQL][TESTS] Check `SparkIllegalArgumentException` instead of `IllegalArgumentException` in `catalyst` 64fa13b5744f is described below commit 64fa13b5744fac11d62c35a9b1e1f8e7917bcc18 Author: Max Gekk <max.g...@gmail.com> AuthorDate: Fri Feb 16 11:44:39 2024 +0300 [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 #45118 from MaxGekk/migrate-IllegalArgumentException-catalyst-tests. Authored-by: Max Gekk <max.g...@gmail.com> Signed-off-by: Max Gekk <max.g...@gmail.com> --- .../spark/sql/catalyst/util/IntervalUtils.scala | 3 +- .../scala/org/apache/spark/sql/RowJsonSuite.scala | 21 +++-- .../test/scala/org/apache/spark/sql/RowTest.scala | 4 +- .../spark/sql/catalyst/csv/CSVExprUtilsSuite.scala | 11 ++- .../sql/catalyst/csv/UnivocityParserSuite.scala | 42 +++++----- .../sql/catalyst/expressions/CastSuiteBase.scala | 73 ++++++++++------- .../expressions/DateExpressionsSuite.scala | 11 ++- .../expressions/ExpressionEvalHelper.scala | 7 ++ .../sql/catalyst/expressions/TimeWindowSuite.scala | 2 +- .../sql/catalyst/expressions/TryCastSuite.scala | 10 ++- .../streaming/InternalOutputModesSuite.scala | 4 +- .../util/DateTimeFormatterHelperSuite.scala | 49 ++++++----- .../sql/catalyst/util/DatetimeFormatterSuite.scala | 6 +- .../spark/sql/connector/catalog/CatalogSuite.scala | 94 +++++++++++----------- .../catalog/SupportsPartitionManagementSuite.scala | 12 +-- .../org/apache/spark/sql/types/DataTypeSuite.scala | 43 +++++----- .../apache/spark/sql/types/StructTypeSuite.scala | 20 +++-- 17 files changed, 238 insertions(+), 174 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala index a016fd7d5881..9b4b0302ee77 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala @@ -22,7 +22,7 @@ import java.util.concurrent.TimeUnit import scala.util.control.NonFatal -import org.apache.spark.SparkIllegalArgumentException +import org.apache.spark.{SparkIllegalArgumentException, SparkThrowable} import org.apache.spark.sql.catalyst.expressions.Literal import org.apache.spark.sql.catalyst.parser.CatalystSqlParser import org.apache.spark.sql.catalyst.util.DateTimeConstants._ @@ -202,6 +202,7 @@ object IntervalUtils extends SparkIntervalUtils { try { f } catch { + case e: SparkThrowable => throw e case NonFatal(e) => throw new SparkIllegalArgumentException( errorClass = "_LEGACY_ERROR_TEMP_3213", diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/RowJsonSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/RowJsonSuite.scala index 1962fca66c05..e5914d4a88d2 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/RowJsonSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/RowJsonSuite.scala @@ -21,7 +21,7 @@ import java.time.LocalDate import org.json4s.JsonAST.{JArray, JBool, JDecimal, JDouble, JLong, JNull, JObject, JString, JValue} -import org.apache.spark.SparkFunSuite +import org.apache.spark.{SparkFunSuite, SparkIllegalArgumentException} import org.apache.spark.sql.catalyst.encoders.{ExamplePoint, ExamplePointUDT} import org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema import org.apache.spark.sql.types._ @@ -128,12 +128,17 @@ class RowJsonSuite extends SparkFunSuite { } test("unsupported type") { - val e = intercept[IllegalArgumentException] { - val row = new GenericRowWithSchema( - Array((1, 2)), - new StructType().add("a", ObjectType(classOf[(Int, Int)]))) - row.jsonValue - } - assert(e.getMessage.contains("Failed to convert value")) + checkError( + exception = intercept[SparkIllegalArgumentException] { + val row = new GenericRowWithSchema( + Array((1, 2)), + new StructType().add("a", ObjectType(classOf[(Int, Int)]))) + row.jsonValue + }, + errorClass = "_LEGACY_ERROR_TEMP_3249", + parameters = Map( + "value" -> "(1,2)", + "valueClass" -> "class scala.Tuple2$mcII$sp", + "dataType" -> "ObjectType(class scala.Tuple2)")) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/RowTest.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/RowTest.scala index 985443773943..dd4d1e5b9d46 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/RowTest.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/RowTest.scala @@ -24,7 +24,7 @@ import org.scalatest.funspec.AnyFunSpec import org.scalatest.matchers.must.Matchers import org.scalatest.matchers.should.Matchers._ -import org.apache.spark.{SparkException, SparkUnsupportedOperationException} +import org.apache.spark.{SparkException, SparkIllegalArgumentException, SparkUnsupportedOperationException} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{GenericRow, GenericRowWithSchema} import org.apache.spark.sql.types._ @@ -66,7 +66,7 @@ class RowTest extends AnyFunSpec with Matchers { } it("Accessing non existent field throws an exception") { - intercept[IllegalArgumentException] { + intercept[SparkIllegalArgumentException] { sampleRow.getAs[String]("non_existent") } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/CSVExprUtilsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/CSVExprUtilsSuite.scala index c1c826d16cf9..b2e4cdfeafe2 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/CSVExprUtilsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/CSVExprUtilsSuite.scala @@ -76,7 +76,7 @@ class CSVExprUtilsSuite extends SparkFunSuite { // backslash, then tab ("""\\t""", Some("""\t"""), None), // invalid special character (dot) - ("""\.""", None, Some("Unsupported special character for delimiter")), + ("""\.""", None, Some("_LEGACY_ERROR_TEMP_3236")), // backslash, then dot ("""\\.""", Some("""\."""), None), // nothing special, just straight conversion @@ -90,17 +90,16 @@ class CSVExprUtilsSuite extends SparkFunSuite { ) test("should correctly produce separator strings, or exceptions, from input") { - forAll(testCases) { (input, separatorStr, expectedErrorMsg) => + forAll(testCases) { (input, separatorStr, expectedErrorClass) => try { val separator = CSVExprUtils.toDelimiterStr(input) assert(separatorStr.isDefined) - assert(expectedErrorMsg.isEmpty) + assert(expectedErrorClass.isEmpty) assert(separator.equals(separatorStr.get)) } catch { - case e: IllegalArgumentException => + case e: SparkIllegalArgumentException => assert(separatorStr.isEmpty) - assert(expectedErrorMsg.isDefined) - assert(e.getMessage.contains(expectedErrorMsg.get)) + assert(e.getErrorClass === expectedErrorClass.get) } } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/UnivocityParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/UnivocityParserSuite.scala index 37605e14b926..943e40d9d455 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/UnivocityParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/UnivocityParserSuite.scala @@ -23,7 +23,7 @@ import java.util.{Locale, TimeZone} import org.apache.commons.lang3.time.FastDateFormat -import org.apache.spark.SparkFunSuite +import org.apache.spark.{SparkFunSuite, SparkIllegalArgumentException} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.plans.SQLHelper import org.apache.spark.sql.catalyst.util.DateTimeConstants._ @@ -304,19 +304,23 @@ class UnivocityParserSuite extends SparkFunSuite with SQLHelper { filters = Seq(EqualTo("d", 3.14)), expected = Some(InternalRow(1, 3.14))) - val errMsg = intercept[IllegalArgumentException] { - check(filters = Seq(EqualTo("invalid attr", 1)), expected = None) - }.getMessage - assert(errMsg.contains("invalid attr does not exist")) - - val errMsg2 = intercept[IllegalArgumentException] { - check( - dataSchema = new StructType(), - requiredSchema = new StructType(), - filters = Seq(EqualTo("i", 1)), - expected = Some(InternalRow.empty)) - }.getMessage - assert(errMsg2.contains("i does not exist")) + checkError( + exception = intercept[SparkIllegalArgumentException] { + check(filters = Seq(EqualTo("invalid attr", 1)), expected = None) + }, + errorClass = "_LEGACY_ERROR_TEMP_3252", + parameters = Map("name" -> "invalid attr", "fieldNames" -> "i")) + + checkError( + exception = intercept[SparkIllegalArgumentException] { + check( + dataSchema = new StructType(), + requiredSchema = new StructType(), + filters = Seq(EqualTo("i", 1)), + expected = Some(InternalRow.empty)) + }, + errorClass = "_LEGACY_ERROR_TEMP_3252", + parameters = Map("name" -> "i", "fieldNames" -> "")) } test("SPARK-30960: parse date/timestamp string with legacy format") { @@ -366,9 +370,11 @@ class UnivocityParserSuite extends SparkFunSuite with SQLHelper { check(new UnivocityParser(StructType(Seq.empty), optionsWithPattern(true))) // With legacy parser disabled, parsing results in error. - val err = intercept[IllegalArgumentException] { - check(new UnivocityParser(StructType(Seq.empty), optionsWithPattern(false))) - } - assert(err.getMessage.contains("Illegal pattern character: n")) + checkError( + exception = intercept[SparkIllegalArgumentException] { + check(new UnivocityParser(StructType(Seq.empty), optionsWithPattern(false))) + }, + errorClass = "_LEGACY_ERROR_TEMP_3258", + parameters = Map("c" -> "n")) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuiteBase.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuiteBase.scala index 1ce311a5544f..2ccb8e50e034 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuiteBase.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuiteBase.scala @@ -22,7 +22,7 @@ import java.time.{Duration, LocalDate, LocalDateTime, Period} import java.time.temporal.ChronoUnit import java.util.{Calendar, Locale, TimeZone} -import org.apache.spark.SparkFunSuite +import org.apache.spark.{SparkFunSuite, SparkIllegalArgumentException} import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.DataTypeMismatch @@ -1105,9 +1105,10 @@ abstract class CastSuiteBase extends SparkFunSuite with ExpressionEvalHelper { Seq("INTERVAL '-178956970-9' YEAR TO MONTH", "INTERVAL '178956970-8' YEAR TO MONTH") .foreach { interval => - checkExceptionInExpression[IllegalArgumentException]( + checkErrorInExpression[SparkIllegalArgumentException]( cast(Literal.create(interval), YearMonthIntervalType()), - "Error parsing interval year-month string: integer overflow") + "_LEGACY_ERROR_TEMP_3213", + Map("interval" -> "year-month", "msg" -> "integer overflow")) } Seq(Byte.MaxValue, Short.MaxValue, Int.MaxValue, Int.MinValue + 1, Int.MinValue) @@ -1173,13 +1174,15 @@ abstract class CastSuiteBase extends SparkFunSuite with ExpressionEvalHelper { Seq("INTERVAL '1-1' YEAR", "INTERVAL '1-1' MONTH").foreach { interval => val dataType = YearMonthIntervalType() - val expectedMsg = s"Interval string does not match year-month format of " + - s"${IntervalUtils.supportedFormat((dataType.startField, dataType.endField)) - .map(format => s"`$format`").mkString(", ")} " + - s"when cast to ${dataType.typeName}: $interval" - checkExceptionInExpression[IllegalArgumentException]( + checkErrorInExpression[SparkIllegalArgumentException]( cast(Literal.create(interval), dataType), - expectedMsg + "_LEGACY_ERROR_TEMP_3214", + Map( + "fallBackNotice" -> "", + "typeName" -> "interval year to month", + "intervalStr" -> "year-month", + "supportedFormat" -> "`[+|-]d h`, `INTERVAL [+|-]'[+|-]d h' DAY TO HOUR`", + "input" -> interval) ) } Seq(("1", YearMonthIntervalType(YEAR, MONTH)), @@ -1193,13 +1196,17 @@ abstract class CastSuiteBase extends SparkFunSuite with ExpressionEvalHelper { ("INTERVAL '1' MONTH", YearMonthIntervalType(YEAR)), ("INTERVAL '1' MONTH", YearMonthIntervalType(YEAR, MONTH))) .foreach { case (interval, dataType) => - val expectedMsg = s"Interval string does not match year-month format of " + - s"${IntervalUtils.supportedFormat((dataType.startField, dataType.endField)) - .map(format => s"`$format`").mkString(", ")} " + - s"when cast to ${dataType.typeName}: $interval" - checkExceptionInExpression[IllegalArgumentException]( + checkErrorInExpression[SparkIllegalArgumentException]( cast(Literal.create(interval), dataType), - expectedMsg) + "_LEGACY_ERROR_TEMP_3214", + Map( + "fallBackNotice" -> "", + "typeName" -> dataType.typeName, + "intervalStr" -> "year-month", + "supportedFormat" -> + IntervalUtils.supportedFormat((dataType.startField, dataType.endField)) + .map(format => s"`$format`").mkString(", "), + "input" -> interval)) } } @@ -1313,15 +1320,17 @@ abstract class CastSuiteBase extends SparkFunSuite with ExpressionEvalHelper { ("1.23", DayTimeIntervalType(MINUTE)), ("1.23", DayTimeIntervalType(MINUTE))) .foreach { case (interval, dataType) => - val expectedMsg = s"Interval string does not match day-time format of " + - s"${IntervalUtils.supportedFormat((dataType.startField, dataType.endField)) - .map(format => s"`$format`").mkString(", ")} " + - s"when cast to ${dataType.typeName}: $interval, " + - s"set ${SQLConf.LEGACY_FROM_DAYTIME_STRING.key} to true " + - "to restore the behavior before Spark 3.0." - checkExceptionInExpression[IllegalArgumentException]( + checkErrorInExpression[SparkIllegalArgumentException]( cast(Literal.create(interval), dataType), - expectedMsg + "_LEGACY_ERROR_TEMP_3214", + Map("fallBackNotice" -> (", set spark.sql.legacy.fromDayTimeString.enabled" + + " to true to restore the behavior before Spark 3.0."), + "intervalStr" -> "day-time", + "typeName" -> dataType.typeName, + "input" -> interval, + "supportedFormat" -> + IntervalUtils.supportedFormat((dataType.startField, dataType.endField)) + .map(format => s"`$format`").mkString(", ")) ) } @@ -1337,15 +1346,17 @@ abstract class CastSuiteBase extends SparkFunSuite with ExpressionEvalHelper { ("INTERVAL '1537228672801:54.7757' MINUTE TO SECOND", DayTimeIntervalType(MINUTE, SECOND)), ("INTERVAL '92233720368541.775807' SECOND", DayTimeIntervalType(SECOND))) .foreach { case (interval, dataType) => - val expectedMsg = "Interval string does not match day-time format of " + - s"${IntervalUtils.supportedFormat((dataType.startField, dataType.endField)) - .map(format => s"`$format`").mkString(", ")} " + - s"when cast to ${dataType.typeName}: $interval, " + - s"set ${SQLConf.LEGACY_FROM_DAYTIME_STRING.key} to true " + - "to restore the behavior before Spark 3.0." - checkExceptionInExpression[IllegalArgumentException]( + checkErrorInExpression[SparkIllegalArgumentException]( cast(Literal.create(interval), dataType), - expectedMsg) + "_LEGACY_ERROR_TEMP_3214", + Map("fallBackNotice" -> (", set spark.sql.legacy.fromDayTimeString.enabled" + + " to true to restore the behavior before Spark 3.0."), + "intervalStr" -> "day-time", + "typeName" -> dataType.typeName, + "input" -> interval, + "supportedFormat" -> + IntervalUtils.supportedFormat((dataType.startField, dataType.endField)) + .map(format => s"`$format`").mkString(", "))) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala index d159d911763b..f1c04c7e3382 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala @@ -28,7 +28,7 @@ import scala.language.postfixOps import scala.reflect.ClassTag import scala.util.Random -import org.apache.spark.{SparkArithmeticException, SparkDateTimeException, SparkException, SparkFunSuite, SparkUpgradeException} +import org.apache.spark.{SparkArithmeticException, SparkDateTimeException, SparkException, SparkFunSuite, SparkIllegalArgumentException, SparkUpgradeException} import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection import org.apache.spark.sql.catalyst.util.{DateTimeUtils, IntervalUtils, TimestampFormatter} @@ -434,9 +434,12 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { } withSQLConf((SQLConf.ANSI_ENABLED.key, "true")) { - checkExceptionInExpression[IllegalArgumentException]( + checkErrorInExpression[SparkIllegalArgumentException]( DateAddInterval(Literal(d), Literal(new CalendarInterval(1, 1, 25 * MICROS_PER_HOUR))), - "Cannot add hours, minutes or seconds, milliseconds, microseconds to a date") + "_LEGACY_ERROR_TEMP_2000", + Map("message" -> + "Cannot add hours, minutes or seconds, milliseconds, microseconds to a date", + "ansiConfig" -> "\"spark.sql.ansi.enabled\"")) } withSQLConf((SQLConf.ANSI_ENABLED.key, "false")) { @@ -1499,7 +1502,7 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { } Seq('q', 'Q', 'e', 'c', 'A', 'n', 'N', 'p').foreach { l => - checkException[IllegalArgumentException](l.toString) + checkException[SparkIllegalArgumentException](l.toString) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala index b71b426ae243..fe21b0ccaad4 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala @@ -154,6 +154,13 @@ trait ExpressionEvalHelper extends ScalaCheckDrivenPropertyChecks with PlanTestB checkErrorInExpression[T](expression, InternalRow.empty, errorClass, parameters) } + protected def checkErrorInExpression[T <: SparkThrowable : ClassTag]( + expression: => Expression, + inputRow: InternalRow, + errorClass: String): Unit = { + checkErrorInExpression[T](expression, inputRow, errorClass, Map.empty[String, String]) + } + protected def checkErrorInExpression[T <: SparkThrowable : ClassTag]( expression: => Expression, inputRow: InternalRow, diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/TimeWindowSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/TimeWindowSuite.scala index 92cffe2152b4..eb89141e0300 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/TimeWindowSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/TimeWindowSuite.scala @@ -188,7 +188,7 @@ class TimeWindowSuite extends SparkFunSuite with ExpressionEvalHelper with Priva "interval 1 year 2 month", "interval '1' year '2' month", "\tinterval '1-2' year to month").foreach { interval => - intercept[IllegalArgumentException] { + intercept[SparkIllegalArgumentException] { TimeWindow(Literal(10L, TimestampType), interval, interval, interval) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/TryCastSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/TryCastSuite.scala index 9ead07566354..f138d9642d1e 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/TryCastSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/TryCastSuite.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.expressions import scala.reflect.ClassTag -import org.apache.spark.SparkFunSuite +import org.apache.spark.{SparkFunSuite, SparkThrowable} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.util.DateTimeTestUtils.UTC_OPT import org.apache.spark.sql.types._ @@ -44,6 +44,14 @@ class TryCastSuite extends CastWithAnsiOnSuite { checkEvaluation(expression, null, inputRow) } + override def checkErrorInExpression[T <: SparkThrowable : ClassTag]( + expression: => Expression, + inputRow: InternalRow, + errorClass: String, + parameters: Map[String, String]): Unit = { + checkEvaluation(expression, null, inputRow) + } + override def checkCastToBooleanError(l: Literal, to: DataType, tryCastResult: Any): Unit = { checkEvaluation(cast(l, to), tryCastResult, InternalRow(l.value)) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/streaming/InternalOutputModesSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/streaming/InternalOutputModesSuite.scala index 3159b541dca7..ca1d64d42f96 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/streaming/InternalOutputModesSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/streaming/InternalOutputModesSuite.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.streaming import java.util.Locale -import org.apache.spark.SparkFunSuite +import org.apache.spark.{SparkFunSuite, SparkIllegalArgumentException} import org.apache.spark.sql.streaming.OutputMode class InternalOutputModesSuite extends SparkFunSuite { @@ -40,7 +40,7 @@ class InternalOutputModesSuite extends SparkFunSuite { test("unsupported strings") { def testMode(outputMode: String): Unit = { val acceptedModes = Seq("append", "update", "complete") - val e = intercept[IllegalArgumentException](InternalOutputModes(outputMode)) + val e = intercept[SparkIllegalArgumentException](InternalOutputModes(outputMode)) (Seq("output mode", "unknown", outputMode) ++ acceptedModes).foreach { s => assert(e.getMessage.toLowerCase(Locale.ROOT).contains(s.toLowerCase(Locale.ROOT))) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeFormatterHelperSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeFormatterHelperSuite.scala index 0b15e49af021..312dc3431249 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeFormatterHelperSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeFormatterHelperSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst.util -import org.apache.spark.SparkFunSuite +import org.apache.spark.{SparkFunSuite, SparkIllegalArgumentException} import org.apache.spark.sql.catalyst.util.DateTimeFormatterHelper._ class DateTimeFormatterHelperSuite extends SparkFunSuite { @@ -38,29 +38,42 @@ class DateTimeFormatterHelperSuite extends SparkFunSuite { assert(convertIncompatiblePattern("yyyy-MM-dd'T'HH:mm:ss.SSSz G") === "yyyy-MM-dd'T'HH:mm:ss.SSSz G") weekBasedLetters.foreach { l => - val e = intercept[IllegalArgumentException](convertIncompatiblePattern(s"yyyy-MM-dd $l G")) - assert(e.getMessage.contains("week-based")) + checkError( + exception = intercept[SparkIllegalArgumentException] { + convertIncompatiblePattern(s"yyyy-MM-dd $l G") + }, + errorClass = "_LEGACY_ERROR_TEMP_3257", + parameters = Map("c" -> l.toString)) } unsupportedLetters.foreach { l => - val e = intercept[IllegalArgumentException](convertIncompatiblePattern(s"yyyy-MM-dd $l G")) - assert(e.getMessage === s"Illegal pattern character: $l") + checkError( + exception = intercept[SparkIllegalArgumentException] { + convertIncompatiblePattern(s"yyyy-MM-dd $l G") + }, + errorClass = "_LEGACY_ERROR_TEMP_3258", + parameters = Map("c" -> l.toString)) } unsupportedLettersForParsing.foreach { l => - val e = intercept[IllegalArgumentException] { - DateTimeFormatterHelper.convertIncompatiblePattern(s"$l", isParsing = true) - } - assert(e.getMessage === s"Illegal pattern character: $l") - assert(convertIncompatiblePattern(s"$l").nonEmpty) + checkError( + exception = intercept[SparkIllegalArgumentException] { + DateTimeFormatterHelper.convertIncompatiblePattern(s"$l", isParsing = true) + }, + errorClass = "_LEGACY_ERROR_TEMP_3258", + parameters = Map("c" -> l.toString)) } unsupportedPatternLengths.foreach { style => - val e1 = intercept[IllegalArgumentException] { - convertIncompatiblePattern(s"yyyy-MM-dd $style") - } - assert(e1.getMessage === s"Too many pattern letters: ${style.head}") - val e2 = intercept[IllegalArgumentException] { - convertIncompatiblePattern(s"yyyy-MM-dd $style${style.head}") - } - assert(e2.getMessage === s"Too many pattern letters: ${style.head}") + checkError( + exception = intercept[SparkIllegalArgumentException] { + convertIncompatiblePattern(s"yyyy-MM-dd $style") + }, + errorClass = "_LEGACY_ERROR_TEMP_3259", + parameters = Map("style" -> style.head.toString)) + checkError( + exception = intercept[SparkIllegalArgumentException] { + convertIncompatiblePattern(s"yyyy-MM-dd $style${style.head}") + }, + errorClass = "_LEGACY_ERROR_TEMP_3259", + parameters = Map("style" -> style.head.toString)) } assert(convertIncompatiblePattern("yyyy-MM-dd EEEE") === "uuuu-MM-dd EEEE") assert(convertIncompatiblePattern("yyyy-MM-dd'e'HH:mm:ss") === "uuuu-MM-dd'e'HH:mm:ss") diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DatetimeFormatterSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DatetimeFormatterSuite.scala index 332568ab2d71..63717a125d2e 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DatetimeFormatterSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DatetimeFormatterSuite.scala @@ -21,7 +21,7 @@ import java.time.DateTimeException import org.scalatest.matchers.must.Matchers -import org.apache.spark.{SparkFunSuite, SparkUpgradeException} +import org.apache.spark.{SparkFunSuite, SparkIllegalArgumentException, SparkUpgradeException} import org.apache.spark.sql.catalyst.plans.SQLHelper import org.apache.spark.sql.catalyst.util.DateTimeTestUtils.{date, UTC} @@ -78,7 +78,7 @@ trait DatetimeFormatterSuite extends SparkFunSuite with SQLHelper with Matchers // not support by the legacy one too val unsupportedBoth = Seq("QQQQQ", "qqqqq", "eeeee", "A", "B", "c", "n", "N", "p", "e") unsupportedBoth.foreach { pattern => - intercept[IllegalArgumentException](checkFormatterCreation(pattern, isParsing)) + intercept[SparkIllegalArgumentException](checkFormatterCreation(pattern, isParsing)) } // supported by the legacy one, then we will suggest users with SparkUpgradeException ((weekBasedLetters ++ unsupportedLetters).map(_.toString) @@ -90,7 +90,7 @@ trait DatetimeFormatterSuite extends SparkFunSuite with SQLHelper with Matchers // not support by the legacy one too val unsupportedBoth = Seq("q", "Q") unsupportedBoth.foreach { pattern => - intercept[IllegalArgumentException](checkFormatterCreation(pattern, true)) + intercept[SparkIllegalArgumentException](checkFormatterCreation(pattern, true)) } // supported by the legacy one, then we will suggest users with SparkUpgradeException (unsupportedLettersForParsing.map(_.toString) -- unsupportedBoth).foreach { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/CatalogSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/CatalogSuite.scala index 42756d91b39e..2935b01649ca 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/CatalogSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/CatalogSuite.scala @@ -22,7 +22,7 @@ import java.util.Collections import scala.jdk.CollectionConverters._ -import org.apache.spark.{SparkFunSuite, SparkUnsupportedOperationException} +import org.apache.spark.{SparkFunSuite, SparkIllegalArgumentException, SparkUnsupportedOperationException} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.{NamespaceAlreadyExistsException, NoSuchFunctionException, NoSuchNamespaceException, NoSuchTableException, TableAlreadyExistsException} import org.apache.spark.sql.catalyst.parser.CatalystSqlParser @@ -335,12 +335,12 @@ class CatalogSuite extends SparkFunSuite { assert(table.schema == schema) - val exc = intercept[IllegalArgumentException] { - catalog.alterTable(testIdent, TableChange.addColumn(Array("data", "ts"), TimestampType)) - } - - assert(exc.getMessage.contains("Not a struct")) - assert(exc.getMessage.contains("data")) + checkError( + exception = intercept[SparkIllegalArgumentException] { + catalog.alterTable(testIdent, TableChange.addColumn(Array("data", "ts"), TimestampType)) + }, + errorClass = "_LEGACY_ERROR_TEMP_3229", + parameters = Map("name" -> "data")) // the table has not changed assert(catalog.loadTable(testIdent).schema == schema) @@ -353,13 +353,13 @@ class CatalogSuite extends SparkFunSuite { assert(table.schema == schema) - val exc = intercept[IllegalArgumentException] { - catalog.alterTable(testIdent, - TableChange.addColumn(Array("missing_col", "new_field"), StringType)) - } - - assert(exc.getMessage.contains("missing_col")) - assert(exc.getMessage.contains("Cannot find")) + checkError( + exception = intercept[SparkIllegalArgumentException] { + catalog.alterTable(testIdent, + TableChange.addColumn(Array("missing_col", "new_field"), StringType)) + }, + errorClass = "_LEGACY_ERROR_TEMP_3227", + parameters = Map("fieldName" -> "missing_col")) } test("alterTable: update column data type") { @@ -399,13 +399,13 @@ class CatalogSuite extends SparkFunSuite { assert(table.schema == schema) - val exc = intercept[IllegalArgumentException] { - catalog.alterTable(testIdent, - TableChange.updateColumnType(Array("missing_col"), LongType)) - } - - assert(exc.getMessage.contains("missing_col")) - assert(exc.getMessage.contains("Cannot find")) + checkError( + exception = intercept[SparkIllegalArgumentException] { + catalog.alterTable(testIdent, + TableChange.updateColumnType(Array("missing_col"), LongType)) + }, + errorClass = "_LEGACY_ERROR_TEMP_3227", + parameters = Map("fieldName" -> "missing_col")) } test("alterTable: add comment") { @@ -450,13 +450,13 @@ class CatalogSuite extends SparkFunSuite { assert(table.schema == schema) - val exc = intercept[IllegalArgumentException] { - catalog.alterTable(testIdent, - TableChange.updateColumnComment(Array("missing_col"), "comment")) - } - - assert(exc.getMessage.contains("missing_col")) - assert(exc.getMessage.contains("Cannot find")) + checkError( + exception = intercept[SparkIllegalArgumentException] { + catalog.alterTable(testIdent, + TableChange.updateColumnComment(Array("missing_col"), "comment")) + }, + errorClass = "_LEGACY_ERROR_TEMP_3227", + parameters = Map("fieldName" -> "missing_col")) } test("alterTable: rename top-level column") { @@ -518,13 +518,13 @@ class CatalogSuite extends SparkFunSuite { assert(table.schema == schema) - val exc = intercept[IllegalArgumentException] { - catalog.alterTable(testIdent, - TableChange.renameColumn(Array("missing_col"), "new_name")) - } - - assert(exc.getMessage.contains("missing_col")) - assert(exc.getMessage.contains("Cannot find")) + checkError( + exception = intercept[SparkIllegalArgumentException] { + catalog.alterTable(testIdent, + TableChange.renameColumn(Array("missing_col"), "new_name")) + }, + errorClass = "_LEGACY_ERROR_TEMP_3227", + parameters = Map("fieldName" -> "missing_col")) } test("alterTable: multiple changes") { @@ -587,12 +587,12 @@ class CatalogSuite extends SparkFunSuite { assert(table.schema == schema) - val exc = intercept[IllegalArgumentException] { - catalog.alterTable(testIdent, TableChange.deleteColumn(Array("missing_col"), false)) - } - - assert(exc.getMessage.contains("missing_col")) - assert(exc.getMessage.contains("Cannot find")) + checkError( + exception = intercept[SparkIllegalArgumentException] { + catalog.alterTable(testIdent, TableChange.deleteColumn(Array("missing_col"), false)) + }, + errorClass = "_LEGACY_ERROR_TEMP_3227", + parameters = Map("fieldName" -> "missing_col")) // with if exists it should pass catalog.alterTable(testIdent, TableChange.deleteColumn(Array("missing_col"), true)) @@ -609,12 +609,12 @@ class CatalogSuite extends SparkFunSuite { assert(table.schema == tableSchema) - val exc = intercept[IllegalArgumentException] { - catalog.alterTable(testIdent, TableChange.deleteColumn(Array("point", "z"), false)) - } - - assert(exc.getMessage.contains("z")) - assert(exc.getMessage.contains("Cannot find")) + checkError( + exception = intercept[SparkIllegalArgumentException] { + catalog.alterTable(testIdent, TableChange.deleteColumn(Array("point", "z"), false)) + }, + errorClass = "_LEGACY_ERROR_TEMP_3227", + parameters = Map("fieldName" -> "z")) // with if exists it should pass catalog.alterTable(testIdent, TableChange.deleteColumn(Array("point", "z"), true)) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/SupportsPartitionManagementSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/SupportsPartitionManagementSuite.scala index 55f676b0c440..698ac5b0ba40 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/SupportsPartitionManagementSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/SupportsPartitionManagementSuite.scala @@ -21,7 +21,7 @@ import java.util import scala.jdk.CollectionConverters._ -import org.apache.spark.{SparkFunSuite, SparkUnsupportedOperationException} +import org.apache.spark.{SparkFunSuite, SparkIllegalArgumentException, SparkUnsupportedOperationException} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.{NoSuchPartitionException, PartitionsAlreadyExistException} import org.apache.spark.sql.connector.expressions.{LogicalExpressions, NamedReference, Transform} @@ -213,10 +213,12 @@ class SupportsPartitionManagementSuite extends SparkFunSuite { assert(!partTable.partitionExists(InternalRow(-1, "def"))) assert(!partTable.partitionExists(InternalRow("abc", "def"))) - val errMsg = intercept[IllegalArgumentException] { - partTable.partitionExists(InternalRow(0)) - }.getMessage - assert(errMsg.contains("The identifier might not refer to one partition")) + checkError( + exception = intercept[SparkIllegalArgumentException] { + partTable.partitionExists(InternalRow(0)) + }, + errorClass = "_LEGACY_ERROR_TEMP_3208", + parameters = Map("numFields" -> "1", "schemaLen" -> "2")) } test("renamePartition") { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DataTypeSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DataTypeSuite.scala index 6daeda693002..472dbdedbc06 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DataTypeSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DataTypeSuite.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.types import com.fasterxml.jackson.core.JsonParseException -import org.apache.spark.{SparkException, SparkFunSuite} +import org.apache.spark.{SparkException, SparkFunSuite, SparkIllegalArgumentException} import org.apache.spark.sql.catalyst.analysis.{caseInsensitiveResolution, caseSensitiveResolution} import org.apache.spark.sql.catalyst.parser.CatalystSqlParser import org.apache.spark.sql.catalyst.types.DataTypeUtils @@ -97,7 +97,7 @@ class DataTypeSuite extends SparkFunSuite { assert(StructField("b", LongType, false) === struct("b")) - intercept[IllegalArgumentException] { + intercept[SparkIllegalArgumentException] { struct("e") } @@ -106,7 +106,7 @@ class DataTypeSuite extends SparkFunSuite { StructField("d", FloatType, true) :: Nil) assert(expectedStruct === struct(Set("b", "d"))) - intercept[IllegalArgumentException] { + intercept[SparkIllegalArgumentException] { struct(Set("b", "d", "e", "f")) } } @@ -119,7 +119,7 @@ class DataTypeSuite extends SparkFunSuite { assert(struct.fieldIndex("a") === 0) assert(struct.fieldIndex("b") === 1) - intercept[IllegalArgumentException] { + intercept[SparkIllegalArgumentException] { struct.fieldIndex("non_existent") } } @@ -292,26 +292,29 @@ class DataTypeSuite extends SparkFunSuite { checkDataTypeFromDDL(structType) test("fromJson throws an exception when given type string is invalid") { - var message = intercept[IllegalArgumentException] { - DataType.fromJson(""""abcd"""") - }.getMessage - assert(message.contains( - "Failed to convert the JSON string 'abcd' to a data type.")) + checkError( + exception = intercept[SparkIllegalArgumentException] { + DataType.fromJson(""""abcd"""") + }, + errorClass = "_LEGACY_ERROR_TEMP_3251", + parameters = Map("other" -> "abcd")) - message = intercept[IllegalArgumentException] { - DataType.fromJson("""{"abcd":"a"}""") - }.getMessage - assert(message.contains( - """Failed to convert the JSON string '{"abcd":"a"}' to a data type""")) + checkError( + exception = intercept[SparkIllegalArgumentException] { + DataType.fromJson("""{"abcd":"a"}""") + }, + errorClass = "_LEGACY_ERROR_TEMP_3251", + parameters = Map("other" -> """{"abcd":"a"}""")) - message = intercept[IllegalArgumentException] { - DataType.fromJson("""{"fields": [{"a":123}], "type": "struct"}""") - }.getMessage - assert(message.contains( - """Failed to convert the JSON string '{"a":123}' to a field.""")) + checkError( + exception = intercept[SparkIllegalArgumentException] { + DataType.fromJson("""{"fields": [{"a":123}], "type": "struct"}""") + }, + errorClass = "_LEGACY_ERROR_TEMP_3250", + parameters = Map("other" -> """{"a":123}""")) // Malformed JSON string - message = intercept[JsonParseException] { + val message = intercept[JsonParseException] { DataType.fromJson("abcd") }.getMessage assert(message.contains("Unrecognized token 'abcd'")) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/StructTypeSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/StructTypeSuite.scala index 1aa0d515d8f9..3ad78a9ecc6e 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/StructTypeSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/StructTypeSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.types -import org.apache.spark.{SparkException, SparkFunSuite} +import org.apache.spark.{SparkException, SparkFunSuite, SparkIllegalArgumentException} import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.analysis.{caseInsensitiveResolution, caseSensitiveResolution} import org.apache.spark.sql.catalyst.parser.ParseException @@ -37,18 +37,24 @@ class StructTypeSuite extends SparkFunSuite with SQLHelper { private val s = StructType.fromDDL("a INT, b STRING") test("lookup a single missing field should output existing fields") { - val e = intercept[IllegalArgumentException](s("c")).getMessage - assert(e.contains("Available: a, b")) + checkError( + exception = intercept[SparkIllegalArgumentException](s("c")), + errorClass = "_LEGACY_ERROR_TEMP_3254", + parameters = Map("name" -> "c", "fieldNames" -> "a, b")) } test("lookup a set of missing fields should output existing fields") { - val e = intercept[IllegalArgumentException](s(Set("a", "c"))).getMessage - assert(e.contains("Available: a, b")) + checkError( + exception = intercept[SparkIllegalArgumentException](s(Set("a", "c"))), + errorClass = "_LEGACY_ERROR_TEMP_3253", + parameters = Map("nonExistFields" -> "c", "fieldNames" -> "a, b")) } test("lookup fieldIndex for missing field should output existing fields") { - val e = intercept[IllegalArgumentException](s.fieldIndex("c")).getMessage - assert(e.contains("Available: a, b")) + checkError( + exception = intercept[SparkIllegalArgumentException](s.fieldIndex("c")), + errorClass = "_LEGACY_ERROR_TEMP_3252", + parameters = Map("name" -> "c", "fieldNames" -> "a, b")) } test("SPARK-24849: toDDL - simple struct") { --------------------------------------------------------------------- To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org