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] cudaErrorIllegalAddress for q95 (3TB) on GCP with ASYNC allocator #4710

Closed
abellina opened this issue Feb 7, 2022 · 17 comments · Fixed by #4947
Closed

[BUG] cudaErrorIllegalAddress for q95 (3TB) on GCP with ASYNC allocator #4710

abellina opened this issue Feb 7, 2022 · 17 comments · Fixed by #4947
Assignees
Labels
bug Something isn't working cudf_dependency An issue or PR with this label depends on a new feature in cudf P0 Must have for release

Comments

@abellina
Copy link
Collaborator

abellina commented Feb 7, 2022

I am seeing cudaErrorIllegalAddress for q95 pretty consistently with ASYNC allocator. This was in dataproc (Spark 3.1.2) with n1-standard-32 instances with 2 T4s attached. The RAPIDS Shuffle Manager was not used, as opposed to #4695.

JARS used:

rapids-4-spark_2.12-22.04.0-20220128.091637-3.jar
cudf-22.04.0-20220128.212138-6-cuda11.jar

I ran q95 100 times with ARENA and I can't reproduce it, with ASYNC I got it to happen 12 times.

I see the cudaErrorIllegalAddress in two stacks:

ai.rapids.cudf.CudfException: transform: failed to synchronize: cudaErrorIllegalAddress: an illegal memory access was encountered
  at ai.rapids.cudf.Table.gather(Native Method)
  at ai.rapids.cudf.Table.gather(Table.java:2214)
  at com.nvidia.spark.rapids.JoinGathererImpl.$anonfun$gatherNext$2(JoinGatherer.scala:537)
Full stack:
22/02/06 21:47:36 ERROR org.apache.spark.executor.Executor: Exception in task 80.0 in stage 42.0 (TID 12756)
ai.rapids.cudf.CudfException: transform: failed to synchronize: cudaErrorIllegalAddress: an illegal memory access was encountered
  at ai.rapids.cudf.Table.gather(Native Method)
  at ai.rapids.cudf.Table.gather(Table.java:2214)
  at com.nvidia.spark.rapids.JoinGathererImpl.$anonfun$gatherNext$2(JoinGatherer.scala:537)
  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.JoinGathererImpl.withResource(JoinGatherer.scala:497)
  at com.nvidia.spark.rapids.JoinGathererImpl.$anonfun$gatherNext$1(JoinGatherer.scala:536)
  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.JoinGathererImpl.withResource(JoinGatherer.scala:497)
  at com.nvidia.spark.rapids.JoinGathererImpl.gatherNext(JoinGatherer.scala:534)
  at com.nvidia.spark.rapids.MultiJoinGather.gatherNext(JoinGatherer.scala:605)
  at com.nvidia.spark.rapids.AbstractGpuJoinIterator.$anonfun$nextCbFromGatherer$2(AbstractGpuJoinIterator.scala:116)
  at scala.Option.map(Option.scala:230)
  at com.nvidia.spark.rapids.AbstractGpuJoinIterator.$anonfun$nextCbFromGatherer$1(AbstractGpuJoinIterator.scala:114)
  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.AbstractGpuJoinIterator.withResource(AbstractGpuJoinIterator.scala:36)
  at com.nvidia.spark.rapids.AbstractGpuJoinIterator.nextCbFromGatherer(AbstractGpuJoinIterator.scala:113)
  at com.nvidia.spark.rapids.AbstractGpuJoinIterator.$anonfun$hasNext$2(AbstractGpuJoinIterator.scala:68)
  at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23)
  at com.nvidia.spark.rapids.GpuMetric.ns(GpuExec.scala:163)
  at com.nvidia.spark.rapids.AbstractGpuJoinIterator.hasNext(AbstractGpuJoinIterator.scala:68)
  at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:460)
  at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:460)
  at com.nvidia.spark.rapids.CollectTimeIterator.$anonfun$hasNext$1(GpuExec.scala:190)
  at com.nvidia.spark.rapids.CollectTimeIterator.$anonfun$hasNext$1$adapted(GpuExec.scala:189)
  at com.nvidia.spark.rapids.Arm.withResource(Arm.scala:28)
  at com.nvidia.spark.rapids.Arm.withResource$(Arm.scala:26)
  at com.nvidia.spark.RebaseHelper$.withResource(RebaseHelper.scala:25)
  at com.nvidia.spark.rapids.CollectTimeIterator.hasNext(GpuExec.scala:189)
  at com.nvidia.spark.rapids.AbstractGpuCoalesceIterator.$anonfun$next$1(GpuCoalesceBatches.scala:340)
  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.AbstractGpuCoalesceIterator.withResource(GpuCoalesceBatches.scala:202)
  at com.nvidia.spark.rapids.AbstractGpuCoalesceIterator.next(GpuCoalesceBatches.scala:322)
  at com.nvidia.spark.rapids.AbstractGpuCoalesceIterator.next(GpuCoalesceBatches.scala:202)
  at com.nvidia.spark.rapids.ConcatAndConsumeAll$.getSingleBatchWithVerification(GpuCoalesceBatches.scala:83)
  at com.nvidia.spark.rapids.GpuShuffledHashJoinExec.$anonfun$doExecuteColumnar$1(GpuShuffledHashJoinExec.scala:140)
  at org.apache.spark.rdd.ZippedPartitionsRDD2.compute(ZippedPartitionsRDD.scala:89)
  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.ZippedPartitionsRDD2.compute(ZippedPartitionsRDD.scala:89)
  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.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.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.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.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)
