[jira] [Commented] (SPARK-28067) Incorrect results in decimal aggregation with whole-stage code gen enabled

2020-10-07 Thread Dongjoon Hyun (Jira)


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

Dongjoon Hyun commented on SPARK-28067:
---

[~anuragmantri] For this one, this is not backported to 3.0.0, too.

> 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
>Assignee: Sunitha Kambhampati
>Priority: Critical
>  Labels: correctness
> Fix For: 3.1.0
>
>
> 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("1000"), 1),
>  (BigDecimal("1000"), 1),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2)).toDF("decNum", "intNum")
> val df2 = df.withColumnRenamed("decNum", "decNum2").join(df, 
> "intNum").agg(sum("decNum"))
> scala> df2.show(40,false)
>  ---
> sum(decNum)
> ---
> 4000.00
> ---
>  
> {code}
>  
> The result should be 104000..
> 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, 10., 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



[jira] [Commented] (SPARK-28067) Incorrect results in decimal aggregation with whole-stage code gen enabled

2020-10-07 Thread Anurag Mantripragada (Jira)


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

Anurag Mantripragada commented on SPARK-28067:
--

I just checked the issue exists in branch-2.4. Since this is a `correctness` 
issue, should we backport it to branch-2.4? 
cc: [~cloud_fan], [~dongjoon]

> 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
>Assignee: Sunitha Kambhampati
>Priority: Critical
>  Labels: correctness
> Fix For: 3.1.0
>
>
> 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("1000"), 1),
>  (BigDecimal("1000"), 1),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2)).toDF("decNum", "intNum")
> val df2 = df.withColumnRenamed("decNum", "decNum2").join(df, 
> "intNum").agg(sum("decNum"))
> scala> df2.show(40,false)
>  ---
> sum(decNum)
> ---
> 4000.00
> ---
>  
> {code}
>  
> The result should be 104000..
> 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, 10., 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



[jira] [Commented] (SPARK-28067) Incorrect results in decimal aggregation with whole-stage code gen enabled

2020-07-07 Thread Apache Spark (Jira)


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

Apache Spark commented on SPARK-28067:
--

User 'cloud-fan' has created a pull request for this issue:
https://github.com/apache/spark/pull/29026

> 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
>Assignee: Sunitha Kambhampati
>Priority: Critical
>  Labels: correctness
> Fix For: 3.1.0
>
>
> 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("1000"), 1),
>  (BigDecimal("1000"), 1),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2)).toDF("decNum", "intNum")
> val df2 = df.withColumnRenamed("decNum", "decNum2").join(df, 
> "intNum").agg(sum("decNum"))
> scala> df2.show(40,false)
>  ---
> sum(decNum)
> ---
> 4000.00
> ---
>  
> {code}
>  
> The result should be 104000..
> 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, 10., 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



[jira] [Commented] (SPARK-28067) Incorrect results in decimal aggregation with whole-stage code gen enabled

2020-02-18 Thread Sunitha Kambhampati (Jira)


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

Sunitha Kambhampati commented on SPARK-28067:
-

I have submitted the two pr's for the two approaches I mention above in my 
comment. 

Approach 1:  Throw exception instead of returning wrong results: 
[https://github.com/apache/spark/pull/27629]

 

Approach 2:

Return null on decimal overflow for aggregate sum when 
spark.sql.ansi.enabled=false,  

Throw exception on decimal overflow for aggregate sum when 
spark.sql.ansi.enabled =true.

[https://github.com/apache/spark/pull/27627] (WIP)

> 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("1000"), 1),
>  (BigDecimal("1000"), 1),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2)).toDF("decNum", "intNum")
> val df2 = df.withColumnRenamed("decNum", "decNum2").join(df, 
> "intNum").agg(sum("decNum"))
> scala> df2.show(40,false)
>  ---
> sum(decNum)
> ---
> 4000.00
> ---
>  
> {code}
>  
> The result should be 104000..
> 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, 10., 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



[jira] [Commented] (SPARK-28067) Incorrect results in decimal aggregation with whole-stage code gen enabled

2020-02-18 Thread Sunitha Kambhampati (Jira)


