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 when running NDS queries with UCX and GDS #2504

Closed
rongou opened this issue May 25, 2021 · 17 comments
Closed

[BUG] OOM when running NDS queries with UCX and GDS #2504

rongou opened this issue May 25, 2021 · 17 comments
Assignees
Labels
bug Something isn't working P0 Must have for release

Comments

@rongou
Copy link
Collaborator

rongou commented May 25, 2021

Describe the bug
When running TPC-DS at scale factor 5000 on a Yarn cluster with 8xA100 40 GB GPUs, seeing out-of-memory errors in libcudf code (see full stacktrace below). Probably caused by native code allocating memory while java/scala is actively spilling.

Steps/Code to reproduce bug
TPC-DS at scale factor 5000, query 14a/b, 16, 24b.

Expected behavior
Should not throw oom errors.

Environment details (please complete the following information)

  • Environment location: YARN
  • Spark configuration settings related to the issue
spark.driver.maxResultSize=2GB
spark.rapids.cudfVersionOverride=true
spark.executor.cores=4
spark.rapids.sql.concurrentGpuTasks=2
spark.executor.resource.gpu.amount=1
spark.task.resource.gpu.amount=0.25
spark.executor.memory=240G
spark.driver.memory=50G
spark.sql.files.maxPartitionBytes=1g
spark.locality.wait=0
spark.shuffle.service.enabled=false
spark.rapids.shuffle.transport.enabled=true
spark.shuffle.manager=com.nvidia.spark.rapids.spark302.RapidsShuffleManager
spark.driver.extraClassPath=$SPARK_RAPIDS_PLUGIN_JAR:$CUDF_JAR:$SPARK_RAPIDS_PLUGIN_INTEGRATION_TEST_JAR
spark.executor.extraClassPath=$SPARK_RAPIDS_PLUGIN_JAR:$CUDF_JAR
spark.executorEnv.UCX_RNDV_SCHEME=put_zcopy
spark.executorEnv.UCX_ERROR_SIGNALS=
spark.executorEnv.UCX_IB_GPU_DIRECT_RDMA=yes
spark.executorEnv.UCX_IB_RX_QUEUE_LEN=1024
spark.rapids.shuffle.ucx.bounceBuffers.size=4MB
spark.rapids.shuffle.ucx.bounceBuffers.device.count=64
spark.rapids.shuffle.ucx.bounceBuffers.host.count=64
spark.rapids.memory.host.spillStorageSize=32G
spark.sql.adaptive.enabled=true
spark.plugins=com.nvidia.spark.SQLPlugin
spark.rapids.memory.pinnedPool.size=8g
spark.rapids.shuffle.maxMetadataSize=1MB
spark.rapids.sql.incompatibleOps.enabled=true
spark.rapids.sql.variableFloatAgg.enabled=true
spark.rapids.sql.hasNans=false
spark.rapids.memory.gpu.direct.storage.spill.enabled=true

Additional context
Full stacktrace:

