-
Notifications
You must be signed in to change notification settings - Fork 13.7k
/
batch.py
547 lines (485 loc) · 21.8 KB
/
batch.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
# 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.
"""AWS Batch services.
.. seealso::
- https://boto3.amazonaws.com/v1/documentation/api/latest/guide/configuration.html
- https://boto3.amazonaws.com/v1/documentation/api/latest/reference/services/batch.html
- https://docs.aws.amazon.com/batch/latest/APIReference/Welcome.html
"""
from __future__ import annotations
import warnings
from datetime import timedelta
from functools import cached_property
from typing import TYPE_CHECKING, Any, Sequence
from airflow.configuration import conf
from airflow.exceptions import AirflowException, AirflowProviderDeprecationWarning
from airflow.models import BaseOperator
from airflow.models.mappedoperator import MappedOperator
from airflow.providers.amazon.aws.hooks.batch_client import BatchClientHook
from airflow.providers.amazon.aws.links.batch import (
BatchJobDefinitionLink,
BatchJobDetailsLink,
BatchJobQueueLink,
)
from airflow.providers.amazon.aws.links.logs import CloudWatchEventsLink
from airflow.providers.amazon.aws.triggers.batch import (
BatchCreateComputeEnvironmentTrigger,
BatchJobTrigger,
)
from airflow.providers.amazon.aws.utils import trim_none_values
from airflow.providers.amazon.aws.utils.task_log_fetcher import AwsTaskLogFetcher
if TYPE_CHECKING:
from airflow.utils.context import Context
class BatchOperator(BaseOperator):
"""Execute a job on AWS Batch.
.. seealso::
For more information on how to use this operator, take a look at the guide:
:ref:`howto/operator:BatchOperator`
:param job_name: the name for the job that will run on AWS Batch (templated)
:param job_definition: the job definition name on AWS Batch
:param job_queue: the queue name on AWS Batch
:param overrides: DEPRECATED, use container_overrides instead with the same value.
:param container_overrides: the `containerOverrides` parameter for boto3 (templated)
:param node_overrides: the `nodeOverrides` parameter for boto3 (templated)
:param share_identifier: The share identifier for the job. Don't specify this parameter if the job queue
doesn't have a scheduling policy.
:param scheduling_priority_override: The scheduling priority for the job.
Jobs with a higher scheduling priority are scheduled before jobs with a lower scheduling priority.
This overrides any scheduling priority in the job definition
:param array_properties: the `arrayProperties` parameter for boto3
:param parameters: the `parameters` for boto3 (templated)
:param job_id: the job ID, usually unknown (None) until the
submit_job operation gets the jobId defined by AWS Batch
:param waiters: an :py:class:`.BatchWaiters` object (see note below);
if None, polling is used with max_retries and status_retries.
:param max_retries: exponential back-off retries, 4200 = 48 hours;
polling is only used when waiters is None
:param status_retries: number of HTTP retries to get job status, 10;
polling is only used when waiters is None
:param aws_conn_id: connection id of AWS credentials / region name. If None,
credential boto3 strategy will be used.
:param region_name: region name to use in AWS Hook.
Override the region_name in connection (if provided)
:param tags: collection of tags to apply to the AWS Batch job submission
if None, no tags are submitted
:param deferrable: Run operator in the deferrable mode.
:param awslogs_enabled: Specifies whether logs from CloudWatch
should be printed or not, False.
If it is an array job, only the logs of the first task will be printed.
:param awslogs_fetch_interval: The interval with which cloudwatch logs are to be fetched, 30 sec.
:param poll_interval: (Deferrable mode only) Time in seconds to wait between polling.
.. note::
Any custom waiters must return a waiter for these calls:
.. code-block:: python
waiter = waiters.get_waiter("JobExists")
waiter = waiters.get_waiter("JobRunning")
waiter = waiters.get_waiter("JobComplete")
"""
ui_color = "#c3dae0"
arn: str | None = None
template_fields: Sequence[str] = (
"job_id",
"job_name",
"job_definition",
"job_queue",
"container_overrides",
"array_properties",
"node_overrides",
"parameters",
"retry_strategy",
"waiters",
"tags",
"wait_for_completion",
"awslogs_enabled",
"awslogs_fetch_interval",
)
template_fields_renderers = {
"container_overrides": "json",
"parameters": "json",
"node_overrides": "json",
"retry_strategy": "json",
}
@property
def operator_extra_links(self):
op_extra_links = [BatchJobDetailsLink()]
if isinstance(self, MappedOperator):
wait_for_completion = self.partial_kwargs.get(
"wait_for_completion"
) or self.expand_input.value.get("wait_for_completion")
array_properties = self.partial_kwargs.get("array_properties") or self.expand_input.value.get(
"array_properties"
)
else:
wait_for_completion = self.wait_for_completion
array_properties = self.array_properties
if wait_for_completion:
op_extra_links.extend([BatchJobDefinitionLink(), BatchJobQueueLink()])
if not array_properties:
# There is no CloudWatch Link to the parent Batch Job available.
op_extra_links.append(CloudWatchEventsLink())
return tuple(op_extra_links)
def __init__(
self,
*,
job_name: str,
job_definition: str,
job_queue: str,
overrides: dict | None = None, # deprecated
container_overrides: dict | None = None,
array_properties: dict | None = None,
node_overrides: dict | None = None,
share_identifier: str | None = None,
scheduling_priority_override: int | None = None,
parameters: dict | None = None,
retry_strategy: dict | None = None,
job_id: str | None = None,
waiters: Any | None = None,
max_retries: int = 4200,
status_retries: int | None = None,
aws_conn_id: str | None = None,
region_name: str | None = None,
tags: dict | None = None,
wait_for_completion: bool = True,
deferrable: bool = conf.getboolean("operators", "default_deferrable", fallback=False),
poll_interval: int = 30,
awslogs_enabled: bool = False,
awslogs_fetch_interval: timedelta = timedelta(seconds=30),
**kwargs,
) -> None:
BaseOperator.__init__(self, **kwargs)
self.job_id = job_id
self.job_name = job_name
self.job_definition = job_definition
self.job_queue = job_queue
self.container_overrides = container_overrides
# handle `overrides` deprecation in favor of `container_overrides`
if overrides:
if container_overrides:
# disallow setting both old and new params
raise AirflowException(
"'container_overrides' replaces the 'overrides' parameter. "
"You cannot specify both. Please remove assignation to the deprecated 'overrides'."
)
self.container_overrides = overrides
warnings.warn(
"Parameter `overrides` is deprecated, Please use `container_overrides` instead.",
AirflowProviderDeprecationWarning,
stacklevel=2,
)
self.node_overrides = node_overrides
self.share_identifier = share_identifier
self.scheduling_priority_override = scheduling_priority_override
self.array_properties = array_properties
self.parameters = parameters or {}
self.retry_strategy = retry_strategy or {}
if not self.retry_strategy.get("attempts", None):
self.retry_strategy["attempts"] = 1
self.waiters = waiters
self.tags = tags or {}
self.wait_for_completion = wait_for_completion
self.deferrable = deferrable
self.poll_interval = poll_interval
self.awslogs_enabled = awslogs_enabled
self.awslogs_fetch_interval = awslogs_fetch_interval
# params for hook
self.max_retries = max_retries
self.status_retries = status_retries
self.aws_conn_id = aws_conn_id
self.region_name = region_name
@cached_property
def hook(self) -> BatchClientHook:
return BatchClientHook(
max_retries=self.max_retries,
status_retries=self.status_retries,
aws_conn_id=self.aws_conn_id,
region_name=self.region_name,
)
def execute(self, context: Context) -> str | None:
"""Submit and monitor an AWS Batch job.
:raises: AirflowException
"""
self.submit_job(context)
if self.deferrable:
if not self.job_id:
raise AirflowException("AWS Batch job - job_id was not found")
job = self.hook.get_job_description(self.job_id)
job_status = job.get("status")
if job_status == self.hook.SUCCESS_STATE:
self.log.info("Job completed.")
return self.job_id
elif job_status == self.hook.FAILURE_STATE:
raise AirflowException(f"Error while running job: {self.job_id} is in {job_status} state")
elif job_status in self.hook.INTERMEDIATE_STATES:
self.defer(
timeout=self.execution_timeout,
trigger=BatchJobTrigger(
job_id=self.job_id,
waiter_max_attempts=self.max_retries,
aws_conn_id=self.aws_conn_id,
region_name=self.region_name,
waiter_delay=self.poll_interval,
),
method_name="execute_complete",
)
raise AirflowException(f"Unexpected status: {job_status}")
if self.wait_for_completion:
self.monitor_job(context)
return self.job_id
def execute_complete(self, context: Context, event: dict[str, Any] | None = None) -> str:
if event is None:
err_msg = "Trigger error: event is None"
self.log.info(err_msg)
raise AirflowException(err_msg)
if event["status"] != "success":
raise AirflowException(f"Error while running job: {event}")
self.log.info("Job completed.")
return event["job_id"]
def on_kill(self):
response = self.hook.client.terminate_job(jobId=self.job_id, reason="Task killed by the user")
self.log.info("AWS Batch job (%s) terminated: %s", self.job_id, response)
def submit_job(self, context: Context):
"""Submit an AWS Batch job.
:raises: AirflowException
"""
self.log.info(
"Running AWS Batch job - job definition: %s - on queue %s",
self.job_definition,
self.job_queue,
)
if self.container_overrides:
self.log.info("AWS Batch job - container overrides: %s", self.container_overrides)
if self.array_properties:
self.log.info("AWS Batch job - array properties: %s", self.array_properties)
if self.node_overrides:
self.log.info("AWS Batch job - node properties: %s", self.node_overrides)
args = {
"jobName": self.job_name,
"jobQueue": self.job_queue,
"jobDefinition": self.job_definition,
"arrayProperties": self.array_properties,
"parameters": self.parameters,
"tags": self.tags,
"containerOverrides": self.container_overrides,
"nodeOverrides": self.node_overrides,
"retryStrategy": self.retry_strategy,
"shareIdentifier": self.share_identifier,
"schedulingPriorityOverride": self.scheduling_priority_override,
}
try:
response = self.hook.client.submit_job(**trim_none_values(args))
except Exception as e:
self.log.error(
"AWS Batch job failed submission - job definition: %s - on queue %s",
self.job_definition,
self.job_queue,
)
raise AirflowException(e)
self.job_id = response["jobId"]
self.log.info("AWS Batch job (%s) started: %s", self.job_id, response)
BatchJobDetailsLink.persist(
context=context,
operator=self,
region_name=self.hook.conn_region_name,
aws_partition=self.hook.conn_partition,
job_id=self.job_id,
)
def monitor_job(self, context: Context):
"""Monitor an AWS Batch job.
This can raise an exception or an AirflowTaskTimeout if the task was
created with ``execution_timeout``.
"""
if not self.job_id:
raise AirflowException("AWS Batch job - job_id was not found")
try:
job_desc = self.hook.get_job_description(self.job_id)
job_definition_arn = job_desc["jobDefinition"]
job_queue_arn = job_desc["jobQueue"]
self.log.info(
"AWS Batch job (%s) Job Definition ARN: %r, Job Queue ARN: %r",
self.job_id,
job_definition_arn,
job_queue_arn,
)
except KeyError:
self.log.warning("AWS Batch job (%s) can't get Job Definition ARN and Job Queue ARN", self.job_id)
else:
BatchJobDefinitionLink.persist(
context=context,
operator=self,
region_name=self.hook.conn_region_name,
aws_partition=self.hook.conn_partition,
job_definition_arn=job_definition_arn,
)
BatchJobQueueLink.persist(
context=context,
operator=self,
region_name=self.hook.conn_region_name,
aws_partition=self.hook.conn_partition,
job_queue_arn=job_queue_arn,
)
if self.awslogs_enabled:
if self.waiters:
self.waiters.wait_for_job(self.job_id, get_batch_log_fetcher=self._get_batch_log_fetcher)
else:
self.hook.wait_for_job(self.job_id, get_batch_log_fetcher=self._get_batch_log_fetcher)
else:
if self.waiters:
self.waiters.wait_for_job(self.job_id)
else:
self.hook.wait_for_job(self.job_id)
awslogs = []
try:
awslogs = self.hook.get_job_all_awslogs_info(self.job_id)
except AirflowException as ae:
self.log.warning("Cannot determine where to find the AWS logs for this Batch job: %s", ae)
if awslogs:
self.log.info("AWS Batch job (%s) CloudWatch Events details found. Links to logs:", self.job_id)
link_builder = CloudWatchEventsLink()
for log in awslogs:
self.log.info(link_builder.format_link(**log))
if len(awslogs) > 1:
# there can be several log streams on multi-node jobs
self.log.warning(
"out of all those logs, we can only link to one in the UI. Using the first one."
)
CloudWatchEventsLink.persist(
context=context,
operator=self,
region_name=self.hook.conn_region_name,
aws_partition=self.hook.conn_partition,
**awslogs[0],
)
self.hook.check_job_success(self.job_id)
self.log.info("AWS Batch job (%s) succeeded", self.job_id)
def _get_batch_log_fetcher(self, job_id: str) -> AwsTaskLogFetcher | None:
awslog_info = self.hook.get_job_awslogs_info(job_id)
if not awslog_info:
return None
return AwsTaskLogFetcher(
aws_conn_id=self.aws_conn_id,
region_name=awslog_info["awslogs_region"],
log_group=awslog_info["awslogs_group"],
log_stream_name=awslog_info["awslogs_stream_name"],
fetch_interval=self.awslogs_fetch_interval,
logger=self.log,
)
class BatchCreateComputeEnvironmentOperator(BaseOperator):
"""Create an AWS Batch compute environment.
.. seealso::
For more information on how to use this operator, take a look at the guide:
:ref:`howto/operator:BatchCreateComputeEnvironmentOperator`
:param compute_environment_name: Name of the AWS batch compute
environment (templated).
:param environment_type: Type of the compute-environment.
:param state: State of the compute-environment.
:param compute_resources: Details about the resources managed by the
compute-environment (templated). More details:
https://boto3.amazonaws.com/v1/documentation/api/latest/reference/services/batch.html#Batch.Client.create_compute_environment
:param unmanaged_v_cpus: Maximum number of vCPU for an unmanaged compute
environment. This parameter is only supported when the ``type``
parameter is set to ``UNMANAGED``.
:param service_role: IAM role that allows Batch to make calls to other AWS
services on your behalf (templated).
:param tags: Tags that you apply to the compute-environment to help you
categorize and organize your resources.
:param poll_interval: How long to wait in seconds between 2 polls at the environment status.
Only useful when deferrable is True.
:param max_retries: How many times to poll for the environment status.
Only useful when deferrable is True.
:param aws_conn_id: Connection ID of AWS credentials / region name. If None,
credential boto3 strategy will be used.
:param region_name: Region name to use in AWS Hook. Overrides the
``region_name`` in connection if provided.
:param deferrable: If True, the operator will wait asynchronously for the environment to be created.
This mode requires aiobotocore module to be installed. (default: False)
"""
template_fields: Sequence[str] = (
"compute_environment_name",
"compute_resources",
"service_role",
)
template_fields_renderers = {"compute_resources": "json"}
def __init__(
self,
compute_environment_name: str,
environment_type: str,
state: str,
compute_resources: dict,
unmanaged_v_cpus: int | None = None,
service_role: str | None = None,
tags: dict | None = None,
poll_interval: int = 30,
max_retries: int | None = None,
aws_conn_id: str | None = None,
region_name: str | None = None,
deferrable: bool = conf.getboolean("operators", "default_deferrable", fallback=False),
**kwargs,
):
if "status_retries" in kwargs:
warnings.warn(
"The `status_retries` parameter is unused and should be removed. "
"It'll be deleted in a future version.",
AirflowProviderDeprecationWarning,
stacklevel=2,
)
kwargs.pop("status_retries") # remove before calling super() to prevent unexpected arg error
super().__init__(**kwargs)
self.compute_environment_name = compute_environment_name
self.environment_type = environment_type
self.state = state
self.unmanaged_v_cpus = unmanaged_v_cpus
self.compute_resources = compute_resources
self.service_role = service_role
self.tags = tags or {}
self.poll_interval = poll_interval
self.max_retries = max_retries or 120
self.aws_conn_id = aws_conn_id
self.region_name = region_name
self.deferrable = deferrable
@cached_property
def hook(self):
"""Create and return a BatchClientHook."""
return BatchClientHook(
aws_conn_id=self.aws_conn_id,
region_name=self.region_name,
)
def execute(self, context: Context):
"""Create an AWS batch compute environment."""
kwargs: dict[str, Any] = {
"computeEnvironmentName": self.compute_environment_name,
"type": self.environment_type,
"state": self.state,
"unmanagedvCpus": self.unmanaged_v_cpus,
"computeResources": self.compute_resources,
"serviceRole": self.service_role,
"tags": self.tags,
}
response = self.hook.client.create_compute_environment(**trim_none_values(kwargs))
arn = response["computeEnvironmentArn"]
if self.deferrable:
self.defer(
trigger=BatchCreateComputeEnvironmentTrigger(
arn, self.poll_interval, self.max_retries, self.aws_conn_id, self.region_name
),
method_name="execute_complete",
)
self.log.info("AWS Batch compute environment created successfully")
return arn
def execute_complete(self, context, event=None):
if event["status"] != "success":
raise AirflowException(f"Error while waiting for the compute environment to be ready: {event}")
return event["value"]