[ 
https://issues.apache.org/jira/browse/SPARK-28067?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17039403#comment-17039403
 ] 

Sunitha Kambhampati commented on SPARK-28067:
---------------------------------------------

I looked into this issue and here are some of my notes. 

*Issue:*

Wrong results are returned for aggregate sum with decimals with whole stage 
codegen enabled 

*Repro:* 

Whole Stage codegen enabled -> Wrong results

Whole Stage codegen disabled -> Returns exception Decimal precision 39 exceeds 
max precision 38 

*Issues:* 

1: Wrong results are returned which is bad 

2: Inconsistency between whole stage enabled and disabled. 

 

*Cause:*

Sum does not take care of possibility of decimal overflow for the intermediate 
steps.  ie the updateExpressions and mergeExpressions.  

 

*Some ways to fix this:* 

+Approach 1:+  Do not return wrong results for this scenario, throw exception 
like whole stage enabled.  DB’s do similar, so there is precedence.  

Pros: 

- No wrong results

- Consistent behavior between wholestage enabled and disabled

- DB’s have similar existing behavior, there is precedence

 

+Approach 2:+ 

By default: Return null on overflow in the sum operation

But if you set spark.sql.ansi.enabled to true, and then it will throw 
exception. 

 

Pros:

- Maybe ok for users who can tolerate sum to be null on overflow. 

- Consistent with the spark.sql.ansi.enabled behavior

 

Cons:

- This will still keep inconsistency between codegen enabled and disabled. 

 

For those interested, there are some JIRA’s that were fixed for v3.0 which do 
the following: 
 * SPARK-23179, Throw null on overflow for decimal operations.   This does not 
kick in for sum for the use case above. 
 * SPARK-28224, that took care of decimal overflow for sum only partially for 2 
values.   If you add another row into the dataset, it will return wrong results

 ------

That said, I think both Approach 1 and  Approach 2 will resolve the wrong 
results which is bad.  

 

Approach 1 is straightforward.   But in the pr’s related to overflow, I think 
the preference is to have it under a spark.sql.ansi.enabled flag which defaults 
to false and return null on overflow. 

I think Approach 2 is not as straightforward.  I have an implementation that 
will fix this. 

I can open 2 prs that implement each of the approach, and would like to get 
comments.  I have run the sql, catalyst and hive tests and they all pass.   

Please let me know your comments.   Thanks. 

 cc [~dongjoon], [~LI,Xiao], [~cloud_fan] [~hyukjin.kwon] [~hvanhovell] 
[~javier_ivanov] [~msirek]

> Incorrect results in decimal aggregation with whole-stage code gen enabled
> --------------------------------------------------------------------------
>
>                 Key: SPARK-28067
>                 URL: https://issues.apache.org/jira/browse/SPARK-28067
>             Project: Spark
>          Issue Type: Bug
>          Components: SQL
>    Affects Versions: 2.0.2, 2.1.3, 2.2.3, 2.3.4, 2.4.4, 3.0.0
>            Reporter: Mark Sirek
>            Priority: Critical
>              Labels: correctness
>
> The following test case involving a join followed by a sum aggregation 
> returns the wrong answer for the sum:
>  
> {code:java}
> val df = Seq(
>  (BigDecimal("10000000000000000000"), 1),
>  (BigDecimal("10000000000000000000"), 1),
>  (BigDecimal("10000000000000000000"), 2),
>  (BigDecimal("10000000000000000000"), 2),
>  (BigDecimal("10000000000000000000"), 2),
>  (BigDecimal("10000000000000000000"), 2),
>  (BigDecimal("10000000000000000000"), 2),
>  (BigDecimal("10000000000000000000"), 2),
>  (BigDecimal("10000000000000000000"), 2),
>  (BigDecimal("10000000000000000000"), 2),
>  (BigDecimal("10000000000000000000"), 2),
>  (BigDecimal("10000000000000000000"), 2)).toDF("decNum", "intNum")
> val df2 = df.withColumnRenamed("decNum", "decNum2").join(df, 
> "intNum").agg(sum("decNum"))
> scala> df2.show(40,false)
>  ---------------------------------------
> sum(decNum)
> ---------------------------------------
> 40000000000000000000.000000000000000000
> ---------------------------------------
>  
> {code}
>  
> The result should be 1040000000000000000000.0000000000000000.
> It appears a partial sum is computed for each join key, as the result 
> returned would be the answer for all rows matching intNum === 1.
> If only the rows with intNum === 2 are included, the answer given is null:
>  
> {code:java}
> scala> val df3 = df.filter($"intNum" === lit(2))
>  df3: org.apache.spark.sql.Dataset[org.apache.spark.sql.Row] = [decNum: 
> decimal(38,18), intNum: int]
> scala> val df4 = df3.withColumnRenamed("decNum", "decNum2").join(df3, 
> "intNum").agg(sum("decNum"))
>  df4: org.apache.spark.sql.DataFrame = [sum(decNum): decimal(38,18)]
> scala> df4.show(40,false)
>  -----------
> sum(decNum)
> -----------
> null
> -----------
>  
> {code}
>  
> The correct answer, 1000000000000000000000.0000000000000000, doesn't fit in 
> the DataType picked for the result, decimal(38,18), so an overflow occurs, 
> which Spark then converts to null.
> The first example, which doesn't filter out the intNum === 1 values should 
> also return null, indicating overflow, but it doesn't.  This may mislead the 
> user to think a valid sum was computed.
> If whole-stage code gen is turned off:
> spark.conf.set("spark.sql.codegen.wholeStage", false)
> ... incorrect results are not returned because the overflow is caught as an 
> exception:
> java.lang.IllegalArgumentException: requirement failed: Decimal precision 39 
> exceeds max precision 38
>  
>  
>  
>  
>  
>  
>  



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org

Reply via email to