-
Notifications
You must be signed in to change notification settings - Fork 16.9k
Expand file tree
/
Copy pathdag.py
More file actions
2530 lines (2219 loc) · 99.3 KB
/
dag.py
File metadata and controls
2530 lines (2219 loc) · 99.3 KB
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
#
# 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.
from __future__ import annotations
import asyncio
import copy
import functools
import logging
import sys
import time
from collections import defaultdict
from collections.abc import Collection, Generator, Iterable, Sequence
from contextlib import ExitStack
from datetime import datetime, timedelta
from functools import cache
from re import Pattern
from typing import (
TYPE_CHECKING,
Any,
Callable,
TypeVar,
Union,
cast,
overload,
)
import attrs
import methodtools
import pendulum
import re2
import sqlalchemy_jsonfield
from dateutil.relativedelta import relativedelta
from packaging import version as packaging_version
from sqlalchemy import (
Boolean,
Column,
ForeignKey,
Index,
Integer,
String,
Text,
and_,
case,
func,
or_,
select,
tuple_,
update,
)
from sqlalchemy.ext.associationproxy import association_proxy
from sqlalchemy.ext.hybrid import hybrid_property
from sqlalchemy.orm import backref, load_only, relationship
from sqlalchemy.sql import Select, expression
from airflow import settings, utils
from airflow.configuration import conf as airflow_conf, secrets_backend_list
from airflow.exceptions import (
AirflowException,
TaskDeferred,
UnknownExecutorException,
)
from airflow.executors.executor_loader import ExecutorLoader
from airflow.models.asset import (
AssetDagRunQueue,
AssetModel,
)
from airflow.models.base import Base, StringID
from airflow.models.baseoperator import BaseOperator
from airflow.models.dag_version import DagVersion
from airflow.models.dagrun import RUN_ID_REGEX, DagRun
from airflow.models.taskinstance import (
Context,
TaskInstance,
TaskInstanceKey,
clear_task_instances,
)
from airflow.models.tasklog import LogTemplate
from airflow.sdk.definitions.asset import Asset, AssetAlias, AssetUniqueKey, BaseAsset
from airflow.sdk.definitions.dag import DAG as TaskSDKDag, dag as task_sdk_dag_decorator
from airflow.secrets.local_filesystem import LocalFilesystemBackend
from airflow.security import permissions
from airflow.settings import json
from airflow.stats import Stats
from airflow.timetables.base import DagRunInfo, DataInterval, TimeRestriction, Timetable
from airflow.timetables.interval import CronDataIntervalTimetable, DeltaDataIntervalTimetable
from airflow.timetables.simple import (
AssetTriggeredTimetable,
NullTimetable,
OnceTimetable,
)
from airflow.utils import timezone
from airflow.utils.dag_cycle_tester import check_cycle
from airflow.utils.log.logging_mixin import LoggingMixin
from airflow.utils.session import NEW_SESSION, provide_session
from airflow.utils.sqlalchemy import UtcDateTime, lock_rows, with_row_locks
from airflow.utils.state import DagRunState, State, TaskInstanceState
from airflow.utils.types import DagRunTriggeredByType, DagRunType
if TYPE_CHECKING:
from sqlalchemy.orm.query import Query
from sqlalchemy.orm.session import Session
from airflow.models.abstractoperator import TaskStateChangeCallback
from airflow.models.dagbag import DagBag
from airflow.models.operator import Operator
from airflow.serialization.serialized_objects import MaybeSerializedDAG
from airflow.typing_compat import Literal
log = logging.getLogger(__name__)
DEFAULT_VIEW_PRESETS = ["grid", "graph", "duration", "gantt", "landing_times"]
ORIENTATION_PRESETS = ["LR", "TB", "RL", "BT"]
AssetT = TypeVar("AssetT", bound=BaseAsset)
TAG_MAX_LEN = 100
DagStateChangeCallback = Callable[[Context], None]
ScheduleInterval = Union[None, str, timedelta, relativedelta]
ScheduleArg = Union[
ScheduleInterval,
Timetable,
BaseAsset,
Collection[Union["Asset", "AssetAlias"]],
]
class InconsistentDataInterval(AirflowException):
"""
Exception raised when a model populates data interval fields incorrectly.
The data interval fields should either both be None (for runs scheduled
prior to AIP-39), or both be datetime (for runs scheduled after AIP-39 is
implemented). This is raised if exactly one of the fields is None.
"""
_template = (
"Inconsistent {cls}: {start[0]}={start[1]!r}, {end[0]}={end[1]!r}, "
"they must be either both None or both datetime"
)
def __init__(self, instance: Any, start_field_name: str, end_field_name: str) -> None:
self._class_name = type(instance).__name__
self._start_field = (start_field_name, getattr(instance, start_field_name))
self._end_field = (end_field_name, getattr(instance, end_field_name))
def __str__(self) -> str:
return self._template.format(cls=self._class_name, start=self._start_field, end=self._end_field)
def _get_model_data_interval(
instance: Any,
start_field_name: str,
end_field_name: str,
) -> DataInterval | None:
start = timezone.coerce_datetime(getattr(instance, start_field_name))
end = timezone.coerce_datetime(getattr(instance, end_field_name))
if start is None:
if end is not None:
raise InconsistentDataInterval(instance, start_field_name, end_field_name)
return None
elif end is None:
raise InconsistentDataInterval(instance, start_field_name, end_field_name)
return DataInterval(start, end)
def get_last_dagrun(dag_id, session, include_externally_triggered=False):
"""
Return the last dag run for a dag, None if there was none.
Last dag run can be any type of run e.g. scheduled or backfilled.
Overridden DagRuns are ignored.
"""
DR = DagRun
query = select(DR).where(DR.dag_id == dag_id, DR.logical_date.is_not(None))
if not include_externally_triggered:
query = query.where(DR.external_trigger == expression.false())
query = query.order_by(DR.logical_date.desc())
return session.scalar(query.limit(1))
def get_asset_triggered_next_run_info(
dag_ids: list[str], *, session: Session
) -> dict[str, dict[str, int | str]]:
"""
Get next run info for a list of dag_ids.
Given a list of dag_ids, get string representing how close any that are asset triggered are
their next run, e.g. "1 of 2 assets updated".
"""
from airflow.models.asset import AssetDagRunQueue as ADRQ, DagScheduleAssetReference
return {
x.dag_id: {
"uri": x.uri,
"ready": x.ready,
"total": x.total,
}
for x in session.execute(
select(
DagScheduleAssetReference.dag_id,
# This is a dirty hack to workaround group by requiring an aggregate,
# since grouping by asset is not what we want to do here...but it works
case((func.count() == 1, func.max(AssetModel.uri)), else_="").label("uri"),
func.count().label("total"),
func.sum(case((ADRQ.target_dag_id.is_not(None), 1), else_=0)).label("ready"),
)
.join(
ADRQ,
and_(
ADRQ.asset_id == DagScheduleAssetReference.asset_id,
ADRQ.target_dag_id == DagScheduleAssetReference.dag_id,
),
isouter=True,
)
.join(AssetModel, AssetModel.id == DagScheduleAssetReference.asset_id)
.group_by(DagScheduleAssetReference.dag_id)
.where(DagScheduleAssetReference.dag_id.in_(dag_ids))
).all()
}
def _triggerer_is_healthy():
from airflow.jobs.triggerer_job_runner import TriggererJobRunner
job = TriggererJobRunner.most_recent_job()
return job and job.is_alive()
@provide_session
def _create_orm_dagrun(
*,
dag: DAG,
run_id: str,
logical_date: datetime | None,
data_interval: DataInterval | None,
run_after: datetime,
start_date: datetime | None,
external_trigger: bool,
conf: Any,
state: DagRunState | None,
run_type: DagRunType,
dag_version: DagVersion | None,
creating_job_id: int | None,
backfill_id: int | None,
triggered_by: DagRunTriggeredByType,
session: Session = NEW_SESSION,
) -> DagRun:
run = DagRun(
dag_id=dag.dag_id,
run_id=run_id,
logical_date=logical_date,
start_date=start_date,
run_after=run_after,
external_trigger=external_trigger,
conf=conf,
state=state,
run_type=run_type,
dag_version=dag_version,
creating_job_id=creating_job_id,
data_interval=data_interval,
triggered_by=triggered_by,
backfill_id=backfill_id,
bundle_version=session.scalar(select(DagModel.bundle_version).where(DagModel.dag_id == dag.dag_id)),
)
# Load defaults into the following two fields to ensure result can be serialized detached
run.log_template_id = int(session.scalar(select(func.max(LogTemplate.__table__.c.id))))
run.consumed_asset_events = []
session.add(run)
session.flush()
run.dag = dag
# create the associated task instances
# state is None at the moment of creation
run.verify_integrity(session=session)
return run
if TYPE_CHECKING:
dag = task_sdk_dag_decorator
else:
def dag(dag_id: str = "", **kwargs):
return task_sdk_dag_decorator(dag_id, __DAG_class=DAG, __warnings_stacklevel_delta=3, **kwargs)
def _convert_max_consecutive_failed_dag_runs(val: int) -> int:
if val == 0:
val = airflow_conf.getint("core", "max_consecutive_failed_dag_runs_per_dag")
if val < 0:
raise ValueError(
f"Invalid max_consecutive_failed_dag_runs: {val}."
f"Requires max_consecutive_failed_dag_runs >= 0"
)
return val
@functools.total_ordering
@attrs.define(hash=False, repr=False, eq=False, slots=False)
class DAG(TaskSDKDag, 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.
Note that if you plan to use time zones all the dates provided should be pendulum
dates. See :ref:`timezone_aware_dags`.
.. versionadded:: 2.4
The *schedule* argument to specify either time-based scheduling logic
(timetable), or asset-driven triggers.
.. versionchanged:: 3.0
The default value of *schedule* has been changed to *None* (no schedule).
The previous default was ``timedelta(days=1)``.
:param dag_id: The id of the DAG; must consist exclusively of alphanumeric
characters, dashes, dots and underscores (all ASCII)
:param description: The description for the DAG to e.g. be shown on the webserver
:param schedule: If provided, this defines the rules according to which DAG
runs are scheduled. Possible values include a cron expression string,
timedelta object, Timetable, or list of Asset objects.
See also :doc:`/howto/timetable`.
:param start_date: The timestamp from which the scheduler will
attempt to backfill. If this is not provided, backfilling must be done
manually with an explicit time range.
:param end_date: A date beyond which your DAG won't run, leave to None
for open-ended scheduling.
: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
:param template_undefined: Template undefined type.
: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.
: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.
: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`.
: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.
:param max_active_tasks: the number of task instances allowed to run
concurrently
: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
:param max_consecutive_failed_dag_runs: (experimental) maximum number of consecutive failed DAG runs,
beyond this the scheduler will disable the DAG
:param dagrun_timeout: Specify the duration a DagRun should be allowed to run before it times out or
fails. Task instances that are running when a DagRun is timed out will be marked as skipped.
:param sla_miss_callback: DEPRECATED - The SLA feature is removed in Airflow 3.0, to be replaced with a new implementation in 3.1
:param default_view: Specify DAG default view (grid, graph, duration,
gantt, landing_times), default grid
:param orientation: Specify DAG orientation in graph view (LR, TB, RL, BT), default LR
:param catchup: Perform scheduler catchup (or only run latest)? Defaults to True
:param on_failure_callback: A function or list of functions to be called when a DagRun of this dag fails.
A context dictionary is passed as a single parameter to this function.
:param on_success_callback: Much like the ``on_failure_callback`` except
that it is executed when the dag succeeds.
:param access_control: Specify optional DAG-level actions, e.g.,
"{'role1': {'can_read'}, 'role2': {'can_read', 'can_edit', 'can_delete'}}"
or it can specify the resource name if there is a DAGs Run resource, e.g.,
"{'role1': {'DAG Runs': {'can_create'}}, 'role2': {'DAGs': {'can_read', 'can_edit', 'can_delete'}}"
: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.
: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/2.11.x/api/#jinja2.Environment>`_
:param render_template_as_native_obj: If True, uses a Jinja ``NativeEnvironment``
to render templates as native Python types. If False, a Jinja
``Environment`` is used to render templates as string values.
:param tags: List of tags to help filtering DAGs in the UI.
:param owner_links: Dict of owners and their links, that will be clickable on the DAGs view UI.
Can be used as an HTTP link (for example the link to your Slack channel), or a mailto link.
e.g: {"dag_owner": "https://airflow.apache.org/"}
:param auto_register: Automatically register this DAG when it is used in a ``with`` block
:param fail_fast: Fails currently running tasks when task in DAG fails.
**Warning**: A fail fast dag can only have tasks with the default trigger rule ("all_success").
An exception will be thrown if any task in a fail fast dag has a non default trigger rule.
:param dag_display_name: The display name of the DAG which appears on the UI.
"""
partial: bool = False
last_loaded: datetime | None = attrs.field(factory=timezone.utcnow)
default_view: str = airflow_conf.get_mandatory_value("webserver", "dag_default_view").lower()
orientation: str = airflow_conf.get_mandatory_value("webserver", "dag_orientation")
# this will only be set at serialization time
# it's only use is for determining the relative fileloc based only on the serialize dag
_processor_dags_folder: str | None = attrs.field(init=False, default=None)
# Override the default from parent class to use config
max_consecutive_failed_dag_runs: int = attrs.field(
default=0,
converter=_convert_max_consecutive_failed_dag_runs,
validator=attrs.validators.instance_of(int),
)
@property
def safe_dag_id(self):
return self.dag_id.replace(".", "__dot__")
def validate(self):
super().validate()
self.validate_executor_field()
def validate_executor_field(self):
for task in self.tasks:
if task.executor:
try:
ExecutorLoader.lookup_executor_name_by_str(task.executor)
except UnknownExecutorException:
raise UnknownExecutorException(
f"The specified executor {task.executor} for task {task.task_id} is not "
"configured. Review the core.executors Airflow configuration to add it or "
"update the executor configuration for this task."
)
@staticmethod
def _upgrade_outdated_dag_access_control(access_control=None):
"""Look for outdated dag level actions in DAG access_controls and replace them with updated actions."""
if access_control is None:
return None
from airflow.providers.fab import __version__ as FAB_VERSION
updated_access_control = {}
for role, perms in access_control.items():
if packaging_version.parse(FAB_VERSION) >= packaging_version.parse("1.3.0"):
updated_access_control[role] = updated_access_control.get(role, {})
if isinstance(perms, (set, list)):
# Support for old-style access_control where only the actions are specified
updated_access_control[role][permissions.RESOURCE_DAG] = set(perms)
else:
updated_access_control[role] = perms
elif isinstance(perms, dict):
# Not allow new access control format with old FAB versions
raise AirflowException(
"Please upgrade the FAB provider to a version >= 1.3.0 to allow "
"use the Dag Level Access Control new format."
)
else:
updated_access_control[role] = set(perms)
return updated_access_control
def get_next_data_interval(self, dag_model: DagModel) -> DataInterval | None:
"""
Get the data interval of the next scheduled run.
For compatibility, this method infers the data interval from the DAG's
schedule if the run does not have an explicit one set, which is possible
for runs created prior to AIP-39.
This function is private to Airflow core and should not be depended on as a
part of the Python API.
:meta private:
"""
if self.dag_id != dag_model.dag_id:
raise ValueError(f"Arguments refer to different DAGs: {self.dag_id} != {dag_model.dag_id}")
if dag_model.next_dagrun is None: # Next run not scheduled.
return None
data_interval = dag_model.next_dagrun_data_interval
if data_interval is not None:
return data_interval
# Compatibility: A run was scheduled without an explicit data interval.
# This means the run was scheduled before AIP-39 implementation. Try to
# infer from the logical date.
return self.infer_automated_data_interval(dag_model.next_dagrun)
def get_run_data_interval(self, run: DagRun) -> DataInterval:
"""
Get the data interval of this run.
For compatibility, this method infers the data interval from the DAG's
schedule if the run does not have an explicit one set, which is possible for
runs created prior to AIP-39.
This function is private to Airflow core and should not be depended on as a
part of the Python API.
:meta private:
"""
if run.dag_id is not None and run.dag_id != self.dag_id:
raise ValueError(f"Arguments refer to different DAGs: {self.dag_id} != {run.dag_id}")
data_interval = _get_model_data_interval(run, "data_interval_start", "data_interval_end")
if data_interval is not None:
return data_interval
# Compatibility: runs created before AIP-39 implementation don't have an
# explicit data interval. Try to infer from the logical date.
return self.infer_automated_data_interval(run.logical_date)
def infer_automated_data_interval(self, logical_date: datetime) -> DataInterval:
"""
Infer a data interval for a run against this DAG.
This method is used to bridge runs created prior to AIP-39
implementation, which do not have an explicit data interval. Therefore,
this method only considers ``schedule_interval`` values valid prior to
Airflow 2.2.
DO NOT call this method if there is a known data interval.
:meta private:
"""
timetable_type = type(self.timetable)
if issubclass(timetable_type, (NullTimetable, OnceTimetable, AssetTriggeredTimetable)):
return DataInterval.exact(timezone.coerce_datetime(logical_date))
start = timezone.coerce_datetime(logical_date)
if issubclass(timetable_type, CronDataIntervalTimetable):
end = cast(CronDataIntervalTimetable, self.timetable)._get_next(start)
elif issubclass(timetable_type, DeltaDataIntervalTimetable):
end = cast(DeltaDataIntervalTimetable, self.timetable)._get_next(start)
# Contributors: When the exception below is raised, you might want to
# add an 'elif' block here to handle custom timetables. Stop! The bug
# you're looking for is instead at when the DAG run (represented by
# logical_date) was created. See GH-31969 for an example:
# * Wrong fix: GH-32074 (modifies this function).
# * Correct fix: GH-32118 (modifies the DAG run creation code).
else:
raise ValueError(f"Not a valid timetable: {self.timetable!r}")
return DataInterval(start, end)
def next_dagrun_info(
self,
last_automated_dagrun: None | DataInterval,
*,
restricted: bool = True,
) -> DagRunInfo | None:
"""
Get information about the next DagRun of this dag after ``date_last_automated_dagrun``.
This calculates what time interval the next DagRun should operate on
(its logical date) and when it can be scheduled, according to the
dag's timetable, start_date, end_date, etc. This doesn't check max
active run or any other "max_active_tasks" type limits, but only
performs calculations based on the various date and interval fields of
this dag and its tasks.
:param last_automated_dagrun: The ``max(logical_date)`` of
existing "automated" DagRuns for this dag (scheduled or backfill,
but not manual).
:param restricted: If set to *False* (default is *True*), ignore
``start_date``, ``end_date``, and ``catchup`` specified on the DAG
or tasks.
:return: DagRunInfo of the next dagrun, or None if a dagrun is not
going to be scheduled.
"""
data_interval = None
if isinstance(last_automated_dagrun, datetime):
raise ValueError(
"Passing a datetime to DAG.next_dagrun_info is not supported anymore. Use a DataInterval instead."
)
data_interval = last_automated_dagrun
if restricted:
restriction = self._time_restriction
else:
restriction = TimeRestriction(earliest=None, latest=None, catchup=True)
try:
info = self.timetable.next_dagrun_info(
last_automated_data_interval=data_interval,
restriction=restriction,
)
except Exception:
self.log.exception(
"Failed to fetch run info after data interval %s for DAG %r",
data_interval,
self.dag_id,
)
info = None
return info
@functools.cached_property
def _time_restriction(self) -> TimeRestriction:
start_dates = [t.start_date for t in self.tasks if t.start_date]
if self.start_date is not None:
start_dates.append(self.start_date)
earliest = None
if start_dates:
earliest = timezone.coerce_datetime(min(start_dates))
latest = timezone.coerce_datetime(self.end_date)
end_dates = [t.end_date for t in self.tasks if t.end_date]
if len(end_dates) == len(self.tasks): # not exists null end_date
if self.end_date is not None:
end_dates.append(self.end_date)
if end_dates:
latest = timezone.coerce_datetime(max(end_dates))
return TimeRestriction(earliest, latest, self.catchup)
def iter_dagrun_infos_between(
self,
earliest: pendulum.DateTime | datetime | None,
latest: pendulum.DateTime | datetime,
*,
align: bool = True,
) -> Iterable[DagRunInfo]:
"""
Yield DagRunInfo using this DAG's timetable between given interval.
DagRunInfo instances yielded if their ``logical_date`` is not earlier
than ``earliest``, nor later than ``latest``. The instances are ordered
by their ``logical_date`` from earliest to latest.
If ``align`` is ``False``, the first run will happen immediately on
``earliest``, even if it does not fall on the logical timetable schedule.
The default is ``True``.
Example: A DAG is scheduled to run every midnight (``0 0 * * *``). If
``earliest`` is ``2021-06-03 23:00:00``, the first DagRunInfo would be
``2021-06-03 23:00:00`` if ``align=False``, and ``2021-06-04 00:00:00``
if ``align=True``.
"""
if earliest is None:
earliest = self._time_restriction.earliest
if earliest is None:
raise ValueError("earliest was None and we had no value in time_restriction to fallback on")
earliest = timezone.coerce_datetime(earliest)
latest = timezone.coerce_datetime(latest)
restriction = TimeRestriction(earliest, latest, catchup=True)
try:
info = self.timetable.next_dagrun_info(
last_automated_data_interval=None,
restriction=restriction,
)
except Exception:
self.log.exception(
"Failed to fetch run info after data interval %s for DAG %r",
None,
self.dag_id,
)
info = None
if info is None:
# No runs to be scheduled between the user-supplied timeframe. But
# if align=False, "invent" a data interval for the timeframe itself.
if not align:
yield DagRunInfo.interval(earliest, latest)
return
# If align=False and earliest does not fall on the timetable's logical
# schedule, "invent" a data interval for it.
if not align and info.logical_date != earliest:
yield DagRunInfo.interval(earliest, info.data_interval.start)
# Generate naturally according to schedule.
while info is not None:
yield info
try:
info = self.timetable.next_dagrun_info(
last_automated_data_interval=info.data_interval,
restriction=restriction,
)
except Exception:
self.log.exception(
"Failed to fetch run info after data interval %s for DAG %r",
info.data_interval if info else "<NONE>",
self.dag_id,
)
break
@provide_session
def get_last_dagrun(self, session=NEW_SESSION, include_externally_triggered=False):
return get_last_dagrun(
self.dag_id, session=session, include_externally_triggered=include_externally_triggered
)
@provide_session
def has_dag_runs(self, session=NEW_SESSION, include_externally_triggered=True) -> bool:
return (
get_last_dagrun(
self.dag_id, session=session, include_externally_triggered=include_externally_triggered
)
is not None
)
@property
def dag_id(self) -> str:
return self._dag_id
@dag_id.setter
def dag_id(self, value: str) -> None:
self._dag_id = value
@property
def timetable_summary(self) -> str:
return self.timetable.summary
@provide_session
def get_concurrency_reached(self, session=NEW_SESSION) -> bool:
"""Return a boolean indicating whether the max_active_tasks limit for this DAG has been reached."""
TI = TaskInstance
total_tasks = session.scalar(
select(func.count(TI.task_id)).where(
TI.dag_id == self.dag_id,
TI.state == TaskInstanceState.RUNNING,
)
)
return total_tasks >= self.max_active_tasks
@provide_session
def get_is_active(self, session=NEW_SESSION) -> None:
"""Return a boolean indicating whether this DAG is active."""
return session.scalar(select(DagModel.is_active).where(DagModel.dag_id == self.dag_id))
@provide_session
def get_is_paused(self, session=NEW_SESSION) -> None:
"""Return a boolean indicating whether this DAG is paused."""
return session.scalar(select(DagModel.is_paused).where(DagModel.dag_id == self.dag_id))
@provide_session
def get_bundle_name(self, session=NEW_SESSION) -> str | None:
"""Return the bundle name this DAG is in."""
return session.scalar(select(DagModel.bundle_name).where(DagModel.dag_id == self.dag_id))
@provide_session
def get_bundle_version(self, session=NEW_SESSION) -> str | None:
"""Return the bundle version that was seen when this dag was processed."""
return session.scalar(select(DagModel.bundle_version).where(DagModel.dag_id == self.dag_id))
@methodtools.lru_cache(maxsize=None)
@classmethod
def get_serialized_fields(cls):
"""Stringified DAGs and operators contain exactly these fields."""
return TaskSDKDag.get_serialized_fields() | {"_processor_dags_folder"}
@staticmethod
@provide_session
def fetch_callback(
dag: DAG,
run_id: str,
success: bool = True,
reason: str | None = None,
*,
session: Session = NEW_SESSION,
) -> tuple[list[TaskStateChangeCallback], Context] | None:
"""
Fetch the appropriate callbacks depending on the value of success.
This method gets the context of a single TaskInstance part of this DagRun and returns it along
the list of callbacks.
:param dag: DAG object
:param run_id: The DAG run ID
:param success: Flag to specify if failure or success callback should be called
:param reason: Completion reason
:param session: Database session
"""
callbacks = dag.on_success_callback if success else dag.on_failure_callback
if callbacks:
dagrun = DAG.fetch_dagrun(dag_id=dag.dag_id, run_id=run_id, session=session)
callbacks = callbacks if isinstance(callbacks, list) else [callbacks]
tis = dagrun.get_task_instances(session=session)
# tis from a dagrun may not be a part of dag.partial_subset,
# since dag.partial_subset is a subset of the dag.
# This ensures that we will only use the accessible TI
# context for the callback.
if dag.partial:
tis = [ti for ti in tis if not ti.state == State.NONE]
# filter out removed tasks
tis = [ti for ti in tis if ti.state != TaskInstanceState.REMOVED]
ti = tis[-1] # get first TaskInstance of DagRun
ti.task = dag.get_task(ti.task_id)
context = ti.get_template_context(session=session)
context["reason"] = reason
return callbacks, context
return None
@provide_session
def handle_callback(self, dagrun: DagRun, success=True, reason=None, session=NEW_SESSION):
"""
Triggers on_failure_callback or on_success_callback as appropriate.
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
"""
callbacks, context = DAG.fetch_callback(
dag=self, run_id=dagrun.run_id, success=success, reason=reason, session=session
) or (None, None)
DAG.execute_callback(callbacks, context, self.dag_id)
@classmethod
def execute_callback(cls, callbacks: list[Callable] | None, context: Context | None, dag_id: str):
"""
Triggers the callbacks with the given context.
:param callbacks: List of callbacks to call
:param context: Context to pass to all callbacks
:param dag_id: The dag_id of the DAG to find.
"""
if callbacks and context:
for callback in callbacks:
cls.logger().info("Executing dag callback function: %s", callback)
try:
callback(context)
except Exception:
cls.logger().exception("failed to invoke dag state update callback")
Stats.incr("dag.callback_exceptions", tags={"dag_id": dag_id})
def get_active_runs(self):
"""
Return a list of dag run logical dates currently running.
:return: List of logical dates
"""
runs = DagRun.find(dag_id=self.dag_id, state=DagRunState.RUNNING)
active_dates = []
for run in runs:
active_dates.append(run.logical_date)
return active_dates
@staticmethod
@provide_session
def fetch_dagrun(dag_id: str, run_id: str, session: Session = NEW_SESSION) -> DagRun:
"""
Return the dag run for a given run_id if it exists, otherwise none.
:param dag_id: The dag_id of the DAG to find.
:param run_id: The run_id of the DagRun to find.
:param session:
:return: The DagRun if found, otherwise None.
"""
return session.scalar(select(DagRun).where(DagRun.dag_id == dag_id, DagRun.run_id == run_id))
@provide_session
def get_dagrun(self, run_id: str, session: Session = NEW_SESSION) -> DagRun:
return DAG.fetch_dagrun(dag_id=self.dag_id, run_id=run_id, session=session)
@provide_session
def get_dagruns_between(self, start_date, end_date, session=NEW_SESSION):
"""
Return the list of dag runs between start_date (inclusive) and end_date (inclusive).
:param start_date: The starting logical date of the DagRun to find.
:param end_date: The ending logical date of the DagRun to find.
:param session:
:return: The list of DagRuns found.
"""
dagruns = session.scalars(
select(DagRun).where(
DagRun.dag_id == self.dag_id,
DagRun.logical_date >= start_date,
DagRun.logical_date <= end_date,
)
).all()
return dagruns
@provide_session
def get_latest_logical_date(self, session: Session = NEW_SESSION) -> pendulum.DateTime | None:
"""Return the latest date for which at least one dag run exists."""
return session.scalar(select(func.max(DagRun.logical_date)).where(DagRun.dag_id == self.dag_id))
@provide_session
def get_task_instances_before(
self,
base_date: datetime,
num: int,
*,
session: Session = NEW_SESSION,
) -> list[TaskInstance]:
"""
Get ``num`` task instances before (including) ``base_date``.
The returned list may contain exactly ``num`` task instances
corresponding to any DagRunType. It can have less if there are
less than ``num`` scheduled DAG runs before ``base_date``.
"""
logical_dates: list[Any] = session.execute(
select(DagRun.logical_date)
.where(
DagRun.dag_id == self.dag_id,
DagRun.logical_date <= base_date,
)
.order_by(DagRun.logical_date.desc())
.limit(num)
).all()
if not logical_dates:
return self.get_task_instances(start_date=base_date, end_date=base_date, session=session)
min_date: datetime | None = logical_dates[-1]._mapping.get(
"logical_date"
) # getting the last value from the list
return self.get_task_instances(start_date=min_date, end_date=base_date, session=session)
@provide_session
def get_task_instances(
self,
start_date: datetime | None = None,
end_date: datetime | None = None,
state: list[TaskInstanceState] | None = None,
session: Session = NEW_SESSION,
) -> list[TaskInstance]:
if not start_date:
start_date = (timezone.utcnow() - timedelta(30)).replace(
hour=0, minute=0, second=0, microsecond=0
)
query = self._get_task_instances(
task_ids=None,
start_date=start_date,
end_date=end_date,
run_id=None,
state=state or (),
include_dependent_dags=False,
exclude_task_ids=(),
exclude_run_ids=None,
session=session,
)
return session.scalars(cast(Select, query).order_by(DagRun.logical_date)).all()
@overload
def _get_task_instances(
self,
*,
task_ids: Collection[str | tuple[str, int]] | None,
start_date: datetime | None,
end_date: datetime | None,
run_id: str | None,
state: TaskInstanceState | Sequence[TaskInstanceState],
include_dependent_dags: bool,
exclude_task_ids: Collection[str | tuple[str, int]] | None,
exclude_run_ids: frozenset[str] | None,
session: Session,
dag_bag: DagBag | None = ...,
) -> Iterable[TaskInstance]: ... # pragma: no cover
@overload
def _get_task_instances(
self,
*,
task_ids: Collection[str | tuple[str, int]] | None,
as_pk_tuple: Literal[True],
start_date: datetime | None,
end_date: datetime | None,
run_id: str | None,