[ 
https://issues.apache.org/jira/browse/SPARK-28067?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=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("1000"), 1),
>  (BigDecimal("1000"), 1),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2)).toDF("decNum", "intNum")
> val df2 = df.withColumnRenamed("decNum", "decNum2").join(df, 
> "intNum").agg(sum("decNum"))
> scala> df2.show(40,false)
>  ---
> sum(decNum)
> ---
> 4000.00
> ---
>  
> {code}
>  
> The result should be 104000..
> 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, 10., doesn't fit in 

[jira] [Commented] (SPARK-28067) Incorrect results in decimal aggregation with whole-stage code gen enabled

2020-02-12 Thread Javier Fuentes (Jira)


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

Javier Fuentes commented on SPARK-28067:


[~dongjoon], [~msirek]  what should be the expected result here in both cases a 
null or ArithmeticException instead?

> 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("1000"), 1),
>  (BigDecimal("1000"), 1),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2)).toDF("decNum", "intNum")
> val df2 = df.withColumnRenamed("decNum", "decNum2").join(df, 
> "intNum").agg(sum("decNum"))
> scala> df2.show(40,false)
>  ---
> sum(decNum)
> ---
> 4000.00
> ---
>  
> {code}
>  
> The result should be 104000..
> 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, 10., 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



[jira] [Commented] (SPARK-28067) Incorrect results in decimal aggregation with whole-stage code gen enabled

2020-02-09 Thread Hyukjin Kwon (Jira)


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

Hyukjin Kwon commented on SPARK-28067:
--

I am lowering the priority given that {{spark.sql.codegen.wholeStage}} defaults 
to {{true}}.

> 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("1000"), 1),
>  (BigDecimal("1000"), 1),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2)).toDF("decNum", "intNum")
> val df2 = df.withColumnRenamed("decNum", "decNum2").join(df, 
> "intNum").agg(sum("decNum"))
> scala> df2.show(40,false)
>  ---
> sum(decNum)
> ---
> 4000.00
> ---
>  
> {code}
>  
> The result should be 104000..
> 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, 10., 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



[jira] [Commented] (SPARK-28067) Incorrect results in decimal aggregation with whole-stage code gen enabled

2020-01-28 Thread Javier Fuentes (Jira)


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

Javier Fuentes commented on SPARK-28067:


The expected result here should be an ArithmeticException for all cases of 
precision > DecimalType.MAX_PRECISION?

> 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: Blocker
>  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("1000"), 1),
>  (BigDecimal("1000"), 1),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2)).toDF("decNum", "intNum")
> val df2 = df.withColumnRenamed("decNum", "decNum2").join(df, 
> "intNum").agg(sum("decNum"))
> scala> df2.show(40,false)
>  ---
> sum(decNum)
> ---
> 4000.00
> ---
>  
> {code}
>  
> The result should be 104000..
> 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, 10., 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



[jira] [Commented] (SPARK-28067) Incorrect results in decimal aggregation with whole-stage code gen enabled

2020-01-22 Thread Dongjoon Hyun (Jira)


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

Dongjoon Hyun commented on SPARK-28067:
---

I reproduced this issue at 2.2.3, 2.3.4 and 2.4.4. As the description says, it 
give different results based on `spark.sql.codegen.wholeStage` value.
{code}
scala> :paste
// Entering paste mode (ctrl-D to finish)

val df = Seq(
 (BigDecimal("1000"), 1),
 (BigDecimal("1000"), 1),
 (BigDecimal("1000"), 2),
 (BigDecimal("1000"), 2),
 (BigDecimal("1000"), 2),
 (BigDecimal("1000"), 2),
 (BigDecimal("1000"), 2),
 (BigDecimal("1000"), 2),
 (BigDecimal("1000"), 2),
 (BigDecimal("1000"), 2),
 (BigDecimal("1000"), 2),
 (BigDecimal("1000"), 2)).toDF("decNum", "intNum")
val df2 = df.withColumnRenamed("decNum", "decNum2").join(df, 
"intNum").agg(sum("decNum"))
df2.show(40,false)

// Exiting paste mode, now interpreting.

+---+
|sum(decNum)|
+---+
|4000.00|
+---+

