-
Notifications
You must be signed in to change notification settings - Fork 726
/
airflow_compiler.py
566 lines (510 loc) · 22.7 KB
/
airflow_compiler.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
import base64
from collections import defaultdict
from datetime import datetime, timedelta
from metaflow.exception import MetaflowException
from metaflow.parameters import deploy_time_eval
from metaflow.util import get_username
from metaflow import R
import sys
from metaflow.util import compress_list, dict_to_cli_options, to_pascalcase
from metaflow.plugins.timeout_decorator import get_run_time_limit_for_task
import os
from metaflow.mflog import bash_capture_logs
import random
import string
import json
from metaflow.decorators import flow_decorators
from metaflow.graph import FlowGraph, DAGNode
from metaflow.plugins.cards.card_modules import chevron
from metaflow.plugins.aws.aws_utils import compute_resource_attributes
from .exceptions import AirflowNotPresent, AirflowException
from .airflow_utils import (
TASK_ID_XCOM_KEY,
Workflow,
AirflowTask,
AirflowDAGArgs,
AIRFLOW_TASK_ID_TEMPLATE_VALUE,
)
from . import airflow_utils as af_utils
from .compute.k8s import create_k8s_args
import metaflow.util as util
# TODO : remove rich at the end.
# Question : Does scheduling interval be a top level argument
# Question : Does The schedule decorator have to be enforced.
AIRFLOW_DEPLOY_TEMPLATE_FILE = os.path.join(os.path.dirname(__file__), "af_deploy.py")
AIRFLOW_PREFIX = "arf"
# Task instance attributes : https://airflow.apache.org/docs/apache-airflow/stable/_api/airflow/models/taskinstance/index.html
class Airflow(object):
parameter_macro = "{{ params | json_dump }}"
task_id = AIRFLOW_TASK_ID_TEMPLATE_VALUE
task_id_arg = "--task-id %s" % task_id
# Airflow run_ids are of the form : "manual__2022-03-15T01:26:41.186781+00:00"
# Such run-ids break the `metaflow.util.decompress_list`; this is why we hash the runid
run_id = "%s-$(echo -n {{ run_id }} | md5sum | awk '{print $1}')" % AIRFLOW_PREFIX
# We do echo -n because emits line breaks and we dont want to consider that since it we want same hash value when retrieved in python.
run_id_arg = "--run-id %s" % run_id
attempt = "{{ task_instance.try_number - 1 }}"
def __init__(
self,
name,
graph,
flow,
code_package_sha,
code_package_url,
metadata,
flow_datastore,
environment,
event_logger,
monitor,
tags=None,
namespace=None,
username=None,
max_workers=None,
is_project=False,
email=None,
start_date=datetime.now(),
description=None,
catchup=False,
file_path=None,
):
self.name = name
self.graph = graph
self.flow = flow
self.code_package_sha = code_package_sha
self.code_package_url = code_package_url
self.metadata = metadata
self.flow_datastore = flow_datastore
self.environment = environment
self.event_logger = event_logger
self.monitor = monitor
self.tags = tags
self.namespace = namespace
self.username = username
self.max_workers = max_workers
self.email = email
self.description = description
self.start_date = start_date
self.catchup = catchup
self.schedule_interval = self._get_schedule()
self._file_path = file_path
self.metaflow_parameters = None
def _get_schedule(self):
schedule = self.flow._flow_decorators.get("schedule")
if schedule:
return schedule.schedule
# Schedule can be None.
# Especially if parameters are provided without defaults from toplevel.
return None
def _k8s_job(self, node, input_paths, env):
# since we are attaching k8s at cli, there will be one for a step.
k8s_deco = [deco for deco in node.decorators if deco.name == "kubernetes"][0]
user_code_retries, _ = self._get_retries(node)
retry_delay = self._get_retry_delay(node)
# This sets timeouts for @timeout decorators.
# The timeout is set as "execution_timeout" for an airflow task.
runtime_limit = get_run_time_limit_for_task(node.decorators)
return create_k8s_args(
self.flow_datastore,
self.metadata,
self.environment,
self.flow.name,
self.run_id,
node.name,
self.task_id,
self.attempt,
self.code_package_url,
self.code_package_sha,
self._step_cli(node, input_paths, self.code_package_url, user_code_retries),
k8s_deco.attributes["image"],
service_account=k8s_deco.attributes["service_account"], # todo set this
secrets=k8s_deco.attributes["secrets"],
node_selector=k8s_deco.attributes["node_selector"],
namespace=k8s_deco.attributes["namespace"],
cpu=k8s_deco.attributes["cpu"],
gpu=k8s_deco.attributes["gpu"],
disk=k8s_deco.attributes["disk"],
memory=k8s_deco.attributes["memory"],
retries=user_code_retries,
run_time_limit=timedelta(seconds=runtime_limit),
retry_delay=retry_delay,
env=env,
user=util.get_username(),
)
def _get_retries(self, node):
max_user_code_retries = 0
max_error_retries = 0
# Different decorators may have different retrying strategies, so take
# the max of them.
for deco in node.decorators:
user_code_retries, error_retries = deco.step_task_retry_count()
max_user_code_retries = max(max_user_code_retries, user_code_retries)
max_error_retries = max(max_error_retries, error_retries)
return max_user_code_retries, max_user_code_retries + max_error_retries
def _get_retry_delay(self, node):
retry_decos = [deco for deco in node.decorators if deco.name == "retry"]
if len(retry_decos) > 0:
retry_mins = retry_decos[0].attributes["minutes_between_retries"]
return timedelta(minutes=int(retry_mins))
return None
def _process_parameters(self):
# Copied from metaflow.plugins.aws.step_functions.step_functions
parameters = []
seen = set()
airflow_params = []
allowed_types = [int, str, bool, float]
type_transform_dict = {
int.__name__: "integer",
str.__name__: "string",
bool.__name__: "string",
float.__name__: "number",
}
type_parser = {bool.__name__: lambda v: str(v)}
for var, param in self.flow._get_parameters():
# Throw an exception if the parameter is specified twice.
norm = param.name.lower()
if norm in seen:
raise MetaflowException(
"Parameter *%s* is specified twice. "
"Note that parameter names are "
"case-insensitive." % param.name
)
seen.add(norm)
is_required = param.kwargs.get("required", False)
# Throw an exception if a schedule is set for a flow with required
# parameters with no defaults. We currently don't have any notion
# of data triggers in AWS Event Bridge.
if "default" not in param.kwargs and is_required:
raise MetaflowException(
"The parameter *%s* does not have a "
"default while having 'required' set to 'True'. "
"A default is required for such parameters when deploying on Airflow."
)
if "default" not in param.kwargs and self.schedule_interval:
raise MetaflowException(
"When @schedule is set with Airflow, Parameters require default values. "
"The parameter *%s* does not have a "
"'default' set"
)
value = deploy_time_eval(param.kwargs.get("default"))
parameters.append(dict(name=param.name, value=value))
# Setting airflow related param args.
param_type = param.kwargs.get("type", None)
airflowparam = dict(
name=param.name,
)
phelp = param.kwargs.get("help", None)
if value is not None:
airflowparam["default"] = value
if phelp:
airflowparam["description"] = phelp
if param_type in allowed_types:
airflowparam["type"] = type_transform_dict[param_type.__name__]
if param_type.__name__ in type_parser and value is not None:
airflowparam["default"] = type_parser[param_type.__name__](value)
airflow_params.append(airflowparam)
self.metaflow_parameters = airflow_params
return parameters
def _to_job(self, node: DAGNode):
# supported compute : k8s (v1), local(v2), batch(v3)
attrs = {
"metaflow.owner": self.username,
"metaflow.flow_name": self.flow.name,
"metaflow.step_name": node.name,
"metaflow.version": self.environment.get_environment_info()[
"metaflow_version"
],
"step_name": node.name,
}
# Making the key conditions to check into human readable variables so
# if statements make sense when reading logic
is_a_foreach = node.type == "foreach"
successors_are_foreach_joins = any(
self.graph[n].type == "join"
and self.graph[self.graph[n].split_parents[-1]].type == "foreach"
for n in node.out_funcs
)
join_in_foreach = (
node.type == "join" and self.graph[node.split_parents[-1]].type == "foreach"
)
any_incoming_node_is_foreach = any(
self.graph[n].type == "foreach" for n in node.in_funcs
)
# Add env vars from the optional @environment decorator.
env_deco = [deco for deco in node.decorators if deco.name == "environment"]
env = {}
if env_deco:
env = env_deco[0].attributes["vars"]
# The Below If/Else Block handle "Input Paths".
# Input Paths help manage dataflow across the graph.
if node.name == "start":
parameters = self._process_parameters()
if parameters:
env["METAFLOW_PARAMETERS"] = self.parameter_macro
default_parameters = {}
for parameter in parameters:
if parameter["value"] is not None:
default_parameters[parameter["name"]] = parameter["value"]
# Dump the default values specified in the flow.
env["METAFLOW_DEFAULT_PARAMETERS"] = json.dumps(default_parameters)
# Initialize parameters for the flow in the `start` step.
# todo : Handle parameters
# `start` step has no upstream input dependencies aside from
# parameters.
input_paths = None
else:
if node.parallel_foreach:
raise AirflowException(
"Parallel steps are not supported yet with Airflow."
)
# Handle foreach join.
if join_in_foreach:
# todo : Handle split values + input_paths
pass
else:
if len(node.in_funcs) == 1:
# set input paths where this is only one parent node
# The parent-task-id is passed via the xcom; There is no other way to get that.
# One key thing about xcoms is that they are immutable and only accepted if the task
# doesn't fail.
# From airflow docs :
# "Note: If the first task run is not succeeded then on every retry task XComs will be cleared to make the task run idempotent."
input_paths = (
# This is set using the `airflow_internal` decorator.
# This will pull the `return_value` xcom which holds a dictionary.
# This helps pass state.
"%s/%s/{{ task_instance.xcom_pull(task_ids='%s')['%s'] }}"
% (
self.run_id,
node.in_funcs[0],
node.in_funcs[0],
TASK_ID_XCOM_KEY,
)
)
else:
# this is a split scenario where there can be more than one input paths.
# todo : set input paths for a split join step
pass
env["METAFLOW_INPUT_PATHS"] = input_paths
if node.is_inside_foreach:
# Todo : Handle This case
pass
if any_incoming_node_is_foreach:
# todo : Handle split index for for-each.
# step.append("--split-index $METAFLOW_SPLIT_INDEX")
pass
# ! HACK : Remove Below Line; Only here temporarily.
env.update({k: v for k, v in os.environ.items() if "AWS" in k})
env["METAFLOW_CODE_URL"] = self.code_package_url
env["METAFLOW_FLOW_NAME"] = attrs["metaflow.flow_name"]
env["METAFLOW_STEP_NAME"] = attrs["metaflow.step_name"]
env["METAFLOW_OWNER"] = attrs["metaflow.owner"]
metadata_env = self.metadata.get_runtime_environment("airflow")
env.update(metadata_env)
metaflow_version = self.environment.get_environment_info()
metaflow_version["flow_name"] = self.graph.name
env["METAFLOW_VERSION"] = json.dumps(metaflow_version)
if (
is_a_foreach
or (node.is_inside_foreach and successors_are_foreach_joins)
or join_in_foreach
):
# Todo : Find ways to pass state using for the below usecases:
# 1. To set the cardinality of foreaches (which are subsequently)
# read prior to the instantiation of the Map state by AWS Step
# Functions.
# 2. To set the input paths from the parent steps of a foreach join.
# 3. To read the input paths in a foreach join.
pass
# Todo : Find and set resource requirements for the decorator.
compute_type = "k8s" # todo : This will become more dynamic in the future.
if compute_type == "k8s":
return self._k8s_job(node, input_paths, env)
def _step_cli(self, node, paths, code_package_url, user_code_retries):
cmds = []
script_name = os.path.basename(sys.argv[0])
executable = self.environment.executable(node.name)
if R.use_r():
entrypoint = [R.entrypoint()]
else:
entrypoint = [executable, script_name]
# Ignore compute decorators since this will already throw stuff there.
top_opts_dict = {
"with": [
decorator.make_decorator_spec()
for decorator in node.decorators
if not decorator.statically_defined and decorator.name != "kubernetes"
]
}
# FlowDecorators can define their own top-level options. They are
# responsible for adding their own top-level options and values through
# the get_top_level_options() hook. See similar logic in runtime.py.
for deco in flow_decorators():
top_opts_dict.update(deco.get_top_level_options())
top_opts = list(dict_to_cli_options(top_opts_dict))
join_in_foreach = (
node.type == "join" and self.graph[node.split_parents[-1]].type == "foreach"
)
any_previous_node_is_foreach = any(
self.graph[n].type == "foreach" for n in node.in_funcs
)
top_level = top_opts + [
"--quiet",
"--metadata=%s" % self.metadata.TYPE,
"--environment=%s" % self.environment.TYPE,
"--datastore=%s" % self.flow_datastore.TYPE,
"--datastore-root=%s" % self.flow_datastore.datastore_root,
"--event-logger=%s" % self.event_logger.logger_type,
"--monitor=%s" % self.monitor.monitor_type,
"--no-pylint",
"--with=airflow_internal",
]
if node.name == "start":
# We need a separate unique ID for the special _parameters task
task_id_params = "%s-params" % self.task_id
# TODO : Currently I am putting this boiler plate because we need to check if parameters are set or not.
# Export user-defined parameters into runtime environment
param_file = "".join(
random.choice(string.ascii_lowercase) for _ in range(10)
)
# Setup Parameters as environment variables which are stored in a dictionary.
export_params = " && ".join(
[
bash_capture_logs(
"python -m metaflow.plugins.airflow.plumbing.set_parameters %s"
% param_file
),
". `pwd`/%s" % param_file,
]
)
# Setting parameters over here.
params = (
entrypoint
+ top_level
+ [
"init",
self.run_id_arg,
"--task-id %s" % task_id_params,
]
)
# Assign tags to run objects.
if self.tags:
params.extend("--tag %s" % tag for tag in self.tags)
# If the start step gets retried, we must be careful not to
# regenerate multiple parameters tasks. Hence we check first if
# _parameters exists already.
exists = entrypoint + [
# Dump the parameters task
"dump",
"--max-value-size=0",
"%s/_parameters/%s" % (self.run_id, task_id_params),
]
cmd = "if ! %s >/dev/null 2>/dev/null; then %s && %s; fi" % (
" ".join(exists),
export_params,
bash_capture_logs(" ".join(params)),
)
cmds.append(cmd)
# set input paths for parameters
paths = "%s/_parameters/%s" % (self.run_id, task_id_params)
if join_in_foreach:
# todo : handle join case
pass
step = [
"step",
node.name,
self.run_id_arg,
self.task_id_arg,
"--retry-count %s" % self.attempt,
"--max-user-code-retries %d" % user_code_retries,
"--input-paths %s" % paths,
]
if any_previous_node_is_foreach:
# todo step.append("--split-index $METAFLOW_SPLIT_INDEX")
pass
if self.tags:
step.extend("--tag %s" % tag for tag in self.tags)
if self.namespace is not None:
step.append("--namespace=%s" % self.namespace)
cmds.append(bash_capture_logs(" ".join(entrypoint + top_level + step)))
return " && ".join(cmds)
def _validate_workflow(self):
# todo : check for batch/ other decorators.
# supported compute : k8s (v1), local(v2), batch(v3),
if self.metadata.TYPE != "service":
raise AirflowException('Metadata of type "service" required with Airflow')
if self.flow_datastore.TYPE != "s3":
raise AirflowException('Datastore of type "s3" required with Airflow')
# supported datastore : s3 (v1)
# supported metadata : service
pass
def compile(self):
# Visit every node of the flow and recursively build the state machine.
def _visit(node: DAGNode, workflow: Workflow, exit_node=None):
if node.parallel_foreach:
raise AirflowException(
"Deploying flows with @parallel decorator(s) "
"to Airflow is not supported currently."
)
state = AirflowTask(node.name).set_operator_args(**self._to_job(node))
if node.type == "end" or exit_node in node.out_funcs:
workflow.add_state(state)
# Continue linear assignment within the (sub)workflow if the node
# doesn't branch or fork.
elif node.type in ("start", "linear", "join"):
workflow.add_state(state.next(node.out_funcs[0]))
_visit(self.graph[node.out_funcs[0]], workflow, exit_node)
elif node.type == "split":
# Todo : handle Taskgroup in this step cardinality in some way
pass
elif node.type == "foreach":
# Todo : handle foreach cardinality in some way
# Continue the traversal from the matching_join.
_visit(self.graph[node.matching_join], workflow, exit_node)
# We shouldn't ideally ever get here.
else:
raise AirflowException(
"Node type *%s* for step *%s* "
"is not currently supported by "
"Airflow." % (node.type, node.name)
)
return workflow
workflow = Workflow(
dag_id=self.name,
default_args=self._create_defaults(),
description=self.description,
schedule_interval=self.schedule_interval,
start_date=self.start_date,
catchup=self.catchup,
tags=self.tags,
file_path=self._file_path,
)
workflow = _visit(self.graph["start"], workflow)
workflow.set_parameters(self.metaflow_parameters)
return self._create_airflow_file(workflow.to_dict())
def _create_airflow_file(self, json_dag):
util_file = None
with open(af_utils.__file__) as f:
util_file = f.read()
with open(AIRFLOW_DEPLOY_TEMPLATE_FILE) as f:
return chevron.render(
f.read(),
dict(
# Converting the configuration to base64 so that there can be no indentation related issues that can be caused because of
# malformed strings / json.
metaflow_workflow_compile_params=json_dag,
AIRFLOW_UTILS=util_file,
),
)
def _create_defaults(self):
return {
"owner": get_username(),
# If set on a task, doesn’t run the task in the current DAG run if the previous run of the task has failed.
"depends_on_past": False,
"email": [] if self.email is None else [self.email],
"email_on_failure": False,
"email_on_retry": False,
"retries": 0,
"execution_timeout": timedelta(days=5),
# check https://airflow.apache.org/docs/apache-airflow/stable/_api/airflow/models/baseoperator/index.html?highlight=retry_delay#airflow.models.baseoperator.BaseOperatorMeta
"retry_delay": timedelta(seconds=5),
}