forked from apache/airflow
-
Notifications
You must be signed in to change notification settings - Fork 0
/
dag_processing.py
1294 lines (1124 loc) · 47.2 KB
/
dag_processing.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 logging
import multiprocessing
import os
import re
import signal
import sys
import time
import zipfile
from abc import ABCMeta, abstractmethod
from collections import defaultdict
from collections import namedtuple
from importlib import import_module
import enum
from typing import Optional
import psutil
from setproctitle import setproctitle
import six
from six.moves import reload_module
from tabulate import tabulate
# To avoid circular imports
import airflow.models
from airflow import configuration as conf
from airflow.dag.base_dag import BaseDag, BaseDagBag
from airflow.exceptions import AirflowException
from airflow.settings import Stats
from airflow.models import errors
from airflow.utils import timezone
from airflow.utils.helpers import reap_process_group
from airflow.utils.db import provide_session
from airflow.utils.log.logging_mixin import LoggingMixin
if six.PY2:
ConnectionError = IOError
class SimpleDag(BaseDag):
"""
A simplified representation of a DAG that contains all attributes
required for instantiating and scheduling its associated tasks.
"""
def __init__(self, dag, pickle_id=None):
"""
:param dag: the DAG
:type dag: airflow.models.DAG
:param pickle_id: ID associated with the pickled version of this DAG.
:type pickle_id: unicode
"""
self._dag_id = dag.dag_id
self._task_ids = [task.task_id for task in dag.tasks]
self._full_filepath = dag.full_filepath
self._is_paused = dag.is_paused
self._concurrency = dag.concurrency
self._pickle_id = pickle_id
self._task_special_args = {}
for task in dag.tasks:
special_args = {}
if task.task_concurrency is not None:
special_args['task_concurrency'] = task.task_concurrency
if len(special_args) > 0:
self._task_special_args[task.task_id] = special_args
@property
def dag_id(self):
"""
:return: the DAG ID
:rtype: unicode
"""
return self._dag_id
@property
def task_ids(self):
"""
:return: A list of task IDs that are in this DAG
:rtype: list[unicode]
"""
return self._task_ids
@property
def full_filepath(self):
"""
:return: The absolute path to the file that contains this DAG's definition
:rtype: unicode
"""
return self._full_filepath
@property
def concurrency(self):
"""
:return: maximum number of tasks that can run simultaneously from this DAG
:rtype: int
"""
return self._concurrency
@property
def is_paused(self):
"""
:return: whether this DAG is paused or not
:rtype: bool
"""
return self._is_paused
@property
def pickle_id(self):
"""
:return: The pickle ID for this DAG, if it has one. Otherwise None.
:rtype: unicode
"""
return self._pickle_id
@property
def task_special_args(self):
return self._task_special_args
def get_task_special_arg(self, task_id, special_arg_name):
if task_id in self._task_special_args and special_arg_name in self._task_special_args[task_id]:
return self._task_special_args[task_id][special_arg_name]
else:
return None
class SimpleTaskInstance(object):
def __init__(self, ti):
self._dag_id = ti.dag_id
self._task_id = ti.task_id
self._execution_date = ti.execution_date
self._start_date = ti.start_date
self._end_date = ti.end_date
self._try_number = ti.try_number
self._state = ti.state
self._executor_config = ti.executor_config
if hasattr(ti, 'run_as_user'):
self._run_as_user = ti.run_as_user
else:
self._run_as_user = None
if hasattr(ti, 'pool'):
self._pool = ti.pool
else:
self._pool = None
if hasattr(ti, 'priority_weight'):
self._priority_weight = ti.priority_weight
else:
self._priority_weight = None
self._queue = ti.queue
self._key = ti.key
@property
def dag_id(self):
return self._dag_id
@property
def task_id(self):
return self._task_id
@property
def execution_date(self):
return self._execution_date
@property
def start_date(self):
return self._start_date
@property
def end_date(self):
return self._end_date
@property
def try_number(self):
return self._try_number
@property
def state(self):
return self._state
@property
def pool(self):
return self._pool
@property
def priority_weight(self):
return self._priority_weight
@property
def queue(self):
return self._queue
@property
def key(self):
return self._key
@property
def executor_config(self):
return self._executor_config
@provide_session
def construct_task_instance(self, session=None, lock_for_update=False):
"""
Construct a TaskInstance from the database based on the primary key
:param session: DB session.
: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 = airflow.models.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()
return ti
class SimpleDagBag(BaseDagBag):
"""
A collection of SimpleDag objects with some convenience methods.
"""
def __init__(self, simple_dags):
"""
Constructor.
:param simple_dags: SimpleDag objects that should be in this
:type list(airflow.utils.dag_processing.SimpleDagBag)
"""
self.simple_dags = simple_dags
self.dag_id_to_simple_dag = {}
for simple_dag in simple_dags:
self.dag_id_to_simple_dag[simple_dag.dag_id] = simple_dag
@property
def dag_ids(self):
"""
:return: IDs of all the DAGs in this
:rtype: list[unicode]
"""
return self.dag_id_to_simple_dag.keys()
def get_dag(self, dag_id):
"""
:param dag_id: DAG ID
:type dag_id: unicode
:return: if the given DAG ID exists in the bag, return the BaseDag
corresponding to that ID. Otherwise, throw an Exception
:rtype: airflow.utils.dag_processing.SimpleDag
"""
if dag_id not in self.dag_id_to_simple_dag:
raise AirflowException("Unknown DAG ID {}".format(dag_id))
return self.dag_id_to_simple_dag[dag_id]
def correct_maybe_zipped(fileloc):
"""
If the path contains a folder with a .zip suffix, then
the folder is treated as a zip archive and path to zip is returned.
"""
_, archive, filename = re.search(
r'((.*\.zip){})?(.*)'.format(re.escape(os.sep)), fileloc).groups()
if archive and zipfile.is_zipfile(archive):
return archive
else:
return fileloc
COMMENT_PATTERN = re.compile(r"\s*#.*")
def list_py_file_paths(directory, safe_mode=True,
include_examples=None):
"""
Traverse a directory and look for Python files.
:param directory: the directory to traverse
:type directory: unicode
:param safe_mode: whether to use a heuristic to determine whether a file
contains Airflow DAG definitions
:return: a list of paths to Python files in the specified directory
:rtype: list[unicode]
"""
if include_examples is None:
include_examples = conf.getboolean('core', 'LOAD_EXAMPLES')
file_paths = []
if directory is None:
return []
elif os.path.isfile(directory):
return [directory]
elif os.path.isdir(directory):
patterns_by_dir = {}
for root, dirs, files in os.walk(directory, followlinks=True):
patterns = patterns_by_dir.get(root, [])
ignore_file = os.path.join(root, '.airflowignore')
if os.path.isfile(ignore_file):
with open(ignore_file, 'r') as f:
# If we have new patterns create a copy so we don't change
# the previous list (which would affect other subdirs)
lines_no_comments = [COMMENT_PATTERN.sub("", line) for line in f.read().split("\n")]
patterns += [re.compile(line) for line in lines_no_comments if line]
# If we can ignore any subdirs entirely we should - fewer paths
# to walk is better. We have to modify the ``dirs`` array in
# place for this to affect os.walk
dirs[:] = [
d
for d in dirs
if not any(p.search(os.path.join(root, d)) for p in patterns)
]
# We want patterns defined in a parent folder's .airflowignore to
# apply to subdirs too
for d in dirs:
patterns_by_dir[os.path.join(root, d)] = patterns
for f in files:
try:
file_path = os.path.join(root, f)
if not os.path.isfile(file_path):
continue
mod_name, file_ext = os.path.splitext(
os.path.split(file_path)[-1])
if file_ext != '.py' and not zipfile.is_zipfile(file_path):
continue
if any([re.findall(p, file_path) for p in patterns]):
continue
# Heuristic that guesses whether a Python file contains an
# Airflow DAG definition.
might_contain_dag = True
if safe_mode and not zipfile.is_zipfile(file_path):
with open(file_path, 'rb') as fp:
content = fp.read()
might_contain_dag = all(
[s in content for s in (b'DAG', b'airflow')])
if not might_contain_dag:
continue
file_paths.append(file_path)
except Exception:
log = LoggingMixin().log
log.exception("Error while examining %s", f)
if include_examples:
import airflow.example_dags
example_dag_folder = airflow.example_dags.__path__[0]
file_paths.extend(list_py_file_paths(example_dag_folder, safe_mode, False))
return file_paths
class AbstractDagFileProcessor(object):
"""
Processes a DAG file. See SchedulerJob.process_file() for more details.
"""
__metaclass__ = ABCMeta
@abstractmethod
def start(self):
"""
Launch the process to process the file
"""
raise NotImplementedError()
@abstractmethod
def terminate(self, sigkill=False):
"""
Terminate (and then kill) the process launched to process the file
"""
raise NotImplementedError()
@property
@abstractmethod
def pid(self):
"""
:return: the PID of the process launched to process the given file
"""
raise NotImplementedError()
@property
@abstractmethod
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
"""
raise NotImplementedError()
@property
@abstractmethod
def done(self):
"""
Check if the process launched to process this file is done.
:return: whether the process is finished running
:rtype: bool
"""
raise NotImplementedError()
@property
@abstractmethod
def result(self):
"""
:return: result of running SchedulerJob.process_file()
:rtype: list[airflow.utils.dag_processing.SimpleDag]
"""
raise NotImplementedError()
@property
@abstractmethod
def start_time(self):
"""
:return: When this started to process the file
:rtype: datetime
"""
raise NotImplementedError()
@property
@abstractmethod
def file_path(self):
"""
:return: the path to the file that this is processing
:rtype: unicode
"""
raise NotImplementedError()
DagParsingStat = namedtuple('DagParsingStat', ['file_paths', 'done', 'all_files_processed'])
class DagParsingSignal(enum.Enum):
AGENT_HEARTBEAT = 'agent_heartbeat'
TERMINATE_MANAGER = 'terminate_manager'
END_MANAGER = 'end_manager'
class DagFileProcessorAgent(LoggingMixin):
"""
Agent for DAG file processing. It is responsible for all DAG parsing
related jobs in scheduler process. Mainly it can spin up DagFileProcessorManager
in a subprocess, collect DAG parsing results from it and communicate
signal/DAG parsing stat with it.
This class runs in the main `airflow scheduler` process.
"""
def __init__(self,
dag_directory,
file_paths,
max_runs,
processor_factory,
processor_timeout,
async_mode):
"""
:param dag_directory: Directory where DAG definitions are kept. All
files in file_paths should be under this directory
:type dag_directory: unicode
:param file_paths: list of file paths that contain DAG definitions
:type file_paths: list[unicode]
:param max_runs: The number of times to parse and schedule each file. -1
for unlimited.
:type max_runs: int
:param processor_factory: function that creates processors for DAG
definition files. Arguments are (dag_definition_path, log_file_path)
:type processor_factory: (unicode, unicode, list) -> (AbstractDagFileProcessor)
:param processor_timeout: How long to wait before timing out a DAG file processor
:type processor_timeout: timedelta
:param async_mode: Whether to start agent in async mode
:type async_mode: bool
"""
self._file_paths = file_paths
self._file_path_queue = []
self._dag_directory = dag_directory
self._max_runs = max_runs
self._processor_factory = processor_factory
self._processor_timeout = processor_timeout
self._async_mode = async_mode
# Map from file path to the processor
self._processors = {}
# Map from file path to the last runtime
self._last_runtime = {}
# Map from file path to the last finish time
self._last_finish_time = {}
# Map from file path to the number of runs
self._run_count = defaultdict(int)
# Pipe for communicating signals
self._process = None
self._done = False
# Initialized as true so we do not deactivate w/o any actual DAG parsing.
self._all_files_processed = True
self._parent_signal_conn = None
self._collected_dag_buffer = []
def start(self):
"""
Launch DagFileProcessorManager processor and start DAG parsing loop in manager.
"""
self._parent_signal_conn, child_signal_conn = multiprocessing.Pipe()
self._process = multiprocessing.Process(
target=type(self)._run_processor_manager,
args=(
self._dag_directory,
self._file_paths,
self._max_runs,
self._processor_factory,
self._processor_timeout,
child_signal_conn,
self._async_mode,
)
)
self._process.start()
self.log.info("Launched DagFileProcessorManager with pid: %s", self._process.pid)
def heartbeat(self):
"""
Should only be used when launched DAG file processor manager in sync mode.
Send agent heartbeat signal to the manager, requesting that it runs one
processing "loop".
Call wait_until_finished to ensure that any launched processors have
finished before continuing
"""
if not self._process.is_alive():
return
try:
self._parent_signal_conn.send(DagParsingSignal.AGENT_HEARTBEAT)
except ConnectionError:
# If this died cos of an error then we will noticed and restarted
# when harvest_simple_dags calls _heartbeat_manager.
pass
def wait_until_finished(self):
while self._parent_signal_conn.poll():
try:
result = self._parent_signal_conn.recv()
except EOFError:
break
self._process_message(result)
if isinstance(result, DagParsingStat):
# In sync mode we don't send this message from the Manager
# until all the running processors have finished
return
@staticmethod
def _run_processor_manager(dag_directory,
file_paths,
max_runs,
processor_factory,
processor_timeout,
signal_conn,
async_mode):
# Make this process start as a new process group - that makes it easy
# to kill all sub-process of this at the OS-level, rather than having
# to iterate the child processes
os.setpgid(0, 0)
setproctitle("airflow scheduler -- DagFileProcessorManager")
# Reload configurations and settings to avoid collision with parent process.
# Because this process may need custom configurations that cannot be shared,
# e.g. RotatingFileHandler. And it can cause connection corruption if we
# do not recreate the SQLA connection pool.
os.environ['CONFIG_PROCESSOR_MANAGER_LOGGER'] = 'True'
os.environ['AIRFLOW__CORE__COLORED_CONSOLE_LOG'] = 'False'
# Replicating the behavior of how logging module was loaded
# in logging_config.py
reload_module(import_module(airflow.settings.LOGGING_CLASS_PATH.rsplit('.', 1)[0]))
reload_module(airflow.settings)
airflow.settings.initialize()
del os.environ['CONFIG_PROCESSOR_MANAGER_LOGGER']
processor_manager = DagFileProcessorManager(dag_directory,
file_paths,
max_runs,
processor_factory,
processor_timeout,
signal_conn,
async_mode)
processor_manager.start()
def harvest_simple_dags(self):
"""
Harvest DAG parsing results from result queue and sync metadata from stat queue.
:return: List of parsing result in SimpleDag format.
"""
# Receive any pending messages before checking if the process has exited.
while self._parent_signal_conn.poll():
try:
result = self._parent_signal_conn.recv()
except (EOFError, ConnectionError):
break
self._process_message(result)
simple_dags = self._collected_dag_buffer
self._collected_dag_buffer = []
# If it died unexpectedly restart the manager process
self._heartbeat_manager()
return simple_dags
def _process_message(self, message):
self.log.debug("Received message of type %s", type(message).__name__)
if isinstance(message, DagParsingStat):
self._sync_metadata(message)
else:
self._collected_dag_buffer.append(message)
def _heartbeat_manager(self):
"""
Heartbeat DAG file processor and restart it if we are not done.
"""
if self._process and not self._process.is_alive():
self._process.join(timeout=0)
if not self.done:
self.log.warning(
"DagFileProcessorManager (PID=%d) exited with exit code %d - re-launching",
self._process.pid, self._process.exitcode
)
self.start()
def _sync_metadata(self, stat):
"""
Sync metadata from stat queue and only keep the latest stat.
"""
self._file_paths = stat.file_paths
self._done = stat.done
self._all_files_processed = stat.all_files_processed
@property
def file_paths(self):
return self._file_paths
@property
def done(self):
return self._done
@property
def all_files_processed(self):
return self._all_files_processed
def terminate(self):
"""
Send termination signal to DAG parsing processor manager
and expect it to terminate all DAG file processors.
"""
if self._process and self._process.is_alive():
self.log.info("Sending termination message to manager.")
self._parent_signal_conn.send(DagParsingSignal.TERMINATE_MANAGER)
def end(self):
"""
Terminate (and then kill) the manager process launched.
:return:
"""
if not self._process:
self.log.warn('Ending without manager process.')
return
reap_process_group(self._process.pid, log=self.log)
self._parent_signal_conn.close()
class DagFileProcessorManager(LoggingMixin):
"""
Given a list of DAG definition files, this kicks off several processors
in parallel to process them and put the results to a multiprocessing.Queue
for DagFileProcessorAgent to harvest. The parallelism is limited and as the
processors finish, more are launched. The files are processed over and
over again, but no more often than the specified interval.
:type _file_path_queue: list[unicode]
:type _processors: dict[unicode, AbstractDagFileProcessor]
:type _last_runtime: dict[unicode, float]
:type _last_finish_time: dict[unicode, datetime.datetime]
"""
def __init__(self,
dag_directory,
file_paths,
max_runs,
processor_factory,
processor_timeout,
signal_conn,
async_mode=True):
"""
:param dag_directory: Directory where DAG definitions are kept. All
files in file_paths should be under this directory
:type dag_directory: unicode
:param file_paths: list of file paths that contain DAG definitions
:type file_paths: list[unicode]
:param max_runs: The number of times to parse and schedule each file. -1
for unlimited.
:type max_runs: int
:param processor_factory: function that creates processors for DAG
definition files. Arguments are (dag_definition_path)
:type processor_factory: (unicode, unicode, list) -> (AbstractDagFileProcessor)
:param processor_timeout: How long to wait before timing out a DAG file processor
:type processor_timeout: timedelta
:param signal_conn: connection to communicate signal with processor agent.
:type signal_conn: airflow.models.connection.Connection
:param async_mode: whether to start the manager in async mode
:type async_mode: bool
"""
self._file_paths = file_paths
self._file_path_queue = []
self._dag_directory = dag_directory
self._max_runs = max_runs
self._processor_factory = processor_factory
self._signal_conn = signal_conn
self._async_mode = async_mode
self._parallelism = conf.getint('scheduler', 'max_threads')
if 'sqlite' in conf.get('core', 'sql_alchemy_conn') and self._parallelism > 1:
self.log.error("Cannot use more than 1 thread when using sqlite. "
"Setting parallelism to 1")
self._parallelism = 1
# Parse and schedule each file no faster than this interval.
self._file_process_interval = conf.getint('scheduler',
'min_file_process_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')
# Map from file path to the processor
self._processors = {}
# Map from file path to the last runtime
self._last_runtime = {}
# Map from file path to the last finish time
self._last_finish_time = {}
self._last_zombie_query_time = timezone.utcnow()
# Last time that the DAG dir was traversed to look for files
self.last_dag_dir_refresh_time = timezone.utcnow()
# Last time stats were printed
self.last_stat_print_time = timezone.datetime(2000, 1, 1)
# TODO: Remove magic number
self._zombie_query_interval = 10
# Map from file path to the number of runs
self._run_count = defaultdict(int)
# Manager heartbeat key.
self._heart_beat_key = 'heart-beat'
# How long to wait before timing out a process to parse a DAG file
self._processor_timeout = processor_timeout
# 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')
self._log = logging.getLogger('airflow.processor_manager')
signal.signal(signal.SIGINT, self._exit_gracefully)
signal.signal(signal.SIGTERM, self._exit_gracefully)
def _exit_gracefully(self, signum, frame):
"""
Helper method to clean up DAG file processors to avoid leaving orphan processes.
"""
self.log.info("Exiting gracefully upon receiving signal %s", signum)
self.terminate()
self.end()
self.log.debug("Finished terminating DAG processors.")
sys.exit(os.EX_OK)
def start(self):
"""
Use multiple processes to parse and generate tasks for the
DAGs in parallel. By processing them in separate processes,
we can get parallelism and isolation from potentially harmful
user code.
"""
self.log.info("Processing files using up to %s processes at a time ", self._parallelism)
self.log.info("Process each file at most once every %s seconds", self._file_process_interval)
self.log.info(
"Checking for new files in %s every %s seconds", self._dag_directory, self.dag_dir_list_interval
)
# In sync mode we want timeout=None -- wait forever until a message is received
poll_time = None # type: Optional[float]
if self._async_mode:
poll_time = 0.0
self.log.debug("Starting DagFileProcessorManager in async mode")
else:
poll_time = None
self.log.debug("Starting DagFileProcessorManager in sync mode")
while True:
loop_start_time = time.time()
if self._signal_conn.poll(poll_time):
agent_signal = self._signal_conn.recv()
self.log.debug("Recived %s singal from DagFileProcessorAgent", agent_signal)
if agent_signal == DagParsingSignal.TERMINATE_MANAGER:
self.terminate()
break
elif agent_signal == DagParsingSignal.END_MANAGER:
self.end()
sys.exit(os.EX_OK)
elif agent_signal == DagParsingSignal.AGENT_HEARTBEAT:
# continue the loop to parse dags
pass
elif not self._async_mode:
# In "sync" mode we don't want to parse the DAGs until we
# are told to (as that would open another connection to the
# SQLite DB which isn't a good practice
continue
self._refresh_dag_dir()
simple_dags = self.heartbeat()
for simple_dag in simple_dags:
self._signal_conn.send(simple_dag)
if not self._async_mode:
self.log.debug(
"Waiting for processors to finish since we're using sqlite")
# Wait until the running DAG processors are finished before
# sending a DagParsingStat message back. This means the Agent
# can tell we've got to the end of this iteration when it sees
# this type of message
self.wait_until_finished()
# Collect anything else that has finished, but don't kick off any more processors
simple_dags = self.collect_results()
for simple_dag in simple_dags:
self._signal_conn.send(simple_dag)
self._print_stat()
all_files_processed = all(self.get_last_finish_time(x) is not None
for x in self.file_paths)
max_runs_reached = self.max_runs_reached()
dag_parsing_stat = DagParsingStat(self._file_paths,
max_runs_reached,
all_files_processed,
)
self._signal_conn.send(dag_parsing_stat)
if max_runs_reached:
self.log.info("Exiting dag parsing loop as all files "
"have been processed %s times", self._max_runs)
break
if self._async_mode:
loop_duration = time.time() - loop_start_time
if loop_duration < 1:
poll_time = 1 - loop_duration
else:
poll_time = 0.0
def _refresh_dag_dir(self):
"""
Refresh file paths from dag dir if we haven't done it for too long.
"""
elapsed_time_since_refresh = (timezone.utcnow() -
self.last_dag_dir_refresh_time).total_seconds()
if elapsed_time_since_refresh > self.dag_dir_list_interval:
# Build up a list of Python files that could contain DAGs
self.log.info("Searching for files in %s", self._dag_directory)
self._file_paths = list_py_file_paths(self._dag_directory)
self.last_dag_dir_refresh_time = timezone.utcnow()
self.log.info("There are %s files in %s", len(self._file_paths), self._dag_directory)
self.set_file_paths(self._file_paths)
try:
self.log.debug("Removing old import errors")
self.clear_nonexistent_import_errors()
except Exception:
self.log.exception("Error removing old import errors")
def _print_stat(self):
"""
Occasionally print out stats about how fast the files are getting processed
"""
if ((timezone.utcnow() - self.last_stat_print_time).total_seconds() >
self.print_stats_interval):
if len(self._file_paths) > 0:
self._log_file_processing_stats(self._file_paths)
self.last_stat_print_time = timezone.utcnow()
@provide_session
def clear_nonexistent_import_errors(self, session):
"""
Clears import errors for files that no longer exist.
:param session: session for ORM operations
:type session: sqlalchemy.orm.session.Session
"""
query = session.query(errors.ImportError)
if self._file_paths:
query = query.filter(
~errors.ImportError.filename.in_(self._file_paths)
)
query.delete(synchronize_session='fetch')
session.commit()
def _log_file_processing_stats(self, known_file_paths):
"""
Print out stats about how files are getting processed.
:param known_file_paths: a list of file paths that may contain Airflow
DAG definitions
:type known_file_paths: list[unicode]
:return: None
"""
# File Path: Path to the file containing the DAG definition
# PID: PID associated with the process that's processing the file. May
# be empty.
# Runtime: If the process is currently running, how long it's been
# running for in seconds.
# Last Runtime: If the process ran before, how long did it take to
# finish in seconds
# Last Run: When the file finished processing in the previous run.
headers = ["File Path",
"PID",
"Runtime",
"Last Runtime",
"Last Run"]
rows = []
for file_path in known_file_paths:
last_runtime = self.get_last_runtime(file_path)
file_name = os.path.basename(file_path)
file_name = os.path.splitext(file_name)[0].replace(os.sep, '.')
if last_runtime:
Stats.gauge(
'dag_processing.last_runtime.{}'.format(file_name),
last_runtime
)
processor_pid = self.get_pid(file_path)
processor_start_time = self.get_start_time(file_path)
runtime = ((timezone.utcnow() - processor_start_time).total_seconds()
if processor_start_time else None)
last_run = self.get_last_finish_time(file_path)
if last_run:
seconds_ago = (timezone.utcnow() - last_run).total_seconds()
Stats.gauge(
'dag_processing.last_run.seconds_ago.{}'.format(file_name),
seconds_ago
)
rows.append((file_path,
processor_pid,
runtime,
last_runtime,
last_run))
# Sort by longest last runtime. (Can't sort None values in python3)
rows = sorted(rows, key=lambda x: x[3] or 0.0)
formatted_rows = []
for file_path, pid, runtime, last_runtime, last_run in rows:
formatted_rows.append((file_path,
pid,
"{:.2f}s".format(runtime)
if runtime else None,
"{:.2f}s".format(last_runtime)
if last_runtime else None,
last_run.strftime("%Y-%m-%dT%H:%M:%S")
if last_run else None))
log_str = ("\n" +
"=" * 80 +
"\n" +
"DAG File Processing Stats\n\n" +
tabulate(formatted_rows, headers=headers) +