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 9ef552691e1d [SPARK-47045][SQL] Replace `IllegalArgumentException` by 
`SparkIllegalArgumentException` in `sql/api`
9ef552691e1d is described below

commit 9ef552691e1d4725d7a64b45e6cdee9e5e75f992
Author: Max Gekk <max.g...@gmail.com>
AuthorDate: Thu Feb 15 10:28:21 2024 +0300

    [SPARK-47045][SQL] Replace `IllegalArgumentException` by 
`SparkIllegalArgumentException` in `sql/api`
    
    ### What changes were proposed in this pull request?
    In the PR, I propose to replace all `IllegalArgumentException` by 
`SparkIllegalArgumentException` in `sql/api` code base, and introduce new 
legacy error classes with the `_LEGACY_ERROR_TEMP_` prefix.
    
    ### Why are the changes needed?
    To unify Spark SQL exception, and port Java exceptions on Spark exceptions 
with error classes.
    
    ### Does this PR introduce _any_ user-facing change?
    Yes, it can if user's code assumes some particular format of 
`IllegalArgumentException` messages.
    
    ### How was this patch tested?
    By running existing test suites like:
    ```
    $ build/sbt "core/testOnly *SparkThrowableSuite"
    ```
    
    ### Was this patch authored or co-authored using generative AI tooling?
    No.
    
    Closes #45098 from MaxGekk/migrate-IllegalArgumentException-sql.
    
    Authored-by: Max Gekk <max.g...@gmail.com>
    Signed-off-by: Max Gekk <max.g...@gmail.com>
---
 R/pkg/tests/fulltests/test_streaming.R             |  3 +-
 .../src/main/resources/error/error-classes.json    | 70 +++++++++++++++++++
 .../src/main/scala/org/apache/spark/sql/Row.scala  | 11 ++-
 .../catalyst/streaming/InternalOutputModes.scala   |  7 +-
 .../catalyst/util/DateTimeFormatterHelper.scala    | 18 +++--
 .../sql/catalyst/util/SparkIntervalUtils.scala     |  8 ++-
 .../sql/catalyst/util/TimestampFormatter.scala     |  6 +-
 .../spark/sql/execution/streaming/Triggers.scala   |  5 +-
 .../org/apache/spark/sql/types/DataType.scala      | 19 ++---
 .../org/apache/spark/sql/types/StructType.scala    | 25 ++++---
 .../results/datetime-formatting-invalid.sql.out    | 81 +++++++++++++++++-----
 .../org/apache/spark/sql/JsonFunctionsSuite.scala  | 13 ++--
 12 files changed, 206 insertions(+), 60 deletions(-)

diff --git a/R/pkg/tests/fulltests/test_streaming.R 
b/R/pkg/tests/fulltests/test_streaming.R
index 8804471e640c..67479726b57c 100644
--- a/R/pkg/tests/fulltests/test_streaming.R
+++ b/R/pkg/tests/fulltests/test_streaming.R
@@ -257,7 +257,8 @@ test_that("Trigger", {
                "Value for trigger.processingTime must be a non-empty string.")
 
   expect_error(write.stream(df, "memory", queryName = "times", outputMode = 
"append",
-               trigger.processingTime = "invalid"), "illegal argument")
+               trigger.processingTime = "invalid"),
+               "Error parsing 'invalid' to interval, unrecognized number 
'invalid'")
 
   expect_error(write.stream(df, "memory", queryName = "times", outputMode = 
"append",
                trigger.once = ""), "Value for trigger.once must be TRUE.")
diff --git a/common/utils/src/main/resources/error/error-classes.json 
b/common/utils/src/main/resources/error/error-classes.json
index 5884c9267119..38161ff87720 100644
--- a/common/utils/src/main/resources/error/error-classes.json
+++ b/common/utils/src/main/resources/error/error-classes.json
@@ -7767,6 +7767,76 @@
       "Single backslash is prohibited. It has special meaning as beginning of 
an escape sequence. To get the backslash character, pass a string with two 
backslashes as the delimiter."
     ]
   },
