forked from apache/airflow
-
Notifications
You must be signed in to change notification settings - Fork 0
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
Add MwaaTriggerDagRunOperator and MwaaHook to Amazon Provider Package (…
…apache#46579) Adds AWS Managed Workflows for Apache Airflow (MWAA) integration to the Amazon Provider Package. The integration includes: - MwaaTriggerDagRunOperator: Allows triggering DAGs in an MWAA environment - MwaaHook: A generic hook to interact with MWAA environments via REST API
- Loading branch information
1 parent
6e9c7a5
commit 9c22411
Showing
9 changed files
with
570 additions
and
0 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -1101,6 +1101,7 @@ muldelete | |
Multinamespace | ||
mutex | ||
mv | ||
mwaa | ||
mypy | ||
Mysql | ||
mysql | ||
|
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,64 @@ | ||
.. 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. | ||
================================================== | ||
Amazon Managed Workflows for Apache Airflow (MWAA) | ||
================================================== | ||
|
||
`Amazon Managed Workflows for Apache Airflow (MWAA) <https://aws.amazon.com/managed-workflows-for-apache-airflow/>`__ | ||
is a managed service for Apache Airflow that lets you use your current, familiar Apache Airflow platform to orchestrate | ||
your workflows. You gain improved scalability, availability, and security without the operational burden of managing | ||
underlying infrastructure. | ||
|
||
Prerequisite Tasks | ||
------------------ | ||
|
||
.. include:: ../_partials/prerequisite_tasks.rst | ||
|
||
Generic Parameters | ||
------------------ | ||
|
||
.. include:: ../_partials/generic_parameters.rst | ||
|
||
Operators | ||
--------- | ||
|
||
.. _howto/operator:MwaaTriggerDagRunOperator: | ||
|
||
Trigger a DAG run in an Amazon MWAA environment | ||
=============================================== | ||
|
||
To trigger a DAG run in an Amazon MWAA environment you can use the | ||
:class:`~airflow.providers.amazon.aws.operators.mwaa.MwaaTriggerDagRunOperator` | ||
|
||
Note: Unlike :class:`~airflow.providers.standard.operators.trigger_dagrun.TriggerDagRunOperator`, this operator is capable of | ||
triggering a DAG in a separate Airflow environment as long as the environment with the DAG being triggered is running on | ||
AWS MWAA. | ||
|
||
In the following example, the task ``trigger_dag_run`` triggers a dag run for a DAG with with the ID ``hello_world`` in | ||
the environment ``MyAirflowEnvironment``. | ||
|
||
.. exampleinclude:: /../../providers/amazon/tests/system/amazon/aws/example_mwaa.py | ||
:language: python | ||
:dedent: 4 | ||
:start-after: [START howto_operator_mwaa_trigger_dag_run] | ||
:end-before: [END howto_operator_mwaa_trigger_dag_run] | ||
|
||
References | ||
---------- | ||
|
||
* `AWS boto3 library documentation for MWAA <https://boto3.amazonaws.com/v1/documentation/api/latest/reference/services/mwaa.html>`__ |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
85 changes: 85 additions & 0 deletions
85
providers/amazon/src/airflow/providers/amazon/aws/hooks/mwaa.py
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,85 @@ | ||
# 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. | ||
"""This module contains AWS MWAA hook.""" | ||
|
||
from __future__ import annotations | ||
|
||
from botocore.exceptions import ClientError | ||
|
||
from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook | ||
|
||
|
||
class MwaaHook(AwsBaseHook): | ||
""" | ||
Interact with AWS Manager Workflows for Apache Airflow. | ||
Provide thin wrapper around :external+boto3:py:class:`boto3.client("mwaa") <MWAA.Client>` | ||
Additional arguments (such as ``aws_conn_id``) may be specified and | ||
are passed down to the underlying AwsBaseHook. | ||
.. seealso:: | ||
- :class:`airflow.providers.amazon.aws.hooks.base_aws.AwsBaseHook` | ||
""" | ||
|
||
def __init__(self, *args, **kwargs) -> None: | ||
kwargs["client_type"] = "mwaa" | ||
super().__init__(*args, **kwargs) | ||
|
||
def invoke_rest_api( | ||
self, | ||
env_name: str, | ||
path: str, | ||
method: str, | ||
body: dict | None = None, | ||
query_params: dict | None = None, | ||
) -> dict: | ||
""" | ||
Invoke the REST API on the Airflow webserver with the specified inputs. | ||
.. seealso:: | ||
- :external+boto3:py:meth:`MWAA.Client.invoke_rest_api` | ||
:param env_name: name of the MWAA environment | ||
:param path: Apache Airflow REST API endpoint path to be called | ||
:param method: HTTP method used for making Airflow REST API calls | ||
:param body: Request body for the Apache Airflow REST API call | ||
:param query_params: Query parameters to be included in the Apache Airflow REST API call | ||
""" | ||
body = body or {} | ||
api_kwargs = { | ||
"Name": env_name, | ||
"Path": path, | ||
"Method": method, | ||
# Filter out keys with None values because Airflow REST API doesn't accept requests otherwise | ||
"Body": {k: v for k, v in body.items() if v is not None}, | ||
"QueryParameters": query_params if query_params else {}, | ||
} | ||
try: | ||
result = self.conn.invoke_rest_api(**api_kwargs) | ||
# ResponseMetadata is removed because it contains data that is either very unlikely to be useful | ||
# in XComs and logs, or redundant given the data already included in the response | ||
result.pop("ResponseMetadata", None) | ||
return result | ||
except ClientError as e: | ||
to_log = e.response | ||
# ResponseMetadata and Error are removed because they contain data that is either very unlikely to | ||
# be useful in XComs and logs, or redundant given the data already included in the response | ||
to_log.pop("ResponseMetadata", None) | ||
to_log.pop("Error", None) | ||
self.log.error(to_log) | ||
raise e |
109 changes: 109 additions & 0 deletions
109
providers/amazon/src/airflow/providers/amazon/aws/operators/mwaa.py
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,109 @@ | ||
# 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. | ||
"""This module contains AWS MWAA operators.""" | ||
|
||
from __future__ import annotations | ||
|
||
from collections.abc import Sequence | ||
from typing import TYPE_CHECKING | ||
|
||
from airflow.providers.amazon.aws.hooks.mwaa import MwaaHook | ||
from airflow.providers.amazon.aws.operators.base_aws import AwsBaseOperator | ||
from airflow.providers.amazon.aws.utils.mixins import aws_template_fields | ||
|
||
if TYPE_CHECKING: | ||
from airflow.utils.context import Context | ||
|
||
|
||
class MwaaTriggerDagRunOperator(AwsBaseOperator[MwaaHook]): | ||
""" | ||
Trigger a Dag Run for a Dag in an Amazon MWAA environment. | ||
.. seealso:: | ||
For more information on how to use this operator, take a look at the guide: | ||
:ref:`howto/operator:MwaaTriggerDagRunOperator` | ||
:param env_name: The MWAA environment name (templated) | ||
:param trigger_dag_id: The ID of the DAG to be triggered (templated) | ||
:param trigger_run_id: The Run ID. This together with trigger_dag_id are a unique key. (templated) | ||
:param logical_date: The logical date (previously called execution date). This is the time or interval | ||
covered by this DAG run, according to the DAG definition. This together with trigger_dag_id are a | ||
unique key. (templated) | ||
:param data_interval_start: The beginning of the interval the DAG run covers | ||
:param data_interval_end: The end of the interval the DAG run covers | ||
:param conf: Additional configuration parameters. The value of this field can be set only when creating | ||
the object. (templated) | ||
:param note: Contains manually entered notes by the user about the DagRun. (templated) | ||
""" | ||
|
||
aws_hook_class = MwaaHook | ||
template_fields: Sequence[str] = aws_template_fields( | ||
"env_name", | ||
"trigger_dag_id", | ||
"trigger_run_id", | ||
"logical_date", | ||
"data_interval_start", | ||
"data_interval_end", | ||
"conf", | ||
"note", | ||
) | ||
template_fields_renderers = {"conf": "json"} | ||
|
||
def __init__( | ||
self, | ||
*, | ||
env_name: str, | ||
trigger_dag_id: str, | ||
trigger_run_id: str | None = None, | ||
logical_date: str | None = None, | ||
data_interval_start: str | None = None, | ||
data_interval_end: str | None = None, | ||
conf: dict | None = None, | ||
note: str | None = None, | ||
**kwargs, | ||
): | ||
super().__init__(**kwargs) | ||
self.env_name = env_name | ||
self.trigger_dag_id = trigger_dag_id | ||
self.trigger_run_id = trigger_run_id | ||
self.logical_date = logical_date | ||
self.data_interval_start = data_interval_start | ||
self.data_interval_end = data_interval_end | ||
self.conf = conf if conf else {} | ||
self.note = note | ||
|
||
def execute(self, context: Context) -> dict: | ||
""" | ||
Trigger a Dag Run for the Dag in the Amazon MWAA environment. | ||
:param context: the Context object | ||
:return: dict with information about the Dag run | ||
For details of the returned dict, see :py:meth:`botocore.client.MWAA.invoke_rest_api` | ||
""" | ||
return self.hook.invoke_rest_api( | ||
env_name=self.env_name, | ||
path=f"/dags/{self.trigger_dag_id}/dagRuns", | ||
method="POST", | ||
body={ | ||
"dag_run_id": self.trigger_run_id, | ||
"logical_date": self.logical_date, | ||
"data_interval_start": self.data_interval_start, | ||
"data_interval_end": self.data_interval_end, | ||
"conf": self.conf, | ||
"note": self.note, | ||
}, | ||
) |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.