cuda11/cpp/build/_deps/rmm-src/include/rmm/mr/device/cuda_async_memory_resource.hpp:153: cudaErrorIllegalAddress an illegal memory access was encountered
  at ai.rapids.cudf.Rmm.allocInternal(Native Method)
  at ai.rapids.cudf.Rmm.alloc(Rmm.java:246)
  at ai.rapids.cudf.DeviceMemoryBuffer.allocate(DeviceMemoryBuffer.java:143)
  at ai.rapids.cudf.DeviceMemoryBuffer.allocate(DeviceMemoryBuffer.java:133)
  at ai.rapids.cudf.JCudfSerialization$HostConcatResult.toContiguousTable(JCudfSerialization.java:378)
  at com.nvidia.spark.rapids.GpuShuffleCoalesceIterator.$anonfun$concatenateTablesBatch$5(GpuShuffleCoalesceExec.scala:200)
Full stack:
22/02/06 21:47:36 ERROR org.apache.spark.executor.Executor: Exception in task 64.0 in stage 42.0 (TID 12740)
ai.rapids.cudf.CudfException: std::bad_alloc: CUDA error at: /home/jenkins/agent/workspace/jenkins-cudf_nightly-dev-github-614-cuda11/cpp/build/_deps/rmm-src/include/rmm/mr/device/cuda_async_memory_resource.hpp:153: cudaErrorIllegalAddress an illegal memory access was encountered
  at ai.rapids.cudf.Rmm.allocInternal(Native Method)
  at ai.rapids.cudf.Rmm.alloc(Rmm.java:246)
  at ai.rapids.cudf.DeviceMemoryBuffer.allocate(DeviceMemoryBuffer.java:143)
  at ai.rapids.cudf.DeviceMemoryBuffer.allocate(DeviceMemoryBuffer.java:133)
  at ai.rapids.cudf.JCudfSerialization$HostConcatResult.toContiguousTable(JCudfSerialization.java:378)
  at com.nvidia.spark.rapids.GpuShuffleCoalesceIterator.$anonfun$concatenateTablesBatch$5(GpuShuffleCoalesceExec.scala:200)
  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.GpuShuffleCoalesceIterator.withResource(GpuShuffleCoalesceExec.scala:78)
  at com.nvidia.spark.rapids.GpuShuffleCoalesceIterator.$anonfun$concatenateTablesBatch$4(GpuShuffleCoalesceExec.scala:199)
  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.GpuShuffleCoalesceIterator.withResource(GpuShuffleCoalesceExec.scala:78)
  at com.nvidia.spark.rapids.GpuShuffleCoalesceIterator.$anonfun$concatenateTablesBatch$3(GpuShuffleCoalesceExec.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.GpuShuffleCoalesceIterator.withResource(GpuShuffleCoalesceExec.scala:78)
  at com.nvidia.spark.rapids.GpuShuffleCoalesceIterator.$anonfun$concatenateTablesBatch$1(GpuShuffleCoalesceExec.scala:195)
  at com.nvidia.spark.rapids.Arm.withResource(Arm.scala:55)
  at com.nvidia.spark.rapids.Arm.withResource$(Arm.scala:53)
  at com.nvidia.spark.rapids.GpuShuffleCoalesceIterator.withResource(GpuShuffleCoalesceExec.scala:78)
  at com.nvidia.spark.rapids.GpuShuffleCoalesceIterator.concatenateTablesBatch(GpuShuffleCoalesceExec.scala:188)
  at com.nvidia.spark.rapids.GpuShuffleCoalesceIterator.$anonfun$concatenateBatch$1(GpuShuffleCoalesceExec.scala:162)
  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.GpuShuffleCoalesceIterator.withResource(GpuShuffleCoalesceExec.scala:78)
  at com.nvidia.spark.rapids.GpuShuffleCoalesceIterator.concatenateBatch(GpuShuffleCoalesceExec.scala:153)
  at com.nvidia.spark.rapids.GpuShuffleCoalesceIterator.next(GpuShuffleCoalesceExec.scala:107)
  at com.nvidia.spark.rapids.GpuShuffleCoalesceIterator.next(GpuShuffleCoalesceExec.scala:78)
  at com.nvidia.spark.rapids.CollectTimeIterator.$anonfun$next$1(GpuExec.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.RebaseHelper$.withResource(RebaseHelper.scala:25)
  at com.nvidia.spark.rapids.CollectTimeIterator.next(GpuExec.scala:195)
  at com.nvidia.spark.rapids.AbstractGpuCoalesceIterator.hasNext(GpuCoalesceBatches.scala:248)
  at com.nvidia.spark.rapids.ConcatAndConsumeAll$.getSingleBatchWithVerification(GpuCoalesceBatches.scala:80)
  at com.nvidia.spark.rapids.GpuShuffledHashJoinExec.$anonfun$doExecuteColumnar$1(GpuShuffledHashJoinExec.scala:140)
  at org.apache.spark.rdd.ZippedPartitionsRDD2.compute(ZippedPartitionsRDD.scala:89)
  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.rdd.ZippedPartitionsRDD2.compute(ZippedPartitionsRDD.scala:89)
  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.ZippedPartitionsRDD2.compute(ZippedPartitionsRDD.scala:89)
  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.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.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.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.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)

Configs used:

spark.rapids.sql.batchSizeBytes=1GB
spark.driver.maxResultSize=2GB
spark.executor.cores=16
spark.locality.wait=0
spark.rapids.sql.concurrentGpuTasks=2
spark.executor.resource.gpu.amount=1
spark.task.resource.gpu.amount=0.0625
spark.executor.memory=16G
spark.driver.memory=50G
spark.sql.shuffle.partitions=200
spark.sql.files.maxPartitionBytes=2gb
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.executor.extraJavaOptions=-Dai.rapids.cudf.nvtx.enabled=true
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=512KB
spark.rapids.sql.incompatibleOps.enabled=true
spark.rapids.sql.variableFloatAgg.enabled=true
spark.rapids.sql.hasNans=false
spark.executor.instances=8
spark.rapids.sql.decimalType.enabled=true
spark.rapids.sql.castFloatToDecimal.enabled=true
spark.scheduler.mode=FIFO
spark.sql.autoBroadcastJoinThreshold=10m
spark.executor.memoryOverhead=16G
@abellina abellina added bug Something isn't working ? - Needs Triage Need team to review and classify labels Feb 7, 2022
@revans2
Copy link
Collaborator

revans2 commented Feb 7, 2022

Can you try running without any pooling? It would help us to know if it is a bug in CUDF/Plugin or if it is in the ASYNC allocator. I suspect it is in CUDF and ARENA is hiding it.

@abellina
Copy link
Collaborator Author

abellina commented Feb 7, 2022

Can you try running without any pooling? It would help us to know if it is a bug in CUDF/Plugin or if it is in the ASYNC allocator. I suspect it is in CUDF and ARENA is hiding it.

Yes, I am trying this now.

@abellina abellina self-assigned this Feb 7, 2022
@abellina
Copy link
Collaborator Author

abellina commented Feb 7, 2022

I have run q95 100 times without pooling, and the issue doesn't reproduce.

@rongou
Copy link
Collaborator

rongou commented Feb 7, 2022

I wonder if it's because the async pool was running out memory. Can you try to add spark.rapids.memory.gpu.allocFraction=0.9 (or even lower) to see if that helps?

@jlowe
Copy link
Contributor

jlowe commented Feb 7, 2022

I wonder if it's because the async pool was running out memory.

Wouldn't that manifest as a CUDA out of memory error code rather than an illegal address error?

@abellina
Copy link
Collaborator Author

abellina commented Feb 7, 2022

I wonder if it's because the async pool was running out memory.

Wouldn't that manifest as a CUDA out of memory error code rather than an illegal address error?

Agree, I don't understand this either, but I tried it with spark.rapids.memory.gpu.allocFraction=0.8 @rongou, and it failed the first time with ASYNC with illegal address.

@sameerz sameerz removed the ? - Needs Triage Need team to review and classify label Feb 8, 2022
@abellina
Copy link
Collaborator Author

abellina commented Feb 9, 2022

If I synchronize before calling gather in the GpuHashJoin code, I can't get it to happen. From my testing, it looks like after obtaining a mixed join gatherer (specifically an inner join), that's where the synchronization is needed.

Here's the shuffled hash join node where I think we are needing this:

(43) GpuShuffledHashJoin
Left output [2]: [ws_warehouse_sk#230, ws_order_number#232L]
Right output [2]: [ws_warehouse_sk#905, ws_order_number#907L]
Arguments: [ws_order_number#232L], [ws_order_number#907L], Inner, GpuBuildRight, NOT (ws_warehouse_sk#230 = ws_warehouse_sk#905), false

This change "fixes" it, but we need to figure out why exactly:

@@ -490,7 +489,9 @@ class ConditionalHashJoinIterator(
               throw new NotImplementedError(s"Joint Type ${joinType.getClass} is not currently" +
                   s" supported")
           }
-          makeGatherer(maps, leftData, rightData, joinType)
+          val gatherer = makeGatherer(maps, leftData, rightData, joinType)
+          Cuda.DEFAULT_STREAM.sync()
+          gatherer
         }
       }
     }

Note that I previously thought it was the non-mixed gatherer code, but adding a synchronize there didn't help. So it seems specific to mixed joins so far.

@abellina
Copy link
Collaborator Author

abellina commented Feb 9, 2022

This also just reproed in our local performance cluster.

@rongou
Copy link
Collaborator

rongou commented Feb 11, 2022

I saw a similar error with q16 as well on the DGX-2:

22/02/10 11:39:45 WARN TaskSetManager: Lost task 84.0 in stage 42.0 (TID 13843) (10.150.30.3 executor 0): ai.rapids.cudf.CudfException: CUDA error at: _deps/rmm-src/include/rmm/device_buffer.hpp:429: cudaErrorIl
legalAddress an illegal memory access was encountered
	at ai.rapids.cudf.Table.mixedLeftSemiJoinGatherMap(Native Method)
	at ai.rapids.cudf.Table.mixedLeftSemiJoinGatherMap(Table.java:3011)
	at org.apache.spark.sql.rapids.execution.ConditionalHashJoinIterator.$anonfun$joinGathererLeftRight$6(GpuHashJoin.scala:488)
	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.AbstractGpuJoinIterator.withResource(AbstractGpuJoinIterator.scala:36)
	at org.apache.spark.sql.rapids.execution.ConditionalHashJoinIterator.$anonfun$joinGathererLeftRight$5(GpuHashJoin.scala:470)
	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.AbstractGpuJoinIterator.withResource(AbstractGpuJoinIterator.scala:36)
	at org.apache.spark.sql.rapids.execution.ConditionalHashJoinIterator.$anonfun$joinGathererLeftRight$4(GpuHashJoin.scala:469)
	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.AbstractGpuJoinIterator.withResource(AbstractGpuJoinIterator.scala:36)
	at org.apache.spark.sql.rapids.execution.ConditionalHashJoinIterator.joinGathererLeftRight(GpuHashJoin.scala:468)
	at org.apache.spark.sql.rapids.execution.BaseHashJoinIterator.$anonfun$joinGathererLeftRight$2(GpuHashJoin.scala:314)
	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.AbstractGpuJoinIterator.withResource(AbstractGpuJoinIterator.scala:36)
	at org.apache.spark.sql.rapids.execution.BaseHashJoinIterator.$anonfun$joinGathererLeftRight$1(GpuHashJoin.scala:313)
	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.AbstractGpuJoinIterator.withResource(AbstractGpuJoinIterator.scala:36)
	at org.apache.spark.sql.rapids.execution.BaseHashJoinIterator.joinGathererLeftRight(GpuHashJoin.scala:312)
	at org.apache.spark.sql.rapids.execution.BaseHashJoinIterator.joinGatherer(GpuHashJoin.scala:328)
	at org.apache.spark.sql.rapids.execution.BaseHashJoinIterator.$anonfun$joinGatherer$2(GpuHashJoin.scala:338)
	at com.nvidia.spark.rapids.Arm.closeOnExcept(Arm.scala:87)
	at com.nvidia.spark.rapids.Arm.closeOnExcept$(Arm.scala:85)
	at com.nvidia.spark.rapids.AbstractGpuJoinIterator.closeOnExcept(AbstractGpuJoinIterator.scala:36)
	at org.apache.spark.sql.rapids.execution.BaseHashJoinIterator.$anonfun$joinGatherer$1(GpuHashJoin.scala:337)
	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.AbstractGpuJoinIterator.withResource(AbstractGpuJoinIterator.scala:36)
	at org.apache.spark.sql.rapids.execution.BaseHashJoinIterator.joinGatherer(GpuHashJoin.scala:336)
	at org.apache.spark.sql.rapids.execution.BaseHashJoinIterator.$anonfun$createGatherer$1(GpuHashJoin.scala:273)
	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.AbstractGpuJoinIterator.withResource(AbstractGpuJoinIterator.scala:36)
	at org.apache.spark.sql.rapids.execution.BaseHashJoinIterator.createGatherer(GpuHashJoin.scala:272)
	at com.nvidia.spark.rapids.SplittableJoinIterator.$anonfun$setupNextGatherer$6(AbstractGpuJoinIterator.scala:215)
	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.AbstractGpuJoinIterator.withResource(AbstractGpuJoinIterator.scala:36)
	at com.nvidia.spark.rapids.SplittableJoinIterator.$anonfun$setupNextGatherer$5(AbstractGpuJoinIterator.scala:197)
	at com.nvidia.spark.rapids.GpuMetric.ns(GpuExec.scala:163)
	at com.nvidia.spark.rapids.SplittableJoinIterator.setupNextGatherer(AbstractGpuJoinIterator.scala:197)
	at com.nvidia.spark.rapids.AbstractGpuJoinIterator.hasNext(AbstractGpuJoinIterator.scala:77)
	at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:460)
	at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:460)
	at com.nvidia.spark.rapids.CollectTimeIterator.$anonfun$hasNext$1(GpuExec.scala:190)
	at com.nvidia.spark.rapids.CollectTimeIterator.$anonfun$hasNext$1$adapted(GpuExec.scala:189)
	at com.nvidia.spark.rapids.Arm.withResource(Arm.scala:28)
	at com.nvidia.spark.rapids.Arm.withResource$(Arm.scala:26)
	at com.nvidia.spark.RebaseHelper$.withResource(RebaseHelper.scala:25)
	at com.nvidia.spark.rapids.CollectTimeIterator.hasNext(GpuExec.scala:189)
	at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:460)
	at com.nvidia.spark.rapids.SplittableJoinIterator.hasNextStreamBatch(AbstractGpuJoinIterator.scala:175)
	at com.nvidia.spark.rapids.AbstractGpuJoinIterator.hasNext(AbstractGpuJoinIterator.scala:71)
	at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:460)
	at com.nvidia.spark.rapids.CollectTimeIterator.$anonfun$hasNext$1(GpuExec.scala:190)
	at com.nvidia.spark.rapids.CollectTimeIterator.$anonfun$hasNext$1$adapted(GpuExec.scala:189)
	at com.nvidia.spark.rapids.Arm.withResource(Arm.scala:28)
	at com.nvidia.spark.rapids.Arm.withResource$(Arm.scala:26)
	at com.nvidia.spark.RebaseHelper$.withResource(RebaseHelper.scala:25)
	at com.nvidia.spark.rapids.CollectTimeIterator.hasNext(GpuExec.scala:189)
	at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:460)
	at com.nvidia.spark.rapids.SplittableJoinIterator.hasNextStreamBatch(AbstractGpuJoinIterator.scala:175)
	at com.nvidia.spark.rapids.AbstractGpuJoinIterator.hasNext(AbstractGpuJoinIterator.scala:71)
	at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:460)
	at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:460)
	at com.nvidia.spark.rapids.CollectTimeIterator.$anonfun$hasNext$1(GpuExec.scala:190)
	at com.nvidia.spark.rapids.CollectTimeIterator.$anonfun$hasNext$1$adapted(GpuExec.scala:189)
	at com.nvidia.spark.rapids.Arm.withResource(Arm.scala:28)
	at com.nvidia.spark.rapids.Arm.withResource$(Arm.scala:26)
	at com.nvidia.spark.RebaseHelper$.withResource(RebaseHelper.scala:25)
	at com.nvidia.spark.rapids.CollectTimeIterator.hasNext(GpuExec.scala:189)
	at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:460)
	at com.nvidia.spark.rapids.SplittableJoinIterator.hasNextStreamBatch(AbstractGpuJoinIterator.scala:175)
	at com.nvidia.spark.rapids.AbstractGpuJoinIterator.hasNext(AbstractGpuJoinIterator.scala:71)
	at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:460)
	at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:460)
	at com.nvidia.spark.rapids.CollectTimeIterator.$anonfun$hasNext$1(GpuExec.scala:190)
	at com.nvidia.spark.rapids.CollectTimeIterator.$anonfun$hasNext$1$adapted(GpuExec.scala:189)
	at com.nvidia.spark.rapids.Arm.withResource(Arm.scala:28)
	at com.nvidia.spark.rapids.Arm.withResource$(Arm.scala:26)
	at com.nvidia.spark.RebaseHelper$.withResource(RebaseHelper.scala:25)
	at com.nvidia.spark.rapids.CollectTimeIterator.hasNext(GpuExec.scala:189)
	at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:460)
	at com.nvidia.spark.rapids.SplittableJoinIterator.hasNextStreamBatch(AbstractGpuJoinIterator.scala:175)
	at com.nvidia.spark.rapids.AbstractGpuJoinIterator.hasNext(AbstractGpuJoinIterator.scala:71)
	at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:460)
	at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:460)
	at com.nvidia.spark.rapids.ColumnarToRowIterator.$anonfun$fetchNextBatch$2(GpuColumnarToRowExec.scala:240)
	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.ColumnarToRowIterator.withResource(GpuColumnarToRowExec.scala:188)
	at com.nvidia.spark.rapids.ColumnarToRowIterator.fetchNextBatch(GpuColumnarToRowExec.scala:239)
	at com.nvidia.spark.rapids.ColumnarToRowIterator.loadNextBatch(GpuColumnarToRowExec.scala:216)
	at com.nvidia.spark.rapids.ColumnarToRowIterator.hasNext(GpuColumnarToRowExec.scala:256)
	at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:460)
	at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage1.agg_doAggregateWithKeys_1$(Unknown Source)
	at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage1.agg_doAggregateWithKeys_0$(Unknown Source)
	at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage1.agg_doAggregateWithoutKey_0$(Unknown Source)
	at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage1.processNext(Unknown Source)
	at org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)
	at org.apache.spark.sql.execution.WholeStageCodegenExec$$anon$1.hasNext(WholeStageCodegenExec.scala:759)
	at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:460)
	at org.apache.spark.shuffle.sort.BypassMergeSortShuffleWriter.write(BypassMergeSortShuffleWriter.java:140)
	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:506)
	at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1462)
	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:509)
	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 the P0 Must have for release label Feb 15, 2022
