Skip to content

Commit 6d85a85

Browse files
Add config option to disable profiling and disable it in many tests per default (#6490)
1 parent 69b798d commit 6d85a85

12 files changed

+205
-36
lines changed

distributed/core.py

Lines changed: 16 additions & 13 deletions
Original file line numberDiff line numberDiff line change
@@ -10,7 +10,7 @@
1010
import uuid
1111
import warnings
1212
import weakref
13-
from collections import defaultdict
13+
from collections import defaultdict, deque
1414
from collections.abc import Container
1515
from contextlib import suppress
1616
from enum import Enum
@@ -195,18 +195,21 @@ def __init__(
195195
self.loop = self.io_loop
196196

197197
if not hasattr(self.io_loop, "profile"):
198-
ref = weakref.ref(self.io_loop)
199-
200-
def stop() -> bool:
201-
loop = ref()
202-
return loop is None or loop.asyncio_loop.is_closed()
203-
204-
self.io_loop.profile = profile.watch(
205-
omit=("profile.py", "selectors.py"),
206-
interval=dask.config.get("distributed.worker.profile.interval"),
207-
cycle=dask.config.get("distributed.worker.profile.cycle"),
208-
stop=stop,
209-
)
198+
if dask.config.get("distributed.worker.profile.enabled"):
199+
ref = weakref.ref(self.io_loop)
200+
201+
def stop() -> bool:
202+
loop = ref()
203+
return loop is None or loop.asyncio_loop.is_closed()
204+
205+
self.io_loop.profile = profile.watch(
206+
omit=("profile.py", "selectors.py"),
207+
interval=dask.config.get("distributed.worker.profile.interval"),
208+
cycle=dask.config.get("distributed.worker.profile.cycle"),
209+
stop=stop,
210+
)
211+
else:
212+
self.io_loop.profile = deque()
210213

211214
# Statistics counters for various events
212215
with suppress(ImportError):

distributed/dashboard/components/shared.py

Lines changed: 16 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -14,6 +14,7 @@
1414
NumeralTickFormatter,
1515
Range1d,
1616
Select,
17+
Title,
1718
)
1819
from bokeh.palettes import Spectral9
1920
from bokeh.plotting import figure
@@ -193,7 +194,6 @@ def __init__(self, server, doc=None, **kwargs):
193194
data = profile.plot_data(self.state, profile_interval)
194195
self.states = data.pop("states")
195196
self.profile_plot, self.source = profile.plot_figure(data, **kwargs)
196-
197197
changing = [False] # avoid repeated changes from within callback
198198

199199
@without_property_validation
@@ -270,6 +270,14 @@ def select_cb(attr, old, new):
270270
**kwargs,
271271
)
272272

273+
self.subtitle = Title(text=" ", text_font_style="italic")
274+
self.profile_plot.add_layout(self.subtitle, "above")
275+
if not dask.config.get("distributed.worker.profile.enabled"):
276+
self.subtitle.text = "Profiling is disabled."
277+
self.select.disabled = True
278+
self.reset_button.disabled = True
279+
self.update_button.disabled = True
280+
273281
@without_property_validation
274282
@log_errors
275283
def update(self, state, metadata=None):
@@ -388,6 +396,13 @@ def ts_change(attr, old, new):
388396
**kwargs,
389397
)
390398

399+
self.subtitle = Title(text=" ", text_font_style="italic")
400+
self.profile_plot.add_layout(self.subtitle, "above")
401+
if not dask.config.get("distributed.worker.profile.enabled"):
402+
self.subtitle.text = "Profiling is disabled."
403+
self.reset_button.disabled = True
404+
self.update_button.disabled = True
405+
391406
@without_property_validation
392407
@log_errors
393408
def update(self, state):

distributed/dashboard/tests/test_components.py

Lines changed: 43 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -1,5 +1,3 @@
1-
import asyncio
2-
31
import pytest
42

