-
Notifications
You must be signed in to change notification settings - Fork 14.1k
/
scheduler_job.py
1312 lines (1107 loc) · 56.5 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 time
import warnings
from collections import defaultdict
from datetime import timedelta
from typing import Collection, DefaultDict, Dict, Iterator, List, Optional, Set, Tuple
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.dag_processing.manager import DagFileProcessorAgent
from airflow.executors.executor_loader import UNPICKLEABLE_EXECUTORS
from airflow.jobs.base_job import BaseJob
from airflow.models import DAG
from airflow.models.dag import DagModel
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, TaskInstance, 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 DagCallbackRequest, TaskCallbackRequest
from airflow.utils.docs import get_docs_url
from airflow.utils.event_scheduler import EventScheduler
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 DagRunState, State, TaskInstanceState
from airflow.utils.types import DagRunType
TI = models.TaskInstance
DR = models.DagRun
DM = models.DagModel
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 scheduler_idle_sleep_time: The number of seconds to wait between
polls of running processors
:type scheduler_idle_sleep_time: 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,
scheduler_idle_sleep_time: float = conf.getfloat('scheduler', 'scheduler_idle_sleep_time'),
do_pickle: bool = False,
log: logging.Logger = None,
processor_poll_interval: Optional[float] = 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
if processor_poll_interval:
# TODO: Remove in Airflow 3.0
warnings.warn(
"The 'processor_poll_interval' parameter is deprecated. "
"Please use 'scheduler_idle_sleep_time'.",
DeprecationWarning,
stacklevel=2,
)
scheduler_idle_sleep_time = processor_poll_interval
self._scheduler_idle_sleep_time = scheduler_idle_sleep_time
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)
if conf.getboolean('smart_sensor', 'use_smart_sensor'):
compatible_sensors = set(
map(lambda l: l.strip(), conf.get('smart_sensor', 'sensors_enabled').split(','))
)
docs_url = get_docs_url('concepts/smart-sensors.html#migrating-to-deferrable-operators')
warnings.warn(
f'Smart sensors are deprecated, yet can be used for {compatible_sensors} sensors.'
f' Please use Deferrable Operators instead. See {docs_url} for more info.',
DeprecationWarning,
)
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 __get_concurrency_maps(
self, states: List[TaskInstanceState], 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 max_active_tasks, 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 []
max_tis = min(max_tis, pool_slots_free)
starved_pools = {pool_name for pool_name, stats in pools.items() if stats['open'] <= 0}
# dag_id to # of running tasks and (dag_id, task_id) to # of running tasks.
dag_active_tasks_map: DefaultDict[str, int]
task_concurrency_map: DefaultDict[Tuple[str, str], int]
dag_active_tasks_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
# dag and task ids that can't be queued because of concurrency limits
starved_dags: Set[str] = set()
starved_tasks: Set[Tuple[str, str]] = set()
pool_num_starving_tasks: DefaultDict[str, int] = defaultdict(int)
for loop_count in itertools.count(start=1):
num_starved_pools = len(starved_pools)
num_starved_dags = len(starved_dags)
num_starved_tasks = len(starved_tasks)
# Get task instances associated with scheduled
# DagRuns which are not backfilled, in the given states,
# and the dag is not paused
query = (
session.query(TI)
.join(TI.dag_run)
.filter(DR.run_type != DagRunType.BACKFILL_JOB, DR.state == DagRunState.RUNNING)
.join(TI.dag_model)
.filter(not_(DM.is_paused))
.filter(TI.state == TaskInstanceState.SCHEDULED)
.options(selectinload('dag_model'))
.order_by(-TI.priority_weight, DR.execution_date)
)
if starved_pools:
query = query.filter(not_(TI.pool.in_(starved_pools)))
if starved_dags:
query = query.filter(not_(TI.dag_id.in_(starved_dags)))
if starved_tasks:
if settings.engine.dialect.name == 'mssql':
task_filter = or_(
and_(
TaskInstance.dag_id == dag_id,
TaskInstance.task_id == task_id,
)
for (dag_id, task_id) in starved_tasks
)
else:
task_filter = tuple_(TaskInstance.dag_id, TaskInstance.task_id).in_(starved_tasks)
query = query.filter(not_(task_filter))
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.")
break
# 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[TI]] = defaultdict(list)
for task_instance in task_instances_to_examine:
pool_to_task_instances[task_instance.pool].append(task_instance)
# 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)
starved_pools.add(pool_name)
continue
pool_total = pools[pool]["total"]
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)
)
for current_index, task_instance in enumerate(priority_sorted_task_instances):
if open_slots <= 0:
self.log.info(
"Not scheduling since there are %s open slots in pool %s", open_slots, pool
)
# Can't schedule any more since there are no more open slots.
num_unhandled = len(priority_sorted_task_instances) - current_index
pool_num_starving_tasks[pool_name] += num_unhandled
num_starving_tasks_total += num_unhandled
starved_pools.add(pool_name)
break
if task_instance.pool_slots > pool_total:
self.log.warning(
"Not executing %s. Requested pool slots (%s) are greater than "
"total pool slots: '%s' for pool: %s.",
task_instance,
task_instance.pool_slots,
pool_total,
pool,
)
starved_tasks.add((task_instance.dag_id, task_instance.task_id))
continue
if task_instance.pool_slots > open_slots:
self.log.info(
"Not executing %s since it requires %s slots "
"but there are %s open slots in the pool %s.",
task_instance,
task_instance.pool_slots,
open_slots,
pool,
)
pool_num_starving_tasks[pool_name] += 1
num_starving_tasks_total += 1
starved_tasks.add((task_instance.dag_id, task_instance.task_id))
# Though we can execute tasks with lower priority if there's enough room
continue
# Check to make sure that the task max_active_tasks of the DAG hasn't been
# reached.
dag_id = task_instance.dag_id
current_active_tasks_per_dag = dag_active_tasks_map[dag_id]
max_active_tasks_per_dag_limit = task_instance.dag_model.max_active_tasks
self.log.info(
"DAG %s has %s/%s running and queued tasks",
dag_id,
current_active_tasks_per_dag,
max_active_tasks_per_dag_limit,
)
if current_active_tasks_per_dag >= max_active_tasks_per_dag_limit:
self.log.info(
"Not executing %s since the number of tasks running or queued "
"from DAG %s is >= to the DAG's max_active_tasks limit of %s",
task_instance,
dag_id,
max_active_tasks_per_dag_limit,
)
starved_dags.add(dag_id)
continue
if task_instance.dag_model.has_task_concurrency_limits:
# Many dags don't have a task_concurrency, so where we can avoid loading the full
# serialized DAG the better.
serialized_dag = self.dagbag.get_dag(dag_id, session=session)
# If the dag is missing, fail the task and continue to the next task.
if not serialized_dag:
self.log.error(
"DAG '%s' for task instance %s not found in serialized_dag table",
dag_id,
task_instance,
)
session.query(TI).filter(TI.dag_id == dag_id, TI.state == State.SCHEDULED).update(
{TI.state: State.FAILED}, synchronize_session='fetch'
)
continue
task_concurrency_limit: Optional[int] = None
if serialized_dag.has_task(task_instance.task_id):
task_concurrency_limit = serialized_dag.get_task(
task_instance.task_id
).max_active_tis_per_dag
if task_concurrency_limit is not None:
current_task_concurrency = task_concurrency_map[
(task_instance.dag_id, task_instance.task_id)
]
if current_task_concurrency >= task_concurrency_limit:
self.log.info(
"Not executing %s since the task concurrency for"
" this task has been reached.",
task_instance,
)
starved_tasks.add((task_instance.dag_id, task_instance.task_id))
continue
executable_tis.append(task_instance)
open_slots -= task_instance.pool_slots
dag_active_tasks_map[dag_id] += 1
task_concurrency_map[(task_instance.dag_id, task_instance.task_id)] += 1
pools[pool]["open"] = open_slots
is_done = executable_tis or len(task_instances_to_examine) < max_tis
# Check this to avoid accidental infinite loops
found_new_filters = (
len(starved_pools) > num_starved_pools
or len(starved_dags) > num_starved_dags
or len(starved_tasks) > num_starved_tasks
)
if is_done or not found_new_filters:
break
self.log.debug(
"Found no task instances to queue on the %s. iteration "
"but there could be more candidate task instances to check.",
loop_count,
)
for pool_name, num_starving_tasks in pool_num_starving_tasks.items():
Stats.gauge(f'pool.starving_tasks.{pool_name}', num_starving_tasks)
Stats.gauge('scheduler.tasks.starving', num_starving_tasks_total)
Stats.gauge('scheduler.tasks.running', num_tasks_in_executor)
Stats.gauge('scheduler.tasks.executable', len(executable_tis))
if len(executable_tis) > 0:
task_instance_str = "\n\t".join(repr(x) for x in executable_tis)
self.log.info("Setting the following tasks to queued state:\n\t%s", task_instance_str)
# set TIs to queued state
filter_for_tis = TI.filter_for_tis(executable_tis)
session.query(TI).filter(filter_for_tis).update(
# TODO[ha]: should we use func.now()? How does that work with DB timezone
# on mysql when it's not UTC?
{TI.state: State.QUEUED, TI.queued_dttm: timezone.utcnow(), TI.queued_by_job_id: self.id},
synchronize_session=False,
)
for ti in executable_tis:
make_transient(ti)
return executable_tis
@provide_session
def _enqueue_task_instances_with_queued_state(
self, task_instances: List[TI], session: Session = None
) -> None:
"""
Takes task_instances, which should have been set to queued, and enqueues them
with the executor.
:param task_instances: TaskInstances to enqueue
:type task_instances: list[TaskInstance]
:param session: The session object
:type session: Session
"""
# actually enqueue them
for ti in task_instances:
if ti.dag_run.state in State.finished:
ti.set_state(State.NONE, session=session)
continue
command = ti.command_as_list(
local=True,
pickle_id=ti.dag_model.pickle_id,
)
priority = ti.priority_weight
queue = ti.queue
self.log.info("Sending %s to executor with priority %s and queue %s", ti.key, priority, queue)
self.executor.queue_command(
ti,
command,
priority=priority,
queue=queue,
)
def _critical_section_execute_task_instances(self, session: Session) -> int:
"""
Attempts to execute TaskInstances that should be executed by the scheduler.
There are three steps:
1. Pick TIs by priority with the constraint that they are in the expected states
and that we do exceed max_active_runs or pool limits.
2. Change the state for the TIs above atomically.
3. Enqueue the TIs in the executor.
HA note: This function is a "critical section" meaning that only a single executor process can execute
this function at the same time. This is achieved by doing ``SELECT ... from pool FOR UPDATE``. For DBs
that support NOWAIT, a "blocked" scheduler will skip this and continue on with other tasks (creating
new DAG runs, progressing TIs from None to SCHEDULED etc.); DBs that don't support this (such as
MariaDB or MySQL 5.x) the other schedulers will wait for the lock before continuing.
:param session:
:type session: sqlalchemy.orm.Session
:return: Number of task instance with state changed.
"""
if self.max_tis_per_query == 0:
max_tis = self.executor.slots_available
else:
max_tis = min(self.max_tis_per_query, self.executor.slots_available)
queued_tis = self._executable_task_instances_to_queued(max_tis, session=session)
self._enqueue_task_instances_with_queued_state(queued_tis, session=session)
return len(queued_tis)
@provide_session
def _process_executor_events(self, session: Session = None) -> int:
"""Respond to executor events."""
if not self.processor_agent:
raise ValueError("Processor agent is not started.")
ti_primary_key_to_try_number_map: Dict[Tuple[str, str, datetime.datetime], int] = {}
event_buffer = self.executor.get_event_buffer()
tis_with_right_state: List[TaskInstanceKey] = []
# Report execution
for ti_key, value in event_buffer.items():
state: str
state, _ = value
# We create map (dag_id, task_id, execution_date) -> in-memory try_number
ti_primary_key_to_try_number_map[ti_key.primary] = ti_key.try_number
self.log.info(
"Executor reports execution of %s.%s run_id=%s exited with status %s for try_number %s",
ti_key.dag_id,
ti_key.task_id,
ti_key.run_id,
state,
ti_key.try_number,
)
if state in (State.FAILED, State.SUCCESS, State.QUEUED):
tis_with_right_state.append(ti_key)
# Return if no finished tasks
if not tis_with_right_state:
return len(event_buffer)
# Check state of finished tasks
filter_for_tis = TI.filter_for_tis(tis_with_right_state)
query = session.query(TI).filter(filter_for_tis).options(selectinload('dag_model'))
# row lock this entire set of taskinstances to make sure the scheduler doesn't fail when we have
# multi-schedulers
tis: Iterator[TI] = with_row_locks(
query,
of=TI,
session=session,
**skip_locked(session=session),
)
for ti in tis:
try_number = ti_primary_key_to_try_number_map[ti.key.primary]
buffer_key = ti.key.with_try_number(try_number)
state, info = event_buffer.pop(buffer_key)
# TODO: should we fail RUNNING as well, as we do in Backfills?
if state == State.QUEUED:
ti.external_executor_id = info
self.log.info("Setting external_id for %s to %s", ti, info)
continue
msg = (
"TaskInstance Finished: dag_id=%s, task_id=%s, run_id=%s, "
"run_start_date=%s, run_end_date=%s, "
"run_duration=%s, state=%s, executor_state=%s, try_number=%s, max_tries=%s, job_id=%s, "
"pool=%s, queue=%s, priority_weight=%d, operator=%s"
)
self.log.info(
msg,
ti.dag_id,
ti.task_id,
ti.run_id,
ti.start_date,
ti.end_date,
ti.duration,
ti.state,
state,
try_number,
ti.max_tries,
ti.job_id,
ti.pool,
ti.queue,
ti.priority_weight,
ti.operator,
)
if ti.try_number == buffer_key.try_number and ti.state == State.QUEUED:
Stats.incr('scheduler.tasks.killed_externally')
msg = (
"Executor reports task instance %s finished (%s) although the "
"task says its %s. (Info: %s) Was the task killed externally?"
)
self.log.error(msg, ti, state, ti.state, info)
# Get task from the Serialized DAG
try:
dag = self.dagbag.get_dag(ti.dag_id)
task = dag.get_task(ti.task_id)
except Exception:
self.log.exception("Marking task instance %s as %s", ti, state)
ti.set_state(state)
continue
ti.task = task
if task.on_retry_callback or task.on_failure_callback:
request = TaskCallbackRequest(
full_filepath=ti.dag_model.fileloc,
simple_task_instance=SimpleTaskInstance(ti),
msg=msg % (ti, state, ti.state, info),
)
self.processor_agent.send_callback_to_execute(request)
else:
ti.handle_failure(error=msg % (ti, state, ti.state, info), session=session)
return len(event_buffer)
def _execute(self) -> None:
self.log.info("Starting the scheduler")
# DAGs can be pickled for easier remote execution by some executors
pickle_dags = self.do_pickle and self.executor_class not in UNPICKLEABLE_EXECUTORS
self.log.info("Processing each file at most %s times", self.num_times_parse_dags)
# When using sqlite, we do not use async_mode
# so the scheduler job and DAG parser don't access the DB at the same time.
async_mode = not self.using_sqlite
processor_timeout_seconds: int = conf.getint('core', 'dag_file_processor_timeout')
processor_timeout = timedelta(seconds=processor_timeout_seconds)
self.processor_agent = DagFileProcessorAgent(
dag_directory=self.subdir,
max_runs=self.num_times_parse_dags,
processor_timeout=processor_timeout,
dag_ids=[],
pickle_dags=pickle_dags,
async_mode=async_mode,
)
try:
self.executor.job_id = self.id
self.executor.start()
self.register_signals()
self.processor_agent.start()
execute_start_time = timezone.utcnow()
self._run_scheduler_loop()
# Stop any processors
self.processor_agent.terminate()
# Verify that all files were processed, and if so, deactivate DAGs that
# haven't been touched by the scheduler as they likely have been
# deleted.
if self.processor_agent.all_files_processed:
self.log.info(
"Deactivating DAGs that haven't been touched since %s", execute_start_time.isoformat()
)
models.DAG.deactivate_stale_dags(execute_start_time)
settings.Session.remove() # type: ignore
except Exception:
self.log.exception("Exception when executing SchedulerJob._run_scheduler_loop")
raise
finally:
try:
self.executor.end()
except Exception:
self.log.exception("Exception when executing Executor.end")
try:
self.processor_agent.end()
except Exception:
self.log.exception("Exception when executing DagFileProcessorAgent.end")
self.log.info("Exited execute loop")
def _run_scheduler_loop(self) -> None:
"""
The actual scheduler loop. The main steps in the loop are:
#. Harvest DAG parsing results through DagFileProcessorAgent
#. Find and queue executable tasks
#. Change task instance state in DB
#. Queue tasks in executor
#. Heartbeat executor
#. Execute queued tasks in executor asynchronously
#. Sync on the states of running tasks
Following is a graphic representation of these steps.
.. image:: ../docs/apache-airflow/img/scheduler_loop.jpg
:rtype: None
"""
if not self.processor_agent:
raise ValueError("Processor agent is not started.")
is_unit_test: bool = conf.getboolean('core', 'unit_test_mode')
timers = EventScheduler()
# Check on start up, then every configured interval
self.adopt_or_reset_orphaned_tasks()
timers.call_regular_interval(
conf.getfloat('scheduler', 'orphaned_tasks_check_interval', fallback=300.0),
self.adopt_or_reset_orphaned_tasks,
)
timers.call_regular_interval(
conf.getfloat('scheduler', 'trigger_timeout_check_interval', fallback=15.0),
self.check_trigger_timeouts,
)
timers.call_regular_interval(
conf.getfloat('scheduler', 'pool_metrics_interval', fallback=5.0),
self._emit_pool_metrics,
)
for loop_count in itertools.count(start=1):
with Stats.timer() as timer:
if self.using_sqlite:
self.processor_agent.run_single_parsing_loop()
# For the sqlite case w/ 1 thread, wait until the processor
# is finished to avoid concurrent access to the DB.
self.log.debug("Waiting for processors to finish since we're using sqlite")
self.processor_agent.wait_until_finished()
with create_session() as session:
num_queued_tis = self._do_scheduling(session)
self.executor.heartbeat()
session.expunge_all()
num_finished_events = self._process_executor_events(session=session)
self.processor_agent.heartbeat()
# Heartbeat the scheduler periodically
self.heartbeat(only_if_necessary=True)
# Run any pending timed events
next_event = timers.run(blocking=False)
self.log.debug("Next timed event is in %f", next_event)
self.log.debug("Ran scheduling loop in %.2f seconds", timer.duration)
if not is_unit_test and not num_queued_tis and not num_finished_events:
# If the scheduler is doing things, don't sleep. This means when there is work to do, the
# scheduler will run "as quick as possible", but when it's stopped, it can sleep, dropping CPU
# usage when "idle"
time.sleep(min(self._scheduler_idle_sleep_time, next_event))
if loop_count >= self.num_runs > 0:
self.log.info(
"Exiting scheduler loop as requested number of runs (%d - got to %d) has been reached",
self.num_runs,
loop_count,
)
break
if self.processor_agent.done:
self.log.info(
"Exiting scheduler loop as requested DAG parse count (%d) has been reached after %d"
" scheduler loops",
self.num_times_parse_dags,
loop_count,
)
break
def _do_scheduling(self, session) -> int:
"""
This function is where the main scheduling decisions take places. It:
- Creates any necessary DAG runs by examining the next_dagrun_create_after column of DagModel
Since creating Dag Runs is a relatively time consuming process, we select only 10 dags by default
(configurable via ``scheduler.max_dagruns_to_create_per_loop`` setting) - putting this higher will
mean one scheduler could spend a chunk of time creating dag runs, and not ever get around to
scheduling tasks.
- Finds the "next n oldest" running DAG Runs to examine for scheduling (n=20 by default, configurable
via ``scheduler.max_dagruns_per_loop_to_schedule`` config setting) and tries to progress state (TIs
to SCHEDULED, or DagRuns to SUCCESS/FAILURE etc)
By "next oldest", we mean hasn't been examined/scheduled in the most time.
The reason we don't select all dagruns at once because the rows are selected with row locks, meaning
that only one scheduler can "process them", even it is waiting behind other dags. Increasing this
limit will allow more throughput for smaller DAGs but will likely slow down throughput for larger
(>500 tasks.) DAGs
- Then, via a Critical Section (locking the rows of the Pool model) we queue tasks, and then send them
to the executor.
See docs of _critical_section_execute_task_instances for more.
:return: Number of TIs enqueued in this iteration
:rtype: int
"""
# Put a check in place to make sure we don't commit unexpectedly
with prohibit_commit(session) as guard:
if settings.USE_JOB_SCHEDULE:
self._create_dagruns_for_dags(guard, session)
self._start_queued_dagruns(session)
guard.commit()
dag_runs = self._get_next_dagruns_to_examine(State.RUNNING, session)
# Bulk fetch the currently active dag runs for the dags we are
# examining, rather than making one query per DagRun
callback_tuples = []
for dag_run in dag_runs:
callback_to_run = self._schedule_dag_run(dag_run, session)
callback_tuples.append((dag_run, callback_to_run))
guard.commit()
# Send the callbacks after we commit to ensure the context is up to date when it gets run
for dag_run, callback_to_run in callback_tuples:
dag = self.dagbag.get_dag(dag_run.dag_id, session=session)
if not dag:
self.log.error("DAG '%s' not found in serialized_dag table", dag_run.dag_id)
continue
self._send_dag_callbacks_to_processor(dag, callback_to_run)
# Without this, the session has an invalid view of the DB
session.expunge_all()
# END: schedule TIs
try:
if self.executor.slots_available <= 0:
# We know we can't do anything here, so don't even try!
self.log.debug("Executor full, skipping critical section")
return 0
timer = Stats.timer('scheduler.critical_section_duration')
timer.start()
# Find anything TIs in state SCHEDULED, try to QUEUE it (send it to the executor)
num_queued_tis = self._critical_section_execute_task_instances(session=session)
# Make sure we only sent this metric if we obtained the lock, otherwise we'll skew the
# metric, way down
timer.stop(send=True)
except OperationalError as e:
timer.stop(send=False)
if is_lock_not_available_error(error=e):
self.log.debug("Critical section lock held by another Scheduler")
Stats.incr('scheduler.critical_section_busy')
session.rollback()
return 0
raise
guard.commit()
return num_queued_tis
@retry_db_transaction
def _get_next_dagruns_to_examine(self, state: DagRunState, session: Session):
"""Get Next DagRuns to Examine with retries"""
return DagRun.next_dagruns_to_examine(state, session)
@retry_db_transaction
def _create_dagruns_for_dags(self, guard, session):
"""Find Dag Models needing DagRuns and Create Dag Runs with retries in case of OperationalError"""
query = DagModel.dags_needing_dagruns(session)
self._create_dag_runs(query.all(), session)
# commit the session - Release the write lock on DagModel table.
guard.commit()
# END: create dagruns
def _create_dag_runs(self, dag_models: Collection[DagModel], session: Session) -> None:
"""
Unconditionally create a DAG run for the given DAG, and update the dag_model's fields to control
if/when the next DAGRun should be created
"""
# Bulk Fetch DagRuns with dag_id and execution_date same
# as DagModel.dag_id and DagModel.next_dagrun
# This list is used to verify if the DagRun already exist so that we don't attempt to create
# duplicate dag runs
if session.bind.dialect.name == 'mssql':
existing_dagruns_filter = or_(
*(
and_(
DagRun.dag_id == dm.dag_id,
DagRun.execution_date == dm.next_dagrun,
)
for dm in dag_models
)
)
else:
existing_dagruns_filter = tuple_(DagRun.dag_id, DagRun.execution_date).in_(
[(dm.dag_id, dm.next_dagrun) for dm in dag_models]
)
existing_dagruns = (
session.query(DagRun.dag_id, DagRun.execution_date).filter(existing_dagruns_filter).all()
)
active_runs_of_dags = defaultdict(
int,
DagRun.active_runs_of_dags(dag_ids=(dm.dag_id for dm in dag_models), session=session),
)
for dag_model in dag_models:
dag = self.dagbag.get_dag(dag_model.dag_id, session=session)
if not dag:
self.log.error("DAG '%s' not found in serialized_dag table", dag_model.dag_id)
continue
dag_hash = self.dagbag.dags_hash.get(dag.dag_id)
data_interval = dag.get_next_data_interval(dag_model)
# Explicitly check if the DagRun already exists. This is an edge case
# where a Dag Run is created but `DagModel.next_dagrun` and `DagModel.next_dagrun_create_after`
# are not updated.
# We opted to check DagRun existence instead
# of catching an Integrity error and rolling back the session i.e
# we need to set dag.next_dagrun_info if the Dag Run already exists or if we