@@ -109,7 +109,14 @@ def test_dataframe_merge_empty_partitions():
109
109
110
110
def check_partitions (df , npartitions ):
111
111
"""Check that all values in `df` hashes to the same"""
112
- hashes = partitioning_index (df , npartitions )
112
+ dtypes = {}
113
+ for col , dtype in df .dtypes .items ():
114
+ if pd .api .types .is_numeric_dtype (dtype ):
115
+ dtypes [col ] = np .float64
116
+ if not dtypes :
117
+ dtypes = None
118
+
119
+ hashes = partitioning_index (df , npartitions , cast_dtype = dtypes )
113
120
if len (hashes ) > 0 :
114
121
return len (hashes .unique ()) == 1
115
122
else :
@@ -132,7 +139,7 @@ def _test_dataframe_shuffle(backend, protocol, n_workers, _partitions):
132
139
all_workers = list (client .get_worker_logs ().keys ())
133
140
comms .default_comms ()
134
141
np .random .seed (42 )
135
- df = pd .DataFrame ({"key" : np .random .random ( 100 )})
142
+ df = pd .DataFrame ({"key" : np .random .randint ( 0 , high = 100 , size = 100 )})
136
143
if backend == "cudf" :
137
144
df = cudf .DataFrame .from_pandas (df )
138
145
@@ -141,15 +148,15 @@ def _test_dataframe_shuffle(backend, protocol, n_workers, _partitions):
141
148
142
149
for input_nparts in range (1 , 5 ):
143
150
for output_nparts in range (1 , 5 ):
144
- ddf = dd .from_pandas (df .copy (), npartitions = input_nparts ).persist (
151
+ ddf1 = dd .from_pandas (df .copy (), npartitions = input_nparts ).persist (
145
152
workers = all_workers
146
153
)
147
154
# To reduce test runtime, we change the batchsizes here instead
148
155
# of using a test parameter.
149
156
for batchsize in (- 1 , 1 , 2 ):
150
157
with dask .config .set (explicit_comms_batchsize = batchsize ):
151
158
ddf = explicit_comms_shuffle (
152
- ddf ,
159
+ ddf1 ,
153
160
["_partitions" ] if _partitions else ["key" ],
154
161
npartitions = output_nparts ,
155
162
batchsize = batchsize ,
@@ -177,6 +184,32 @@ def _test_dataframe_shuffle(backend, protocol, n_workers, _partitions):
177
184
got = ddf .compute ().sort_values ("key" )
178
185
assert_eq (got , expected )
179
186
187
+ # Check that partitioning is consistent with "tasks"
188
+ ddf_tasks = ddf1 .shuffle (
189
+ ["key" ],
190
+ npartitions = output_nparts ,
191
+ shuffle_method = "tasks" ,
192
+ )
193
+ for i in range (output_nparts ):
194
+ expected_partition = ddf_tasks .partitions [
195
+ i
196
+ ].compute ()["key" ]
197
+ actual_partition = ddf .partitions [i ].compute ()[
198
+ "key"
199
+ ]
200
+ if backend == "cudf" :
201
+ expected_partition = (
202
+ expected_partition .values_host
203
+ )
204
+ actual_partition = actual_partition .values_host
205
+ else :
206
+ expected_partition = expected_partition .values
207
+ actual_partition = actual_partition .values
208
+ assert all (
209
+ np .sort (expected_partition )
210
+ == np .sort (actual_partition )
211
+ )
212
+
180
213
181
214
@pytest .mark .parametrize ("nworkers" , [1 , 2 , 3 ])
182
215
@pytest .mark .parametrize ("backend" , ["pandas" , "cudf" ])
0 commit comments