+  "_LEGACY_ERROR_TEMP_3249" : {
+    "message" : [
+      "Failed to convert value <value> (class of <valueClass>}) with the type 
of <dataType> to JSON."
+    ]
+  },
+  "_LEGACY_ERROR_TEMP_3250" : {
+    "message" : [
+      "Failed to convert the JSON string '<other>' to a field."
+    ]
+  },
+  "_LEGACY_ERROR_TEMP_3251" : {
+    "message" : [
+      "Failed to convert the JSON string '<other>' to a data type."
+    ]
+  },
+  "_LEGACY_ERROR_TEMP_3252" : {
+    "message" : [
+      "<name> does not exist. Available: <fieldNames>"
+    ]
+  },
+  "_LEGACY_ERROR_TEMP_3253" : {
+    "message" : [
+      "<nonExistFields> do(es) not exist. Available: <fieldNames>"
+    ]
+  },
+  "_LEGACY_ERROR_TEMP_3254" : {
+    "message" : [
+      "<name> does not exist. Available: <fieldNames>"
+    ]
+  },
+  "_LEGACY_ERROR_TEMP_3255" : {
+    "message" : [
+      "Error parsing '<input>' to interval, <msg>"
+    ]
+  },
+  "_LEGACY_ERROR_TEMP_3256" : {
+    "message" : [
+      "Unrecognized datetime pattern: <pattern>"
+    ]
+  },
+  "_LEGACY_ERROR_TEMP_3257" : {
+    "message" : [
+      "All week-based patterns are unsupported since Spark 3.0, detected: <c>, 
Please use the SQL function EXTRACT instead"
+    ]
+  },
+  "_LEGACY_ERROR_TEMP_3258" : {
+    "message" : [
+      "Illegal pattern character: <c>"
+    ]
+  },
+  "_LEGACY_ERROR_TEMP_3259" : {
+    "message" : [
+      "Too many pattern letters: <style>"
+    ]
+  },
+  "_LEGACY_ERROR_TEMP_3260" : {
+    "message" : [
+      "'<s>' is an invalid timestamp"
+    ]
+  },
+  "_LEGACY_ERROR_TEMP_3261" : {
+    "message" : [
+      "Unknown output mode <outputMode>. Accepted output modes are 'append', 
'complete', 'update'"
+    ]
+  },
+  "_LEGACY_ERROR_TEMP_3262" : {
+    "message" : [
+      "Doesn't support month or year interval: <interval>"
+    ]
+  },
   "_LEGACY_ERROR_USER_RAISED_EXCEPTION" : {
     "message" : [
       "<errorMessage>"
diff --git a/sql/api/src/main/scala/org/apache/spark/sql/Row.scala 
b/sql/api/src/main/scala/org/apache/spark/sql/Row.scala
index a1735aa250eb..954bf6973eb1 100644
--- a/sql/api/src/main/scala/org/apache/spark/sql/Row.scala
+++ b/sql/api/src/main/scala/org/apache/spark/sql/Row.scala
@@ -29,6 +29,7 @@ import org.json4s.{JArray, JBool, JDecimal, JDouble, JField, 
JLong, JNull, JObje
 import org.json4s.JsonAST.JValue
 import org.json4s.jackson.JsonMethods.{compact, pretty, render}
 
+import org.apache.spark.SparkIllegalArgumentException
 import org.apache.spark.annotation.{Stable, Unstable}
 import org.apache.spark.sql.catalyst.expressions.GenericRow
 import org.apache.spark.sql.catalyst.util.{DateFormatter, SparkDateTimeUtils, 
TimestampFormatter, UDTUtils}
@@ -609,9 +610,13 @@ trait Row extends Serializable {
         new JObject(elements.toList)
       case (v: Any, udt: UserDefinedType[Any @unchecked]) =>
         toJson(UDTUtils.toRow(v, udt), udt.sqlType)
-      case _ =>
-        throw new IllegalArgumentException(s"Failed to convert value $value " +
-          s"(class of ${value.getClass}}) with the type of $dataType to JSON.")
+      case _ => throw new SparkIllegalArgumentException(
+        errorClass = "_LEGACY_ERROR_TEMP_3249",
+        messageParameters = Map(
+          "value" -> value.toString,
+          "valueClass" -> value.getClass.toString,
+          "dataType" -> dataType.toString)
+      )
     }
     toJson(this, schema)
   }
diff --git 
a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/streaming/InternalOutputModes.scala
 
b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/streaming/InternalOutputModes.scala
index 3cd6970ebefb..bac64689053e 100644
--- 
a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/streaming/InternalOutputModes.scala
+++ 
b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/streaming/InternalOutputModes.scala
@@ -19,6 +19,7 @@ package org.apache.spark.sql.catalyst.streaming
 
 import java.util.Locale
 
+import org.apache.spark.SparkIllegalArgumentException
 import org.apache.spark.sql.streaming.OutputMode
 
 /**
@@ -56,9 +57,9 @@ private[sql] object InternalOutputModes {
         OutputMode.Complete
       case "update" =>
         OutputMode.Update
-      case _ =>
-        throw new IllegalArgumentException(s"Unknown output mode $outputMode. 
" +
-          "Accepted output modes are 'append', 'complete', 'update'")
+      case _ => throw new SparkIllegalArgumentException(
+          errorClass = "_LEGACY_ERROR_TEMP_3261",
+          messageParameters = Map("outputMode" -> outputMode))
     }
   }
 }
diff --git 
a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeFormatterHelper.scala
 
b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeFormatterHelper.scala
index e2a897a32111..645e103c0282 100644
--- 
a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeFormatterHelper.scala
+++ 
b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeFormatterHelper.scala
@@ -24,6 +24,7 @@ import java.time.temporal.{ChronoField, TemporalAccessor, 
TemporalQueries}
 import java.util
 import java.util.{Collections, Date, Locale}
 
+import org.apache.spark.SparkIllegalArgumentException
 import org.apache.spark.sql.catalyst.util.DateTimeFormatterHelper._
 import org.apache.spark.sql.errors.ExecutionErrors
 import org.apache.spark.sql.internal.LegacyBehaviorPolicy._
@@ -231,7 +232,9 @@ private object DateTimeFormatterHelper {
                 builder.appendFraction(ChronoField.NANO_OF_SECOND, 1, 
secondFraction.length, false)
               }
               rest = suffix
-            case _ => throw new IllegalArgumentException(s"Unrecognized 
datetime pattern: $pattern")
+            case _ => throw new SparkIllegalArgumentException(
+              errorClass = "_LEGACY_ERROR_TEMP_3256",
+              messageParameters = Map("pattern" -> pattern))
           }
         }
       case (patternPart, _) => builder.appendLiteral(patternPart)
@@ -309,15 +312,20 @@ private object DateTimeFormatterHelper {
       case (patternPart, index) =>
         if (index % 2 == 0) {
           for (c <- patternPart if weekBasedLetters.contains(c)) {
-            throw new IllegalArgumentException(s"All week-based patterns are 
unsupported since" +
-              s" Spark 3.0, detected: $c, Please use the SQL function EXTRACT 
instead")
+            throw new SparkIllegalArgumentException(
+              errorClass = "_LEGACY_ERROR_TEMP_3257",
+              messageParameters = Map("c" -> c.toString))
           }
           for (c <- patternPart if unsupportedLetters.contains(c) ||
             (isParsing && unsupportedLettersForParsing.contains(c))) {
-            throw new IllegalArgumentException(s"Illegal pattern character: 
$c")
+            throw new SparkIllegalArgumentException(
+              errorClass = "_LEGACY_ERROR_TEMP_3258",
+              messageParameters = Map("c" -> c.toString))
           }
           for (style <- unsupportedPatternLengths if 
patternPart.contains(style)) {
-            throw new IllegalArgumentException(s"Too many pattern letters: 
${style.head}")
+            throw new SparkIllegalArgumentException(
+              errorClass = "_LEGACY_ERROR_TEMP_3259",
+              messageParameters = Map("style" -> style.head.toString))
           }
           // In DateTimeFormatter, 'u' supports negative years. We substitute 
'y' to 'u' here for
           // keeping the support in Spark 3.0. If parse failed in Spark 3.0, 
fall back to 'y'.
diff --git 
a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/SparkIntervalUtils.scala
 
b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/SparkIntervalUtils.scala
index f9c132ebf375..f5911ebf95a1 100644
--- 
a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/SparkIntervalUtils.scala
+++ 
b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/SparkIntervalUtils.scala
@@ -21,6 +21,7 @@ import java.time.temporal.ChronoUnit
 
 import scala.collection.mutable
 
+import org.apache.spark.SparkIllegalArgumentException
 import org.apache.spark.sql.catalyst.util.DateTimeConstants._
 import org.apache.spark.sql.catalyst.util.IntervalStringStyles.{ANSI_STYLE, 
HIVE_STYLE, IntervalStyle}
 import org.apache.spark.sql.types.{DayTimeIntervalType => DT, 
YearMonthIntervalType => YM}
@@ -131,7 +132,12 @@ trait SparkIntervalUtils {
   def stringToInterval(input: UTF8String): CalendarInterval = {
     import ParseState._
     def throwIAE(msg: String, e: Exception = null) = {
-      throw new IllegalArgumentException(s"Error parsing '$input' to interval, 
$msg", e)
+      throw new SparkIllegalArgumentException(
+        errorClass = "_LEGACY_ERROR_TEMP_3255",
+        messageParameters = Map(
+          "input" -> Option(input).map(_.toString).getOrElse("null"),
+          "msg" -> msg),
+        cause = e)
     }
 
     if (input == null) {
diff --git 
a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/TimestampFormatter.scala
 
b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/TimestampFormatter.scala
index b8dac2df8687..d59b52a3818a 100644
--- 
a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/TimestampFormatter.scala
+++ 
b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/TimestampFormatter.scala
@@ -29,7 +29,7 @@ import scala.util.control.NonFatal
 
 import org.apache.commons.lang3.time.FastDateFormat
 
-import org.apache.spark.SparkException
+import org.apache.spark.{SparkException, SparkIllegalArgumentException}
 import org.apache.spark.sql.catalyst.util.DateTimeConstants._
 import org.apache.spark.sql.catalyst.util.LegacyDateFormats.{LegacyDateFormat, 
LENIENT_SIMPLE_DATE_FORMAT}
 import org.apache.spark.sql.catalyst.util.RebaseDateTime._
@@ -408,7 +408,9 @@ class LegacyFastTimestampFormatter(
   override def parse(s: String): Long = {
     cal.clear() // Clear the calendar because it can be re-used many times
     if (!fastDateFormat.parse(s, new ParsePosition(0), cal)) {
-      throw new IllegalArgumentException(s"'$s' is an invalid timestamp")
+      throw new SparkIllegalArgumentException(
+        errorClass = "_LEGACY_ERROR_TEMP_3260",
+        messageParameters = Map("s" -> s))
     }
     extractMicros(cal)
   }
diff --git 
a/sql/api/src/main/scala/org/apache/spark/sql/execution/streaming/Triggers.scala
 
b/sql/api/src/main/scala/org/apache/spark/sql/execution/streaming/Triggers.scala
index 37c5b314978b..146012b4266d 100644
--- 
a/sql/api/src/main/scala/org/apache/spark/sql/execution/streaming/Triggers.scala
+++ 
b/sql/api/src/main/scala/org/apache/spark/sql/execution/streaming/Triggers.scala
@@ -21,6 +21,7 @@ import java.util.concurrent.TimeUnit
 
 import scala.concurrent.duration.Duration
 
+import org.apache.spark.SparkIllegalArgumentException
 import org.apache.spark.sql.catalyst.util.DateTimeConstants.MICROS_PER_DAY
 import org.apache.spark.sql.catalyst.util.SparkDateTimeUtils.microsToMillis
 import org.apache.spark.sql.catalyst.util.SparkIntervalUtils
@@ -35,7 +36,9 @@ private object Triggers {
   def convert(interval: String): Long = {
     val cal = 
SparkIntervalUtils.stringToInterval(UTF8String.fromString(interval))
     if (cal.months != 0) {
-      throw new IllegalArgumentException(s"Doesn't support month or year 
interval: $interval")
+      throw new SparkIllegalArgumentException(
+        errorClass = "_LEGACY_ERROR_TEMP_3262",
+        messageParameters = Map("interval" -> interval))
     }
     val microsInDays = Math.multiplyExact(cal.days, MICROS_PER_DAY)
     microsToMillis(Math.addExact(cal.microseconds, microsInDays))
diff --git a/sql/api/src/main/scala/org/apache/spark/sql/types/DataType.scala 
b/sql/api/src/main/scala/org/apache/spark/sql/types/DataType.scala
index 94252de48d1e..319a5eccbb6d 100644
--- a/sql/api/src/main/scala/org/apache/spark/sql/types/DataType.scala
+++ b/sql/api/src/main/scala/org/apache/spark/sql/types/DataType.scala
@@ -27,7 +27,7 @@ import org.json4s.JsonAST.JValue
 import org.json4s.JsonDSL._
 import org.json4s.jackson.JsonMethods._
 
-import org.apache.spark.SparkThrowable
+import org.apache.spark.{SparkIllegalArgumentException, SparkThrowable}
 import org.apache.spark.annotation.Stable
 import org.apache.spark.sql.catalyst.analysis.SqlApiAnalysis
 import org.apache.spark.sql.catalyst.parser.DataTypeParser
@@ -190,8 +190,9 @@ object DataType {
       case "timestamp_ltz" => TimestampType
       case other => otherTypes.getOrElse(
         other,
-        throw new IllegalArgumentException(
-          s"Failed to convert the JSON string '$name' to a data type."))
+        throw new SparkIllegalArgumentException(
+          errorClass = "_LEGACY_ERROR_TEMP_3251",
+          messageParameters = Map("other" -> name)))
     }
   }
 
@@ -241,9 +242,9 @@ object DataType {
     ("type", JString("udt"))) =>
         new PythonUserDefinedType(parseDataType(v), pyClass, serialized)
 
-    case other =>
-      throw new IllegalArgumentException(
-        s"Failed to convert the JSON string '${compact(render(other))}' to a 
data type.")
+    case other => throw new SparkIllegalArgumentException(
+      errorClass = "_LEGACY_ERROR_TEMP_3251",
+      messageParameters = Map("other" -> compact(render(other))))
   }
 
   private def parseStructField(json: JValue): StructField = json match {
@@ -264,9 +265,9 @@ object DataType {
     ("name", JString(name)),
     ("type", dataType: JValue)) =>
       StructField(name, parseDataType(dataType))
-    case other =>
-      throw new IllegalArgumentException(
-        s"Failed to convert the JSON string '${compact(render(other))}' to a 
field.")
+    case other => throw new SparkIllegalArgumentException(
+      errorClass = "_LEGACY_ERROR_TEMP_3250",
+      messageParameters = Map("other" -> compact(render(other))))
   }
 
   protected[types] def buildFormattedString(
diff --git a/sql/api/src/main/scala/org/apache/spark/sql/types/StructType.scala 
b/sql/api/src/main/scala/org/apache/spark/sql/types/StructType.scala
index aac2f1cd737e..f6cbeefa11c0 100644
--- a/sql/api/src/main/scala/org/apache/spark/sql/types/StructType.scala
+++ b/sql/api/src/main/scala/org/apache/spark/sql/types/StructType.scala
@@ -19,12 +19,13 @@ package org.apache.spark.sql.types
 
 import java.util.Locale
 
-import scala.collection.{mutable, Map}
+import scala.collection.{immutable, mutable, Map}
 import scala.util.Try
 import scala.util.control.NonFatal
 
 import org.json4s.JsonDSL._
 
+import org.apache.spark.SparkIllegalArgumentException
 import org.apache.spark.annotation.Stable
 import org.apache.spark.sql.catalyst.analysis.SqlApiAnalysis
 import org.apache.spark.sql.catalyst.parser.{DataTypeParser, 
LegacyTypeStringParser}
@@ -281,8 +282,11 @@ case class StructType(fields: Array[StructField]) extends 
DataType with Seq[Stru
    */
   def apply(name: String): StructField = {
     nameToField.getOrElse(name,
-      throw new IllegalArgumentException(
-        s"$name does not exist. Available: ${fieldNames.mkString(", ")}"))
+      throw new SparkIllegalArgumentException(
+        errorClass = "_LEGACY_ERROR_TEMP_3254",
+        messageParameters = immutable.Map(
+          "name" -> name,
+          "fieldNames" -> fieldNames.mkString(", "))))
   }
 
   /**
@@ -294,9 +298,11 @@ case class StructType(fields: Array[StructField]) extends 
DataType with Seq[Stru
   def apply(names: Set[String]): StructType = {
     val nonExistFields = names -- fieldNamesSet
     if (nonExistFields.nonEmpty) {
-      throw new IllegalArgumentException(
-        s"${nonExistFields.mkString(", ")} do(es) not exist. " +
-          s"Available: ${fieldNames.mkString(", ")}")
+      throw new SparkIllegalArgumentException(
+        errorClass = "_LEGACY_ERROR_TEMP_3253",
+        messageParameters = immutable.Map(
+          "nonExistFields" -> nonExistFields.mkString(", "),
+          "fieldNames" -> fieldNames.mkString(", ")))
     }
     // Preserve the original order of fields.
     StructType(fields.filter(f => names.contains(f.name)))
@@ -309,8 +315,11 @@ case class StructType(fields: Array[StructField]) extends 
DataType with Seq[Stru
    */
   def fieldIndex(name: String): Int = {
     nameToIndex.getOrElse(name,
-      throw new IllegalArgumentException(
-        s"$name does not exist. Available: ${fieldNames.mkString(", ")}"))
+      throw new SparkIllegalArgumentException(
+        errorClass = "_LEGACY_ERROR_TEMP_3252",
+        messageParameters = immutable.Map(
+          "name" -> name,
+          "fieldNames" -> fieldNames.mkString(", "))))
   }
 
   private[sql] def getFieldIndex(name: String): Option[Int] = {
diff --git 
a/sql/core/src/test/resources/sql-tests/results/datetime-formatting-invalid.sql.out
 
b/sql/core/src/test/resources/sql-tests/results/datetime-formatting-invalid.sql.out
index 3a69857e5e54..1eca7c5e69c1 100644
--- 
a/sql/core/src/test/resources/sql-tests/results/datetime-formatting-invalid.sql.out
+++ 
b/sql/core/src/test/resources/sql-tests/results/datetime-formatting-invalid.sql.out
@@ -38,8 +38,13 @@ select date_format('2018-11-17 13:33:33.333', 'qqqqq')
 -- !query schema
 struct<>
 -- !query output
-java.lang.IllegalArgumentException
-Too many pattern letters: q
+org.apache.spark.SparkIllegalArgumentException
+{
+  "errorClass" : "_LEGACY_ERROR_TEMP_3259",
+  "messageParameters" : {
+    "style" : "q"
+  }
+}
 
 
 -- !query
@@ -47,8 +52,13 @@ select date_format('2018-11-17 13:33:33.333', 'QQQQQ')
 -- !query schema
 struct<>
 -- !query output
-java.lang.IllegalArgumentException
-Too many pattern letters: Q
+org.apache.spark.SparkIllegalArgumentException
+{
+  "errorClass" : "_LEGACY_ERROR_TEMP_3259",
+  "messageParameters" : {
+    "style" : "Q"
+  }
+}
 
 
 -- !query
@@ -364,8 +374,13 @@ select date_format('2018-11-17 13:33:33.333', 'A')
 -- !query schema
 struct<>
 -- !query output
-java.lang.IllegalArgumentException
-Illegal pattern character: A
+org.apache.spark.SparkIllegalArgumentException
+{
+  "errorClass" : "_LEGACY_ERROR_TEMP_3258",
+  "messageParameters" : {
+    "c" : "A"
+  }
+}
 
 
 -- !query
@@ -373,8 +388,13 @@ select date_format('2018-11-17 13:33:33.333', 'n')
 -- !query schema
 struct<>
 -- !query output
-java.lang.IllegalArgumentException
-Illegal pattern character: n
+org.apache.spark.SparkIllegalArgumentException
+{
+  "errorClass" : "_LEGACY_ERROR_TEMP_3258",
+  "messageParameters" : {
+    "c" : "n"
+  }
+}
 
 
 -- !query
@@ -382,8 +402,13 @@ select date_format('2018-11-17 13:33:33.333', 'N')
 -- !query schema
 struct<>
 -- !query output
-java.lang.IllegalArgumentException
-Illegal pattern character: N
+org.apache.spark.SparkIllegalArgumentException
+{
+  "errorClass" : "_LEGACY_ERROR_TEMP_3258",
+  "messageParameters" : {
+    "c" : "N"
+  }
+}
 
 
 -- !query
@@ -391,8 +416,13 @@ select date_format('2018-11-17 13:33:33.333', 'p')
 -- !query schema
 struct<>
 -- !query output
-java.lang.IllegalArgumentException
-Illegal pattern character: p
+org.apache.spark.SparkIllegalArgumentException
+{
+  "errorClass" : "_LEGACY_ERROR_TEMP_3258",
+  "messageParameters" : {
+    "c" : "p"
+  }
+}
 
 
 -- !query
@@ -468,8 +498,13 @@ select date_format('2018-11-17 13:33:33.333', 'e')
 -- !query schema
 struct<>
 -- !query output
-java.lang.IllegalArgumentException
-All week-based patterns are unsupported since Spark 3.0, detected: e, Please 
use the SQL function EXTRACT instead
+org.apache.spark.SparkIllegalArgumentException
+{
+  "errorClass" : "_LEGACY_ERROR_TEMP_3257",
+  "messageParameters" : {
+    "c" : "e"
+  }
+}
 
 
 -- !query
@@ -477,8 +512,13 @@ select date_format('2018-11-17 13:33:33.333', 'c')
 -- !query schema
 struct<>
 -- !query output
-java.lang.IllegalArgumentException
-All week-based patterns are unsupported since Spark 3.0, detected: c, Please 
use the SQL function EXTRACT instead
+org.apache.spark.SparkIllegalArgumentException
+{
+  "errorClass" : "_LEGACY_ERROR_TEMP_3257",
+  "messageParameters" : {
+    "c" : "c"
+  }
+}
 
 
 -- !query
@@ -486,8 +526,13 @@ select date_format('2018-11-17 13:33:33.333', 'B')
 -- !query schema
 struct<>
 -- !query output
-java.lang.IllegalArgumentException
-Illegal pattern character: B
+org.apache.spark.SparkIllegalArgumentException
+{
+  "errorClass" : "_LEGACY_ERROR_TEMP_3258",
+  "messageParameters" : {
+    "c" : "B"
+  }
+}
 
 
 -- !query
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala 
b/sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala
index f2daa8339cfe..3c2543866615 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala
@@ -23,7 +23,7 @@ import java.util.Locale
 
 import scala.jdk.CollectionConverters._
 
-import org.apache.spark.{SparkException, SparkRuntimeException}
+import org.apache.spark.{SparkException, SparkIllegalArgumentException, 
SparkRuntimeException}
 import org.apache.spark.sql.catalyst.InternalRow
 import org.apache.spark.sql.catalyst.expressions.{Literal, StructsToJson}
 import org.apache.spark.sql.catalyst.expressions.Cast._
@@ -1157,17 +1157,12 @@ class JsonFunctionsSuite extends QueryTest with 
SharedSparkSession {
   test("SPARK-33286: from_json - combined error messages") {
     val df = Seq("""{"a":1}""").toDF("json")
     val invalidJsonSchema = """{"fields": [{"a":123}], "type": "struct"}"""
-    val invalidJsonSchemaReason = "Failed to convert the JSON string 
'{\"a\":123}' to a field."
     checkError(
-      exception = intercept[AnalysisException] {
+      exception = intercept[SparkIllegalArgumentException] {
         df.select(from_json($"json", invalidJsonSchema, Map.empty[String, 
String])).collect()
       },
-      errorClass = "INVALID_SCHEMA.PARSE_ERROR",
-      parameters = Map(
-        "inputSchema" -> "\"{\"fields\": [{\"a\":123}], \"type\": 
\"struct\"}\"",
-        "reason" -> invalidJsonSchemaReason
-      )
-    )
+      errorClass = "_LEGACY_ERROR_TEMP_3250",
+      parameters = Map("other" -> """{"a":123}"""))
 
     val invalidDataType = "MAP<INT, cow>"
     val invalidDataTypeReason = "Unrecognized token 'MAP': " +


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

Reply via email to