@abellina
Copy link
Collaborator Author

abellina commented Feb 16, 2022

An update on this issue. I believe a lot of this has to do with running out of memory. The original issue can be reproduced pretty easily if the number of concurrent threads in q95 is set high (16 concurrent allowed on the gpu for example). One issue looks to be that we are loosing stream information given that we use PTDS, and additionally we don't set the stream for gatherMap and for contiguous_split buffers (these DeviceMemoryBuffers are instantiated using DeviceMemoryBuffer.fromRmm, which doesn't take as tream). With PTDS, that stream is the per thread stream identifier, so it's not of much use.

ARENA has a per-thread tracking system when PTDS is on (not per stream). So it is adding synchronization in cases where an allocation was created in thread A and freed in thread B. This happens during spill pretty often, and that looks to be one of the areas where we see issues. I think this implies that there are paths where we are not synchronizing appropriately before we hand a buffer to ASYNC to free on stream B. When I run the workers under compute-sanitizer, these errors go from "illegal address" to being reported as out of memory, which I think is interesting (perhaps there is some synchronization happening under compute-sanitizer).

That said, I am able to see an illegal access with ARENA in a single threaded way as well, this may be a different issue, but it is happening in q95. For this issue I do have a compute-sanitizer log. I see it in the concatenate kernel invocation from ConcatAndConsumeAll.buildNonEmptyBatch. I dumped to parquet tables that caused the crash and tried to concatenate them outside of the process, and that worked fine. My guess is that serializing the tables to parquet is "fixing" them so they don't fail again. I'll need to try to dump the actual buffers.

