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] cudaErrorInvalidDevice in ParquetChunkedReader with a small number of executor cores #11565

Open
jihoonson opened this issue Oct 8, 2024 · 11 comments
Assignees
Labels
bug Something isn't working

Comments

@jihoonson
Copy link
Collaborator

jihoonson commented Oct 8, 2024

Describe the bug

A simple join query fails with the error below.

24/10/07 17:06:59 WARN TaskSetManager: Lost task 0.0 in stage 31.0 (TID 12031) (10.110.47.50 executor 4): java.io.IOException: Error when processing path: file:///home/jihoons/data/tpcds/sf=10/parquet/store/part-00000-5b8288c9-c0f1-494a-9871-9d7fa7bd822e-c000.snappy.parquet, range: 0-17505, partition values: [empty row]
	at com.nvidia.spark.rapids.ParquetTableReader.$anonfun$next$1(GpuParquetScan.scala:2701)
	at com.nvidia.spark.rapids.Arm$.withResource(Arm.scala:30)
	at com.nvidia.spark.rapids.ParquetTableReader.next(GpuParquetScan.scala:2692)
	at com.nvidia.spark.rapids.ParquetTableReader.next(GpuParquetScan.scala:2664)
	at com.nvidia.spark.rapids.CachedGpuBatchIterator$.$anonfun$apply$1(GpuDataProducer.scala:159)
	at com.nvidia.spark.rapids.Arm$.withResource(Arm.scala:30)
	at com.nvidia.spark.rapids.CachedGpuBatchIterator$.apply(GpuDataProducer.scala:156)
	at com.nvidia.spark.rapids.MultiFileCoalescingPartitionReaderBase.$anonfun$readBatch$4(GpuMultiFileReader.scala:1065)
	at com.nvidia.spark.rapids.RmmRapidsRetryIterator$AutoCloseableAttemptSpliterator.next(RmmRapidsRetryIterator.scala:477)
	at com.nvidia.spark.rapids.RmmRapidsRetryIterator$RmmRapidsRetryIterator.next(RmmRapidsRetryIterator.scala:613)
	at com.nvidia.spark.rapids.RmmRapidsRetryIterator$RmmRapidsRetryAutoCloseableIterator.next(RmmRapidsRetryIterator.scala:517)
	at com.nvidia.spark.rapids.RmmRapidsRetryIterator$.drainSingleWithVerification(RmmRapidsRetryIterator.scala:291)
	at com.nvidia.spark.rapids.RmmRapidsRetryIterator$.withRetryNoSplit(RmmRapidsRetryIterator.scala:132)
	at com.nvidia.spark.rapids.MultiFileCoalescingPartitionReaderBase.$anonfun$readBatch$1(GpuMultiFileReader.scala:1058)
	at com.nvidia.spark.rapids.Arm$.withResource(Arm.scala:30)
	at com.nvidia.spark.rapids.MultiFileCoalescingPartitionReaderBase.readBatch(GpuMultiFileReader.scala:1031)
	at com.nvidia.spark.rapids.MultiFileCoalescingPartitionReaderBase.next(GpuMultiFileReader.scala:1011)
	at com.nvidia.spark.rapids.PartitionIterator.hasNext(dataSourceUtil.scala:29)
	at com.nvidia.spark.rapids.MetricsBatchIterator.hasNext(dataSourceUtil.scala:46)
	at com.nvidia.spark.rapids.shims.GpuDataSourceRDD$$anon$1.$anonfun$hasNext$1(GpuDataSourceRDD.scala:74)
	at com.nvidia.spark.rapids.shims.GpuDataSourceRDD$$anon$1.$anonfun$hasNext$1$adapted(GpuDataSourceRDD.scala:74)
	at scala.Option.exists(Option.scala:376)
	at com.nvidia.spark.rapids.shims.GpuDataSourceRDD$$anon$1.hasNext(GpuDataSourceRDD.scala:74)
	at com.nvidia.spark.rapids.shims.GpuDataSourceRDD$$anon$1.advanceToNextIter(GpuDataSourceRDD.scala:98)
	at com.nvidia.spark.rapids.shims.GpuDataSourceRDD$$anon$1.hasNext(GpuDataSourceRDD.scala:74)
	at org.apache.spark.InterruptibleIterator.hasNext(InterruptibleIterator.scala:37)
	at org.apache.spark.sql.rapids.GpuFileSourceScanExec$$anon$1.hasNext(GpuFileSourceScanExec.scala:477)
	at scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:491)
	at com.nvidia.spark.rapids.CollectTimeIterator.$anonfun$hasNext$1(GpuExec.scala:254)
	at com.nvidia.spark.rapids.CollectTimeIterator.$anonfun$hasNext$1$adapted(GpuExec.scala:253)
	at com.nvidia.spark.rapids.Arm$.withResource(Arm.scala:30)
	at com.nvidia.spark.rapids.CollectTimeIterator.hasNext(GpuExec.scala:253)
	at com.nvidia.spark.rapids.AbstractGpuCoalesceIterator.getHasOnDeck(GpuCoalesceBatches.scala:313)
	at com.nvidia.spark.rapids.AbstractGpuCoalesceIterator.hasNext(GpuCoalesceBatches.scala:330)
	at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:460)
	at scala.collection.Iterator.foreach(Iterator.scala:943)
	at scala.collection.Iterator.foreach$(Iterator.scala:943)
	at scala.collection.AbstractIterator.foreach(Iterator.scala:1431)
	at scala.collection.generic.Growable.$plus$plus$eq(Growable.scala:62)
	at scala.collection.generic.Growable.$plus$plus$eq$(Growable.scala:53)
	at scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:105)
	at scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:49)
	at scala.collection.TraversableOnce.to(TraversableOnce.scala:366)
	at scala.collection.TraversableOnce.to$(TraversableOnce.scala:364)
	at scala.collection.AbstractIterator.to(Iterator.scala:1431)
	at scala.collection.TraversableOnce.toBuffer(TraversableOnce.scala:358)
	at scala.collection.TraversableOnce.toBuffer$(TraversableOnce.scala:358)
	at scala.collection.AbstractIterator.toBuffer(Iterator.scala:1431)
	at scala.collection.TraversableOnce.toArray(TraversableOnce.scala:345)
	at scala.collection.TraversableOnce.toArray$(TraversableOnce.scala:339)
	at scala.collection.AbstractIterator.toArray(Iterator.scala:1431)
	at org.apache.spark.rdd.RDD.$anonfun$collect$2(RDD.scala:1049)
	at org.apache.spark.SparkContext.$anonfun$runJob$5(SparkContext.scala:2438)
	at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:93)
	at org.apache.spark.TaskContext.runTaskWithListeners(TaskContext.scala:166)
	at org.apache.spark.scheduler.Task.run(Task.scala:141)
	at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$4(Executor.scala:620)
	at org.apache.spark.util.SparkErrorUtils.tryWithSafeFinally(SparkErrorUtils.scala:64)
	at org.apache.spark.util.SparkErrorUtils.tryWithSafeFinally$(SparkErrorUtils.scala:61)
	at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:94)
	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:623)
	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:750)
