-
Notifications
You must be signed in to change notification settings - Fork 5
/
Copy pathmodels.py
executable file
·4239 lines (3704 loc) · 147 KB
/
models.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 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
from future.standard_library import install_aliases
install_aliases()
from builtins import str
from builtins import object, bytes
import copy
from collections import namedtuple
from datetime import datetime, timedelta
import dill
import functools
import getpass
import imp
import importlib
import inspect
import zipfile
import jinja2
import json
import logging
import os
import pickle
import re
import signal
import socket
import sys
import textwrap
import traceback
import warnings
import hashlib
from urllib.parse import urlparse
from sqlalchemy import (
Column, Integer, String, DateTime, Text, Boolean, ForeignKey, PickleType,
Index, Float)
from sqlalchemy import func, or_, and_
from sqlalchemy.ext.declarative import declarative_base, declared_attr
from sqlalchemy.dialects.mysql import LONGTEXT
from sqlalchemy.orm import reconstructor, relationship, synonym
from croniter import croniter
import six
from airflow import settings, utils
from airflow.executors import DEFAULT_EXECUTOR, LocalExecutor
from airflow import configuration
from airflow.exceptions import AirflowException, AirflowSkipException, AirflowTaskTimeout
from airflow.dag.base_dag import BaseDag, BaseDagBag
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.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 (
as_tuple, is_container, is_in, validate_key, pprinttable)
from airflow.utils.logging import LoggingMixin
from airflow.utils.operator_resources import Resources
from airflow.utils.state import State
from airflow.utils.timeout import timeout
from airflow.utils.trigger_rule import TriggerRule
Base = declarative_base()
ID_LEN = 250
XCOM_RETURN_KEY = 'return_value'
Stats = settings.Stats
ENCRYPTION_ON = False
try:
from cryptography.fernet import Fernet
FERNET = Fernet(configuration.get('core', 'FERNET_KEY').encode('utf-8'))
ENCRYPTION_ON = True
except:
pass
if 'mysql' in settings.SQL_ALCHEMY_CONN:
LongText = LONGTEXT
else:
LongText = Text
# used by DAG context_managers
_CONTEXT_MANAGER_DAG = None
def clear_task_instances(tis, session, activate_dag_runs=True):
"""
Clears a set of task instances, but makes sure the running ones
get killed.
"""
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)
# todo: this creates an issue with the webui tests
# elif ti.state != State.REMOVED:
# ti.state = State.NONE
# session.merge(ti)
else:
session.delete(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:
execution_dates = {ti.execution_date for ti in tis}
dag_ids = {ti.dag_id for ti in tis}
drs = session.query(DagRun).filter(
DagRun.dag_id.in_(dag_ids),
DagRun.execution_date.in_(execution_dates),
).all()
for dr in drs:
dr.state = State.RUNNING
dr.start_date = datetime.now()
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 sync_to_db: whether to sync the properties of the DAGs to
the metadata DB while finding them, typically should be done
by the scheduler job only
:type sync_to_db: bool
"""
def __init__(
self,
dag_folder=None,
executor=DEFAULT_EXECUTOR,
include_examples=configuration.getboolean('core', 'LOAD_EXAMPLES')):
dag_folder = dag_folder or settings.DAGS_FOLDER
self.logger.info("Filling up the DagBag from {}".format(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 = {}
if include_examples:
example_dag_folder = os.path.join(
os.path.dirname(__file__),
'example_dags')
self.collect_dags(example_dag_folder)
self.collect_dags(dag_folder)
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 [dag.dag_id for 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 = []
# todo: raise exception?
if 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:
logging.exception(e)
return found_dags
mods = []
if not zipfile.is_zipfile(filepath):
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
return found_dags
self.logger.debug("Importing {}".format(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.getint('core', "DAGBAG_IMPORT_TIMEOUT")):
try:
m = imp.load_source(mod_name, filepath)
mods.append(m)
except Exception as e:
self.logger.exception("Failed to import: " + 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.logger.warning("Found __init__.{0} at root of {1}".
format(ext, filepath))
if safe_mode:
with zip_file.open(mod.filename) as zf:
self.logger.debug("Reading {} from {}".
format(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
return found_dags
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.logger.exception("Failed to import: " + 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
dag.is_subdag = False
self.bag_dag(dag, parent_dag=dag, root_dag=dag)
found_dags.append(dag)
found_dags += dag.subdags
self.file_last_changed[filepath] = file_last_changed_on_disk
return found_dags
@provide_session
def kill_zombies(self, session=None):
"""
Fails tasks that haven't had a heartbeat in too long
"""
from airflow.jobs import LocalTaskJob as LJ
self.logger.info("Finding 'running' jobs without a recent heartbeat")
TI = TaskInstance
secs = (
configuration.getint('scheduler', 'scheduler_zombie_task_threshold'))
limit_dttm = datetime.now() - timedelta(seconds=secs)
self.logger.info(
"Failing jobs without heartbeat after {}".format(limit_dttm))
tis = (
session.query(TI)
.join(LJ, TI.job_id == LJ.id)
.filter(TI.state == State.RUNNING)
.filter(
or_(
LJ.state != State.RUNNING,
LJ.latest_heartbeat < limit_dttm,
))
.all()
)
for ti in tis:
if ti and ti.dag_id in self.dags:
dag = self.dags[ti.dag_id]
if ti.task_id in dag.task_ids:
task = dag.get_task(ti.task_id)
ti.task = task
ti.handle_failure("{} killed as zombie".format(ti))
self.logger.info(
'Marked zombie job {} as failed'.format(ti))
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.
"""
self.dags[dag.dag_id] = dag
dag.resolve_template_files()
dag.last_loaded = datetime.now()
for task in dag.tasks:
settings.policy(task)
for subdag in dag.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.logger.debug('Loaded DAG {dag}'.format(**locals()))
def collect_dags(
self,
dag_folder=None,
only_if_updated=True):
"""
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 behaves much like a .gitignore does,
ignoring files that match any of the regex patterns specified
in the file.
"""
start_dttm = datetime.now()
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")
if os.path.isfile(dag_folder):
self.process_file(dag_folder, only_if_updated=only_if_updated)
elif os.path.isdir(dag_folder):
patterns = []
for root, dirs, files in os.walk(dag_folder, followlinks=True):
ignore_file = [f for f in files if f == '.airflowignore']
if ignore_file:
f = open(os.path.join(root, ignore_file[0]), 'r')
patterns += [p for p in f.read().split('\n') if p]
f.close()
for f in files:
try:
filepath = os.path.join(root, f)
if not os.path.isfile(filepath):
continue
mod_name, file_ext = os.path.splitext(
os.path.split(filepath)[-1])
if file_ext != '.py' and not zipfile.is_zipfile(filepath):
continue
if not any(
[re.findall(p, filepath) for p in patterns]):
ts = datetime.now()
found_dags = self.process_file(
filepath, only_if_updated=only_if_updated)
td = datetime.now() - 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:
logging.warning(e)
Stats.gauge(
'collect_dags', (datetime.now() - 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.dag_num for o in stats]),
table=pprinttable(stats),
)
def deactivate_inactive_dags(self):
active_dag_ids = [dag.dag_id for dag in list(self.dags.values())]
session = settings.Session()
for dag in session.query(
DagModel).filter(~DagModel.dag_id.in_(active_dag_ids)).all():
dag.is_active = False
session.merge(dag)
session.commit()
session.close()
def paused_dags(self):
session = settings.Session()
dag_ids = [dp.dag_id for dp in session.query(DagModel).filter(
DagModel.is_paused.is_(True))]
session.commit()
session.close()
return dag_ids
class User(Base):
__tablename__ = "users"
id = Column(Integer, primary_key=True)
username = Column(String(ID_LEN), unique=True)
email = Column(String(500))
superuser = Column('superuser', Boolean, default=False)
def __repr__(self):
return self.username
def get_id(self):
return str(self.id)
def is_superuser(self):
return self.superuser
class Connection(Base):
"""
Placeholder to store information about different database instances
connection information. The idea here is that scripts use references to
database instances (conn_id) instead of hard coding hostname, logins and
passwords when using operators or hooks.
"""
__tablename__ = "connection"
id = Column(Integer(), primary_key=True)
conn_id = Column(String(ID_LEN))
conn_type = Column(String(500))
host = Column(String(500))
schema = Column(String(500))
login = Column(String(500))
_password = Column('password', String(5000))
port = Column(Integer())
is_encrypted = Column(Boolean, unique=False, default=False)
is_extra_encrypted = Column(Boolean, unique=False, default=False)
_extra = Column('extra', String(5000))
_types = [
('fs', 'File (path)'),
('ftp', 'FTP',),
('google_cloud_platform', 'Google Cloud Platform'),
('hdfs', 'HDFS',),
('http', 'HTTP',),
('hive_cli', 'Hive Client Wrapper',),
('hive_metastore', 'Hive Metastore Thrift',),
('hiveserver2', 'Hive Server 2 Thrift',),
('jdbc', 'Jdbc Connection',),
('mysql', 'MySQL',),
('postgres', 'Postgres',),
('oracle', 'Oracle',),
('vertica', 'Vertica',),
('presto', 'Presto',),
('s3', 'S3',),
('samba', 'Samba',),
('sqlite', 'Sqlite',),
('ssh', 'SSH',),
('cloudant', 'IBM Cloudant',),
('mssql', 'Microsoft SQL Server'),
('mesos_framework-id', 'Mesos Framework ID'),
('jira', 'JIRA',),
]
def __init__(
self, conn_id=None, conn_type=None,
host=None, login=None, password=None,
schema=None, port=None, extra=None,
uri=None):
self.conn_id = conn_id
if uri:
self.parse_from_uri(uri)
else:
self.conn_type = conn_type
self.host = host
self.login = login
self.password = password
self.schema = schema
self.port = port
self.extra = extra
def parse_from_uri(self, uri):
temp_uri = urlparse(uri)
hostname = temp_uri.hostname or ''
if '%2f' in hostname:
hostname = hostname.replace('%2f', '/').replace('%2F', '/')
conn_type = temp_uri.scheme
if conn_type == 'postgresql':
conn_type = 'postgres'
self.conn_type = conn_type
self.host = hostname
self.schema = temp_uri.path[1:]
self.login = temp_uri.username
self.password = temp_uri.password
self.port = temp_uri.port
def get_password(self):
if self._password and self.is_encrypted:
if not ENCRYPTION_ON:
raise AirflowException(
"Can't decrypt encrypted password for login={}, \
FERNET_KEY configuration is missing".format(self.login))
return FERNET.decrypt(bytes(self._password, 'utf-8')).decode()
else:
return self._password
def set_password(self, value):
if value:
try:
self._password = FERNET.encrypt(bytes(value, 'utf-8')).decode()
self.is_encrypted = True
except NameError:
self._password = value
self.is_encrypted = False
@declared_attr
def password(cls):
return synonym('_password',
descriptor=property(cls.get_password, cls.set_password))
def get_extra(self):
if self._extra and self.is_extra_encrypted:
if not ENCRYPTION_ON:
raise AirflowException(
"Can't decrypt `extra` params for login={},\
FERNET_KEY configuration is missing".format(self.login))
return FERNET.decrypt(bytes(self._extra, 'utf-8')).decode()
else:
return self._extra
def set_extra(self, value):
if value:
try:
self._extra = FERNET.encrypt(bytes(value, 'utf-8')).decode()
self.is_extra_encrypted = True
except NameError:
self._extra = value
self.is_extra_encrypted = False
@declared_attr
def extra(cls):
return synonym('_extra',
descriptor=property(cls.get_extra, cls.set_extra))
def get_hook(self):
try:
if self.conn_type == 'mysql':
from airflow.hooks.mysql_hook import MySqlHook
return MySqlHook(mysql_conn_id=self.conn_id)
elif self.conn_type == 'google_cloud_platform':
from airflow.contrib.hooks.bigquery_hook import BigQueryHook
return BigQueryHook(bigquery_conn_id=self.conn_id)
elif self.conn_type == 'postgres':
from airflow.hooks.postgres_hook import PostgresHook
return PostgresHook(postgres_conn_id=self.conn_id)
elif self.conn_type == 'hive_cli':
from airflow.hooks.hive_hooks import HiveCliHook
return HiveCliHook(hive_cli_conn_id=self.conn_id)
elif self.conn_type == 'presto':
from airflow.hooks.presto_hook import PrestoHook
return PrestoHook(presto_conn_id=self.conn_id)
elif self.conn_type == 'hiveserver2':
from airflow.hooks.hive_hooks import HiveServer2Hook
return HiveServer2Hook(hiveserver2_conn_id=self.conn_id)
elif self.conn_type == 'sqlite':
from airflow.hooks.sqlite_hook import SqliteHook
return SqliteHook(sqlite_conn_id=self.conn_id)
elif self.conn_type == 'jdbc':
from airflow.hooks.jdbc_hook import JdbcHook
return JdbcHook(jdbc_conn_id=self.conn_id)
elif self.conn_type == 'mssql':
from airflow.hooks.mssql_hook import MsSqlHook
return MsSqlHook(mssql_conn_id=self.conn_id)
elif self.conn_type == 'oracle':
from airflow.hooks.oracle_hook import OracleHook
return OracleHook(oracle_conn_id=self.conn_id)
elif self.conn_type == 'vertica':
from airflow.contrib.hooks.vertica_hook import VerticaHook
return VerticaHook(vertica_conn_id=self.conn_id)
elif self.conn_type == 'cloudant':
from airflow.contrib.hooks.cloudant_hook import CloudantHook
return CloudantHook(cloudant_conn_id=self.conn_id)
elif self.conn_type == 'jira':
from airflow.contrib.hooks.jira_hook import JiraHook
return JiraHook(jira_conn_id=self.conn_id)
except:
pass
def __repr__(self):
return self.conn_id
@property
def extra_dejson(self):
"""Returns the extra property by deserializing json."""
obj = {}
if self.extra:
try:
obj = json.loads(self.extra)
except Exception as e:
logging.exception(e)
logging.error("Failed parsing the json for conn_id %s", self.conn_id)
return obj
class DagPickle(Base):
"""
Dags can originate from different places (user repos, master repo, ...)
and also get executed in different places (different executors). This
object represents a version of a DAG and becomes a source of truth for
a BackfillJob execution. A pickle is a native python serialized object,
and in this case gets stored in the database for the duration of the job.
The executors pick up the DagPickle id and read the dag definition from
the database.
"""
id = Column(Integer, primary_key=True)
pickle = Column(PickleType(pickler=dill))
created_dttm = Column(DateTime, default=func.now())
pickle_hash = Column(Text)
__tablename__ = "dag_pickle"
def __init__(self, dag):
self.dag_id = dag.dag_id
if hasattr(dag, 'template_env'):
dag.template_env = None
self.pickle_hash = hash(dag)
self.pickle = dag
class TaskInstance(Base):
"""
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 SqlAchemy 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(DateTime, primary_key=True)
start_date = Column(DateTime)
end_date = Column(DateTime)
duration = Column(Float)
state = Column(String(20))
try_number = Column(Integer, default=0)
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(DateTime)
pid = Column(Integer)
__table_args__ = (
Index('ti_dag_state', dag_id, state),
Index('ti_state', state),
Index('ti_state_lkp', dag_id, task_id, execution_date, state),
Index('ti_pool', pool, state, priority_weight),
)
def __init__(self, task, execution_date, state=None):
self.dag_id = task.dag_id
self.task_id = task.task_id
self.execution_date = execution_date
self.task = task
self.queue = task.queue
self.pool = task.pool
self.priority_weight = task.priority_weight_total
self.try_number = 0
self.unixname = getpass.getuser()
self.run_as_user = task.run_as_user
if state:
self.state = state
self.hostname = ''
self.init_on_load()
@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'
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 ignoreable dependencies.
Overrides the other ignore_* parameters.
:type ignore_all_deps: boolean
:param ignore_depends_on_past: Ignore depends_on_past parameter of DAGs
(e.g. for Backfills)
:type ignore_depends_on_past: boolean
:param ignore_task_deps: Ignore task-specific dependencies such as depends_on_past
and trigger rule
:type ignore_task_deps: boolean
:param ignore_ti_state: Ignore the task instance's previous failure/success
:type ignore_ti_state: boolean
: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
: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.get('core', 'BASE_LOG_FOLDER'))
return (
"{log}/{self.dag_id}/{self.task_id}/{iso}.log".format(**locals()))
@property
def log_url(self):
iso = self.execution_date.isoformat()
BASE_URL = configuration.get('webserver', 'BASE_URL')
return BASE_URL + (
"/admin/airflow/log"
"?dag_id={self.dag_id}"
"&task_id={self.task_id}"
"&execution_date={iso}"
).format(**locals())
@property
def mark_success_url(self):
iso = self.execution_date.isoformat()
BASE_URL = configuration.get('webserver', 'BASE_URL')
return BASE_URL + (
"/admin/airflow/action"
"?action=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.
"""
logging.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
self.try_number = ti.try_number
self.hostname = ti.hostname
self.pid = ti.pid