Executor task launch worker for task 15902 21/05/24 20:34:48:745 ERROR Executor: Exception in task 58.0 in stage 54.0 (TID 15902)
java.lang.OutOfMemoryError: Could not allocate native memory: std::bad_alloc: RMM failure at:/home/jenkins/agent/workspace/jenkins-cudf_nightly-dev-github-307-cuda11/cpp/build/_deps/rmm-src/include/rmm/mr/device
/detail/arena.hpp:382: Maximum pool size exceeded
        at ai.rapids.cudf.ColumnView.ifElseVS(Native Method)
        at ai.rapids.cudf.ColumnView.ifElse(ColumnView.java:450)
        at com.nvidia.spark.rapids.GpuNvl$.$anonfun$apply$2(nullExpressions.scala:37)
        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.GpuNvl$.withResource(nullExpressions.scala:28)
        at com.nvidia.spark.rapids.GpuNvl$.apply(nullExpressions.scala:36)
        at com.nvidia.spark.rapids.GpuCoalesce.$anonfun$columnarEval$1(nullExpressions.scala:59)
        at com.nvidia.spark.rapids.GpuCoalesce.$anonfun$columnarEval$1$adapted(nullExpressions.scala:50)
        at scala.collection.immutable.List.foreach(List.scala:392)
        at com.nvidia.spark.rapids.GpuCoalesce.columnarEval(nullExpressions.scala:50)
        at com.nvidia.spark.rapids.RapidsPluginImplicits$ReallyAGpuExpression.columnarEval(implicits.scala:36)
        at com.nvidia.spark.rapids.GpuExpressionsUtils$.columnarEvalToColumn(GpuExpressions.scala:91)
        at com.nvidia.spark.rapids.GpuProjectExec$.projectSingle(basicPhysicalOperators.scala:70)
        at com.nvidia.spark.rapids.GpuProjectExec$.$anonfun$project$1(basicPhysicalOperators.scala:77)
        at com.nvidia.spark.rapids.RapidsPluginImplicits$MapsSafely.$anonfun$safeMap$1(implicits.scala:162)
        at com.nvidia.spark.rapids.RapidsPluginImplicits$MapsSafely.$anonfun$safeMap$1$adapted(implicits.scala:159)
        at scala.collection.immutable.List.foreach(List.scala:392)
        at com.nvidia.spark.rapids.RapidsPluginImplicits$MapsSafely.safeMap(implicits.scala:159)
        at com.nvidia.spark.rapids.RapidsPluginImplicits$AutoCloseableProducingSeq.safeMap(implicits.scala:194)
        at com.nvidia.spark.rapids.GpuProjectExec$.project(basicPhysicalOperators.scala:77)
        at org.apache.spark.sql.rapids.execution.HashJoinIterator.joinGatherer(GpuHashJoin.scala:330)
        at org.apache.spark.sql.rapids.execution.HashJoinIterator.$anonfun$hasNext$4(GpuHashJoin.scala:352)
        at org.apache.spark.sql.rapids.execution.HashJoinIterator.$anonfun$hasNext$4$adapted(GpuHashJoin.scala:351)
        at com.nvidia.spark.rapids.Arm.withResource(Arm.scala:28)
        at com.nvidia.spark.rapids.Arm.withResource$(Arm.scala:26)
        at org.apache.spark.sql.rapids.execution.HashJoinIterator.withResource(GpuHashJoin.scala:179)
        at org.apache.spark.sql.rapids.execution.HashJoinIterator.$anonfun$hasNext$3(GpuHashJoin.scala:351)
        at org.apache.spark.sql.rapids.execution.HashJoinIterator.$anonfun$hasNext$3$adapted(GpuHashJoin.scala:349)
        at com.nvidia.spark.rapids.Arm.withResource(Arm.scala:28)
        at com.nvidia.spark.rapids.Arm.withResource$(Arm.scala:26)
        at org.apache.spark.sql.rapids.execution.HashJoinIterator.withResource(GpuHashJoin.scala:179)
        at org.apache.spark.sql.rapids.execution.HashJoinIterator.hasNext(GpuHashJoin.scala:349)
        at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:458)
        at org.apache.spark.sql.rapids.execution.HashJoinIterator.hasNext(GpuHashJoin.scala:345)
        at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:458)
        at com.nvidia.spark.rapids.GpuHashAggregateExec.$anonfun$doExecuteColumnar$1(aggregate.scala:345)
        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:462)
        at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1377)
        at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:465)
        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)

@jlowe @abellina @revans2

@rongou rongou added bug Something isn't working ? - Needs Triage Need team to review and classify labels May 25, 2021
@sameerz sameerz added P0 Must have for release and removed ? - Needs Triage Need team to review and classify labels May 25, 2021
@jlowe jlowe changed the title [BUG] out-of-memory error in native code [BUG] OOM when running NDS queries with UCX and GDS May 26, 2021
@jlowe
Copy link
Member

jlowe commented May 26, 2021

Probably caused by native code allocating memory while java/scala is actively spilling.

