-
Notifications
You must be signed in to change notification settings - Fork 1.3k
/
queued_run_coordinator_daemon.py
224 lines (178 loc) · 8.05 KB
/
queued_run_coordinator_daemon.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
import logging
import sys
import time
from collections import defaultdict
from typing import Dict
from dagster import DagsterEvent, DagsterEventType, check
from dagster.core.events.log import EventLogEntry
from dagster.core.instance import DagsterInstance
from dagster.core.storage.pipeline_run import (
IN_PROGRESS_RUN_STATUSES,
PipelineRun,
PipelineRunStatus,
PipelineRunsFilter,
)
from dagster.core.storage.tags import PRIORITY_TAG
from dagster.core.workspace import IWorkspace
from dagster.daemon.daemon import DagsterDaemon
from dagster.utils.error import serializable_error_info_from_exc_info
class _TagConcurrencyLimitsCounter:
"""
Helper object that keeps track of when the tag concurrency limits are met
"""
def __init__(self, tag_concurrency_limits, in_progress_runs):
check.opt_list_param(tag_concurrency_limits, "tag_concurrency_limits", of_type=dict)
check.list_param(in_progress_runs, "in_progress_runs", of_type=PipelineRun)
self._key_limits: Dict[str, int] = {}
self._key_value_limits: Dict[(str, str), int] = {}
self._unique_value_limits: Dict[str, int] = {}
for tag_limit in tag_concurrency_limits:
key = tag_limit["key"]
value = tag_limit.get("value")
limit = tag_limit["limit"]
if isinstance(value, str):
self._key_value_limits[(key, value)] = limit
elif not value or not value["applyLimitPerUniqueValue"]:
self._key_limits[key] = limit
else:
self._unique_value_limits[key] = limit
self._key_counts: Dict[str, int] = defaultdict(lambda: 0)
self._key_value_counts: Dict[(str, str), int] = defaultdict(lambda: 0)
self._unique_value_counts: Dict[(str, str), int] = defaultdict(lambda: 0)
# initialize counters based on current in progress runs
for run in in_progress_runs:
self.update_counters_with_launched_run(run)
def is_run_blocked(self, run):
"""
True if there are in progress runs which are blocking this run based on tag limits
"""
for key, value in run.tags.items():
if key in self._key_limits and self._key_counts[key] >= self._key_limits[key]:
return True
tag_tuple = (key, value)
if (
tag_tuple in self._key_value_limits
and self._key_value_counts[tag_tuple] >= self._key_value_limits[tag_tuple]
):
return True
if (
key in self._unique_value_limits
and self._unique_value_counts[tag_tuple] >= self._unique_value_limits[key]
):
return True
return False
def update_counters_with_launched_run(self, run):
"""
Add a new in progress run to the counters
"""
for key, value in run.tags.items():
if key in self._key_limits:
self._key_counts[key] += 1
tag_tuple = (key, value)
if tag_tuple in self._key_value_limits:
self._key_value_counts[tag_tuple] += 1
if key in self._unique_value_limits:
self._unique_value_counts[tag_tuple] += 1
class QueuedRunCoordinatorDaemon(DagsterDaemon):
"""
Used with the QueuedRunCoordinator on the instance. This process finds queued runs from the run
store and launches them.
"""
@classmethod
def daemon_type(cls):
return "QUEUED_RUN_COORDINATOR"
def run_iteration(self, instance, workspace):
check.inst_param(instance, "instance", DagsterInstance)
check.inst_param(workspace, "workspace", IWorkspace)
max_concurrent_runs = instance.run_coordinator.max_concurrent_runs
tag_concurrency_limits = instance.run_coordinator.tag_concurrency_limits
in_progress_runs = self._get_in_progress_runs(instance)
max_runs_to_launch = max_concurrent_runs - len(in_progress_runs)
# Possibly under 0 if runs were launched without queuing
if max_runs_to_launch <= 0:
self._logger.info(
"{} runs are currently in progress. Maximum is {}, won't launch more.".format(
len(in_progress_runs), max_concurrent_runs
)
)
return
queued_runs = self._get_queued_runs(instance)
if not queued_runs:
self._logger.info("Poll returned no queued runs.")
else:
self._logger.info("Retrieved {} queued runs, checking limits.".format(len(queued_runs)))
# place in order
sorted_runs = self._priority_sort(queued_runs)
# launch until blocked by limit rules
num_dequeued_runs = 0
tag_concurrency_limits_counter = _TagConcurrencyLimitsCounter(
tag_concurrency_limits, in_progress_runs
)
for run in sorted_runs:
if num_dequeued_runs >= max_runs_to_launch:
break
if tag_concurrency_limits_counter.is_run_blocked(run):
continue
error_info = None
try:
self._dequeue_run(instance, run, workspace)
except Exception:
error_info = serializable_error_info_from_exc_info(sys.exc_info())
message = (
f"Caught an error for run {run.run_id} while removing it from the queue."
" Marking the run as failed and dropping it from the queue"
)
message_with_full_error = f"{message}: {error_info.to_string()}"
self._logger.error(message_with_full_error)
instance.report_run_failed(run, message_with_full_error)
# modify the original error, so that the extra message appears in heartbeats
error_info = error_info._replace(message=f"{message}: {error_info.message}")
else:
tag_concurrency_limits_counter.update_counters_with_launched_run(run)
num_dequeued_runs += 1
yield error_info
self._logger.info("Launched {} runs.".format(num_dequeued_runs))
def _get_queued_runs(self, instance):
queued_runs_filter = PipelineRunsFilter(statuses=[PipelineRunStatus.QUEUED])
# Reversed for fifo ordering
# Note: should add a maximum fetch limit https://github.com/dagster-io/dagster/issues/3339
runs = instance.get_runs(filters=queued_runs_filter)[::-1]
return runs
def _get_in_progress_runs(self, instance):
# Note: should add a maximum fetch limit https://github.com/dagster-io/dagster/issues/3339
return instance.get_runs(filters=PipelineRunsFilter(statuses=IN_PROGRESS_RUN_STATUSES))
def _priority_sort(self, runs):
def get_priority(run):
priority_tag_value = run.tags.get(PRIORITY_TAG, "0")
try:
return int(priority_tag_value)
except ValueError:
return 0
# sorted is stable, so fifo is maintained
return sorted(runs, key=get_priority, reverse=True)
def _dequeue_run(self, instance, run, workspace):
# double check that the run is still queued before dequeing
reloaded_run = instance.get_run_by_id(run.run_id)
if reloaded_run.status != PipelineRunStatus.QUEUED:
self._logger.info(
"Run {run_id} is now {status} instead of QUEUED, skipping".format(
run_id=reloaded_run.run_id, status=reloaded_run.status
)
)
return
dequeued_event = DagsterEvent(
event_type_value=DagsterEventType.PIPELINE_DEQUEUED.value,
pipeline_name=run.pipeline_name,
)
event_record = EventLogEntry(
message="",
user_message="",
level=logging.INFO,
pipeline_name=run.pipeline_name,
run_id=run.run_id,
error_info=None,
timestamp=time.time(),
dagster_event=dequeued_event,
)
instance.handle_new_event(event_record)
instance.launch_run(run.run_id, workspace)