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-30919][SQL] Make interval multiply and divide's overflow behavior consistent with other operations #27672

Closed
wants to merge 9 commits into from

Conversation

yaooqinn
Copy link
Member

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

…ior consisitent with other interval operations
@SparkQA
Copy link

SparkQA commented Feb 21, 2020

Test build #118803 has finished for PR 27672 at commit ae9869c.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds the following public classes (experimental):
  • abstract class IntervalNumOperation(interval: Expression, num: Expression)

@SparkQA
Copy link

SparkQA commented Feb 22, 2020

Test build #118814 has finished for PR 27672 at commit a05af57.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@yaooqinn
Copy link
Member Author

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)
Copy link
Contributor

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.

Copy link
Contributor

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

Copy link
Member Author

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
Copy link
Contributor

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?

Copy link
Member Author

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)
Copy link
Contributor

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)
Copy link
Contributor

Choose a reason for hiding this comment

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

ditto

@cloud-fan
Copy link
Contributor

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
Copy link
Contributor

Choose a reason for hiding this comment

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

@SparkQA
Copy link

SparkQA commented Feb 24, 2020

Test build #118860 has finished for PR 27672 at commit 0c82efe.

  • This patch fails SparkR unit tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@@ -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)
Copy link
Contributor

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.

@SparkQA
Copy link

SparkQA commented Feb 24, 2020

Test build #118865 has finished for PR 27672 at commit 2bf07a3.

  • This patch fails Spark unit tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Feb 24, 2020

Test build #118871 has finished for PR 27672 at commit ebb0011.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds the following public classes (experimental):
  • case class MultiplyInterval(
  • case class DivideInterval(

@SparkQA
Copy link

SparkQA commented Feb 24, 2020

Test build #118878 has finished for PR 27672 at commit e96135f.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds the following public classes (experimental):
  • abstract class IntervalNumOperation(

@@ -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)
Copy link
Contributor

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)
Copy link
Contributor

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 = {
Copy link
Contributor

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.

@SparkQA
Copy link

SparkQA commented Feb 25, 2020

Test build #118898 has finished for PR 27672 at commit 7ece786.

  • This patch fails due to an unknown error code, -9.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Feb 25, 2020

Test build #118906 has finished for PR 27672 at commit 1a67228.

  • This patch fails due to an unknown error code, -9.
  • This patch merges cleanly.
  • This patch adds no public classes.

assert(new CalendarInterval(-2, 0, -MICROS_PER_MINUTE) === func(interval, 0.5))
}

assert(divide(new CalendarInterval(Int.MaxValue, Int.MaxValue, 0), 0.9) ===
Copy link
Contributor

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
Copy link
Contributor

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)
Copy link
Contributor

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.

@SparkQA
Copy link

SparkQA commented Feb 25, 2020

Test build #118910 has finished for PR 27672 at commit 9db1f37.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@cloud-fan
Copy link
Contributor

thanks, merging to master/3.0!

@cloud-fan cloud-fan closed this in 761209c Feb 25, 2020
@cloud-fan
Copy link
Contributor

feel free to address #27672 (comment) in your next PR :)

cloud-fan pushed a commit that referenced this pull request Feb 25, 2020
…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>
maropu pushed a commit that referenced this pull request Feb 27, 2020
…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>
maropu pushed a commit that referenced this pull request Feb 27, 2020
…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>
sjincho pushed a commit to sjincho/spark that referenced this pull request Apr 15, 2020
…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>
sjincho pushed a commit to sjincho/spark that referenced this pull request Apr 15, 2020
…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>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
3 participants