libcudf allocations go through RMM, and all RMM allocations are wrapped via the JNI memory handler that traps OOM exceptions and calls back into the JVM to try to spill. Therefore libcudf allocations should be covered, and allocating while spilling should not be a problem as long as there are buffers to spill.

Was the spill storage exhausted before the OOM was ultimately thrown to the application? The executor logs should have a warning message logged when the OOM is thrown that looks something like this:

          logWarning(s"Device store exhausted, unable to allocate $allocSize bytes. " +
              s"Total RMM allocated is ${Rmm.getTotalBytesAllocated} bytes.")

And just before that you should see one or more info log messages that show how much the device spill store had available at the time of each OOM that was trapped:

        logInfo(s"Device allocation of $allocSize bytes failed, device store has " +
            s"$storeSize bytes. Total RMM allocated is ${Rmm.getTotalBytesAllocated} bytes.")

@rongou
Copy link
Collaborator Author

rongou commented May 26, 2021

Ah that makes sense. Because gpu concurrency is set to 2, spilling lines are interspersed with adding new tables. Here is the output from DeviceMemoryEventHandler:

Executor task launch worker for task 15904 21/05/24 20:34:38:560 INFO DeviceMemoryEventHandler: Device allocation of 922253344 bytes failed, device store has 33414782208 bytes. Total RMM allocated is 39091921664 bytes.
Executor task launch worker for task 15904 21/05/24 20:34:38:868 INFO DeviceMemoryEventHandler: Spilled 957535936 bytes from the device store
Executor task launch worker for task 15904 21/05/24 20:34:38:868 INFO DeviceMemoryEventHandler: Device allocation of 922253344 bytes failed, device store has 32511059648 bytes. Total RMM allocated is 38166767104 bytes.
Executor task launch worker for task 15904 21/05/24 20:34:39:84 INFO DeviceMemoryEventHandler: Spilled 922488000 bytes from the device store
Executor task launch worker for task 15904 21/05/24 20:34:39:85 INFO DeviceMemoryEventHandler: Device allocation of 922253344 bytes failed, device store has 31592134400 bytes. Total RMM allocated is 37248090112 bytes.
Executor task launch worker for task 15904 21/05/24 20:34:39:314 INFO DeviceMemoryEventHandler: Spilled 923951424 bytes from the device store
Executor task launch worker for task 15904 21/05/24 20:34:39:314 INFO DeviceMemoryEventHandler: Device allocation of 922253344 bytes failed, device store has 30671030144 bytes. Total RMM allocated is 36324637184 bytes.
Executor task launch worker for task 15904 21/05/24 20:34:39:576 INFO DeviceMemoryEventHandler: Spilled 770921856 bytes from the device store
Executor task launch worker for task 15904 21/05/24 20:34:39:576 INFO DeviceMemoryEventHandler: Device allocation of 922253344 bytes failed, device store has 29749299776 bytes. Total RMM allocated is 35249146112 bytes.
Executor task launch worker for task 15904 21/05/24 20:34:40:267 INFO DeviceMemoryEventHandler: Spilled 923513280 bytes from the device store
Executor task launch worker for task 15904 21/05/24 20:34:40:267 INFO DeviceMemoryEventHandler: Device allocation of 922253344 bytes failed, device store has 28831180736 bytes. Total RMM allocated is 34325514240 bytes.
Executor task launch worker for task 15904 21/05/24 20:34:40:510 INFO DeviceMemoryEventHandler: Spilled 924609408 bytes from the device store
Executor task launch worker for task 15904 21/05/24 20:34:40:511 INFO DeviceMemoryEventHandler: Device allocation of 922253344 bytes failed, device store has 27911556992 bytes. Total RMM allocated is 33400769792 bytes.
Executor task launch worker for task 15904 21/05/24 20:34:40:752 INFO DeviceMemoryEventHandler: Spilled 923612928 bytes from the device store
Executor task launch worker for task 15904 21/05/24 20:34:40:752 INFO DeviceMemoryEventHandler: Device allocation of 922253344 bytes failed, device store has 26994072704 bytes. Total RMM allocated is 32477029376 bytes.
Executor task launch worker for task 15904 21/05/24 20:34:41:01 INFO DeviceMemoryEventHandler: Spilled 924478080 bytes from the device store
Executor task launch worker for task 15904 21/05/24 20:34:41:02 INFO DeviceMemoryEventHandler: Device allocation of 922253344 bytes failed, device store has 26074123520 bytes. Total RMM allocated is 31552404992 bytes.
Executor task launch worker for task 15904 21/05/24 20:34:41:254 INFO DeviceMemoryEventHandler: Spilled 922651776 bytes from the device store
Executor task launch worker for task 15904 21/05/24 20:34:41:255 INFO DeviceMemoryEventHandler: Device allocation of 922253344 bytes failed, device store has 25155440768 bytes. Total RMM allocated is 30634233344 bytes.
Executor task launch worker for task 15904 21/05/24 20:34:41:501 INFO DeviceMemoryEventHandler: Spilled 923119872 bytes from the device store
Executor task launch worker for task 15904 21/05/24 20:34:41:502 INFO DeviceMemoryEventHandler: Device allocation of 922253344 bytes failed, device store has 24236214080 bytes. Total RMM allocated is 29715189248 bytes.
Executor task launch worker for task 15904 21/05/24 20:34:41:743 INFO DeviceMemoryEventHandler: Spilled 922323648 bytes from the device store
Executor task launch worker for task 15904 21/05/24 20:34:41:743 INFO DeviceMemoryEventHandler: Device allocation of 922253344 bytes failed, device store has 23317938560 bytes. Total RMM allocated is 28797192704 bytes.
Executor task launch worker for task 15904 21/05/24 20:34:41:997 INFO DeviceMemoryEventHandler: Spilled 922265664 bytes from the device store
Executor task launch worker for task 15904 21/05/24 20:34:41:997 INFO DeviceMemoryEventHandler: Device allocation of 922253344 bytes failed, device store has 22399751360 bytes. Total RMM allocated is 27879741696 bytes.
Executor task launch worker for task 15904 21/05/24 20:34:42:240 INFO DeviceMemoryEventHandler: Spilled 923620608 bytes from the device store
Executor task launch worker for task 15904 21/05/24 20:34:42:240 INFO DeviceMemoryEventHandler: Device allocation of 922253344 bytes failed, device store has 21479970944 bytes. Total RMM allocated is 26961197312 bytes.
Executor task launch worker for task 15904 21/05/24 20:34:42:473 INFO DeviceMemoryEventHandler: Spilled 922864128 bytes from the device store
Executor task launch worker for task 15904 21/05/24 20:34:42:473 INFO DeviceMemoryEventHandler: Device allocation of 922253344 bytes failed, device store has 20561154752 bytes. Total RMM allocated is 26041150976 bytes.
Executor task launch worker for task 15904 21/05/24 20:34:42:718 INFO DeviceMemoryEventHandler: Spilled 922604352 bytes from the device store
Executor task launch worker for task 15904 21/05/24 20:34:42:719 INFO DeviceMemoryEventHandler: Device allocation of 922253344 bytes failed, device store has 19642611008 bytes. Total RMM allocated is 25122911744 bytes.
Executor task launch worker for task 15904 21/05/24 20:34:42:955 INFO DeviceMemoryEventHandler: Spilled 922385472 bytes from the device store
Executor task launch worker for task 15904 21/05/24 20:34:42:955 INFO DeviceMemoryEventHandler: Device allocation of 922253344 bytes failed, device store has 18724193792 bytes. Total RMM allocated is 24204867840 bytes.
Executor task launch worker for task 15904 21/05/24 20:34:43:190 INFO DeviceMemoryEventHandler: Spilled 922324800 bytes from the device store
Executor task launch worker for task 15904 21/05/24 20:34:43:191 INFO DeviceMemoryEventHandler: Device allocation of 922253344 bytes failed, device store has 17806062464 bytes. Total RMM allocated is 23287089920 bytes.
Executor task launch worker for task 15904 21/05/24 20:34:43:430 INFO DeviceMemoryEventHandler: Spilled 923287488 bytes from the device store
Executor task launch worker for task 15904 21/05/24 20:34:43:431 INFO DeviceMemoryEventHandler: Device allocation of 922253344 bytes failed, device store has 16886832512 bytes. Total RMM allocated is 22368237824 bytes.
Executor task launch worker for task 15904 21/05/24 20:34:43:671 INFO DeviceMemoryEventHandler: Spilled 922334400 bytes from the device store
Executor task launch worker for task 15904 21/05/24 20:34:43:672 INFO DeviceMemoryEventHandler: Device allocation of 922253344 bytes failed, device store has 15968340032 bytes. Total RMM allocated is 21450115584 bytes.
Executor task launch worker for task 15904 21/05/24 20:34:43:918 INFO DeviceMemoryEventHandler: Spilled 924839040 bytes from the device store
Executor task launch worker for task 15904 21/05/24 20:34:43:918 INFO DeviceMemoryEventHandler: Device allocation of 922253344 bytes failed, device store has 15047275904 bytes. Total RMM allocated is 20528381184 bytes.
Executor task launch worker for task 15904 21/05/24 20:34:44:424 INFO DeviceMemoryEventHandler: Spilled 922786816 bytes from the device store
Executor task launch worker for task 15904 21/05/24 20:34:44:424 INFO DeviceMemoryEventHandler: Device allocation of 922253344 bytes failed, device store has 14128526656 bytes. Total RMM allocated is 19609404160 bytes.
Executor task launch worker for task 15904 21/05/24 20:34:44:668 INFO DeviceMemoryEventHandler: Spilled 922778304 bytes from the device store
Executor task launch worker for task 15904 21/05/24 20:34:44:669 INFO DeviceMemoryEventHandler: Device allocation of 922253344 bytes failed, device store has 13209531136 bytes. Total RMM allocated is 18690748160 bytes.
Executor task launch worker for task 15904 21/05/24 20:34:44:921 INFO DeviceMemoryEventHandler: Spilled 923547456 bytes from the device store
Executor task launch worker for task 15904 21/05/24 20:34:44:922 INFO DeviceMemoryEventHandler: Device allocation of 922253344 bytes failed, device store has 12289936768 bytes. Total RMM allocated is 17771073792 bytes.
Executor task launch worker for task 15904 21/05/24 20:34:45:368 INFO DeviceMemoryEventHandler: Spilled 923549376 bytes from the device store
Executor task launch worker for task 15904 21/05/24 20:34:45:368 INFO DeviceMemoryEventHandler: Device allocation of 922253344 bytes failed, device store has 11369205760 bytes. Total RMM allocated is 16851182336 bytes.
Executor task launch worker for task 15904 21/05/24 20:34:45:874 INFO DeviceMemoryEventHandler: Spilled 923342784 bytes from the device store
Executor task launch worker for task 15904 21/05/24 20:34:45:875 INFO DeviceMemoryEventHandler: Device allocation of 922253344 bytes failed, device store has 10449756736 bytes. Total RMM allocated is 15931006208 bytes.
Executor task launch worker for task 15904 21/05/24 20:34:46:117 INFO DeviceMemoryEventHandler: Spilled 922883520 bytes from the device store
Executor task launch worker for task 15904 21/05/24 20:34:46:117 INFO DeviceMemoryEventHandler: Device allocation of 922253344 bytes failed, device store has 9530909248 bytes. Total RMM allocated is 15012487936 bytes.
Executor task launch worker for task 15904 21/05/24 20:34:46:345 INFO DeviceMemoryEventHandler: Spilled 922749696 bytes from the device store
Executor task launch worker for task 15904 21/05/24 20:34:46:345 INFO DeviceMemoryEventHandler: Device allocation of 922253344 bytes failed, device store has 8612094976 bytes. Total RMM allocated is 14094076416 bytes.
Executor task launch worker for task 15904 21/05/24 20:34:46:580 INFO DeviceMemoryEventHandler: Spilled 922618688 bytes from the device store
Executor task launch worker for task 15904 21/05/24 20:34:46:581 INFO DeviceMemoryEventHandler: Device allocation of 922253344 bytes failed, device store has 7693204736 bytes. Total RMM allocated is 13198959872 bytes.
Executor task launch worker for task 15904 21/05/24 20:34:46:830 INFO DeviceMemoryEventHandler: Spilled 924134720 bytes from the device store
Executor task launch worker for task 15904 21/05/24 20:34:46:830 INFO DeviceMemoryEventHandler: Device allocation of 922253344 bytes failed, device store has 6775202112 bytes. Total RMM allocated is 12279469312 bytes.
Executor task launch worker for task 15904 21/05/24 20:34:47:85 INFO DeviceMemoryEventHandler: Spilled 923583680 bytes from the device store
Executor task launch worker for task 15904 21/05/24 20:34:47:85 INFO DeviceMemoryEventHandler: Device allocation of 922253344 bytes failed, device store has 5855502400 bytes. Total RMM allocated is 11360095744 bytes.
Executor task launch worker for task 15904 21/05/24 20:34:47:329 INFO DeviceMemoryEventHandler: Spilled 922276480 bytes from the device store
Executor task launch worker for task 15904 21/05/24 20:34:47:330 INFO DeviceMemoryEventHandler: Device allocation of 922253344 bytes failed, device store has 4937363712 bytes. Total RMM allocated is 10442732032 bytes.
Executor task launch worker for task 15904 21/05/24 20:34:47:582 INFO DeviceMemoryEventHandler: Spilled 923401216 bytes from the device store
Executor task launch worker for task 15904 21/05/24 20:34:47:583 INFO DeviceMemoryEventHandler: Device allocation of 922253344 bytes failed, device store has 4018006976 bytes. Total RMM allocated is 9524222208 bytes.
Executor task launch worker for task 15904 21/05/24 20:34:47:827 INFO DeviceMemoryEventHandler: Spilled 922386816 bytes from the device store
Executor task launch worker for task 15904 21/05/24 20:34:47:828 INFO DeviceMemoryEventHandler: Device allocation of 922253344 bytes failed, device store has 3099492224 bytes. Total RMM allocated is 8604730880 bytes.
Executor task launch worker for task 15904 21/05/24 20:34:48:98 INFO DeviceMemoryEventHandler: Spilled 923372800 bytes from the device store
Executor task launch worker for task 15904 21/05/24 20:34:48:99 INFO DeviceMemoryEventHandler: Device allocation of 922253344 bytes failed, device store has 2180239168 bytes. Total RMM allocated is 7685363712 bytes.
Executor task launch worker for task 15904 21/05/24 20:34:48:349 INFO DeviceMemoryEventHandler: Spilled 922616960 bytes from the device store
Executor task launch worker for task 15904 21/05/24 20:34:48:349 INFO DeviceMemoryEventHandler: Device allocation of 922253344 bytes failed, device store has 1261718912 bytes. Total RMM allocated is 6767174144 bytes.
Executor task launch worker for task 15904 21/05/24 20:34:48:606 INFO DeviceMemoryEventHandler: Spilled 923679424 bytes from the device store
Executor task launch worker for task 15904 21/05/24 20:34:48:606 INFO DeviceMemoryEventHandler: Device allocation of 922253344 bytes failed, device store has 342021952 bytes. Total RMM allocated is 5847780864 bytes.
Executor task launch worker for task 15904 21/05/24 20:34:48:717 INFO DeviceMemoryEventHandler: Spilled 342021952 bytes from the device store
Executor task launch worker for task 15904 21/05/24 20:34:48:717 INFO DeviceMemoryEventHandler: Device allocation of 922253344 bytes failed, device store has 3473664 bytes. Total RMM allocated is 5510065920 bytes.
Executor task launch worker for task 15904 21/05/24 20:34:48:722 INFO DeviceMemoryEventHandler: Spilled 3472320 bytes from the device store
Executor task launch worker for task 15904 21/05/24 20:34:48:722 INFO DeviceMemoryEventHandler: Device allocation of 922253344 bytes failed, device store has 4190400 bytes. Total RMM allocated is 5510883840 bytes.
Executor task launch worker for task 15904 21/05/24 20:34:48:725 INFO DeviceMemoryEventHandler: Spilled 4190400 bytes from the device store
Executor task launch worker for task 15904 21/05/24 20:34:48:725 INFO DeviceMemoryEventHandler: Device allocation of 922253344 bytes failed, device store has 494592 bytes. Total RMM allocated is 5507719936 bytes.
Executor task launch worker for task 15904 21/05/24 20:34:48:725 INFO DeviceMemoryEventHandler: Spilled 494592 bytes from the device store
Executor task launch worker for task 15904 21/05/24 20:34:48:725 INFO DeviceMemoryEventHandler: Device allocation of 922253344 bytes failed, device store has 0 bytes. Total RMM allocated is 5509950464 bytes.
Executor task launch worker for task 15904 21/05/24 20:34:48:725 WARN DeviceMemoryEventHandler: Device store exhausted, unable to allocate 922253344 bytes. Total RMM allocated is 5524215552 bytes.

