Skip to content

Commit

Permalink
Add Deferrable mode to Emr Add Steps operator (#30928)
Browse files Browse the repository at this point in the history
* Add logging to waiters

* Update docs to mention availability of deferrable mode for EmrAddStepsOperator
  • Loading branch information
syedahsn authored Jun 8, 2023
1 parent 8891c69 commit 3a7c9f2
Show file tree
Hide file tree
Showing 6 changed files with 340 additions and 3 deletions.
31 changes: 29 additions & 2 deletions airflow/providers/amazon/aws/operators/emr.py
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@
from airflow.models import BaseOperator
from airflow.providers.amazon.aws.hooks.emr import EmrContainerHook, EmrHook, EmrServerlessHook
from airflow.providers.amazon.aws.links.emr import EmrClusterLink, EmrLogsLink, get_log_uri
from airflow.providers.amazon.aws.triggers.emr import EmrAddStepsTrigger
from airflow.providers.amazon.aws.utils.waiter import waiter
from airflow.utils.helpers import exactly_one, prune_dict
from airflow.utils.types import NOTSET, ArgNotSet
Expand Down Expand Up @@ -55,6 +56,10 @@ class EmrAddStepsOperator(BaseOperator):
:param wait_for_completion: If True, the operator will wait for all the steps to be completed.
:param execution_role_arn: The ARN of the runtime role for a step on the cluster.
:param do_xcom_push: if True, job_flow_id is pushed to XCom with key job_flow_id.
:param wait_for_completion: Whether to wait for job run completion. (default: True)
:param deferrable: If True, the operator will wait asynchronously for the job to complete.
This implies waiting for completion. This mode requires aiobotocore module to be installed.
(default: False)
"""

template_fields: Sequence[str] = (
Expand Down Expand Up @@ -84,6 +89,7 @@ def __init__(
waiter_delay: int | None = None,
waiter_max_attempts: int | None = None,
execution_role_arn: str | None = None,
deferrable: bool = False,
**kwargs,
):
if not exactly_one(job_flow_id is None, job_flow_name is None):
Expand All @@ -96,10 +102,11 @@ def __init__(
self.job_flow_name = job_flow_name
self.cluster_states = cluster_states
self.steps = steps
self.wait_for_completion = wait_for_completion
self.wait_for_completion = False if deferrable else wait_for_completion
self.waiter_delay = waiter_delay
self.waiter_max_attempts = waiter_max_attempts
self.execution_role_arn = execution_role_arn
self.deferrable = deferrable

def execute(self, context: Context) -> list[str]:
emr_hook = EmrHook(aws_conn_id=self.aws_conn_id)
Expand Down Expand Up @@ -137,14 +144,34 @@ def execute(self, context: Context) -> list[str]:
steps = self.steps
if isinstance(steps, str):
steps = ast.literal_eval(steps)
return emr_hook.add_job_flow_steps(
step_ids = emr_hook.add_job_flow_steps(
job_flow_id=job_flow_id,
steps=steps,
wait_for_completion=self.wait_for_completion,
waiter_delay=self.waiter_delay,
waiter_max_attempts=self.waiter_max_attempts,
execution_role_arn=self.execution_role_arn,
)
if self.deferrable:
self.defer(
trigger=EmrAddStepsTrigger(
job_flow_id=job_flow_id,
step_ids=step_ids,
aws_conn_id=self.aws_conn_id,
max_attempts=self.waiter_max_attempts,
poll_interval=self.waiter_delay,
),
method_name="execute_complete",
)

return step_ids

def execute_complete(self, context, event=None):
if event["status"] != "success":
raise AirflowException(f"Error resuming cluster: {event}")
else:
self.log.info("Steps completed successfully")
return event["step_ids"]


class EmrStartNotebookExecutionOperator(BaseOperator):
Expand Down
99 changes: 99 additions & 0 deletions airflow/providers/amazon/aws/triggers/emr.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,99 @@
# 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 annotations

import asyncio
from typing import Any

from botocore.exceptions import WaiterError

from airflow.providers.amazon.aws.hooks.emr import EmrHook
from airflow.triggers.base import BaseTrigger, TriggerEvent


class EmrAddStepsTrigger(BaseTrigger):
"""
AWS Emr Add Steps Trigger
The trigger will asynchronously poll the boto3 API and wait for the
steps to finish executing.
:param job_flow_id: The id of the job flow.
:param step_ids: The id of the steps being waited upon.
:param poll_interval: The amount of time in seconds to wait between attempts.
:param max_attempts: The maximum number of attempts to be made.
:param aws_conn_id: The Airflow connection used for AWS credentials.
"""

def __init__(
self,
job_flow_id: str,
step_ids: list[str],
aws_conn_id: str,
max_attempts: int | None,
poll_interval: int | None,
):
self.job_flow_id = job_flow_id
self.step_ids = step_ids
self.aws_conn_id = aws_conn_id
self.max_attempts = max_attempts
self.poll_interval = poll_interval

def serialize(self) -> tuple[str, dict[str, Any]]:
return (
"airflow.providers.amazon.aws.triggers.emr.EmrAddStepsTrigger",
{
"job_flow_id": str(self.job_flow_id),
"step_ids": self.step_ids,
"poll_interval": str(self.poll_interval),
"max_attempts": str(self.max_attempts),
"aws_conn_id": str(self.aws_conn_id),
},
)

async def run(self):
self.hook = EmrHook(aws_conn_id=self.aws_conn_id)
async with self.hook.async_conn as client:
for step_id in self.step_ids:
attempt = 0
waiter = client.get_waiter("step_complete")
while attempt < int(self.max_attempts):
attempt += 1
try:
await waiter.wait(
ClusterId=self.job_flow_id,
StepId=step_id,
WaiterConfig={
"Delay": int(self.poll_interval),
"MaxAttempts": 1,
},
)
break
except WaiterError as error:
if "terminal failure" in str(error):
yield TriggerEvent(
{"status": "failure", "message": f"Step {step_id} failed: {error}"}
)
break
self.log.info(
"Status of step is %s - %s",
error.last_response["Step"]["Status"]["State"],
error.last_response["Step"]["Status"]["StateChangeReason"],
)
await asyncio.sleep(int(self.poll_interval))
if attempt >= int(self.max_attempts):
yield TriggerEvent({"status": "failure", "message": "Steps failed: max attempts reached"})
else:
yield TriggerEvent({"status": "success", "message": "Steps completed", "step_ids": self.step_ids})
3 changes: 3 additions & 0 deletions airflow/providers/amazon/provider.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -522,6 +522,9 @@ triggers:
python-modules:
- airflow.providers.amazon.aws.triggers.glue
- airflow.providers.amazon.aws.triggers.glue_crawler
- integration-name: Amazon EMR
python-modules:
- airflow.providers.amazon.aws.triggers.emr

transfers:
- source-integration-name: Amazon DynamoDB
Expand Down
4 changes: 4 additions & 0 deletions docs/apache-airflow-providers-amazon/operators/emr/emr.rst
Original file line number Diff line number Diff line change
Expand Up @@ -89,6 +89,10 @@ Add Steps to an EMR job flow

To add steps to an existing EMR Job flow you can use
:class:`~airflow.providers.amazon.aws.operators.emr.EmrAddStepsOperator`.
This operator can be run in deferrable mode by passing ``deferrable=True`` as a parameter.
Using ``deferrable`` mode will release worker slots and leads to efficient utilization of
resources within Airflow cluster.However this mode will need the Airflow triggerer to be
available in your deployment.

.. exampleinclude:: /../../tests/system/providers/amazon/aws/example_emr.py
:language: python
Expand Down
36 changes: 35 additions & 1 deletion tests/providers/amazon/aws/operators/test_emr_add_steps.py
Original file line number Diff line number Diff line change
Expand Up @@ -25,11 +25,12 @@
import pytest
from jinja2 import StrictUndefined

from airflow.exceptions import AirflowException
from airflow.exceptions import AirflowException, TaskDeferred
from airflow.models import DAG, DagRun, TaskInstance
from airflow.providers.amazon.aws.hooks.emr import EmrHook
from airflow.providers.amazon.aws.hooks.s3 import S3Hook
from airflow.providers.amazon.aws.operators.emr import EmrAddStepsOperator
from airflow.providers.amazon.aws.triggers.emr import EmrAddStepsTrigger
from airflow.utils import timezone
from tests.test_utils import AIRFLOW_MAIN_FOLDER

Expand Down Expand Up @@ -244,3 +245,36 @@ def test_wait_for_completion(self, mock_add_job_flow_steps, *_):
waiter_max_attempts=None,
execution_role_arn=None,
)

def test_wait_for_completion_false_with_deferrable(self):
job_flow_id = "j-8989898989"
operator = EmrAddStepsOperator(
task_id="test_task",
job_flow_id=job_flow_id,
aws_conn_id="aws_default",
dag=DAG("test_dag_id", default_args=self.args),
wait_for_completion=True,
deferrable=True,
)

assert operator.wait_for_completion is False

@patch("airflow.providers.amazon.aws.operators.emr.get_log_uri")
@patch("airflow.providers.amazon.aws.hooks.emr.EmrHook.add_job_flow_steps")
def test_emr_add_steps_deferrable(self, mock_add_job_flow_steps, mock_get_log_uri):
mock_add_job_flow_steps.return_value = "test_step_id"
mock_get_log_uri.return_value = "test/log/uri"
job_flow_id = "j-8989898989"
operator = EmrAddStepsOperator(
task_id="test_task",
job_flow_id=job_flow_id,
aws_conn_id="aws_default",
dag=DAG("test_dag_id", default_args=self.args),
wait_for_completion=True,
deferrable=True,
)

with pytest.raises(TaskDeferred) as exc:
operator.execute(self.mock_context)

assert isinstance(exc.value.trigger, EmrAddStepsTrigger), "Trigger is not a EmrAddStepsTrigger"
Loading

0 comments on commit 3a7c9f2

Please sign in to comment.