async def test_add_partition(s: Scheduler, *workers: Worker):
    exts: dict[str, ShuffleWorkerExtension] = {
        w.address: w.extensions["shuffle"]
        for w in workers
    }

    new_metadata = NewShuffleMetadata(
        ShuffleId("foo"),
        pd.DataFrame({
            "A": [],
            "partition": []
        }),
        "partition",
        8,
    )

    ext = next(iter(exts.values()))
    metadata = await ext._create_shuffle(new_metadata)
    partition = pd.DataFrame({
        "A": ["a", "b", "c", "d", "e", "f", "g", "h"],
        "partition": [0, 1, 2, 3, 4, 5, 6, 7],
    })
    await ext._add_partition(partition, new_metadata.id)

    with pytest.raises(ValueError, match="not registered"):
        await ext._add_partition(partition, ShuffleId("bar"))

    for i, data in partition.groupby(new_metadata.column):
        addr = metadata.worker_for(int(i))
        ext = exts[addr]
        received = ext.shuffles[metadata.id].output_partitions[int(i)]
        assert len(received) == 1
        dd.utils.assert_eq(data, received[0])
async def test_create(s: Scheduler, *workers: Worker):
    exts: list[ShuffleWorkerExtension] = [
        w.extensions["shuffle"] for w in workers
    ]

    new_metadata = NewShuffleMetadata(
        ShuffleId("foo"),
        pd.DataFrame({"A": []}),
        "A",
        5,
    )

    metadata = await exts[0]._create_shuffle(new_metadata)
    assert sorted(metadata.workers) == sorted(w.address for w in workers)

    # Check shuffle was created on all workers
    for ext in exts:
        assert len(ext.shuffles) == 1
        shuffle = ext.shuffles[new_metadata.id]
        assert shuffle.metadata.workers == metadata.workers

    # TODO (resilience stage) what happens if some workers already have
    # the ID registered, but others don't?

    with pytest.raises(ValueError, match="already registered"):
        await exts[0]._create_shuffle(new_metadata)
async def test_get_partition(c: Client, s: Scheduler, *workers: Worker):
    exts: dict[str, ShuffleWorkerExtension] = {
        w.address: w.extensions["shuffle"]
        for w in workers
    }

    new_metadata = NewShuffleMetadata(
        ShuffleId("foo"),
        pd.DataFrame({
            "A": [],
            "partition": []
        }),
        "partition",
        8,
    )
    _ = await add_dummy_unpack_keys(new_metadata, c)

    ext = next(iter(exts.values()))
    metadata = await ext._create_shuffle(new_metadata)
    p1 = pd.DataFrame({
        "A": ["a", "b", "c", "d", "e", "f", "g", "h"],
        "partition": [0, 1, 2, 3, 4, 5, 6, 6],
    })
    p2 = pd.DataFrame({
        "A": ["a", "b", "c", "d", "e", "f", "g", "h"],
        "partition": [0, 1, 2, 3, 0, 0, 2, 3],
    })
    await asyncio.gather(ext._add_partition(p1, metadata.id),
                         ext._add_partition(p2, metadata.id))
    await ext._barrier(metadata.id)

    for addr, ext in exts.items():
        if metadata.worker_for(0) != addr:
            with pytest.raises(AssertionError, match="belongs on"):
                ext.get_output_partition(metadata.id, 0)

    full = pd.concat([p1, p2])
    expected_groups = full.groupby("partition")
    for output_i in range(metadata.npartitions):
        addr = metadata.worker_for(output_i)
        ext = exts[addr]
        result = ext.get_output_partition(metadata.id, output_i)
        try:
            expected = expected_groups.get_group(output_i)
        except KeyError:
            expected = metadata.empty
        dd.utils.assert_eq(expected, result)
        # ^ NOTE: use `assert_eq` instead of `pd.testing.assert_frame_equal` directly
        # to ignore order of the rows (`assert_eq` pre-sorts its inputs).

    # Once all partitions are retrieved, shuffles are cleaned up
    for ext in exts.values():
        assert not ext.shuffles
        with pytest.raises(ValueError, match="not registered"):
            ext.get_output_partition(metadata.id, 0)
