-
Notifications
You must be signed in to change notification settings - Fork 13.7k
/
taskinstance.py
3581 lines (3103 loc) · 138 KB
/
taskinstance.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
#
# 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 collections.abc
import contextlib
import hashlib
import itertools
import logging
import math
import operator
import os
import signal
import warnings
from collections import defaultdict
from datetime import timedelta
from enum import Enum
from typing import TYPE_CHECKING, Any, Callable, Collection, Generator, Iterable, Tuple
from urllib.parse import quote
import dill
import jinja2
import lazy_object_proxy
import pendulum
from jinja2 import TemplateAssertionError, UndefinedError
from sqlalchemy import (
Column,
DateTime,
Float,
ForeignKey,
ForeignKeyConstraint,
Index,
Integer,
PrimaryKeyConstraint,
String,
Text,
and_,
delete,
false,
func,
inspect,
or_,
text,
update,
)
from sqlalchemy.ext.associationproxy import association_proxy
from sqlalchemy.ext.mutable import MutableDict
from sqlalchemy.orm import reconstructor, relationship
from sqlalchemy.orm.attributes import NO_VALUE, set_committed_value
from sqlalchemy.sql.expression import case
from airflow import settings
from airflow.api_internal.internal_api_call import internal_api_call
from airflow.compat.functools import cache
from airflow.configuration import conf
from airflow.datasets import Dataset
from airflow.datasets.manager import dataset_manager
from airflow.exceptions import (
AirflowException,
AirflowFailException,
AirflowRescheduleException,
AirflowSensorTimeout,
AirflowSkipException,
AirflowTaskTimeout,
DagRunNotFound,
RemovedInAirflow3Warning,
TaskDeferred,
UnmappableXComLengthPushed,
UnmappableXComTypePushed,
XComForMappingNotPushed,
)
from airflow.listeners.listener import get_listener_manager
from airflow.models.base import Base, StringID
from airflow.models.dagbag import DagBag
from airflow.models.log import Log
from airflow.models.mappedoperator import MappedOperator
from airflow.models.param import process_params
from airflow.models.taskfail import TaskFail
from airflow.models.taskinstancekey import TaskInstanceKey
from airflow.models.taskmap import TaskMap
from airflow.models.taskreschedule import TaskReschedule
from airflow.models.xcom import LazyXComAccess, XCom
from airflow.plugins_manager import integrate_macros_plugins
from airflow.sentry import Sentry
from airflow.stats import Stats
from airflow.templates import SandboxedEnvironment
from airflow.ti_deps.dep_context import DepContext
from airflow.ti_deps.dependencies_deps import REQUEUEABLE_DEPS, RUNNING_DEPS
from airflow.utils import timezone
from airflow.utils.context import ConnectionAccessor, Context, VariableAccessor, context_merge
from airflow.utils.email import send_email
from airflow.utils.helpers import prune_dict, render_template_to_string
from airflow.utils.log.logging_mixin import LoggingMixin
from airflow.utils.module_loading import qualname
from airflow.utils.net import get_hostname
from airflow.utils.operator_helpers import context_to_airflow_vars
from airflow.utils.platform import getuser
from airflow.utils.retries import run_with_db_retries
from airflow.utils.session import NEW_SESSION, create_session, provide_session
from airflow.utils.sqlalchemy import (
ExecutorConfigType,
ExtendedJSON,
UtcDateTime,
tuple_in_condition,
with_row_locks,
)
from airflow.utils.state import DagRunState, JobState, State, TaskInstanceState
from airflow.utils.task_group import MappedTaskGroup
from airflow.utils.task_instance_session import set_current_task_instance_session
from airflow.utils.timeout import timeout
from airflow.utils.xcom import XCOM_RETURN_KEY
TR = TaskReschedule
_CURRENT_CONTEXT: list[Context] = []
log = logging.getLogger(__name__)
if TYPE_CHECKING:
from datetime import datetime
from pathlib import PurePath
from types import TracebackType
from sqlalchemy.orm.session import Session
from sqlalchemy.sql.elements import BooleanClauseList
from sqlalchemy.sql.expression import ColumnOperators
from airflow.models.abstractoperator import TaskStateChangeCallback
from airflow.models.baseoperator import BaseOperator
from airflow.models.dag import DAG, DagModel
from airflow.models.dagrun import DagRun
from airflow.models.dataset import DatasetEvent
from airflow.models.operator import Operator
from airflow.serialization.pydantic.dag import DagModelPydantic
from airflow.serialization.pydantic.taskinstance import TaskInstancePydantic
from airflow.timetables.base import DataInterval
from airflow.typing_compat import Literal, TypeGuard
from airflow.utils.task_group import TaskGroup
# This is a workaround because mypy doesn't work with hybrid_property
# TODO: remove this hack and move hybrid_property back to main import block
# See https://github.com/python/mypy/issues/4430
hybrid_property = property
else:
from sqlalchemy.ext.hybrid import hybrid_property
PAST_DEPENDS_MET = "past_depends_met"
class TaskReturnCode(Enum):
"""
Enum to signal manner of exit for task run command.
:meta private:
"""
DEFERRED = 100
"""When task exits with deferral to trigger."""
@contextlib.contextmanager
def set_current_context(context: Context) -> Generator[Context, None, None]:
"""
Set the current execution context to the provided context object.
This method should be called once per Task execution, before calling operator.execute.
"""
_CURRENT_CONTEXT.append(context)
try:
yield context
finally:
expected_state = _CURRENT_CONTEXT.pop()
if expected_state != context:
log.warning(
"Current context is not equal to the state at context stack. Expected=%s, got=%s",
context,
expected_state,
)
def _stop_remaining_tasks(*, task_instance: TaskInstance | TaskInstancePydantic, session: Session):
"""
Stop non-teardown tasks in dag.
:meta private:
"""
if not task_instance.dag_run:
raise ValueError("``task_instance`` must have ``dag_run`` set")
tis = task_instance.dag_run.get_task_instances(session=session)
if TYPE_CHECKING:
assert isinstance(task_instance.task.dag, DAG)
for ti in tis:
if ti.task_id == task_instance.task_id or ti.state in (
TaskInstanceState.SUCCESS,
TaskInstanceState.FAILED,
):
continue
task = task_instance.task.dag.task_dict[ti.task_id]
if not task.is_teardown:
if ti.state == TaskInstanceState.RUNNING:
log.info("Forcing task %s to fail due to dag's `fail_stop` setting", ti.task_id)
ti.error(session)
else:
log.info("Setting task %s to SKIPPED due to dag's `fail_stop` setting.", ti.task_id)
ti.set_state(state=TaskInstanceState.SKIPPED, session=session)
else:
log.info("Not skipping teardown task '%s'", ti.task_id)
def clear_task_instances(
tis: list[TaskInstance],
session: Session,
activate_dag_runs: None = None,
dag: DAG | None = None,
dag_run_state: DagRunState | Literal[False] = DagRunState.QUEUED,
) -> None:
"""
Clear a set of task instances, but make sure the running ones get killed.
Also sets Dagrun's `state` to QUEUED and `start_date` to the time of execution.
But only for finished DRs (SUCCESS and FAILED).
Doesn't clear DR's `state` and `start_date`for running
DRs (QUEUED and RUNNING) because clearing the state for already
running DR is redundant and clearing `start_date` affects DR's duration.
:param tis: a list of task instances
:param session: current session
:param dag_run_state: state to set finished DagRuns to.
If set to False, DagRuns state will not be changed.
:param dag: DAG object
:param activate_dag_runs: Deprecated parameter, do not pass
"""
job_ids = []
# Keys: dag_id -> run_id -> map_indexes -> try_numbers -> task_id
task_id_by_key: dict[str, dict[str, dict[int, dict[int, set[str]]]]] = defaultdict(
lambda: defaultdict(lambda: defaultdict(lambda: defaultdict(set)))
)
dag_bag = DagBag(read_dags_from_db=True)
for ti in tis:
if ti.state == TaskInstanceState.RUNNING:
if ti.job_id:
# If a task is cleared when running, set its state to RESTARTING so that
# the task is terminated and becomes eligible for retry.
ti.state = TaskInstanceState.RESTARTING
job_ids.append(ti.job_id)
else:
ti_dag = dag if dag and dag.dag_id == ti.dag_id else dag_bag.get_dag(ti.dag_id, session=session)
task_id = ti.task_id
if ti_dag and ti_dag.has_task(task_id):
task = ti_dag.get_task(task_id)
ti.refresh_from_task(task)
task_retries = task.retries
ti.max_tries = ti.try_number + task_retries - 1
else:
# Ignore errors when updating max_tries if the DAG or
# task are not found since database records could be
# outdated. We make max_tries the maximum value of its
# original max_tries or the last attempted try number.
ti.max_tries = max(ti.max_tries, ti.prev_attempted_tries)
ti.state = None
ti.external_executor_id = None
ti.clear_next_method_args()
session.merge(ti)
task_id_by_key[ti.dag_id][ti.run_id][ti.map_index][ti.try_number].add(ti.task_id)
if task_id_by_key:
# Clear all reschedules related to the ti to clear
# This is an optimization for the common case where all tis are for a small number
# of dag_id, run_id, try_number, and map_index. Use a nested dict of dag_id,
# run_id, try_number, map_index, and task_id to construct the where clause in a
# hierarchical manner. This speeds up the delete statement by more than 40x for
# large number of tis (50k+).
conditions = or_(
and_(
TR.dag_id == dag_id,
or_(
and_(
TR.run_id == run_id,
or_(
and_(
TR.map_index == map_index,
or_(
and_(TR.try_number == try_number, TR.task_id.in_(task_ids))
for try_number, task_ids in task_tries.items()
),
)
for map_index, task_tries in map_indexes.items()
),
)
for run_id, map_indexes in run_ids.items()
),
)
for dag_id, run_ids in task_id_by_key.items()
)
delete_qry = TR.__table__.delete().where(conditions)
session.execute(delete_qry)
if job_ids:
from airflow.jobs.job import Job
session.execute(update(Job).where(Job.id.in_(job_ids)).values(state=JobState.RESTARTING))
if activate_dag_runs is not None:
warnings.warn(
"`activate_dag_runs` parameter to clear_task_instances function is deprecated. "
"Please use `dag_run_state`",
RemovedInAirflow3Warning,
stacklevel=2,
)
if not activate_dag_runs:
dag_run_state = False
if dag_run_state is not False and tis:
from airflow.models.dagrun import DagRun # Avoid circular import
run_ids_by_dag_id = defaultdict(set)
for instance in tis:
run_ids_by_dag_id[instance.dag_id].add(instance.run_id)
drs = (
session.query(DagRun)
.filter(
or_(
and_(DagRun.dag_id == dag_id, DagRun.run_id.in_(run_ids))
for dag_id, run_ids in run_ids_by_dag_id.items()
)
)
.all()
)
dag_run_state = DagRunState(dag_run_state) # Validate the state value.
for dr in drs:
if dr.state in State.finished_dr_states:
dr.state = dag_run_state
dr.start_date = timezone.utcnow()
if dag_run_state == DagRunState.QUEUED:
dr.last_scheduling_decision = None
dr.start_date = None
dr.clear_number += 1
session.flush()
def _is_mappable_value(value: Any) -> TypeGuard[Collection]:
"""Whether a value can be used for task mapping.
We only allow collections with guaranteed ordering, but exclude character
sequences since that's usually not what users would expect to be mappable.
"""
if not isinstance(value, (collections.abc.Sequence, dict)):
return False
if isinstance(value, (bytearray, bytes, str)):
return False
return True
def _creator_note(val):
"""Creator the ``note`` association proxy."""
if isinstance(val, str):
return TaskInstanceNote(content=val)
elif isinstance(val, dict):
return TaskInstanceNote(**val)
else:
return TaskInstanceNote(*val)
def _execute_task(task_instance, context, task_orig):
"""
Execute Task (optionally with a Timeout) and push Xcom results.
:param task_instance: the task instance
:param context: Jinja2 context
:param task_orig: origin task
:meta private:
"""
task_to_execute = task_instance.task
if isinstance(task_to_execute, MappedOperator):
raise AirflowException("MappedOperator cannot be executed.")
# If the task has been deferred and is being executed due to a trigger,
# then we need to pick the right method to come back to, otherwise
# we go for the default execute
execute_callable_kwargs = {}
if task_instance.next_method:
if task_instance.next_method:
execute_callable = task_to_execute.resume_execution
execute_callable_kwargs["next_method"] = task_instance.next_method
execute_callable_kwargs["next_kwargs"] = task_instance.next_kwargs
else:
execute_callable = task_to_execute.execute
# If a timeout is specified for the task, make it fail
# if it goes beyond
if task_to_execute.execution_timeout:
# If we are coming in with a next_method (i.e. from a deferral),
# calculate the timeout from our start_date.
if task_instance.next_method:
timeout_seconds = (
task_to_execute.execution_timeout - (timezone.utcnow() - task_instance.start_date)
).total_seconds()
else:
timeout_seconds = task_to_execute.execution_timeout.total_seconds()
try:
# It's possible we're already timed out, so fast-fail if true
if timeout_seconds <= 0:
raise AirflowTaskTimeout()
# Run task in timeout wrapper
with timeout(timeout_seconds):
result = execute_callable(context=context, **execute_callable_kwargs)
except AirflowTaskTimeout:
task_to_execute.on_kill()
raise
else:
result = execute_callable(context=context, **execute_callable_kwargs)
with create_session() as session:
if task_to_execute.do_xcom_push:
xcom_value = result
else:
xcom_value = None
if xcom_value is not None: # If the task returns a result, push an XCom containing it.
task_instance.xcom_push(key=XCOM_RETURN_KEY, value=xcom_value, session=session)
_record_task_map_for_downstreams(
task_instance=task_instance, task=task_orig, value=xcom_value, session=session
)
return result
def _refresh_from_db(
*, task_instance: TaskInstance | TaskInstancePydantic, session: Session, lock_for_update: bool = False
) -> None:
"""
Refreshes the task instance from the database based on the primary key.
:param task_instance: the task instance
:param session: SQLAlchemy ORM Session
:param lock_for_update: if True, indicates that the database should
lock the TaskInstance (issuing a FOR UPDATE clause) until the
session is committed.
:meta private:
"""
if task_instance in session:
session.refresh(task_instance, TaskInstance.__mapper__.column_attrs.keys())
ti = TaskInstance.get_task_instance(
dag_id=task_instance.dag_id,
task_id=task_instance.task_id,
run_id=task_instance.run_id,
map_index=task_instance.map_index,
select_columns=True,
lock_for_update=lock_for_update,
session=session,
)
if ti:
# Fields ordered per model definition
task_instance.start_date = ti.start_date
task_instance.end_date = ti.end_date
task_instance.duration = ti.duration
task_instance.state = ti.state
# Since we selected columns, not the object, this is the raw value
task_instance.try_number = ti.try_number
task_instance.max_tries = ti.max_tries
task_instance.hostname = ti.hostname
task_instance.unixname = ti.unixname
task_instance.job_id = ti.job_id
task_instance.pool = ti.pool
task_instance.pool_slots = ti.pool_slots or 1
task_instance.queue = ti.queue
task_instance.priority_weight = ti.priority_weight
task_instance.operator = ti.operator
task_instance.custom_operator_name = ti.custom_operator_name
task_instance.queued_dttm = ti.queued_dttm
task_instance.queued_by_job_id = ti.queued_by_job_id
task_instance.pid = ti.pid
task_instance.executor_config = ti.executor_config
task_instance.external_executor_id = ti.external_executor_id
task_instance.trigger_id = ti.trigger_id
task_instance.next_method = ti.next_method
task_instance.next_kwargs = ti.next_kwargs
else:
task_instance.state = None
def _set_duration(*, task_instance: TaskInstance | TaskInstancePydantic) -> None:
"""
Set task instance duration.
:param task_instance: the task instance
:meta private:
"""
if task_instance.end_date and task_instance.start_date:
task_instance.duration = (task_instance.end_date - task_instance.start_date).total_seconds()
else:
task_instance.duration = None
log.debug("Task Duration set to %s", task_instance.duration)
def _stats_tags(*, task_instance: TaskInstance | TaskInstancePydantic) -> dict[str, str]:
"""
Returns task instance tags.
:param task_instance: the task instance
:meta private:
"""
return prune_dict({"dag_id": task_instance.dag_id, "task_id": task_instance.task_id})
def _clear_next_method_args(*, task_instance: TaskInstance | TaskInstancePydantic) -> None:
"""
Ensure we unset next_method and next_kwargs to ensure that any retries don't reuse them.
:param task_instance: the task instance
:meta private:
"""
log.debug("Clearing next_method and next_kwargs.")
task_instance.next_method = None
task_instance.next_kwargs = None
def _get_template_context(
*,
task_instance,
session: Session | None = None,
ignore_param_exceptions: bool = True,
) -> Context:
"""
Return TI Context.
:param task_instance: the task instance
:param session: SQLAlchemy ORM Session
:param ignore_param_exceptions: flag to suppress value exceptions while initializing the ParamsDict
:meta private:
"""
# Do not use provide_session here -- it expunges everything on exit!
if not session:
session = settings.Session()
from airflow import macros
from airflow.models.abstractoperator import NotMapped
integrate_macros_plugins()
task = task_instance.task
if TYPE_CHECKING:
assert task.dag
dag: DAG = task.dag
dag_run = task_instance.get_dagrun(session)
data_interval = dag.get_run_data_interval(dag_run)
validated_params = process_params(dag, task, dag_run, suppress_exception=ignore_param_exceptions)
logical_date = timezone.coerce_datetime(task_instance.execution_date)
ds = logical_date.strftime("%Y-%m-%d")
ds_nodash = ds.replace("-", "")
ts = logical_date.isoformat()
ts_nodash = logical_date.strftime("%Y%m%dT%H%M%S")
ts_nodash_with_tz = ts.replace("-", "").replace(":", "")
@cache # Prevent multiple database access.
def _get_previous_dagrun_success() -> DagRun | None:
return task_instance.get_previous_dagrun(state=DagRunState.SUCCESS, session=session)
def _get_previous_dagrun_data_interval_success() -> DataInterval | None:
dagrun = _get_previous_dagrun_success()
if dagrun is None:
return None
return dag.get_run_data_interval(dagrun)
def get_prev_data_interval_start_success() -> pendulum.DateTime | None:
data_interval = _get_previous_dagrun_data_interval_success()
if data_interval is None:
return None
return data_interval.start
def get_prev_data_interval_end_success() -> pendulum.DateTime | None:
data_interval = _get_previous_dagrun_data_interval_success()
if data_interval is None:
return None
return data_interval.end
def get_prev_start_date_success() -> pendulum.DateTime | None:
dagrun = _get_previous_dagrun_success()
if dagrun is None:
return None
return timezone.coerce_datetime(dagrun.start_date)
def get_prev_end_date_success() -> pendulum.DateTime | None:
dagrun = _get_previous_dagrun_success()
if dagrun is None:
return None
return timezone.coerce_datetime(dagrun.end_date)
@cache
def get_yesterday_ds() -> str:
return (logical_date - timedelta(1)).strftime("%Y-%m-%d")
def get_yesterday_ds_nodash() -> str:
return get_yesterday_ds().replace("-", "")
@cache
def get_tomorrow_ds() -> str:
return (logical_date + timedelta(1)).strftime("%Y-%m-%d")
def get_tomorrow_ds_nodash() -> str:
return get_tomorrow_ds().replace("-", "")
@cache
def get_next_execution_date() -> pendulum.DateTime | None:
# For manually triggered dagruns that aren't run on a schedule,
# the "next" execution date doesn't make sense, and should be set
# to execution date for consistency with how execution_date is set
# for manually triggered tasks, i.e. triggered_date == execution_date.
if dag_run.external_trigger:
return logical_date
if dag is None:
return None
next_info = dag.next_dagrun_info(data_interval, restricted=False)
if next_info is None:
return None
return timezone.coerce_datetime(next_info.logical_date)
def get_next_ds() -> str | None:
execution_date = get_next_execution_date()
if execution_date is None:
return None
return execution_date.strftime("%Y-%m-%d")
def get_next_ds_nodash() -> str | None:
ds = get_next_ds()
if ds is None:
return ds
return ds.replace("-", "")
@cache
def get_prev_execution_date():
# For manually triggered dagruns that aren't run on a schedule,
# the "previous" execution date doesn't make sense, and should be set
# to execution date for consistency with how execution_date is set
# for manually triggered tasks, i.e. triggered_date == execution_date.
if dag_run.external_trigger:
return logical_date
with warnings.catch_warnings():
warnings.simplefilter("ignore", RemovedInAirflow3Warning)
return dag.previous_schedule(logical_date)
@cache
def get_prev_ds() -> str | None:
execution_date = get_prev_execution_date()
if execution_date is None:
return None
return execution_date.strftime("%Y-%m-%d")
def get_prev_ds_nodash() -> str | None:
prev_ds = get_prev_ds()
if prev_ds is None:
return None
return prev_ds.replace("-", "")
def get_triggering_events() -> dict[str, list[DatasetEvent]]:
if TYPE_CHECKING:
assert session is not None
# The dag_run may not be attached to the session anymore since the
# code base is over-zealous with use of session.expunge_all().
# Re-attach it if we get called.
nonlocal dag_run
if dag_run not in session:
dag_run = session.merge(dag_run, load=False)
dataset_events = dag_run.consumed_dataset_events
triggering_events: dict[str, list[DatasetEvent]] = defaultdict(list)
for event in dataset_events:
triggering_events[event.dataset.uri].append(event)
return triggering_events
try:
expanded_ti_count: int | None = task.get_mapped_ti_count(task_instance.run_id, session=session)
except NotMapped:
expanded_ti_count = None
# NOTE: If you add anything to this dict, make sure to also update the
# definition in airflow/utils/context.pyi, and KNOWN_CONTEXT_KEYS in
# airflow/utils/context.py!
context = {
"conf": conf,
"dag": dag,
"dag_run": dag_run,
"data_interval_end": timezone.coerce_datetime(data_interval.end),
"data_interval_start": timezone.coerce_datetime(data_interval.start),
"ds": ds,
"ds_nodash": ds_nodash,
"execution_date": logical_date,
"expanded_ti_count": expanded_ti_count,
"inlets": task.inlets,
"logical_date": logical_date,
"macros": macros,
"next_ds": get_next_ds(),
"next_ds_nodash": get_next_ds_nodash(),
"next_execution_date": get_next_execution_date(),
"outlets": task.outlets,
"params": validated_params,
"prev_data_interval_start_success": get_prev_data_interval_start_success(),
"prev_data_interval_end_success": get_prev_data_interval_end_success(),
"prev_ds": get_prev_ds(),
"prev_ds_nodash": get_prev_ds_nodash(),
"prev_execution_date": get_prev_execution_date(),
"prev_execution_date_success": task_instance.get_previous_execution_date(
state=DagRunState.SUCCESS,
session=session,
),
"prev_start_date_success": get_prev_start_date_success(),
"prev_end_date_success": get_prev_end_date_success(),
"run_id": task_instance.run_id,
"task": task,
"task_instance": task_instance,
"task_instance_key_str": f"{task.dag_id}__{task.task_id}__{ds_nodash}",
"test_mode": task_instance.test_mode,
"ti": task_instance,
"tomorrow_ds": get_tomorrow_ds(),
"tomorrow_ds_nodash": get_tomorrow_ds_nodash(),
"triggering_dataset_events": lazy_object_proxy.Proxy(get_triggering_events),
"ts": ts,
"ts_nodash": ts_nodash,
"ts_nodash_with_tz": ts_nodash_with_tz,
"var": {
"json": VariableAccessor(deserialize_json=True),
"value": VariableAccessor(deserialize_json=False),
},
"conn": ConnectionAccessor(),
"yesterday_ds": get_yesterday_ds(),
"yesterday_ds_nodash": get_yesterday_ds_nodash(),
}
# Mypy doesn't like turning existing dicts in to a TypeDict -- and we "lie" in the type stub to say it
# is one, but in practice it isn't. See https://github.com/python/mypy/issues/8890
return Context(context) # type: ignore
def _is_eligible_to_retry(*, task_instance: TaskInstance | TaskInstancePydantic):
"""
Is task instance is eligible for retry.
:param task_instance: the task instance
:meta private:
"""
if task_instance.state == TaskInstanceState.RESTARTING:
# If a task is cleared when running, it goes into RESTARTING state and is always
# eligible for retry
return True
if not getattr(task_instance, "task", None):
# Couldn't load the task, don't know number of retries, guess:
return task_instance.try_number <= task_instance.max_tries
return task_instance.task.retries and task_instance.try_number <= task_instance.max_tries
def _handle_failure(
*,
task_instance: TaskInstance | TaskInstancePydantic,
error: None | str | Exception | KeyboardInterrupt,
session: Session,
test_mode: bool | None = None,
context: Context | None = None,
force_fail: bool = False,
) -> None:
"""
Handle Failure for a task instance.
:param task_instance: the task instance
:param error: if specified, log the specific exception if thrown
:param session: SQLAlchemy ORM Session
:param test_mode: doesn't record success or failure in the DB if True
:param context: Jinja2 context
:param force_fail: if True, task does not retry
:meta private:
"""
if test_mode is None:
test_mode = task_instance.test_mode
failure_context = TaskInstance.fetch_handle_failure_context(
ti=task_instance,
error=error,
test_mode=test_mode,
context=context,
force_fail=force_fail,
session=session,
)
_log_state(task_instance=task_instance, lead_msg="Immediate failure requested. " if force_fail else "")
if (
failure_context["task"]
and failure_context["email_for_state"](failure_context["task"])
and failure_context["task"].email
):
try:
task_instance.email_alert(error, failure_context["task"])
except Exception:
log.exception("Failed to send email to: %s", failure_context["task"].email)
if failure_context["callbacks"] and failure_context["context"]:
_run_finished_callback(
callbacks=failure_context["callbacks"],
context=failure_context["context"],
)
if not test_mode:
TaskInstance.save_to_db(failure_context["ti"], session)
def _get_try_number(*, task_instance: TaskInstance | TaskInstancePydantic):
"""
Return the try number that a task number will be when it is actually run.
If the TaskInstance is currently running, this will match the column in the
database, in all other cases this will be incremented.
This is designed so that task logs end up in the right file.
:param task_instance: the task instance
:meta private:
"""
if task_instance.state == TaskInstanceState.RUNNING.RUNNING:
return task_instance._try_number
return task_instance._try_number + 1
def _set_try_number(*, task_instance: TaskInstance | TaskInstancePydantic, value: int) -> None:
"""
Set a task try number.
:param task_instance: the task instance
:param value: the try number
:meta private:
"""
task_instance._try_number = value
def _refresh_from_task(
*, task_instance: TaskInstance | TaskInstancePydantic, task: Operator, pool_override: str | None = None
) -> None:
"""
Copy common attributes from the given task.
:param task_instance: the task instance
:param task: The task object to copy from
:param pool_override: Use the pool_override instead of task's pool
:meta private:
"""
task_instance.task = task
task_instance.queue = task.queue
task_instance.pool = pool_override or task.pool
task_instance.pool_slots = task.pool_slots
task_instance.priority_weight = task.priority_weight_total
task_instance.run_as_user = task.run_as_user
# Do not set max_tries to task.retries here because max_tries is a cumulative
# value that needs to be stored in the db.
task_instance.executor_config = task.executor_config
task_instance.operator = task.task_type
task_instance.custom_operator_name = getattr(task, "custom_operator_name", None)
def _record_task_map_for_downstreams(
*, task_instance: TaskInstance | TaskInstancePydantic, task: Operator, value: Any, session: Session
) -> None:
"""
Record the task map for downstream tasks.
:param task_instance: the task instance
:param task: The task object
:param value: The value
:param session: SQLAlchemy ORM Session
:meta private:
"""
if next(task.iter_mapped_dependants(), None) is None: # No mapped dependants, no need to validate.
return
# TODO: We don't push TaskMap for mapped task instances because it's not
# currently possible for a downstream to depend on one individual mapped
# task instance. This will change when we implement task mapping inside
# a mapped task group, and we'll need to further analyze the case.
if isinstance(task, MappedOperator):
return
if value is None:
raise XComForMappingNotPushed()
if not _is_mappable_value(value):
raise UnmappableXComTypePushed(value)
task_map = TaskMap.from_task_instance_xcom(task_instance, value)
max_map_length = conf.getint("core", "max_map_length", fallback=1024)
if task_map.length > max_map_length:
raise UnmappableXComLengthPushed(value, max_map_length)
session.merge(task_map)
def _get_previous_dagrun(
*,
task_instance: TaskInstance | TaskInstancePydantic,
state: DagRunState | None = None,
session: Session | None = None,
) -> DagRun | None:
"""
The DagRun that ran before this task instance's DagRun.
:param task_instance: the task instance
:param state: If passed, it only take into account instances of a specific state.
:param session: SQLAlchemy ORM Session.
:meta private:
"""
dag = task_instance.task.dag
if dag is None:
return None
dr = task_instance.get_dagrun(session=session)
dr.dag = dag
from airflow.models.dagrun import DagRun # Avoid circular import
# We always ignore schedule in dagrun lookup when `state` is given
# or the DAG is never scheduled. For legacy reasons, when
# `catchup=True`, we use `get_previous_scheduled_dagrun` unless
# `ignore_schedule` is `True`.
ignore_schedule = state is not None or not dag.timetable.can_be_scheduled
if dag.catchup is True and not ignore_schedule:
last_dagrun = DagRun.get_previous_scheduled_dagrun(dr.id, session=session)
else:
last_dagrun = DagRun.get_previous_dagrun(dag_run=dr, session=session, state=state)
if last_dagrun:
return last_dagrun
return None
def _get_previous_execution_date(
*,
task_instance: TaskInstance | TaskInstancePydantic,
state: DagRunState | None,
session: Session,
) -> pendulum.DateTime | None:
"""
The execution date from property previous_ti_success.
:param task_instance: the task instance
:param session: SQLAlchemy ORM Session
:param state: If passed, it only take into account instances of a specific state.
:meta private:
"""
log.debug("previous_execution_date was called")
prev_ti = task_instance.get_previous_ti(state=state, session=session)
return pendulum.instance(prev_ti.execution_date) if prev_ti and prev_ti.execution_date else None
def _email_alert(
*, task_instance: TaskInstance | TaskInstancePydantic, exception, task: BaseOperator
) -> None:
"""
Send alert email with exception information.
:param task_instance: the task instance
:param exception: the exception
:param task: task related to the exception
:meta private:
"""
subject, html_content, html_content_err = task_instance.get_email_subject_content(exception, task=task)
assert task.email
try:
send_email(task.email, subject, html_content)