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] TPC-DS-like query 24a and 24b at scale=3TB fails with OOM #1628

Closed
abellina opened this issue Jan 29, 2021 · 4 comments
Closed

[BUG] TPC-DS-like query 24a and 24b at scale=3TB fails with OOM #1628

abellina opened this issue Jan 29, 2021 · 4 comments
Labels
bug Something isn't working cudf_dependency An issue or PR with this label depends on a new feature in cudf

Comments

@abellina
Copy link
Collaborator

I have seen this with and without the RapidsShuffleManager. In this case, the device store I see two tasks wanting to allocate ~1GB each.

This is with 8 executors (each with an A100 with 40GB) and 4 concurrent tasks (4 cores/exec)

21/01/28 22:44:17 INFO DeviceMemoryEventHandler: Device allocation of 1325112360 bytes failed, device store has 0 bytes. Total RMM allocated is 26554743808 bytes.
21/01/28 22:44:17 INFO DeviceMemoryEventHandler: Device allocation of 1013508408 bytes failed, device store has 0 bytes. Total RMM allocated is 26554923008 bytes.
java.lang.OutOfMemoryError: Could not allocate native memory: std::bad_alloc: CUDA error at: /usr/local/rapids/include/rmm/mr/device/cuda_memory_resource.hpp:69: cudaErrorMemoryAllocation out of memory
        at ai.rapids.cudf.Table.innerJoin(Native Method)
        at ai.rapids.cudf.Table.access$3500(Table.java:44)
        at ai.rapids.cudf.Table$TableOperation.innerJoin(Table.java:2233)
        at com.nvidia.spark.rapids.shims.spark300.GpuHashJoin.doJoinLeftRight(GpuHashJoin.scala:307)
        at com.nvidia.spark.rapids.shims.spark300.GpuHashJoin.com$nvidia$spark$rapids$shims$spark300$GpuHashJoin$$doJoin(GpuHashJoin.scala:274)
        at com.nvidia.spark.rapids.shims.spark300.GpuHashJoin$$anon$1.hasNext(GpuHashJoin.scala:223)
        at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:458)
        at com.nvidia.spark.rapids.GpuHashAggregateExec.$anonfun$doExecuteColumnar$1(aggregate.scala:433)
        at org.apache.spark.rdd.RDD.$anonfun$mapPartitions$2(RDD.scala:837)
        at org.apache.spark.rdd.RDD.$anonfun$mapPartitions$2$adapted(RDD.scala:837)
        at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
        at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:349)
        at org.apache.spark.rdd.RDD.iterator(RDD.scala:313)
        at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
        at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:349)
        at org.apache.spark.rdd.RDD.iterator(RDD.scala:313)
        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:127)
        at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:446)
        at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1377)
        at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:449)
        at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
        at java.lang.Thread.run(Thread.java:748)
@abellina abellina added bug Something isn't working ? - Needs Triage Need team to review and classify labels Jan 29, 2021
@abellina abellina changed the title [BUG] TPC-DS-like query 24a at scale=3TB fails with OOM [BUG] TPC-DS-like query 24a and 24b at scale=3TB fails with OOM Jan 29, 2021
@sameerz sameerz removed the ? - Needs Triage Need team to review and classify label Feb 2, 2021
@jlowe
Copy link
Member

jlowe commented Feb 3, 2021

These two queries have the problem of horrific skew on join keys followed by an exploding join. One of the join conditions in both queries is c_birth_country == upper(ca_country). There's only two distinct values for upper(ca_country) which is null and UNITED STATES, leading to bad join key skew. The join also explodes since it mixes in a store zip join with customer zip before this, so just a few tasks end up joining two reasonably-sized tables into a very large result (few million rows becomes billions of rows within the same task).

We will likely need some kind of chunked join output functionality from libcudf to handle this.

@revans2
Copy link
Collaborator

revans2 commented May 3, 2021

I believe that this is likely fixed now that #2310 has been merged in. I was able to run both query 24a and 24b at scale factor 200, but with only 2 shuffle partitions. This should be equivalent to running at scale factor 3000 with 30 partitions. But because this deals with skewed data (specifically upper(ca_country)) it is likely not really equivalent. @jlowe or @abellina can we try and rerun these no and see if they are still failing?

@abellina
Copy link
Collaborator Author

abellina commented May 5, 2021

@revans2 sorry I missed this comment. I ran both 24a and 24b myself at 3TB and they are both passing for me.

I used 200 shuffle partitions (default), and ditto with batchSizeBytes (left it alone -> 2GB)

--conf 'spark.rapids.sql.concurrentGpuTasks=2'
--conf 'spark.rapids.memory.pinnedPool.size=8g' 
--conf 'spark.rapids.memory.host.spillStorageSize=32g'
--conf 'spark.sql.files.maxPartitionBytes=1g'

@jlowe
Copy link
Member

jlowe commented May 5, 2021

Thanks for the update, @abellina! Based on them now passing with defaults, closing this as fixed.

@sameerz sameerz closed this as completed May 18, 2021
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
bug Something isn't working cudf_dependency An issue or PR with this label depends on a new feature in cudf
Projects
None yet
Development

No branches or pull requests

4 participants