Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[SPARK-47045][SQL] Replace IllegalArgumentException by SparkIllegalArgumentException in sql/api #45098

Closed
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
3 changes: 2 additions & 1 deletion R/pkg/tests/fulltests/test_streaming.R
Original file line number Diff line number Diff line change
Expand Up @@ -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.")
Expand Down
70 changes: 70 additions & 0 deletions common/utils/src/main/resources/error/error-classes.json
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: punctuation in detected: <c>, Please could be improved, feel free to ignore though :)

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I preserved punctuation of the original error intentionally:
https://github.com/apache/spark/pull/45098/files#diff-1ada8897c412e27c3f73c8f5449f62f1fdc805b979cc7cfa35fcf8ad031529bbL313
to don't break existing tests. We should improve errors while assigning proper error class names, and writing tests for the errors.

]
},
"_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>"
Expand Down
11 changes: 8 additions & 3 deletions sql/api/src/main/scala/org/apache/spark/sql/Row.scala
Original file line number Diff line number Diff line change
Expand Up @@ -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}
Expand Down Expand Up @@ -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)
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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

/**
Expand Down Expand Up @@ -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))
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -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._
Expand Down Expand Up @@ -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)
Expand Down Expand Up @@ -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'.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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}
Expand Down Expand Up @@ -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) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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._
Expand Down Expand Up @@ -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)
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand All @@ -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))
Expand Down
19 changes: 10 additions & 9 deletions sql/api/src/main/scala/org/apache/spark/sql/types/DataType.scala
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down Expand Up @@ -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)))
}
}

Expand Down Expand Up @@ -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 {
Expand All @@ -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(
Expand Down
25 changes: 17 additions & 8 deletions sql/api/src/main/scala/org/apache/spark/sql/types/StructType.scala
Original file line number Diff line number Diff line change
Expand Up @@ -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}
Expand Down Expand Up @@ -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(", "))))
}

/**
Expand All @@ -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)))
Expand All @@ -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] = {
Expand Down
Loading