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

Preserve contextvars during comm offload #5486

Merged
merged 2 commits into from
Nov 15, 2021
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
13 changes: 13 additions & 0 deletions distributed/tests/test_utils.py
Original file line number Diff line number Diff line change
@@ -1,5 +1,6 @@
import array
import asyncio
import contextvars
import functools
import io
import os
Expand Down Expand Up @@ -554,6 +555,18 @@ async def test_offload():
assert (await offload(lambda x, y: x + y, 1, y=2)) == 3


@pytest.mark.asyncio
async def test_offload_preserves_contextvars():
gjoseph92 marked this conversation as resolved.
Show resolved Hide resolved
var = contextvars.ContextVar("var")

async def set_var(v: str):
var.set(v)
r = await offload(var.get)
assert r == v

await asyncio.gather(set_var("foo"), set_var("bar"))


def test_serialize_for_cli_deprecated():
with pytest.warns(FutureWarning, match="serialize_for_cli is deprecated"):
from distributed.utils import serialize_for_cli
Expand Down
7 changes: 6 additions & 1 deletion distributed/utils.py
Original file line number Diff line number Diff line change
@@ -1,6 +1,7 @@
from __future__ import annotations

import asyncio
import contextvars
import functools
import importlib
import inspect
Expand Down Expand Up @@ -1322,7 +1323,11 @@ def import_term(name: str):

async def offload(fn, *args, **kwargs):
loop = asyncio.get_event_loop()
return await loop.run_in_executor(_offload_executor, lambda: fn(*args, **kwargs))
# Retain context vars while deserializing; see https://bugs.python.org/issue34014
context = contextvars.copy_context()
return await loop.run_in_executor(
_offload_executor, lambda: context.run(fn, *args, **kwargs)
)


class EmptyContext:
Expand Down