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

restore support for yield unsafe Client context managers and deprecate that support #7987

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.
Jump to
Jump to file
Failed to load files.
Diff view
Diff view
24 changes: 22 additions & 2 deletions distributed/client.py
Expand Up @@ -1502,7 +1502,17 @@ async def __aenter__(self):

async def __aexit__(self, exc_type, exc_value, traceback):
if self._previous_as_current:
_current_client.reset(self._previous_as_current)
try:
_current_client.reset(self._previous_as_current)
except ValueError as e:
if not e.args[0].endswith(" was created in a different Context"):
raise # pragma: nocover
warnings.warn(
"It is deprecated to enter and exit the Client context "
"manager from different tasks",
Copy link
Member

Choose a reason for hiding this comment

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

Can we phrase this in a way that tells the user how to replace their anti-pattern?

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'm not sure how to phrase that, it's usually because someone has used async with Client( inside an async generator, the specific case it's likely to occur is in pytest-asyncio where async fixtures are not run in a way that can support ContextVars, TaskGroups and Timeouts

related discussion is here: https://discuss.python.org/t/preventing-yield-inside-certain-context-managers/1091

Copy link
Member

Choose a reason for hiding this comment

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

Yeah, I don't have any suggestions, either. Let's see if this pops up frequently then and add more context/a link to some Github discussion if need be.

DeprecationWarning,
stacklevel=2,
)
await self._close(
# if we're handling an exception, we assume that it's more
# important to deliver that exception than shutdown gracefully.
Expand All @@ -1512,7 +1522,17 @@ async def __aexit__(self, exc_type, exc_value, traceback):

def __exit__(self, exc_type, exc_value, traceback):
if self._previous_as_current:
_current_client.reset(self._previous_as_current)
try:
_current_client.reset(self._previous_as_current)
except ValueError as e:
if not e.args[0].endswith(" was created in a different Context"):
raise # pragma: nocover
warnings.warn(
"It is deprecated to enter and exit the Client context "
Copy link
Member

Choose a reason for hiding this comment

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

See above

"manager from different threads",
DeprecationWarning,
stacklevel=2,
)
self.close()

def __del__(self):
Expand Down
27 changes: 27 additions & 0 deletions distributed/tests/test_client.py
Expand Up @@ -1274,6 +1274,33 @@ async def client_2():
await asyncio.gather(client_1(), client_2())


@gen_cluster(client=False, nthreads=[])
async def test_context_manager_used_from_different_tasks(s):
c = Client(s.address, asynchronous=True)
await asyncio.create_task(c.__aenter__())
with pytest.warns(
DeprecationWarning,
match=r"It is deprecated to enter and exit the Client context manager "
"from different tasks",
):
await asyncio.create_task(c.__aexit__(None, None, None))


def test_context_manager_used_from_different_threads(s, loop):
c = Client(s["address"])
with (
concurrent.futures.ThreadPoolExecutor(1) as tp1,
concurrent.futures.ThreadPoolExecutor(1) as tp2,
):
tp1.submit(c.__enter__).result()
with pytest.warns(
DeprecationWarning,
match=r"It is deprecated to enter and exit the Client context manager "
"from different threads",
):
tp2.submit(c.__exit__, None, None, None).result()


def test_global_clients(loop):
assert _get_global_client() is None
with pytest.raises(
Expand Down