53
pytest.importorskip("bokeh")
@@ -21,7 +19,11 @@ def test_basic(Component):
2119
assert isinstance(c.root, Model)
2220

2321

24-
@gen_cluster(client=True, clean_kwargs={"threads": False})
22+
@gen_cluster(
23+
client=True,
24+
clean_kwargs={"threads": False},
25+
config={"distributed.worker.profile.enabled": True},
26+
)
2527
async def test_profile_plot(c, s, a, b):
2628
p = ProfilePlot()
2729
assert not p.source.data["left"]
@@ -30,20 +32,54 @@ async def test_profile_plot(c, s, a, b):
3032
p.update(a.profile_recent)
3133

3234

33-
@gen_cluster(client=True, clean_kwargs={"threads": False})
35+
@gen_cluster(
36+
client=True,
37+
clean_kwargs={"threads": False},
38+
config={
39+
"distributed.worker.profile.enabled": True,
40+
"distributed.worker.profile.interval": "10ms",
41+
"distributed.worker.profile.cycle": "50ms",
42+
},
43+
)
3444
async def test_profile_time_plot(c, s, a, b):
3545
from bokeh.io import curdoc
3646

3747
sp = ProfileTimePlot(s, doc=curdoc())
48+
assert "disabled" not in sp.subtitle.text
3849
sp.trigger_update()
3950

4051
ap = ProfileTimePlot(a, doc=curdoc())
52+
assert "disabled" not in sp.subtitle.text
4153
ap.trigger_update()
4254

43-
assert not len(sp.source.data["left"])
44-
assert not len(ap.source.data["left"])
55+
assert len(sp.source.data["left"]) == 0
56+
assert len(ap.source.data["left"]) == 0
4557

4658
await c.gather(c.map(slowinc, range(10), delay=0.05))
59+
4760
ap.trigger_update()
4861
sp.trigger_update()
49-
await asyncio.sleep(0.05)
62+
63+
64+
@gen_cluster(
65+
client=True,
66+
clean_kwargs={"threads": False},
67+
config={
68+
"distributed.worker.profile.enabled": False,
69+
"distributed.worker.profile.interval": "10ms",
70+
"distributed.worker.profile.cycle": "50ms",
71+
},
72+
)
73+
async def test_profile_time_plot_disabled(c, s, a, b):
74+
from bokeh.io import curdoc
75+
76+
sp = ProfileTimePlot(s, doc=curdoc())
77+
assert "disabled" in sp.subtitle.text
78+
sp.trigger_update()
79+
80+
ap = ProfileTimePlot(a, doc=curdoc())
81+
assert "disabled" in sp.subtitle.text
82+
ap.trigger_update()
83+
84+
assert len(sp.source.data["left"]) == 0
85+
assert len(ap.source.data["left"]) == 0

distributed/dashboard/tests/test_scheduler_bokeh.py

Lines changed: 20 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -788,12 +788,14 @@ async def test_TaskGroupGraph_arrows(c, s, a, b):
788788
@gen_cluster(
789789
client=True,
790790
config={
791+
"distributed.worker.profile.enabled": True,
791792
"distributed.worker.profile.interval": "10ms",
792793
"distributed.worker.profile.cycle": "50ms",
793794
},
794795
)
795796
async def test_profile_server(c, s, a, b):
796797
ptp = ProfileServer(s)
798+
assert "disabled" not in ptp.subtitle.text
797799
start = time()
798800
await asyncio.sleep(0.100)
799801
while len(ptp.ts_source.data["time"]) < 2:
@@ -802,6 +804,24 @@ async def test_profile_server(c, s, a, b):
802804
assert time() < start + 2
803805

804806