df: org.apache.spark.sql.DataFrame = [decNum: decimal(38,18), intNum: int]
df2: org.apache.spark.sql.DataFrame = [sum(decNum): decimal(38,18)]

scala> spark.conf.set("spark.sql.codegen.wholeStage", false)

scala> df2.show(40,false)
20/01/22 20:18:10 ERROR Executor: Exception in task 1.0 in stage 4.0 (TID 20)
java.lang.IllegalArgumentException: requirement failed: Decimal precision 39 
exceeds max precision 38
{code}

> 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: Spark Core
>Affects Versions: 2.3.0, 2.4.0, 2.4.4
> Environment: Ubuntu LTS 16.04
> Oracle Java 1.8.0_201
> spark-2.4.3-bin-without-hadoop
> spark-shell
>Reporter: Mark Sirek
>Priority: Minor
>  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("1000"), 1),
>  (BigDecimal("1000"), 1),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2)).toDF("decNum", "intNum")
> val df2 = df.withColumnRenamed("decNum", "decNum2").join(df, 
> "intNum").agg(sum("decNum"))
> scala> df2.show(40,false)
>  ---
> sum(decNum)
> ---
> 4000.00
> ---
>  
> {code}
>  
> The result should be 104000..
> 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, 10., 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)


[jira] [Commented] (SPARK-28067) Incorrect results in decimal aggregation with whole-stage code gen enabled

2019-07-08 Thread Marco Gaido (JIRA)


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

Marco Gaido commented on SPARK-28067:
-

I cannot reproduce in 2.4.0 either:

{code}

spark-2.4.0-bin-hadoop2.7 xxx$ ./bin/spark-shell
2019-07-08 22:52:11 WARN  NativeCodeLoader:62 - Unable to load native-hadoop 
library for your platform... using builtin-java classes where applicable
Setting default log level to "WARN".
To adjust logging level use sc.setLogLevel(newLevel). For SparkR, use 
setLogLevel(newLevel).
Spark context Web UI available at http://xxx:4040
Spark context available as 'sc' (master = local[*], app id = 
local-1562619141279).
Spark session available as 'spark'.
Welcome to
    __
 / __/__  ___ _/ /__
