Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
29 changes: 29 additions & 0 deletions airflow/providers/amazon/aws/sensors/emr.py
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@
# under the License.
from __future__ import annotations

from datetime import timedelta
from typing import TYPE_CHECKING, Any, Iterable, Sequence

from deprecated import deprecated
Expand All @@ -25,6 +26,7 @@
from airflow.providers.amazon.aws.hooks.emr import EmrContainerHook, EmrHook, EmrServerlessHook
from airflow.providers.amazon.aws.hooks.s3 import S3Hook
from airflow.providers.amazon.aws.links.emr import EmrLogsLink
from airflow.providers.amazon.aws.triggers.emr import EmrStepSensorTrigger
from airflow.sensors.base import BaseSensorOperator

if TYPE_CHECKING:
Expand Down Expand Up @@ -468,6 +470,7 @@ class EmrStepSensor(EmrBaseSensor):
step reaches any of these states
:param failed_states: the failure states, sensor fails when
step reaches any of these states
:param deferrable: Run sensor in the deferrable mode.
"""

template_fields: Sequence[str] = ("job_flow_id", "step_id", "target_states", "failed_states")
Expand All @@ -480,13 +483,17 @@ def __init__(
step_id: str,
target_states: Iterable[str] | None = None,
failed_states: Iterable[str] | None = None,
deferrable: bool = False,
max_attempts: int = 60,
**kwargs,
):
super().__init__(**kwargs)
self.job_flow_id = job_flow_id
self.step_id = step_id
self.target_states = target_states or ["COMPLETED"]
self.failed_states = failed_states or ["CANCELLED", "FAILED", "INTERRUPTED"]
self.deferrable = deferrable
self.max_attempts = max_attempts

def get_emr_response(self, context: Context) -> dict[str, Any]:
"""
Expand Down Expand Up @@ -527,3 +534,25 @@ def failure_message_from_response(response: dict[str, Any]) -> str | None:
f"with message {fail_details.get('Message')} and log file {fail_details.get('LogFile')}"
)
return None

def execute(self, context: Context) -> None:
"""Deferred and give control to trigger"""
if self.deferrable and not self.poke(context):
self.defer(
timeout=timedelta(seconds=self.timeout),
trigger=EmrStepSensorTrigger(
job_flow_id=self.job_flow_id,
step_id=self.step_id,
target_states=self.target_states,
aws_conn_id=self.aws_conn_id,
poke_interval=self.poke_interval,
max_attempts=self.max_attempts,
),
method_name="execute_complete",
)

def execute_complete(self, context, event=None):
if event["status"] != "success":
raise AirflowException(f"Error while running job: {event}")
else:
self.log.info("Job completed.")
90 changes: 90 additions & 0 deletions airflow/providers/amazon/aws/triggers/emr.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,90 @@
# 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

from typing import Any, AsyncIterator, Iterable

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


class EmrStepSensorTrigger(BaseTrigger):
"""
Poll for the status of EMR container until reaches terminal state

:param virtual_cluster_id: Reference Emr cluster id
:param job_id: job_id to check the state
:param max_tries: maximum try attempts for polling the status
:param aws_conn_id: Reference to AWS connection id
:param poll_interval: polling period in seconds to check for the status
"""

def __init__(
self,
job_flow_id: str,
step_id: str,
target_states: Iterable[str],
aws_conn_id: str = "aws_default",
poll_interval: int = 30,
max_attempts: int = 60,
**kwargs: Any,
):
self.job_flow_id = job_flow_id
self.step_id = step_id
self.target_states = target_states
self.aws_conn_id = aws_conn_id
self.poll_interval = poll_interval
self.max_attempts = max_attempts
super().__init__(**kwargs)

@cached_property
def hook(self) -> EmrHook:
return EmrHook(self.aws_conn_id)

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

async def run(self) -> AsyncIterator[TriggerEvent]:
async with self.hook.async_conn as client:
waiter = self.hook.get_waiter("job_step_wait_for_terminal", deferrable=True, client=client)
await waiter.wait(
ClusterId=self.job_flow_id,
StepId=self.step_id,
WaiterConfig={
"Delay": self.poll_interval,
"MaxAttempts": self.max_attempts,
},
)

