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-26218][SQL] Overflow on arithmetic operations returns incorrect result #21599

Closed
wants to merge 29 commits into from

Conversation

mgaido91
Copy link
Contributor

What changes were proposed in this pull request?

When an overflow occurs performing an arithmetic operation, we are returning an incorrect value. Instead, we should throw an exception, as stated in the SQL standard.

How was this patch tested?

added UT + existing UTs (improved)

@SparkQA
Copy link

SparkQA commented Jun 20, 2018

Test build #92132 has finished for PR 21599 at commit 5c662f6.

  • This patch fails Scala style tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Jun 20, 2018

Test build #92133 has finished for PR 21599 at commit fad75fa.

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

@SparkQA
Copy link

SparkQA commented Jun 20, 2018

Test build #92140 has finished for PR 21599 at commit 8591417.

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

@mgaido91
Copy link
Contributor Author

@cloud-fan @gatorsmile the main issue which is causing the UT failures, now, is that since before we were allowing overflows, in aggregations we could have an overflow eventually fixed by another overflow on the opposite direction (see the two UT failures in the Range suite).

The problem here is that we can know whether an overflow occurs only checking at the incoming operands, so we cannot defer after the whole aggregation is performed to check if the overflow occurred or not. The way MySQL deals with this case is returning a DECIMAL as result of the aggregation (it the starting datatype is BIGINT, then a DECIMAL(41, 0)). Probably we can do something similar, but that would be a quite huge behavior change and I am not sure it is acceptable in a minor release.

Therefore, I'd like to know your thoughts about this, since on one side it is a pretty serious bug which should be fixed, on the other I cannot think of any solution which doesn't involve behavior changes.
Thank you.

@cloud-fan
Copy link
Contributor

is this a regression in Spark 2.3? which commit caused it?

@mgaido91
Copy link
Contributor Author

@cloud-fan it is not a regression, it has been like that at least since 2.0...

@SparkQA
Copy link

SparkQA commented Jun 21, 2018

Test build #92172 has finished for PR 21599 at commit 9c3df7d.

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

override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = dataType match {
case _: DecimalType =>
defineCodeGen(ctx, ev, (eval1, eval2) => s"$eval1.$decimalMethod($eval2)")
case CalendarIntervalType =>
defineCodeGen(ctx, ev, (eval1, eval2) => s"$eval1.$calendarIntervalMethod($eval2)")
// In the following cases, overflow can happen, so we need to check the result is valid.
// Otherwise we throw an ArithmeticException
Copy link
Contributor

Choose a reason for hiding this comment

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

In current Spark we are very conservative about runtime error, as it may break the data pipeline middle away, and returning null is a commonly used strategy. Shall we follow it here? We can throw exception when we have a strict mode.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Personally, I am quite against returning null. It is not something a user expects, so he/she is likely not to check for it (when I see a NULL myself, I think that one of the 2 operands was NULL, not that an overflow occurred), so he/she won't realize the issue and would find corrupted data. Moreover, this is not how RDBMS behaves and it is against SQL standard. So I think that the behavior which was chosen for DECIMAL was wrong and I'd prefer not to introduce the same behavior also in other places.

Anyway I see your point about consistency over the codebase and it makes sense.

I'd love to know @gatorsmile and @hvanhovell's opinions too.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

@gatorsmile @hvanhovell do you have time to check this and give your opinion here? Thanks.

Copy link
Member

Choose a reason for hiding this comment

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

Why don't we fix it to return null and throw an exception when the configuration is on? Overflowed value is already quite pointless and changing the behaviour to return null by default might not be so harmful.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Thanks for your comment @HyukjinKwon . The issue with returning null is described in #21599 (comment) (moreover this behavior would be against SQL standard, but that's a minor point).

Copy link
Member

Choose a reason for hiding this comment

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

As I am working on #25300, +1 for returning null on overflow by default with @cloud-fan @HyukjinKwon . This makes arithmetic operations and casting consistent.
I think null is better than a non-sense number on overflow.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

