-
-
Notifications
You must be signed in to change notification settings - Fork 719
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
Enable configuration of prometheus metrics namespace (#4722)
* Enable configuration of prometheus metrics namespace * Undo editor formatting * Refactor into base class and remove private function import * Rename semaphore subsystem * Remove serving metric and rename connections metric
- Loading branch information
1 parent
4a883a7
commit 42e3f22
Showing
9 changed files
with
199 additions
and
158 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,17 @@ | ||
import dask.config | ||
|
||
|
||
class PrometheusCollector: | ||
def __init__(self, server): | ||
self.server = server | ||
self.namespace = dask.config.get("distributed.dashboard.prometheus.namespace") | ||
self.subsystem = None | ||
|
||
def build_name(self, name): | ||
full_name = [] | ||
if self.namespace: | ||
full_name.append(self.namespace) | ||
if self.subsystem: | ||
full_name.append(self.subsystem) | ||
full_name.append(name) | ||
return "_".join(full_name) |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -1,105 +1,3 @@ | ||
import toolz | ||
|
||
from distributed.http.utils import RequestHandler | ||
from distributed.scheduler import ALL_TASK_STATES | ||
|
||
from .semaphore import SemaphoreMetricExtension | ||
|
||
|
||
class _PrometheusCollector: | ||
def __init__(self, dask_server): | ||
self.server = dask_server | ||
|
||
def collect(self): | ||
from prometheus_client.core import CounterMetricFamily, GaugeMetricFamily | ||
|
||
yield GaugeMetricFamily( | ||
"dask_scheduler_clients", | ||
"Number of clients connected.", | ||
value=len([k for k in self.server.clients if k != "fire-and-forget"]), | ||
) | ||
|
||
yield GaugeMetricFamily( | ||
"dask_scheduler_desired_workers", | ||
"Number of workers scheduler needs for task graph.", | ||
value=self.server.adaptive_target(), | ||
) | ||
|
||
worker_states = GaugeMetricFamily( | ||
"dask_scheduler_workers", | ||
"Number of workers known by scheduler.", | ||
labels=["state"], | ||
) | ||
worker_states.add_metric(["connected"], len(self.server.workers)) | ||
worker_states.add_metric(["saturated"], len(self.server.saturated)) | ||
worker_states.add_metric(["idle"], len(self.server.idle)) | ||
yield worker_states | ||
|
||
tasks = GaugeMetricFamily( | ||
"dask_scheduler_tasks", | ||
"Number of tasks known by scheduler.", | ||
labels=["state"], | ||
) | ||
|
||
task_counter = toolz.merge_with( | ||
sum, (tp.states for tp in self.server.task_prefixes.values()) | ||
) | ||
|
||
suspicious_tasks = CounterMetricFamily( | ||
"dask_scheduler_tasks_suspicious", | ||
"Total number of times a task has been marked suspicious", | ||
labels=["task_prefix_name"], | ||
) | ||
|
||
for tp in self.server.task_prefixes.values(): | ||
suspicious_tasks.add_metric([tp.name], tp.suspicious) | ||
yield suspicious_tasks | ||
|
||
yield CounterMetricFamily( | ||
"dask_scheduler_tasks_forgotten", | ||
( | ||
"Total number of processed tasks no longer in memory and already " | ||
"removed from the scheduler job queue. Note task groups on the " | ||
"scheduler which have all tasks in the forgotten state are not included." | ||
), | ||
value=task_counter.get("forgotten", 0.0), | ||
) | ||
|
||
for state in ALL_TASK_STATES: | ||
tasks.add_metric([state], task_counter.get(state, 0.0)) | ||
yield tasks | ||
|
||
|
||
COLLECTORS = [_PrometheusCollector, SemaphoreMetricExtension] | ||
|
||
|
||
class PrometheusHandler(RequestHandler): | ||
_collectors = None | ||
|
||
def __init__(self, *args, dask_server=None, **kwargs): | ||
import prometheus_client | ||
|
||
super().__init__(*args, dask_server=dask_server, **kwargs) | ||
|
||
if PrometheusHandler._collectors: | ||
# Especially during testing, multiple schedulers are started | ||
# sequentially in the same python process | ||
for _collector in PrometheusHandler._collectors: | ||
_collector.server = self.server | ||
return | ||
|
||
PrometheusHandler._collectors = tuple( | ||
collector(self.server) for collector in COLLECTORS | ||
) | ||
# Register collectors | ||
for instantiated_collector in PrometheusHandler._collectors: | ||
prometheus_client.REGISTRY.register(instantiated_collector) | ||
|
||
def get(self): | ||
import prometheus_client | ||
|
||
self.write(prometheus_client.generate_latest()) | ||
self.set_header("Content-Type", "text/plain; version=0.0.4") | ||
|
||
from .core import PrometheusHandler | ||
|
||
routes = [("/metrics", PrometheusHandler, {})] |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,104 @@ | ||
import toolz | ||
|
||
from distributed.http.prometheus import PrometheusCollector | ||
from distributed.http.utils import RequestHandler | ||
from distributed.scheduler import ALL_TASK_STATES | ||
|
||
from .semaphore import SemaphoreMetricCollector | ||
|
||
|
||
class SchedulerMetricCollector(PrometheusCollector): | ||
def __init__(self, server): | ||
super().__init__(server) | ||
self.subsystem = "scheduler" | ||
|
||
def collect(self): | ||
from prometheus_client.core import CounterMetricFamily, GaugeMetricFamily | ||
|
||
yield GaugeMetricFamily( | ||
self.build_name("clients"), | ||
"Number of clients connected.", | ||
value=len([k for k in self.server.clients if k != "fire-and-forget"]), | ||
) | ||
|
||
yield GaugeMetricFamily( | ||
self.build_name("desired_workers"), | ||
"Number of workers scheduler needs for task graph.", | ||
value=self.server.adaptive_target(), | ||
) | ||
|
||
worker_states = GaugeMetricFamily( | ||
self.build_name("workers"), | ||
"Number of workers known by scheduler.", | ||
labels=["state"], | ||
) | ||
worker_states.add_metric(["connected"], len(self.server.workers)) | ||
worker_states.add_metric(["saturated"], len(self.server.saturated)) | ||
worker_states.add_metric(["idle"], len(self.server.idle)) | ||
yield worker_states | ||
|
||
tasks = GaugeMetricFamily( | ||
self.build_name("tasks"), | ||
"Number of tasks known by scheduler.", | ||
labels=["state"], | ||
) | ||
|
||
task_counter = toolz.merge_with( | ||
sum, (tp.states for tp in self.server.task_prefixes.values()) | ||
) | ||
|
||
suspicious_tasks = CounterMetricFamily( | ||
self.build_name("tasks_suspicious"), | ||
"Total number of times a task has been marked suspicious", | ||
labels=["task_prefix_name"], | ||
) | ||
|
||
for tp in self.server.task_prefixes.values(): | ||
suspicious_tasks.add_metric([tp.name], tp.suspicious) | ||
yield suspicious_tasks | ||
|
||
yield CounterMetricFamily( | ||
self.build_name("tasks_forgotten"), | ||
( | ||
"Total number of processed tasks no longer in memory and already " | ||
"removed from the scheduler job queue. Note task groups on the " | ||
"scheduler which have all tasks in the forgotten state are not included." | ||
), | ||
value=task_counter.get("forgotten", 0.0), | ||
) | ||
|
||
for state in ALL_TASK_STATES: | ||
tasks.add_metric([state], task_counter.get(state, 0.0)) | ||
yield tasks | ||
|
||
|
||
COLLECTORS = [SchedulerMetricCollector, SemaphoreMetricCollector] | ||
|
||
|
||
class PrometheusHandler(RequestHandler): | ||
_collectors = None | ||
|
||
def __init__(self, *args, dask_server=None, **kwargs): | ||
import prometheus_client | ||
|
||
super().__init__(*args, dask_server=dask_server, **kwargs) | ||
|
||
if PrometheusHandler._collectors: | ||
# Especially during testing, multiple schedulers are started | ||
# sequentially in the same python process | ||
for _collector in PrometheusHandler._collectors: | ||
_collector.server = self.server | ||
return | ||
|
||
PrometheusHandler._collectors = tuple( | ||
collector(self.server) for collector in COLLECTORS | ||
) | ||
# Register collectors | ||
for instantiated_collector in PrometheusHandler._collectors: | ||
prometheus_client.REGISTRY.register(instantiated_collector) | ||
|
||
def get(self): | ||
import prometheus_client | ||
|
||
self.write(prometheus_client.generate_latest()) | ||
self.set_header("Content-Type", "text/plain; version=0.0.4") |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,3 @@ | ||
from .core import PrometheusHandler | ||
|
||
routes = [("/metrics", PrometheusHandler, {})] |
Oops, something went wrong.