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
Regression: P2P shuffle skeleton (#5520) causes test flakiness #5688
Comments
I've identified and fixed the flaky shuffle test in #5689. The rebalance tests I'm less sure about, and less sure that shuffling is the culprit. The only change #5520 makes that I think could have any effect on other tests is that Looking at the logs you posted, I frequently see something like this in stderr for the failing rebalance tests:
https://github.com/dask/distributed/runs/4922504753?check_suite_focus=true#step:12:1338
Basically it seems that the worker from which we want to rebalance is now:
Unless the memory usage was previously so close to the limit that adding a few extra Python objects in I notice #5665 was merged right after the shuffle PR. Is it possible that's related? |
The test is responsible for 512 MiB worth of managed memory and can cope with up to 204 MiB worth of unmanaged memory before it starts misbehaving:
I would expect <50 MiB worth of unmanaged memory on a spawned process; instead here we've having 877 - 512 = 365 MiB. How much did you have before your PR? |
Reduce memory spikes during data transfer with `test_rebalance`, by using NumPy arrays (zero-copy) and more, smaller keys. Fixes dask#5688
We determined that the shuffle extension increases worker unmanaged memory by ~10MiB, because it imports pandas. It seems that this slight increase, combined with the memory spiking during the rebalance, was enough to push the worker over its spill threshold (or sometimes even limit), leading to incomplete rebalancing because some data would get spilled to disk (or the worker would restart). I've refactored the |
In dask#5688, we discovered that dask#5520 was increasing worker unmanaged memory usage by ~10MiB at startup (on my mac). I suspect that this came from importing pandas. The shuffle extension already worked if pandas wasn't available: it just wouldn't install itself on the worker. However, to check if pandas was available, it was importing pandas—so if pandas _was_ available, every worker would have to spend the time and memory to import it at startup, even if it wasn't used at all. With this PR, all use of pandas is deferred. There's also now an explicit test that importing the shuffle does not import pandas.
If this causes issues, we can consider reverting the PR |
Reopening; test_get_partition is still flaky although on a different point. https://github.com/dask/distributed/runs/4965019825?check_suite_focus=true
|
Fixed by #5706 |
Since #5520 was merged earlier today, several tests that I never observed failing before became very flaky:
FAILED distributed/shuffle/tests/test_shuffle_extension.py::test_get_partition
FAILED distributed/shuffle/tests/test_shuffle_extension.py::test_get_partition
FAILED distributed/shuffle/tests/test_shuffle_extension.py::test_get_partition
FAILED distributed/tests/test_client.py::test_rebalance - assert 0 == 10
FAILED distributed/tests/test_client.py::test_rebalance - assert 0 == 10
FAILED distributed/tests/test_scheduler.py::test_rebalance - AssertionError: ...
FAILED distributed/tests/test_scheduler.py::test_rebalance - AssertionError: ...
FAILED distributed/tests/test_tls_functional.py::test_rebalance - assert 0 == 10
logs:
https://github.com/dask/distributed/actions/runs/1739731969
https://github.com/dask/distributed/actions/runs/1740153471
https://github.com/dask/distributed/actions/runs/1740658508
CC @gjoseph92
The text was updated successfully, but these errors were encountered: