Skip to content

Commit

Permalink
Weigh gilknocker Prometheus metric by duration (#8558)
Browse files Browse the repository at this point in the history
  • Loading branch information
crusaderky committed Mar 8, 2024
1 parent 91350ab commit de166f6
Show file tree
Hide file tree
Showing 8 changed files with 28 additions and 22 deletions.
7 changes: 4 additions & 3 deletions distributed/dashboard/components/scheduler.py
Original file line number Diff line number Diff line change
Expand Up @@ -3911,19 +3911,20 @@ def __init__(self, scheduler, **kwargs):
@log_errors
def update(self):
s = self.scheduler
monitor_gil = s.monitor.monitor_gil_contention

self.data["values"] = [
s._tick_interval_observed,
self.gil_contention_scheduler,
sum(w.metrics["event_loop_interval"] for w in s.workers.values())
/ (len(s.workers) or 1),
self.gil_contention_workers,
][:: 1 if monitor_gil else 2]
][:: 1 if s.monitor.monitor_gil_contention else 2]

# Format event loop as time and GIL (if configured) as %
self.data["text"] = [
f"{x * 100:.1f}%" if i % 2 and monitor_gil else format_time(x)
f"{x * 100:.1f}%"
if i % 2 and s.monitor.monitor_gil_contention
else format_time(x)
for i, x in enumerate(self.data["values"])
]
update(self.source, self.data)
Expand Down
3 changes: 2 additions & 1 deletion distributed/http/scheduler/prometheus/core.py
Original file line number Diff line number Diff line change
Expand Up @@ -56,7 +56,8 @@ def collect(self) -> Iterator[GaugeMetricFamily | CounterMetricFamily]:
yield CounterMetricFamily(
self.build_name("gil_contention"),
"GIL contention metric",
value=self.server.monitor._cumulative_gil_contention,
value=self.server.monitor.cumulative_gil_contention,
unit="seconds",
)

yield CounterMetricFamily(
Expand Down
2 changes: 1 addition & 1 deletion distributed/http/scheduler/tests/test_scheduler_http.py
Original file line number Diff line number Diff line change
Expand Up @@ -132,7 +132,7 @@ async def test_prometheus(c, s, a, b):
except ImportError:
pass # pragma: nocover
else:
expected_metrics.add("dask_scheduler_gil_contention")
expected_metrics.add("dask_scheduler_gil_contention_seconds")

assert set(active_metrics.keys()) == expected_metrics
assert active_metrics["dask_scheduler_clients"].samples[0].value == 1.0
Expand Down
4 changes: 2 additions & 2 deletions distributed/http/tests/test_core.py
Original file line number Diff line number Diff line change
Expand Up @@ -73,8 +73,8 @@ async def test_prometheus_api_doc(c, s, a, _):
gil_metrics = set() # Already in worker_metrics
except ImportError:
gil_metrics = {
"dask_scheduler_gil_contention_total",
"dask_worker_gil_contention_total",
"dask_scheduler_gil_contention_seconds_total",
"dask_worker_gil_contention_seconds_total",
}

implemented = scheduler_metrics | worker_metrics | crick_metrics | gil_metrics
Expand Down
3 changes: 2 additions & 1 deletion distributed/http/worker/prometheus/core.py
Original file line number Diff line number Diff line change
Expand Up @@ -63,7 +63,8 @@ def collect(self) -> Iterator[Metric]:
yield CounterMetricFamily(
self.build_name("gil_contention"),
"GIL contention metric",
value=self.server.monitor._cumulative_gil_contention,
value=self.server.monitor.cumulative_gil_contention,
unit="seconds",
)

yield GaugeMetricFamily(
Expand Down
2 changes: 1 addition & 1 deletion distributed/http/worker/tests/test_worker_http.py
Original file line number Diff line number Diff line change
Expand Up @@ -59,7 +59,7 @@ async def test_prometheus(c, s, a):
except ImportError:
pass # pragma: nocover
else:
expected_metrics.add("dask_worker_gil_contention_total")
expected_metrics.add("dask_worker_gil_contention_seconds_total")

try:
import crick # noqa: F401
Expand Down
13 changes: 6 additions & 7 deletions distributed/system_monitor.py
Original file line number Diff line number Diff line change
Expand Up @@ -31,7 +31,7 @@ class SystemMonitor:
_last_host_cpu_counters: Any # dynamically-defined psutil namedtuple
_last_gil_contention: float # 0-1 value

_cumulative_gil_contention: float
cumulative_gil_contention: float

gpu_name: str | None
gpu_memory_total: int
Expand Down Expand Up @@ -114,12 +114,11 @@ def __init__(
self.monitor_gil_contention = False
else:
self.quantities["gil_contention"] = deque(maxlen=maxlen)
self._cumulative_gil_contention = 0.0
self.cumulative_gil_contention = 0.0
raw_interval = dask.config.get(
"distributed.admin.system-monitor.gil.interval",
)
interval = parse_timedelta(raw_interval, default="us") * 1e6

interval = parse_timedelta(raw_interval) * 1e6
self._gilknocker = KnockKnock(polling_interval_micros=int(interval))
self._gilknocker.start()

Expand Down Expand Up @@ -197,10 +196,10 @@ def update(self) -> dict[str, Any]:
self._last_host_cpu_counters = host_cpu

if self.monitor_gil_contention:
self._last_gil_contention = self._gilknocker.contention_metric
self._cumulative_gil_contention += self._last_gil_contention
result["gil_contention"] = self._last_gil_contention
gil_contention = self._gilknocker.contention_metric
self._gilknocker.reset_contention_metric()
result["gil_contention"] = self._last_gil_contention = gil_contention
self.cumulative_gil_contention += duration * gil_contention

# Note: WINDOWS constant doesn't work with `mypy --platform win32`
if sys.platform != "win32":
Expand Down
16 changes: 10 additions & 6 deletions docs/source/prometheus.rst
Original file line number Diff line number Diff line change
Expand Up @@ -26,9 +26,11 @@ dask_scheduler_clients
Number of clients connected
dask_scheduler_desired_workers
Number of workers scheduler needs for task graph
dask_scheduler_gil_contention_total
Value representing cumulative total of GIL contention,
in the form of summed percentages.
dask_scheduler_gil_contention_seconds_total
Value representing cumulative total of *potential* GIL contention,
in the form of cumulative seconds during which any thread held the GIL locked.
Other threads may or may not have been actually trying to acquire the GIL in the
meantime.

.. note::
Requires ``gilknocker`` to be installed, and
Expand Down Expand Up @@ -128,9 +130,11 @@ dask_worker_tasks
Number of tasks at worker
dask_worker_threads
Number of worker threads
dask_worker_gil_contention_total
Value representing cumulative total GIL contention on worker,
in the form of summed percentages.
dask_worker_gil_contention_seconds_total
Value representing cumulative total of *potential* GIL contention,
in the form of cumulative seconds during which any thread held the GIL locked.
Other threads may or may not have been actually trying to acquire the GIL in the
meantime.

.. note::
Requires ``gilknocker`` to be installed, and
Expand Down

0 comments on commit de166f6

Please sign in to comment.