I don't agree on this proposal for the reasons explained in the comment I mentioned earlier. Making all arithmetic operations nullable is a too broad change and I think it is not worth for the little benefit.

@SparkQA
Copy link

SparkQA commented Jun 22, 2018

Test build #92208 has finished for PR 21599 at commit ebdaf61.

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

@SparkQA
Copy link

SparkQA commented Jun 22, 2018

Test build #92216 has finished for PR 21599 at commit a0b862e.

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

@mgaido91
Copy link
Contributor Author

mgaido91 commented Jul 9, 2018

kindly ping @gatorsmile @hvanhovell

@mgaido91
Copy link
Contributor Author

@cloud-fan @gatorsmile @hvanhovell what do you think about deciding a strategy about how to handle this case and try to make it happen in 2.4 as this is a correctness bug? Thanks.

@SparkQA
Copy link

SparkQA commented Jul 16, 2018

Test build #93107 has finished for PR 21599 at commit 7bba22f.

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

@mgaido91
Copy link
Contributor Author

kindly ping @cloud-fan @gatorsmile @hvanhovell

@cloud-fan
Copy link
Contributor

I prefer returning null, and introduce a strict mode in Spark 3.0. We can revisit all the returning null cases and think if we should fail if strict mode is on.

@mgaido91
Copy link
Contributor Author