_\ \/ _ \/ _ `/ __/  '_/
   /___/ .__/\_,_/_/ /_/\_\   version 2.4.0
  /_/
 
Using Scala version 2.11.12 (Java HotSpot(TM) 64-Bit Server VM, Java 1.8.0_152)
Type in expressions to have them evaluated.
Type :help for more information.

scala> val df = Seq(
 |  (BigDecimal("1000"), 1),
 |  (BigDecimal("1000"), 1),
 |  (BigDecimal("1000"), 2),
 |  (BigDecimal("1000"), 2),
 |  (BigDecimal("1000"), 2),
 |  (BigDecimal("1000"), 2),
 |  (BigDecimal("1000"), 2),
 |  (BigDecimal("1000"), 2),
 |  (BigDecimal("1000"), 2),
 |  (BigDecimal("1000"), 2),
 |  (BigDecimal("1000"), 2),
 |  (BigDecimal("1000"), 2)).toDF("decNum", "intNum")
df: org.apache.spark.sql.DataFrame = [decNum: decimal(38,18), intNum: int]

scala> val df2 = df.withColumnRenamed("decNum", "decNum2").join(df, 
"intNum").agg(sum("decNum"))
df2: org.apache.spark.sql.DataFrame = [sum(decNum): decimal(38,18)]

scala> df2.show(40,false)
+---+
|sum(decNum)|
+---+
|null   |
+---+
{code}

> 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: Spark Core
>Affects Versions: 2.3.0, 2.4.0
> Environment: Ubuntu LTS 16.04
> Oracle Java 1.8.0_201
> spark-2.4.3-bin-without-hadoop
> spark-shell
>Reporter: Mark Sirek
>Priority: Minor
>  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("1000"), 1),
>  (BigDecimal("1000"), 1),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2)).toDF("decNum", "intNum")
> val df2 = df.withColumnRenamed("decNum", "decNum2").join(df, 
> "intNum").agg(sum("decNum"))
> scala> df2.show(40,false)
>  ---
> sum(decNum)
> ---
> 4000.00
> ---
>  
> {code}
>  
> The result should be 104000..
> 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, 10., 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 

[jira] [Commented] (SPARK-28067) Incorrect results in decimal aggregation with whole-stage code gen enabled

2019-07-06 Thread Mark Sirek (JIRA)


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

Mark Sirek commented on SPARK-28067:


I tried the test on 4 different systems, all immediately after downloading 
Spark, and changing no settings, so they should all be the defaults.  None of 
the tests return null.  I'm not sure which config settings I should change.  I 
wouldn't think it's expected behavior for Spark to return an incorrect answer 
with a certain config setting, unless there's a setting which controls the 
hiding or overflows.

> 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: Spark Core
>Affects Versions: 2.3.0, 2.4.0
> Environment: Ubuntu LTS 16.04
> Oracle Java 1.8.0_201
> spark-2.4.3-bin-without-hadoop
> spark-shell
>Reporter: Mark Sirek
>Priority: Minor
>  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("1000"), 1),
>  (BigDecimal("1000"), 1),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2)).toDF("decNum", "intNum")
> val df2 = df.withColumnRenamed("decNum", "decNum2").join(df, 
> "intNum").agg(sum("decNum"))
> scala> df2.show(40,false)
>  ---
> sum(decNum)
> ---
> 4000.00
> ---
>  
> {code}
>  
> The result should be 104000..
> 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, 10., 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
(v7.6.3#76005)

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



[jira] [Commented] (SPARK-28067) Incorrect results in decimal aggregation with whole-stage code gen enabled

2019-06-24 Thread Marco Gaido (JIRA)


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

Marco Gaido commented on SPARK-28067:
-

No, it is the same. Are you sure about your configs?

{code}
macmarco:spark mark9$ git log -5 --oneline
5ad1053f3e (HEAD, apache/master) [SPARK-28128][PYTHON][SQL] Pandas Grouped UDFs 
skip empty partitions
113f8c8d13 [SPARK-28132][PYTHON] Update document type conversion for Pandas 
UDFs (pyarrow 0.13.0, pandas 0.24.2, Python 3.7)
9b9d81b821 [SPARK-28131][PYTHON] Update document type conversion between Python 
data and SQL types in normal UDFs (Python 3.7)
54da3bbfb2 [SPARK-28127][SQL] Micro optimization on TreeNode's mapChildren 
method
47f54b1ec7 [SPARK-28118][CORE] Add `spark.eventLog.compression.codec` 
configuration
macmarco:spark mark9$ ./bin/spark-shell 
19/06/24 09:17:58 WARN NativeCodeLoader: Unable to load native-hadoop library 
for your platform... using builtin-java classes where applicable
Using Spark's default log4j profile: org/apache/spark/log4j-defaults.properties
Setting default log level to "WARN".
To adjust logging level use sc.setLogLevel(newLevel). For SparkR, use 
setLogLevel(newLevel).
Spark context Web UI available at http://.:4040
Spark context available as 'sc' (master = local[*], app id = 
local-1561360686725).
Spark session available as 'spark'.
Welcome to
    __
 / __/__  ___ _/ /__
_\ \/ _ \/ _ `/ __/  '_/
   /___/ .__/\_,_/_/ /_/\_\   version 3.0.0-SNAPSHOT
  /_/
 
Using Scala version 2.12.8 (Java HotSpot(TM) 64-Bit Server VM, Java 1.8.0_152)
Type in expressions to have them evaluated.
Type :help for more information.

scala> val df = Seq(
 |  (BigDecimal("1000"), 1),
 |  (BigDecimal("1000"), 1),
 |  (BigDecimal("1000"), 2),
 |  (BigDecimal("1000"), 2),
 |  (BigDecimal("1000"), 2),
 |  (BigDecimal("1000"), 2),
 |  (BigDecimal("1000"), 2),
 |  (BigDecimal("1000"), 2),
 |  (BigDecimal("1000"), 2),
 |  (BigDecimal("1000"), 2),
 |  (BigDecimal("1000"), 2),
 |  (BigDecimal("1000"), 2)).toDF("decNum", "intNum")
df: org.apache.spark.sql.DataFrame = [decNum: decimal(38,18), intNum: int]

