/
scheduler_job.py
1913 lines (1634 loc) · 80.2 KB
/
scheduler_job.py
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
#
# Licensed to the Apache Software Foundation (ASF) under one
# or more contributor license agreements. See the NOTICE file
# distributed with this work for additional information
# regarding copyright ownership. The ASF licenses this file
# to you under the Apache License, Version 2.0 (the
# "License"); you may not use this file except in compliance
# with the License. You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing,
# software distributed under the License is distributed on an
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
# KIND, either express or implied. See the License for the
# specific language governing permissions and limitations
# under the License.
#
import datetime
import itertools
import logging
import multiprocessing
import os
import signal
import sys
import threading
import time
from collections import defaultdict
from contextlib import redirect_stderr, redirect_stdout, suppress
from datetime import timedelta
from multiprocessing.connection import Connection as MultiprocessingConnection
from typing import DefaultDict, Dict, Iterable, List, Optional, Set, Tuple
from setproctitle import setproctitle
from sqlalchemy import and_, func, not_, or_, tuple_
from sqlalchemy.exc import OperationalError
from sqlalchemy.orm import load_only, selectinload
from sqlalchemy.orm.session import Session, make_transient
from airflow import models, settings
from airflow.configuration import conf
from airflow.exceptions import AirflowException, SerializedDagNotFound, TaskNotFound
from airflow.executors.executor_loader import UNPICKLEABLE_EXECUTORS
from airflow.jobs.base_job import BaseJob
from airflow.models import DAG, DagModel, SlaMiss, errors
from airflow.models.dagbag import DagBag
from airflow.models.dagrun import DagRun
from airflow.models.serialized_dag import SerializedDagModel
from airflow.models.taskinstance import SimpleTaskInstance, TaskInstanceKey
from airflow.stats import Stats
from airflow.ti_deps.dependencies_states import EXECUTION_STATES
from airflow.utils import timezone
from airflow.utils.callback_requests import (
CallbackRequest,
DagCallbackRequest,
SlaCallbackRequest,
TaskCallbackRequest,
)
from airflow.utils.dag_processing import AbstractDagFileProcessorProcess, DagFileProcessorAgent
from airflow.utils.email import get_email_address_list, send_email
from airflow.utils.event_scheduler import EventScheduler
from airflow.utils.log.logging_mixin import LoggingMixin, StreamLogWriter, set_context
from airflow.utils.mixins import MultiprocessingStartMethodMixin
from airflow.utils.retries import MAX_DB_RETRIES, retry_db_transaction, run_with_db_retries
from airflow.utils.session import create_session, provide_session
from airflow.utils.sqlalchemy import is_lock_not_available_error, prohibit_commit, skip_locked, with_row_locks
from airflow.utils.state import State
from airflow.utils.types import DagRunType
TI = models.TaskInstance
DR = models.DagRun
DM = models.DagModel
class DagFileProcessorProcess(AbstractDagFileProcessorProcess, LoggingMixin, MultiprocessingStartMethodMixin):
"""Runs DAG processing in a separate process using DagFileProcessor
:param file_path: a Python file containing Airflow DAG definitions
:type file_path: str
:param pickle_dags: whether to serialize the DAG objects to the DB
:type pickle_dags: bool
:param dag_ids: If specified, only look at these DAG ID's
:type dag_ids: List[str]
:param callback_requests: failure callback to execute
:type callback_requests: List[airflow.utils.callback_requests.CallbackRequest]
"""
# Counter that increments every time an instance of this class is created
class_creation_counter = 0
def __init__(
self,
file_path: str,
pickle_dags: bool,
dag_ids: Optional[List[str]],
callback_requests: List[CallbackRequest],
):
super().__init__()
self._file_path = file_path
self._pickle_dags = pickle_dags
self._dag_ids = dag_ids
self._callback_requests = callback_requests
# The process that was launched to process the given .
self._process: Optional[multiprocessing.process.BaseProcess] = None
# The result of DagFileProcessor.process_file(file_path).
self._result: Optional[Tuple[int, int]] = None
# Whether the process is done running.
self._done = False
# When the process started.
self._start_time: Optional[datetime.datetime] = None
# This ID is use to uniquely name the process / thread that's launched
# by this processor instance
self._instance_id = DagFileProcessorProcess.class_creation_counter
self._parent_channel: Optional[MultiprocessingConnection] = None
DagFileProcessorProcess.class_creation_counter += 1
@property
def file_path(self) -> str:
return self._file_path
@staticmethod
def _run_file_processor(
result_channel: MultiprocessingConnection,
parent_channel: MultiprocessingConnection,
file_path: str,
pickle_dags: bool,
dag_ids: Optional[List[str]],
thread_name: str,
callback_requests: List[CallbackRequest],
) -> None:
"""
Process the given file.
:param result_channel: the connection to use for passing back the result
:type result_channel: multiprocessing.Connection
:param parent_channel: the parent end of the channel to close in the child
:type parent_channel: multiprocessing.Connection
:param file_path: the file to process
:type file_path: str
:param pickle_dags: whether to pickle the DAGs found in the file and
save them to the DB
:type pickle_dags: bool
:param dag_ids: if specified, only examine DAG ID's that are
in this list
:type dag_ids: list[str]
:param thread_name: the name to use for the process that is launched
:type thread_name: str
:param callback_requests: failure callback to execute
:type callback_requests: List[airflow.utils.callback_requests.CallbackRequest]
:return: the process that was launched
:rtype: multiprocessing.Process
"""
# This helper runs in the newly created process
log: logging.Logger = logging.getLogger("airflow.processor")
# Since we share all open FDs from the parent, we need to close the parent side of the pipe here in
# the child, else it won't get closed properly until we exit.
log.info("Closing parent pipe")
parent_channel.close()
del parent_channel
set_context(log, file_path)
setproctitle(f"airflow scheduler - DagFileProcessor {file_path}")
try:
# redirect stdout/stderr to log
with redirect_stdout(StreamLogWriter(log, logging.INFO)), redirect_stderr(
StreamLogWriter(log, logging.WARN)
), Stats.timer() as timer:
# 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
log.info("Started process (PID=%s) to work on %s", os.getpid(), file_path)
dag_file_processor = DagFileProcessor(dag_ids=dag_ids, log=log)
result: Tuple[int, int] = dag_file_processor.process_file(
file_path=file_path,
pickle_dags=pickle_dags,
callback_requests=callback_requests,
)
result_channel.send(result)
log.info("Processing %s took %.3f seconds", file_path, timer.duration)
except Exception: # pylint: disable=broad-except
# Log exceptions through the logging framework.
log.exception("Got an exception! Propagating...")
raise
finally:
# We re-initialized the ORM within this Process above so we need to
# tear it down manually here
settings.dispose_orm()
result_channel.close()
def start(self) -> None:
"""Launch the process and start processing the DAG."""
start_method = self._get_multiprocessing_start_method()
context = multiprocessing.get_context(start_method)
_parent_channel, _child_channel = context.Pipe(duplex=False)
process = context.Process(
target=type(self)._run_file_processor,
args=(
_child_channel,
_parent_channel,
self.file_path,
self._pickle_dags,
self._dag_ids,
f"DagFileProcessor{self._instance_id}",
self._callback_requests,
),
name=f"DagFileProcessor{self._instance_id}-Process",
)
self._process = process
self._start_time = timezone.utcnow()
process.start()
# Close the child side of the pipe now the subprocess has started -- otherwise this would prevent it
# from closing in some cases
_child_channel.close()
del _child_channel
# Don't store it on self until after we've started the child process - we don't want to keep it from
# getting GCd/closed
self._parent_channel = _parent_channel
def kill(self) -> None:
"""Kill the process launched to process the file, and ensure consistent state."""
if self._process is None:
raise AirflowException("Tried to kill before starting!")
self._kill_process()
def terminate(self, sigkill: bool = False) -> None:
"""
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 or self._parent_channel is None:
raise AirflowException("Tried to call terminate before starting!")
self._process.terminate()
# Arbitrarily wait 5s for the process to die
with suppress(TimeoutError):
self._process._popen.wait(5) # type: ignore # pylint: disable=protected-access
if sigkill:
self._kill_process()
self._parent_channel.close()
def _kill_process(self) -> None:
if self._process is None:
raise AirflowException("Tried to kill process before starting!")
if self._process.is_alive() and self._process.pid:
self.log.warning("Killing DAGFileProcessorProcess (PID=%d)", self._process.pid)
os.kill(self._process.pid, signal.SIGKILL)
if self._parent_channel:
self._parent_channel.close()
@property
def pid(self) -> int:
"""
:return: the PID of the process launched to process the given file
:rtype: int
"""
if self._process is None or self._process.pid is None:
raise AirflowException("Tried to get PID before starting!")
return self._process.pid
@property
def exit_code(self) -> Optional[int]:
"""
After the process is finished, this can be called to get the return code
:return: the exit code of the process
:rtype: int
"""
if self._process is None:
raise AirflowException("Tried to get exit code before starting!")
if not self._done:
raise AirflowException("Tried to call retcode before process was finished!")
return self._process.exitcode
@property
def done(self) -> bool:
"""
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 or self._parent_channel is None:
raise AirflowException("Tried to see if it's done before starting!")
if self._done:
return True
if self._parent_channel.poll():
try:
self._result = self._parent_channel.recv()
self._done = True
self.log.debug("Waiting for %s", self._process)
self._process.join()
self._parent_channel.close()
return True
except EOFError:
# If we get an EOFError, it means the child end of the pipe has been closed. This only happens
# in the finally block. But due to a possible race condition, the process may have not yet
# terminated (it could be doing cleanup/python shutdown still). So we kill it here after a
# "suitable" timeout.
self._done = True
# Arbitrary timeout -- error/race condition only, so this doesn't need to be tunable.
self._process.join(timeout=5)
if self._process.is_alive():
# Didn't shut down cleanly - kill it
self._kill_process()
if not self._process.is_alive():
self._done = True
self.log.debug("Waiting for %s", self._process)
self._process.join()
self._parent_channel.close()
return True
return False
@property
def result(self) -> Optional[Tuple[int, int]]:
"""
:return: result of running DagFileProcessor.process_file()
:rtype: tuple[int, int] or None
"""
if not self.done:
raise AirflowException("Tried to get the result before it's done!")
return self._result
@property
def start_time(self) -> datetime.datetime:
"""
: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
@property
def waitable_handle(self):
return self._process.sentinel
class DagFileProcessor(LoggingMixin):
"""
Process a Python file containing Airflow DAGs.
This includes:
1. Execute the file and look for DAG objects in the namespace.
2. Execute any Callbacks if passed to DagFileProcessor.process_file
3. Serialize the DAGs and save it to DB (or update existing record in the DB).
4. Pickle the DAG and save it to the DB (if necessary).
5. Record any errors importing the file into ORM
Returns a tuple of 'number of dags found' and 'the count of import errors'
:param dag_ids: If specified, only look at these DAG ID's
:type dag_ids: List[str]
:param log: Logger to save the processing process
:type log: logging.Logger
"""
UNIT_TEST_MODE: bool = conf.getboolean('core', 'UNIT_TEST_MODE')
def __init__(self, dag_ids: Optional[List[str]], log: logging.Logger):
super().__init__()
self.dag_ids = dag_ids
self._log = log
@provide_session
def manage_slas(self, dag: DAG, session: Session = None) -> None:
"""
Finding all tasks that have SLAs defined, and sending alert emails
where needed. New SLA misses are also recorded in the database.
We are assuming that the scheduler runs often, so we only check for
tasks that should have succeeded in the past hour.
"""
self.log.info("Running SLA Checks for %s", dag.dag_id)
if not any(isinstance(ti.sla, timedelta) for ti in dag.tasks):
self.log.info("Skipping SLA check for %s because no tasks in DAG have SLAs", dag)
return
qry = (
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: List[TI] = (
session.query(TI)
.filter(
TI.dag_id == dag.dag_id,
TI.task_id == qry.c.task_id,
TI.execution_date == qry.c.max_ti,
)
.all()
)
ts = timezone.utcnow()
for ti in max_tis:
task = dag.get_task(ti.task_id)
if task.sla and not isinstance(task.sla, timedelta):
raise TypeError(
f"SLA is expected to be timedelta object, got "
f"{type(task.sla)} in {task.dag_id}:{task.task_id}"
)
dttm = dag.following_schedule(ti.execution_date)
while dttm < timezone.utcnow():
following_schedule = dag.following_schedule(dttm)
if following_schedule + task.sla < timezone.utcnow():
session.merge(
SlaMiss(task_id=ti.task_id, dag_id=ti.dag_id, execution_date=dttm, timestamp=ts)
)
dttm = dag.following_schedule(dttm)
session.commit()
# pylint: disable=singleton-comparison
slas: List[SlaMiss] = (
session.query(SlaMiss)
.filter(SlaMiss.notification_sent == False, SlaMiss.dag_id == dag.dag_id) # noqa
.all()
)
# pylint: enable=singleton-comparison
if slas: # pylint: disable=too-many-nested-blocks
sla_dates: List[datetime.datetime] = [sla.execution_date for sla in slas]
fetched_tis: List[TI] = (
session.query(TI)
.filter(TI.state != State.SUCCESS, TI.execution_date.in_(sla_dates), TI.dag_id == dag.dag_id)
.all()
)
blocking_tis: List[TI] = []
for ti in fetched_tis:
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('Calling SLA miss callback')
try:
dag.sla_miss_callback(dag, task_list, blocking_task_list, slas, blocking_tis)
notification_sent = True
except Exception: # pylint: disable=broad-except
self.log.exception("Could not call sla_miss_callback for DAG %s", dag.dag_id)
email_content = f"""\
Here's a list of tasks that missed their SLAs:
<pre><code>{task_list}\n<code></pre>
Blocking tasks:
<pre><code>{blocking_task_list}<code></pre>
Airflow Webserver URL: {conf.get(section='webserver', key='base_url')}
"""
tasks_missed_sla = []
for sla in slas:
try:
task = dag.get_task(sla.task_id)
except TaskNotFound:
# task already deleted from DAG, skip it
self.log.warning(
"Task %s doesn't exist in DAG anymore, skipping SLA miss notification.", sla.task_id
)
continue
tasks_missed_sla.append(task)
emails: Set[str] = set()
for task in tasks_missed_sla:
if task.email:
if isinstance(task.email, str):
emails |= set(get_email_address_list(task.email))
elif isinstance(task.email, (list, tuple)):
emails |= set(task.email)
if emails:
try:
send_email(emails, f"[airflow] SLA miss on DAG={dag.dag_id}", email_content)
email_sent = True
notification_sent = True
except Exception: # pylint: disable=broad-except
Stats.incr('sla_email_notification_failure')
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:
sla.email_sent = email_sent
sla.notification_sent = True
session.merge(sla)
session.commit()
@staticmethod
def update_import_errors(session: Session, dagbag: DagBag) -> None:
"""
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: airflow.DagBag
"""
# Clear the errors of the processed files
for dagbag_file in dagbag.file_last_changed:
session.query(errors.ImportError).filter(errors.ImportError.filename == dagbag_file).delete()
# Add the errors of the processed files
for filename, stacktrace in dagbag.import_errors.items():
session.add(
errors.ImportError(filename=filename, timestamp=timezone.utcnow(), stacktrace=stacktrace)
)
session.commit()
@provide_session
def execute_callbacks(
self, dagbag: DagBag, callback_requests: List[CallbackRequest], session: Session = None
) -> None:
"""
Execute on failure callbacks. These objects can come from SchedulerJob or from
DagFileProcessorManager.
:param dagbag: Dag Bag of dags
:param callback_requests: failure callbacks to execute
:type callback_requests: List[airflow.utils.callback_requests.CallbackRequest]
:param session: DB session.
"""
for request in callback_requests:
self.log.debug("Processing Callback Request: %s", request)
try:
if isinstance(request, TaskCallbackRequest):
self._execute_task_callbacks(dagbag, request)
elif isinstance(request, SlaCallbackRequest):
self.manage_slas(dagbag.dags.get(request.dag_id))
elif isinstance(request, DagCallbackRequest):
self._execute_dag_callbacks(dagbag, request, session)
except Exception: # pylint: disable=broad-except
self.log.exception(
"Error executing %s callback for file: %s",
request.__class__.__name__,
request.full_filepath,
)
session.commit()
@provide_session
def _execute_dag_callbacks(self, dagbag: DagBag, request: DagCallbackRequest, session: Session):
dag = dagbag.dags[request.dag_id]
dag_run = dag.get_dagrun(execution_date=request.execution_date, session=session)
dag.handle_callback(
dagrun=dag_run, success=not request.is_failure_callback, reason=request.msg, session=session
)
def _execute_task_callbacks(self, dagbag: DagBag, request: TaskCallbackRequest):
simple_ti = request.simple_task_instance
if simple_ti.dag_id in dagbag.dags:
dag = dagbag.dags[simple_ti.dag_id]
if simple_ti.task_id in dag.task_ids:
task = dag.get_task(simple_ti.task_id)
ti = TI(task, simple_ti.execution_date)
# Get properties needed for failure handling from SimpleTaskInstance.
ti.start_date = simple_ti.start_date
ti.end_date = simple_ti.end_date
ti.try_number = simple_ti.try_number
ti.state = simple_ti.state
ti.test_mode = self.UNIT_TEST_MODE
if request.is_failure_callback:
ti.handle_failure_with_callback(error=request.msg, test_mode=ti.test_mode)
self.log.info('Executed failure callback for %s in state %s', ti, ti.state)
@provide_session
def process_file(
self,
file_path: str,
callback_requests: List[CallbackRequest],
pickle_dags: bool = False,
session: Session = None,
) -> Tuple[int, int]:
"""
Process a Python file containing Airflow DAGs.
This includes:
1. Execute the file and look for DAG objects in the namespace.
2. Execute any Callbacks if passed to this method.
3. Serialize the DAGs and save it to DB (or update existing record in the DB).
4. Pickle the DAG and save it to the DB (if necessary).
5. Record any errors importing the file into ORM
:param file_path: the path to the Python file that should be executed
:type file_path: str
:param callback_requests: failure callback to execute
:type callback_requests: List[airflow.utils.dag_processing.CallbackRequest]
:param pickle_dags: whether serialize the DAGs found in the file and
save them to the db
:type pickle_dags: bool
:param session: Sqlalchemy ORM Session
:type session: Session
:return: number of dags found, count of import errors
:rtype: Tuple[int, int]
"""
self.log.info("Processing file %s for tasks to queue", file_path)
try:
dagbag = DagBag(file_path, include_examples=False, include_smart_sensor=False)
except Exception: # pylint: disable=broad-except
self.log.exception("Failed at reloading the DAG file %s", file_path)
Stats.incr('dag_file_refresh_error', 1, 1)
return 0, 0
if len(dagbag.dags) > 0:
self.log.info("DAG(s) %s retrieved from %s", dagbag.dags.keys(), file_path)
else:
self.log.warning("No viable dags retrieved from %s", file_path)
self.update_import_errors(session, dagbag)
return 0, len(dagbag.import_errors)
self.execute_callbacks(dagbag, callback_requests)
# Save individual DAGs in the ORM
dagbag.sync_to_db()
if pickle_dags:
paused_dag_ids = DagModel.get_paused_dag_ids(dag_ids=dagbag.dag_ids)
unpaused_dags: List[DAG] = [
dag for dag_id, dag in dagbag.dags.items() if dag_id not in paused_dag_ids
]
for dag in unpaused_dags:
dag.pickle(session)
# Record import errors into the ORM
try:
self.update_import_errors(session, dagbag)
except Exception: # pylint: disable=broad-except
self.log.exception("Error logging import errors!")
return len(dagbag.dags), len(dagbag.import_errors)
def _is_parent_process():
"""
Returns True if the current process is the parent process. False if the current process is a child
process started by multiprocessing.
"""
return multiprocessing.current_process().name == 'MainProcess'
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.
:param subdir: directory containing Python files with Airflow DAG
definitions, or a specific path to a file
:type subdir: str
:param num_runs: The number of times to run the scheduling loop. If you
have a large number of DAG files this could complete before each file
has been parsed. -1 for unlimited times.
:type num_runs: int
:param num_times_parse_dags: The number of times to try to parse each DAG file.
-1 for unlimited times.
:type num_times_parse_dags: int
:param processor_poll_interval: The number of seconds to wait between
polls of running processors
:type processor_poll_interval: 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
:param log: override the default Logger
:type log: logging.Logger
"""
__mapper_args__ = {'polymorphic_identity': 'SchedulerJob'}
heartrate: int = conf.getint('scheduler', 'SCHEDULER_HEARTBEAT_SEC')
def __init__(
self,
subdir: str = settings.DAGS_FOLDER,
num_runs: int = conf.getint('scheduler', 'num_runs'),
num_times_parse_dags: int = -1,
processor_poll_interval: float = conf.getfloat('scheduler', 'processor_poll_interval'),
do_pickle: bool = False,
log: logging.Logger = None,
*args,
**kwargs,
):
self.subdir = subdir
self.num_runs = num_runs
# In specific tests, we want to stop the parse loop after the _files_ have been parsed a certain
# number of times. This is only to support testing, and isn't something a user is likely to want to
# configure -- they'll want num_runs
self.num_times_parse_dags = num_times_parse_dags
self._processor_poll_interval = processor_poll_interval
self.do_pickle = do_pickle
super().__init__(*args, **kwargs)
if log:
self._log = log
# Check what SQL backend we use
sql_conn: str = conf.get('core', 'sql_alchemy_conn').lower()
self.using_sqlite = sql_conn.startswith('sqlite')
self.using_mysql = sql_conn.startswith('mysql')
self.max_tis_per_query: int = conf.getint('scheduler', 'max_tis_per_query')
self.processor_agent: Optional[DagFileProcessorAgent] = None
self.dagbag = DagBag(dag_folder=self.subdir, read_dags_from_db=True, load_op_links=False)
def register_signals(self) -> None:
"""Register signals that stop child processes"""
signal.signal(signal.SIGINT, self._exit_gracefully)
signal.signal(signal.SIGTERM, self._exit_gracefully)
signal.signal(signal.SIGUSR2, self._debug_dump)
def _exit_gracefully(self, signum, frame) -> None:
"""Helper method to clean up processor_agent to avoid leaving orphan processes."""
if not _is_parent_process():
# Only the parent process should perform the cleanup.
return
self.log.info("Exiting gracefully upon receiving signal %s", signum)
if self.processor_agent:
self.processor_agent.end()
sys.exit(os.EX_OK)
def _debug_dump(self, signum, frame):
if not _is_parent_process():
# Only the parent process should perform the debug dump.
return
try:
sig_name = signal.Signals(signum).name
except Exception:
sig_name = str(signum)
self.log.info("%s\n%s received, printing debug\n%s", "-" * 80, sig_name, "-" * 80)
self.executor.debug_dump()
self.log.info("-" * 80)
def is_alive(self, grace_multiplier: Optional[float] = None) -> bool:
"""
Is this SchedulerJob alive?
We define alive as in a state of running and a heartbeat within the
threshold defined in the ``scheduler_health_check_threshold`` config
setting.
``grace_multiplier`` is accepted for compatibility with the parent class.
:rtype: boolean
"""
if grace_multiplier is not None:
# Accept the same behaviour as superclass
return super().is_alive(grace_multiplier=grace_multiplier)
scheduler_health_check_threshold: int = conf.getint('scheduler', 'scheduler_health_check_threshold')
return (
self.state == State.RUNNING
and (timezone.utcnow() - self.latest_heartbeat).total_seconds() < scheduler_health_check_threshold
)
@provide_session
def _change_state_for_tis_without_dagrun(
self, old_states: List[str], new_state: str, session: Session = None
) -> None:
"""
For all DAG IDs in the DagBag, 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_states: list[airflow.utils.state.State]
:param new_state: set TaskInstances to this state
:type new_state: airflow.utils.state.State
"""
tis_changed = 0
query = (
session.query(models.TaskInstance)
.outerjoin(models.TaskInstance.dag_run)
.filter(models.TaskInstance.dag_id.in_(list(self.dagbag.dag_ids)))
.filter(models.TaskInstance.state.in_(old_states))
.filter(
or_(
models.DagRun.state != State.RUNNING,
models.DagRun.state.is_(None),
)
)
)
# We need to do this for mysql as well because it can cause deadlocks
# as discussed in https://issues.apache.org/jira/browse/AIRFLOW-2516
if self.using_sqlite or self.using_mysql:
tis_to_change: List[TI] = with_row_locks(
query, of=TI, session=session, **skip_locked(session=session)
).all()
for ti in tis_to_change:
ti.set_state(new_state, session=session)
tis_changed += 1
else:
subq = query.subquery()
current_time = timezone.utcnow()
ti_prop_update = {
models.TaskInstance.state: new_state,
models.TaskInstance.start_date: current_time,
}
# Only add end_date and duration if the new_state is 'success', 'failed' or 'skipped'
if new_state in State.finished:
ti_prop_update.update(
{
models.TaskInstance.end_date: current_time,
models.TaskInstance.duration: 0,
}
)
tis_changed = (
session.query(models.TaskInstance)
.filter(
models.TaskInstance.dag_id == subq.c.dag_id,
models.TaskInstance.task_id == subq.c.task_id,
models.TaskInstance.execution_date == subq.c.execution_date,
)
.update(ti_prop_update, synchronize_session=False)
)
if tis_changed > 0:
session.flush()
self.log.warning(
"Set %s task instances to state=%s as their associated DagRun was not in RUNNING state",
tis_changed,
new_state,
)
Stats.gauge('scheduler.tasks.without_dagrun', tis_changed)
@provide_session
def __get_concurrency_maps(
self, states: List[str], session: Session = None
) -> Tuple[DefaultDict[str, int], DefaultDict[Tuple[str, str], int]]:
"""
Get the concurrency maps.
:param states: List of states to query for
:type states: list[airflow.utils.state.State]
:return: A map from (dag_id, task_id) to # of task instances and
a map from (dag_id, task_id) to # of task instances in the given state list
:rtype: tuple[dict[str, int], dict[tuple[str, str], int]]
"""
ti_concurrency_query: List[Tuple[str, str, int]] = (
session.query(TI.task_id, TI.dag_id, func.count('*'))
.filter(TI.state.in_(states))
.group_by(TI.task_id, TI.dag_id)
).all()
dag_map: DefaultDict[str, int] = defaultdict(int)
task_map: DefaultDict[Tuple[str, str], int] = defaultdict(int)
for result in ti_concurrency_query:
task_id, dag_id, count = result
dag_map[dag_id] += count
task_map[(dag_id, task_id)] = count
return dag_map, task_map
@provide_session
def _executable_task_instances_to_queued(self, max_tis: int, session: Session = None) -> List[TI]:
"""
Finds TIs that are ready for execution with respect to pool limits,
dag concurrency, executor state, and priority.
:param max_tis: Maximum number of TIs to queue in this loop.
:type max_tis: int
:return: list[airflow.models.TaskInstance]
"""
executable_tis: List[TI] = []
# Get the pool settings. We get a lock on the pool rows, treating this as a "critical section"
# Throws an exception if lock cannot be obtained, rather than blocking
pools = models.Pool.slots_stats(lock_rows=True, session=session)
# If the pools are full, there is no point doing anything!
# If _somehow_ the pool is overfull, don't let the limit go negative - it breaks SQL
pool_slots_free = max(0, sum(pool['open'] for pool in pools.values()))
if pool_slots_free == 0:
self.log.debug("All pools are full!")
return executable_tis
max_tis = min(max_tis, pool_slots_free)
# Get all task instances associated with scheduled
# DagRuns which are not backfilled, in the given states,
# and the dag is not paused
query = (
session.query(TI)
.outerjoin(TI.dag_run)
.filter(or_(DR.run_id.is_(None), DR.run_type != DagRunType.BACKFILL_JOB))
.join(TI.dag_model)
.filter(not_(DM.is_paused))
.filter(TI.state == State.SCHEDULED)
.options(selectinload('dag_model'))
.order_by(-TI.priority_weight, TI.execution_date)
)
starved_pools = [pool_name for pool_name, stats in pools.items() if stats['open'] <= 0]
if starved_pools:
query = query.filter(not_(TI.pool.in_(starved_pools)))
query = query.limit(max_tis)
task_instances_to_examine: List[TI] = with_row_locks(
query,
of=TI,
session=session,
**skip_locked(session=session),
).all()
# TODO[HA]: This was wrong before anyway, as it only looked at a sub-set of dags, not everything.
# Stats.gauge('scheduler.tasks.pending', len(task_instances_to_examine))
if len(task_instances_to_examine) == 0:
self.log.debug("No tasks to consider for execution.")
return executable_tis
# Put one task instance on each line
task_instance_str = "\n\t".join(repr(x) for x in task_instances_to_examine)
self.log.info("%s tasks up for execution:\n\t%s", len(task_instances_to_examine), task_instance_str)
pool_to_task_instances: DefaultDict[str, List[models.Pool]] = defaultdict(list)
for task_instance in task_instances_to_examine:
pool_to_task_instances[task_instance.pool].append(task_instance)
# dag_id to # of running tasks and (dag_id, task_id) to # of running tasks.
dag_concurrency_map: DefaultDict[str, int]
task_concurrency_map: DefaultDict[Tuple[str, str], int]
dag_concurrency_map, task_concurrency_map = self.__get_concurrency_maps(
states=list(EXECUTION_STATES), session=session
)
num_tasks_in_executor = 0
# Number of tasks that cannot be scheduled because of no open slot in pool
num_starving_tasks_total = 0
# Go through each pool, and queue up a task for execution if there are
# any open slots in the pool.
for pool, task_instances in pool_to_task_instances.items():
pool_name = pool
if pool not in pools:
self.log.warning("Tasks using non-existent pool '%s' will not be scheduled", pool)
continue
open_slots = pools[pool]["open"]
num_ready = len(task_instances)
self.log.info(
"Figuring out tasks to run in Pool(name=%s) with %s open slots "
"and %s task instances ready to be queued",
pool,
open_slots,
num_ready,
)
priority_sorted_task_instances = sorted(
task_instances, key=lambda ti: (-ti.priority_weight, ti.execution_date)
)