Skip to content
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
2 changes: 1 addition & 1 deletion distributed/shuffle/_arrow.py
Original file line number Diff line number Diff line change
Expand Up @@ -55,7 +55,7 @@ def convert_partition(data: bytes, meta: pd.DataFrame) -> pd.DataFrame:
while file.tell() < end:
sr = pa.RecordBatchStreamReader(file)
shards.append(sr.read_all())
table = pa.concat_tables(shards)
table = pa.concat_tables(shards, promote=True)
df = table.to_pandas(self_destruct=True)

def default_types_mapper(pyarrow_dtype: pa.DataType) -> object:
Expand Down
19 changes: 19 additions & 0 deletions distributed/shuffle/tests/test_shuffle.py
Original file line number Diff line number Diff line change
Expand Up @@ -1826,3 +1826,22 @@ async def test_closed_worker_returns_before_barrier(c, s):
await c.close()
await asyncio.gather(*[clean_worker(w) for w in workers])
await clean_scheduler(s)


@gen_cluster(client=True)
async def test_handle_null_partitions_p2p_shuffling(c, s, *workers):
data = [
{"companies": [], "id": "a", "x": None},
{"companies": [{"id": 3}, {"id": 5}], "id": "b", "x": None},
{"companies": [{"id": 3}, {"id": 4}, {"id": 5}], "id": "c", "x": "b"},
{"companies": [{"id": 9}], "id": "a", "x": "a"},
]
df = pd.DataFrame(data)
ddf = dd.from_pandas(df, npartitions=2)
ddf = ddf.shuffle(on="id", shuffle="p2p", ignore_index=True)
result = await c.compute(ddf)
dd.assert_eq(result, df)

await c.close()
await asyncio.gather(*[clean_worker(w) for w in workers])
await clean_scheduler(s)