Skip to content

Commit

Permalink
Disable profiler on python 3.11
Browse files Browse the repository at this point in the history
  • Loading branch information
fjetter committed Oct 29, 2024
1 parent 2953090 commit 1a11d3d
Show file tree
Hide file tree
Showing 8 changed files with 31 additions and 1 deletion.
5 changes: 4 additions & 1 deletion distributed/core.py
Original file line number Diff line number Diff line change
Expand Up @@ -312,7 +312,10 @@ def __init__(
self.io_loop = self.loop = IOLoop.current()

if not hasattr(self.io_loop, "profile"):
if dask.config.get("distributed.worker.profile.enabled"):
if (
dask.config.get("distributed.worker.profile.enabled")
and sys.version_info.minor != 11
):
ref = weakref.ref(self.io_loop)

def stop() -> bool:
Expand Down
11 changes: 11 additions & 0 deletions distributed/dashboard/components/shared.py
Original file line number Diff line number Diff line change
@@ -1,6 +1,7 @@
from __future__ import annotations

import asyncio
import sys

Check warning on line 4 in distributed/dashboard/components/shared.py

View check run for this annotation

Codecov / codecov/patch

distributed/dashboard/components/shared.py#L4

Added line #L4 was not covered by tests
import weakref

import tlz as toolz
Expand Down Expand Up @@ -275,6 +276,12 @@ def select_cb(attr, old, new):
self.profile_plot.add_layout(self.subtitle, "above")
if not dask.config.get("distributed.worker.profile.enabled"):
self.subtitle.text = "Profiling is disabled."

self.select.disabled = True
self.reset_button.disabled = True
self.update_button.disabled = True
elif sys.version_info.minor == 11:
self.subtitle.text = "Profiling is disabled due to a known deadlock in CPython 3.11 that can be triggered by the profiler. See https://github.com/dask/distributed/issues/8616 for more information."

Check warning on line 284 in distributed/dashboard/components/shared.py

View check run for this annotation

Codecov / codecov/patch

distributed/dashboard/components/shared.py#L280-L284

Added lines #L280 - L284 were not covered by tests
self.select.disabled = True
self.reset_button.disabled = True
self.update_button.disabled = True
Expand Down Expand Up @@ -403,6 +410,10 @@ def ts_change(attr, old, new):
self.subtitle.text = "Profiling is disabled."
self.reset_button.disabled = True
self.update_button.disabled = True
elif sys.version_info.minor == 11:
self.subtitle.text = "Profiling is disabled due to a known deadlock in CPython 3.11 that can be triggered by the profiler. See https://github.com/dask/distributed/issues/8616 for more information."
self.reset_button.disabled = True
self.update_button.disabled = True

Check warning on line 416 in distributed/dashboard/components/shared.py

View check run for this annotation

Codecov / codecov/patch

distributed/dashboard/components/shared.py#L413-L416

Added lines #L413 - L416 were not covered by tests

@without_property_validation
@log_errors
Expand Down
4 changes: 4 additions & 0 deletions distributed/dashboard/tests/test_components.py
Original file line number Diff line number Diff line change
Expand Up @@ -4,6 +4,8 @@

pytest.importorskip("bokeh")

import sys

from bokeh.models import ColumnDataSource, Model

from distributed.dashboard.components.shared import (
Expand All @@ -21,6 +23,7 @@ def test_basic(Component):
assert isinstance(c.root, Model)


@pytest.mark.skipif(sys.version_info.minor == 11, reason="Profiler disabled")
@gen_cluster(
client=True,
clean_kwargs={"threads": False},
Expand All @@ -34,6 +37,7 @@ async def test_profile_plot(c, s, a, b):
p.update(a.profile_recent)


@pytest.mark.skipif(sys.version_info.minor == 11, reason="Profiler disabled")
@gen_cluster(
client=True,
clean_kwargs={"threads": False},
Expand Down
1 change: 1 addition & 0 deletions distributed/dashboard/tests/test_scheduler_bokeh.py
Original file line number Diff line number Diff line change
Expand Up @@ -1010,6 +1010,7 @@ async def test_TaskGroupGraph_arrows(c, s, a, b):
assert not any(tgg.arrows_source.data.values())


@pytest.mark.skipif(sys.version_info.minor == 11, reason="Profiler disabled")
@gen_cluster(
client=True,
config={
Expand Down
2 changes: 2 additions & 0 deletions distributed/tests/test_actor.py
Original file line number Diff line number Diff line change
Expand Up @@ -2,6 +2,7 @@

import asyncio
import operator
import sys
from time import sleep

import pytest
Expand Down Expand Up @@ -526,6 +527,7 @@ def check(dask_worker):
assert time() < start + 30


@pytest.mark.skipif(sys.version_info.minor == 11, reason="Profiler disabled")
@gen_cluster(
client=True,
nthreads=[("127.0.0.1", 1)],
Expand Down
2 changes: 2 additions & 0 deletions distributed/tests/test_client.py
Original file line number Diff line number Diff line change
Expand Up @@ -5521,6 +5521,7 @@ async def test_call_stack_collections_all(c, s, a, b):
assert result


@pytest.mark.skipif(sys.version_info.minor == 11, reason="Profiler disabled")
@pytest.mark.flaky(condition=WINDOWS, reruns=10, reruns_delay=5)
@gen_cluster(
client=True,
Expand Down Expand Up @@ -6374,6 +6375,7 @@ async def test_futures_of_sorted(c, s, a, b):
assert [fut.key for fut in futures] == [k for k in b.__dask_keys__()]


@pytest.mark.skipif(sys.version_info.minor == 11, reason="Profiler disabled")
@gen_cluster(
client=True,
config={
Expand Down
4 changes: 4 additions & 0 deletions distributed/tests/test_scheduler.py
Original file line number Diff line number Diff line change
Expand Up @@ -1973,6 +1973,7 @@ async def test_profile_metadata(c, s, a, b):
assert not meta["counts"][-1][1]


@pytest.mark.skipif(sys.version_info.minor == 11, reason="Profiler disabled")
@gen_cluster(
client=True,
config={
Expand Down Expand Up @@ -2000,6 +2001,7 @@ def raise_timeout(*args, **kwargs):
assert not meta["counts"][-1][1]


@pytest.mark.skipif(sys.version_info.minor == 11, reason="Profiler disabled")
@gen_cluster(
client=True,
config={
Expand All @@ -2019,6 +2021,7 @@ async def test_profile_metadata_keys(c, s, a, b):
)


@pytest.mark.skipif(sys.version_info.minor == 11, reason="Profiler disabled")
@gen_cluster(
client=True,
config={
Expand All @@ -2036,6 +2039,7 @@ async def test_statistical_profiling(c, s, a, b):
assert profile["count"]


@pytest.mark.skipif(sys.version_info.minor == 11, reason="Profiler disabled")
@gen_cluster(
client=True,
config={
Expand Down
3 changes: 3 additions & 0 deletions distributed/tests/test_worker.py
Original file line number Diff line number Diff line change
Expand Up @@ -1161,6 +1161,7 @@ async def test_scheduler_delay(c, s, a, b):
assert a.scheduler_delay != old


@pytest.mark.skipif(sys.version_info.minor == 11, reason="Profiler disabled")
@pytest.mark.flaky(reruns=10, reruns_delay=5)
@gen_cluster(
client=True,
Expand All @@ -1176,6 +1177,7 @@ async def test_statistical_profiling(c, s, a, b):
assert profile["count"]


@pytest.mark.skipif(sys.version_info.minor == 11, reason="Profiler disabled")
@pytest.mark.slow
@nodebug
@gen_cluster(
Expand All @@ -1202,6 +1204,7 @@ async def test_statistical_profiling_2(c, s, a, b):
break


@pytest.mark.skipif(sys.version_info.minor == 11, reason="Profiler disabled")
@gen_cluster(
client=True,
config={
Expand Down

0 comments on commit 1a11d3d

Please sign in to comment.