Maybe the memory is just super fragmented? I do see a lot of tiny shuffle buffers.

@jlowe
Copy link
Member

jlowe commented May 26, 2021

There is a config feature to perform a heap dump on GPU OOM. It may be useful to take a look at the resulting heap dump and see what DeviceMemoryBuffers are remaining and if we can trace them to owners that would give a clue as to why they are being retained. This can also be used to verify the extreme fragmentation theory.

@rongou
Copy link
Collaborator Author

rongou commented May 27, 2021

Looks like these errors happen with GDS disabled as well, so it's not specific to GDS spilling.

@abellina
Copy link
Collaborator

Agree with @jlowe. The heap dump on OOM feature would help figure out what's going on here.

@rongou rongou mentioned this issue Jun 4, 2021
11 tasks
@abellina
Copy link
Collaborator

abellina commented Jun 18, 2021

We see a similar issue in Q64 TPCDS @ 3TB. In this case, it is a query that normally passes, but earlier in the month, it failed with what looks like a fragmented pool:

Executor task launch worker for task 13527 21/06/04 07:29:47:837 INFO DeviceMemoryEventHandler: Device allocation of 1064613504 bytes failed, device store has 0 bytes. Total RMM allocated is 2682843136 bytes.

I'll take a look at running this query and capturing the heap at various places to see if we have leaks, it may overlap with this task.

