Skip to content
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
8 changes: 7 additions & 1 deletion core/src/main/resources/error/error-classes.json
Original file line number Diff line number Diff line change
Expand Up @@ -78,7 +78,7 @@
},
"DIVIDE_BY_ZERO" : {
"message" : [
"Division by zero. Use `try_divide` to tolerate divisor being 0 and return NULL instead. If necessary set <config> to \"false\" (except for ANSI interval type) to bypass this error."
"Division by zero. Use `try_divide` to tolerate divisor being 0 and return NULL instead. If necessary set <config> to \"false\" to bypass this error."
],
"sqlState" : "22012"
},
Expand Down Expand Up @@ -210,6 +210,12 @@
"<message>"
]
},
"INTERVAL_DIVIDED_BY_ZERO" : {
"message" : [
"Division by zero. Use `try_divide` to tolerate divisor being 0 and return NULL instead."
],
"sqlState" : "22012"
},
"INVALID_ARRAY_INDEX" : {
"message" : [
"The index <indexValue> is out of bounds. The array has <arraySize> elements. Use `try_element_at` and increase the array index by 1(the starting array index is 1 for `try_element_at`) to tolerate accessing element at invalid index and return NULL instead. If necessary set <ansiConfig> to \"false\" to bypass this error."
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -168,7 +168,7 @@ class SparkThrowableSuite extends SparkFunSuite {
"[DIVIDE_BY_ZERO] Division by zero. " +
"Use `try_divide` to tolerate divisor being 0 and return NULL instead. " +
"If necessary set foo to \"false\" " +
"(except for ANSI interval type) to bypass this error.")
"to bypass this error.")
}