I should say I tried the concatenate in a sequential way:

  def buildNonEmptyBatch(arrayOfBatches: Array[ColumnarBatch],
      schema: StructType): ColumnarBatch = {
    if (arrayOfBatches.length == 1) {
      arrayOfBatches(0)
    } else {
      val tables = arrayOfBatches.map(GpuColumnVector.from)
      try {
        Cuda.DEFAULT_STREAM.sync()
        logInfo(s"Total allocated=${Rmm.getTotalBytesAllocated}")
        logInfo(s"Concatenating: ${tables.length} tables")
        (1 until tables.length).foreach { ix =>
          withResource(Table.concatenate(tables.slice(0, ix+1).toArray:_*)) { _ =>
            logInfo(s"Concatenated $ix tables!")
          }
        }

        val combined = Table.concatenate(tables: _*)
        Cuda.DEFAULT_STREAM.sync()
        try {
          GpuColumnVector.from(combined, GpuColumnVector.extractTypes(schema))
        } finally {
          combined.close()
        }
      } finally {
        tables.foreach(_.close())
        arrayOfBatches.foreach(_.close())
      }
    }
  }

And I am able to get through some of these concats, indicating I haven't reached a bad table, or I am somehow running out of memory in the fused_concatenate_kernel and it's getting ignored. I suspect this is all related to the ASYNC issue.

