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]OOM happened when do cube operations #3226

Closed
nvliyuan opened this issue Aug 13, 2021 · 7 comments
Closed

[BUG]OOM happened when do cube operations #3226

nvliyuan opened this issue Aug 13, 2021 · 7 comments
Assignees
Labels
bug Something isn't working P0 Must have for release

Comments

@nvliyuan
Copy link
Collaborator

nvliyuan commented Aug 13, 2021

Describe the bug
There is an OOM error when cube too many columns.

Steps/Code to reproduce bug
#load TPCDS 3000 data as a dataframe
val df = spark.read.parquet("hdfs:///XXXX/useDecimal=false,useDate=true,filterNull=false/web_returns")
#when cube 6 columns, it works well
val result_cube6=df.cube($"wr_returned_date_sk",$"wr_refunded_customer_sk",$"wr_refunded_cdemo_sk",$"wr_refunded_hdemo_sk",$"wr_refunded_addr_sk",$"wr_returning_customer_sk").count().show(10)
#but when cube 7 columns, OOM happened
val result_cube7=df.cube($"wr_returned_date_sk",$"wr_refunded_customer_sk",$"wr_refunded_cdemo_sk",$"wr_refunded_hdemo_sk",$"wr_refunded_addr_sk",$"wr_returning_customer_sk",$"wr_web_page_sk").count().show()

here is the config params:
NUM_EXECUTOR_CORES=16
RESOURCE_GPU_AMT=0.0625
EXECUTOR_MEMORY=240
DRIVER_MEMORY=50
USEAQE=true

--driver-memory ${DRIVER_MEMORY}G
--executor-cores $NUM_EXECUTOR_CORES
--executor-memory ${EXECUTOR_MEMORY}G
--conf spark.executor.resource.gpu.amount=1
--conf spark.rapids.sql.concurrentGpuTasks=2
--conf spark.locality.wait=0
--conf spark.rapids.memory.pinnedPool.size=8g
--conf spark.sql.files.maxPartitionBytes=1g
--conf spark.dynamicAllocation.enabled=false
--conf spark.task.resource.gpu.amount=$RESOURCE_GPU_AMT
--conf spark.rapids.sql.enabled=True
--conf spark.plugins=com.nvidia.spark.SQLPlugin
--conf spark.rapids.sql.variableFloatAgg.enabled=true
--conf spark.sql.adaptive.enabled=$USEAQE
--conf spark.rapids.sql.explain=NOT_ON_GPU
--conf spark.shuffle.manager=com.nvidia.spark.rapids.spark311.RapidsShuffleManager
--conf spark.shuffle.service.enabled=false
--conf spark.executorEnv.UCX_TLS=cuda_copy,rc,tcp,sm
--conf spark.executorEnv.UCX_ERROR_SIGNALS=
--conf spark.executorEnv.UCX_RNDV_SCHEME=put_zcopy
--conf spark.executorEnv.UCX_MAX_RNDV_RAILS=1
--conf spark.executorEnv.UCX_MEMTYPE_CACHE=n
--conf spark.executorEnv.UCX_IB_RX_QUEUE_LEN=1024
--conf spark.rapids.shuffle.ucx.bounceBuffers.size=4MB
--conf spark.rapids.shuffle.ucx.bounceBuffers.host.count=64
--conf spark.rapids.shuffle.ucx.bounceBuffers.device.count=64
--conf spark.rapids.shuffle.transport.enabled=true
--conf spark.rapids.memory.host.spillStorageSize=32G \

Error:
executor 6): java.lang.OutOfMemoryError: Could not allocate native memory: std::bad_alloc: RMM failure at:/XXX/XXX...../rmm/mr/device/detail/arena.hpp:379: Maximum pool size exceeded
at ai.rapids.cudf.Table.groupByAggregate(Native Method)
at ai.rapids.cudf.Table.access$3000(Table.java:38)
at ai.rapids.cudf.Table$GroupByOperation.aggregate(Table.java:2349)
at com.nvidia.spark.rapids.GpuHashAggregateExec.$anonfun$computeAggregate$7(aggregate.scala:729)
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.GpuHashAggregateExec.withResource(aggregate.scala:240)
at com.nvidia.spark.rapids.GpuHashAggregateExec.$anonfun$computeAggregate$1(aggregate.scala:728)
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.GpuHashAggregateExec.withResource(aggregate.scala:240)