@abellina abellina self-assigned this Jun 24, 2021
@rongou
Copy link
Collaborator Author

rongou commented Jun 25, 2021

I got a hprof dump from one of the OOM queries. Looks like all the DeviceMemoryBuffers are closed. I wonder where all the rest of the RMM allocation is from.

@abellina
Copy link
Collaborator

abellina commented Jun 25, 2021

I got a hprof dump from one of the OOM queries. Looks like all the DeviceMemoryBuffers are closed. I wonder where all the rest of the RMM allocation is from.

That's great @rongou. As far as I understand, spill is blocking, so if we wanted to allocate X, and we freed Y where Y >= X, but we still can't allocate, it probably means that RMM is fragmented. But, I can't recall what happens right after we free to target Y. Do we allocate in the same stack (holding a lock) or do we free the lock? If we return and release the lock then there could be a race with another task.

In this case, the key seems to be in the first, and the last two lines of the log:

Executor task launch worker for task 15904 21/05/24 20:34:38:560 INFO DeviceMemoryEventHandler: Device allocation of 922253344 bytes failed, device store has 33414782208 bytes. Total RMM allocated is 39091921664 bytes.

...

Executor task launch worker for task 15904 21/05/24 20:34:48:725 INFO DeviceMemoryEventHandler: Device allocation of 922253344 bytes failed, device store has 0 bytes. Total RMM allocated is 5509950464 bytes.
Executor task launch worker for task 15904 21/05/24 20:34:48:725 WARN DeviceMemoryEventHandler: Device store exhausted, unable to allocate 922253344 bytes. Total RMM allocated is 5524215552 bytes.

