NVIDIA / spark-rapids

Spark RAPIDS plugin - accelerate Apache Spark with GPUs
https://nvidia.github.io/spark-rapids
Apache License 2.0
822 stars 235 forks source link

[BUG] The size of the device buffer being spilled is less than being required when OOM happens. #11742

Open firestarman opened 1 day ago

firestarman commented 1 day ago

We saw Plugin sometimes spilled less device buffer than being required in a customer query. See as below, it asked for 536558080 (~ 500MB) bytes buffer, but only 263708544 (~250MB) bytes were spilled. While the remaining spillable buffer was still big enough (3954541056 bytes, ~3.9G) for this requirement.

24/11/18 15:25:06 INFO DeviceMemoryEventHandler: Device allocation of 536558080 bytes failed, device store has 7115063488 total and 4744781312 spillable bytes. Attempt 2. Total RMM allocated is 14093593344 bytes. 
24/11/18 15:25:06 WARN RapidsDeviceMemoryStore: Targeting a device memory size of 4208223232. Current total 7115063488. Current spillable 4744781312
24/11/18 15:25:06 WARN RapidsDeviceMemoryStore: device memory store spilling to reduce usage from 7115063488 total (4744781312 spillable) to 4208223232 bytes

24/11/18 15:25:06 INFO DeviceMemoryEventHandler: Device allocation of 122912992 bytes failed, device store has 7115063488 total and 3954541056 spillable bytes. First attempt. Total RMM allocated is 14792092416 bytes.
24/11/18 15:25:07 INFO DeviceMemoryEventHandler: Spilled 263708544 bytes from the device store 

24/11/18 15:25:07 WARN RapidsDeviceMemoryStore: Targeting a device memory size of 3831628064. Current total 6851354944. Current spillable 3954541056

24/11/18 15:25:07 INFO DeviceMemoryEventHandler: Device allocation of 536558080 bytes failed, device store has 6851354944 total and 3954541056 spillable bytes. Attempt 3. Total RMM allocated is 14528383744 bytes. 
24/11/18 15:25:07 WARN RapidsDeviceMemoryStore: device memory store spilling to reduce usage from 6851354944 total (3954541056 spillable) to 3831628064 bytes 

24/11/18 15:25:08 INFO DeviceMemoryEventHandler: Spilled 263192384 bytes from the device store 

And the concurrentGpuTask was set to 2.

This will not fail the query, but it may lead to bad perf because one or more allocation retries and buffer spillings will take place to satisfy this allocation size.

abellina commented 1 day ago

I think you are seeing a race where a spillable buffer was spillable at some point, but then when we tried to spill we couldn't spill it because it was acquired, but I need to double check the old code. Note that we are rewriting the spill framework from scratch, and that also has the race. I don't believe this is a horrible race to have, but I could be wrong.

With the async allocator we will tend to free and device synchronize and the synchronize will trigger a defragment of the pool. It might be worth to retry the allocation at that point, but we don't have hard that on whether that's usually a win or a loss. In terms of the race, we are erring on the side of "don't copy things that are currently acquired, and are therefore not spillable", because they won't free any memory. If I have incRefCounted a DeviceMemoryBuffer and I copy it to "spill" it and then I close the buffer, I still have a +1 ref count, so it won't actually free anything => and I wasted time on that copy.

So I think we should get some data around this. In the new framework we are going to have all task threads independently try to spill, rather than the "1 thread spills approach", I am afraid that might overspill, but we need to gather that data.

We are going to have to study this closely soon, so I can take this and get some statistics on different strategies. That said, I am not sure if this is a bug yet, I'd want to verify that the math is right when we are spilling and that it's the spillability race.

firestarman commented 1 day ago

Free to remove the BUG in title or close this issue if it is not a bug.

Since I am not sure whether we need to fix it either. At least, it won't fail the query.