forked from apache/airflow
-
Notifications
You must be signed in to change notification settings - Fork 0
/
Copy pathcli.py
2140 lines (1904 loc) · 76.9 KB
/
cli.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
#!/usr/bin/env python
# -*- 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 print_function
from backports.configparser import NoSectionError
import logging
import os
import subprocess
import textwrap
import random
import string
from importlib import import_module
import getpass
import reprlib
import argparse
from builtins import input
from collections import namedtuple
from airflow.utils.timezone import parse as parsedate
import json
from tabulate import tabulate
import daemon
from daemon.pidfile import TimeoutPIDLockFile
import signal
import sys
import threading
import traceback
import time
import psutil
import re
from urllib.parse import urlunparse
import airflow
from airflow import api
from airflow import jobs, settings
from airflow import configuration as conf
from airflow.exceptions import AirflowException, AirflowWebServerTimeout
from airflow.executors import GetDefaultExecutor
from airflow.models import (DagModel, DagBag, TaskInstance,
DagPickle, DagRun, Variable, DagStat,
Connection, DAG)
from airflow.ti_deps.dep_context import (DepContext, SCHEDULER_DEPS)
from airflow.utils import cli as cli_utils
from airflow.utils import db as db_utils
from airflow.utils.net import get_hostname
from airflow.utils.log.logging_mixin import (LoggingMixin, redirect_stderr,
redirect_stdout)
from airflow.www.app import (cached_app, create_app)
from airflow.www_rbac.app import cached_app as cached_app_rbac
from airflow.www_rbac.app import create_app as create_app_rbac
from airflow.www_rbac.app import cached_appbuilder
from sqlalchemy import func
from sqlalchemy.orm import exc
api.load_auth()
api_module = import_module(conf.get('cli', 'api_client'))
api_client = api_module.Client(api_base_url=conf.get('cli', 'endpoint_url'),
auth=api.api_auth.client_auth)
log = LoggingMixin().log
DAGS_FOLDER = settings.DAGS_FOLDER
if "BUILDING_AIRFLOW_DOCS" in os.environ:
DAGS_FOLDER = '[AIRFLOW_HOME]/dags'
def sigint_handler(sig, frame):
sys.exit(0)
def sigquit_handler(sig, frame):
"""Helps debug deadlocks by printing stacktraces when this gets a SIGQUIT
e.g. kill -s QUIT <PID> or CTRL+\
"""
print("Dumping stack traces for all threads in PID {}".format(os.getpid()))
id_to_name = dict([(th.ident, th.name) for th in threading.enumerate()])
code = []
for thread_id, stack in sys._current_frames().items():
code.append("\n# Thread: {}({})"
.format(id_to_name.get(thread_id, ""), thread_id))
for filename, line_number, name, line in traceback.extract_stack(stack):
code.append('File: "{}", line {}, in {}'
.format(filename, line_number, name))
if line:
code.append(" {}".format(line.strip()))
print("\n".join(code))
def setup_logging(filename):
root = logging.getLogger()
handler = logging.FileHandler(filename)
formatter = logging.Formatter(settings.SIMPLE_LOG_FORMAT)
handler.setFormatter(formatter)
root.addHandler(handler)
root.setLevel(settings.LOGGING_LEVEL)
return handler.stream
def setup_locations(process, pid=None, stdout=None, stderr=None, log=None):
if not stderr:
stderr = os.path.join(os.path.expanduser(settings.AIRFLOW_HOME),
'airflow-{}.err'.format(process))
if not stdout:
stdout = os.path.join(os.path.expanduser(settings.AIRFLOW_HOME),
'airflow-{}.out'.format(process))
if not log:
log = os.path.join(os.path.expanduser(settings.AIRFLOW_HOME),
'airflow-{}.log'.format(process))
if not pid:
pid = os.path.join(os.path.expanduser(settings.AIRFLOW_HOME),
'airflow-{}.pid'.format(process))
return pid, stdout, stderr, log
def process_subdir(subdir):
if subdir:
subdir = subdir.replace('DAGS_FOLDER', DAGS_FOLDER)
subdir = os.path.abspath(os.path.expanduser(subdir))
return subdir
def get_dag(args):
dagbag = DagBag(process_subdir(args.subdir))
if args.dag_id not in dagbag.dags:
raise AirflowException(
'dag_id could not be found: {}. Either the dag did not exist or it failed to '
'parse.'.format(args.dag_id))
return dagbag.dags[args.dag_id]
def get_dags(args):
if not args.dag_regex:
return [get_dag(args)]
dagbag = DagBag(process_subdir(args.subdir))
matched_dags = [dag for dag in dagbag.dags.values() if re.search(
args.dag_id, dag.dag_id)]
if not matched_dags:
raise AirflowException(
'dag_id could not be found with regex: {}. Either the dag did not exist '
'or it failed to parse.'.format(args.dag_id))
return matched_dags
@cli_utils.action_logging
def backfill(args, dag=None):
logging.basicConfig(
level=settings.LOGGING_LEVEL,
format=settings.SIMPLE_LOG_FORMAT)
dag = dag or get_dag(args)
if not args.start_date and not args.end_date:
raise AirflowException("Provide a start_date and/or end_date")
# If only one date is passed, using same as start and end
args.end_date = args.end_date or args.start_date
args.start_date = args.start_date or args.end_date
if args.task_regex:
dag = dag.sub_dag(
task_regex=args.task_regex,
include_upstream=not args.ignore_dependencies)
run_conf = None
if args.conf:
run_conf = json.loads(args.conf)
if args.dry_run:
print("Dry run of DAG {0} on {1}".format(args.dag_id,
args.start_date))
for task in dag.tasks:
print("Task {0}".format(task.task_id))
ti = TaskInstance(task, args.start_date)
ti.dry_run()
else:
if args.reset_dagruns:
DAG.clear_dags(
[dag],
start_date=args.start_date,
end_date=args.end_date,
confirm_prompt=True,
include_subdags=True,
)
dag.run(
start_date=args.start_date,
end_date=args.end_date,
mark_success=args.mark_success,
local=args.local,
donot_pickle=(args.donot_pickle or
conf.getboolean('core', 'donot_pickle')),
ignore_first_depends_on_past=args.ignore_first_depends_on_past,
ignore_task_deps=args.ignore_dependencies,
pool=args.pool,
delay_on_limit_secs=args.delay_on_limit,
verbose=args.verbose,
conf=run_conf,
rerun_failed_tasks=args.rerun_failed_tasks,
)
@cli_utils.action_logging
def trigger_dag(args):
"""
Creates a dag run for the specified dag
:param args:
:return:
"""
log = LoggingMixin().log
try:
message = api_client.trigger_dag(dag_id=args.dag_id,
run_id=args.run_id,
conf=args.conf,
execution_date=args.exec_date)
except IOError as err:
log.error(err)
raise AirflowException(err)
log.info(message)
@cli_utils.action_logging
def delete_dag(args):
"""
Deletes all DB records related to the specified dag
:param args:
:return:
"""
log = LoggingMixin().log
if args.yes or input(
"This will drop all existing records related to the specified DAG. "
"Proceed? (y/n)").upper() == "Y":
try:
message = api_client.delete_dag(dag_id=args.dag_id)
except IOError as err:
log.error(err)
raise AirflowException(err)
log.info(message)
else:
print("Bail.")
@cli_utils.action_logging
def pool(args):
log = LoggingMixin().log
def _tabulate(pools):
return "\n%s" % tabulate(pools, ['Pool', 'Slots', 'Description'],
tablefmt="fancy_grid")
try:
imp = getattr(args, 'import')
if args.get is not None:
pools = [api_client.get_pool(name=args.get)]
elif args.set:
pools = [api_client.create_pool(name=args.set[0],
slots=args.set[1],
description=args.set[2])]
elif args.delete:
pools = [api_client.delete_pool(name=args.delete)]
elif imp:
if os.path.exists(imp):
pools = pool_import_helper(imp)
else:
print("Missing pools file.")
pools = api_client.get_pools()
elif args.export:
pools = pool_export_helper(args.export)
else:
pools = api_client.get_pools()
except (AirflowException, IOError) as err:
log.error(err)
else:
log.info(_tabulate(pools=pools))
def pool_import_helper(filepath):
with open(filepath, 'r') as poolfile:
pl = poolfile.read()
try:
d = json.loads(pl)
except Exception as e:
print("Please check the validity of the json file: " + str(e))
else:
try:
pools = []
n = 0
for k, v in d.items():
if isinstance(v, dict) and len(v) == 2:
pools.append(api_client.create_pool(name=k,
slots=v["slots"],
description=v["description"]))
n += 1
else:
pass
except Exception:
pass
finally:
print("{} of {} pool(s) successfully updated.".format(n, len(d)))
return pools
def pool_export_helper(filepath):
pool_dict = {}
pools = api_client.get_pools()
for pool in pools:
pool_dict[pool[0]] = {"slots": pool[1], "description": pool[2]}
with open(filepath, 'w') as poolfile:
poolfile.write(json.dumps(pool_dict, sort_keys=True, indent=4))
print("{} pools successfully exported to {}".format(len(pool_dict), filepath))
return pools
@cli_utils.action_logging
def variables(args):
if args.get:
try:
var = Variable.get(args.get,
deserialize_json=args.json,
default_var=args.default)
print(var)
except ValueError as e:
print(e)
if args.delete:
session = settings.Session()
session.query(Variable).filter_by(key=args.delete).delete()
session.commit()
session.close()
if args.set:
Variable.set(args.set[0], args.set[1])
# Work around 'import' as a reserved keyword
imp = getattr(args, 'import')
if imp:
if os.path.exists(imp):
import_helper(imp)
else:
print("Missing variables file.")
if args.export:
export_helper(args.export)
if not (args.set or args.get or imp or args.export or args.delete):
# list all variables
session = settings.Session()
vars = session.query(Variable)
msg = "\n".join(var.key for var in vars)
print(msg)
def import_helper(filepath):
with open(filepath, 'r') as varfile:
var = varfile.read()
try:
d = json.loads(var)
except Exception:
print("Invalid variables file.")
else:
try:
n = 0
for k, v in d.items():
if isinstance(v, dict):
Variable.set(k, v, serialize_json=True)
else:
Variable.set(k, v)
n += 1
except Exception:
pass
finally:
print("{} of {} variables successfully updated.".format(n, len(d)))
def export_helper(filepath):
session = settings.Session()
qry = session.query(Variable).all()
session.close()
var_dict = {}
d = json.JSONDecoder()
for var in qry:
val = None
try:
val = d.decode(var.val)
except Exception:
val = var.val
var_dict[var.key] = val
with open(filepath, 'w') as varfile:
varfile.write(json.dumps(var_dict, sort_keys=True, indent=4))
print("{} variables successfully exported to {}".format(len(var_dict), filepath))
@cli_utils.action_logging
def pause(args, dag=None):
set_is_paused(True, args, dag)
@cli_utils.action_logging
def unpause(args, dag=None):
set_is_paused(False, args, dag)
def set_is_paused(is_paused, args, dag=None):
dag = dag or get_dag(args)
session = settings.Session()
dm = session.query(DagModel).filter(
DagModel.dag_id == dag.dag_id).first()
dm.is_paused = is_paused
session.commit()
msg = "Dag: {}, paused: {}".format(dag, str(dag.is_paused))
print(msg)
def _run(args, dag, ti):
if args.local:
run_job = jobs.LocalTaskJob(
task_instance=ti,
mark_success=args.mark_success,
pickle_id=args.pickle,
ignore_all_deps=args.ignore_all_dependencies,
ignore_depends_on_past=args.ignore_depends_on_past,
ignore_task_deps=args.ignore_dependencies,
ignore_ti_state=args.force,
pool=args.pool)
run_job.run()
elif args.raw:
ti._run_raw_task(
mark_success=args.mark_success,
job_id=args.job_id,
pool=args.pool,
)
else:
pickle_id = None
if args.ship_dag:
try:
# Running remotely, so pickling the DAG
session = settings.Session()
pickle = DagPickle(dag)
session.add(pickle)
session.commit()
pickle_id = pickle.id
# TODO: This should be written to a log
print('Pickled dag {dag} as pickle_id:{pickle_id}'
.format(**locals()))
except Exception as e:
print('Could not pickle the DAG')
print(e)
raise e
executor = GetDefaultExecutor()
executor.start()
print("Sending to executor.")
executor.queue_task_instance(
ti,
mark_success=args.mark_success,
pickle_id=pickle_id,
ignore_all_deps=args.ignore_all_dependencies,
ignore_depends_on_past=args.ignore_depends_on_past,
ignore_task_deps=args.ignore_dependencies,
ignore_ti_state=args.force,
pool=args.pool)
executor.heartbeat()
executor.end()
@cli_utils.action_logging
def run(args, dag=None):
# Optional sections won't log an error if they're missing in airflow.cfg.
OPTIONAL_AIRFLOW_CFG_SECTIONS = [
'atlas',
'celery',
'celery_broker_transport_options',
'dask',
'elasticsearch',
'github_enterprise',
'hive',
'kerberos',
'kubernetes',
'kubernetes_node_selectors',
'kubernetes_secrets',
'ldap',
'lineage',
'mesos',
]
if dag:
args.dag_id = dag.dag_id
log = LoggingMixin().log
# Load custom airflow config
if args.cfg_path:
with open(args.cfg_path, 'r') as conf_file:
conf_dict = json.load(conf_file)
if os.path.exists(args.cfg_path):
os.remove(args.cfg_path)
# Do not log these properties since some may contain passwords.
# This may also set default values for database properties like
# core.sql_alchemy_pool_size
# core.sql_alchemy_pool_recycle
for section, config in conf_dict.items():
for option, value in config.items():
try:
conf.set(section, option, value)
except NoSectionError:
no_section_msg = (
'Section {section} Option {option} '
'does not exist in the config!'
).format(section=section, option=option)
if section in OPTIONAL_AIRFLOW_CFG_SECTIONS:
log.debug(no_section_msg)
else:
log.error(no_section_msg)
settings.configure_vars()
# IMPORTANT, have to use the NullPool, otherwise, each "run" command may leave
# behind multiple open sleeping connections while heartbeating, which could
# easily exceed the database connection limit when
# processing hundreds of simultaneous tasks.
settings.configure_orm(disable_connection_pool=True)
if not args.pickle and not dag:
dag = get_dag(args)
elif not dag:
session = settings.Session()
log.info('Loading pickle id {args.pickle}'.format(args=args))
dag_pickle = session.query(
DagPickle).filter(DagPickle.id == args.pickle).first()
if not dag_pickle:
raise AirflowException("Who hid the pickle!? [missing pickle]")
dag = dag_pickle.pickle
task = dag.get_task(task_id=args.task_id)
ti = TaskInstance(task, args.execution_date)
ti.refresh_from_db()
ti.init_run_context(raw=args.raw)
hostname = get_hostname()
log.info("Running %s on host %s", ti, hostname)
if args.interactive:
_run(args, dag, ti)
else:
with redirect_stdout(ti.log, logging.INFO), redirect_stderr(ti.log, logging.WARN):
_run(args, dag, ti)
logging.shutdown()
@cli_utils.action_logging
def task_failed_deps(args):
"""
Returns the unmet dependencies for a task instance from the perspective of the
scheduler (i.e. why a task instance doesn't get scheduled and then queued by the
scheduler, and then run by an executor).
>>> airflow task_failed_deps tutorial sleep 2015-01-01
Task instance dependencies not met:
Dagrun Running: Task instance's dagrun did not exist: Unknown reason
Trigger Rule: Task's trigger rule 'all_success' requires all upstream tasks
to have succeeded, but found 1 non-success(es).
"""
dag = get_dag(args)
task = dag.get_task(task_id=args.task_id)
ti = TaskInstance(task, args.execution_date)
dep_context = DepContext(deps=SCHEDULER_DEPS)
failed_deps = list(ti.get_failed_dep_statuses(dep_context=dep_context))
# TODO, Do we want to print or log this
if failed_deps:
print("Task instance dependencies not met:")
for dep in failed_deps:
print("{}: {}".format(dep.dep_name, dep.reason))
else:
print("Task instance dependencies are all met.")
@cli_utils.action_logging
def task_state(args):
"""
Returns the state of a TaskInstance at the command line.
>>> airflow task_state tutorial sleep 2015-01-01
success
"""
dag = get_dag(args)
task = dag.get_task(task_id=args.task_id)
ti = TaskInstance(task, args.execution_date)
print(ti.current_state())
@cli_utils.action_logging
def dag_state(args):
"""
Returns the state of a DagRun at the command line.
>>> airflow dag_state tutorial 2015-01-01T00:00:00.000000
running
"""
dag = get_dag(args)
dr = DagRun.find(dag.dag_id, execution_date=args.execution_date)
print(dr[0].state if len(dr) > 0 else None)
@cli_utils.action_logging
def next_execution(args):
"""
Returns the next execution datetime of a DAG at the command line.
>>> airflow next_execution tutorial
2018-08-31 10:38:00
"""
dag = get_dag(args)
if dag.is_paused:
print("[INFO] Please be reminded this DAG is PAUSED now.")
if dag.latest_execution_date:
next_execution_dttm = dag.following_schedule(dag.latest_execution_date)
if next_execution_dttm is None:
print("[WARN] No following schedule can be found. " +
"This DAG may have schedule interval '@once' or `None`.")
print(next_execution_dttm)
else:
print("[WARN] Only applicable when there is execution record found for the DAG.")
print(None)
@cli_utils.action_logging
def list_dags(args):
dagbag = DagBag(process_subdir(args.subdir))
s = textwrap.dedent("""\n
-------------------------------------------------------------------
DAGS
-------------------------------------------------------------------
{dag_list}
""")
dag_list = "\n".join(sorted(dagbag.dags))
print(s.format(dag_list=dag_list))
if args.report:
print(dagbag.dagbag_report())
@cli_utils.action_logging
def list_tasks(args, dag=None):
dag = dag or get_dag(args)
if args.tree:
dag.tree_view()
else:
tasks = sorted([t.task_id for t in dag.tasks])
print("\n".join(sorted(tasks)))
@cli_utils.action_logging
def test(args, dag=None):
dag = dag or get_dag(args)
task = dag.get_task(task_id=args.task_id)
# Add CLI provided task_params to task.params
if args.task_params:
passed_in_params = json.loads(args.task_params)
task.params.update(passed_in_params)
ti = TaskInstance(task, args.execution_date)
if args.dry_run:
ti.dry_run()
else:
ti.run(ignore_task_deps=True, ignore_ti_state=True, test_mode=True)
@cli_utils.action_logging
def render(args):
dag = get_dag(args)
task = dag.get_task(task_id=args.task_id)
ti = TaskInstance(task, args.execution_date)
ti.render_templates()
for attr in task.__class__.template_fields:
print(textwrap.dedent("""\
# ----------------------------------------------------------
# property: {}
# ----------------------------------------------------------
{}
""".format(attr, getattr(task, attr))))
@cli_utils.action_logging
def clear(args):
logging.basicConfig(
level=settings.LOGGING_LEVEL,
format=settings.SIMPLE_LOG_FORMAT)
dags = get_dags(args)
if args.task_regex:
for idx, dag in enumerate(dags):
dags[idx] = dag.sub_dag(
task_regex=args.task_regex,
include_downstream=args.downstream,
include_upstream=args.upstream)
DAG.clear_dags(
dags,
start_date=args.start_date,
end_date=args.end_date,
only_failed=args.only_failed,
only_running=args.only_running,
confirm_prompt=not args.no_confirm,
include_subdags=not args.exclude_subdags,
include_parentdag=not args.exclude_parentdag,
)
def get_num_ready_workers_running(gunicorn_master_proc):
workers = psutil.Process(gunicorn_master_proc.pid).children()
def ready_prefix_on_cmdline(proc):
try:
cmdline = proc.cmdline()
if len(cmdline) > 0:
return settings.GUNICORN_WORKER_READY_PREFIX in cmdline[0]
except psutil.NoSuchProcess:
pass
return False
ready_workers = [proc for proc in workers if ready_prefix_on_cmdline(proc)]
return len(ready_workers)
def get_num_workers_running(gunicorn_master_proc):
workers = psutil.Process(gunicorn_master_proc.pid).children()
return len(workers)
def restart_workers(gunicorn_master_proc, num_workers_expected, master_timeout):
"""
Runs forever, monitoring the child processes of @gunicorn_master_proc and
restarting workers occasionally.
Each iteration of the loop traverses one edge of this state transition
diagram, where each state (node) represents
[ num_ready_workers_running / num_workers_running ]. We expect most time to
be spent in [n / n]. `bs` is the setting webserver.worker_refresh_batch_size.
The horizontal transition at ? happens after the new worker parses all the
dags (so it could take a while!)
V ────────────────────────────────────────────────────────────────────────┐
[n / n] ──TTIN──> [ [n, n+bs) / n + bs ] ────?───> [n + bs / n + bs] ──TTOU─┘
^ ^───────────────┘
│
│ ┌────────────────v
└──────┴────── [ [0, n) / n ] <─── start
We change the number of workers by sending TTIN and TTOU to the gunicorn
master process, which increases and decreases the number of child workers
respectively. Gunicorn guarantees that on TTOU workers are terminated
gracefully and that the oldest worker is terminated.
"""
def wait_until_true(fn, timeout=0):
"""
Sleeps until fn is true
"""
t = time.time()
while not fn():
if 0 < timeout and timeout <= time.time() - t:
raise AirflowWebServerTimeout(
"No response from gunicorn master within {0} seconds"
.format(timeout))
time.sleep(0.1)
def start_refresh(gunicorn_master_proc):
batch_size = conf.getint('webserver', 'worker_refresh_batch_size')
log.debug('%s doing a refresh of %s workers', state, batch_size)
sys.stdout.flush()
sys.stderr.flush()
excess = 0
for _ in range(batch_size):
gunicorn_master_proc.send_signal(signal.SIGTTIN)
excess += 1
wait_until_true(lambda: num_workers_expected + excess ==
get_num_workers_running(gunicorn_master_proc),
master_timeout)
try:
wait_until_true(lambda: num_workers_expected ==
get_num_workers_running(gunicorn_master_proc),
master_timeout)
while True:
num_workers_running = get_num_workers_running(gunicorn_master_proc)
num_ready_workers_running = \
get_num_ready_workers_running(gunicorn_master_proc)
state = '[{0} / {1}]'.format(num_ready_workers_running, num_workers_running)
# Whenever some workers are not ready, wait until all workers are ready
if num_ready_workers_running < num_workers_running:
log.debug('%s some workers are starting up, waiting...', state)
sys.stdout.flush()
time.sleep(1)
# Kill a worker gracefully by asking gunicorn to reduce number of workers
elif num_workers_running > num_workers_expected:
excess = num_workers_running - num_workers_expected
log.debug('%s killing %s workers', state, excess)
for _ in range(excess):
gunicorn_master_proc.send_signal(signal.SIGTTOU)
excess -= 1
wait_until_true(lambda: num_workers_expected + excess ==
get_num_workers_running(gunicorn_master_proc),
master_timeout)
# Start a new worker by asking gunicorn to increase number of workers
elif num_workers_running == num_workers_expected:
refresh_interval = conf.getint('webserver', 'worker_refresh_interval')
log.debug(
'%s sleeping for %ss starting doing a refresh...',
state, refresh_interval
)
time.sleep(refresh_interval)
start_refresh(gunicorn_master_proc)
else:
# num_ready_workers_running == num_workers_running < num_workers_expected
log.error((
"%s some workers seem to have died and gunicorn"
"did not restart them as expected"
), state)
time.sleep(10)
if len(
psutil.Process(gunicorn_master_proc.pid).children()
) < num_workers_expected:
start_refresh(gunicorn_master_proc)
except (AirflowWebServerTimeout, OSError) as err:
log.error(err)
log.error("Shutting down webserver")
try:
gunicorn_master_proc.terminate()
gunicorn_master_proc.wait()
finally:
sys.exit(1)
@cli_utils.action_logging
def webserver(args):
print(settings.HEADER)
access_logfile = args.access_logfile or conf.get('webserver', 'access_logfile')
error_logfile = args.error_logfile or conf.get('webserver', 'error_logfile')
num_workers = args.workers or conf.get('webserver', 'workers')
worker_timeout = (args.worker_timeout or
conf.get('webserver', 'web_server_worker_timeout'))
ssl_cert = args.ssl_cert or conf.get('webserver', 'web_server_ssl_cert')
ssl_key = args.ssl_key or conf.get('webserver', 'web_server_ssl_key')
if not ssl_cert and ssl_key:
raise AirflowException(
'An SSL certificate must also be provided for use with ' + ssl_key)
if ssl_cert and not ssl_key:
raise AirflowException(
'An SSL key must also be provided for use with ' + ssl_cert)
if args.debug:
print(
"Starting the web server on port {0} and host {1}.".format(
args.port, args.hostname))
app = create_app_rbac(conf) if settings.RBAC else create_app(conf)
app.run(debug=True, port=args.port, host=args.hostname,
ssl_context=(ssl_cert, ssl_key) if ssl_cert and ssl_key else None)
else:
os.environ['SKIP_DAGS_PARSING'] = 'True'
app = cached_app_rbac(conf) if settings.RBAC else cached_app(conf)
pid, stdout, stderr, log_file = setup_locations(
"webserver", args.pid, args.stdout, args.stderr, args.log_file)
os.environ.pop('SKIP_DAGS_PARSING')
if args.daemon:
handle = setup_logging(log_file)
stdout = open(stdout, 'w+')
stderr = open(stderr, 'w+')
print(
textwrap.dedent('''\
Running the Gunicorn Server with:
Workers: {num_workers} {args.workerclass}
Host: {args.hostname}:{args.port}
Timeout: {worker_timeout}
Logfiles: {access_logfile} {error_logfile}
=================================================================\
'''.format(**locals())))
run_args = [
'gunicorn',
'-w', str(num_workers),
'-k', str(args.workerclass),
'-t', str(worker_timeout),
'-b', args.hostname + ':' + str(args.port),
'-n', 'airflow-webserver',
'-p', str(pid),
'-c', 'python:airflow.www.gunicorn_config',
]
if args.access_logfile:
run_args += ['--access-logfile', str(args.access_logfile)]
if args.error_logfile:
run_args += ['--error-logfile', str(args.error_logfile)]
if args.daemon:
run_args += ['-D']
if ssl_cert:
run_args += ['--certfile', ssl_cert, '--keyfile', ssl_key]
webserver_module = 'www_rbac' if settings.RBAC else 'www'
run_args += ["airflow." + webserver_module + ".app:cached_app()"]
gunicorn_master_proc = None
def kill_proc(dummy_signum, dummy_frame):
gunicorn_master_proc.terminate()
gunicorn_master_proc.wait()
sys.exit(0)
def monitor_gunicorn(gunicorn_master_proc):
# These run forever until SIG{INT, TERM, KILL, ...} signal is sent
if conf.getint('webserver', 'worker_refresh_interval') > 0:
master_timeout = conf.getint('webserver', 'web_server_master_timeout')
restart_workers(gunicorn_master_proc, num_workers, master_timeout)
else:
while gunicorn_master_proc.poll() is None:
time.sleep(1)
sys.exit(gunicorn_master_proc.returncode)
if args.daemon:
base, ext = os.path.splitext(pid)
ctx = daemon.DaemonContext(
pidfile=TimeoutPIDLockFile(base + "-monitor" + ext, -1),
files_preserve=[handle],
stdout=stdout,
stderr=stderr,
signal_map={
signal.SIGINT: kill_proc,
signal.SIGTERM: kill_proc
},
)
with ctx:
subprocess.Popen(run_args, close_fds=True)
# Reading pid file directly, since Popen#pid doesn't
# seem to return the right value with DaemonContext.
while True:
try:
with open(pid) as f:
gunicorn_master_proc_pid = int(f.read())
break
except IOError:
log.debug("Waiting for gunicorn's pid file to be created.")
time.sleep(0.1)
gunicorn_master_proc = psutil.Process(gunicorn_master_proc_pid)
monitor_gunicorn(gunicorn_master_proc)
stdout.close()
stderr.close()
else:
gunicorn_master_proc = subprocess.Popen(run_args, close_fds=True)
signal.signal(signal.SIGINT, kill_proc)
signal.signal(signal.SIGTERM, kill_proc)
monitor_gunicorn(gunicorn_master_proc)
@cli_utils.action_logging
def scheduler(args):
print(settings.HEADER)
job = jobs.SchedulerJob(
dag_id=args.dag_id,