-
Notifications
You must be signed in to change notification settings - Fork 93
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
Fix partitioning in explicit-comms shuffle #1356
Changes from all commits
51079a7
463ab8d
8bc7570
2344777
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -109,7 +109,14 @@ def test_dataframe_merge_empty_partitions(): | |
|
||
def check_partitions(df, npartitions): | ||
"""Check that all values in `df` hashes to the same""" | ||
hashes = partitioning_index(df, npartitions) | ||
dtypes = {} | ||
for col, dtype in df.dtypes.items(): | ||
if pd.api.types.is_numeric_dtype(dtype): | ||
dtypes[col] = np.float64 | ||
if not dtypes: | ||
dtypes = None | ||
|
||
hashes = partitioning_index(df, npartitions, cast_dtype=dtypes) | ||
if len(hashes) > 0: | ||
return len(hashes.unique()) == 1 | ||
else: | ||
|
@@ -128,11 +135,10 @@ def _test_dataframe_shuffle(backend, protocol, n_workers, _partitions): | |
worker_class=IncreasedCloseTimeoutNanny, | ||
processes=True, | ||
) as cluster: | ||
with Client(cluster) as client: | ||
all_workers = list(client.get_worker_logs().keys()) | ||
with Client(cluster): | ||
comms.default_comms() | ||
np.random.seed(42) | ||
df = pd.DataFrame({"key": np.random.random(100)}) | ||
df = pd.DataFrame({"key": np.random.randint(0, high=100, size=100)}) | ||
if backend == "cudf": | ||
df = cudf.DataFrame.from_pandas(df) | ||
|
||
|
@@ -141,15 +147,13 @@ def _test_dataframe_shuffle(backend, protocol, n_workers, _partitions): | |
|
||
for input_nparts in range(1, 5): | ||
for output_nparts in range(1, 5): | ||
ddf = dd.from_pandas(df.copy(), npartitions=input_nparts).persist( | ||
workers=all_workers | ||
) | ||
ddf1 = dd.from_pandas(df.copy(), npartitions=input_nparts) | ||
Comment on lines
-144
to
+150
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Strange Finding: Even without the changes in this PR, I get a "cancelled" error when I persist There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Are we ok still moving ahead with the changes even though there's a cancelation error? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Yes. The changes in this PR are independent of the cancellation error (it happens in Another important detail: I only get the error when query-planning is disabled. Therefore, I assume the problem has to do with a key-name collision that doesn't happen with dask-expr (which is much more disciplined about key names than the legacy API is). |
||
# To reduce test runtime, we change the batchsizes here instead | ||
# of using a test parameter. | ||
for batchsize in (-1, 1, 2): | ||
with dask.config.set(explicit_comms_batchsize=batchsize): | ||
ddf = explicit_comms_shuffle( | ||
ddf, | ||
ddf1, | ||
Comment on lines
-152
to
+156
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Seems a bit confusing to me that we were previously modifying the initial collection. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Makes sense to avoid that. I think @madsbk may have an idea if this was an oversight or was intentional when he gets back. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I think it was a oversight :) |
||
["_partitions"] if _partitions else ["key"], | ||
npartitions=output_nparts, | ||
batchsize=batchsize, | ||
|
@@ -177,6 +181,32 @@ def _test_dataframe_shuffle(backend, protocol, n_workers, _partitions): | |
got = ddf.compute().sort_values("key") | ||
assert_eq(got, expected) | ||
|
||
# Check that partitioning is consistent with "tasks" | ||
ddf_tasks = ddf1.shuffle( | ||
["key"], | ||
npartitions=output_nparts, | ||
shuffle_method="tasks", | ||
) | ||
for i in range(output_nparts): | ||
expected_partition = ddf_tasks.partitions[ | ||
i | ||
].compute()["key"] | ||
actual_partition = ddf.partitions[i].compute()[ | ||
"key" | ||
] | ||
if backend == "cudf": | ||
expected_partition = ( | ||
expected_partition.values_host | ||
) | ||
actual_partition = actual_partition.values_host | ||
else: | ||
expected_partition = expected_partition.values | ||
actual_partition = actual_partition.values | ||
assert all( | ||
np.sort(expected_partition) | ||
== np.sort(actual_partition) | ||
) | ||
|
||
|
||
@pytest.mark.parametrize("nworkers", [1, 2, 3]) | ||
@pytest.mark.parametrize("backend", ["pandas", "cudf"]) | ||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
@ayushdg - FYI: I think this means the ordering of partitions could have been "wrong" even before the dtype-casting change in dask :/
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Ah I see. Based on the limited testing I've done the biggest change in results on my tests were from #1323 , even when paired with an older version of Dask prior to the dtype change.