-
Notifications
You must be signed in to change notification settings - Fork 13.7k
/
dag.py
1633 lines (1422 loc) · 60.3 KB
/
dag.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 -*-
#
# Licensed to the Apache Software Foundation (ASF) under one
# or more contributor license agreements. See the NOTICE file
# distributed with this work for additional information
# regarding copyright ownership. The ASF licenses this file
# to you 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.
import copy
import functools
import os
import pickle
import re
import sys
import traceback
import warnings
from collections import OrderedDict, defaultdict
from datetime import datetime, timedelta
from typing import TYPE_CHECKING, Callable, Dict, Iterable, List, Optional, Type, Union
import jinja2
import pendulum
from croniter import croniter
from dateutil.relativedelta import relativedelta
from sqlalchemy import Boolean, Column, Integer, String, Text, func, or_
from airflow import settings, utils
from airflow.configuration import conf
from airflow.dag.base_dag import BaseDag
from airflow.exceptions import AirflowDagCycleException, AirflowException
from airflow.executors import LocalExecutor, get_default_executor
from airflow.models.base import ID_LEN, Base
from airflow.models.dagbag import DagBag
from airflow.models.dagpickle import DagPickle
from airflow.models.dagrun import DagRun
from airflow.models.taskinstance import TaskInstance, clear_task_instances
from airflow.utils import timezone
from airflow.utils.dates import cron_presets, date_range as utils_date_range
from airflow.utils.db import provide_session
from airflow.utils.helpers import validate_key
from airflow.utils.log.logging_mixin import LoggingMixin
from airflow.utils.sqlalchemy import Interval, UtcDateTime
from airflow.utils.state import State
if TYPE_CHECKING:
from airflow.models.baseoperator import BaseOperator # Avoid circular dependency
ScheduleInterval = Union[str, timedelta, relativedelta]
def get_last_dagrun(dag_id, session, include_externally_triggered=False):
"""
Returns the last dag run for a dag, None if there was none.
Last dag run can be any type of run eg. scheduled or backfilled.
Overridden DagRuns are ignored.
"""
DR = DagRun
query = session.query(DR).filter(DR.dag_id == dag_id)
if not include_externally_triggered:
query = query.filter(DR.external_trigger == False) # noqa pylint: disable=singleton-comparison
query = query.order_by(DR.execution_date.desc())
return query.first()
@functools.total_ordering
class DAG(BaseDag, LoggingMixin):
"""
A dag (directed acyclic graph) is a collection of tasks with directional
dependencies. A dag also has a schedule, a start date and an end date
(optional). For each schedule, (say daily or hourly), the DAG needs to run
each individual tasks as their dependencies are met. Certain tasks have
the property of depending on their own past, meaning that they can't run
until their previous schedule (and upstream tasks) are completed.
DAGs essentially act as namespaces for tasks. A task_id can only be
added once to a DAG.
:param dag_id: The id of the DAG
:type dag_id: str
:param description: The description for the DAG to e.g. be shown on the webserver
:type description: str
:param schedule_interval: Defines how often that DAG runs, this
timedelta object gets added to your latest task instance's
execution_date to figure out the next schedule
:type schedule_interval: datetime.timedelta or
dateutil.relativedelta.relativedelta or str that acts as a cron
expression
:param start_date: The timestamp from which the scheduler will
attempt to backfill
:type start_date: datetime.datetime
:param end_date: A date beyond which your DAG won't run, leave to None
for open ended scheduling
:type end_date: datetime.datetime
:param template_searchpath: This list of folders (non relative)
defines where jinja will look for your templates. Order matters.
Note that jinja/airflow includes the path of your DAG file by
default
:type template_searchpath: str or list[str]
:param template_undefined: Template undefined type.
:type template_undefined: jinja2.Undefined
:param user_defined_macros: a dictionary of macros that will be exposed
in your jinja templates. For example, passing ``dict(foo='bar')``
to this argument allows you to ``{{ foo }}`` in all jinja
templates related to this DAG. Note that you can pass any
type of object here.
:type user_defined_macros: dict
:param user_defined_filters: a dictionary of filters that will be exposed
in your jinja templates. For example, passing
``dict(hello=lambda name: 'Hello %s' % name)`` to this argument allows
you to ``{{ 'world' | hello }}`` in all jinja templates related to
this DAG.
:type user_defined_filters: dict
:param default_args: A dictionary of default parameters to be used
as constructor keyword parameters when initialising operators.
Note that operators have the same hook, and precede those defined
here, meaning that if your dict contains `'depends_on_past': True`
here and `'depends_on_past': False` in the operator's call
`default_args`, the actual value will be `False`.
:type default_args: dict
:param params: a dictionary of DAG level parameters that are made
accessible in templates, namespaced under `params`. These
params can be overridden at the task level.
:type params: dict
:param concurrency: the number of task instances allowed to run
concurrently
:type concurrency: int
:param max_active_runs: maximum number of active DAG runs, beyond this
number of DAG runs in a running state, the scheduler won't create
new active DAG runs
:type max_active_runs: int
:param dagrun_timeout: specify how long a DagRun should be up before
timing out / failing, so that new DagRuns can be created. The timeout
is only enforced for scheduled DagRuns, and only once the
# of active DagRuns == max_active_runs.
:type dagrun_timeout: datetime.timedelta
:param sla_miss_callback: specify a function to call when reporting SLA
timeouts.
:type sla_miss_callback: types.FunctionType
:param default_view: Specify DAG default view (tree, graph, duration,
gantt, landing_times)
:type default_view: str
:param orientation: Specify DAG orientation in graph view (LR, TB, RL, BT)
:type orientation: str
:param catchup: Perform scheduler catchup (or only run latest)? Defaults to True
:type catchup: bool
:param on_failure_callback: A function to be called when a DagRun of this dag fails.
A context dictionary is passed as a single parameter to this function.
:type on_failure_callback: callable
:param on_success_callback: Much like the ``on_failure_callback`` except
that it is executed when the dag succeeds.
:type on_success_callback: callable
:param access_control: Specify optional DAG-level permissions, e.g.,
"{'role1': {'can_dag_read'}, 'role2': {'can_dag_read', 'can_dag_edit'}}"
:type access_control: dict
:param is_paused_upon_creation: Specifies if the dag is paused when created for the first time.
If the dag exists already, this flag will be ignored. If this optional parameter
is not specified, the global config setting will be used.
:type is_paused_upon_creation: bool or None
:param jinja_environment_kwargs: additional configuration options to be passed to Jinja
``Environment`` for template rendering
**Example**: to avoid Jinja from removing a trailing newline from template strings ::
DAG(dag_id='my-dag',
jinja_environment_kwargs={
'keep_trailing_newline': True,
# some other jinja2 Environment options here
}
)
**See**: `Jinja Environment documentation
<https://jinja.palletsprojects.com/en/master/api/#jinja2.Environment>`_
:type jinja_environment_kwargs: dict
"""
_comps = {
'dag_id',
'task_ids',
'parent_dag',
'start_date',
'schedule_interval',
'full_filepath',
'template_searchpath',
'last_loaded',
}
def __init__(
self,
dag_id: str,
description: str = '',
schedule_interval: Optional[ScheduleInterval] = timedelta(days=1),
start_date: Optional[datetime] = None,
end_date: Optional[datetime] = None,
full_filepath: Optional[str] = None,
template_searchpath: Optional[Union[str, Iterable[str]]] = None,
template_undefined: Type[jinja2.Undefined] = jinja2.Undefined,
user_defined_macros: Optional[Dict] = None,
user_defined_filters: Optional[Dict] = None,
default_args: Optional[Dict] = None,
concurrency: int = conf.getint('core', 'dag_concurrency'),
max_active_runs: int = conf.getint('core', 'max_active_runs_per_dag'),
dagrun_timeout: Optional[timedelta] = None,
sla_miss_callback: Optional[Callable] = None,
default_view: Optional[str] = None,
orientation: str = conf.get('webserver', 'dag_orientation'),
catchup: bool = conf.getboolean('scheduler', 'catchup_by_default'),
on_success_callback: Optional[Callable] = None,
on_failure_callback: Optional[Callable] = None,
doc_md: Optional[str] = None,
params: Optional[Dict] = None,
access_control: Optional[Dict] = None,
is_paused_upon_creation: Optional[bool] = None,
jinja_environment_kwargs: Optional[Dict] = None
):
self.user_defined_macros = user_defined_macros
self.user_defined_filters = user_defined_filters
self.default_args = copy.deepcopy(default_args or {})
self.params = params or {}
# merging potentially conflicting default_args['params'] into params
if 'params' in self.default_args:
self.params.update(self.default_args['params'])
del self.default_args['params']
validate_key(dag_id)
# Properties from BaseDag
self._dag_id = dag_id
self._full_filepath = full_filepath if full_filepath else ''
self._concurrency = concurrency
self._pickle_id = None
self._description = description
# set file location to caller source path
self.fileloc = sys._getframe().f_back.f_code.co_filename
self.task_dict = dict() # type: Dict[str, BaseOperator]
# set timezone from start_date
if start_date and start_date.tzinfo:
self.timezone = start_date.tzinfo
elif 'start_date' in self.default_args and self.default_args['start_date']:
if isinstance(self.default_args['start_date'], str):
self.default_args['start_date'] = (
timezone.parse(self.default_args['start_date'])
)
self.timezone = self.default_args['start_date'].tzinfo
if not hasattr(self, 'timezone') or not self.timezone:
self.timezone = settings.TIMEZONE
# Apply the timezone we settled on to end_date if it wasn't supplied
if 'end_date' in self.default_args and self.default_args['end_date']:
if isinstance(self.default_args['end_date'], str):
self.default_args['end_date'] = (
timezone.parse(self.default_args['end_date'], timezone=self.timezone)
)
self.start_date = timezone.convert_to_utc(start_date)
self.end_date = timezone.convert_to_utc(end_date)
# also convert tasks
if 'start_date' in self.default_args:
self.default_args['start_date'] = (
timezone.convert_to_utc(self.default_args['start_date'])
)
if 'end_date' in self.default_args:
self.default_args['end_date'] = (
timezone.convert_to_utc(self.default_args['end_date'])
)
self.schedule_interval = schedule_interval
if isinstance(schedule_interval, str) and schedule_interval in cron_presets:
self._schedule_interval = cron_presets.get(schedule_interval) # type: Optional[ScheduleInterval]
elif schedule_interval == '@once':
self._schedule_interval = None
else:
self._schedule_interval = schedule_interval
if isinstance(template_searchpath, str):
template_searchpath = [template_searchpath]
self.template_searchpath = template_searchpath
self.template_undefined = template_undefined
self.parent_dag = None # Gets set when DAGs are loaded
self.last_loaded = timezone.utcnow()
self.safe_dag_id = dag_id.replace('.', '__dot__')
self.max_active_runs = max_active_runs
self.dagrun_timeout = dagrun_timeout
self.sla_miss_callback = sla_miss_callback
self._default_view = default_view
self.orientation = orientation
self.catchup = catchup
self.is_subdag = False # DagBag.bag_dag() will set this to True if appropriate
self.partial = False
self.on_success_callback = on_success_callback
self.on_failure_callback = on_failure_callback
self.doc_md = doc_md
self._old_context_manager_dags = [] # type: Iterable[DAG]
self._access_control = access_control
self.is_paused_upon_creation = is_paused_upon_creation
self.jinja_environment_kwargs = jinja_environment_kwargs
def __repr__(self):
return "<DAG: {self.dag_id}>".format(self=self)
def __eq__(self, other):
if (type(self) == type(other) and
self.dag_id == other.dag_id):
# Use getattr() instead of __dict__ as __dict__ doesn't return
# correct values for properties.
return all(getattr(self, c, None) == getattr(other, c, None) for c in self._comps)
return False
def __ne__(self, other):
return not self == other
def __lt__(self, other):
return self.dag_id < other.dag_id
def __hash__(self):
hash_components = [type(self)]
for c in self._comps:
# task_ids returns a list and lists can't be hashed
if c == 'task_ids':
val = tuple(self.task_dict.keys())
else:
val = getattr(self, c, None)
try:
hash(val)
hash_components.append(val)
except TypeError:
hash_components.append(repr(val))
return hash(tuple(hash_components))
# Context Manager -----------------------------------------------
def __enter__(self):
self._old_context_manager_dags.append(settings.CONTEXT_MANAGER_DAG)
settings.CONTEXT_MANAGER_DAG = self
return self
def __exit__(self, _type, _value, _tb):
settings.CONTEXT_MANAGER_DAG = self._old_context_manager_dags.pop()
# /Context Manager ----------------------------------------------
def get_default_view(self):
"""This is only there for backward compatible jinja2 templates"""
if self._default_view is None:
return conf.get('webserver', 'dag_default_view').lower()
else:
return self._default_view
def date_range(self, start_date, num=None, end_date=timezone.utcnow()):
if num:
end_date = None
return utils_date_range(
start_date=start_date, end_date=end_date,
num=num, delta=self._schedule_interval)
def is_fixed_time_schedule(self):
"""
Figures out if the DAG schedule has a fixed time (e.g. 3 AM).
:return: True if the schedule has a fixed time, False if not.
"""
now = datetime.now()
cron = croniter(self._schedule_interval, now)
start = cron.get_next(datetime)
cron_next = cron.get_next(datetime)
if cron_next.minute == start.minute and cron_next.hour == start.hour:
return True
return False
def following_schedule(self, dttm):
"""
Calculates the following schedule for this dag in UTC.
:param dttm: utc datetime
:return: utc datetime
"""
if isinstance(self._schedule_interval, str):
# we don't want to rely on the transitions created by
# croniter as they are not always correct
dttm = pendulum.instance(dttm)
naive = timezone.make_naive(dttm, self.timezone)
cron = croniter(self._schedule_interval, naive)
# We assume that DST transitions happen on the minute/hour
if not self.is_fixed_time_schedule():
# relative offset (eg. every 5 minutes)
delta = cron.get_next(datetime) - naive
following = dttm.in_timezone(self.timezone).add_timedelta(delta)
else:
# absolute (e.g. 3 AM)
naive = cron.get_next(datetime)
tz = pendulum.timezone(self.timezone.name)
following = timezone.make_aware(naive, tz)
return timezone.convert_to_utc(following)
elif self._schedule_interval is not None:
return dttm + self._schedule_interval
def previous_schedule(self, dttm):
"""
Calculates the previous schedule for this dag in UTC
:param dttm: utc datetime
:return: utc datetime
"""
if isinstance(self._schedule_interval, str):
# we don't want to rely on the transitions created by
# croniter as they are not always correct
dttm = pendulum.instance(dttm)
naive = timezone.make_naive(dttm, self.timezone)
cron = croniter(self._schedule_interval, naive)
# We assume that DST transitions happen on the minute/hour
if not self.is_fixed_time_schedule():
# relative offset (eg. every 5 minutes)
delta = naive - cron.get_prev(datetime)
previous = dttm.in_timezone(self.timezone).subtract_timedelta(delta)
else:
# absolute (e.g. 3 AM)
naive = cron.get_prev(datetime)
tz = pendulum.timezone(self.timezone.name)
previous = timezone.make_aware(naive, tz)
return timezone.convert_to_utc(previous)
elif self._schedule_interval is not None:
return dttm - self._schedule_interval
def get_run_dates(self, start_date, end_date=None):
"""
Returns a list of dates between the interval received as parameter using this
dag's schedule interval. Returned dates can be used for execution dates.
:param start_date: the start date of the interval
:type start_date: datetime
:param end_date: the end date of the interval, defaults to timezone.utcnow()
:type end_date: datetime
:return: a list of dates within the interval following the dag's schedule
:rtype: list
"""
run_dates = []
using_start_date = start_date
using_end_date = end_date
# dates for dag runs
using_start_date = using_start_date or min([t.start_date for t in self.tasks])
using_end_date = using_end_date or timezone.utcnow()
# next run date for a subdag isn't relevant (schedule_interval for subdags
# is ignored) so we use the dag run's start date in the case of a subdag
next_run_date = (self.normalize_schedule(using_start_date)
if not self.is_subdag else using_start_date)
while next_run_date and next_run_date <= using_end_date:
run_dates.append(next_run_date)
next_run_date = self.following_schedule(next_run_date)
return run_dates
def normalize_schedule(self, dttm):
"""
Returns dttm + interval unless dttm is first interval then it returns dttm
"""
following = self.following_schedule(dttm)
# in case of @once
if not following:
return dttm
if self.previous_schedule(following) != dttm:
return following
return dttm
@provide_session
def get_last_dagrun(self, session=None, include_externally_triggered=False):
return get_last_dagrun(self.dag_id, session=session,
include_externally_triggered=include_externally_triggered)
@property
def dag_id(self):
return self._dag_id
@dag_id.setter
def dag_id(self, value):
self._dag_id = value
@property
def full_filepath(self):
return self._full_filepath
@full_filepath.setter
def full_filepath(self, value):
self._full_filepath = value
@property
def concurrency(self):
return self._concurrency
@concurrency.setter
def concurrency(self, value):
self._concurrency = value
@property
def access_control(self):
return self._access_control
@access_control.setter
def access_control(self, value):
self._access_control = value
@property
def description(self):
return self._description
@property
def pickle_id(self):
return self._pickle_id
@pickle_id.setter
def pickle_id(self, value):
self._pickle_id = value
@property
def tasks(self):
return list(self.task_dict.values())
@tasks.setter
def tasks(self, val):
raise AttributeError(
'DAG.tasks can not be modified. Use dag.add_task() instead.')
@property
def task_ids(self):
return list(self.task_dict.keys())
@property
def filepath(self):
"""
File location of where the dag object is instantiated
"""
fn = self.full_filepath.replace(settings.DAGS_FOLDER + '/', '')
fn = fn.replace(os.path.dirname(__file__) + '/', '')
return fn
@property
def folder(self):
"""Folder location of where the DAG object is instantiated."""
return os.path.dirname(self.full_filepath)
@property
def owner(self):
"""
Return list of all owners found in DAG tasks.
:return: Comma separated list of owners in DAG tasks
:rtype: str
"""
return ", ".join({t.owner for t in self.tasks})
@provide_session
def _get_concurrency_reached(self, session=None):
TI = TaskInstance
qry = session.query(func.count(TI.task_id)).filter(
TI.dag_id == self.dag_id,
TI.state == State.RUNNING,
)
return qry.scalar() >= self.concurrency
@property
def concurrency_reached(self):
"""
Returns a boolean indicating whether the concurrency limit for this DAG
has been reached
"""
return self._get_concurrency_reached()
@provide_session
def _get_is_paused(self, session=None):
qry = session.query(DagModel).filter(
DagModel.dag_id == self.dag_id)
return qry.value('is_paused')
@property
def is_paused(self):
"""
Returns a boolean indicating whether this DAG is paused
"""
return self._get_is_paused()
@provide_session
def handle_callback(self, dagrun, success=True, reason=None, session=None):
"""
Triggers the appropriate callback depending on the value of success, namely the
on_failure_callback or on_success_callback. This method gets the context of a
single TaskInstance part of this DagRun and passes that to the callable along
with a 'reason', primarily to differentiate DagRun failures.
.. note: The logs end up in
``$AIRFLOW_HOME/logs/scheduler/latest/PROJECT/DAG_FILE.py.log``
:param dagrun: DagRun object
:param success: Flag to specify if failure or success callback should be called
:param reason: Completion reason
:param session: Database session
"""
callback = self.on_success_callback if success else self.on_failure_callback
if callback:
self.log.info('Executing dag callback function: {}'.format(callback))
tis = dagrun.get_task_instances()
ti = tis[-1] # get first TaskInstance of DagRun
ti.task = self.get_task(ti.task_id)
context = ti.get_template_context(session=session)
context.update({'reason': reason})
callback(context)
def get_active_runs(self):
"""
Returns a list of dag run execution dates currently running
:return: List of execution dates
"""
runs = DagRun.find(dag_id=self.dag_id, state=State.RUNNING)
active_dates = []
for run in runs:
active_dates.append(run.execution_date)
return active_dates
@provide_session
def get_num_active_runs(self, external_trigger=None, session=None):
"""
Returns the number of active "running" dag runs
:param external_trigger: True for externally triggered active dag runs
:type external_trigger: bool
:param session:
:return: number greater than 0 for active dag runs
"""
query = (session
.query(DagRun)
.filter(DagRun.dag_id == self.dag_id)
.filter(DagRun.state == State.RUNNING))
if external_trigger is not None:
query = query.filter(DagRun.external_trigger == external_trigger)
return query.count()
@provide_session
def get_dagrun(self, execution_date, session=None):
"""
Returns the dag run for a given execution date if it exists, otherwise
none.
:param execution_date: The execution date of the DagRun to find.
:param session:
:return: The DagRun if found, otherwise None.
"""
dagrun = (
session.query(DagRun)
.filter(
DagRun.dag_id == self.dag_id,
DagRun.execution_date == execution_date)
.first())
return dagrun
@provide_session
def get_dagruns_between(self, start_date, end_date, session=None):
"""
Returns the list of dag runs between start_date (inclusive) and end_date (inclusive).
:param start_date: The starting execution date of the DagRun to find.
:param end_date: The ending execution date of the DagRun to find.
:param session:
:return: The list of DagRuns found.
"""
dagruns = (
session.query(DagRun)
.filter(
DagRun.dag_id == self.dag_id,
DagRun.execution_date >= start_date,
DagRun.execution_date <= end_date)
.all())
return dagruns
@provide_session
def _get_latest_execution_date(self, session=None):
return session.query(func.max(DagRun.execution_date)).filter(
DagRun.dag_id == self.dag_id
).scalar()
@property
def latest_execution_date(self):
"""
Returns the latest date for which at least one dag run exists
"""
return self._get_latest_execution_date()
@property
def subdags(self):
"""
Returns a list of the subdag objects associated to this DAG
"""
# Check SubDag for class but don't check class directly
from airflow.operators.subdag_operator import SubDagOperator
subdag_lst = []
for task in self.tasks:
if (isinstance(task, SubDagOperator) or
# TODO remove in Airflow 2.0
type(task).__name__ == 'SubDagOperator'):
subdag_lst.append(task.subdag)
subdag_lst += task.subdag.subdags
return subdag_lst
def resolve_template_files(self):
for t in self.tasks:
t.resolve_template_files()
def get_template_env(self) -> jinja2.Environment:
"""Build a Jinja2 environment."""
# Collect directories to search for template files
searchpath = [self.folder]
if self.template_searchpath:
searchpath += self.template_searchpath
# Default values (for backward compatibility)
jinja_env_options = {
'loader': jinja2.FileSystemLoader(searchpath),
'undefined': self.template_undefined,
'extensions': ["jinja2.ext.do"],
'cache_size': 0
}
if self.jinja_environment_kwargs:
jinja_env_options.update(self.jinja_environment_kwargs)
env = jinja2.Environment(**jinja_env_options) # type: ignore
# Add any user defined items. Safe to edit globals as long as no templates are rendered yet.
# http://jinja.pocoo.org/docs/2.10/api/#jinja2.Environment.globals
if self.user_defined_macros:
env.globals.update(self.user_defined_macros)
if self.user_defined_filters:
env.filters.update(self.user_defined_filters)
return env
def set_dependency(self, upstream_task_id, downstream_task_id):
"""
Simple utility method to set dependency between two tasks that
already have been added to the DAG using add_task()
"""
self.get_task(upstream_task_id).set_downstream(
self.get_task(downstream_task_id))
@provide_session
def get_task_instances(
self, start_date=None, end_date=None, state=None, session=None):
if not start_date:
start_date = (timezone.utcnow() - timedelta(30)).date()
start_date = timezone.make_aware(
datetime.combine(start_date, datetime.min.time()))
end_date = end_date or timezone.utcnow()
tis = session.query(TaskInstance).filter(
TaskInstance.dag_id == self.dag_id,
TaskInstance.execution_date >= start_date,
TaskInstance.execution_date <= end_date,
TaskInstance.task_id.in_([t.task_id for t in self.tasks]),
)
if state:
if isinstance(state, str):
tis = tis.filter(TaskInstance.state == state)
else:
# this is required to deal with NULL values
if None in state:
tis = tis.filter(
or_(TaskInstance.state.in_(state),
TaskInstance.state.is_(None))
)
else:
tis = tis.filter(TaskInstance.state.in_(state))
tis = tis.order_by(TaskInstance.execution_date).all()
return tis
@property
def roots(self) -> List["BaseOperator"]:
"""Return nodes with no parents. These are first to execute and are called roots or root nodes."""
return [task for task in self.tasks if not task.upstream_list]
@property
def leaves(self) -> List["BaseOperator"]:
"""Return nodes with no children. These are last to execute and are called leaves or leaf nodes."""
return [task for task in self.tasks if not task.downstream_list]
def topological_sort(self, include_subdag_tasks: bool = False):
"""
Sorts tasks in topographical order, such that a task comes after any of its
upstream dependencies.
Heavily inspired by:
http://blog.jupo.org/2012/04/06/topological-sorting-acyclic-directed-graphs/
:param include_subdag_tasks: whether to include tasks in subdags, default to False
:return: list of tasks in topological order
"""
from airflow.operators.subdag_operator import SubDagOperator # Avoid circular import
# convert into an OrderedDict to speedup lookup while keeping order the same
graph_unsorted = OrderedDict((task.task_id, task) for task in self.tasks)
graph_sorted = [] # type: List[BaseOperator]
# special case
if len(self.tasks) == 0:
return tuple(graph_sorted)
# Run until the unsorted graph is empty.
while graph_unsorted:
# Go through each of the node/edges pairs in the unsorted
# graph. If a set of edges doesn't contain any nodes that
# haven't been resolved, that is, that are still in the
# unsorted graph, remove the pair from the unsorted graph,
# and append it to the sorted graph. Note here that by using
# using the items() method for iterating, a copy of the
# unsorted graph is used, allowing us to modify the unsorted
# graph as we move through it. We also keep a flag for
# checking that that graph is acyclic, which is true if any
# nodes are resolved during each pass through the graph. If
# not, we need to bail out as the graph therefore can't be
# sorted.
acyclic = False
for node in list(graph_unsorted.values()):
for edge in node.upstream_list:
if edge.task_id in graph_unsorted:
break
# no edges in upstream tasks
else:
acyclic = True
del graph_unsorted[node.task_id]
graph_sorted.append(node)
if include_subdag_tasks and isinstance(node, SubDagOperator):
graph_sorted.extend(node.subdag.topological_sort(include_subdag_tasks=True))
if not acyclic:
raise AirflowException("A cyclic dependency occurred in dag: {}"
.format(self.dag_id))
return tuple(graph_sorted)
@provide_session
def set_dag_runs_state(
self,
state=State.RUNNING,
session=None,
start_date=None,
end_date=None,
):
query = session.query(DagRun).filter_by(dag_id=self.dag_id)
if start_date:
query = query.filter(DagRun.execution_date >= start_date)
if end_date:
query = query.filter(DagRun.execution_date <= end_date)
drs = query.all()
dirty_ids = []
for dr in drs:
dr.state = state
dirty_ids.append(dr.dag_id)
@provide_session
def clear(
self, start_date=None, end_date=None,
only_failed=False,
only_running=False,
confirm_prompt=False,
include_subdags=True,
include_parentdag=True,
reset_dag_runs=True,
dry_run=False,
session=None,
get_tis=False,
):
"""
Clears a set of task instances associated with the current dag for
a specified date range.
"""
TI = TaskInstance
tis = session.query(TI)
if include_subdags:
# Crafting the right filter for dag_id and task_ids combo
conditions = []
for dag in self.subdags + [self]:
conditions.append(
TI.dag_id.like(dag.dag_id) &
TI.task_id.in_(dag.task_ids)
)
tis = tis.filter(or_(*conditions))
else:
tis = session.query(TI).filter(TI.dag_id == self.dag_id)
tis = tis.filter(TI.task_id.in_(self.task_ids))
if include_parentdag and self.is_subdag:
p_dag = self.parent_dag.sub_dag(
task_regex=r"^{}$".format(self.dag_id.split('.')[1]),
include_upstream=False,
include_downstream=True)
tis = tis.union(p_dag.clear(
start_date=start_date, end_date=end_date,
only_failed=only_failed,
only_running=only_running,
confirm_prompt=confirm_prompt,
include_subdags=include_subdags,
include_parentdag=False,
reset_dag_runs=reset_dag_runs,
get_tis=True,
session=session,
))
if start_date:
tis = tis.filter(TI.execution_date >= start_date)
if end_date:
tis = tis.filter(TI.execution_date <= end_date)
if only_failed:
tis = tis.filter(or_(
TI.state == State.FAILED,
TI.state == State.UPSTREAM_FAILED))
if only_running:
tis = tis.filter(TI.state == State.RUNNING)
if get_tis:
return tis
if dry_run:
tis = tis.all()
session.expunge_all()
return tis
count = tis.count()
do_it = True
if count == 0:
return 0
if confirm_prompt:
ti_list = "\n".join([str(t) for t in tis])
question = (
"You are about to delete these {count} tasks:\n"
"{ti_list}\n\n"
"Are you sure? (yes/no): ").format(count=count, ti_list=ti_list)
do_it = utils.helpers.ask_yesno(question)
if do_it:
clear_task_instances(tis.all(),
session,
dag=self,
)
if reset_dag_runs:
self.set_dag_runs_state(session=session,
start_date=start_date,
end_date=end_date,
)
else:
count = 0
print("Bail. Nothing was cleared.")
session.commit()
return count
@classmethod
def clear_dags(
cls, dags,
start_date=None,
end_date=None,
only_failed=False,
only_running=False,
confirm_prompt=False,
include_subdags=True,