response = self.hook.conn.describe_step(ClusterId=self.job_flow_id, StepId=self.step_id)
state = response["Step"]["Status"]["State"]
if state in self.target_states:
yield TriggerEvent({"status": "success"})
else:
yield TriggerEvent({"status": "failed", "response": response})
31 changes: 31 additions & 0 deletions airflow/providers/amazon/aws/waiters/emr.json
Original file line number Diff line number Diff line change
Expand Up @@ -75,6 +75,37 @@
"state": "failure"
}
]
},
"job_step_wait_for_terminal": {
"operation": "DescribeStep",
"delay": 30,
"maxAttempts": 60,
"acceptors": [
{
"matcher": "path",
"argument": "step.status",
"expected": "COMPLETED",
"state": "success"
},
{
"matcher": "path",
"argument": "step.status",
"expected": "CANCELLED",
"state": "success"
},
{
"matcher": "path",
"argument": "step.status",
"expected": "FAILED",
"state": "failure"
},
{
"matcher": "path",
"argument": "step.status",
"expected": "INTERRUPTED",
"state": "failure"
}
]
}
}
}
19 changes: 18 additions & 1 deletion tests/providers/amazon/aws/sensors/test_emr_step.py
Original file line number Diff line number Diff line change
Expand Up @@ -24,8 +24,9 @@
import pytest
from dateutil.tz import tzlocal

from airflow.exceptions import AirflowException
from airflow.exceptions import AirflowException, TaskDeferred
from airflow.providers.amazon.aws.sensors.emr import EmrStepSensor
from airflow.providers.amazon.aws.triggers.emr import EmrStepSensorTrigger

DESCRIBE_JOB_STEP_RUNNING_RETURN = {
"ResponseMetadata": {"HTTPStatusCode": 200, "RequestId": "8dee8db2-3719-11e6-9e20-35b2f861a2a6"},
Expand Down Expand Up @@ -216,3 +217,19 @@ def test_step_interrupted(self):
mock_isinstance.return_value = True
with pytest.raises(AirflowException):
self.sensor.execute(None)

@mock.patch("airflow.providers.amazon.aws.sensors.emr.EmrStepSensor.poke")
def test_sensor_defer(self, mock_poke):
sensor = EmrStepSensor(
task_id="test_task",
poke_interval=0,
job_flow_id="j-8989898989",
step_id="s-VK57YR1Z9Z5N",
aws_conn_id="aws_default",
deferrable=True,
)

mock_poke.return_value = False
with pytest.raises(TaskDeferred) as exc:
sensor.execute(context=None)
assert isinstance(exc.value.trigger, EmrStepSensorTrigger), "Trigger is not a EmrStepSensorTrigger"
77 changes: 77 additions & 0 deletions tests/providers/amazon/aws/triggers/test_emr.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,77 @@
# 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

from unittest.mock import PropertyMock

import pytest

from airflow.providers.amazon.aws.triggers.emr import EmrStepSensorTrigger
from tests.providers.amazon.aws.utils.compat import AsyncMock, async_mock

JOB__FLOW_ID = "job-1234"
STEP_ID = "s-1234"
TARGET_STATE = ["TERMINATED"]
AWS_CONN_ID = "aws_emr_conn"
POLL_INTERVAL = 60
MAX_ATTEMPTS = 5


class TestEmrStepSensorTrigger:
def test_emr_job_step_sensor_trigger_serialize(self):
emr_trigger = EmrStepSensorTrigger(
job_flow_id=JOB__FLOW_ID,
step_id=STEP_ID,
target_states=TARGET_STATE,
aws_conn_id=AWS_CONN_ID,
poll_interval=POLL_INTERVAL,
max_attempts=MAX_ATTEMPTS,
)
class_path, args = emr_trigger.serialize()
assert class_path == "airflow.providers.amazon.aws.triggers.emr.EmrStepSensorTrigger"
assert args["job_flow_id"] == JOB__FLOW_ID
assert args["step_id"] == STEP_ID
assert args["target_states"] == TARGET_STATE
assert args["aws_conn_id"] == AWS_CONN_ID
assert args["poll_interval"] == POLL_INTERVAL
assert args["max_attempts"] == MAX_ATTEMPTS

@pytest.mark.asyncio
@async_mock.patch(
"airflow.providers.amazon.aws.hooks.base_aws.AwsGenericHook.conn", new_callable=PropertyMock
)
@async_mock.patch("airflow.providers.amazon.aws.hooks.emr.EmrHook.get_waiter")
@async_mock.patch("airflow.providers.amazon.aws.hooks.emr.EmrHook.async_conn")
async def test_emr_step_sensor_trigger_run(self, mock_async_conn, mock_get_waiter, mock_conn):
mock = async_mock.MagicMock()
mock_async_conn.__aenter__.return_value = mock

mock_get_waiter().wait = AsyncMock()

emr_trigger = EmrStepSensorTrigger(
job_flow_id=JOB__FLOW_ID,
target_states=TARGET_STATE,
step_id=STEP_ID,
aws_conn_id=AWS_CONN_ID,
poll_interval=POLL_INTERVAL,
max_attempts=MAX_ATTEMPTS,
)

generator = emr_trigger.run()
await generator.asend(None)

assert mock_conn.return_value.describe_step.called