Skip to content
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

Ensure client_desires_keys does not corrupt Scheduler state #8827

Merged
merged 10 commits into from
Aug 20, 2024

Conversation

fjetter
Copy link
Member

@fjetter fjetter commented Aug 12, 2024

I ran into this over in dask/dask#11248 where I was somehow triggering this condition in a very different way.

The test test_futures_of_cancelled_raises is actually broken on main. It is indeed raising a CancelledError but not for the reason we'd like it to. What happens under the hood is that this behavior is triggering an AssertionError during transitioning which will close the network connection to the scheduler and raises a FutureCancelledError but not a "This task has been cancelled" CancelledError 🙄

The reason for this is that client_desires_keys instantiates a new TaskState object if the key is unknown. This is a pretty breaking behavior in general but so far has been required to make Variable, Queue, etc. work the way they do. Variables and the like are communicating via an unordered RPC to the scheduler causing the key often to not be registered yet and this premature initialization just worked because typically the state corruption would only last for a brief moment.

However, the case that cannot be corrected easily (and something I'd like to fix but is more work than I currently care to invest) is that Future objects are _inform_ing the scheduler about their existence whenever they are instantiated. This is important to allow the submission of persisted collections (e.g. used by publish/get_dataset) or when storing/recovering stored futures in a variable.
This mechanism is typically disabled on ordinary clusters because there is no client in the context of the scheduler. However, when running in async mode, the async test client is shared with the scheduler and therefore the future in the scheduler context is also calling client_desires_keys even though it was already released...

Alas, this is only a small patch but it should make things more reliable. Eventually this should be cleaned up...

closes #7498

Copy link
Contributor

github-actions bot commented Aug 12, 2024

Unit Test Results

See test report for an extended history of previous test failures. This is useful for diagnosing flaky tests.

    25 files  ± 0      25 suites  ±0   10h 15m 23s ⏱️ + 8m 13s
 4 101 tests  -  4   3 978 ✅  -  9    113 💤 ±0  10 ❌ +5 
47 390 runs   - 48  45 248 ✅  - 56  2 132 💤 +3  10 ❌ +5 

For more details on these failures, see this check.

Results for commit fc085a7. ± Comparison against base commit f12cc4f.

This pull request removes 8 and adds 4 tests. Note that renamed tests count towards both.
distributed.tests.test_client ‑ test_future_auto_inform
distributed.tests.test_client ‑ test_future_defaults_to_default_client
distributed.tests.test_client ‑ test_rebalance_raises_on_explicit_missing_data
distributed.tests.test_client ‑ test_serialize_future
distributed.tests.test_client ‑ test_serialize_future_without_client
distributed.tests.test_scheduler ‑ test_client_desires_keys_creates_ts
distributed.tests.test_spans ‑ test_client_desires_keys_creates_tg
distributed.tests.test_spans ‑ test_client_desires_keys_creates_ts
distributed.tests.test_client ‑ test_worker_clients_do_not_claim_ownership_of_serialize_futures[False]
distributed.tests.test_client ‑ test_worker_clients_do_not_claim_ownership_of_serialize_futures[True]
distributed.tests.test_queues ‑ test_set_cancelled_future
distributed.tests.test_variable ‑ test_set_cancelled_future

♻️ This comment has been updated with latest results.

@fjetter
Copy link
Member Author

fjetter commented Aug 12, 2024

FAILED distributed/tests/test_client.py::test_futures_of_cancelled_raises - AssertionError: Regex pattern did not match.
Regex: '(reason: unknown|testreason)'
Input: 'inc-2ac8f30d27eb33df3f76277392063a38 cancelled for reason: scheduler-connection-lost.\nClient lost the connection to the scheduler. Please check your connection and re-run your work.'

aha... will have to keep digging, it seems

@fjetter
Copy link
Member Author

fjetter commented Aug 12, 2024

ok, the failure I was writing about above was due to the warning being raised. We're raising on all warnings.

I also found a couple of tests that explicitly implemented the behavior I am now forbidding. I ended up deleting those tests, especially since we're now discouraging users from instantiating Futures themselves

@fjetter fjetter force-pushed the client_desires_must_not_instantiate_task branch from 8c77c11 to 90a39e6 Compare August 15, 2024 12:43
distributed/client.py Outdated Show resolved Hide resolved
@fjetter fjetter self-assigned this Aug 15, 2024
@hendrikmakait hendrikmakait self-requested a review August 16, 2024 09:11
Copy link
Member

@hendrikmakait hendrikmakait left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks, @fjetter. Overall these changes make sense to me, I'm not sure if I have a full understanding of the subtleties of the change but CI looks good, so that's enough for me.

distributed/queues.py Outdated Show resolved Hide resolved
distributed/tests/test_queues.py Outdated Show resolved Hide resolved
distributed/variable.py Outdated Show resolved Hide resolved

@pytest.mark.slow()
@pytest.mark.parametrize("do_wait", [True, False])
def test_worker_clients_do_not_claim_ownership_of_serialize_futures(c, do_wait):
Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think this test describes the subtleties involved in this change. The cases that raise a CancelledError could previously still work, depending on timing. If the futures would unpack on the worker before the client side release reached the scheduler, the futures would still be referenced.
In an async test, this would rather trivially be true since the scheduler also deserializes the future and given that the test client and scheduler run in the same thread, the get_client discovery would detect the client even while inside of the scheduler which is the "bug/feature" I had to get rid of here.

#7498 describes this race condition in detail

@fjetter fjetter merged commit fe79a36 into dask:main Aug 20, 2024
22 of 32 checks passed
@fjetter fjetter deleted the client_desires_must_not_instantiate_task branch August 20, 2024 11:29
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
2 participants