========= Invalid __global__ read of size 8 bytes
=========     at 0x660 in void cudf::detail::fused_concatenate_kernel<long, (int)256, (bool)0>(const cudf::column_device_view *, const unsigned long *, int, cudf::mutable_column_device_view, int *)
=========     by thread (233,0,0) in block (3894854,0,0)
=========     Address 0x7ed40b7c7590 is out of bounds
=========     Saved host backtrace up to driver entry point at kernel launch time
=========     Host Frame: [0x22c12a]
=========                in /usr/lib/x86_64-linux-gnu/libcuda.so.1
=========     Host Frame: [0x2b4f35b]
=========                in /tmp/cudf5505390038906264702.so
=========     Host Frame: [0x2b8caa8]
=========                in /tmp/cudf5505390038906264702.so
=========     Host Frame:void cudf::detail::fused_concatenate_kernel<long, int=256, bool=0>(cudf::column_device_view const *, unsigned long const *, int, cudf::mutable_column_device_view, int*) [0x1210e01]
=========                in /tmp/cudf5505390038906264702.so
=========     Host Frame:std::unique_ptr<cudf::column, std::default_delete<std::unique_ptr>> cudf::detail::fused_concatenate<long>(cudf::host_span<cudf::column_view const , unsigned long=18446744073709551615>, bool, rmm::cuda_stream_view, cudf::host_span<cudf::column_view const , unsigned long=18446744073709551615>::mr::device_memory_resource*) [0x12189eb]
=========                in /tmp/cudf5505390038906264702.so
=========     Host Frame:cudf::detail::concatenate(cudf::host_span<cudf::column_view const , unsigned long=18446744073709551615>, rmm::cuda_stream_view, rmm::mr::device_memory_resource*) [0x120db86]
=========                in /tmp/cudf5505390038906264702.so
=========     Host Frame:cudf::detail::concatenate(cudf::host_span<cudf::table_view const , unsigned long=18446744073709551615>, rmm::cuda_stream_view, rmm::mr::device_memory_resource*) [0x120e4b0]
=========                in /tmp/cudf5505390038906264702.so
=========     Host Frame:cudf::concatenate(cudf::host_span<cudf::table_view const , unsigned long=18446744073709551615>, rmm::mr::device_memory_resource*) [0x120e7e2]
=========                in /tmp/cudf5505390038906264702.so
=========     Host Frame:Java_ai_rapids_cudf_Table_concatenate [0xf806f3]
=========                in /tmp/cudf5505390038906264702.so

