Skip to content
Merged
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
16 changes: 16 additions & 0 deletions airflow/providers/amazon/aws/notifications/__init__.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,16 @@
# 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.
61 changes: 61 additions & 0 deletions airflow/providers/amazon/aws/notifications/chime.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,61 @@
# Licensed to the Apache Software Foundation (ASF) under one
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Should we name the directory notifiers instead? To be consistent with operators, hooks, ...

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Notifications was consistent with how other providers have named such as Slack and Discord. But I am open to renaming if there is consensus.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Lets stay with the current convention for now.
Notifiers is a new experience and I prefer to gather feedback before we start deprecating.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Sounds good

# 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 functools import cached_property

from airflow.exceptions import AirflowOptionalProviderFeatureException
from airflow.providers.amazon.aws.hooks.chime import ChimeWebhookHook
from airflow.utils.context import Context

try:
from airflow.notifications.basenotifier import BaseNotifier
except ImportError:
raise AirflowOptionalProviderFeatureException(
"Failed to import BaseNotifier. This feature is only available in Airflow versions >= 2.6.0"
)


class ChimeNotifier(BaseNotifier):
"""
Chime notifier to send messages to a chime room via callbacks.

:param: chime_conn_id: The chime connection to use with Endpoint as "https://hooks.chime.aws" and
the webhook token in the form of ```{webhook.id}?token{webhook.token}```
:param: message: The message to send to the chime room associated with the webhook.

"""

template_fields = ("message",)

def __init__(self, *, chime_conn_id: str, message: str = "This is the default chime notifier message"):
super().__init__()
self.chime_conn_id = chime_conn_id
self.message = message

@cached_property
def hook(self):
"""To reduce overhead cache the hook for the notifier."""
return ChimeWebhookHook(chime_conn_id=self.chime_conn_id)

def notify(self, context: Context) -> None:
"""Send a message to a Chime Chat Room."""
self.hook.send_message(message=self.message)


send_chime_notification = ChimeNotifier
3 changes: 3 additions & 0 deletions airflow/providers/amazon/provider.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -642,6 +642,9 @@ connection-types:
- hook-class-name: airflow.providers.amazon.aws.hooks.redshift_sql.RedshiftSQLHook
connection-type: redshift

notifications:
- airflow.providers.amazon.aws.notifications.chime.ChimeNotifier


secrets-backends:
- airflow.providers.amazon.aws.secrets.secrets_manager.SecretsManagerBackend
Expand Down
1 change: 1 addition & 0 deletions docs/apache-airflow-providers-amazon/index.rst
Original file line number Diff line number Diff line change
Expand Up @@ -34,6 +34,7 @@
:caption: Guides

Connection types <connections/index>
Notifications <notifications/index>
Operators <operators/index>
Transfers <transfer/index>
Deferrable Operators <deferrable>
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,53 @@
.. 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.

How-to Guide for Chime notifications
====================================

Introduction
------------
Chime notifier (:class:`airflow.providers.amazon.aws.notifications.chime.ChimeNotifier`) allows users to send
messages to a Chime chat room setup via a webhook using the various ``on_*_callbacks`` at both the DAG level and Task level

You can also use a notifier with ``sla_miss_callback``.

.. note::
When notifiers are used with `sla_miss_callback` the context will contain only values passed to the callback, refer :ref:`sla_miss_callback<concepts:sla_miss_callback>`.

Example Code:
-------------

.. code-block:: python

from datetime import datetime
from airflow import DAG
from airflow.operators.bash import BashOperator
from airflow.providers.amazon.aws.notifications.chime import send_chime_notification

with DAG(
start_date=datetime(2023, 6, 27),
on_success_callback=[
send_chime_notification(chime_conn_id="my_chime_conn", message="The DAG {{ dag.dag_id }} succeeded")
],
):
BashOperator(
task_id="mytask",
on_failure_callback=[
send_chime_notification(chime_conn_id="my_chime_conn", message="The task {{ ti.task_id }} failed")
],
bash_command="fail",
)
28 changes: 28 additions & 0 deletions docs/apache-airflow-providers-amazon/notifications/index.rst
Original file line number Diff line number Diff line change
@@ -0,0 +1,28 @@
.. 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 AWS Notifications
========================


.. toctree::
:maxdepth: 1
:glob:

*
16 changes: 16 additions & 0 deletions tests/providers/amazon/aws/notifications/__init__.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,16 @@
# 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.
73 changes: 73 additions & 0 deletions tests/providers/amazon/aws/notifications/test_chime.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,73 @@
# 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 import mock

from airflow.models import Connection
from airflow.operators.empty import EmptyOperator
from airflow.providers.amazon.aws.hooks.chime import ChimeWebhookHook
from airflow.providers.amazon.aws.notifications.chime import ChimeNotifier, send_chime_notification
from airflow.utils import db


class TestChimeNotifier:
# Chime webhooks can't really have a default connection, so we need to create one for tests.
def setup_method(self):
db.merge_conn(
Connection(
conn_id="default-chime-webhook",
conn_type="chime",
host="hooks.chime.aws/incomingwebhooks/",
password="abcd-1134-ZeDA?token=somechimetoken111",
schema="https",
)
)

@mock.patch.object(ChimeWebhookHook, "send_message")
def test_chime_notifier(self, mock_chime_hook, dag_maker):
with dag_maker("test_chime_notifier") as dag:
EmptyOperator(task_id="task1")

notifier = send_chime_notification(
chime_conn_id="default-chime-webhook", message="Chime Test Message"
)
notifier({"dag": dag})
mock_chime_hook.assert_called_once_with(message="Chime Test Message")

@mock.patch.object(ChimeWebhookHook, "send_message")
def test_chime_notifier_with_notifier_class(self, mock_chime_hook, dag_maker):
with dag_maker("test_chime_notifier") as dag:
EmptyOperator(task_id="task1")

notifier = ChimeNotifier(
chime_conn_id="default-chime-webhook", message="Test Chime Message for Class"
)
notifier({"dag": dag})
mock_chime_hook.assert_called_once_with(message="Test Chime Message for Class")

@mock.patch.object(ChimeWebhookHook, "send_message")
def test_chime_notifier_templated(self, mock_chime_hook, dag_maker):
with dag_maker("test_chime_notifier") as dag:
EmptyOperator(task_id="task1")

notifier = send_chime_notification(
chime_conn_id="default-chime-webhook", message="Test Chime Message. Dag is {{ dag.dag_id }}."
)
notifier({"dag": dag})
mock_chime_hook.assert_called_once_with(message="Test Chime Message. Dag is test_chime_notifier.")