-
Notifications
You must be signed in to change notification settings - Fork 13.7k
/
__init__.py
executable file
·4744 lines (4147 loc) · 173 KB
/
__init__.py
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
# -*- coding: utf-8 -*-
#
# Licensed to the Apache Software Foundation (ASF) under one
# or more contributor license agreements. See the NOTICE file
# distributed with this work for additional information
# regarding copyright ownership. The ASF licenses this file
# to you under the Apache License, Version 2.0 (the
# "License"); you may not use this file except in compliance
# with the License. You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing,
# software distributed under the License is distributed on an
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
# KIND, either express or implied. See the License for the
# specific language governing permissions and limitations
# under the License.
from __future__ import absolute_import
from __future__ import division
from __future__ import print_function
from __future__ import unicode_literals
import copy
from collections import defaultdict, namedtuple, OrderedDict
from builtins import ImportError as BuiltinImportError, bytes, object, str
from future.standard_library import install_aliases
from airflow.models.base import Base, ID_LEN
try:
# Fix Python > 3.7 deprecation
from collections.abc import Hashable
except ImportError:
# Preserve Python < 3.3 compatibility
from collections import Hashable
from datetime import timedelta
import dill
import functools
import getpass
import imp
import importlib
import zipfile
import jinja2
import json
import logging
import os
import pendulum
import pickle
import re
import signal
import sys
import textwrap
import traceback
import warnings
import hashlib
from datetime import datetime
from urllib.parse import quote
from sqlalchemy import (
Boolean, Column, DateTime, Float, Index, Integer, PickleType, String,
Text, UniqueConstraint, and_, func, or_
)
from sqlalchemy.ext.declarative import declared_attr
from sqlalchemy.orm import reconstructor, synonym
from croniter import (
croniter, CroniterBadCronError, CroniterBadDateError, CroniterNotAlphaError
)
import six
from airflow import settings, utils
from airflow.executors import get_default_executor, LocalExecutor
from airflow import configuration
from airflow.exceptions import (
AirflowDagCycleException, AirflowException, AirflowSkipException, AirflowTaskTimeout,
AirflowRescheduleException
)
from airflow.dag.base_dag import BaseDag, BaseDagBag
from airflow.lineage import apply_lineage, prepare_lineage
from airflow.models.dagpickle import DagPickle
from airflow.models.kubernetes import KubeWorkerIdentifier, KubeResourceVersion # noqa: F401
from airflow.models.log import Log
from airflow.models.taskfail import TaskFail
from airflow.models.taskreschedule import TaskReschedule
from airflow.models.xcom import XCom
from airflow.ti_deps.deps.not_in_retry_period_dep import NotInRetryPeriodDep
from airflow.ti_deps.deps.prev_dagrun_dep import PrevDagrunDep
from airflow.ti_deps.deps.trigger_rule_dep import TriggerRuleDep
from airflow.ti_deps.dep_context import DepContext, QUEUE_DEPS, RUN_DEPS
from airflow.utils import timezone
from airflow.utils.dag_processing import list_py_file_paths
from airflow.utils.dates import cron_presets, date_range as utils_date_range
from airflow.utils.db import provide_session
from airflow.utils.decorators import apply_defaults
from airflow.utils.email import send_email
from airflow.utils.helpers import is_container, validate_key, pprinttable
from airflow.utils.operator_resources import Resources
from airflow.utils.state import State
from airflow.utils.sqlalchemy import UtcDateTime, Interval
from airflow.utils.timeout import timeout
from airflow.utils.trigger_rule import TriggerRule
from airflow.utils.weight_rule import WeightRule
from airflow.utils.net import get_hostname
from airflow.utils.log.logging_mixin import LoggingMixin
install_aliases()
XCOM_RETURN_KEY = 'return_value'
Stats = settings.Stats
class InvalidFernetToken(Exception):
# If Fernet isn't loaded we need a valid exception class to catch. If it is
# loaded this will get reset to the actual class once get_fernet() is called
pass
class NullFernet(object):
"""
A "Null" encryptor class that doesn't encrypt or decrypt but that presents
a similar interface to Fernet.
The purpose of this is to make the rest of the code not have to know the
difference, and to only display the message once, not 20 times when
`airflow initdb` is ran.
"""
is_encrypted = False
def decrpyt(self, b):
return b
def encrypt(self, b):
return b
_fernet = None
def get_fernet():
"""
Deferred load of Fernet key.
This function could fail either because Cryptography is not installed
or because the Fernet key is invalid.
:return: Fernet object
:raises: airflow.exceptions.AirflowException if there's a problem trying to load Fernet
"""
global _fernet
log = LoggingMixin().log
if _fernet:
return _fernet
try:
from cryptography.fernet import Fernet, MultiFernet, InvalidToken
global InvalidFernetToken
InvalidFernetToken = InvalidToken
except BuiltinImportError:
log.warning(
"cryptography not found - values will not be stored encrypted."
)
_fernet = NullFernet()
return _fernet
try:
fernet_key = configuration.conf.get('core', 'FERNET_KEY')
if not fernet_key:
log.warning(
"empty cryptography key - values will not be stored encrypted."
)
_fernet = NullFernet()
else:
_fernet = MultiFernet([
Fernet(fernet_part.encode('utf-8'))
for fernet_part in fernet_key.split(',')
])
_fernet.is_encrypted = True
except (ValueError, TypeError) as ve:
raise AirflowException("Could not create Fernet object: {}".format(ve))
return _fernet
# Used by DAG context_managers
_CONTEXT_MANAGER_DAG = None
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 current task try number.
ti.max_tries = max(ti.max_tries, ti.try_number - 1)
ti.state = State.NONE
session.merge(ti)
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:
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()
def get_last_dagrun(dag_id, session, include_externally_triggered=False):
"""
Returns the last dag run for a dag, None if there was none.
Last dag run can be any type of run eg. scheduled or backfilled.
Overridden DagRuns are ignored.
"""
DR = DagRun
query = session.query(DR).filter(DR.dag_id == dag_id)
if not include_externally_triggered:
query = query.filter(DR.external_trigger == False) # noqa
query = query.order_by(DR.execution_date.desc())
return query.first()
class DagBag(BaseDagBag, LoggingMixin):
"""
A dagbag is a collection of dags, parsed out of a folder tree and has high
level configuration settings, like what database to use as a backend and
what executor to use to fire off tasks. This makes it easier to run
distinct environments for say production and development, tests, or for
different teams or security profiles. What would have been system level
settings are now dagbag level so that one system can run multiple,
independent settings sets.
:param dag_folder: the folder to scan to find DAGs
:type dag_folder: unicode
:param executor: the executor to use when executing task instances
in this DagBag
:param include_examples: whether to include the examples that ship
with airflow or not
:type include_examples: bool
:param has_logged: an instance boolean that gets flipped from False to True after a
file has been skipped. This is to prevent overloading the user with logging
messages about skipped files. Therefore only once per DagBag is a file logged
being skipped.
"""
# static class variables to detetct dag cycle
CYCLE_NEW = 0
CYCLE_IN_PROGRESS = 1
CYCLE_DONE = 2
def __init__(
self,
dag_folder=None,
executor=None,
include_examples=configuration.conf.getboolean('core', 'LOAD_EXAMPLES'),
safe_mode=configuration.conf.getboolean('core', 'DAG_DISCOVERY_SAFE_MODE')):
# do not use default arg in signature, to fix import cycle on plugin load
if executor is None:
executor = get_default_executor()
dag_folder = dag_folder or settings.DAGS_FOLDER
self.log.info("Filling up the DagBag from %s", dag_folder)
self.dag_folder = dag_folder
self.dags = {}
# the file's last modified timestamp when we last read it
self.file_last_changed = {}
self.executor = executor
self.import_errors = {}
self.has_logged = False
self.collect_dags(
dag_folder=dag_folder,
include_examples=include_examples,
safe_mode=safe_mode)
def size(self):
"""
:return: the amount of dags contained in this dagbag
"""
return len(self.dags)
def get_dag(self, dag_id):
"""
Gets the DAG out of the dictionary, and refreshes it if expired
"""
# If asking for a known subdag, we want to refresh the parent
root_dag_id = dag_id
if dag_id in self.dags:
dag = self.dags[dag_id]
if dag.is_subdag:
root_dag_id = dag.parent_dag.dag_id
# If the dag corresponding to root_dag_id is absent or expired
orm_dag = DagModel.get_current(root_dag_id)
if orm_dag and (
root_dag_id not in self.dags or
(
orm_dag.last_expired and
dag.last_loaded < orm_dag.last_expired
)
):
# Reprocess source file
found_dags = self.process_file(
filepath=orm_dag.fileloc, only_if_updated=False)
# If the source file no longer exports `dag_id`, delete it from self.dags
if found_dags and dag_id in [found_dag.dag_id for found_dag in found_dags]:
return self.dags[dag_id]
elif dag_id in self.dags:
del self.dags[dag_id]
return self.dags.get(dag_id)
def process_file(self, filepath, only_if_updated=True, safe_mode=True):
"""
Given a path to a python module or zip file, this method imports
the module and look for dag objects within it.
"""
found_dags = []
# if the source file no longer exists in the DB or in the filesystem,
# return an empty list
# todo: raise exception?
if filepath is None or not os.path.isfile(filepath):
return found_dags
try:
# This failed before in what may have been a git sync
# race condition
file_last_changed_on_disk = datetime.fromtimestamp(os.path.getmtime(filepath))
if only_if_updated \
and filepath in self.file_last_changed \
and file_last_changed_on_disk == self.file_last_changed[filepath]:
return found_dags
except Exception as e:
self.log.exception(e)
return found_dags
mods = []
is_zipfile = zipfile.is_zipfile(filepath)
if not is_zipfile:
if safe_mode and os.path.isfile(filepath):
with open(filepath, 'rb') as f:
content = f.read()
if not all([s in content for s in (b'DAG', b'airflow')]):
self.file_last_changed[filepath] = file_last_changed_on_disk
# Don't want to spam user with skip messages
if not self.has_logged:
self.has_logged = True
self.log.info(
"File %s assumed to contain no DAGs. Skipping.",
filepath)
return found_dags
self.log.debug("Importing %s", filepath)
org_mod_name, _ = os.path.splitext(os.path.split(filepath)[-1])
mod_name = ('unusual_prefix_' +
hashlib.sha1(filepath.encode('utf-8')).hexdigest() +
'_' + org_mod_name)
if mod_name in sys.modules:
del sys.modules[mod_name]
with timeout(configuration.conf.getint('core', "DAGBAG_IMPORT_TIMEOUT")):
try:
m = imp.load_source(mod_name, filepath)
mods.append(m)
except Exception as e:
self.log.exception("Failed to import: %s", filepath)
self.import_errors[filepath] = str(e)
self.file_last_changed[filepath] = file_last_changed_on_disk
else:
zip_file = zipfile.ZipFile(filepath)
for mod in zip_file.infolist():
head, _ = os.path.split(mod.filename)
mod_name, ext = os.path.splitext(mod.filename)
if not head and (ext == '.py' or ext == '.pyc'):
if mod_name == '__init__':
self.log.warning("Found __init__.%s at root of %s", ext, filepath)
if safe_mode:
with zip_file.open(mod.filename) as zf:
self.log.debug("Reading %s from %s", mod.filename, filepath)
content = zf.read()
if not all([s in content for s in (b'DAG', b'airflow')]):
self.file_last_changed[filepath] = (
file_last_changed_on_disk)
# todo: create ignore list
# Don't want to spam user with skip messages
if not self.has_logged:
self.has_logged = True
self.log.info(
"File %s assumed to contain no DAGs. Skipping.",
filepath)
if mod_name in sys.modules:
del sys.modules[mod_name]
try:
sys.path.insert(0, filepath)
m = importlib.import_module(mod_name)
mods.append(m)
except Exception as e:
self.log.exception("Failed to import: %s", filepath)
self.import_errors[filepath] = str(e)
self.file_last_changed[filepath] = file_last_changed_on_disk
for m in mods:
for dag in list(m.__dict__.values()):
if isinstance(dag, DAG):
if not dag.full_filepath:
dag.full_filepath = filepath
if dag.fileloc != filepath and not is_zipfile:
dag.fileloc = filepath
try:
dag.is_subdag = False
self.bag_dag(dag, parent_dag=dag, root_dag=dag)
if isinstance(dag._schedule_interval, six.string_types):
croniter(dag._schedule_interval)
found_dags.append(dag)
found_dags += dag.subdags
except (CroniterBadCronError,
CroniterBadDateError,
CroniterNotAlphaError) as cron_e:
self.log.exception("Failed to bag_dag: %s", dag.full_filepath)
self.import_errors[dag.full_filepath] = \
"Invalid Cron expression: " + str(cron_e)
self.file_last_changed[dag.full_filepath] = \
file_last_changed_on_disk
except AirflowDagCycleException as cycle_exception:
self.log.exception("Failed to bag_dag: %s", dag.full_filepath)
self.import_errors[dag.full_filepath] = str(cycle_exception)
self.file_last_changed[dag.full_filepath] = \
file_last_changed_on_disk
self.file_last_changed[filepath] = file_last_changed_on_disk
return found_dags
@provide_session
def kill_zombies(self, zombies, session=None):
"""
Fail given zombie tasks, which are tasks that haven't
had a heartbeat for too long, in the current DagBag.
:param zombies: zombie task instances to kill.
:type zombies: airflow.utils.dag_processing.SimpleTaskInstance
:param session: DB session.
:type session: sqlalchemy.orm.session.Session
"""
for zombie in zombies:
if zombie.dag_id in self.dags:
dag = self.dags[zombie.dag_id]
if zombie.task_id in dag.task_ids:
task = dag.get_task(zombie.task_id)
ti = TaskInstance(task, zombie.execution_date)
# Get properties needed for failure handling from SimpleTaskInstance.
ti.start_date = zombie.start_date
ti.end_date = zombie.end_date
ti.try_number = zombie.try_number
ti.state = zombie.state
ti.test_mode = configuration.getboolean('core', 'unit_test_mode')
ti.handle_failure("{} detected as zombie".format(ti),
ti.test_mode, ti.get_template_context())
self.log.info(
'Marked zombie job %s as %s', ti, ti.state)
Stats.incr('zombies_killed')
session.commit()
def bag_dag(self, dag, parent_dag, root_dag):
"""
Adds the DAG into the bag, recurses into sub dags.
Throws AirflowDagCycleException if a cycle is detected in this dag or its subdags
"""
dag.test_cycle() # throws if a task cycle is found
dag.resolve_template_files()
dag.last_loaded = timezone.utcnow()
for task in dag.tasks:
settings.policy(task)
subdags = dag.subdags
try:
for subdag in subdags:
subdag.full_filepath = dag.full_filepath
subdag.parent_dag = dag
subdag.is_subdag = True
self.bag_dag(subdag, parent_dag=dag, root_dag=root_dag)
self.dags[dag.dag_id] = dag
self.log.debug('Loaded DAG {dag}'.format(**locals()))
except AirflowDagCycleException as cycle_exception:
# There was an error in bagging the dag. Remove it from the list of dags
self.log.exception('Exception bagging dag: {dag.dag_id}'.format(**locals()))
# Only necessary at the root level since DAG.subdags automatically
# performs DFS to search through all subdags
if dag == root_dag:
for subdag in subdags:
if subdag.dag_id in self.dags:
del self.dags[subdag.dag_id]
raise cycle_exception
def collect_dags(
self,
dag_folder=None,
only_if_updated=True,
include_examples=configuration.conf.getboolean('core', 'LOAD_EXAMPLES'),
safe_mode=configuration.conf.getboolean('core', 'DAG_DISCOVERY_SAFE_MODE')):
"""
Given a file path or a folder, this method looks for python modules,
imports them and adds them to the dagbag collection.
Note that if a ``.airflowignore`` file is found while processing
the directory, it will behave much like a ``.gitignore``,
ignoring files that match any of the regex patterns specified
in the file.
**Note**: The patterns in .airflowignore are treated as
un-anchored regexes, not shell-like glob patterns.
"""
start_dttm = timezone.utcnow()
dag_folder = dag_folder or self.dag_folder
# Used to store stats around DagBag processing
stats = []
FileLoadStat = namedtuple(
'FileLoadStat', "file duration dag_num task_num dags")
for filepath in list_py_file_paths(dag_folder, safe_mode=safe_mode,
include_examples=include_examples):
try:
ts = timezone.utcnow()
found_dags = self.process_file(
filepath, only_if_updated=only_if_updated,
safe_mode=safe_mode)
td = timezone.utcnow() - ts
td = td.total_seconds() + (
float(td.microseconds) / 1000000)
stats.append(FileLoadStat(
filepath.replace(dag_folder, ''),
td,
len(found_dags),
sum([len(dag.tasks) for dag in found_dags]),
str([dag.dag_id for dag in found_dags]),
))
except Exception as e:
self.log.exception(e)
Stats.gauge(
'collect_dags', (timezone.utcnow() - start_dttm).total_seconds(), 1)
Stats.gauge(
'dagbag_size', len(self.dags), 1)
Stats.gauge(
'dagbag_import_errors', len(self.import_errors), 1)
self.dagbag_stats = sorted(
stats, key=lambda x: x.duration, reverse=True)
def dagbag_report(self):
"""Prints a report around DagBag loading stats"""
report = textwrap.dedent("""\n
-------------------------------------------------------------------
DagBag loading stats for {dag_folder}
-------------------------------------------------------------------
Number of DAGs: {dag_num}
Total task number: {task_num}
DagBag parsing time: {duration}
{table}
""")
stats = self.dagbag_stats
return report.format(
dag_folder=self.dag_folder,
duration=sum([o.duration for o in stats]),
dag_num=sum([o.dag_num for o in stats]),
task_num=sum([o.task_num for o in stats]),
table=pprinttable(stats),
)
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))
queue = Column(String(50))
priority_weight = Column(Integer)
operator = Column(String(1000))
queued_dttm = Column(UtcDateTime)
pid = Column(Integer)
executor_config = Column(PickleType(pickler=dill))
__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 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 TI is currently running, this will match the column in the
databse, in all othercases this will be incremenetd
"""
# 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 next_try_number(self):
return self._try_number + 1
def command(
self,
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,
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.
"""
return " ".join(self.command_as_list(
mark_success=mark_success,
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,
local=local,
pickle_id=pickle_id,
raw=raw,
job_id=job_id,
pool=pool,
cfg_path=cfg_path))
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
: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: basestring
:return: shell command that can be used to run the task instance
"""
iso = execution_date.isoformat()
cmd = ["airflow", "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(configuration.conf.get('core', 'BASE_LOG_FOLDER'))
return (
"{log}/{self.dag_id}/{self.task_id}/{iso}.log".format(**locals()))
@property
def log_url(self):
iso = quote(self.execution_date.isoformat())
base_url = configuration.conf.get('webserver', 'BASE_URL')
return base_url + (
"/log?"
"execution_date={iso}"
"&task_id={self.task_id}"
"&dag_id={self.dag_id}"
).format(**locals())
@property
def mark_success_url(self):
iso = quote(self.execution_date.isoformat())
base_url = configuration.conf.get('webserver', 'BASE_URL')
return base_url + (
"/success"
"?task_id={self.task_id}"
"&dag_id={self.dag_id}"
"&execution_date={iso}"
"&upstream=false"
"&downstream=false"
).format(**locals())
@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 = TaskInstance
ti = session.query(TI).filter(
TI.dag_id == self.dag_id,
TI.task_id == self.task_id,
TI.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):
"""
Refreshes the task instance from the database based on the primary key
:param lock_for_update: if True, indicates that the database should
lock the TaskInstance (issuing a FOR UPDATE clause) until the
session is committed.
"""
TI = TaskInstance
qry = session.query(TI).filter(
TI.dag_id == self.dag_id,
TI.task_id == self.task_id,
TI.execution_date == self.execution_date)
if lock_for_update:
ti = qry.with_for_update().first()
else:
ti = qry.first()
if ti:
self.state = ti.state
self.start_date = ti.start_date
self.end_date = ti.end_date
# Get the raw value of try_number column, don't read through the
# accessor here otherwise it will be incremeneted by one already.
self.try_number = ti._try_number
self.max_tries = ti.max_tries
self.hostname = ti.hostname
self.pid = ti.pid
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):
"""
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):
self.state = state
self.start_date = timezone.utcnow()
self.end_date = timezone.utcnow()
session.merge(self)
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