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

[BUG] java.lang.ArithmeticException: divide by zero when spark.sql.ansi.enabled=true #2078

Closed
viadea opened this issue Apr 4, 2021 · 7 comments · Fixed by #2130
Closed
Assignees
Labels
bug Something isn't working P0 Must have for release

Comments

@viadea
Copy link
Collaborator

viadea commented Apr 4, 2021

Describe the bug
A clear and concise description of what the bug is.

When spark.sql.ansi.enabled=true , NDS query Q7 will fail with below stacktrace:

Caused by: java.lang.ArithmeticException: divide by zero
  at org.apache.spark.sql.rapids.GpuDivModLike.divByZeroError(arithmetic.scala:189)
  at org.apache.spark.sql.rapids.GpuDivModLike.$anonfun$doColumnar$3(arithmetic.scala:196)
  at com.nvidia.spark.rapids.Arm.withResource(Arm.scala:28)
  at com.nvidia.spark.rapids.Arm.withResource$(Arm.scala:26)
  at com.nvidia.spark.rapids.CudfBinaryOperator.withResource(GpuExpressions.scala:261)
  at org.apache.spark.sql.rapids.GpuDivModLike.doColumnar(arithmetic.scala:194)
  at org.apache.spark.sql.rapids.GpuDivModLike.doColumnar$(arithmetic.scala:192)
  at org.apache.spark.sql.rapids.GpuDivide.doColumnar(arithmetic.scala:243)
  at com.nvidia.spark.rapids.GpuBinaryExpression.columnarEval(GpuExpressions.scala:167)
  at com.nvidia.spark.rapids.GpuBinaryExpression.columnarEval$(GpuExpressions.scala:158)
  at com.nvidia.spark.rapids.CudfBinaryOperator.columnarEval(GpuExpressions.scala:261)
  at com.nvidia.spark.rapids.GpuHashAggregateExec.$anonfun$doExecuteColumnar$5(aggregate.scala:502)
  at scala.collection.TraversableLike.$anonfun$map$1(TraversableLike.scala:238)
  at scala.collection.mutable.ResizableArray.foreach(ResizableArray.scala:62)
  at scala.collection.mutable.ResizableArray.foreach$(ResizableArray.scala:55)
  at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:49)
  at scala.collection.TraversableLike.map(TraversableLike.scala:238)
  at scala.collection.TraversableLike.map$(TraversableLike.scala:231)
  at scala.collection.AbstractTraversable.map(Traversable.scala:108)
  at com.nvidia.spark.rapids.GpuHashAggregateExec.$anonfun$doExecuteColumnar$1(aggregate.scala:498)
  at org.apache.spark.rdd.RDD.$anonfun$mapPartitions$2(RDD.scala:863)
  at org.apache.spark.rdd.RDD.$anonfun$mapPartitions$2$adapted(RDD.scala:863)
  at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
  at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:373)
  at org.apache.spark.rdd.RDD.iterator(RDD.scala:337)
  at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
  at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:373)
  at org.apache.spark.rdd.RDD.iterator(RDD.scala:337)
  at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
  at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:373)
  at org.apache.spark.rdd.RDD.iterator(RDD.scala:337)
  at org.apache.spark.shuffle.ShuffleWriteProcessor.write(ShuffleWriteProcessor.scala:59)
  at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:99)
  at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:52)
  at org.apache.spark.scheduler.Task.run(Task.scala:131)
  at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:497)
  at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1439)
  at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:500)
  at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
  at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
  at java.base/java.lang.Thread.run(Thread.java:834)

Steps/Code to reproduce bug
Please provide a list of steps or a code sample to reproduce the issue.
Avoid posting private or sensitive data.

Turn on spark.sql.ansi.enabled=true and run NDS Q7.

Expected behavior
A clear and concise description of what you expected to happen.

NDS Q7 should run fine when spark.sql.ansi.enabled=true in GPU Mode.