@abellina
Copy link
Collaborator Author

I'll build again with lineinfo to see if we can get to the bottom of the concatenate issue. There is an old issue in cuDF that has a similar output (rapidsai/cudf#7722), but I am not sure if it is related.

@abellina
Copy link
Collaborator Author

Ok, with linenumber, I get that this is the line for the exception:

https://github.com/rapidsai/cudf/blob/branch-22.04/cpp/src/copying/concatenate.cu#L186

========= Invalid __global__ read of size 8 bytes
=========     at 0x660 in /root/cudf/cpp/src/copying/concatenate.cu:186:void cudf::detail::fused_concatenate_kernel<long, (int)256, (bool)0>(const cudf::column_device_view *, const unsigned long *, int, cudf::mutable_column_device_view, int *)

@abellina
Copy link
Collaborator Author

The concatenate issue that I have reported here is specific to the configuration used to try and reproduce the ASYNC issue.

The reason for this is I am starting q95 with 1 executor core, and I used 16 shuffle partitions, which changes the size of the batches that are materialized and sent to cuDF. The invalid access I'll link once I have a small repro likely a PR fix in cuDF.

@abellina
Copy link
Collaborator Author

This is the cuDF concatenate issue: rapidsai/cudf#10333.

@sameerz sameerz added the cudf_dependency An issue or PR with this label depends on a new feature in cudf label Feb 21, 2022
@sameerz sameerz added this to the Feb 14 - Feb 25 milestone Feb 21, 2022
@abellina
Copy link
Collaborator Author

