forked from spotify/luigi
/
scheduler.py
1650 lines (1375 loc) · 64 KB
/
scheduler.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
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
# -*- coding: utf-8 -*-
#
# Copyright 2012-2015 Spotify AB
#
# Licensed under the Apache License, Version 2.0 (the "License");
# you may not use this file except in compliance with the License.
# You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS,
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
# See the License for the specific language governing permissions and
# limitations under the License.
#
"""
The system for scheduling tasks and executing them in order.
Deals with dependencies, priorities, resources, etc.
The :py:class:`~luigi.worker.Worker` pulls tasks from the scheduler (usually over the REST interface) and executes them.
See :doc:`/central_scheduler` for more info.
"""
import collections
import json
from luigi.batch_notifier import BatchNotifier
try:
import cPickle as pickle
except ImportError:
import pickle
import functools
import hashlib
import itertools
import logging
import os
import re
import time
import uuid
from luigi import six
from luigi import configuration
from luigi import notifications
from luigi import parameter
from luigi import task_history as history
from luigi.task_status import DISABLED, DONE, FAILED, PENDING, RUNNING, SUSPENDED, UNKNOWN, \
BATCH_RUNNING
from luigi.task import Config
from luigi.parameter import ParameterVisibility
from luigi.metrics import MetricsCollectors
logger = logging.getLogger(__name__)
UPSTREAM_RUNNING = 'UPSTREAM_RUNNING'
UPSTREAM_MISSING_INPUT = 'UPSTREAM_MISSING_INPUT'
UPSTREAM_FAILED = 'UPSTREAM_FAILED'
UPSTREAM_DISABLED = 'UPSTREAM_DISABLED'
UPSTREAM_SEVERITY_ORDER = (
'',
UPSTREAM_RUNNING,
UPSTREAM_MISSING_INPUT,
UPSTREAM_FAILED,
UPSTREAM_DISABLED,
)
UPSTREAM_SEVERITY_KEY = UPSTREAM_SEVERITY_ORDER.index
STATUS_TO_UPSTREAM_MAP = {
FAILED: UPSTREAM_FAILED,
RUNNING: UPSTREAM_RUNNING,
BATCH_RUNNING: UPSTREAM_RUNNING,
PENDING: UPSTREAM_MISSING_INPUT,
DISABLED: UPSTREAM_DISABLED,
}
WORKER_STATE_DISABLED = 'disabled'
WORKER_STATE_ACTIVE = 'active'
TASK_FAMILY_RE = re.compile(r'([^(_]+)[(_]')
RPC_METHODS = {}
_retry_policy_fields = [
"retry_count",
"disable_hard_timeout",
"disable_window",
]
RetryPolicy = collections.namedtuple("RetryPolicy", _retry_policy_fields)
def _get_empty_retry_policy():
return RetryPolicy(*[None] * len(_retry_policy_fields))
def rpc_method(**request_args):
def _rpc_method(fn):
# If request args are passed, return this function again for use as
# the decorator function with the request args attached.
fn_args = six.getargspec(fn)
assert not fn_args.varargs
assert fn_args.args[0] == 'self'
all_args = fn_args.args[1:]
defaults = dict(zip(reversed(all_args), reversed(fn_args.defaults or ())))
required_args = frozenset(arg for arg in all_args if arg not in defaults)
fn_name = fn.__name__
@functools.wraps(fn)
def rpc_func(self, *args, **kwargs):
actual_args = defaults.copy()
actual_args.update(dict(zip(all_args, args)))
actual_args.update(kwargs)
if not all(arg in actual_args for arg in required_args):
raise TypeError('{} takes {} arguments ({} given)'.format(
fn_name, len(all_args), len(actual_args)))
return self._request('/api/{}'.format(fn_name), actual_args, **request_args)
RPC_METHODS[fn_name] = rpc_func
return fn
return _rpc_method
class scheduler(Config):
retry_delay = parameter.FloatParameter(default=900.0)
remove_delay = parameter.FloatParameter(default=600.0)
worker_disconnect_delay = parameter.FloatParameter(default=60.0)
state_path = parameter.Parameter(default='/var/lib/luigi-server/state.pickle')
batch_emails = parameter.BoolParameter(default=False, description="Send e-mails in batches rather than immediately")
# Jobs are disabled if we see more than retry_count failures in disable_window seconds.
# These disables last for disable_persist seconds.
disable_window = parameter.IntParameter(default=3600)
retry_count = parameter.IntParameter(default=999999999)
disable_hard_timeout = parameter.IntParameter(default=999999999)
disable_persist = parameter.IntParameter(default=86400)
max_shown_tasks = parameter.IntParameter(default=100000)
max_graph_nodes = parameter.IntParameter(default=100000)
record_task_history = parameter.BoolParameter(default=False)
prune_on_get_work = parameter.BoolParameter(default=False)
pause_enabled = parameter.BoolParameter(default=True)
send_messages = parameter.BoolParameter(default=True)
metrics_collector = parameter.EnumParameter(enum=MetricsCollectors, default=MetricsCollectors.default)
def _get_retry_policy(self):
return RetryPolicy(self.retry_count, self.disable_hard_timeout, self.disable_window)
class Failures(object):
"""
This class tracks the number of failures in a given time window.
Failures added are marked with the current timestamp, and this class counts
the number of failures in a sliding time window ending at the present.
"""
def __init__(self, window):
"""
Initialize with the given window.
:param window: how long to track failures for, as a float (number of seconds).
"""
self.window = window
self.failures = collections.deque()
self.first_failure_time = None
def add_failure(self):
"""
Add a failure event with the current timestamp.
"""
failure_time = time.time()
if not self.first_failure_time:
self.first_failure_time = failure_time
self.failures.append(failure_time)
def num_failures(self):
"""
Return the number of failures in the window.
"""
min_time = time.time() - self.window
while self.failures and self.failures[0] < min_time:
self.failures.popleft()
return len(self.failures)
def clear(self):
"""
Clear the failure queue.
"""
self.failures.clear()
def _get_default(x, default):
if x is not None:
return x
else:
return default
class OrderedSet(collections.MutableSet):
"""
Standard Python OrderedSet recipe found at http://code.activestate.com/recipes/576694/
Modified to include a peek function to get the last element
"""
def __init__(self, iterable=None):
self.end = end = []
end += [None, end, end] # sentinel node for doubly linked list
self.map = {} # key --> [key, prev, next]
if iterable is not None:
self |= iterable
def __len__(self):
return len(self.map)
def __contains__(self, key):
return key in self.map
def add(self, key):
if key not in self.map:
end = self.end
curr = end[1]
curr[2] = end[1] = self.map[key] = [key, curr, end]
def discard(self, key):
if key in self.map:
key, prev, next = self.map.pop(key)
prev[2] = next
next[1] = prev
def __iter__(self):
end = self.end
curr = end[2]
while curr is not end:
yield curr[0]
curr = curr[2]
def __reversed__(self):
end = self.end
curr = end[1]
while curr is not end:
yield curr[0]
curr = curr[1]
def peek(self, last=True):
if not self:
raise KeyError('set is empty')
key = self.end[1][0] if last else self.end[2][0]
return key
def pop(self, last=True):
key = self.peek(last)
self.discard(key)
return key
def __repr__(self):
if not self:
return '%s()' % (self.__class__.__name__,)
return '%s(%r)' % (self.__class__.__name__, list(self))
def __eq__(self, other):
if isinstance(other, OrderedSet):
return len(self) == len(other) and list(self) == list(other)
return set(self) == set(other)
class Task(object):
def __init__(self, task_id, status, deps, resources=None, priority=0, family='', module=None,
params=None, param_visibilities=None, accepts_messages=False, tracking_url=None, status_message=None,
progress_percentage=None, retry_policy='notoptional'):
self.id = task_id
self.stakeholders = set() # workers ids that are somehow related to this task (i.e. don't prune while any of these workers are still active)
self.workers = OrderedSet() # workers ids that can perform task - task is 'BROKEN' if none of these workers are active
if deps is None:
self.deps = set()
else:
self.deps = set(deps)
self.status = status # PENDING, RUNNING, FAILED or DONE
self.time = time.time() # Timestamp when task was first added
self.updated = self.time
self.retry = None
self.remove = None
self.worker_running = None # the worker id that is currently running the task or None
self.time_running = None # Timestamp when picked up by worker
self.expl = None
self.priority = priority
self.resources = _get_default(resources, {})
self.family = family
self.module = module
self.param_visibilities = _get_default(param_visibilities, {})
self.params = {}
self.public_params = {}
self.hidden_params = {}
self.set_params(params)
self.accepts_messages = accepts_messages
self.retry_policy = retry_policy
self.failures = Failures(self.retry_policy.disable_window)
self.tracking_url = tracking_url
self.status_message = status_message
self.progress_percentage = progress_percentage
self.scheduler_message_responses = {}
self.scheduler_disable_time = None
self.runnable = False
self.batchable = False
self.batch_id = None
def __repr__(self):
return "Task(%r)" % vars(self)
def set_params(self, params):
self.params = _get_default(params, {})
self.public_params = {key: value for key, value in self.params.items() if
self.param_visibilities.get(key, ParameterVisibility.PUBLIC) == ParameterVisibility.PUBLIC}
self.hidden_params = {key: value for key, value in self.params.items() if
self.param_visibilities.get(key, ParameterVisibility.PUBLIC) == ParameterVisibility.HIDDEN}
# TODO(2017-08-10) replace this function with direct calls to batchable
# this only exists for backward compatibility
def is_batchable(self):
try:
return self.batchable
except AttributeError:
return False
def add_failure(self):
self.failures.add_failure()
def has_excessive_failures(self):
if self.failures.first_failure_time is not None:
if (time.time() >= self.failures.first_failure_time + self.retry_policy.disable_hard_timeout):
return True
logger.debug('%s task num failures is %s and limit is %s', self.id, self.failures.num_failures(), self.retry_policy.retry_count)
if self.failures.num_failures() >= self.retry_policy.retry_count:
logger.debug('%s task num failures limit(%s) is exceeded', self.id, self.retry_policy.retry_count)
return True
return False
@property
def pretty_id(self):
param_str = ', '.join(u'{}={}'.format(key, value) for key, value in sorted(self.public_params.items()))
return u'{}({})'.format(self.family, param_str)
class Worker(object):
"""
Structure for tracking worker activity and keeping their references.
"""
def __init__(self, worker_id, last_active=None):
self.id = worker_id
self.reference = None # reference to the worker in the real world. (Currently a dict containing just the host)
self.last_active = last_active or time.time() # seconds since epoch
self.last_get_work = None
self.started = time.time() # seconds since epoch
self.tasks = set() # task objects
self.info = {}
self.disabled = False
self.rpc_messages = []
def add_info(self, info):
self.info.update(info)
def update(self, worker_reference, get_work=False):
if worker_reference:
self.reference = worker_reference
self.last_active = time.time()
if get_work:
self.last_get_work = time.time()
def prune(self, config):
# Delete workers that haven't said anything for a while (probably killed)
if self.last_active + config.worker_disconnect_delay < time.time():
return True
def get_tasks(self, state, *statuses):
num_self_tasks = len(self.tasks)
num_state_tasks = sum(len(state._status_tasks[status]) for status in statuses)
if num_self_tasks < num_state_tasks:
return six.moves.filter(lambda task: task.status in statuses, self.tasks)
else:
return six.moves.filter(lambda task: self.id in task.workers, state.get_active_tasks_by_status(*statuses))
def is_trivial_worker(self, state):
"""
If it's not an assistant having only tasks that are without
requirements.
We have to pass the state parameter for optimization reasons.
"""
if self.assistant:
return False
return all(not task.resources for task in self.get_tasks(state, PENDING))
@property
def assistant(self):
return self.info.get('assistant', False)
@property
def enabled(self):
return not self.disabled
@property
def state(self):
if self.enabled:
return WORKER_STATE_ACTIVE
else:
return WORKER_STATE_DISABLED
def add_rpc_message(self, name, **kwargs):
# the message has the format {'name': <function_name>, 'kwargs': <function_kwargs>}
self.rpc_messages.append({'name': name, 'kwargs': kwargs})
def fetch_rpc_messages(self):
messages = self.rpc_messages[:]
del self.rpc_messages[:]
return messages
def __str__(self):
return self.id
class SimpleTaskState(object):
"""
Keep track of the current state and handle persistance.
The point of this class is to enable other ways to keep state, eg. by using a database
These will be implemented by creating an abstract base class that this and other classes
inherit from.
"""
def __init__(self, state_path):
self._state_path = state_path
self._tasks = {} # map from id to a Task object
self._status_tasks = collections.defaultdict(dict)
self._active_workers = {} # map from id to a Worker object
self._task_batchers = {}
self._metrics_collector = None
def get_state(self):
return self._tasks, self._active_workers, self._task_batchers
def set_state(self, state):
self._tasks, self._active_workers = state[:2]
if len(state) >= 3:
self._task_batchers = state[2]
def dump(self):
try:
with open(self._state_path, 'wb') as fobj:
pickle.dump(self.get_state(), fobj)
except IOError:
logger.warning("Failed saving scheduler state", exc_info=1)
else:
logger.info("Saved state in %s", self._state_path)
# prone to lead to crashes when old state is unpickled with updated code. TODO some kind of version control?
def load(self):
if os.path.exists(self._state_path):
logger.info("Attempting to load state from %s", self._state_path)
try:
with open(self._state_path, 'rb') as fobj:
state = pickle.load(fobj)
except BaseException:
logger.exception("Error when loading state. Starting from empty state.")
return
self.set_state(state)
self._status_tasks = collections.defaultdict(dict)
for task in six.itervalues(self._tasks):
self._status_tasks[task.status][task.id] = task
else:
logger.info("No prior state file exists at %s. Starting with empty state", self._state_path)
def get_active_tasks(self):
return six.itervalues(self._tasks)
def get_active_tasks_by_status(self, *statuses):
return itertools.chain.from_iterable(six.itervalues(self._status_tasks[status]) for status in statuses)
def get_active_task_count_for_status(self, status):
if status:
return len(self._status_tasks[status])
else:
return len(self._tasks)
def get_batch_running_tasks(self, batch_id):
assert batch_id is not None
return [
task for task in self.get_active_tasks_by_status(BATCH_RUNNING)
if task.batch_id == batch_id
]
def set_batcher(self, worker_id, family, batcher_args, max_batch_size):
self._task_batchers.setdefault(worker_id, {})
self._task_batchers[worker_id][family] = (batcher_args, max_batch_size)
def get_batcher(self, worker_id, family):
return self._task_batchers.get(worker_id, {}).get(family, (None, 1))
def num_pending_tasks(self):
"""
Return how many tasks are PENDING + RUNNING. O(1).
"""
return len(self._status_tasks[PENDING]) + len(self._status_tasks[RUNNING])
def get_task(self, task_id, default=None, setdefault=None):
if setdefault:
task = self._tasks.setdefault(task_id, setdefault)
self._status_tasks[task.status][task.id] = task
return task
else:
return self._tasks.get(task_id, default)
def has_task(self, task_id):
return task_id in self._tasks
def re_enable(self, task, config=None):
task.scheduler_disable_time = None
task.failures.clear()
if config:
self.set_status(task, FAILED, config)
task.failures.clear()
def set_batch_running(self, task, batch_id, worker_id):
self.set_status(task, BATCH_RUNNING)
task.batch_id = batch_id
task.worker_running = worker_id
task.resources_running = task.resources
task.time_running = time.time()
def set_status(self, task, new_status, config=None):
if new_status == FAILED:
assert config is not None
if new_status == DISABLED and task.status in (RUNNING, BATCH_RUNNING):
return
remove_on_failure = task.batch_id is not None and not task.batchable
if task.status == DISABLED:
if new_status == DONE:
self.re_enable(task)
# don't allow workers to override a scheduler disable
elif task.scheduler_disable_time is not None and new_status != DISABLED:
return
if task.status == RUNNING and task.batch_id is not None and new_status != RUNNING:
for batch_task in self.get_batch_running_tasks(task.batch_id):
self.set_status(batch_task, new_status, config)
batch_task.batch_id = None
task.batch_id = None
if new_status == FAILED and task.status != DISABLED:
task.add_failure()
self.update_metrics_task_failed(task)
if task.has_excessive_failures():
task.scheduler_disable_time = time.time()
new_status = DISABLED
self.update_metrics_task_disabled(task, config)
if not config.batch_emails:
notifications.send_error_email(
'Luigi Scheduler: DISABLED {task} due to excessive failures'.format(task=task.id),
'{task} failed {failures} times in the last {window} seconds, so it is being '
'disabled for {persist} seconds'.format(
failures=task.retry_policy.retry_count,
task=task.id,
window=config.disable_window,
persist=config.disable_persist,
))
elif new_status == DISABLED:
task.scheduler_disable_time = None
if new_status != task.status:
self._status_tasks[task.status].pop(task.id)
self._status_tasks[new_status][task.id] = task
task.status = new_status
task.updated = time.time()
if new_status == DONE:
self.update_metrics_task_done(task)
if new_status == FAILED:
task.retry = time.time() + config.retry_delay
if remove_on_failure:
task.remove = time.time()
def fail_dead_worker_task(self, task, config, assistants):
# If a running worker disconnects, tag all its jobs as FAILED and subject it to the same retry logic
if task.status in (BATCH_RUNNING, RUNNING) and task.worker_running and task.worker_running not in task.stakeholders | assistants:
logger.info("Task %r is marked as running by disconnected worker %r -> marking as "
"FAILED with retry delay of %rs", task.id, task.worker_running,
config.retry_delay)
task.worker_running = None
self.set_status(task, FAILED, config)
task.retry = time.time() + config.retry_delay
def update_status(self, task, config):
# Mark tasks with no remaining active stakeholders for deletion
if (not task.stakeholders) and (task.remove is None) and (task.status != RUNNING):
# We don't check for the RUNNING case, because that is already handled
# by the fail_dead_worker_task function.
logger.debug("Task %r has no stakeholders anymore -> might remove "
"task in %s seconds", task.id, config.remove_delay)
task.remove = time.time() + config.remove_delay
# Re-enable task after the disable time expires
if task.status == DISABLED and task.scheduler_disable_time is not None:
if time.time() - task.scheduler_disable_time > config.disable_persist:
self.re_enable(task, config)
# Reset FAILED tasks to PENDING if max timeout is reached, and retry delay is >= 0
if task.status == FAILED and config.retry_delay >= 0 and task.retry < time.time():
self.set_status(task, PENDING, config)
def may_prune(self, task):
return task.remove and time.time() >= task.remove
def inactivate_tasks(self, delete_tasks):
# The terminology is a bit confusing: we used to "delete" tasks when they became inactive,
# but with a pluggable state storage, you might very well want to keep some history of
# older tasks as well. That's why we call it "inactivate" (as in the verb)
for task in delete_tasks:
task_obj = self._tasks.pop(task)
self._status_tasks[task_obj.status].pop(task)
def get_active_workers(self, last_active_lt=None, last_get_work_gt=None):
for worker in six.itervalues(self._active_workers):
if last_active_lt is not None and worker.last_active >= last_active_lt:
continue
last_get_work = worker.last_get_work
if last_get_work_gt is not None and (
last_get_work is None or last_get_work <= last_get_work_gt):
continue
yield worker
def get_assistants(self, last_active_lt=None):
return filter(lambda w: w.assistant, self.get_active_workers(last_active_lt))
def get_worker_ids(self):
return self._active_workers.keys() # only used for unit tests
def get_worker(self, worker_id):
return self._active_workers.setdefault(worker_id, Worker(worker_id))
def inactivate_workers(self, delete_workers):
# Mark workers as inactive
for worker in delete_workers:
self._active_workers.pop(worker)
self._remove_workers_from_tasks(delete_workers)
def _remove_workers_from_tasks(self, workers, remove_stakeholders=True):
for task in self.get_active_tasks():
if remove_stakeholders:
task.stakeholders.difference_update(workers)
task.workers -= workers
def disable_workers(self, worker_ids):
self._remove_workers_from_tasks(worker_ids, remove_stakeholders=False)
for worker_id in worker_ids:
worker = self.get_worker(worker_id)
worker.disabled = True
worker.tasks.clear()
def update_metrics_task_started(self, task):
self._metrics_collector.handle_task_started(task)
def update_metrics_task_disabled(self, task, config):
self._metrics_collector.handle_task_disabled(task, config)
def update_metrics_task_failed(self, task):
self._metrics_collector.handle_task_failed(task)
def update_metrics_task_done(self, task):
self._metrics_collector.handle_task_done(task)
class Scheduler(object):
"""
Async scheduler that can handle multiple workers, etc.
Can be run locally or on a server (using RemoteScheduler + server.Server).
"""
def __init__(self, config=None, resources=None, task_history_impl=None, **kwargs):
"""
Keyword Arguments:
:param config: an object of class "scheduler" or None (in which the global instance will be used)
:param resources: a dict of str->int constraints
:param task_history_impl: ignore config and use this object as the task history
"""
self._config = config or scheduler(**kwargs)
self._state = SimpleTaskState(self._config.state_path)
if task_history_impl:
self._task_history = task_history_impl
elif self._config.record_task_history:
from luigi import db_task_history # Needs sqlalchemy, thus imported here
self._task_history = db_task_history.DbTaskHistory()
else:
self._task_history = history.NopHistory()
self._resources = resources or configuration.get_config().getintdict('resources') # TODO: Can we make this a Parameter?
self._make_task = functools.partial(Task, retry_policy=self._config._get_retry_policy())
self._worker_requests = {}
self._paused = False
if self._config.batch_emails:
self._email_batcher = BatchNotifier()
self._state._metrics_collector = MetricsCollectors.get(self._config.metrics_collector)
def load(self):
self._state.load()
def dump(self):
self._state.dump()
if self._config.batch_emails:
self._email_batcher.send_email()
@rpc_method()
def prune(self):
logger.debug("Starting pruning of task graph")
self._prune_workers()
self._prune_tasks()
self._prune_emails()
logger.debug("Done pruning task graph")
def _prune_workers(self):
remove_workers = []
for worker in self._state.get_active_workers():
if worker.prune(self._config):
logger.debug("Worker %s timed out (no contact for >=%ss)", worker, self._config.worker_disconnect_delay)
remove_workers.append(worker.id)
self._state.inactivate_workers(remove_workers)
def _prune_tasks(self):
assistant_ids = {w.id for w in self._state.get_assistants()}
remove_tasks = []
for task in self._state.get_active_tasks():
self._state.fail_dead_worker_task(task, self._config, assistant_ids)
self._state.update_status(task, self._config)
if self._state.may_prune(task):
logger.info("Removing task %r", task.id)
remove_tasks.append(task.id)
self._state.inactivate_tasks(remove_tasks)
def _prune_emails(self):
if self._config.batch_emails:
self._email_batcher.update()
def _update_worker(self, worker_id, worker_reference=None, get_work=False):
# Keep track of whenever the worker was last active.
# For convenience also return the worker object.
worker = self._state.get_worker(worker_id)
worker.update(worker_reference, get_work=get_work)
return worker
def _update_priority(self, task, prio, worker):
"""
Update priority of the given task.
Priority can only be increased.
If the task doesn't exist, a placeholder task is created to preserve priority when the task is later scheduled.
"""
task.priority = prio = max(prio, task.priority)
for dep in task.deps or []:
t = self._state.get_task(dep)
if t is not None and prio > t.priority:
self._update_priority(t, prio, worker)
@rpc_method()
def add_task_batcher(self, worker, task_family, batched_args, max_batch_size=float('inf')):
self._state.set_batcher(worker, task_family, batched_args, max_batch_size)
@rpc_method()
def forgive_failures(self, task_id=None):
status = PENDING
task = self._state.get_task(task_id)
if task is None:
return {"task_id": task_id, "status": None}
# we forgive only failures
if task.status == FAILED:
# forgive but do not forget
self._update_task_history(task, status)
self._state.set_status(task, status, self._config)
return {"task_id": task_id, "status": task.status}
@rpc_method()
def add_task(self, task_id=None, status=PENDING, runnable=True,
deps=None, new_deps=None, expl=None, resources=None,
priority=0, family='', module=None, params=None, param_visibilities=None, accepts_messages=False,
assistant=False, tracking_url=None, worker=None, batchable=None,
batch_id=None, retry_policy_dict=None, owners=None, **kwargs):
"""
* add task identified by task_id if it doesn't exist
* if deps is not None, update dependency list
* update status of task
* add additional workers/stakeholders
* update priority when needed
"""
assert worker is not None
worker_id = worker
worker = self._update_worker(worker_id)
resources = {} if resources is None else resources.copy()
if retry_policy_dict is None:
retry_policy_dict = {}
retry_policy = self._generate_retry_policy(retry_policy_dict)
if worker.enabled:
_default_task = self._make_task(
task_id=task_id, status=PENDING, deps=deps, resources=resources,
priority=priority, family=family, module=module, params=params, param_visibilities=param_visibilities,
)
else:
_default_task = None
task = self._state.get_task(task_id, setdefault=_default_task)
if task is None or (task.status != RUNNING and not worker.enabled):
return
# for setting priority, we'll sometimes create tasks with unset family and params
if not task.family:
task.family = family
if not getattr(task, 'module', None):
task.module = module
if not getattr(task, 'param_visibilities', None):
task.param_visibilities = _get_default(param_visibilities, {})
if not task.params:
task.set_params(params)
if batch_id is not None:
task.batch_id = batch_id
if status == RUNNING and not task.worker_running:
task.worker_running = worker_id
if batch_id:
# copy resources_running of the first batch task
batch_tasks = self._state.get_batch_running_tasks(batch_id)
task.resources_running = batch_tasks[0].resources_running.copy()
task.time_running = time.time()
if accepts_messages is not None:
task.accepts_messages = accepts_messages
if tracking_url is not None or task.status != RUNNING:
task.tracking_url = tracking_url
if task.batch_id is not None:
for batch_task in self._state.get_batch_running_tasks(task.batch_id):
batch_task.tracking_url = tracking_url
if batchable is not None:
task.batchable = batchable
if task.remove is not None:
task.remove = None # unmark task for removal so it isn't removed after being added
if expl is not None:
task.expl = expl
if task.batch_id is not None:
for batch_task in self._state.get_batch_running_tasks(task.batch_id):
batch_task.expl = expl
task_is_not_running = task.status not in (RUNNING, BATCH_RUNNING)
task_started_a_run = status in (DONE, FAILED, RUNNING)
running_on_this_worker = task.worker_running == worker_id
if task_is_not_running or (task_started_a_run and running_on_this_worker) or new_deps:
# don't allow re-scheduling of task while it is running, it must either fail or succeed on the worker actually running it
if status != task.status or status == PENDING:
# Update the DB only if there was a acctual change, to prevent noise.
# We also check for status == PENDING b/c that's the default value
# (so checking for status != task.status woule lie)
self._update_task_history(task, status)
self._state.set_status(task, PENDING if status == SUSPENDED else status, self._config)
if status == FAILED and self._config.batch_emails:
batched_params, _ = self._state.get_batcher(worker_id, family)
if batched_params:
unbatched_params = {
param: value
for param, value in six.iteritems(task.params)
if param not in batched_params
}
else:
unbatched_params = task.params
try:
expl_raw = json.loads(expl)
except ValueError:
expl_raw = expl
self._email_batcher.add_failure(
task.pretty_id, task.family, unbatched_params, expl_raw, owners)
if task.status == DISABLED:
self._email_batcher.add_disable(
task.pretty_id, task.family, unbatched_params, owners)
if deps is not None:
task.deps = set(deps)
if new_deps is not None:
task.deps.update(new_deps)
if resources is not None:
task.resources = resources
if worker.enabled and not assistant:
task.stakeholders.add(worker_id)
# Task dependencies might not exist yet. Let's create dummy tasks for them for now.
# Otherwise the task dependencies might end up being pruned if scheduling takes a long time
for dep in task.deps or []:
t = self._state.get_task(dep, setdefault=self._make_task(task_id=dep, status=UNKNOWN, deps=None, priority=priority))
t.stakeholders.add(worker_id)
self._update_priority(task, priority, worker_id)
# Because some tasks (non-dynamic dependencies) are `_make_task`ed
# before we know their retry_policy, we always set it here
task.retry_policy = retry_policy
if runnable and status != FAILED and worker.enabled:
task.workers.add(worker_id)
self._state.get_worker(worker_id).tasks.add(task)
task.runnable = runnable
@rpc_method()
def announce_scheduling_failure(self, task_name, family, params, expl, owners, **kwargs):
if not self._config.batch_emails:
return
worker_id = kwargs['worker']
batched_params, _ = self._state.get_batcher(worker_id, family)
if batched_params:
unbatched_params = {
param: value
for param, value in six.iteritems(params)
if param not in batched_params
}
else:
unbatched_params = params
self._email_batcher.add_scheduling_fail(task_name, family, unbatched_params, expl, owners)
@rpc_method()
def add_worker(self, worker, info, **kwargs):
self._state.get_worker(worker).add_info(info)
@rpc_method()
def disable_worker(self, worker):
self._state.disable_workers({worker})
@rpc_method()
def set_worker_processes(self, worker, n):
self._state.get_worker(worker).add_rpc_message('set_worker_processes', n=n)
@rpc_method()
def send_scheduler_message(self, worker, task, content):
if not self._config.send_messages:
return {"message_id": None}
message_id = str(uuid.uuid4())
self._state.get_worker(worker).add_rpc_message('dispatch_scheduler_message', task_id=task,
message_id=message_id, content=content)
return {"message_id": message_id}
@rpc_method()
def add_scheduler_message_response(self, task_id, message_id, response):
if self._state.has_task(task_id):
task = self._state.get_task(task_id)
task.scheduler_message_responses[message_id] = response
@rpc_method()
def get_scheduler_message_response(self, task_id, message_id):
response = None
if self._state.has_task(task_id):
task = self._state.get_task(task_id)
response = task.scheduler_message_responses.pop(message_id, None)
return {"response": response}
@rpc_method()
def is_pause_enabled(self):