scala> val df2 = df.withColumnRenamed("decNum", "decNum2").join(df, 
"intNum").agg(sum("decNum"))
df2: org.apache.spark.sql.DataFrame = [sum(decNum): decimal(38,18)]

scala> df2.explain
== Physical Plan ==
*(2) HashAggregate(keys=[], functions=[sum(decNum#14)])
+- Exchange SinglePartition
   +- *(1) HashAggregate(keys=[], functions=[partial_sum(decNum#14)])
  +- *(1) Project [decNum#14]
 +- *(1) BroadcastHashJoin [intNum#8], [intNum#15], Inner, BuildLeft
:- BroadcastExchange HashedRelationBroadcastMode(List(cast(input[0, 
int, false] as bigint)))
:  +- LocalTableScan [intNum#8]
+- LocalTableScan [decNum#14, intNum#15]



scala> df2.show(40,false)
+---+   
|sum(decNum)|
+---+
|null   |
+---+
{code}

> 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: Spark Core
>Affects Versions: 2.3.0, 2.4.0
> Environment: Ubuntu LTS 16.04
> Oracle Java 1.8.0_201
> spark-2.4.3-bin-without-hadoop
> spark-shell
>Reporter: Mark Sirek
>Priority: Minor
>  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("1000"), 1),
>  (BigDecimal("1000"), 1),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2)).toDF("decNum", "intNum")
> val df2 = df.withColumnRenamed("decNum", "decNum2").join(df, 
> "intNum").agg(sum("decNum"))
> scala> df2.show(40,false)
>  ---
> sum(decNum)
> ---
> 4000.00
> ---
>  
> {code}
>  
> The result should be 

[jira] [Commented] (SPARK-28067) Incorrect results in decimal aggregation with whole-stage code gen enabled

2019-06-22 Thread Mark Sirek (JIRA)


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

Mark Sirek commented on SPARK-28067:


[~mgaido] Here is the physical plan I'm getting.  Maybe yours is different?  I 
tried on master this time...

 

 
{code:java}
msirek@skylake16:~/IdeaProjects/spark$ git status
On branch master
Your branch is up to date with 'origin/master'.
nothing to commit, working tree clean
msirek@skylake16:~/IdeaProjects/spark$ git log -5 
--pretty=format:"%h%x09%an%x09%ad%x09%s"
870f972dcc Yuming Wang Sat Jun 22 09:15:07 2019 -0700 [SPARK-28104][SQL] 
Implement Spark's own GetColumnsOperation
5ad1053f3e Bryan Cutler Sat Jun 22 11:20:35 2019 +0900 
[SPARK-28128][PYTHON][SQL] Pandas Grouped UDFs skip empty partitions
113f8c8d13 HyukjinKwon Fri Jun 21 10:47:54 2019 -0700 [SPARK-28132][PYTHON] 
Update document type conversion for Pandas UDFs (pyarrow 0.13.0, pandas 0.24.2, 
Python 3.7)
9b9d81b821 HyukjinKwon Fri Jun 21 10:27:18 2019 -0700 [SPARK-28131][PYTHON] 
Update document type conversion between Python data and SQL types in normal 
UDFs (Python 3.7)
54da3bbfb2 Yesheng Ma Thu Jun 20 19:45:59 2019 -0700 [SPARK-28127][SQL] Micro 
optimization on TreeNode's mapChildren method

msirek@skylake16:~/IdeaProjects/spark$ ./bin/spark-shell
SLF4J: Class path contains multiple SLF4J bindings.
SLF4J: Found binding in 
[jar:file:/home/msirek/IdeaProjects/spark/assembly/target/scala-2.12/jars/slf4j-log4j12-1.7.16.jar!/org/slf4j/impl/StaticLoggerBinder.class]
SLF4J: Found binding in 
[jar:file:/opt/cloudera/parcels/CDH-5.16.1-1.cdh5.16.1.p0.3/jars/slf4j-log4j12-1.7.5.jar!/org/slf4j/impl/StaticLoggerBinder.class]
SLF4J: See http://www.slf4j.org/codes.html#multiple_bindings for an explanation.
SLF4J: Actual binding is of type [org.slf4j.impl.Log4jLoggerFactory]
Setting default log level to "WARN".
To adjust logging level use sc.setLogLevel(newLevel). For SparkR, use 
setLogLevel(newLevel).
19/06/22 22:13:39 WARN util.Utils: Service 'SparkUI' could not bind on port 
4040. Attempting port 4041.
Spark context Web UI available at http://skylake16.home.colo:4041
Spark context available as 'sc' (master = local[*], app id = 
local-1561266819220).
Spark session available as 'spark'.
Welcome to
  __
 / __/__ ___ _/ /__
 _\ \/ _ \/ _ `/ __/ '_/
 /___/ .__/\_,_/_/ /_/\_\ version 3.0.0-SNAPSHOT
 /_/
 
Using Scala version 2.12.8 (Java HotSpot(TM) 64-Bit Server VM, Java 1.8.0_201)
Type in expressions to have them evaluated.
Type :help for more information.
scala> val df = Seq(
 | (BigDecimal("1000"), 1),
 | (BigDecimal("1000"), 1),
 | (BigDecimal("1000"), 2),
 | (BigDecimal("1000"), 2),
 | (BigDecimal("1000"), 2),
 | (BigDecimal("1000"), 2),
 | (BigDecimal("1000"), 2),
 | (BigDecimal("1000"), 2),
 | (BigDecimal("1000"), 2),
 | (BigDecimal("1000"), 2),
 | (BigDecimal("1000"), 2),
 | (BigDecimal("1000"), 2)).toDF("decNum", "intNum")
df: org.apache.spark.sql.DataFrame = [decNum: decimal(38,18), intNum: int]
scala> val df2 = df.withColumnRenamed("decNum", "decNum2").join(df, 
"intNum").agg(sum("decNum"))
df2: org.apache.spark.sql.DataFrame = [sum(decNum): decimal(38,18)]
scala> df2.explain
== Physical Plan ==
*(2) HashAggregate(keys=[], functions=[sum(decNum#14)])
+- Exchange SinglePartition
 +- *(1) HashAggregate(keys=[], functions=[partial_sum(decNum#14)])
 +- *(1) Project [decNum#14]
 +- *(1) BroadcastHashJoin [intNum#8], [intNum#15], Inner, BuildLeft
 :- BroadcastExchange HashedRelationBroadcastMode(List(cast(input[0, int, 
false] as bigint)))
 : +- LocalTableScan [intNum#8]
 +- LocalTableScan [decNum#14, intNum#15]
 
scala> df2.show(40,false)
+---+
|sum(decNum) |
+---+
|4000.00|
+---+
{code}
 

 

> 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: Spark Core
>Affects Versions: 2.3.0, 2.4.0
> Environment: Ubuntu LTS 16.04
> Oracle Java 1.8.0_201
> spark-2.4.3-bin-without-hadoop
> spark-shell
>Reporter: Mark Sirek
>Priority: Minor
>  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("1000"), 1),
>  (BigDecimal("1000"), 1),
>  (BigDecimal("1000"), 2),
>  

[jira] [Commented] (SPARK-28067) Incorrect results in decimal aggregation with whole-stage code gen enabled

2019-06-22 Thread Marco Gaido (JIRA)


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

Marco Gaido commented on SPARK-28067:
-

I cannot reproduce on master. It always returns null with whole stage codegen 
enabled.

> 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: Spark Core
>Affects Versions: 2.3.0, 2.4.0
> Environment: Ubuntu LTS 16.04
> Oracle Java 1.8.0_201
> spark-2.4.3-bin-without-hadoop
> spark-shell
>Reporter: Mark Sirek
>Priority: Minor
>  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("1000"), 1),
>  (BigDecimal("1000"), 1),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2),
>  (BigDecimal("1000"), 2)).toDF("decNum", "intNum")
> val df2 = df.withColumnRenamed("decNum", "decNum2").join(df, 
> "intNum").agg(sum("decNum"))
> scala> df2.show(40,false)
>  ---
> sum(decNum)
> ---
> 4000.00
> ---
>  
> {code}
>  
> The result should be 104000..
> 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, 10., 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
(v7.6.3#76005)

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