test("Error message is formatted") {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -617,18 +617,22 @@ trait IntervalDivide {
num: Any,
context: Option[SQLQueryContext]): Unit = dataType match {
case _: DecimalType =>
if (num.asInstanceOf[Decimal].isZero) throw QueryExecutionErrors.divideByZeroError(context)
case _ => if (num == 0) throw QueryExecutionErrors.divideByZeroError(context)
if (num.asInstanceOf[Decimal].isZero) {
throw QueryExecutionErrors.intervalDividedByZeroError(context)
}
case _ => if (num == 0) throw QueryExecutionErrors.intervalDividedByZeroError(context)
}

def divideByZeroCheckCodegen(
dataType: DataType,
value: String,
errorContextReference: String): String = dataType match {
// scalastyle:off line.size.limit
case _: DecimalType =>
s"if ($value.isZero()) throw QueryExecutionErrors.divideByZeroError($errorContextReference);"
s"if ($value.isZero()) throw QueryExecutionErrors.intervalDividedByZeroError($errorContextReference);"
case _ =>
s"if ($value == 0) throw QueryExecutionErrors.divideByZeroError($errorContextReference);"
s"if ($value == 0) throw QueryExecutionErrors.intervalDividedByZeroError($errorContextReference);"
// scalastyle:on line.size.limit
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -733,7 +733,7 @@ object IntervalUtils {
* @throws ArithmeticException if the result overflows any field value or divided by zero
*/
def divideExact(interval: CalendarInterval, num: Double): CalendarInterval = {
if (num == 0) throw QueryExecutionErrors.divideByZeroError(None)
if (num == 0) throw QueryExecutionErrors.intervalDividedByZeroError(None)
fromDoubles(interval.months / num, interval.days / num, interval.microseconds / num)
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -204,6 +204,14 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase {
summary = getSummary(context))
}

def intervalDividedByZeroError(context: Option[SQLQueryContext]): ArithmeticException = {
new SparkArithmeticException(
errorClass = "INTERVAL_DIVIDED_BY_ZERO",
messageParameters = Array.empty,
context = context,
summary = getSummary(context))
}

def invalidArrayIndexError(
index: Int,
numElements: Int,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -225,7 +225,7 @@ select interval '2 seconds' / 0
struct<>
-- !query output
org.apache.spark.SparkArithmeticException
[DIVIDE_BY_ZERO] Division by zero. Use `try_divide` to tolerate divisor being 0 and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" (except for ANSI interval type) to bypass this error.
[INTERVAL_DIVIDED_BY_ZERO] Division by zero. Use `try_divide` to tolerate divisor being 0 and return NULL instead.
== SQL(line 1, position 8) ==
select interval '2 seconds' / 0
^^^^^^^^^^^^^^^^^^^^^^^^
Expand Down Expand Up @@ -261,7 +261,7 @@ select interval '2' year / 0
struct<>
-- !query output
org.apache.spark.SparkArithmeticException
[DIVIDE_BY_ZERO] Division by zero. Use `try_divide` to tolerate divisor being 0 and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" (except for ANSI interval type) to bypass this error.
[INTERVAL_DIVIDED_BY_ZERO] Division by zero. Use `try_divide` to tolerate divisor being 0 and return NULL instead.
== SQL(line 1, position 8) ==
select interval '2' year / 0
^^^^^^^^^^^^^^^^^^^^^
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -201,7 +201,7 @@ select interval '2 seconds' / 0
struct<>
-- !query output
org.apache.spark.SparkArithmeticException
[DIVIDE_BY_ZERO] Division by zero. Use `try_divide` to tolerate divisor being 0 and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" (except for ANSI interval type) to bypass this error.
[INTERVAL_DIVIDED_BY_ZERO] Division by zero. Use `try_divide` to tolerate divisor being 0 and return NULL instead.
== SQL(line 1, position 8) ==
select interval '2 seconds' / 0
^^^^^^^^^^^^^^^^^^^^^^^^
Expand Down Expand Up @@ -237,7 +237,7 @@ select interval '2' year / 0
struct<>
-- !query output
org.apache.spark.SparkArithmeticException
[DIVIDE_BY_ZERO] Division by zero. Use `try_divide` to tolerate divisor being 0 and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" (except for ANSI interval type) to bypass this error.
[INTERVAL_DIVIDED_BY_ZERO] Division by zero. Use `try_divide` to tolerate divisor being 0 and return NULL instead.
== SQL(line 1, position 8) ==
select interval '2' year / 0
^^^^^^^^^^^^^^^^^^^^^
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -647,7 +647,7 @@ select bigint('9223372036854775800') / bigint('0')
struct<>
-- !query output
org.apache.spark.SparkArithmeticException
[DIVIDE_BY_ZERO] Division by zero. Use `try_divide` to tolerate divisor being 0 and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" (except for ANSI interval type) to bypass this error.
[DIVIDE_BY_ZERO] Division by zero. Use `try_divide` to tolerate divisor being 0 and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error.
== SQL(line 1, position 8) ==
select bigint('9223372036854775800') / bigint('0')
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
Expand All @@ -659,7 +659,7 @@ select bigint('-9223372036854775808') / smallint('0')
struct<>
-- !query output
org.apache.spark.SparkArithmeticException
[DIVIDE_BY_ZERO] Division by zero. Use `try_divide` to tolerate divisor being 0 and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" (except for ANSI interval type) to bypass this error.
[DIVIDE_BY_ZERO] Division by zero. Use `try_divide` to tolerate divisor being 0 and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error.
== SQL(line 1, position 8) ==
select bigint('-9223372036854775808') / smallint('0')
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
Expand All @@ -671,7 +671,7 @@ select smallint('100') / bigint('0')
struct<>
-- !query output
org.apache.spark.SparkArithmeticException
[DIVIDE_BY_ZERO] Division by zero. Use `try_divide` to tolerate divisor being 0 and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" (except for ANSI interval type) to bypass this error.
[DIVIDE_BY_ZERO] Division by zero. Use `try_divide` to tolerate divisor being 0 and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error.
== SQL(line 1, position 8) ==
select smallint('100') / bigint('0')
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -174,7 +174,7 @@ SELECT 1 AS one FROM test_having WHERE 1/a = 1 HAVING 1 < 2
struct<>
-- !query output
org.apache.spark.SparkArithmeticException
[DIVIDE_BY_ZERO] Division by zero. Use `try_divide` to tolerate divisor being 0 and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" (except for ANSI interval type) to bypass this error.
[DIVIDE_BY_ZERO] Division by zero. Use `try_divide` to tolerate divisor being 0 and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error.
== SQL(line 1, position 40) ==
...1 AS one FROM test_having WHERE 1/a = 1 HAVING 1 < 2
^^^
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -174,7 +174,7 @@ SELECT 1 AS one FROM test_having WHERE 1/udf(a) = 1 HAVING 1 < 2
struct<>
-- !query output
org.apache.spark.SparkArithmeticException
[DIVIDE_BY_ZERO] Division by zero. Use `try_divide` to tolerate divisor being 0 and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" (except for ANSI interval type) to bypass this error.
[DIVIDE_BY_ZERO] Division by zero. Use `try_divide` to tolerate divisor being 0 and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error.
== SQL(line 1, position 40) ==
...1 AS one FROM test_having WHERE 1/udf(a) = 1 HAVING 1 < 2
^^^^^^^^
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -48,7 +48,7 @@ class QueryExecutionAnsiErrorsSuite extends QueryTest with QueryErrorsSuiteBase
msg =
"Division by zero. Use `try_divide` to tolerate divisor being 0 and return NULL instead. " +
"If necessary set " +
s"""$ansiConf to "false" (except for ANSI interval type) to bypass this error.""" +
s"""$ansiConf to "false" to bypass this error.""" +
"""
|== SQL(line 1, position 8) ==
|select 6/0
Expand All @@ -57,6 +57,16 @@ class QueryExecutionAnsiErrorsSuite extends QueryTest with QueryErrorsSuiteBase
sqlState = Some("22012"))
}

test("INTERVAL_DIVIDED_BY_ZERO: interval divided by zero") {
checkError(
exception = intercept[SparkArithmeticException] {
sql("select interval 1 day / 0").collect()
},
errorClass = "INTERVAL_DIVIDED_BY_ZERO",
parameters = Map.empty
)
}

test("INVALID_FRACTION_OF_SECOND: in the function make_timestamp") {
checkError(
exception = intercept[SparkDateTimeException] {
Expand Down