def test_worker_for_distribution(npartitions: int, n_workers: int):
    "Test that `worker_for` distributes evenly"
    metadata = ShuffleMetadata(
        ShuffleId("foo"),
        pd.DataFrame({"A": []}),
        "A",
        npartitions,
        list(string.ascii_lowercase[:n_workers]),
    )

    with pytest.raises(AssertionError, match="Negative"):
        metadata.worker_for(-1)

    assignments = [metadata.worker_for(i) for i in range(metadata.npartitions)]

    # Test internal `_partition_range` method
    for w in metadata.workers:
        first, last = metadata._partition_range(w)
        assert all([
            first <= p_i <= last if a == w else p_i < first or p_i > last
            for p_i, a in enumerate(assignments)
        ])

    counter = Counter(assignments)
    assert len(counter) == min(npartitions, n_workers)

    # Test `npartitions_for`
    calculated_counter = {
        w: metadata.npartitions_for(w)
        for w in metadata.workers
    }
    assert counter == {
        w: count
        for w, count in calculated_counter.items() if count != 0
    }
    assert calculated_counter.keys() == set(metadata.workers)
    # ^ this also checks that workers receiving 0 output partitions were calculated properly

    # Test the distribution of worker assignments.
    # All workers should be assigned the same number of partitions, or if
    # there's an odd number, some workers will be assigned only one extra partition.
    counts = set(counter.values())
    assert len(counts) <= 2
    if len(counts) == 2:
        lo, hi = sorted(counts)
        assert lo == hi - 1

    with pytest.raises(IndexError, match="does not exist"):
        metadata.worker_for(npartitions)
async def test_init(s: Scheduler, worker: Worker):
    ext: ShuffleWorkerExtension = worker.extensions["shuffle"]
    assert not ext.shuffles
    metadata = ShuffleMetadata(
        ShuffleId("foo"),
        pd.DataFrame({"A": []}),
        "A",
        5,
        [worker.address],
    )

    ext.shuffle_init(None, metadata)
    assert list(ext.shuffles) == [metadata.id]

    with pytest.raises(ValueError, match="already registered"):
        ext.shuffle_init(None, metadata)

    assert list(ext.shuffles) == [metadata.id]
async def test_barrier(c: Client, s: Scheduler, *workers: Worker):
    exts: dict[str, ShuffleWorkerExtension] = {
        w.address: w.extensions["shuffle"]
        for w in workers
    }

    new_metadata = NewShuffleMetadata(
        ShuffleId("foo"),
        pd.DataFrame({
            "A": [],
            "partition": []
        }),
        "partition",
        4,
    )
    fs = await add_dummy_unpack_keys(new_metadata, c)

    ext = next(iter(exts.values()))
    metadata = await ext._create_shuffle(new_metadata)
    partition = pd.DataFrame({
        "A": ["a", "b", "c"],
        "partition": [0, 1, 2],
    })
    await ext._add_partition(partition, metadata.id)

    await ext._barrier(metadata.id)

    # Check scheduler restrictions were set for unpack tasks
    for i, key in enumerate(fs):
        assert s.tasks[key].worker_restrictions == {metadata.worker_for(i)}

    # Check all workers have been informed of the barrier
    for addr, ext in exts.items():
        if metadata.npartitions_for(addr):
            shuffle = ext.shuffles[metadata.id]
            assert shuffle.transferred
            assert not shuffle.done()
        else:
            # No output partitions on this worker; shuffle already cleaned up
            assert not ext.shuffles
Beispiel #7
0
def rearrange_by_column_p2p(
    df: DataFrame,
    column: str,
    npartitions: int | None = None,
):
    from dask.dataframe import DataFrame

    npartitions = npartitions or df.npartitions
    token = tokenize(df, column, npartitions)

    setup = delayed(shuffle_setup, pure=True)(NewShuffleMetadata(
        ShuffleId(token),
        df._meta,
        column,
        npartitions,
    ))

    transferred = df.map_partitions(
        shuffle_transfer,
        token,
        setup,
        meta=df,
        enforce_metadata=False,
        transform_divisions=False,
    )

    barrier_key = "shuffle-barrier-" + token
    barrier_dsk = {
        barrier_key: (shuffle_barrier, token, transferred.__dask_keys__())
    }
    barrier = Delayed(
        barrier_key,
        HighLevelGraph.from_collections(barrier_key,
                                        barrier_dsk,
                                        dependencies=[transferred]),
    )

    name = "shuffle-unpack-" + token
    dsk = {(name, i): (shuffle_unpack, token, i, barrier_key)
           for i in range(npartitions)}
    # TODO: update to use blockwise.
    # Changes task names, so breaks setting worker restrictions at the moment.
    # Also maybe would be nice if the `DataFrameIOLayer` interface supported this?
    # dsk = blockwise(
    #     shuffle_unpack,
    #     name,
    #     "i",
    #     token,
    #     None,
    #     BlockwiseDepDict({(i,): i for i in range(npartitions)}),
    #     "i",
    #     barrier_key,
    #     None,
    #     numblocks={},
    # )

    return DataFrame(
        HighLevelGraph.from_collections(name, dsk, [barrier]),
        name,
        df._meta,
        [None] * (npartitions + 1),
    )