-
Notifications
You must be signed in to change notification settings - Fork 14.4k
/
taskinstance.py
1523 lines (1338 loc) · 56.9 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.
import copy
import getpass
import hashlib
import logging
import math
import os
import signal
import time
from datetime import datetime, timedelta
from typing import Any, Dict, Iterable, Optional, Tuple, Union
from urllib.parse import quote
import dill
import lazy_object_proxy
import pendulum
from sqlalchemy import Column, Float, Index, Integer, PickleType, String, func
from sqlalchemy.orm import reconstructor
from sqlalchemy.orm.session import Session
from airflow import settings
from airflow.configuration import conf
from airflow.exceptions import (
AirflowException, AirflowRescheduleException, AirflowSkipException, AirflowTaskTimeout,
)
from airflow.models.base import ID_LEN, Base
from airflow.models.log import Log
from airflow.models.pool import Pool
from airflow.models.taskfail import TaskFail
from airflow.models.taskreschedule import TaskReschedule
from airflow.models.variable import Variable
from airflow.models.xcom import XCOM_RETURN_KEY, XCom
from airflow.sentry import Sentry
from airflow.stats import Stats
from airflow.ti_deps.dep_context import REQUEUEABLE_DEPS, RUNNING_DEPS, DepContext
from airflow.utils import timezone
from airflow.utils.db import provide_session
from airflow.utils.email import send_email
from airflow.utils.helpers import is_container
from airflow.utils.log.logging_mixin import LoggingMixin
from airflow.utils.net import get_hostname
from airflow.utils.sqlalchemy import UtcDateTime
from airflow.utils.state import State
from airflow.utils.timeout import timeout
def clear_task_instances(tis,
session,
activate_dag_runs=True,
dag=None,
):
"""
Clears a set of task instances, but makes sure the running ones
get killed.
:param tis: a list of task instances
:param session: current session
:param activate_dag_runs: flag to check for active dag run
:param dag: DAG object
"""
job_ids = []
for ti in tis:
if ti.state == State.RUNNING:
if ti.job_id:
ti.state = State.SHUTDOWN
job_ids.append(ti.job_id)
else:
task_id = ti.task_id
if dag and dag.has_task(task_id):
task = dag.get_task(task_id)
task_retries = task.retries
ti.max_tries = ti.try_number + task_retries - 1
else:
# Ignore errors when updating max_tries if dag is None or
# task not found in dag 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 = State.NONE
session.merge(ti)
# Clear all reschedules related to the ti to clear
TR = TaskReschedule
session.query(TR).filter(
TR.dag_id == ti.dag_id,
TR.task_id == ti.task_id,
TR.execution_date == ti.execution_date,
TR.try_number == ti.try_number
).delete()
if job_ids:
from airflow.jobs import BaseJob as BJ
for job in session.query(BJ).filter(BJ.id.in_(job_ids)).all():
job.state = State.SHUTDOWN
if activate_dag_runs and tis:
from airflow.models.dagrun import DagRun # Avoid circular import
drs = session.query(DagRun).filter(
DagRun.dag_id.in_({ti.dag_id for ti in tis}),
DagRun.execution_date.in_({ti.execution_date for ti in tis}),
).all()
for dr in drs:
dr.state = State.RUNNING
dr.start_date = timezone.utcnow()
# Key used to identify task instance
# Tuple of: dag_id, task_id, execution_date, try_number
TaskInstanceKeyType = Tuple[str, str, datetime, int]
class TaskInstance(Base, LoggingMixin):
"""
Task instances store the state of a task instance. This table is the
authority and single source of truth around what tasks have run and the
state they are in.
The SqlAlchemy model doesn't have a SqlAlchemy foreign key to the task or
dag model deliberately to have more control over transactions.
Database transactions on this table should insure double triggers and
any confusion around what task instances are or aren't ready to run
even while multiple schedulers may be firing task instances.
"""
__tablename__ = "task_instance"
task_id = Column(String(ID_LEN), primary_key=True)
dag_id = Column(String(ID_LEN), primary_key=True)
execution_date = Column(UtcDateTime, primary_key=True)
start_date = Column(UtcDateTime)
end_date = Column(UtcDateTime)
duration = Column(Float)
state = Column(String(20))
_try_number = Column('try_number', Integer, default=0)
max_tries = Column(Integer)
hostname = Column(String(1000))
unixname = Column(String(1000))
job_id = Column(Integer)
pool = Column(String(50), nullable=False)
queue = Column(String(256))
priority_weight = Column(Integer)
operator = Column(String(1000))
queued_dttm = Column(UtcDateTime)
pid = Column(Integer)
executor_config = Column(PickleType(pickler=dill))
# If adding new fields here then remember to add them to
# refresh_from_db() or they wont display in the UI correctly
__table_args__ = (
Index('ti_dag_state', dag_id, state),
Index('ti_dag_date', dag_id, execution_date),
Index('ti_state', state),
Index('ti_state_lkp', dag_id, task_id, execution_date, state),
Index('ti_pool', pool, state, priority_weight),
Index('ti_job_id', job_id),
)
def __init__(self, task, execution_date, state=None):
self.dag_id = task.dag_id
self.task_id = task.task_id
self.task = task
self._log = logging.getLogger("airflow.task")
# make sure we have a localized execution_date stored in UTC
if execution_date and not timezone.is_localized(execution_date):
self.log.warning("execution date %s has no timezone information. Using "
"default from dag or system", execution_date)
if self.task.has_dag():
execution_date = timezone.make_aware(execution_date,
self.task.dag.timezone)
else:
execution_date = timezone.make_aware(execution_date)
execution_date = timezone.convert_to_utc(execution_date)
self.execution_date = execution_date
self.queue = task.queue
self.pool = task.pool
self.priority_weight = task.priority_weight_total
self.try_number = 0
self.max_tries = self.task.retries
self.unixname = getpass.getuser()
self.run_as_user = task.run_as_user
if state:
self.state = state
self.hostname = ''
self.executor_config = task.executor_config
self.init_on_load()
# Is this TaskInstance being currently running within `airflow tasks run --raw`.
# Not persisted to the database so only valid for the current process
self.raw = False
@reconstructor
def init_on_load(self):
""" Initialize the attributes that aren't stored in the DB. """
self.test_mode = False # can be changed when calling 'run'
@property
def try_number(self):
"""
Return the try number that this 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.
if self.state == State.RUNNING:
return self._try_number
return self._try_number + 1
@try_number.setter
def try_number(self, value):
self._try_number = value
@property
def prev_attempted_tries(self):
"""
Based on this instance's try_number, this will calculate
the number of previously attempted tries, defaulting to 0.
"""
# Expose this for the Task Tries and Gantt graph views.
# Using `try_number` throws off the counts for non-running tasks.
# Also useful in error logging contexts to get
# the try number for the last try that was attempted.
# https://issues.apache.org/jira/browse/AIRFLOW-2143
return self._try_number
@property
def next_try_number(self):
return self._try_number + 1
def command_as_list(
self,
mark_success=False,
ignore_all_deps=False,
ignore_task_deps=False,
ignore_depends_on_past=False,
ignore_ti_state=False,
local=False,
pickle_id=None,
raw=False,
job_id=None,
pool=None,
cfg_path=None):
"""
Returns a command that can be executed anywhere where airflow is
installed. This command is part of the message sent to executors by
the orchestrator.
"""
dag = self.task.dag
should_pass_filepath = not pickle_id and dag
if should_pass_filepath and dag.full_filepath != dag.filepath:
path = "DAGS_FOLDER/{}".format(dag.filepath)
elif should_pass_filepath and dag.full_filepath:
path = dag.full_filepath
else:
path = None
return TaskInstance.generate_command(
self.dag_id,
self.task_id,
self.execution_date,
mark_success=mark_success,
ignore_all_deps=ignore_all_deps,
ignore_task_deps=ignore_task_deps,
ignore_depends_on_past=ignore_depends_on_past,
ignore_ti_state=ignore_ti_state,
local=local,
pickle_id=pickle_id,
file_path=path,
raw=raw,
job_id=job_id,
pool=pool,
cfg_path=cfg_path)
@staticmethod
def generate_command(dag_id,
task_id,
execution_date,
mark_success=False,
ignore_all_deps=False,
ignore_depends_on_past=False,
ignore_task_deps=False,
ignore_ti_state=False,
local=False,
pickle_id=None,
file_path=None,
raw=False,
job_id=None,
pool=None,
cfg_path=None
):
"""
Generates the shell command required to execute this task instance.
:param dag_id: DAG ID
:type dag_id: unicode
:param task_id: Task ID
:type task_id: unicode
:param execution_date: Execution date for the task
:type execution_date: datetime.datetime
:param mark_success: Whether to mark the task as successful
:type mark_success: bool
:param ignore_all_deps: Ignore all ignorable dependencies.
Overrides the other ignore_* parameters.
:type ignore_all_deps: bool
:param ignore_depends_on_past: Ignore depends_on_past parameter of DAGs
(e.g. for Backfills)
:type ignore_depends_on_past: bool
:param ignore_task_deps: Ignore task-specific dependencies such as depends_on_past
and trigger rule
:type ignore_task_deps: bool
:param ignore_ti_state: Ignore the task instance's previous failure/success
:type ignore_ti_state: bool
:param local: Whether to run the task locally
:type local: bool
:param pickle_id: If the DAG was serialized to the DB, the ID
associated with the pickled DAG
:type pickle_id: unicode
:param file_path: path to the file containing the DAG definition
:param raw: raw mode (needs more details)
:param job_id: job ID (needs more details)
:param pool: the Airflow pool that the task should run in
:type pool: unicode
:param cfg_path: the Path to the configuration file
:type cfg_path: str
:return: shell command that can be used to run the task instance
"""
iso = execution_date.isoformat()
cmd = ["airflow", "tasks", "run", str(dag_id), str(task_id), str(iso)]
cmd.extend(["--mark_success"]) if mark_success else None
cmd.extend(["--pickle", str(pickle_id)]) if pickle_id else None
cmd.extend(["--job_id", str(job_id)]) if job_id else None
cmd.extend(["-A"]) if ignore_all_deps else None
cmd.extend(["-i"]) if ignore_task_deps else None
cmd.extend(["-I"]) if ignore_depends_on_past else None
cmd.extend(["--force"]) if ignore_ti_state else None
cmd.extend(["--local"]) if local else None
cmd.extend(["--pool", pool]) if pool else None
cmd.extend(["--raw"]) if raw else None
cmd.extend(["-sd", file_path]) if file_path else None
cmd.extend(["--cfg_path", cfg_path]) if cfg_path else None
return cmd
@property
def log_filepath(self):
iso = self.execution_date.isoformat()
log = os.path.expanduser(conf.get('core', 'BASE_LOG_FOLDER'))
return ("{log}/{dag_id}/{task_id}/{iso}.log".format(
log=log, dag_id=self.dag_id, task_id=self.task_id, iso=iso))
@property
def log_url(self):
iso = quote(self.execution_date.isoformat())
base_url = conf.get('webserver', 'BASE_URL')
return base_url + (
"/log?"
"execution_date={iso}"
"&task_id={task_id}"
"&dag_id={dag_id}"
).format(iso=iso, task_id=self.task_id, dag_id=self.dag_id)
@property
def mark_success_url(self):
iso = quote(self.execution_date.isoformat())
base_url = conf.get('webserver', 'BASE_URL')
return base_url + (
"/success"
"?task_id={task_id}"
"&dag_id={dag_id}"
"&execution_date={iso}"
"&upstream=false"
"&downstream=false"
).format(task_id=self.task_id, dag_id=self.dag_id, iso=iso)
@provide_session
def current_state(self, session=None):
"""
Get the very latest state from the database, if a session is passed,
we use and looking up the state becomes part of the session, otherwise
a new session is used.
"""
ti = session.query(TaskInstance).filter(
TaskInstance.dag_id == self.dag_id,
TaskInstance.task_id == self.task_id,
TaskInstance.execution_date == self.execution_date,
).all()
if ti:
state = ti[0].state
else:
state = None
return state
@provide_session
def error(self, session=None):
"""
Forces the task instance's state to FAILED in the database.
"""
self.log.error("Recording the task instance as FAILED")
self.state = State.FAILED
session.merge(self)
session.commit()
@provide_session
def refresh_from_db(self, session=None, lock_for_update=False, refresh_executor_config=False) -> None:
"""
Refreshes the task instance from the database based on the primary key
:param refresh_executor_config: if True, revert executor config to
result from DB. Often, however, we will want to keep the newest
version
:param lock_for_update: if True, indicates that the database should
lock the TaskInstance (issuing a FOR UPDATE clause) until the
session is committed.
"""
qry = session.query(TaskInstance).filter(
TaskInstance.dag_id == self.dag_id,
TaskInstance.task_id == self.task_id,
TaskInstance.execution_date == self.execution_date)
if lock_for_update:
ti = qry.with_for_update().first()
else:
ti = qry.first()
if ti:
# Fields ordered per model definition
self.start_date = ti.start_date
self.end_date = ti.end_date
self.duration = ti.duration
self.state = ti.state
# Get the raw value of try_number column, don't read through the
# accessor here otherwise it will be incremented by one already.
self.try_number = ti._try_number
self.max_tries = ti.max_tries
self.hostname = ti.hostname
self.unixname = ti.unixname
self.job_id = ti.job_id
self.pool = ti.pool
self.queue = ti.queue
self.priority_weight = ti.priority_weight
self.operator = ti.operator
self.queued_dttm = ti.queued_dttm
self.pid = ti.pid
if refresh_executor_config:
self.executor_config = ti.executor_config
else:
self.state = None
@provide_session
def clear_xcom_data(self, session=None):
"""
Clears all XCom data from the database for the task instance
"""
session.query(XCom).filter(
XCom.dag_id == self.dag_id,
XCom.task_id == self.task_id,
XCom.execution_date == self.execution_date
).delete()
session.commit()
@property
def key(self) -> TaskInstanceKeyType:
"""
Returns a tuple that identifies the task instance uniquely
"""
return self.dag_id, self.task_id, self.execution_date, self.try_number
@provide_session
def set_state(self, state, session=None, commit=True):
self.state = state
self.start_date = timezone.utcnow()
self.end_date = timezone.utcnow()
session.merge(self)
if commit:
session.commit()
@property
def is_premature(self):
"""
Returns whether a task is in UP_FOR_RETRY state and its retry interval
has elapsed.
"""
# is the task still in the retry waiting period?
return self.state == State.UP_FOR_RETRY and not self.ready_for_retry()
@provide_session
def are_dependents_done(self, session=None):
"""
Checks whether the dependents of this task instance have all succeeded.
This is meant to be used by wait_for_downstream.
This is useful when you do not want to start processing the next
schedule of a task until the dependents are done. For instance,
if the task DROPs and recreates a table.
"""
task = self.task
if not task.downstream_task_ids:
return True
ti = session.query(func.count(TaskInstance.task_id)).filter(
TaskInstance.dag_id == self.dag_id,
TaskInstance.task_id.in_(task.downstream_task_ids),
TaskInstance.execution_date == self.execution_date,
TaskInstance.state == State.SUCCESS,
)
count = ti[0][0]
return count == len(task.downstream_task_ids)
@provide_session
def _get_previous_ti(
self,
state: Optional[str] = None,
session: Session = None
) -> Optional['TaskInstance']:
dag = self.task.dag
if dag:
dr = self.get_dagrun(session=session)
# LEGACY: most likely running from unit tests
if not dr:
# Means that this TaskInstance is NOT being run from a DR, but from a catchup
previous_scheduled_date = dag.previous_schedule(self.execution_date)
if not previous_scheduled_date:
return None
return TaskInstance(task=self.task, execution_date=previous_scheduled_date)
dr.dag = dag
# We always ignore schedule in dagrun lookup when `state` is given or `schedule_interval is None`.
# For legacy reasons, when `catchup=True`, we use `get_previous_scheduled_dagrun` unless
# `ignore_schedule` is `True`.
ignore_schedule = state is not None or dag.schedule_interval is None
if dag.catchup is True and not ignore_schedule:
last_dagrun = dr.get_previous_scheduled_dagrun(session=session)
else:
last_dagrun = dr.get_previous_dagrun(session=session, state=state)
if last_dagrun:
return last_dagrun.get_task_instance(self.task_id, session=session)
return None
@property
def previous_ti(self) -> Optional['TaskInstance']:
"""The task instance for the task that ran before this task instance."""
return self._get_previous_ti()
@property
def previous_ti_success(self) -> Optional['TaskInstance']:
"""The ti from prior succesful dag run for this task, by execution date."""
return self._get_previous_ti(state=State.SUCCESS)
@property
def previous_execution_date_success(self) -> Optional[pendulum.datetime]:
"""The execution date from property previous_ti_success."""
self.log.debug("previous_execution_date_success was called")
prev_ti = self._get_previous_ti(state=State.SUCCESS)
return prev_ti and prev_ti.execution_date
@property
def previous_start_date_success(self) -> Optional[pendulum.datetime]:
"""The start date from property previous_ti_success."""
self.log.debug("previous_start_date_success was called")
prev_ti = self._get_previous_ti(state=State.SUCCESS)
return prev_ti and prev_ti.start_date
@provide_session
def are_dependencies_met(
self,
dep_context=None,
session=None,
verbose=False):
"""
Returns whether or not all the conditions are met for this task instance to be run
given the context for the dependencies (e.g. a task instance being force run from
the UI will ignore some dependencies).
:param dep_context: The execution context that determines the dependencies that
should be evaluated.
:type dep_context: DepContext
:param session: database session
:type session: sqlalchemy.orm.session.Session
:param verbose: whether log details on failed dependencies on
info or debug log level
:type verbose: bool
"""
dep_context = dep_context or DepContext()
failed = False
verbose_aware_logger = self.log.info if verbose else self.log.debug
for dep_status in self.get_failed_dep_statuses(
dep_context=dep_context,
session=session):
failed = True
verbose_aware_logger(
"Dependencies not met for %s, dependency '%s' FAILED: %s",
self, dep_status.dep_name, dep_status.reason
)
if failed:
return False
verbose_aware_logger("Dependencies all met for %s", self)
return True
@provide_session
def get_failed_dep_statuses(
self,
dep_context=None,
session=None):
dep_context = dep_context or DepContext()
for dep in dep_context.deps | self.task.deps:
for dep_status in dep.get_dep_statuses(
self,
session,
dep_context):
self.log.debug(
"%s dependency '%s' PASSED: %s, %s",
self, dep_status.dep_name, dep_status.passed, dep_status.reason
)
if not dep_status.passed:
yield dep_status
def __repr__(self):
return (
"<TaskInstance: {ti.dag_id}.{ti.task_id} "
"{ti.execution_date} [{ti.state}]>"
).format(ti=self)
def next_retry_datetime(self):
"""
Get datetime of the next retry if the task instance fails. For exponential
backoff, retry_delay is used as base and will be converted to seconds.
"""
delay = self.task.retry_delay
if self.task.retry_exponential_backoff:
# If the min_backoff calculation is below 1, it will be converted to 0 via int. Thus,
# we must round up prior to converting to an int, otherwise a divide by zero error
# will occurr in the modded_hash calculation.
min_backoff = int(math.ceil(delay.total_seconds() * (2 ** (self.try_number - 2))))
# deterministic per task instance
hash = int(hashlib.sha1("{}#{}#{}#{}".format(self.dag_id,
self.task_id,
self.execution_date,
self.try_number)
.encode('utf-8')).hexdigest(), 16)
# between 1 and 1.0 * delay * (2^retry_number)
modded_hash = min_backoff + hash % min_backoff
# timedelta has a maximum representable value. The exponentiation
# here means this value can be exceeded after a certain number
# of tries (around 50 if the initial delay is 1s, even fewer if
# the delay is larger). Cap the value here before creating a
# timedelta object so the operation doesn't fail.
delay_backoff_in_seconds = min(
modded_hash,
timedelta.max.total_seconds() - 1
)
delay = timedelta(seconds=delay_backoff_in_seconds)
if self.task.max_retry_delay:
delay = min(self.task.max_retry_delay, delay)
return self.end_date + delay
def ready_for_retry(self):
"""
Checks on whether the task instance is in the right state and timeframe
to be retried.
"""
return (self.state == State.UP_FOR_RETRY and
self.next_retry_datetime() < timezone.utcnow())
@provide_session
def pool_full(self, session):
"""
Returns a boolean as to whether the slot pool has room for this
task to run
"""
if not self.task.pool:
return False
pool = (
session
.query(Pool)
.filter(Pool.pool == self.task.pool)
.first()
)
if not pool:
return False
open_slots = pool.open_slots(session=session)
return open_slots <= 0
@provide_session
def get_dagrun(self, session):
"""
Returns the DagRun for this TaskInstance
:param session:
:return: DagRun
"""
from airflow.models.dagrun import DagRun # Avoid circular import
dr = session.query(DagRun).filter(
DagRun.dag_id == self.dag_id,
DagRun.execution_date == self.execution_date
).first()
return dr
@provide_session
def _check_and_change_state_before_execution(
self,
verbose: bool = True,
ignore_all_deps: bool = False,
ignore_depends_on_past: bool = False,
ignore_task_deps: bool = False,
ignore_ti_state: bool = False,
mark_success: bool = False,
test_mode: bool = False,
job_id: Optional[str] = None,
pool: Optional[str] = None,
session=None) -> bool:
"""
Checks dependencies and then sets state to RUNNING if they are met. Returns
True if and only if state is set to RUNNING, which implies that task should be
executed, in preparation for _run_raw_task
:param verbose: whether to turn on more verbose logging
:type verbose: bool
:param ignore_all_deps: Ignore all of the non-critical dependencies, just runs
:type ignore_all_deps: bool
:param ignore_depends_on_past: Ignore depends_on_past DAG attribute
:type ignore_depends_on_past: bool
:param ignore_task_deps: Don't check the dependencies of this TaskInstance's task
:type ignore_task_deps: bool
:param ignore_ti_state: Disregards previous task instance state
:type ignore_ti_state: bool
:param mark_success: Don't run the task, mark its state as success
:type mark_success: bool
:param test_mode: Doesn't record success or failure in the DB
:type test_mode: bool
:param pool: specifies the pool to use to run the task instance
:type pool: str
:return: whether the state was changed to running or not
:rtype: bool
"""
task = self.task
self.pool = pool or task.pool
self.test_mode = test_mode
self.refresh_from_db(session=session, lock_for_update=True)
self.job_id = job_id
self.hostname = get_hostname()
self.operator = task.__class__.__name__
if not ignore_all_deps and not ignore_ti_state and self.state == State.SUCCESS:
Stats.incr('previously_succeeded', 1, 1)
# TODO: Logging needs cleanup, not clear what is being printed
hr = "\n" + ("-" * 80) # Line break
if not mark_success:
# Firstly find non-runnable and non-requeueable tis.
# Since mark_success is not set, we do nothing.
non_requeueable_dep_context = DepContext(
deps=RUNNING_DEPS - REQUEUEABLE_DEPS,
ignore_all_deps=ignore_all_deps,
ignore_ti_state=ignore_ti_state,
ignore_depends_on_past=ignore_depends_on_past,
ignore_task_deps=ignore_task_deps)
if not self.are_dependencies_met(
dep_context=non_requeueable_dep_context,
session=session,
verbose=True):
session.commit()
return False
# For reporting purposes, we report based on 1-indexed,
# not 0-indexed lists (i.e. Attempt 1 instead of
# Attempt 0 for the first attempt).
# Set the task start date. In case it was re-scheduled use the initial
# start date that is recorded in task_reschedule table
self.start_date = timezone.utcnow()
task_reschedules = TaskReschedule.find_for_task_instance(self, session)
if task_reschedules:
self.start_date = task_reschedules[0].start_date
# Secondly we find non-runnable but requeueable tis. We reset its state.
# This is because we might have hit concurrency limits,
# e.g. because of backfilling.
dep_context = DepContext(
deps=REQUEUEABLE_DEPS,
ignore_all_deps=ignore_all_deps,
ignore_depends_on_past=ignore_depends_on_past,
ignore_task_deps=ignore_task_deps,
ignore_ti_state=ignore_ti_state)
if not self.are_dependencies_met(
dep_context=dep_context,
session=session,
verbose=True):
self.state = State.NONE
self.log.warning(hr)
self.log.warning(
"Rescheduling due to concurrency limits reached "
"at task runtime. Attempt %s of "
"%s. State set to NONE.", self.try_number, self.max_tries + 1
)
self.log.warning(hr)
self.queued_dttm = timezone.utcnow()
session.merge(self)
session.commit()
return False
# print status message
self.log.info(hr)
self.log.info("Starting attempt %s of %s", self.try_number, self.max_tries + 1)
self.log.info(hr)
self._try_number += 1
if not test_mode:
session.add(Log(State.RUNNING, self))
self.state = State.RUNNING
self.pid = os.getpid()
self.end_date = None
if not test_mode:
session.merge(self)
session.commit()
# Closing all pooled connections to prevent
# "max number of connections reached"
settings.engine.dispose() # type: ignore
if verbose:
if mark_success:
self.log.info("Marking success for %s on %s", self.task, self.execution_date)
else:
self.log.info("Executing %s on %s", self.task, self.execution_date)
return True
@provide_session
@Sentry.enrich_errors
def _run_raw_task(
self,
mark_success: bool = False,
test_mode: bool = False,
job_id: Optional[str] = None,
pool: Optional[str] = None,
session=None) -> None:
"""
Immediately runs the task (without checking or changing db state
before execution) and then sets the appropriate final state after
completion and runs any post-execute callbacks. Meant to be called
only after another function changes the state to running.
:param mark_success: Don't run the task, mark its state as success
:type mark_success: bool
:param test_mode: Doesn't record success or failure in the DB
:type test_mode: bool
:param pool: specifies the pool to use to run the task instance
:type pool: str
"""
from airflow.sensors.base_sensor_operator import BaseSensorOperator
task = self.task
self.pool = pool or task.pool
self.test_mode = test_mode
self.refresh_from_db(session=session)
self.job_id = job_id
self.hostname = get_hostname()
self.operator = task.__class__.__name__
context = {} # type: Dict
actual_start_date = timezone.utcnow()
try:
if not mark_success:
context = self.get_template_context()
task_copy = copy.copy(task)
# Sensors in `poke` mode can block execution of DAGs when running
# with single process executor, thus we change the mode to`reschedule`
# to allow parallel task being scheduled and executed
if isinstance(task_copy, BaseSensorOperator) and \
conf.get('core', 'executor') == "DebugExecutor":
self.log.warning("DebugExecutor changes sensor mode to 'reschedule'.")
task_copy.mode = 'reschedule'
self.task = task_copy
def signal_handler(signum, frame):
self.log.error("Received SIGTERM. Terminating subprocesses.")
task_copy.on_kill()
raise AirflowException("Task received SIGTERM signal")
signal.signal(signal.SIGTERM, signal_handler)
# Don't clear Xcom until the task is certain to execute
self.clear_xcom_data()
start_time = time.time()
self.render_templates(context=context)
task_copy.pre_execute(context=context)
# If a timeout is specified for the task, make it fail
# if it goes beyond
result = None
if task_copy.execution_timeout:
try:
with timeout(int(
task_copy.execution_timeout.total_seconds())):
result = task_copy.execute(context=context)
except AirflowTaskTimeout:
task_copy.on_kill()
raise
else:
result = task_copy.execute(context=context)
# If the task returns a result, push an XCom containing it
if task_copy.do_xcom_push and result is not None:
self.xcom_push(key=XCOM_RETURN_KEY, value=result)
task_copy.post_execute(context=context, result=result)
end_time = time.time()
duration = end_time - start_time
Stats.timing(
'dag.{dag_id}.{task_id}.duration'.format(
dag_id=task_copy.dag_id,
task_id=task_copy.task_id),
duration)
Stats.incr('operator_successes_{}'.format(
self.task.__class__.__name__), 1, 1)
Stats.incr('ti_successes')
self.refresh_from_db(lock_for_update=True)
self.state = State.SUCCESS
except AirflowSkipException as e:
# log only if exception has any arguments to prevent log flooding
if e.args:
self.log.info(e)
self.refresh_from_db(lock_for_update=True)
self.state = State.SKIPPED
except AirflowRescheduleException as reschedule_exception:
self.refresh_from_db()
self._handle_reschedule(actual_start_date, reschedule_exception, test_mode, context)
return
except AirflowException as e:
self.refresh_from_db()
# for case when task is marked as success/failed externally
# current behavior doesn't hit the success callback
if self.state in {State.SUCCESS, State.FAILED}:
return
else:
self.handle_failure(e, test_mode, context)
raise
except (Exception, KeyboardInterrupt) as e:
self.handle_failure(e, test_mode, context)
raise
# Success callback
try:
if task.on_success_callback:
task.on_success_callback(context)
except Exception as e3:
self.log.error("Failed when executing success callback")
self.log.exception(e3)
# Recording SUCCESS
self.end_date = timezone.utcnow()
self.set_duration()
if not test_mode:
session.add(Log(self.state, self))
session.merge(self)
session.commit()
@provide_session
def run(
self,