Unfortunately I have seen the error again in dataproc, even when I am synchronizing events #4818 in the spill framework (I also didn't see us spilling in this case). Again in the gather stack.

Note that this is the error from dmesg:

[Thu Feb 24 15:47:43 2022] NVRM: Xid (PCI:0000:00:05): 31, pid=16262, Ch 00000008, intr 00000000. MMU Fault: ENGINE GRAPHICS GPCCLIENT_T1_0 faulted @ 0x6_5959f000. Fault is of type FAULT_PDE ACCESS_TYPE_VIRT_READ

These are T4s running driver 460.106.00 (11.2), so this should be a supported configuration.

I am failing to see why ASYNC would change things, other than it is another corruption like rapidsai/cudf#10333, that is hiding. I'll try to run with compute-sanitize in the dataproc cluster, I should be able to make that work.

@abellina
Copy link
Collaborator Author

This specific issue is not reproducible for drivers at or above 11.4.3. The PR (#4947), is making sure we don't use ASYNC by mistake in drivers < 11.5.0, as that's the minimum driver version we can easily test for.

@rongou
Copy link
Collaborator

rongou commented Mar 14, 2022

I'm adding a check in RMM: rapidsai/rmm#993

rapids-bot bot pushed a commit to rapidsai/rmm that referenced this issue Mar 16, 2022
With NVIDIA/spark-rapids#4710 we found some issues with the async pool that may cause memory errors with older drivers. This was confirmed with the cuda team. For driver version < 11.5, we'll disable `cudaMemPoolReuseAllowOpportunistic`.

@abellina

Authors:
  - Rong Ou (https://github.com/rongou)

Approvers:
  - Alessandro Bellina (https://github.com/abellina)
  - Jake Hemstad (https://github.com/jrhemstad)
  - Mark Harris (https://github.com/harrism)
  - Leo Fang (https://github.com/leofang)

URL: #993
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 P0 Must have for release
Projects
None yet
Development

Successfully merging a pull request may close this issue.

5 participants