-
Notifications
You must be signed in to change notification settings - Fork 13.7k
/
dag_command.py
602 lines (514 loc) · 20 KB
/
dag_command.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
# 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.
"""Dag sub-commands."""
from __future__ import annotations
import ast
import errno
import json
import logging
import operator
import signal
import subprocess
import sys
import warnings
from typing import TYPE_CHECKING
from sqlalchemy import delete, select
from airflow import settings
from airflow.api.client import get_current_api_client
from airflow.api_connexion.schemas.dag_schema import dag_schema
from airflow.cli.simple_table import AirflowConsole
from airflow.configuration import conf
from airflow.exceptions import AirflowException, RemovedInAirflow3Warning
from airflow.jobs.job import Job
from airflow.models import DagBag, DagModel, DagRun, TaskInstance
from airflow.models.dag import DAG
from airflow.models.serialized_dag import SerializedDagModel
from airflow.utils import cli as cli_utils, timezone
from airflow.utils.cli import get_dag, get_dags, process_subdir, sigint_handler, suppress_logs_and_warning
from airflow.utils.dot_renderer import render_dag, render_dag_dependencies
from airflow.utils.providers_configuration_loader import providers_configuration_loaded
from airflow.utils.session import NEW_SESSION, create_session, provide_session
from airflow.utils.state import DagRunState
if TYPE_CHECKING:
from graphviz.dot import Dot
from sqlalchemy.orm import Session
from airflow.timetables.base import DataInterval
log = logging.getLogger(__name__)
def _run_dag_backfill(dags: list[DAG], args) -> None:
# 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
run_conf = None
if args.conf:
run_conf = json.loads(args.conf)
for dag in dags:
if args.task_regex:
dag = dag.partial_subset(
task_ids_or_regex=args.task_regex, include_upstream=not args.ignore_dependencies
)
if not dag.task_dict:
raise AirflowException(
f"There are no tasks that match '{args.task_regex}' regex. Nothing to run, exiting..."
)
if args.dry_run:
print(f"Dry run of DAG {dag.dag_id} on {args.start_date}")
dagrun_infos = dag.iter_dagrun_infos_between(earliest=args.start_date, latest=args.end_date)
for dagrun_info in dagrun_infos:
dr = DagRun(
dag.dag_id,
execution_date=dagrun_info.logical_date,
data_interval=dagrun_info.data_interval,
)
for task in dag.tasks:
print(f"Task {task.task_id} located in DAG {dag.dag_id}")
ti = TaskInstance(task, run_id=None)
ti.dag_run = dr
ti.dry_run()
else:
if args.reset_dagruns:
DAG.clear_dags(
[dag],
start_date=args.start_date,
end_date=args.end_date,
confirm_prompt=not args.yes,
include_subdags=True,
dag_run_state=DagRunState.QUEUED,
)
try:
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,
run_backwards=args.run_backwards,
continue_on_failures=args.continue_on_failures,
disable_retry=args.disable_retry,
)
except ValueError as vr:
print(str(vr))
sys.exit(1)
@cli_utils.action_cli
@providers_configuration_loaded
def dag_backfill(args, dag: list[DAG] | DAG | None = None) -> None:
"""Create backfill job or dry run for a DAG or list of DAGs using regex."""
logging.basicConfig(level=settings.LOGGING_LEVEL, format=settings.SIMPLE_LOG_FORMAT)
signal.signal(signal.SIGTERM, sigint_handler)
warnings.warn(
"--ignore-first-depends-on-past is deprecated as the value is always set to True",
category=RemovedInAirflow3Warning,
)
if args.ignore_first_depends_on_past is False:
args.ignore_first_depends_on_past = True
if not args.start_date and not args.end_date:
raise AirflowException("Provide a start_date and/or end_date")
if not dag:
dags = get_dags(args.subdir, dag_id=args.dag_id, use_regex=args.treat_dag_as_regex)
elif isinstance(dag, list):
dags = dag
else:
dags = [dag]
del dag
dags.sort(key=lambda d: d.dag_id)
_run_dag_backfill(dags, args)
if len(dags) > 1:
log.info("All of the backfills are done.")
@cli_utils.action_cli
@providers_configuration_loaded
def dag_trigger(args) -> None:
"""Create a dag run for the specified dag."""
api_client = get_current_api_client()
try:
message = api_client.trigger_dag(
dag_id=args.dag_id,
run_id=args.run_id,
conf=args.conf,
execution_date=args.exec_date,
replace_microseconds=args.replace_microseconds,
)
AirflowConsole().print_as(
data=[message] if message is not None else [],
output=args.output,
)
except OSError as err:
raise AirflowException(err)
@cli_utils.action_cli
@providers_configuration_loaded
def dag_delete(args) -> None:
"""Delete all DB records related to the specified dag."""
api_client = get_current_api_client()
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)
print(message)
except OSError as err:
raise AirflowException(err)
else:
print("Cancelled")
@cli_utils.action_cli
@providers_configuration_loaded
def dag_pause(args) -> None:
"""Pauses a DAG."""
set_is_paused(True, args)
@cli_utils.action_cli
@providers_configuration_loaded
def dag_unpause(args) -> None:
"""Unpauses a DAG."""
set_is_paused(False, args)
@providers_configuration_loaded
def set_is_paused(is_paused: bool, args) -> None:
"""Set is_paused for DAG by a given dag_id."""
dag = DagModel.get_dagmodel(args.dag_id)
if not dag:
raise SystemExit(f"DAG: {args.dag_id} does not exist in 'dag' table")
dag.set_is_paused(is_paused=is_paused)
print(f"Dag: {args.dag_id}, paused: {is_paused}")
@providers_configuration_loaded
def dag_dependencies_show(args) -> None:
"""Display DAG dependencies, save to file or show as imgcat image."""
dot = render_dag_dependencies(SerializedDagModel.get_dag_dependencies())
filename = args.save
imgcat = args.imgcat
if filename and imgcat:
raise SystemExit(
"Option --save and --imgcat are mutually exclusive. "
"Please remove one option to execute the command.",
)
elif filename:
_save_dot_to_file(dot, filename)
elif imgcat:
_display_dot_via_imgcat(dot)
else:
print(dot.source)
@providers_configuration_loaded
def dag_show(args) -> None:
"""Display DAG or saves its graphic representation to the file."""
dag = get_dag(args.subdir, args.dag_id)
dot = render_dag(dag)
filename = args.save
imgcat = args.imgcat
if filename and imgcat:
raise SystemExit(
"Option --save and --imgcat are mutually exclusive. "
"Please remove one option to execute the command.",
)
elif filename:
_save_dot_to_file(dot, filename)
elif imgcat:
_display_dot_via_imgcat(dot)
else:
print(dot.source)
def _display_dot_via_imgcat(dot: Dot) -> None:
data = dot.pipe(format="png")
try:
with subprocess.Popen("imgcat", stdout=subprocess.PIPE, stdin=subprocess.PIPE) as proc:
out, err = proc.communicate(data)
if out:
print(out.decode("utf-8"))
if err:
print(err.decode("utf-8"))
except OSError as e:
if e.errno == errno.ENOENT:
raise SystemExit("Failed to execute. Make sure the imgcat executables are on your systems 'PATH'")
else:
raise
def _save_dot_to_file(dot: Dot, filename: str) -> None:
filename_without_ext, _, ext = filename.rpartition(".")
dot.render(filename=filename_without_ext, format=ext, cleanup=True)
print(f"File {filename} saved")
def _get_dagbag_dag_details(dag: DAG) -> dict:
"""Return a dagbag dag details dict."""
return {
"dag_id": dag.dag_id,
"root_dag_id": dag.parent_dag.dag_id if dag.parent_dag else None,
"is_paused": dag.get_is_paused(),
"is_active": dag.get_is_active(),
"is_subdag": dag.is_subdag,
"last_parsed_time": None,
"last_pickled": None,
"last_expired": None,
"scheduler_lock": None,
"pickle_id": dag.pickle_id,
"default_view": dag.default_view,
"fileloc": dag.fileloc,
"file_token": None,
"owners": dag.owner,
"description": dag.description,
"schedule_interval": dag.schedule_interval,
"timetable_description": dag.timetable.description,
"tags": dag.tags,
"max_active_tasks": dag.max_active_tasks,
"max_active_runs": dag.max_active_runs,
"has_task_concurrency_limits": any(
t.max_active_tis_per_dag is not None or t.max_active_tis_per_dagrun is not None for t in dag.tasks
),
"has_import_errors": False,
"next_dagrun": None,
"next_dagrun_data_interval_start": None,
"next_dagrun_data_interval_end": None,
"next_dagrun_create_after": None,
}
@cli_utils.action_cli
@providers_configuration_loaded
@provide_session
def dag_state(args, session: Session = NEW_SESSION) -> None:
"""
Return the state (and conf if exists) of a DagRun at the command line.
>>> airflow dags state tutorial 2015-01-01T00:00:00.000000
running
>>> airflow dags state a_dag_with_conf_passed 2015-01-01T00:00:00.000000
failed, {"name": "bob", "age": "42"}
"""
dag = DagModel.get_dagmodel(args.dag_id, session=session)
if not dag:
raise SystemExit(f"DAG: {args.dag_id} does not exist in 'dag' table")
dr = session.scalar(select(DagRun).filter_by(dag_id=args.dag_id, execution_date=args.execution_date))
out = dr.state if dr else None
conf_out = ""
if out and dr.conf:
conf_out = ", " + json.dumps(dr.conf)
print(str(out) + conf_out)
@cli_utils.action_cli
@providers_configuration_loaded
def dag_next_execution(args) -> None:
"""
Return the next execution datetime of a DAG at the command line.
>>> airflow dags next-execution tutorial
2018-08-31 10:38:00
"""
dag = get_dag(args.subdir, args.dag_id)
with create_session() as session:
last_parsed_dag: DagModel = session.scalars(
select(DagModel).where(DagModel.dag_id == dag.dag_id)
).one()
if last_parsed_dag.get_is_paused():
print("[INFO] Please be reminded this DAG is PAUSED now.", file=sys.stderr)
def print_execution_interval(interval: DataInterval | None):
if interval is None:
print(
"[WARN] No following schedule can be found. "
"This DAG may have schedule interval '@once' or `None`.",
file=sys.stderr,
)
print(None)
return
print(interval.start.isoformat())
next_interval = dag.get_next_data_interval(last_parsed_dag)
print_execution_interval(next_interval)
for _ in range(1, args.num_executions):
next_info = dag.next_dagrun_info(next_interval, restricted=False)
next_interval = None if next_info is None else next_info.data_interval
print_execution_interval(next_interval)
@cli_utils.action_cli
@suppress_logs_and_warning
@providers_configuration_loaded
@provide_session
def dag_list_dags(args, session=NEW_SESSION) -> None:
"""Display dags with or without stats at the command line."""
cols = args.columns if args.columns else []
invalid_cols = [c for c in cols if c not in dag_schema.fields]
valid_cols = [c for c in cols if c in dag_schema.fields]
if invalid_cols:
from rich import print as rich_print
rich_print(
f"[red][bold]Error:[/bold] Ignoring the following invalid columns: {invalid_cols}. "
f"List of valid columns: {list(dag_schema.fields.keys())}",
file=sys.stderr,
)
dagbag = DagBag(process_subdir(args.subdir))
if dagbag.import_errors:
from rich import print as rich_print
rich_print(
"[red][bold]Error:[/bold] Failed to load all files. "
"For details, run `airflow dags list-import-errors`",
file=sys.stderr,
)
def get_dag_detail(dag: DAG) -> dict:
dag_model = DagModel.get_dagmodel(dag.dag_id, session=session)
if dag_model:
dag_detail = dag_schema.dump(dag_model)
else:
dag_detail = _get_dagbag_dag_details(dag)
return {col: dag_detail[col] for col in valid_cols}
AirflowConsole().print_as(
data=sorted(dagbag.dags.values(), key=operator.attrgetter("dag_id")),
output=args.output,
mapper=get_dag_detail,
)
@cli_utils.action_cli
@suppress_logs_and_warning
@providers_configuration_loaded
@provide_session
def dag_details(args, session=NEW_SESSION):
"""Get DAG details given a DAG id."""
dag = DagModel.get_dagmodel(args.dag_id, session=session)
if not dag:
raise SystemExit(f"DAG: {args.dag_id} does not exist in 'dag' table")
dag_detail = dag_schema.dump(dag)
if args.output in ["table", "plain"]:
data = [{"property_name": key, "property_value": value} for key, value in dag_detail.items()]
else:
data = [dag_detail]
AirflowConsole().print_as(
data=data,
output=args.output,
)
@cli_utils.action_cli
@suppress_logs_and_warning
@providers_configuration_loaded
def dag_list_import_errors(args) -> None:
"""Display dags with import errors on the command line."""
dagbag = DagBag(process_subdir(args.subdir))
data = []
for filename, errors in dagbag.import_errors.items():
data.append({"filepath": filename, "error": errors})
AirflowConsole().print_as(
data=data,
output=args.output,
)
if data:
sys.exit(1)
@cli_utils.action_cli
@suppress_logs_and_warning
@providers_configuration_loaded
def dag_report(args) -> None:
"""Display dagbag stats at the command line."""
dagbag = DagBag(process_subdir(args.subdir))
AirflowConsole().print_as(
data=dagbag.dagbag_stats,
output=args.output,
mapper=lambda x: {
"file": x.file,
"duration": x.duration,
"dag_num": x.dag_num,
"task_num": x.task_num,
"dags": sorted(ast.literal_eval(x.dags)),
},
)
@cli_utils.action_cli
@suppress_logs_and_warning
@providers_configuration_loaded
@provide_session
def dag_list_jobs(args, dag: DAG | None = None, session: Session = NEW_SESSION) -> None:
"""List latest n jobs."""
queries = []
if dag:
args.dag_id = dag.dag_id
if args.dag_id:
dag = DagModel.get_dagmodel(args.dag_id, session=session)
if not dag:
raise SystemExit(f"DAG: {args.dag_id} does not exist in 'dag' table")
queries.append(Job.dag_id == args.dag_id)
if args.state:
queries.append(Job.state == args.state)
fields = ["dag_id", "state", "job_type", "start_date", "end_date"]
all_jobs_iter = session.scalars(
select(Job).where(*queries).order_by(Job.start_date.desc()).limit(args.limit)
)
all_jobs = [{f: str(job.__getattribute__(f)) for f in fields} for job in all_jobs_iter]
AirflowConsole().print_as(
data=all_jobs,
output=args.output,
)
@cli_utils.action_cli
@suppress_logs_and_warning
@providers_configuration_loaded
@provide_session
def dag_list_dag_runs(args, dag: DAG | None = None, session: Session = NEW_SESSION) -> None:
"""List dag runs for a given DAG."""
if dag:
args.dag_id = dag.dag_id
else:
dag = DagModel.get_dagmodel(args.dag_id, session=session)
if not dag:
raise SystemExit(f"DAG: {args.dag_id} does not exist in 'dag' table")
state = args.state.lower() if args.state else None
dag_runs = DagRun.find(
dag_id=args.dag_id,
state=state,
no_backfills=args.no_backfill,
execution_start_date=args.start_date,
execution_end_date=args.end_date,
session=session,
)
dag_runs.sort(key=lambda x: x.execution_date, reverse=True)
AirflowConsole().print_as(
data=dag_runs,
output=args.output,
mapper=lambda dr: {
"dag_id": dr.dag_id,
"run_id": dr.run_id,
"state": dr.state,
"execution_date": dr.execution_date.isoformat(),
"start_date": dr.start_date.isoformat() if dr.start_date else "",
"end_date": dr.end_date.isoformat() if dr.end_date else "",
},
)
@cli_utils.action_cli
@providers_configuration_loaded
@provide_session
def dag_test(args, dag: DAG | None = None, session: Session = NEW_SESSION) -> None:
"""Execute one single DagRun for a given DAG and execution date."""
run_conf = None
if args.conf:
try:
run_conf = json.loads(args.conf)
except ValueError as e:
raise SystemExit(f"Configuration {args.conf!r} is not valid JSON. Error: {e}")
execution_date = args.execution_date or timezone.utcnow()
dag = dag or get_dag(subdir=args.subdir, dag_id=args.dag_id)
dr: DagRun = dag.test(execution_date=execution_date, run_conf=run_conf, session=session)
show_dagrun = args.show_dagrun
imgcat = args.imgcat_dagrun
filename = args.save_dagrun
if show_dagrun or imgcat or filename:
tis = session.scalars(
select(TaskInstance).where(
TaskInstance.dag_id == args.dag_id,
TaskInstance.execution_date == execution_date,
)
).all()
dot_graph = render_dag(dag, tis=tis)
print()
if filename:
_save_dot_to_file(dot_graph, filename)
if imgcat:
_display_dot_via_imgcat(dot_graph)
if show_dagrun:
print(dot_graph.source)
if dr and dr.state == DagRunState.FAILED:
raise SystemExit("DagRun failed")
@cli_utils.action_cli
@providers_configuration_loaded
@provide_session
def dag_reserialize(args, session: Session = NEW_SESSION) -> None:
"""Serialize a DAG instance."""
session.execute(delete(SerializedDagModel).execution_options(synchronize_session=False))
if not args.clear_only:
dagbag = DagBag(process_subdir(args.subdir))
dagbag.sync_to_db(session=session)