@cloud-fan ok, but then all arithmetic operations should be always nullable = true. I am not sure whether this can introduce performance regression (the additional checks are also going to worsen performance, but I don't see any way to avoid it).

If there are no objections, I am going to update the PR according to your suggestion. Thanks.

@cloud-fan
Copy link
Contributor

ah that's a good point. Then maybe we should keep this "java style behavior" and add "strict mode" later.

cc @rxin @rednaxelafx

@cloud-fan
Copy link
Contributor

I don't think we can change behavior like this without introducing a config. In Spark 3.0 we will add a "strict mode" and closely follow SQL standard. But for Spark 2.4, how about we keep the "java style behavior" as it is? We can add some document about it though.

@mgaido91
Copy link
Contributor Author

mgaido91 commented Aug 2, 2018

@cloud-fan I think this is just a wrong behavior (being against SQL standard is not the only issue), so I am not sure that makes sense to have it controlled by a config. Anyway, I agree that as it is a behavior change, we can target it for 3.0 and add some documentation for it now, in order to let users know about this issue.
Thanks.

@mgaido91 mgaido91 changed the title [SPARK-24598][SQL] Overflow on arithmetic operations returns incorrect result [SPARK-26218][SQL] Overflow on arithmetic operations returns incorrect result Nov 29, 2018
* This method test against binary expressions by feeding them arbitrary literals of `dataType1`
* and `dataType2`.
*/
def checkConsistencyBetweenInterpretedAndCodegenAllowingException(
Copy link
Contributor

Choose a reason for hiding this comment

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

it seems to me that checkConsistencyBetweenInterpretedAndCodegen should handle exceptions as well, as that's part of the consistency check.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

I am not sure what is you suggestion here. Are you suggesting that we should collapse these 2 methods?

checkValues(Decimal(Double.MaxValue), Double.MaxValue, 0L)
checkValues(Decimal(Double.MinValue), Double.MinValue, 0L)
assert(Decimal(Double.MaxValue).toDouble == Double.MaxValue)
assert(Decimal(Double.MinValue).toDouble == Double.MinValue)
Copy link
Contributor

Choose a reason for hiding this comment

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

why change this?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

because toLong overflows in this case

Copy link
Contributor

Choose a reason for hiding this comment

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

I think we have reverted the changes in Decimal.toLong?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

yes, I can revert these changes and the tests pass, but here we are basically checking that we are returning wrong values, so I don't think it makes any sense. Anyway I can revert the change if you think we should.

Copy link
Contributor

Choose a reason for hiding this comment

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

Yes we know Decimal.toLong can return wrong result, and we are going to fix it by making cast follow SQL standard. Let's revert it and fix it in the cast PR.

-- We cannot test this when checkOverflow=true here
-- because exception happens in the executors and the
-- output stacktrace cannot have an exact match
set spark.sql.arithmetic.checkOverflow=false;
Copy link
Contributor

Choose a reason for hiding this comment

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

is it necessary? It's false by default.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

I put it for enforcing this so that eventual changes on the default value won't affect here.

@cloud-fan
Copy link
Contributor

LGTM except a few comments, thanks for working on it!

@mgaido91
Copy link
Contributor Author

thank you @cloud-fan for you help on this!

@SparkQA
Copy link

SparkQA commented Jul 30, 2019

Test build #108409 has finished for PR 21599 at commit 582d148.

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

@SparkQA
Copy link

SparkQA commented Jul 30, 2019

Test build #108418 has finished for PR 21599 at commit b809a3f.

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

@maropu
Copy link
Member

maropu commented Jul 31, 2019

retest this please

@SparkQA
Copy link

SparkQA commented Jul 31, 2019

Test build #108443 has finished for PR 21599 at commit b809a3f.

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

@maropu
Copy link
Member

maropu commented Jul 31, 2019

retest this please

Copy link
Member

@gengliangwang gengliangwang left a comment

Choose a reason for hiding this comment

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

@mgaido91 Thanks for the work. The PR overall LGTM.
I am aware that it has been a lot of work. But could you add corner test cases for the overflow of byte/short/int types?

@SparkQA
Copy link

SparkQA commented Jul 31, 2019

Test build #108453 has finished for PR 21599 at commit b809a3f.

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

@mgaido91
Copy link
Contributor Author

thanks @gengliangwang , I added them.

val e3 = Multiply(maxLongLiteral, Literal(2L))
val e4 = Add(minLongLiteral, minLongLiteral)
val e5 = Subtract(minLongLiteral, maxLongLiteral)
val e6 = Multiply(minLongLiteral, minLongLiteral)
Copy link
Contributor

Choose a reason for hiding this comment

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

shall we also test UnaryMinus? BTW do you know why the previous mistake in UnaryMinus code was not caught by the existing tests?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

yes, it was my fault, I was not testing "normal" cases with the flag turned on. I fixed it. UnaryMinus cases are already tested in its UT, I think it is useless to add them here too.

Copy link
Contributor

Choose a reason for hiding this comment

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

makes sense, thanks!

@SparkQA
Copy link

SparkQA commented Jul 31, 2019

Test build #108467 has finished for PR 21599 at commit ce3ed2b.

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

@cloud-fan
Copy link
Contributor

thanks, merging to master!

@cloud-fan cloud-fan closed this in ee41001 Aug 1, 2019
@mgaido91
Copy link
Contributor Author

mgaido91 commented Aug 3, 2019

thank you all for your great help on this!

cloud-fan pushed a commit that referenced this pull request Sep 18, 2019
…here is a corresponding function in Math

### What changes were proposed in this pull request?

1. After #21599, if the option "spark.sql.failOnIntegralTypeOverflow" is enabled, all the Binary Arithmetic operator will used the exact version function.
However, only `Add`/`Substract`/`Multiply` has a corresponding exact function in java.lang.Math . When the option "spark.sql.failOnIntegralTypeOverflow" is enabled, a runtime exception "BinaryArithmetics must override either exactMathMethod or genCode" is thrown if the other Binary Arithmetic operators are used, such as "Divide", "Remainder".
The exact math method should be called only when there is a corresponding function in `java.lang.Math`
2. Revise the log output of casting to `Int`/`Short`
3. Enable `spark.sql.failOnIntegralTypeOverflow` for pgSQL tests in `SQLQueryTestSuite`.

### Why are the changes needed?

1. Fix the bugs of #21599
2. The test case of pgSQL intends to check the overflow of integer/long type. We should enable `spark.sql.failOnIntegralTypeOverflow`.

### Does this PR introduce any user-facing change?

No

### How was this patch tested?

Unit test.

Closes #25804 from gengliangwang/enableIntegerOverflowInSQLTest.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
Projects
None yet
8 participants