Environment details (please complete the following information)

  • Environment location: [Standalone, YARN, Kubernetes, Cloud(specify cloud provider)]
  • Spark configuration settings related to the issue

Standalone Spark 3.1.1 single-node cluster.
rapids accelerator 0.4.1 GA release.

Additional context
Add any other context about the problem here.

@viadea viadea added bug Something isn't working ? - Needs Triage Need team to review and classify labels Apr 4, 2021
@andygrove andygrove self-assigned this Apr 5, 2021
@andygrove andygrove added this to the Mar 30 - Apr 9 milestone Apr 5, 2021
@gerashegalov
Copy link
Collaborator

@viadea can you please add the Spark version to the PR description?

Presumably it's Spark 3.1.1. It's related to this audit issue #1464

@sameerz sameerz added P0 Must have for release and removed ? - Needs Triage Need team to review and classify labels Apr 6, 2021
@viadea
Copy link
Collaborator Author

viadea commented Apr 6, 2021

@viadea can you please add the Spark version to the PR description?

Presumably it's Spark 3.1.1. It's related to this audit issue #1464

Yes Spark 3.1.1 with Rapids accelerator 0.4.1

@andygrove
Copy link
Contributor

@viadea I could not reproduce this locally with SF=100 data set. Which data set were you running against?

@andygrove
Copy link
Contributor

It is possible that due to rounding differences that we get a zero on GPU but not on CPU which could explain the difference here. I can explore more once I know which data set to test with. It would also be good to know whether decimal support was enabled or not.

@viadea
Copy link
Collaborator Author

viadea commented Apr 9, 2021

@andygrove i will try to see if i can find a minimum repro for this one. Once I found it i will update here.

@viadea
Copy link
Collaborator Author

viadea commented Apr 9, 2021

@andygrove I narrowed down Q7 to a minimum reproduce.
Basically I tried to materialize the data like below:

create table testbug_parquet STORED AS PARQUET as 
 SELECT ss_quantity,i_item_id
 FROM store_sales, customer_demographics, date_dim, item, promotion
 WHERE ss_sold_date_sk = d_date_sk AND
       ss_item_sk = i_item_sk AND
       ss_cdemo_sk = cd_demo_sk AND
       ss_promo_sk = p_promo_sk AND
       cd_gender = 'M' AND
       cd_marital_status = 'S' AND
       cd_education_status = 'College' AND
       (p_channel_email = 'N' or p_channel_event = 'N') AND
       d_year = 2000;

And then the result table testbug_parquet only has 2 columns: 1 string + 1 int:

spark-sql> desc testbug_parquet
         > ;
ss_quantity	int	NULL
i_item_id	string	NULL
Time taken: 0.361 seconds, Fetched 2 row(s)

Then I can reproduce the issue using below SQL:

set spark.sql.ansi.enabled=true;
select avg(ss_quantity) from testbug_parquet
GROUP BY i_item_id;

Or spark-shell version:

spark.conf.set("spark.sql.ansi.enabled",true)
spark.read.parquet("/xxx/testbug_parquet").createOrReplaceTempView("df")
spark.sql("select avg(ss_quantity) from df GROUP BY i_item_id").show

After removing set spark.sql.ansi.enabled=true;, then it works fine in GPU mode.
I can transfer the final parquet dataset for testbug table which is 3MB(testbug.zip).

@andygrove
Copy link
Contributor

Thanks for the repro case @viadea. I have been debugging this with @abellina and we discovered that the issue is that the GpuAverage operation when called from GpuHashAggregateExec will sometimes divide by zero and expects non-ansi behavior where null is returned rather than getting an exception.

Spark Average passes a failOnError=false flag to Divide to disable ansi mode, and we are missing that on the GPU implementation.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
bug Something isn't working P0 Must have for release
Projects
None yet
Development

Successfully merging a pull request may close this issue.

5 participants