807+
@pytest.mark.slow
808+
@gen_cluster(
809+
client=True,
810+
config={
811+
"distributed.worker.profile.enabled": False,
812+
"distributed.worker.profile.interval": "5ms",
813+
"distributed.worker.profile.cycle": "10ms",
814+
},
815+
)
816+
async def test_profile_server_disabled(c, s, a, b):
817+
ptp = ProfileServer(s)
818+
assert "disabled" in ptp.subtitle.text
819+
start = time()
820+
await asyncio.sleep(0.1)
821+
ptp.trigger_update()
822+
assert len(ptp.ts_source.data["time"]) == 0
823+
824+
805825
@gen_cluster(client=True, scheduler_kwargs={"dashboard": True})
806826
async def test_root_redirect(c, s, a, b):
807827
http_client = AsyncHTTPClient()

distributed/distributed-schema.yaml

Lines changed: 4 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -405,6 +405,10 @@ properties:
405405
This data gets collected into statistical profiling information,
406406
which is then periodically bundled together and sent along to the scheduler.
407407
properties:
408+
enabled:
409+
type: boolean
410+
description: |
411+
Whether or not to enable profiling
408412
interval:
409413
type: string
410414
description: |

distributed/distributed.yaml

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -91,6 +91,7 @@ distributed:
9191
restart: False # Do we ressurrect the worker after the lifetime deadline?
9292

9393
profile:
94+
enabled: True # Whether or not to enable profiling
9495
interval: 10ms # Time between statistical profiling queries
9596
cycle: 1000ms # Time between starting new profile
9697
low-level: False # Whether or not to include low-level functions

distributed/tests/test_actor.py

Lines changed: 4 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -530,7 +530,10 @@ def check(dask_worker):
530530
@gen_cluster(
531531
client=True,
532532
nthreads=[("127.0.0.1", 1)],
533-
config={"distributed.worker.profile.interval": "1ms"},
533+
config={
534+
"distributed.worker.profile.enabled": True,
535+
"distributed.worker.profile.interval": "1ms",
536+
},
534537
)
535538
async def test_actors_in_profile(c, s, a):
536539
class Sleeper:

distributed/tests/test_client.py

Lines changed: 62 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -5422,7 +5422,13 @@ async def test_call_stack_collections_all(c, s, a, b):
54225422

54235423

54245424
@pytest.mark.flaky(condition=WINDOWS, reruns=10, reruns_delay=5)
5425-
@gen_cluster(client=True, worker_kwargs={"profile_cycle_interval": "100ms"})
5425+
@gen_cluster(
5426+
client=True,
5427+
config={
5428+
"distributed.worker.profile.enabled": True,
5429+
"distributed.worker.profile.cycle": "100ms",
5430+
},
5431+
)
54265432
async def test_profile(c, s, a, b):
54275433
futures = c.map(slowinc, range(10), delay=0.05, workers=a.address)
54285434
await wait(futures)
@@ -5444,7 +5450,33 @@ async def test_profile(c, s, a, b):
54445450
assert not result["count"]
54455451

54465452

5447-
@gen_cluster(client=True, worker_kwargs={"profile_cycle_interval": "100ms"})
5453+
@gen_cluster(
5454+
client=True,
5455+
config={
5456+
"distributed.worker.profile.enabled": False,
5457+
"distributed.worker.profile.cycle": "100ms",
5458+
},
5459+
)
5460+
async def test_profile_disabled(c, s, a, b):
5461+
futures = c.map(slowinc, range(10), delay=0.05, workers=a.address)
5462+
await wait(futures)
5463+
5464+
x = await c.profile(start=time() + 10, stop=time() + 20)
5465+
assert x["count"] == 0
5466+
5467+
x = await c.profile(start=0, stop=time())
5468+
assert x["count"] == 0
5469+
5470+
y = await c.profile(start=time() - 0.300, stop=time())
5471+
assert 0 == y["count"] == x["count"]
5472+
5473+
5474+
@gen_cluster(
5475+
client=True,
5476+
config={
5477+
"distributed.worker.profile.cycle": "100ms",
5478+
},
5479+
)
54485480
async def test_profile_keys(c, s, a, b):
54495481
x = c.map(slowinc, range(10), delay=0.05, workers=a.address)
54505482
y = c.map(slowdec, range(10), delay=0.05, workers=a.address)
@@ -6169,7 +6201,13 @@ async def test_futures_of_sorted(c, s, a, b):
61696201