Caused by: ai.rapids.cudf.CudfException: copy_if failed on 1st step: cudaErrorInvalidDevice: invalid device ordinal
	at ai.rapids.cudf.ParquetChunkedReader.readChunk(Native Method)
	at ai.rapids.cudf.ParquetChunkedReader.readChunk(ParquetChunkedReader.java:170)
	at com.nvidia.spark.rapids.ParquetTableReader.$anonfun$next$1(GpuParquetScan.scala:2694)
	... 63 more

Steps/Code to reproduce bug

Here is the data and query I'm using.

  • Data: TPC-DS at sf=10
  • Query:
select s.* 
from store s, store_sales ss 
where s.s_store_sk <= ss.ss_store_sk and s.s_store_sk + 21 > ss.ss_store_sk;

Per my observation so far, this query fails when spark.executor.cores is set to less than 10. It works fine otherwise. When the executor core count was set to 10, there were 6 executors created since my machine has 64 cpu cores. When the executor core count was 9, there were 7 executors created.

Expected behavior
Ideally, the query should run successfully even with a small number of executor cores. Or, if this is some edge case that the plugin cannot execute the query, then the query should fail with a more user-friendly error message.

Environment details (please complete the following information)

  • Environment location: single node standalone cluster. The node has 64 CPU cores.
  • Spark configuration settings related to the issue
SPARK_WORKER_OPTS="-Dspark.worker.resource.gpu.amount=1 -Dspark.worker.resource.gpu.discoveryScript=/path/to/getGpusResources.sh"

