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

Explicit-comms-shuffle: fine control of task scheduling #1025

Merged
merged 5 commits into from
Oct 28, 2022
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
41 changes: 26 additions & 15 deletions dask_cuda/explicit_comms/dataframe/shuffle.py
Original file line number Diff line number Diff line change
Expand Up @@ -11,7 +11,6 @@

import dask
import dask.dataframe
import distributed
from dask.base import compute_as_if_collection, tokenize
from dask.dataframe.core import DataFrame, _concat as dd_concat, new_dd_object
from dask.dataframe.shuffle import shuffle_group
Expand Down Expand Up @@ -325,25 +324,37 @@ def shuffle(
rank_to_out_part_ids,
ignore_index,
)
distributed.wait(list(result_futures.values()))
del df_groups
wait(list(result_futures.values()))

# Step (c): extract individual dataframe-partitions
# Release dataframes from step (a)
for fut in df_groups:
fut.release()

# Step (c): extract individual dataframe-partitions. We use `submit()`
# to control where the tasks are executed.
# TODO: can we do this without using `submit()` to avoid the overhead
# of creating a Future for each dataframe partition?
name = f"explicit-comms-shuffle-getitem-{tokenize(name)}"
dsk = {}
meta = None
for rank, parts in rank_to_out_part_ids.items():
for i, part_id in enumerate(parts):
dsk[(name, part_id)] = (getitem, result_futures[rank], i)
if meta is None:
# Get the meta from the first output partition
meta = delayed(make_meta)(
delayed(getitem)(result_futures[rank], i)
).compute()
assert meta is not None
for rank, worker in enumerate(c.worker_addresses):
if rank in workers:
for i, part_id in enumerate(rank_to_out_part_ids[rank]):
dsk[(name, part_id)] = c.client.submit(
getitem, result_futures[rank], i, workers=[worker]
)
wence- marked this conversation as resolved.
Show resolved Hide resolved

# Get the meta from the first output partition
meta = delayed(make_meta)(next(iter(dsk.values()))).compute()

# Create a distributed Dataframe from all the pieces
divs = [None] * (len(dsk) + 1)
return new_dd_object(dsk, name, meta, divs).persist()
ret = new_dd_object(dsk, name, meta, divs).persist()
wait(ret)

# Release all temporary dataframes
for fut in [*result_futures.values(), *dsk.values()]:
fut.release()
return ret


def get_rearrange_by_column_tasks_wrapper(func):
Expand Down
5 changes: 3 additions & 2 deletions docs/source/conf.py
Original file line number Diff line number Diff line change
Expand Up @@ -189,7 +189,8 @@
# -- Extension configuration -------------------------------------------------



def setup(app):
app.add_css_file("https://docs.rapids.ai/assets/css/custom.css")
app.add_js_file("https://docs.rapids.ai/assets/js/custom.js", loading_method="defer")
app.add_js_file(
"https://docs.rapids.ai/assets/js/custom.js", loading_method="defer"
)