@nvliyuan nvliyuan added ? - Needs Triage Need team to review and classify bug Something isn't working labels Aug 13, 2021
@Salonijain27 Salonijain27 added P0 Must have for release and removed ? - Needs Triage Need team to review and classify labels Aug 17, 2021
@nvliyuan
Copy link
Collaborator Author

executors num:8
GPU: A100

@abellina
Copy link
Collaborator

abellina commented Aug 18, 2021

Thanks @nvliyuan, I believe/hope this is another instance of: #2504.

I'll try to repro this issue with/without a fix for the issue linked to see if we can handle it the same way.

@abellina
Copy link
Collaborator

The issue seems to fail even with a fix for fragmentation where every block is returned to the global arena. I tried with the same GPUs and same number of them:

Executor task launch worker for task 124.0 in stage 5.0 (TID 2441) 21/08/18 17:22:59:692 WARN DeviceMemoryEventHandler: Device store exhausted, unable to allocate 2131623104 bytes. Total RMM allocated is 13514738432 bytes.
Executor task launch worker for task 4.0 in stage 5.0 (TID 2321) 21/08/18 17:22:59:693 WARN DeviceMemoryEventHandler: Device store exhausted, unable to allocate 2083201792 bytes. Total RMM allocated is 13514738432 bytes.

I'll have to try with the extra debugging that @rongou is adding, will get back to you.

@abellina
Copy link
Collaborator

abellina commented Aug 27, 2021

In this scenario, I have tested with arena and RMM and I don't see leaks. Arena is also getting improved to manage things better in the OOM scenario, which improves this case, but doesn't entirely fix it. What is happening in this case is real fragmentation, while we have outstanding allocations (not closed, and not leaked). This is a natural fragmentation of the pool, and the only allocator I know of that would address is the CUDA Async Allocator, as it can defrag its memory when it fails to service an allocation, or the request of more GPU resources.

So at this point, I don't think this query can be performed as is, with the resources requested, it just doesn't fit. I have a PoC that includes the CUDA Async Allocator, and I'll test that next to see if it helps with this query.

@abellina
Copy link
Collaborator

abellina commented Aug 27, 2021

With CUDA Async Allocator, this query is succeeding consistently, meaning that it is able to defrag itself. Note I am not even System.GC'ing between runs, this allocator seems promising, I propose we spin this issue as a new FEA ticket to add support and test this allocator.

@Salonijain27 Salonijain27 added this to the Sep 27 - Oct 1 milestone Sep 24, 2021
@sameerz sameerz modified the milestones: Sep 27 - Oct 1, Oct 4 - Oct 15 Oct 8, 2021
@Salonijain27 Salonijain27 removed this from the Nov 1 - Nov 12 milestone Nov 15, 2021
@rongou
Copy link
Collaborator

rongou commented Dec 13, 2021

I was able to reproduce this with rapidsai/rmm#916. This still seems like a fragmentation issue. Here is the memory dump right before the oom error:

[2021-12-13 11:45:32.702] [arena_memory_dump] [info] **************************************************
[2021-12-13 11:45:32.702] [arena_memory_dump] [info] Ran out of memory trying to allocate 129.531 MiB.
[2021-12-13 11:45:32.702] [arena_memory_dump] [info] **************************************************
[2021-12-13 11:45:32.702] [arena_memory_dump] [info] Global arena:
[2021-12-13 11:45:32.702] [arena_memory_dump] [info]   Arena size: 22.7189 GiB
[2021-12-13 11:45:32.702] [arena_memory_dump] [info]   # superblocks: 230
[2021-12-13 11:45:32.702] [arena_memory_dump] [info]   Total free memory: 1.70405 GiB
[2021-12-13 11:45:32.702] [arena_memory_dump] [info]   Largest block of free memory: 106.298 MiB
[2021-12-13 11:45:32.702] [arena_memory_dump] [info]   Fragmentation: 93.91%

@rongou
Copy link
Collaborator

rongou commented Jan 12, 2022

Switching to async allocator #4515

@rongou rongou closed this as completed Jan 12, 2022
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

No branches or pull requests

5 participants