export SPARK_CONF=("--master" "spark:/my-master:7077"
                   "--conf" "spark.driver.maxResultSize=2GB"
                   "--conf" "spark.driver.memory=50G"
                   "--conf" "spark.executor.cores=9" <== setting this to 10 or a greater can avoid the error
                   "--conf" "spark.executor.instances=1" <== this did not take effect
                   "--conf" "spark.executor.memory=16G"
                   "--conf" "spark.sql.adaptive.enabled=true"
                   "--conf" "spark.plugins=com.nvidia.spark.SQLPlugin"
                   "--conf" "spark.rapids.memory.host.spillStorageSize=1MB"
                   "--conf" "spark.rapids.memory.pinnedPool.size=8g"
                   "--conf" "spark.rapids.sql.concurrentGpuTasks=4"
                   "--conf" "spark.rapids.sql.explain=all"
                   "--conf" "spark.eventLog.enabled=true"
                   "--conf" "spark.eventLog.dir=/tmp/spark-events"
                   "--conf" "spark.driver.extraClassPath=$SPARK_RAPIDS_PLUGIN_JAR"
                   "--conf" "spark.executor.extraClassPath=$SPARK_RAPIDS_PLUGIN_JAR"
                   "--conf" "spark.executor.extraJavaOptions=-Dai.rapids.cudf.nvtx.enabled=true"
                   "--conf" "spark.rapids.sql.batchSizeBytes=256MB"
                   "--conf" "spark.files.maxPartitionBytes=268435456"
                   "--conf" "spark.rapids.memory.gpu.allocSize=3758096384"
                   "--conf" "spark.rapids.sql.metrics.level=DEBUG")
@jihoonson jihoonson added ? - Needs Triage Need team to review and classify bug Something isn't working labels Oct 8, 2024
@jihoonson
Copy link
Collaborator Author

I am aware of #11215 and rapidsai/cudf#16186, but not entirely sure if these are the same issue.

@mattahrens
Copy link
Collaborator

What version of the plugin jar are you running with?

@mattahrens mattahrens removed the ? - Needs Triage Need team to review and classify label Oct 8, 2024
@jihoonson
Copy link
Collaborator Author

@mattahrens I've been using a branch in my repo which is based on branch-24.10. I will try with the latest branch and see if the problem still remains.

@jihoonson
Copy link
Collaborator Author

Just tested. branch-24.12 has the same issue.

@jihoonson
Copy link
Collaborator Author

It turns out that the issue is not about the GPU device setting (thanks to @abellina's help for debugging). It's about the memory, especially the so-called "reserve" memory. Because the GPU memory size is fixed in my setting, the total amount of GPU memory used by all executors grows as more executors are created. The amount assigned to each executor was 3.5G and my machine has 24G of total GPU memory. When the executor core count was set to 9, 7 executors were created (because my machine has 64 cpu cores), which would have left only few memory for GPU kernel execution unless otherwise it has exhausted memory. This likely has caused some kernel failure. So, the failure is legit as we cannot proceed query processing in this case, but the error message should be improved.

@revans2
Copy link
Collaborator

revans2 commented Oct 10, 2024

@jihoonson Are you running multiple spark processes sharing a single GPU? Was this on purpose?

@jihoonson
Copy link
Collaborator Author

jihoonson commented Oct 11, 2024

@revans2 yes I was, but it was not on purpose. It rather accidentally happened. I wanted to limit the executor process count to 1, but spark.executor.instances=1 did not work as I expected. I did not realize that I had multiple executor processes running until I looked at the logs more closely to debug this invalid device error.

@revans2
Copy link
Collaborator

revans2 commented Oct 14, 2024

@abellina and @jlowe do we want to update GpuDeviceManager so that spark.rapids.memory.gpu.allocSize will still honor the things like the min/max fraction? I am conflicted here because the allocSize is an internal only config because there were no checks and it was supposed to be for internal testing. But if devs are running into issues with it, should we try and find ways to avoid the foot-gun problem?

@abellina
Copy link
Collaborator

abellina commented Oct 14, 2024

With reserve, in general, I'd be interested to know if there is a way to make the error more user friendly as that's the part that was really confusing here.

I think allocSize we could change so that it warns loudly (on the driver... never mind it's got to be on the executor), "it looks like you went above reserve!" or something like that?

@jlowe
Copy link
Member

jlowe commented Oct 14, 2024

I'm personally OK if a spark.rapids.memory.gpu.allocSize setting causes a crash if it goes above the max fraction (or otherwise conflicts with other settings). Essentially you'd have to increase max alloc fraction and/or reduce reserve amount configs to allocate beyond normal upper limit even with explicit allocSize setting.

@revans2
Copy link
Collaborator

revans2 commented Oct 14, 2024

The other problem is that there is a race condition on start up. We check the reserve based on the memory that is currently free on the GPU. We don't check it after the pool has been initialized, so if there are multiple tasks trying to grab memory at the same time we might get into trouble with detecting reserved memory. That said some checks that are racy are better than no checks at all.

Another thing that concerns me is that I have seen the async allocator treat the pool size as a suggestion more than a hard limit. We limit the total number of bytes that can be allocated, but the async pool is not the one that is doing the limiting. If there are a lot of threads/streams then fragmentation between these sub-pools used for each stream can make it grow beyond the limit we set. This might be the cause of a similar failure we saw in a customer's query. It might be that we actually did run out of GPU memory. Not sure though.

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

No branches or pull requests

5 participants