61706202

61716203
@pytest.mark.flaky(reruns=10, reruns_delay=5)
6172-
@gen_cluster(client=True, worker_kwargs={"profile_cycle_interval": "10ms"})
6204+
@gen_cluster(
6205+
client=True,
6206+
config={
6207+
"distributed.worker.profile.enabled": True,
6208+
"distributed.worker.profile.cycle": "10ms",
6209+
},
6210+
)
61736211
async def test_profile_server(c, s, a, b):
61746212
for i in range(5):
61756213
try:
@@ -6193,6 +6231,27 @@ async def test_profile_server(c, s, a, b):
61936231
break
61946232

61956233

6234+
@gen_cluster(
6235+
client=True,
6236+
config={
6237+
"distributed.worker.profile.enabled": False,
6238+
"distributed.worker.profile.cycle": "10ms",
6239+
},
6240+
)
6241+
async def test_profile_server_disabled(c, s, a, b):
6242+
x = c.map(slowinc, range(10), delay=0.01, workers=a.address, pure=False)
6243+
await wait(x)
6244+
await asyncio.gather(
6245+
c.run(slowinc, 1, delay=0.5), c.run_on_scheduler(slowdec, 1, delay=0.5)
6246+
)
6247+
6248+
p = await c.profile(server=True) # All worker servers
6249+
assert "slowinc" not in str(p)
6250+
6251+
p = await c.profile(scheduler=True) # Scheduler
6252+
assert "slowdec" not in str(p)
6253+
6254+
61966255
@gen_cluster(client=True)
61976256
async def test_await_future(c, s, a, b):
61986257
future = c.submit(inc, 1)

distributed/tests/test_scheduler.py

Lines changed: 16 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -1300,7 +1300,13 @@ async def test_profile_metadata(c, s, a, b):
13001300
assert not meta["counts"][-1][1]
13011301

13021302

1303-
@gen_cluster(client=True, worker_kwargs={"profile_cycle_interval": "100ms"})
1303+
@gen_cluster(
1304+
client=True,
1305+
config={
1306+
"distributed.worker.profile.enabled": True,
1307+
"distributed.worker.profile.cycle": "100ms",
1308+
},
1309+
)
13041310
async def test_profile_metadata_timeout(c, s, a, b):
13051311
start = time() - 1
13061312

@@ -1321,7 +1327,13 @@ def raise_timeout(*args, **kwargs):
13211327
assert not meta["counts"][-1][1]
13221328

13231329

1324-
@gen_cluster(client=True, worker_kwargs={"profile_cycle_interval": "100ms"})
1330+
@gen_cluster(
1331+
client=True,
1332+
config={
1333+
"distributed.worker.profile.enabled": True,
1334+
"distributed.worker.profile.cycle": "100ms",
1335+
},
1336+
)
13251337
async def test_profile_metadata_keys(c, s, a, b):
13261338
x = c.map(slowinc, range(10), delay=0.05)
13271339
y = c.map(slowdec, range(10), delay=0.05)
@@ -1337,6 +1349,7 @@ async def test_profile_metadata_keys(c, s, a, b):
13371349
@gen_cluster(
13381350
client=True,
13391351
config={
1352+
"distributed.worker.profile.enabled": True,
13401353
"distributed.worker.profile.interval": "1ms",
13411354
"distributed.worker.profile.cycle": "100ms",
13421355
},
@@ -1353,6 +1366,7 @@ async def test_statistical_profiling(c, s, a, b):
13531366
@gen_cluster(
13541367
client=True,
13551368
config={
1369+
"distributed.worker.profile.enabled": True,
13561370
"distributed.worker.profile.interval": "1ms",
13571371
"distributed.worker.profile.cycle": "100ms",
13581372
},

0 commit comments

Comments
 (0)