-
Notifications
You must be signed in to change notification settings - Fork 234
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
[QST] How to configure to run multiple Concurrent Tasks on single GPU for Spark RAPIDS Jobs? #11586
Comments
I think i found the issue is due to task cpu set to 4. When i changed this from 4 to 1 then i could see the parallel tasks "spark.task.cpus": "4" to "spark.task.cpus": "1" |
Yeah
As long as there are multiple tasks on an executor (spark.executor.cores > 1 and concurrentGpuTasks > 1). Artificially changing the cpu allotment per task (spark.task.cpus) makes this math harder.
There is a lot in this question and it greatly depends in the workload. We have tools to help you decide what the configs should be, and you can always ask questions here. To get you started:
I think these question are about There are lots of configs in your original question. Here is some feedback on them: For L4 GPU we have a recommendation around 3 concurrent, but this is a guideline. I'd keep it in mind when suggesting 4 concurrent, as we may see OOM in some jobs. That said, it really depends on the queries:
We have spent some time with the number of shuffle partitions. Usually, 200 is a really good default. Note that 201+ is a different shuffle algorithm in Apache Spark. It might make sense, for small data, to have 10 partitions, especially if you have 10 spark.executor.cores total (every core has some work to do and they all finish at once).
Related to the above, set this to
If you configure this, without a "mode" you are using a multi-threaded shuffle, not the UCX shuffle (I see UCX shuffle referenced in your github repo). See this for more configs around UCX mode: https://docs.nvidia.com/spark-rapids/user-guide/latest/additional-functionality/rapids-shuffle.html#ucx-mode.
This is the default:
I don't think you have to set this, it should be auto populated by the discovery script:
Note that if you don't use UCX mode (if you use the default multi-threaded mode) you can use
s3a is already part of the
ASYNC allocator is the default, we shouldn't need to set any of this:
This is actually a conf bug. There is a config we don't actually use:
I would keep this value default unless you start seeing OOM near the scan side (parquet/orc reading). It limits the amount of memory we are going to be batching up on the read.
Don't set this unless you really want to see more metrics:
These are conflicting with each other. The first one, would log all of explain output with GPU annotations. The second one would log only the pieces of the plan that we don't put on the GPU:
Why was this enabled? I am mostly curious if you do need a stable sort (https://github.com/NVIDIA/spark-rapids/blob/4866941bbf70f93931d9e23ee8f9a65b0f7c6c01/docs/compatibility.md#ordering-of-output):
This applies to joins where we are using floating point as the key. Do you have this use case? (https://github.com/NVIDIA/spark-rapids/blob/4866941bbf70f93931d9e23ee8f9a65b0f7c6c01/docs/compatibility.md#floating-point)
This enabled operations where we are not 100% compatible with the CPU. Please go to https://nvidia.github.io/spark-rapids/docs/additional-functionality/advanced_configs.html and look for the operations where we are not 100% compatible. Do you want/need them turned on? I'd recommend leaving this set to
This should be default:
|
Hi, I would like to leverage this discussion to ask about how Spark RAPIDS runs multiple concurrent tasks per GPU. In particular, what kind of GPU space sharing technique does it use? MPS or MIG? Also, is there any way to configure how the GPU is shared directly in Spark-RAPIDS? For instance, MIG currently offers a total of 18 choices for resources partitions on the A100. Is there any spark config parameter where I can change the GPU partition? |
The way we run multiple tasks per GPU is given by the number of cores that a spark executor is given. For example, spark.executor.cores=4 means that spark is aware that each executor has 4 possible tasks in parallel, so it will divide work accordingly. This is default behavior in spark, whether or not you are using spark-rapids. We have a separate config that we use to control out of spark.executor.cores, how many of those spark tasks can grab the GPU and submit GPU work (kernels, memory operations, etc.). This config is: MPS allows multiple processes to run on the same GPU, even if the GPU is marked as exclusive process. This is a daemon service that has its own lifecycle, and you are free to use it, but it's something we don't test and is not part of our plugin. Please check out: https://docs.nvidia.com/deploy/mps/index.html for more info. On MIG, please take a look at this entry in our FAQ: https://docs.nvidia.com/spark-rapids/user-guide/latest/faq.html#is-multi-instance-gpu-mig-supported and https://github.com/NVIDIA/spark-rapids-examples/tree/main/examples/MIG-Support. Essentially, if the scheduler supports isolation it should allow a container running in such a scheduler with a MIG slice. Let us know if MIG/k8s is a requirement for you and need more info here. With either MPS or MIG and spark-rapids note that we are going to hard partition the GPU by default. We have a memory pool that we allocate at startup, so if you have 2 processes in MPS you'll have to configure the pools to use ~1/2 of the GPU, and MIG slices will see a GPU that is 1/2 the size. With the hard partitioning, one of the issues is that a task that has skew may run OOM more easily because now we have a smaller ceiling. Overall, I am curious on your use cases for MPS/MIG, so please share and we are happy to help. |
@abellina, thank you for your detailed response. I was also taking a look at the Tuning Guide and the semaphore restricts the access of a task to a GPU, making tasks run on the GPU in a round-robin fashion. Contrary to space-sharing techniques (MIG and MPS), this seems to be a time-sharing technique because only one task can be executed on a GPU at a time. My understanding is that if I have multiple tasks and each task has multiple GPU kernels, I can only run in parallel kernels from the same task in the GPU because each task will have exclusive access on the GPU per time based on the semaphore. However, does RAPIDS use concurrent streams so that the kernels of a task can be executed in parallel in the GPU? Or will the kernels of a task be executed sequentially in the GPU? I am just starting to use Spark and RAPIDS and I am curious to know how they work when running multiple tasks per GPU. I do not have any concrete use case at the moment, but a potential one would be when only one task is not enough to saturate one GPU. So instead of processing one task per GPU, it might be better (in terms of GPU utilization) to run multiple tasks on the same GPU (to saturate it as much as possible) before requesting more GPUs. This way, other GPUs in the cluster would be free to process other tasks. |
Our semaphore (https://github.com/NVIDIA/spark-rapids/blob/branch-24.12/sql-plugin/src/main/scala/com/nvidia/spark/rapids/PrioritySemaphore.scala#L22) allows multiple tasks to run on the GPU, and the number of tasks that are allowed to execute on the GPU concurrently is controlled by Following this 4 concurrent example, we will let each of the 4 tasks submit GPU work concurrently. For example, we might see all 4 tasks submitting an unsnap_kernel, or a parquet decode kernel, all submitted at the same time. The GPU will then schedule the work as lower level resources free up (shared mem, SMs, registers, etc). On your question around CUDA streams, yes we use streams. We have 1 stream per executor thread, and then cuDF (our underlying library) creates other streams as it needs them. We use something called "per-thread-default-stream" (see https://developer.nvidia.com/blog/gpu-pro-tip-cuda-7-streams-simplify-concurrency/), which means that any thread that wants to submit CUDA work gets to submit in its own stream to improve concurrency. |
@abellina thanks again for your very detailed explanation and also sharing the link for the semaphore code. So the semaphore is indeed a space sharing technique as multiple tasks are allowed to execute on the GPU concurrently. For some reason I thought MPS and MIG were the only two space sharing techniques available on NVIDIA GPUs, but there are other alternatives like your semaphore. I will keep studying the documentation and also run some tests to get familiar with these GPU concurrency features in Spark-RAPIDS. |
What is your question?
I am currently running PySpark data processing jobs to read partitioned CSV data from S3 and perform some aggregations. The same script is being run on both CPU and GPU instances using Spark RAPIDS. However, I am noticing that
Spark RAPIDS on GPU is significantly slower than CPU
, especially when loading partitioned CSV data from S3 using the s3a protocol.Issue:
When running on CPUs, I can see that all 4 cores are being fully utilized, running 4+ tasks in parallel per executor. However, when using GPUs with Spark RAPIDS, each GPU node is only running 1 task per node, resulting in underutilization and much slower performance (up to 4 times slower than CPU).
GPU Spark Rapids config
CPU PySpark Config
Despite trying multiple configurations, the GPU seems to be running only 1 task per node, while CPUs are efficiently running 4+ tasks per node.
Screenshots:
In the attached screenshots, you can see the difference between the Spark History Server for both CPU and GPU runs:
Left (CPU): Shows 5 active tasks per node.
Right (GPU): Shows only 1 active task per node.
Questions:
How can I ensure that multiple tasks can run concurrently on a single GPU, similar to how multiple tasks run on multiple CPU cores?
Is there any specific configuration I am missing that could enable better utilization of the GPU for Spark workloads?
Are there any known limitations or best practices for running Spark tasks on GPUs that might explain this behavior?
Is it recommended to use one instance with multiple GPUs instead of using 4 instances with 1 GPU each?
The text was updated successfully, but these errors were encountered: