-
Notifications
You must be signed in to change notification settings - Fork 28.1k
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-30919][SQL] Make interval multiply and divide's overflow behavior consistent with other operations #27672
Conversation
…ior consisitent with other interval operations
Test build #118803 has finished for PR 27672 at commit
|
Test build #118814 has finished for PR 27672 at commit
|
cc @cloud-fan @gatorsmile, thanks for reviewing. |
extends IntervalNumOperation(interval, num) { | ||
|
||
override protected def operation(interval: CalendarInterval, num: Double): CalendarInterval = { | ||
if (checkOverflow) multiplyExact(interval, num) else multiply(interval, num) |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
we shouldn't check the ansi flag for each row.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
we should define operation
as a val: val operation: (CalendarInterval, Double) => CalendarInterval
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
ah, thanks
} else if (daysWithFraction < Int.MinValue) { | ||
Int.MinValue | ||
} else { | ||
daysWithFraction.toInt |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
what if we just call daysWithFraction.toInt
?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
OK
extends IntervalNumOperation(interval, num) { | ||
|
||
override protected val operation: (CalendarInterval, Double) => CalendarInterval = | ||
(interval, num) => if (checkOverflow) multiplyExact(interval, num) else multiply(interval, num) |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This still checks overflow per-row...
if (checkOverflow) multiplyExact else multiply
extends IntervalNumOperation(interval, num) { | ||
|
||
override protected val operation: (CalendarInterval, Double) => CalendarInterval = | ||
(interval, num) => if (checkOverflow) divideExact(interval, num) else divide(interval, num) |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
ditto
LGTM except one comment |
extends BinaryExpression with ImplicitCastInputTypes with Serializable { | ||
override def left: Expression = interval | ||
override def right: Expression = num | ||
|
||
protected val checkOverflow: Boolean = SQLConf.get.ansiEnabled |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
can we follow https://github.com/apache/spark/pull/27683/files#diff-1516b10738479bbe190fb4e239258473R36 and make it a parameter?
Test build #118860 has finished for PR 27672 at commit
|
@@ -308,12 +308,12 @@ class Analyzer( | |||
case _ => s | |||
} | |||
case m @ Multiply(l, r) if m.childrenResolved => (l.dataType, r.dataType) match { | |||
case (CalendarIntervalType, _) => MultiplyInterval(l, r) | |||
case (_, CalendarIntervalType) => MultiplyInterval(r, l) | |||
case (CalendarIntervalType, _) => MultiplyInterval(l, r, conf.ansiEnabled) |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
SQLConf.get
is reliable now. Let's follow other PRs and use the default parameter value.
Test build #118865 has finished for PR 27672 at commit
|
Test build #118871 has finished for PR 27672 at commit
|
Test build #118878 has finished for PR 27672 at commit
|
@@ -113,12 +114,14 @@ object ExtractIntervalPart { | |||
abstract class IntervalNumOperation( | |||
interval: Expression, | |||
num: Expression, | |||
operation: (CalendarInterval, Double) => CalendarInterval, | |||
operationName: String) | |||
val checkOverflow: Boolean = SQLConf.get.ansiEnabled) |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
hey wait. Does it mean this value can change when we transform/copy MultiplyInterval
/DivideInterval
?
assert(new CalendarInterval(0, 0, -15 * MICROS_PER_SECOND) === func(interval, 2)) | ||
assert(new CalendarInterval(-2, 0, -MICROS_PER_MINUTE) === func(interval, 0.5)) | ||
} | ||
assert(divide(new CalendarInterval(123, 456, 789), 0) === null) |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
can we test one more case that divide
returns Int.Max when overflow?
interval: String, | ||
num: Double, | ||
expected: String, | ||
configs: Seq[String] = Seq("true", "false")): Unit = { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
nit: isAnsi: Option[Boolean]
? If it's None then we test both configs.
Test build #118898 has finished for PR 27672 at commit
|
Test build #118906 has finished for PR 27672 at commit
|
assert(new CalendarInterval(-2, 0, -MICROS_PER_MINUTE) === func(interval, 0.5)) | ||
} | ||
|
||
assert(divide(new CalendarInterval(Int.MaxValue, Int.MaxValue, 0), 0.9) === |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
we should test divideExact
as well.
val expectedRes = safeStringToInterval(expected) | ||
Seq("true", "false").foreach { v => | ||
val configs = if (isAnsi.isEmpty) { Seq("true", "false") } else isAnsi.map(_.toString).toSeq |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
super nit: { Seq("true", "false") }
-> Seq("true", "false")
assert(divide(interval, 0.9) === new CalendarInterval(Int.MaxValue, Int.MaxValue, | ||
((Int.MaxValue / 9.0) * MICROS_PER_DAY).round)) | ||
try { | ||
divideExact(interval, 0.9) |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
we should use intercept
to assert a failure. I'm OK to leave it as we already use try-catch
in this test case.
Test build #118910 has finished for PR 27672 at commit
|
thanks, merging to master/3.0! |
feel free to address #27672 (comment) in your next PR :) |
…ior consistent with other operations ### What changes were proposed in this pull request? The current behavior of interval multiply and divide follows the ANSI SQL standard when overflow, it is compatible with other operations when `spark.sql.ansi.enabled` is true, but not compatible when `spark.sql.ansi.enabled` is false. When `spark.sql.ansi.enabled` is false, as the factor is a double value, so it should use java's rounding or truncation behavior for casting double to integrals. when divided by zero, it returns `null`. we also follow the natural rules for intervals as defined in the Gregorian calendar, so we do not add the month fraction to days but add days fraction to microseconds. ### Why are the changes needed? Make interval multiply and divide's overflow behavior consistent with other interval operations ### Does this PR introduce any user-facing change? no, these are new features in 3.0 ### How was this patch tested? add uts Closes #27672 from yaooqinn/SPARK-30919. Authored-by: Kent Yao <yaooqinn@hotmail.com> Signed-off-by: Wenchen Fan <wenchen@databricks.com> (cherry picked from commit 761209c) Signed-off-by: Wenchen Fan <wenchen@databricks.com>
…t failures in IntervalUtilsSuite ### What changes were proposed in this pull request? In this PR, I addressed the comment from #27672 (comment) to use `intercept` instead of `try-catch` block to assert failures in the IntervalUtilsSuite ### Why are the changes needed? improve tests ### Does this PR introduce any user-facing change? no ### How was this patch tested? Nah Closes #27700 from yaooqinn/intervaltest. Authored-by: Kent Yao <yaooqinn@hotmail.com> Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
…t failures in IntervalUtilsSuite ### What changes were proposed in this pull request? In this PR, I addressed the comment from #27672 (comment) to use `intercept` instead of `try-catch` block to assert failures in the IntervalUtilsSuite ### Why are the changes needed? improve tests ### Does this PR introduce any user-facing change? no ### How was this patch tested? Nah Closes #27700 from yaooqinn/intervaltest. Authored-by: Kent Yao <yaooqinn@hotmail.com> Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org> (cherry picked from commit 2d2706c) Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
…ior consistent with other operations ### What changes were proposed in this pull request? The current behavior of interval multiply and divide follows the ANSI SQL standard when overflow, it is compatible with other operations when `spark.sql.ansi.enabled` is true, but not compatible when `spark.sql.ansi.enabled` is false. When `spark.sql.ansi.enabled` is false, as the factor is a double value, so it should use java's rounding or truncation behavior for casting double to integrals. when divided by zero, it returns `null`. we also follow the natural rules for intervals as defined in the Gregorian calendar, so we do not add the month fraction to days but add days fraction to microseconds. ### Why are the changes needed? Make interval multiply and divide's overflow behavior consistent with other interval operations ### Does this PR introduce any user-facing change? no, these are new features in 3.0 ### How was this patch tested? add uts Closes apache#27672 from yaooqinn/SPARK-30919. Authored-by: Kent Yao <yaooqinn@hotmail.com> Signed-off-by: Wenchen Fan <wenchen@databricks.com>
…t failures in IntervalUtilsSuite ### What changes were proposed in this pull request? In this PR, I addressed the comment from apache#27672 (comment) to use `intercept` instead of `try-catch` block to assert failures in the IntervalUtilsSuite ### Why are the changes needed? improve tests ### Does this PR introduce any user-facing change? no ### How was this patch tested? Nah Closes apache#27700 from yaooqinn/intervaltest. Authored-by: Kent Yao <yaooqinn@hotmail.com> Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
What changes were proposed in this pull request?
The current behavior of interval multiply and divide follows the ANSI SQL standard when overflow, it is compatible with other operations when
spark.sql.ansi.enabled
is true, but not compatible whenspark.sql.ansi.enabled
is false.When
spark.sql.ansi.enabled
is false, as the factor is a double value, so it should use java's rounding or truncation behavior for casting double to integrals. when divided by zero, it returnsnull
. we also follow the natural rules for intervals as defined in the Gregorian calendar, so we do not add the month fraction to days but add days fraction to microseconds.Why are the changes needed?
Make interval multiply and divide's overflow behavior consistent with other interval operations
Does this PR introduce any user-facing change?
no, these are new features in 3.0
How was this patch tested?
add uts