forked from apache/airflow
-
Notifications
You must be signed in to change notification settings - Fork 0
/
jobs.py
2699 lines (2377 loc) · 110 KB
/
jobs.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 getpass
import logging
import multiprocessing
import os
import psutil
import signal
import six
import sys
import threading
import time
import datetime
from collections import defaultdict
from past.builtins import basestring
from sqlalchemy import (
Column, Integer, String, func, Index, or_, and_, not_)
from sqlalchemy.exc import OperationalError
from sqlalchemy.orm.session import make_transient
from tabulate import tabulate
from time import sleep
from airflow import configuration as conf
from airflow import executors, models, settings
from airflow.exceptions import AirflowException
from airflow.models import DAG, DagRun
from airflow.settings import Stats
from airflow.task.task_runner import get_task_runner
from airflow.ti_deps.dep_context import DepContext, QUEUE_DEPS, RUN_DEPS
from airflow.utils import asciiart, helpers, timezone
from airflow.utils.configuration import tmp_configuration_copy
from airflow.utils.dag_processing import (AbstractDagFileProcessor,
DagFileProcessorManager,
SimpleDag,
SimpleDagBag,
list_py_file_paths)
from airflow.utils.db import create_session, provide_session
from airflow.utils.email import send_email
from airflow.utils.log.logging_mixin import LoggingMixin, set_context, StreamLogWriter
from airflow.utils.net import get_hostname
from airflow.utils.state import State
from airflow.utils.sqlalchemy import UtcDateTime
Base = models.Base
ID_LEN = models.ID_LEN
class BaseJob(Base, LoggingMixin):
"""
Abstract class to be derived for jobs. Jobs are processing items with state
and duration that aren't task instances. For instance a BackfillJob is
a collection of task instance runs, but should have its own state, start
and end time.
"""
__tablename__ = "job"
id = Column(Integer, primary_key=True)
dag_id = Column(String(ID_LEN),)
state = Column(String(20))
job_type = Column(String(30))
start_date = Column(UtcDateTime())
end_date = Column(UtcDateTime())
latest_heartbeat = Column(UtcDateTime())
executor_class = Column(String(500))
hostname = Column(String(500))
unixname = Column(String(1000))
__mapper_args__ = {
'polymorphic_on': job_type,
'polymorphic_identity': 'BaseJob'
}
__table_args__ = (
Index('job_type_heart', job_type, latest_heartbeat),
Index('idx_job_state_heartbeat', state, latest_heartbeat),
)
def __init__(
self,
executor=executors.GetDefaultExecutor(),
heartrate=conf.getfloat('scheduler', 'JOB_HEARTBEAT_SEC'),
*args, **kwargs):
self.hostname = get_hostname()
self.executor = executor
self.executor_class = executor.__class__.__name__
self.start_date = timezone.utcnow()
self.latest_heartbeat = timezone.utcnow()
self.heartrate = heartrate
self.unixname = getpass.getuser()
self.max_tis_per_query = conf.getint('scheduler', 'max_tis_per_query')
super(BaseJob, self).__init__(*args, **kwargs)
def is_alive(self):
return (
(timezone.utcnow() - self.latest_heartbeat).seconds <
(conf.getint('scheduler', 'JOB_HEARTBEAT_SEC') * 2.1)
)
@provide_session
def kill(self, session=None):
job = session.query(BaseJob).filter(BaseJob.id == self.id).first()
job.end_date = timezone.utcnow()
try:
self.on_kill()
except Exception as e:
self.log.error('on_kill() method failed: {}'.format(e))
session.merge(job)
session.commit()
raise AirflowException("Job shut down externally.")
def on_kill(self):
"""
Will be called when an external kill command is received
"""
pass
def heartbeat_callback(self, session=None):
pass
def heartbeat(self):
"""
Heartbeats update the job's entry in the database with a timestamp
for the latest_heartbeat and allows for the job to be killed
externally. This allows at the system level to monitor what is
actually active.
For instance, an old heartbeat for SchedulerJob would mean something
is wrong.
This also allows for any job to be killed externally, regardless
of who is running it or on which machine it is running.
Note that if your heartbeat is set to 60 seconds and you call this
method after 10 seconds of processing since the last heartbeat, it
will sleep 50 seconds to complete the 60 seconds and keep a steady
heart rate. If you go over 60 seconds before calling it, it won't
sleep at all.
"""
with create_session() as session:
job = session.query(BaseJob).filter_by(id=self.id).one()
make_transient(job)
session.commit()
if job.state == State.SHUTDOWN:
self.kill()
# Figure out how long to sleep for
sleep_for = 0
if job.latest_heartbeat:
sleep_for = max(
0,
self.heartrate - (
timezone.utcnow() - job.latest_heartbeat).total_seconds())
sleep(sleep_for)
# Update last heartbeat time
with create_session() as session:
job = session.query(BaseJob).filter(BaseJob.id == self.id).first()
job.latest_heartbeat = timezone.utcnow()
session.merge(job)
session.commit()
self.heartbeat_callback(session=session)
self.log.debug('[heartbeat]')
def run(self):
Stats.incr(self.__class__.__name__.lower() + '_start', 1, 1)
# Adding an entry in the DB
with create_session() as session:
self.state = State.RUNNING
session.add(self)
session.commit()
id_ = self.id
make_transient(self)
self.id = id_
try:
self._execute()
# In case of max runs or max duration
self.state = State.SUCCESS
except SystemExit as e:
# In case of ^C or SIGTERM
self.state = State.SUCCESS
except Exception as e:
self.state = State.FAILED
raise
finally:
self.end_date = timezone.utcnow()
session.merge(self)
session.commit()
Stats.incr(self.__class__.__name__.lower() + '_end', 1, 1)
def _execute(self):
raise NotImplementedError("This method needs to be overridden")
@provide_session
def reset_state_for_orphaned_tasks(self, filter_by_dag_run=None, session=None):
"""
This function checks if there are any tasks in the dagrun (or all)
that have a scheduled state but are not known by the
executor. If it finds those it will reset the state to None
so they will get picked up again.
The batch option is for performance reasons as the queries are made in
sequence.
:param filter_by_dag_run: the dag_run we want to process, None if all
:type filter_by_dag_run: models.DagRun
:return: the TIs reset (in expired SQLAlchemy state)
:rtype: List(TaskInstance)
"""
queued_tis = self.executor.queued_tasks
# also consider running as the state might not have changed in the db yet
running_tis = self.executor.running
resettable_states = [State.SCHEDULED, State.QUEUED]
TI = models.TaskInstance
DR = models.DagRun
if filter_by_dag_run is None:
resettable_tis = (
session
.query(TI)
.join(
DR,
and_(
TI.dag_id == DR.dag_id,
TI.execution_date == DR.execution_date))
.filter(
DR.state == State.RUNNING,
DR.run_id.notlike(BackfillJob.ID_PREFIX + '%'),
TI.state.in_(resettable_states))).all()
else:
resettable_tis = filter_by_dag_run.get_task_instances(state=resettable_states,
session=session)
tis_to_reset = []
# Can't use an update here since it doesn't support joins
for ti in resettable_tis:
if ti.key not in queued_tis and ti.key not in running_tis:
tis_to_reset.append(ti)
if len(tis_to_reset) == 0:
return []
def query(result, items):
filter_for_tis = ([and_(TI.dag_id == ti.dag_id,
TI.task_id == ti.task_id,
TI.execution_date == ti.execution_date)
for ti in items])
reset_tis = (
session
.query(TI)
.filter(or_(*filter_for_tis), TI.state.in_(resettable_states))
.with_for_update()
.all())
for ti in reset_tis:
ti.state = State.NONE
session.merge(ti)
return result + reset_tis
reset_tis = helpers.reduce_in_chunks(query,
tis_to_reset,
[],
self.max_tis_per_query)
task_instance_str = '\n\t'.join(
["{}".format(x) for x in reset_tis])
session.commit()
self.log.info(
"Reset the following %s TaskInstances:\n\t%s",
len(reset_tis), task_instance_str
)
return reset_tis
class DagFileProcessor(AbstractDagFileProcessor, LoggingMixin):
"""Helps call SchedulerJob.process_file() in a separate process."""
# Counter that increments everytime an instance of this class is created
class_creation_counter = 0
def __init__(self, file_path, pickle_dags, dag_id_white_list):
"""
:param file_path: a Python file containing Airflow DAG definitions
:type file_path: unicode
:param pickle_dags: whether to serialize the DAG objects to the DB
:type pickle_dags: bool
:param dag_id_whitelist: If specified, only look at these DAG ID's
:type dag_id_whitelist: list[unicode]
"""
self._file_path = file_path
# Queue that's used to pass results from the child process.
self._result_queue = multiprocessing.Queue()
# The process that was launched to process the given .
self._process = None
self._dag_id_white_list = dag_id_white_list
self._pickle_dags = pickle_dags
# The result of Scheduler.process_file(file_path).
self._result = None
# Whether the process is done running.
self._done = False
# When the process started.
self._start_time = None
# This ID is use to uniquely name the process / thread that's launched
# by this processor instance
self._instance_id = DagFileProcessor.class_creation_counter
DagFileProcessor.class_creation_counter += 1
@property
def file_path(self):
return self._file_path
@staticmethod
def _launch_process(result_queue,
file_path,
pickle_dags,
dag_id_white_list,
thread_name):
"""
Launch a process to process the given file.
:param result_queue: the queue to use for passing back the result
:type result_queue: multiprocessing.Queue
:param file_path: the file to process
:type file_path: unicode
:param pickle_dags: whether to pickle the DAGs found in the file and
save them to the DB
:type pickle_dags: bool
:param dag_id_white_list: if specified, only examine DAG ID's that are
in this list
:type dag_id_white_list: list[unicode]
:param thread_name: the name to use for the process that is launched
:type thread_name: unicode
:return: the process that was launched
:rtype: multiprocessing.Process
"""
def helper():
# This helper runs in the newly created process
log = logging.getLogger("airflow.processor")
stdout = StreamLogWriter(log, logging.INFO)
stderr = StreamLogWriter(log, logging.WARN)
set_context(log, file_path)
try:
# redirect stdout/stderr to log
sys.stdout = stdout
sys.stderr = stderr
# Re-configure the ORM engine as there are issues with multiple processes
settings.configure_orm()
# Change the thread name to differentiate log lines. This is
# really a separate process, but changing the name of the
# process doesn't work, so changing the thread name instead.
threading.current_thread().name = thread_name
start_time = time.time()
log.info("Started process (PID=%s) to work on %s",
os.getpid(), file_path)
scheduler_job = SchedulerJob(dag_ids=dag_id_white_list, log=log)
result = scheduler_job.process_file(file_path,
pickle_dags)
result_queue.put(result)
end_time = time.time()
log.info(
"Processing %s took %.3f seconds", file_path, end_time - start_time
)
except Exception:
# Log exceptions through the logging framework.
log.exception("Got an exception! Propagating...")
raise
finally:
sys.stdout = sys.__stdout__
sys.stderr = sys.__stderr__
# We re-initialized the ORM within this Process above so we need to
# tear it down manually here
settings.dispose_orm()
p = multiprocessing.Process(target=helper,
args=(),
name="{}-Process".format(thread_name))
p.start()
return p
def start(self):
"""
Launch the process and start processing the DAG.
"""
self._process = DagFileProcessor._launch_process(
self._result_queue,
self.file_path,
self._pickle_dags,
self._dag_id_white_list,
"DagFileProcessor{}".format(self._instance_id))
self._start_time = timezone.utcnow()
def terminate(self, sigkill=False):
"""
Terminate (and then kill) the process launched to process the file.
:param sigkill: whether to issue a SIGKILL if SIGTERM doesn't work.
:type sigkill: bool
"""
if self._process is None:
raise AirflowException("Tried to call stop before starting!")
# The queue will likely get corrupted, so remove the reference
self._result_queue = None
self._process.terminate()
# Arbitrarily wait 5s for the process to die
self._process.join(5)
if sigkill and self._process.is_alive():
self.log.warning("Killing PID %s", self._process.pid)
os.kill(self._process.pid, signal.SIGKILL)
@property
def pid(self):
"""
:return: the PID of the process launched to process the given file
:rtype: int
"""
if self._process is None:
raise AirflowException("Tried to get PID before starting!")
return self._process.pid
@property
def exit_code(self):
"""
After the process is finished, this can be called to get the return code
:return: the exit code of the process
:rtype: int
"""
if not self._done:
raise AirflowException("Tried to call retcode before process was finished!")
return self._process.exitcode
@property
def done(self):
"""
Check if the process launched to process this file is done.
:return: whether the process is finished running
:rtype: bool
"""
if self._process is None:
raise AirflowException("Tried to see if it's done before starting!")
if self._done:
return True
if not self._result_queue.empty():
self._result = self._result_queue.get_nowait()
self._done = True
self.log.debug("Waiting for %s", self._process)
self._process.join()
return True
# Potential error case when process dies
if not self._process.is_alive():
self._done = True
# Get the object from the queue or else join() can hang.
if not self._result_queue.empty():
self._result = self._result_queue.get_nowait()
self.log.debug("Waiting for %s", self._process)
self._process.join()
return True
return False
@property
def result(self):
"""
:return: result of running SchedulerJob.process_file()
:rtype: SimpleDag
"""
if not self.done:
raise AirflowException("Tried to get the result before it's done!")
return self._result
@property
def start_time(self):
"""
:return: when this started to process the file
:rtype: datetime
"""
if self._start_time is None:
raise AirflowException("Tried to get start time before it started!")
return self._start_time
class SchedulerJob(BaseJob):
"""
This SchedulerJob runs for a specific time interval and schedules the jobs
that are ready to run. It figures out the latest runs for each
task and sees if the dependencies for the next schedules are met.
If so, it creates appropriate TaskInstances and sends run commands to the
executor. It does this for each task in each DAG and repeats.
"""
__mapper_args__ = {
'polymorphic_identity': 'SchedulerJob'
}
def __init__(
self,
dag_id=None,
dag_ids=None,
subdir=settings.DAGS_FOLDER,
num_runs=-1,
file_process_interval=conf.getint('scheduler',
'min_file_process_interval'),
processor_poll_interval=1.0,
run_duration=None,
do_pickle=False,
log=None,
*args, **kwargs):
"""
:param dag_id: if specified, only schedule tasks with this DAG ID
:type dag_id: unicode
:param dag_ids: if specified, only schedule tasks with these DAG IDs
:type dag_ids: list[unicode]
:param subdir: directory containing Python files with Airflow DAG
definitions, or a specific path to a file
:type subdir: unicode
:param num_runs: The number of times to try to schedule each DAG file.
-1 for unlimited within the run_duration.
:param processor_poll_interval: The number of seconds to wait between
polls of running processors
:param run_duration: how long to run (in seconds) before exiting
:type run_duration: int
:param do_pickle: once a DAG object is obtained by executing the Python
file, whether to serialize the DAG object to the DB
:type do_pickle: bool
"""
# for BaseJob compatibility
self.dag_id = dag_id
self.dag_ids = [dag_id] if dag_id else []
if dag_ids:
self.dag_ids.extend(dag_ids)
self.subdir = subdir
self.num_runs = num_runs
self.run_duration = run_duration
self._processor_poll_interval = processor_poll_interval
self.do_pickle = do_pickle
super(SchedulerJob, self).__init__(*args, **kwargs)
self.heartrate = conf.getint('scheduler', 'SCHEDULER_HEARTBEAT_SEC')
self.max_threads = conf.getint('scheduler', 'max_threads')
if log:
self._log = log
self.using_sqlite = False
if 'sqlite' in conf.get('core', 'sql_alchemy_conn'):
if self.max_threads > 1:
self.log.error("Cannot use more than 1 thread when using sqlite. Setting max_threads to 1")
self.max_threads = 1
self.using_sqlite = True
# How often to scan the DAGs directory for new files. Default to 5 minutes.
self.dag_dir_list_interval = conf.getint('scheduler',
'dag_dir_list_interval')
# How often to print out DAG file processing stats to the log. Default to
# 30 seconds.
self.print_stats_interval = conf.getint('scheduler',
'print_stats_interval')
self.file_process_interval = file_process_interval
self.max_tis_per_query = conf.getint('scheduler', 'max_tis_per_query')
if run_duration is None:
self.run_duration = conf.getint('scheduler',
'run_duration')
@provide_session
def manage_slas(self, dag, session=None):
"""
Finding all tasks that have SLAs defined, and sending alert emails
where needed. New SLA misses are also recorded in the database.
Where assuming that the scheduler runs often, so we only check for
tasks that should have succeeded in the past hour.
"""
if not any([ti.sla for ti in dag.tasks]):
self.log.info(
"Skipping SLA check for %s because no tasks in DAG have SLAs",
dag
)
return
TI = models.TaskInstance
sq = (
session
.query(
TI.task_id,
func.max(TI.execution_date).label('max_ti'))
.with_hint(TI, 'USE INDEX (PRIMARY)', dialect_name='mysql')
.filter(TI.dag_id == dag.dag_id)
.filter(or_(
TI.state == State.SUCCESS,
TI.state == State.SKIPPED))
.filter(TI.task_id.in_(dag.task_ids))
.group_by(TI.task_id).subquery('sq')
)
max_tis = session.query(TI).filter(
TI.dag_id == dag.dag_id,
TI.task_id == sq.c.task_id,
TI.execution_date == sq.c.max_ti,
).all()
ts = timezone.utcnow()
SlaMiss = models.SlaMiss
for ti in max_tis:
task = dag.get_task(ti.task_id)
dttm = ti.execution_date
if task.sla:
dttm = dag.following_schedule(dttm)
while dttm < timezone.utcnow():
following_schedule = dag.following_schedule(dttm)
if following_schedule + task.sla < timezone.utcnow():
session.merge(models.SlaMiss(
task_id=ti.task_id,
dag_id=ti.dag_id,
execution_date=dttm,
timestamp=ts))
dttm = dag.following_schedule(dttm)
session.commit()
slas = (
session
.query(SlaMiss)
.filter(SlaMiss.notification_sent == False) # noqa: E712
.filter(SlaMiss.dag_id == dag.dag_id)
.all()
)
if slas:
sla_dates = [sla.execution_date for sla in slas]
qry = (
session
.query(TI)
.filter(TI.state != State.SUCCESS)
.filter(TI.execution_date.in_(sla_dates))
.filter(TI.dag_id == dag.dag_id)
.all()
)
blocking_tis = []
for ti in qry:
if ti.task_id in dag.task_ids:
ti.task = dag.get_task(ti.task_id)
blocking_tis.append(ti)
else:
session.delete(ti)
session.commit()
task_list = "\n".join([
sla.task_id + ' on ' + sla.execution_date.isoformat()
for sla in slas])
blocking_task_list = "\n".join([
ti.task_id + ' on ' + ti.execution_date.isoformat()
for ti in blocking_tis])
# Track whether email or any alert notification sent
# We consider email or the alert callback as notifications
email_sent = False
notification_sent = False
if dag.sla_miss_callback:
# Execute the alert callback
self.log.info(' --------------> ABOUT TO CALL SLA MISS CALL BACK ')
try:
dag.sla_miss_callback(dag, task_list, blocking_task_list, slas,
blocking_tis)
notification_sent = True
except Exception:
self.log.exception("Could not call sla_miss_callback for DAG %s",
dag.dag_id)
email_content = """\
Here's a list of tasks that missed their SLAs:
<pre><code>{task_list}\n<code></pre>
Blocking tasks:
<pre><code>{blocking_task_list}\n{bug}<code></pre>
""".format(bug=asciiart.bug, **locals())
emails = set()
for task in dag.tasks:
if task.email:
if isinstance(task.email, basestring):
emails.add(task.email)
elif isinstance(task.email, (list, tuple)):
emails |= set(task.email)
if emails and len(slas):
try:
send_email(
emails,
"[airflow] SLA miss on DAG=" + dag.dag_id,
email_content)
email_sent = True
notification_sent = True
except Exception:
self.log.exception("Could not send SLA Miss email notification for"
" DAG %s", dag.dag_id)
# If we sent any notification, update the sla_miss table
if notification_sent:
for sla in slas:
if email_sent:
sla.email_sent = True
sla.notification_sent = True
session.merge(sla)
session.commit()
@staticmethod
@provide_session
def clear_nonexistent_import_errors(session, known_file_paths):
"""
Clears import errors for files that no longer exist.
:param session: session for ORM operations
:type session: sqlalchemy.orm.session.Session
:param known_file_paths: The list of existing files that are parsed for DAGs
:type known_file_paths: list[unicode]
"""
query = session.query(models.ImportError)
if known_file_paths:
query = query.filter(
~models.ImportError.filename.in_(known_file_paths)
)
query.delete(synchronize_session='fetch')
session.commit()
@staticmethod
def update_import_errors(session, dagbag):
"""
For the DAGs in the given DagBag, record any associated import errors and clears
errors for files that no longer have them. These are usually displayed through the
Airflow UI so that users know that there are issues parsing DAGs.
:param session: session for ORM operations
:type session: sqlalchemy.orm.session.Session
:param dagbag: DagBag containing DAGs with import errors
:type dagbag: models.Dagbag
"""
# Clear the errors of the processed files
for dagbag_file in dagbag.file_last_changed:
session.query(models.ImportError).filter(
models.ImportError.filename == dagbag_file
).delete()
# Add the errors of the processed files
for filename, stacktrace in six.iteritems(dagbag.import_errors):
session.add(models.ImportError(
filename=filename,
stacktrace=stacktrace))
session.commit()
@provide_session
def create_dag_run(self, dag, session=None):
"""
This method checks whether a new DagRun needs to be created
for a DAG based on scheduling interval
Returns DagRun if one is scheduled. Otherwise returns None.
"""
if dag.schedule_interval:
active_runs = DagRun.find(
dag_id=dag.dag_id,
state=State.RUNNING,
external_trigger=False,
session=session
)
# return if already reached maximum active runs and no timeout setting
if len(active_runs) >= dag.max_active_runs and not dag.dagrun_timeout:
return
timedout_runs = 0
for dr in active_runs:
if (
dr.start_date and dag.dagrun_timeout and
dr.start_date < timezone.utcnow() - dag.dagrun_timeout):
dr.state = State.FAILED
dr.end_date = timezone.utcnow()
dag.handle_callback(dr, success=False, reason='dagrun_timeout',
session=session)
timedout_runs += 1
session.commit()
if len(active_runs) - timedout_runs >= dag.max_active_runs:
return
# this query should be replaced by find dagrun
qry = (
session.query(func.max(DagRun.execution_date))
.filter_by(dag_id=dag.dag_id)
.filter(or_(
DagRun.external_trigger == False, # noqa: E712
# add % as a wildcard for the like query
DagRun.run_id.like(DagRun.ID_PREFIX + '%')
))
)
last_scheduled_run = qry.scalar()
# don't schedule @once again
if dag.schedule_interval == '@once' and last_scheduled_run:
return None
# don't do scheduler catchup for dag's that don't have dag.catchup = True
if not (dag.catchup or dag.schedule_interval == '@once'):
# The logic is that we move start_date up until
# one period before, so that timezone.utcnow() is AFTER
# the period end, and the job can be created...
now = timezone.utcnow()
next_start = dag.following_schedule(now)
last_start = dag.previous_schedule(now)
if next_start <= now:
new_start = last_start
else:
new_start = dag.previous_schedule(last_start)
if dag.start_date:
if new_start >= dag.start_date:
dag.start_date = new_start
else:
dag.start_date = new_start
next_run_date = None
if not last_scheduled_run:
# First run
task_start_dates = [t.start_date for t in dag.tasks]
if task_start_dates:
next_run_date = dag.normalize_schedule(min(task_start_dates))
self.log.debug(
"Next run date based on tasks %s",
next_run_date
)
else:
next_run_date = dag.following_schedule(last_scheduled_run)
# make sure backfills are also considered
last_run = dag.get_last_dagrun(session=session)
if last_run and next_run_date:
while next_run_date <= last_run.execution_date:
next_run_date = dag.following_schedule(next_run_date)
# don't ever schedule prior to the dag's start_date
if dag.start_date:
next_run_date = (dag.start_date if not next_run_date
else max(next_run_date, dag.start_date))
if next_run_date == dag.start_date:
next_run_date = dag.normalize_schedule(dag.start_date)
self.log.debug(
"Dag start date: %s. Next run date: %s",
dag.start_date, next_run_date
)
# don't ever schedule in the future
if next_run_date > timezone.utcnow():
return
# this structure is necessary to avoid a TypeError from concatenating
# NoneType
if dag.schedule_interval == '@once':
period_end = next_run_date
elif next_run_date:
period_end = dag.following_schedule(next_run_date)
# Don't schedule a dag beyond its end_date (as specified by the dag param)
if next_run_date and dag.end_date and next_run_date > dag.end_date:
return
# Don't schedule a dag beyond its end_date (as specified by the task params)
# Get the min task end date, which may come from the dag.default_args
min_task_end_date = []
task_end_dates = [t.end_date for t in dag.tasks if t.end_date]
if task_end_dates:
min_task_end_date = min(task_end_dates)
if next_run_date and min_task_end_date and next_run_date > min_task_end_date:
return
if next_run_date and period_end and period_end <= timezone.utcnow():
next_run = dag.create_dagrun(
run_id=DagRun.ID_PREFIX + next_run_date.isoformat(),
execution_date=next_run_date,
start_date=timezone.utcnow(),
state=State.RUNNING,
external_trigger=False
)
return next_run
@provide_session
def _process_task_instances(self, dag, queue, session=None):
"""
This method schedules the tasks for a single DAG by looking at the
active DAG runs and adding task instances that should run to the
queue.
"""
# update the state of the previously active dag runs
dag_runs = DagRun.find(dag_id=dag.dag_id, state=State.RUNNING, session=session)
active_dag_runs = []
for run in dag_runs:
self.log.info("Examining DAG run %s", run)
# don't consider runs that are executed in the future
if run.execution_date > timezone.utcnow():
self.log.error(
"Execution date is in future: %s",
run.execution_date
)
continue
if len(active_dag_runs) >= dag.max_active_runs:
self.log.info("Active dag runs > max_active_run.")
continue
# skip backfill dagruns for now as long as they are not really scheduled
if run.is_backfill:
continue
# todo: run.dag is transient but needs to be set
run.dag = dag
# todo: preferably the integrity check happens at dag collection time
run.verify_integrity(session=session)
run.update_state(session=session)
if run.state == State.RUNNING:
make_transient(run)
active_dag_runs.append(run)
for run in active_dag_runs:
self.log.debug("Examining active DAG run: %s", run)
# this needs a fresh session sometimes tis get detached
tis = run.get_task_instances(state=(State.NONE,
State.UP_FOR_RETRY))
# this loop is quite slow as it uses are_dependencies_met for
# every task (in ti.is_runnable). This is also called in
# update_state above which has already checked these tasks
for ti in tis:
task = dag.get_task(ti.task_id)
# fixme: ti.task is transient but needs to be set
ti.task = task
# future: remove adhoc
if task.adhoc:
continue
if ti.are_dependencies_met(
dep_context=DepContext(flag_upstream_failed=True),
session=session):
self.log.debug('Queuing task: %s', ti)
queue.append(ti.key)
@provide_session
def _change_state_for_tis_without_dagrun(self,
simple_dag_bag,
old_states,
new_state,
session=None):
"""
For all DAG IDs in the SimpleDagBag, look for task instances in the
old_states and set them to new_state if the corresponding DagRun
does not exist or exists but is not in the running state. This
normally should not happen, but it can if the state of DagRuns are
changed manually.
:param old_states: examine TaskInstances in this state
:type old_state: list[State]
:param new_state: set TaskInstances to this state
:type new_state: State
:param simple_dag_bag: TaskInstances associated with DAGs in the
simple_dag_bag and with states in the old_state will be examined
:type simple_dag_bag: SimpleDagBag
"""
tis_changed = 0
query = session \
.query(models.TaskInstance) \
.outerjoin(models.DagRun, and_(
models.TaskInstance.dag_id == models.DagRun.dag_id,