This was a ~40GB GPU, and was mostly full at 20:34:38:560. At 20:34:48:725, we gave up because we freed 33GB from the GPU store. But we still can't allocate 900MB in the last entry, and RMM is telling us it sees 5GB allocated.

So either the RMM allocated number is not accurate (because of races or some other issue), or we have enough fragmentation that a 900MB block can't be found.

@rongou rongou self-assigned this Jun 28, 2021
@rongou
Copy link
Collaborator Author

rongou commented Jun 28, 2021

Looking at tweaking the arena allocator that might help with this.

@abellina
Copy link
Collaborator

Filed this rapidsai/rmm#813 to be able to get more info from ARENA in cases like this, I believe that is a pre-requisite for this issue.

@jlowe
Copy link
Member

jlowe commented Jul 16, 2021

I think we may be able to learn quite a bit by leveraging the tracking allocator to build a rough approximation of the memory map. Minimally it could be used to help verify we don't have a small GPU memory leak that is aggravating the fragmentation.

@abellina
Copy link
Collaborator

Thanks @jlowe will take a look at that then to make progress.

@abellina
Copy link
Collaborator

I've made some progress on this and have a theory. If I set the minimum_superblock_size (defaults to 256KiB) to 0, I can get a query to succeed where it fails consistently at 256KiB (q14a). I ran it 20 times in a row with the same JVMs without issues, where before it would fail within 5 iterations.

