-
Notifications
You must be signed in to change notification settings - Fork 77
/
Copy pathprometheus_exporter.py
530 lines (466 loc) · 16.1 KB
/
prometheus_exporter.py
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
"""Prometheus exporter for Airflow."""
import json
import pickle
from contextlib import contextmanager
from airflow.configuration import conf
from airflow.models import DagModel, DagRun, TaskInstance, TaskFail, XCom
from airflow.plugins_manager import AirflowPlugin
from airflow.settings import Session
from airflow.utils.state import State
from airflow.utils.log.logging_mixin import LoggingMixin
from flask import Response
from flask_admin import BaseView, expose
from prometheus_client import generate_latest, REGISTRY
from prometheus_client.core import GaugeMetricFamily
from sqlalchemy import and_, func
from airflow_prometheus_exporter.xcom_config import load_xcom_config
CANARY_DAG = "canary_dag"
RBAC = "True"
@contextmanager
def session_scope(session):
"""Provide a transactional scope around a series of operations."""
try:
yield session
finally:
session.close()
######################
# DAG Related Metrics
######################
def get_dag_state_info():
"""Number of DAG Runs with particular state."""
with session_scope(Session) as session:
dag_status_query = (
session.query(
DagRun.dag_id,
DagRun.state,
func.count(DagRun.state).label("count"),
)
.group_by(DagRun.dag_id, DagRun.state)
.subquery()
)
return (
session.query(
dag_status_query.c.dag_id,
dag_status_query.c.state,
dag_status_query.c.count,
DagModel.owners,
)
.join(DagModel, DagModel.dag_id == dag_status_query.c.dag_id)
.filter(
DagModel.is_active == True, # noqa
DagModel.is_paused == False,
)
.all()
)
def get_dag_duration_info():
"""Duration of successful DAG Runs."""
with session_scope(Session) as session:
max_execution_dt_query = (
session.query(
DagRun.dag_id,
func.max(DagRun.execution_date).label("max_execution_dt"),
)
.join(DagModel, DagModel.dag_id == DagRun.dag_id)
.filter(
DagModel.is_active == True, # noqa
DagModel.is_paused == False,
DagRun.state == State.SUCCESS,
DagRun.end_date.isnot(None),
)
.group_by(DagRun.dag_id)
.subquery()
)
dag_start_dt_query = (
session.query(
max_execution_dt_query.c.dag_id,
max_execution_dt_query.c.max_execution_dt.label(
"execution_date"
),
func.min(TaskInstance.start_date).label("start_date"),
)
.join(
TaskInstance,
and_(
TaskInstance.dag_id == max_execution_dt_query.c.dag_id,
(
TaskInstance.execution_date
== max_execution_dt_query.c.max_execution_dt
),
),
)
.group_by(
max_execution_dt_query.c.dag_id,
max_execution_dt_query.c.max_execution_dt,
)
.subquery()
)
return (
session.query(
dag_start_dt_query.c.dag_id,
dag_start_dt_query.c.start_date,
DagRun.end_date,
)
.join(
DagRun,
and_(
DagRun.dag_id == dag_start_dt_query.c.dag_id,
DagRun.execution_date
== dag_start_dt_query.c.execution_date,
),
)
.filter(
TaskInstance.start_date.isnot(None),
TaskInstance.end_date.isnot(None),
)
.all()
)
######################
# Task Related Metrics
######################
def get_task_state_info():
"""Number of task instances with particular state."""
with session_scope(Session) as session:
task_status_query = (
session.query(
TaskInstance.dag_id,
TaskInstance.task_id,
TaskInstance.state,
func.count(TaskInstance.dag_id).label("value"),
)
.group_by(
TaskInstance.dag_id, TaskInstance.task_id, TaskInstance.state
)
.subquery()
)
return (
session.query(
task_status_query.c.dag_id,
task_status_query.c.task_id,
task_status_query.c.state,
task_status_query.c.value,
DagModel.owners,
)
.join(DagModel, DagModel.dag_id == task_status_query.c.dag_id)
.filter(
DagModel.is_active == True, # noqa
DagModel.is_paused == False,
)
.all()
)
def get_task_failure_counts():
"""Compute Task Failure Counts."""
with session_scope(Session) as session:
return (
session.query(
TaskFail.dag_id,
TaskFail.task_id,
func.count(TaskFail.dag_id).label("count"),
)
.join(DagModel, DagModel.dag_id == TaskFail.dag_id,)
.filter(
DagModel.is_active == True, # noqa
DagModel.is_paused == False,
)
.group_by(TaskFail.dag_id, TaskFail.task_id,)
)
def get_xcom_params(task_id):
"""XCom parameters for matching task_id's for the latest run of a DAG."""
with session_scope(Session) as session:
max_execution_dt_query = (
session.query(
DagRun.dag_id,
func.max(DagRun.execution_date).label("max_execution_dt"),
)
.group_by(DagRun.dag_id)
.subquery()
)
query = session.query(XCom.dag_id, XCom.task_id, XCom.value).join(
max_execution_dt_query,
and_(
(XCom.dag_id == max_execution_dt_query.c.dag_id),
(
XCom.execution_date
== max_execution_dt_query.c.max_execution_dt
),
),
)
if task_id == "all":
return query.all()
else:
return query.filter(XCom.task_id == task_id).all()
def extract_xcom_parameter(value):
"""Deserializes value stored in xcom table."""
enable_pickling = conf.getboolean("core", "enable_xcom_pickling")
if enable_pickling:
value = pickle.loads(value)
try:
value = json.loads(value)
return value
except Exception:
return {}
else:
try:
return json.loads(value.decode("UTF-8"))
except ValueError:
log = LoggingMixin().log
log.error(
"Could not deserialize the XCOM value from JSON. "
"If you are using pickles instead of JSON "
"for XCOM, then you need to enable pickle "
"support for XCOM in your airflow config."
)
return {}
def get_task_duration_info():
"""Duration of successful tasks in seconds."""
with session_scope(Session) as session:
max_execution_dt_query = (
session.query(
DagRun.dag_id,
func.max(DagRun.execution_date).label("max_execution_dt"),
)
.join(DagModel, DagModel.dag_id == DagRun.dag_id,)
.filter(
DagModel.is_active == True, # noqa
DagModel.is_paused == False,
DagRun.state == State.SUCCESS,
DagRun.end_date.isnot(None),
)
.group_by(DagRun.dag_id)
.subquery()
)
return (
session.query(
TaskInstance.dag_id,
TaskInstance.task_id,
TaskInstance.start_date,
TaskInstance.end_date,
DagRun.execution_date,
)
.join(
DagRun, TaskInstance.run_id == DagRun.run_id
)
.join(
max_execution_dt_query,
and_(
(TaskInstance.dag_id == max_execution_dt_query.c.dag_id),
(
TaskInstance.execution_date
== max_execution_dt_query.c.max_execution_dt
),
),
)
.filter(
TaskInstance.state == State.SUCCESS,
TaskInstance.start_date.isnot(None),
TaskInstance.end_date.isnot(None),
)
.all()
)
######################
# Scheduler Related Metrics
######################
def get_dag_scheduler_delay():
"""Compute DAG scheduling delay."""
with session_scope(Session) as session:
return (
session.query(
DagRun.dag_id, DagRun.execution_date, DagRun.start_date,
)
.filter(DagRun.dag_id == CANARY_DAG,)
.order_by(DagRun.execution_date.desc())
.limit(1)
.all()
)
def get_task_scheduler_delay():
"""Compute Task scheduling delay."""
with session_scope(Session) as session:
task_status_query = (
session.query(
TaskInstance.queue,
func.max(TaskInstance.start_date).label("max_start"),
)
.filter(
TaskInstance.dag_id == CANARY_DAG,
TaskInstance.queued_dttm.isnot(None),
)
.group_by(TaskInstance.queue)
.subquery()
)
return (
session.query(
task_status_query.c.queue,
DagRun.execution_date,
TaskInstance.queued_dttm,
task_status_query.c.max_start.label("start_date"),
)
.join(
TaskInstance,
and_(
TaskInstance.queue == task_status_query.c.queue,
TaskInstance.start_date == task_status_query.c.max_start,
),
)
.join(
DagRun, TaskInstance.run_id == DagRun.run_id
)
.filter(
TaskInstance.dag_id
== CANARY_DAG, # Redundant, for performance.
)
.all()
)
def get_num_queued_tasks():
"""Number of queued tasks currently."""
with session_scope(Session) as session:
return (
session.query(TaskInstance)
.filter(TaskInstance.state == State.QUEUED)
.count()
)
class MetricsCollector(object):
"""Metrics Collector for prometheus."""
def describe(self):
return []
def collect(self):
"""Collect metrics."""
# Task metrics
task_info = get_task_state_info()
t_state = GaugeMetricFamily(
"airflow_task_status",
"Shows the number of task instances with particular status",
labels=["dag_id", "task_id", "owner", "status"],
)
for task in task_info:
t_state.add_metric(
[task.dag_id, task.task_id, task.owners, task.state or "none"],
task.value,
)
yield t_state
task_duration = GaugeMetricFamily(
"airflow_task_duration",
"Duration of successful tasks in seconds",
labels=["task_id", "dag_id", "execution_date"],
)
for task in get_task_duration_info():
task_duration_value = (
task.end_date - task.start_date
).total_seconds()
task_duration.add_metric(
[task.task_id, task.dag_id, str(task.execution_date.date())],
task_duration_value,
)
yield task_duration
task_failure_count = GaugeMetricFamily(
"airflow_task_fail_count",
"Count of failed tasks",
labels=["dag_id", "task_id"],
)
for task in get_task_failure_counts():
task_failure_count.add_metric(
[task.dag_id, task.task_id], task.count
)
yield task_failure_count
# Dag Metrics
dag_info = get_dag_state_info()
d_state = GaugeMetricFamily(
"airflow_dag_status",
"Shows the number of dag starts with this status",
labels=["dag_id", "owner", "status"],
)
for dag in dag_info:
d_state.add_metric([dag.dag_id, dag.owners, dag.state], dag.count)
yield d_state
dag_duration = GaugeMetricFamily(
"airflow_dag_run_duration",
"Duration of successful dag_runs in seconds",
labels=["dag_id"],
)
for dag in get_dag_duration_info():
dag_duration_value = (
dag.end_date - dag.start_date
).total_seconds()
dag_duration.add_metric([dag.dag_id], dag_duration_value)
yield dag_duration
# Scheduler Metrics
dag_scheduler_delay = GaugeMetricFamily(
"airflow_dag_scheduler_delay",
"Airflow DAG scheduling delay",
labels=["dag_id"],
)
for dag in get_dag_scheduler_delay():
dag_scheduling_delay_value = (
dag.start_date - dag.execution_date
).total_seconds()
dag_scheduler_delay.add_metric(
[dag.dag_id], dag_scheduling_delay_value
)
yield dag_scheduler_delay
# XCOM parameters
xcom_params = GaugeMetricFamily(
"airflow_xcom_parameter",
"Airflow Xcom Parameter",
labels=["dag_id", "task_id"],
)
xcom_config = load_xcom_config()
for tasks in xcom_config.get("xcom_params", []):
for param in get_xcom_params(tasks["task_id"]):
xcom_value = extract_xcom_parameter(param.value)
if tasks["key"] in xcom_value:
xcom_params.add_metric(
[param.dag_id, param.task_id], xcom_value[tasks["key"]]
)
yield xcom_params
task_scheduler_delay = GaugeMetricFamily(
"airflow_task_scheduler_delay",
"Airflow Task scheduling delay",
labels=["queue"],
)
for task in get_task_scheduler_delay():
task_scheduling_delay_value = (
task.start_date - task.queued_dttm
).total_seconds()
task_scheduler_delay.add_metric(
[task.queue], task_scheduling_delay_value
)
yield task_scheduler_delay
num_queued_tasks_metric = GaugeMetricFamily(
"airflow_num_queued_tasks", "Airflow Number of Queued Tasks",
)
num_queued_tasks = get_num_queued_tasks()
num_queued_tasks_metric.add_metric([], num_queued_tasks)
yield num_queued_tasks_metric
REGISTRY.register(MetricsCollector())
if RBAC:
from flask_appbuilder import BaseView as FABBaseView, expose as FABexpose
class RBACMetrics(FABBaseView):
route_base = "/admin/metrics/"
@FABexpose('/')
def list(self):
return Response(generate_latest(), mimetype='text')
# Metrics View for Flask app builder used in airflow with rbac enabled
RBACmetricsView = {
"view": RBACMetrics(),
"name": "Metrics",
"category": "Public"
}
ADMIN_VIEW = []
RBAC_VIEW = [RBACmetricsView]
else:
class Metrics(BaseView):
@expose("/")
def index(self):
return Response(generate_latest(), mimetype="text/plain")
ADMIN_VIEW = [Metrics(category="Prometheus exporter", name="Metrics")]
RBAC_VIEW = []
class AirflowPrometheusPlugin(AirflowPlugin):
"""Airflow Plugin for collecting metrics."""
name = "airflow_prometheus_plugin"
operators = []
hooks = []
executors = []
macros = []
admin_views = ADMIN_VIEW
flask_blueprints = []
menu_links = []
appbuilder_views = RBAC_VIEW
appbuilder_menu_items = []