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

Dask-cudf multi partition merge slows down with ucx #402

Closed
VibhuJawa opened this issue Jan 29, 2020 · 123 comments
Closed

Dask-cudf multi partition merge slows down with ucx #402

VibhuJawa opened this issue Jan 29, 2020 · 123 comments

Comments

@VibhuJawa
Copy link
Member

VibhuJawa commented Jan 29, 2020

Dask-cudf multi partition merge slows down with ucx .

Dask-cudf merge seems to slow down with ucx .

Wall time: (15.4 seconds on tcp) vs (37.8 s on ucx) (exp-01)

In the attached example we see a slow down with ucx vs just using tcp .

Wall Times on exp-01

UCX Time

CPU times: user 19.3 s, sys: 1.97 s, total: 21.2 s
Wall time: 38.4 s
2945293


CPU times: user 16.7 s, sys: 1.71 s, total: 18.4 s
Wall time: 37.8 s
2943379

TCP times

CPU times: user 10.8 s, sys: 815 ms, total: 11.6 s
Wall time: 15.7 s
2944022

CPU times: user 10.9 s, sys: 807 ms, total: 11.7 s
Wall time: 15.4 s
2943697 

Repro Code:

Helper Function to create distributed dask-cudf frame

import dask_cudf
import cudf
import os
import time
import dask.dataframe as dd
import dask.array as da

from dask_cuda import LocalCUDACluster
from dask.distributed import Client,wait
from dask.utils import parse_bytes

def create_random_data(n_rows=1_000,n_parts = 10, n_keys_index_1=100_000,n_keys_index_2=100,n_keys_index_3=100, col_prefix = 'a'):
    
    chunks = n_rows//n_parts

    df = dd.concat([
        da.random.random(n_rows, chunks = chunks).to_dask_dataframe(columns= col_prefix + '_non_merge_1'),
        da.random.random(n_rows, chunks = chunks).to_dask_dataframe(columns= col_prefix + '_non_merge_2'),
        da.random.random(n_rows, chunks = chunks).to_dask_dataframe(columns= col_prefix + '_non_merge_3'),
        da.random.randint(0, n_keys_index_1, size=n_rows,chunks = chunks ).to_dask_dataframe(columns= col_prefix + '_0'),
        da.random.randint(0, n_keys_index_2, size=n_rows, chunks = chunks ).to_dask_dataframe(columns= col_prefix +'_1'),
        da.random.randint(0, n_keys_index_3, size=n_rows, chunks = chunks ).to_dask_dataframe(columns= col_prefix +'_2'),
        
    ], axis=1).persist()
    
    gdf = df.map_partitions(cudf.from_pandas)
    gdf =  gdf.persist()
    _ = wait(gdf)
    return gdf

RMM Setup:

def setup_rmm_pool(client):
    client.run(
        cudf.set_allocator,
        pool=True,
        initial_pool_size= parse_bytes("26GB"),
        allocator="default"
    )
    return None

setup_rmm_pool(client)

Merge Code:

The slow down happens on the merge step.

rows_1, parts_1 = 140_176_770, 245
rows_2, parts_2 = 21_004_393, 171

df_1 = create_random_data(n_rows= rows_1, n_parts = parts_1, col_prefix = 'a')
df_2 = create_random_data(n_rows= rows_2,  n_parts = parts_2, col_prefix = 'b')

merged_df = df_1.merge(df_2, left_on = ['a_0','a_1','a_2'], right_on = ['b_0','b_1','b_2'])
%time len(merged_df)

Additional Context:

There has been discussion about this on our internal slack channel, please see for more context.

@quasiben
Copy link
Member

quasiben commented Jan 29, 2020

Adding the dask reports for ucx/tcp:

The notable issue in the task graph is that for TCP there is more communication but the workers are fully engaged (no white space):

Screen Shot 2020-01-29 at 1 48 16 PM

For UCX, there is less communication workers are not actively working on a particular task (white space):

Screen Shot 2020-01-29 at 1 47 36 PM

@jakirkham
Copy link
Member

Thanks Vibhu for filing and Ben for including the profiling plot.