I came to this variable because of the logic on shrink_arena. After looking at some RMM logs, what I think is happening is that we have tiny allocations that are less than minimum_superblock_size very consistently, and are never returned to the global arena because shrink_arena wants to conserve these inside of the stream arena. Because we normally run with many threads (PTDS streams), we can have interleaving streams for these allocations (i.e. free blocks), so they are also not getting coalesced. When looking at RMM logs we have a ton of space left on the GPU (i.e. GBs left for an allocation int the MB range), but it is likely made up of tiny non-contiguous free blocks that are freed below the threshold into the stream arena. I'd like to prove this, I don't have a map of the GPU memory showing this.

If I set the minimum_superblock_size to 0, it works, and I think it works because we always relinquish the block to the global arena, which then can merge and provide bigger blocks for a future allocation.

Because of the interleaving, the blocks likely cannot be merged https://github.com/rapidsai/rmm/blob/branch-21.10/include/rmm/mr/device/detail/arena.hpp#L201, as they are not contiguous in VA space.

@abellina
Copy link
Collaborator

abellina commented Aug 17, 2021

I tracked allocations and frees and their respective sizes, especially separating the blocks under minimum_superblock_size. Over time, for q14a (with the setting at 256KiB) I see the blocks accumulate.

I am also tracking gaps between blocks: (i.e. [block1 ---- gap_1_2 ---- block2]). If I count these "non-superblocks" as allocated blocks, which they essentially become with more and more likelihood from a non-owning stream, then the gap shortens to single-digit MiB quickly even with 30GiB worth of memory free in a 40GiB GPU, so this confirms the fragmentation, as long as the tracking is right.

@abellina
Copy link
Collaborator

@rongou has opened a PR for this issue here: rapidsai/rmm#845, so we are mostly working there. I'll close this issue once we have a cuDF that includes the fix.

@abellina
Copy link
Collaborator

At this point this issue is waiting for Rong's PR. I made a comment that is going to take a little bit of work, but @rongou mentioned he will be able to get to it (rapidsai/rmm#845 (comment)). The main thing is if local arenas can't reach 0 free blocks, we could very well end up in a similar place as before, but the thought is to fix that so we can achieve 0 free blocks in all arenas on OOM, which ensures arena can get back to initial state.

@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 Author

rongou commented Jan 12, 2022

We've improved the arena allocator as much as we could, the oom issue is better now but not completely eliminated. Switching to the async allocator should further help. #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