It looks like the read time on the worker is taking ~1.5x longer in the UCX case than the TCP case. This may be helped by releasing the GIL during these operations. ( #391 )

We also are spending a lot of time in generating the __cuda_array_interface__ in Numba. Not sure why that is the case. Will look at this more closely.

@beckernick
Copy link
Member

beckernick commented Jan 29, 2020

We are also spending more time working in the UCX report (3-4x the time). I'm going to be raising a similar issue with a reproducible example for groupby-aggregation behavior, today

@jakirkham
Copy link
Member

Ah was looking at the gaps initially based on Ben's comment. Should I be looking at something else?

@beckernick
Copy link
Member

Ah was looking at the gaps initially based on Ben's comment. Should I be looking at something else?

I just wanted to highlight the actual time spent doing compute is materially different, which may or may not related but also naively feels like a potential contributor to the slowdown.

@jakirkham
Copy link
Member

Meaning the overall runtime? If so, then I think Ben has the right idea to look at gaps. If not, some clarity on the compute time you are looking at would be helpful 🙂

@beckernick
Copy link
Member

beckernick commented Jan 29, 2020

Yes, sorry for being unclear 😄

From the worker profile in the linked performance report, we spend 130 total seconds doing "compute" with UCX on across the 16 GPUs. We only spend 40 total seconds with TCP across the 16 GPUs.

  • 62 seconds doing _concat with UCX, vs 13 seconds with TCP
  • 19 vs 9 for shuffle_group

@quasiben
Copy link
Member

quasiben commented Jan 29, 2020

Another interesting thing to note is that deletion of dependencies takes quite a bit of time in UCX compared with TCP. This can be seen in the Worker Profile (administrative) page:

UCX
Screen Shot 2020-01-29 at 3 17 07 PM

TCP
Screen Shot 2020-01-29 at 3 17 25 PM

@jakirkham
Copy link
Member

No worries. Thanks Nick! Just wanted to make sure we are focusing on the pain point 😉

Yeah that's what I was looking at above, Ben. Agree this is what we should focus on.

@quasiben
Copy link
Member

It want to note that these issue could be a symptom of poor work stealing and we are still digging into this idea

@jakirkham
Copy link
Member

That may be true.

There is also some cost in building Numba arrays, which we are doing. I've done some work to speed this up. Am still trying to figure out the right way to hook this into what we are doing here.

In particular please see this rough benchmark.

@pentschev
Copy link
Member

pentschev commented Jan 29, 2020

Here are some timings I gathered with this code:

TCP
Create time: 10.193811178207397
Merge time: 15.630461931228638
Create time: 10.504480838775635
Merge time: 15.304280996322632
Create time: 10.684969425201416
Merge time: 16.014200448989868

UCX (Master):
Create time: 10.919841766357422
Merge time: 22.28807020187378
Create time: 11.757182836532593
Merge time: 23.014573097229004
Create time: 11.073307514190674
Merge time: 22.2984356880188

UCX (v1.7.0):
Create time: 22.65068030357361
Merge time: 45.892472982406616
Create time: 21.901089429855347
Merge time: 44.46129822731018
Create time: 22.363646984100342
Merge time: 44.13629865646362

UCX (v1.7.0 + https://github.com/openucx/ucx/pull/4646):
Create time: 10.750166177749634
Merge time: 23.12590527534485
Create time: 11.505124568939209
Merge time: 22.653675079345703
Create time: 11.365516662597656
Merge time: 20.664494276046753

As we can see above, openucx/ucx#4646 helps (and if cherry-picked to v1.7.0, matches performance of master) but TCP still outperforms UCX.

@quasiben and I explored the __delitem__ that was mentioned before, and just for a rough test we ended up commenting it out just to see if that would affect performance, but it turns out that it doesn't change performance at all.

@quasiben
Copy link
Member

In looking more thoroughly through the worker profile there seems to be a fair amount of time managing cuda contexts when using UCX:

Screen Shot 2020-01-29 at 9 52 54 PM

@kkraus14 do you happen to have any insights here ?

@jakirkham
Copy link
Member

Yeah that's what I was trying to get at with this comment.

Am still trying to find the right way to hook Dask into higher level RMM/dask-cuda. PR ( rapidsai/rmm#264 ) is a start at that.

@jakirkham
Copy link
Member

Also after a fairly long rabbit hole earlier today, I think rmm.auto_device may be causing some of the problems here. Basically it might be copying data unnecessarily ( rapidsai/rmm#265 ). It's mostly unused except for this one deserialization line in cuDF when using strings (probably affects us here). Keith and I are trying to remove that currently ( rapidsai/cudf#4003 ).

@quasiben
Copy link
Member

thanks @jakirkham for looking into this

@jakirkham
Copy link
Member

Have added PR ( rapidsai/rmm#268 ) to RMM. This should make it easier to directly copy from a device pointer to a host buffer without going through a bunch of Numba machinery first. Am hoping this will be useful in cuDF and dask-cuda where we can benefit from removing some of this overhead.

@quasiben
Copy link
Member

I setup two dask-cuda-workers manually and ran with nvprof to bette understand what was happening in cuda land:

==11522== Profiling application: /datasets/bzaitlen/miniconda3/envs/rapidsai-latest/bin/python /datasets/bzaitlen/miniconda3/envs/rapidsai-latest/bin/dask-cuda-worker ucx://10.33.227.163:8786 --enable-nvlink --enable-tcp-over-ucx
==11522== Profiling result:
No kernels were profiled.
            Type  Time(%)      Time     Calls       Avg       Min       Max  Name
      API calls:   98.50%  324.77ms         1  324.77ms  324.77ms  324.77ms  cuDevicePrimaryCtxRetain
                    0.90%  2.9561ms        97  30.474us     138ns  1.2527ms  cuDeviceGetAttribute
                    0.47%  1.5458ms         1  1.5458ms  1.5458ms  1.5458ms  cuDeviceTotalMem
                    0.08%  274.87us         2  137.43us  128.12us  146.75us  cuDeviceGetName
                    0.05%  162.31us         1  162.31us  162.31us  162.31us  cuMemGetInfo
                    0.00%  4.4480us         6     741ns     182ns  1.7090us  cuDeviceGetCount
                    0.00%  3.9780us         1  3.9780us  3.9780us  3.9780us  cuDeviceGetPCIBusId
                    0.00%  2.3980us         1  2.3980us  2.3980us  2.3980us  cuCtxPushCurrent
                    0.00%  1.7690us         1  1.7690us  1.7690us  1.7690us  cuInit
                    0.00%  1.7600us         4     440ns     150ns     799ns  cudaGetDeviceCount
                    0.00%  1.6120us         3     537ns     480ns     611ns  cuDeviceGet
                    0.00%  1.0650us         1  1.0650us  1.0650us  1.0650us  cuDriverGetVersion
                    0.00%     951ns         1     951ns     951ns     951ns  cuCtxGetCurrent
                    0.00%     742ns         1     742ns     742ns     742ns  cuDeviceComputeCapability
                    0.00%     251ns         1     251ns     251ns     251ns  cuDeviceGetUuid

We are spending a lot of time with cuDevicePrimaryCtxRetain this is happening within numba and we can see it in the worker profile as well:

Screen Shot 2020-01-30 at 8 00 56 PM

I believe this is known to both @jakirkham and @kkraus14

@jakirkham
Copy link
Member

jakirkham commented Jan 31, 2020

Thanks for doing that Ben! 😄

This coming from Numba makes sense. Though it's nice to have the additional detail provided here.

Just to make sure we are not missing anything, do we know of any other situations (outside of Numba array creation) where cuDevicePrimaryCtxRetain would be called?

Edit: Should add we are hoping PR ( rapidsai/rmm#268 ) paves a short path for us to dispense with this overhead.

@quasiben
Copy link
Member

I checked cudf/rmm/ucx and I didn't see cuDevicePrimaryCtxRetain anywhere except numba. It's worth noting that at this point we already should have a context created from the initialization step in dask-cuda. I also want to point out that the code lower in the stack comes from these two locations:

https://github.com/rapidsai/rmm/blob/branch-0.12/python/rmm/rmm.py#L142-L164

which calls:

https://github.com/rapidsai/rmm/blob/2e309eb37d2928db16ba5413fd25c9636e20c80f/python/rmm/rmm.py#L31

@jakirkham
Copy link
Member

FWIW I tried adding __cuda_array_interface__ to Buffer and coercing Buffer objects to arrays as needed with PR ( rapidsai/cudf#4023 ). This may help and is probably worth trying (assuming CI passes though can fix that tomorrow). It's probably a useful change for building other things in cuDF in the future (feedback on this is welcome).

That said, I wouldn't be surprised if we still see a fair amount of time spent in Numba due to legacy RMM allocation methods. So am planning on pushing on PR ( rapidsai/rmm#268 ) more tomorrow in the hopes of quickly integrating this into cuDF and dask-cuda for further testing.

@jakirkham
Copy link
Member

jakirkham commented Jan 31, 2020

One thing that confuses me about the flame graph above (and hopefully someone can help answer this 🙂), it appears that creating an rmm.DeviceBuffer is taking a lot of time. However these are generally very fast to construct. In fact it's much faster than anything else that we might use to allocate device memory (please see benchmark below). Given this, why does it appear that line is taking a long time? 🤔

In [1]: import rmm                                                              

In [2]: rmm.reinitialize(pool_allocator=True, 
   ...:                  initial_pool_size=int(2 * 2**30))                      
Out[2]: 0

In [3]: %timeit rmm.DeviceBuffer(size=50_000_000)                               
360 ns ± 1.47 ns per loop (mean ± std. dev. of 7 runs, 1000000 loops each)

In [4]: import cupy                                                             

In [5]: %timeit cupy.empty((50_000_000,), dtype="u1")                           
2.15 µs ± 412 ns per loop (mean ± std. dev. of 7 runs, 1000000 loops each)

In [6]: import numba.cuda                                                       

In [7]: %timeit numba.cuda.device_array((50_000_000,), dtype="u1")              
1.39 ms ± 2.38 µs per loop (mean ± std. dev. of 7 runs, 1000 loops each)

@jakirkham
Copy link
Member

Of course if the pool were not actually enabled, that would present a problem.

In [1]: import rmm                                                              

In [2]: rmm.reinitialize(pool_allocator=False)                                  
Out[2]: 0

In [3]: %timeit rmm.DeviceBuffer(size=50_000_000)                               
1.12 ms ± 145 µs per loop (mean ± std. dev. of 7 runs, 1 loop each)

@pentschev
Copy link
Member

Note that profiling dask-cuda-worker with nvprof will only gather information from the parent process. Internally, dask-cuda-worker uses Nanny, and that will fork/spawn new processes which are not catched by nvprof. Also note that the output from #402 (comment) doesn't have any calls to cudaMalloc nor kernels launches, which hints that that process is not doing any useful CUDA work. Finally, as already mentioned, there's only a single call to cuDevicePrimaryCtxRetain, but we would be more interested to see a number of calls to the same function during runtime, with a probable accumulated time in the order of seconds that could be causing a slowdown.

@pentschev
Copy link
Member

As for rmm.DeviceBuffer, I was able yesterday to confirm the same numbers from Ben's flame graph. Apart from that I can confirm with 110% certainty that we do have the RMM pool enabled, there are two ways I can confirm:

  1. If you watch the memory usage at runtime you can see the initial pool size gets allocated for each GPU at startup and never grows beyond that;
  2. When you disable the pool, the flame graph reports the following:
rmm.DeviceBuffer
- TCP: 332s
- UCX: 287s

@kkraus14
Copy link

@quasiben also mentioned to me that we had ~81k calls to rmm.DeviceBuffer in that flamechart. I believe the pool allocator of rmm (cnmem) is known to have performance issues with freeing memory once there's a certain number of allocations being managed. As a quick test it may be worth testing what happens when you create 10/100k allocations, store them somewhere, and then see how long it takes to delete the references.

@kkraus14
Copy link

kkraus14 commented Jan 31, 2020

Wrote a quick test:

import rmm
import time

rmm.reinitialize(pool_allocator=True, initial_pool_size=int(2**34))

num_allocations = 10000
allocations = [None] * num_allocations

start = time.time()
for i in range(num_allocations):
    allocations[i] = rmm.DeviceBuffer(size=100)
end = time.time()
time_result = (end - start) * 1000

print(f"Time taken for allocating {num_allocations} buffers: {time_result}ms")


start = time.time()
for i in range(num_allocations):
    allocations[i] = None
end = time.time()
time_result = (end - start) * 1000

print(f"Time taken for freeing {num_allocations} buffers: {time_result}ms")

Results:

Time taken for allocating 100000 buffers: 44.32559013366699ms
Time taken for freeing 100000 buffers: 22098.9887714386ms

Time taken for allocating 50000 buffers: 23.27561378479004ms
Time taken for freeing 50000 buffers: 5765.538692474365ms

Time taken for allocating 25000 buffers: 11.168956756591797ms
Time taken for freeing 25000 buffers: 1489.1653060913086ms

Time taken for allocating 10000 buffers: 5.175113677978516ms
Time taken for freeing 10000 buffers: 360.734224319458ms

@jakirkham
Copy link
Member

Note that profiling dask-cuda-worker with nvprof will only gather information from the parent process. Internally, dask-cuda-worker uses Nanny, and that will fork/spawn new processes which are not catched by nvprof. Also note that the output from #402 (comment) doesn't have any calls to cudaMalloc nor kernels launches, which hints that that process is not doing any useful CUDA work. Finally, as already mentioned, there's only a single call to cuDevicePrimaryCtxRetain, but we would be more interested to see a number of calls to the same function during runtime, with a probable accumulated time in the order of seconds that could be causing a slowdown.

IDK if this would help, but one could add a sitecustomize.py in the site-packages directory of the Conda environment and add custom Python code that will be run by each interpreter at startup. This could allow us to gather and dump a bunch of profiling statistics from all interpreters in some common directory. Though I don't know if one can configure nvprof to inspect an existing process. If so, there is probably some ugly ctypes code we could write to enable profiling that process.

Barring that one might rename the pythonX.Y executable to something like pythonX.Y-orig and add a shell script that starts the Python interpreter with nvprof. Would probably also need to fix all symlinks (like python, pythonX) to point at this script. Not sure how well this would work with forking, but it could be another way to profile the cluster of processes.

@jakirkham
Copy link
Member

Adding some plots below to hopefully give more context. This is for one worker, but other workers look similar.

The first plot shows a histogram of the number of allocations for a particular number of bytes. The second plot shows how many allocations are alive over in "time steps" (when an allocation occurs).

Allocations (#) vs  Size (bytes)

Alive Allocations (#) vs  Time (Operations)

@kkraus14
Copy link

This looks like we're keep the number of allocations to a very reasonable amount.

@harrism
Copy link
Member

harrism commented Feb 13, 2020

Any chance of seeing where on that timeline you see the slowdown?

@jakirkham
Copy link
Member

My guess (though will try to back this up with data) is it lines up with where we see the very large spikes. The first half is mostly loading data and the second half is performing the merge. This is also where all of the data transfer is happening.

@jakirkham
Copy link
Member

There are workers that see a few larger allocations, but this seems to be a rare occurrence.

Allocations (#) vs  Size (bytes) 2

Alive Allocations (#) vs  Time (Operations) 2

@jakirkham
Copy link
Member

Looking over all of these the mean is ~0.5MB, the standard deviation ~1MB, the median is ~0.25MB, and the max ~5MB. The standard deviation and mean seem to be pretty large based on these infrequent, large allocations.

@harrism
Copy link
Member

harrism commented Feb 14, 2020

Is this example representative of important workloads? I am surprised that the largest allocation in a big data application is 5MB. What is the peak memory usage in bytes?

@harrism
Copy link
Member

harrism commented Feb 14, 2020

For the last histogram, I'd like to see a more detailed breakdown of that bottom bin. Is it mostly <1KB allocations like in the first histogram? Or are they closer to 500KB?

@pentschev
Copy link
Member

Is this example representative of important workloads? I am surprised that the largest allocation in a big data application is 5MB.

It is representative of this particular problem, but granted this is on a small scale dataset. I'm hoping we'll see larger allocations for larger problems. However, there's a bunch of small allocations regardless that happen probably in CUB when it's doing some DtoH<->HtoD transfers, probably transferring some 32 bit counters or something similar.

@pentschev
Copy link
Member

@jakirkham you asked for reports from Azure, here they are:

TCP
UCX + NVLink
UCX + NVLink + InfiniBand (without nv_peer_mem)

@jakirkham
Copy link
Member

It looks like UCX + NVLink + InfiniBand is a bit faster, but is experiencing the memory allocation problem much worse than UCX + NVLink. It seems memory allocation is taking roughly twice as much time in the former than the latter. Though I would expect both perform the same number of allocations. Why should one take so much longer? 🤔

@kkraus14
Copy link

It looks like UCX + NVLink + InfiniBand is a bit faster, but is experiencing the memory allocation problem much worse than UCX + NVLink. It seems memory allocation is taking roughly twice as much time in the former than the latter. Though I would expect both perform the same number of allocations. Why should one take so much longer? 🤔

At this point I think we should start getting some NSight profiles and looking at them to see where we're spending the time. If we need to add additional nvtx ranges into cuDF / RMM we can.

@Akshay-Venkatesh
Copy link
Contributor

Akshay-Venkatesh commented Feb 16, 2020 via email

@harrism
Copy link
Member

harrism commented Feb 16, 2020

Which slowdown? The overall reason for this issue? Or the slowdown of UCX + NVLink + IB vs. UCX + NVLink?

@pentschev
Copy link
Member

I think there are some confusions, let me attempt to clarify:

  1. The overall slowdown for this issue refers to UCX being generally slower (independent of transport) than TCP via Python sockets -- this remains an issue that is possibly on the Python side but still unconfirmed and thus we have people working on many fronts;
  2. The patch Akshay is referring improves InfiniBand specifically which is important for us, but not the main topic of this issues.

Finally, there has been a general trend assuming a large part of slowdowns comes from memory allocation. While that may be true, I think we need to be extra careful on interpreting our profile reports as all of them are limited in not including all the information we need (i.e., we don't have any profile that includes C++, CUDA and Python information all at once). For instance, I think one such assumptions is coming from the fact that rmm.DeviceBuffer appears as one of most time-consuming operations, and that is a call coming from distributed.protocol.ucx.read which hints it's not only the allocation that takes place, but the copy as well, and the copy may include additional overhead, such as (but not limited to) cuIpcOpenMemHandle if that's the first time a worker is reading another worker's memory pool data.

While I'm not saying the above is actually happening, we need to be careful to identify exactly what's going on, since we don't have the full picture yet, we may be looking at the wrong piece of the puzzle.

@Akshay-Venkatesh
Copy link
Contributor

I think there are some confusions, let me attempt to clarify:

  1. The overall slowdown for this issue refers to UCX being generally slower (independent of transport) than TCP via Python sockets -- this remains an issue that is possibly on the Python side but still unconfirmed and thus we have people working on many fronts;
  2. The patch Akshay is referring improves InfiniBand specifically which is important for us, but not the main topic of this issues.

Finally, there has been a general trend assuming a large part of slowdowns comes from memory allocation. While that may be true, I think we need to be extra careful on interpreting our profile reports as all of them are limited in not including all the information we need (i.e., we don't have any profile that includes C++, CUDA and Python information all at once). For instance, I think one such assumptions is coming from the fact that rmm.DeviceBuffer appears as one of most time-consuming operations, and that is a call coming from distributed.protocol.ucx.read which hints it's not only the allocation that takes place, but the copy as well, and the copy may include additional overhead, such as (but not limited to) cuIpcOpenMemHandle if that's the first time a worker is reading another worker's memory pool data.

While I'm not saying the above is actually happening, we need to be careful to identify exactly what's going on, since we don't have the full picture yet, we may be looking at the wrong piece of the puzzle.

Thanks much for the summary, @pentschev

@jakirkham
Copy link
Member

jakirkham commented Feb 21, 2020

Have rerun using the same setup as before.

One small tweak was to bump up the RMM pool by about 2GB (so 28GB total).

The main difference here is that TCP now avoids pickling the data as it uses the "dask" serializers, which are basically the same as the "ucx" serializers. The "dask" serializers incur a penalty due to moving the data to/from host memory before/after transmission by TCP (UCX should avoid this data movement). Though this penalty on TCP was already implicitly present before due to pickling (which did additional work after moving the data to host).


Here are the overall runtimes extracted from the profile plots:

Machine Protocol Duration (s)
DGX-1 TCP 61.68
DGX-1 UCX 71.33
DGX-2 TCP 29.94
DGX-2 UCX 29.81

Here are the profiles I got for the DGX-1 and DGX-2 using TCP and UCX:

DGX-1 TCP Profile

DGX-1 UCX Profile

DGX-2 TCP Profile

DGX-2 UCX Profile

@VibhuJawa
Copy link
Member Author

One small tweak was to bump up the RMM pool by about 2GB (so 28GB total).

Machine Protocol Duration (s)
DGX-1 TCP 61.68
DGX-1 UCX 71.33
DGX-2 TCP 29.94
DGX-2 UCX 29.81

From the issue description this seems like a slow down for tcp no as when i tested this on ?
See timing in the issue which states tcp used to take 15.7 s on dgx-2.

@jakirkham
Copy link
Member

It's worth noting those numbers come from the Dask Performance Reports themselves not %time. So am not exactly sure how they relate.

@VibhuJawa
Copy link
Member Author

VibhuJawa commented Feb 21, 2020

So the previous reports did include the creation time, updated times for the merge are:

DGX-2 TCP : 14.7 s

DGX-2 UCX : 18.6 s

TCP Profile

UCX Profile

With setting UCX_RNDV_THRESH=8192 the time is brought down to 14.1 s.

UCX Profile 8192

@jakirkham
Copy link
Member

So there may be issues from my build or my environment generally. It would be good to get others to replicate this and see what they get. However this is what I'm seeing using the suballocator work on a DGX-1. This uses UCX_RNDV_THRESH=8192.

Machine Protocol Duration (s)
DGX-1 TCP 32
DGX-1 UCX 37.5

TCP Profile

UCX Profile

@VibhuJawa
Copy link
Member Author

So there may be issues from my build or my environment generally. It would be good to get others to replicate this and see what they get. However this is what I'm seeing using the suballocator work on a DGX-1. This uses UCX_RNDV_THRESH=8192.

Machine Protocol Duration (s)
DGX-1 TCP 32
DGX-1 UCX 37.5
[TCP Profile]

So the timings here include both creation and merging, can you try to profile just the merge and creation separately, as the sub-allocator was supposed to help memory allocations that were happening during merge and provides context for that .

@jakirkham
Copy link
Member

I'm going to investigate and retest today. There still seem to be some weird things here that I've seen in previous runs (despite all new packages). Maybe it's time to purge Miniconda and reinstall everything from scratch?

@pentschev
Copy link
Member

I checked how things are looking currently, and below are the results I got with RAPIDS 0.20 and UCX 1.9 on a DGX-2:

TCP:

%time len(merged_df)
CPU times: user 8.22 s, sys: 576 ms, total: 8.8 s
Wall time: 35.2 s

%time len(merged_df)
CPU times: user 8.26 s, sys: 924 ms, total: 9.18 s
Wall time: 35.3 s

UCX + NVLink:

%time len(merged_df)
CPU times: user 8.04 s, sys: 975 ms, total: 9.01 s
Wall time: 21 s

%time len(merged_df)
CPU times: user 7.86 s, sys: 857 ms, total: 8.72 s
Wall time: 20.4 s

It seems like there has been a regression in performance with TCP, and UCX is outperforming TCP now (as we would expect). From a UCX-Py side, I think the original issue here has been resolved over the past year, do you think we can close this @VibhuJawa ? It may be worth investigating the potential regression in this merge workflow, but at this point I don't think there's anything to be done on the UCX front.

@VibhuJawa
Copy link
Member Author

I checked how things are looking currently, and below are the results I got with RAPIDS 0.20 and UCX 1.9 on a DGX-2:

TCP:

%time len(merged_df)
CPU times: user 8.22 s, sys: 576 ms, total: 8.8 s
Wall time: 35.2 s

%time len(merged_df)
CPU times: user 8.26 s, sys: 924 ms, total: 9.18 s
Wall time: 35.3 s

UCX + NVLink:

%time len(merged_df)
CPU times: user 8.04 s, sys: 975 ms, total: 9.01 s
Wall time: 21 s

%time len(merged_df)
CPU times: user 7.86 s, sys: 857 ms, total: 8.72 s
Wall time: 20.4 s

It seems like there has been a regression in performance with TCP, and UCX is outperforming TCP now (as we would expect). From a UCX-Py side, I think the original issue here has been resolved over the past year, do you think we can close this @VibhuJawa ? It may be worth investigating the potential regression in this merge workflow, but at this point I don't think there's anything to be done on the UCX front.

Thanks for following up on this Peter. Agree that we should close this on UCX-PY as there is no action item left on that front.

Might be worth exploring the TCP slow down though as the 2x slow don seems concerning (CC: @randerzander / @beckernick ) .

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

No branches or pull requests

10 participants