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/smtp/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.
106 changes: 106 additions & 0 deletions airflow/providers/smtp/notifications/smtp.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,106 @@
# 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 functools import cached_property
from typing import Any, Iterable

from airflow.exceptions import AirflowOptionalProviderFeatureException

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"
)

from airflow.providers.smtp.hooks.smtp import SmtpHook


class SmtpNotifier(BaseNotifier):
"""
SMTP Notifier

:param smtp_conn_id: The :ref:`smtp connection id <howto/connection:smtp>`
that contains the information used to authenticate the client.
"""

template_fields = (
"from_email",
"to",
"subject",
"html_content",
"files",
"cc",
"bcc",
"mime_subtype",
"mime_charset",
"custom_headers",
)

def __init__(
self,
from_email: str | None,
to: str | Iterable[str],
subject: str,
html_content: str,
files: list[str] | None = None,
cc: str | Iterable[str] | None = None,
bcc: str | Iterable[str] | None = None,
mime_subtype: str = "mixed",
mime_charset: str = "utf-8",
custom_headers: dict[str, Any] | None = None,
smtp_conn_id: str = SmtpHook.default_conn_name,
):
super().__init__()
self.smtp_conn_id = smtp_conn_id
self.from_email = from_email
self.to = to
self.subject = subject
self.html_content = html_content
self.files = files
self.cc = cc
self.bcc = bcc
self.mime_subtype = mime_subtype
self.mime_charset = mime_charset
self.custom_headers = custom_headers

@cached_property
def hook(self) -> SmtpHook:
"""Smtp Events Hook"""
return SmtpHook(smtp_conn_id=self.smtp_conn_id)

def notify(self, context):
"""Send a email via smtp server"""
with self.hook as smtp:
smtp.send_email_smtp(
smtp_conn_id=self.smtp_conn_id,
from_email=self.from_email,
to=self.to,
subject=self.subject,
html_content=self.html_content,
files=self.files,
cc=self.cc,
bcc=self.bcc,
mime_subtype=self.mime_subtype,
mime_charset=self.mime_charset,
custom_headers=self.custom_headers,
)


send_smtp_notification = SmtpNotifier
3 changes: 3 additions & 0 deletions airflow/providers/smtp/provider.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -50,3 +50,6 @@ hooks:
connection-types:
- hook-class-name: airflow.providers.smtp.hooks.smtp.SmtpHook
connection-type: smtp

notifications:
- airflow.providers.smtp.notifications.smtp.SmtpNotifier
7 changes: 7 additions & 0 deletions docs/apache-airflow-providers-smtp/index.rst
Original file line number Diff line number Diff line change
Expand Up @@ -46,6 +46,13 @@ Content
Detailed list of commits <commits>


.. toctree::
:maxdepth: 1
:caption: Guides

SMTP Notifications <notifications/smtp_notifier_howto_guide>


Package apache-airflow-providers-smtp
------------------------------------------------------

Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,61 @@
.. 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 SMTP notifications
===================================

Introduction
------------
The SMTP notifier (:class:`airflow.providers.smtp.notifications.smtp.SmtpNotifier`) allows users to send
messages to SMTP servers using the various ``on_*_callbacks`` at both the DAG level and Task level.

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

.. code-block:: python

from datetime import datetime
from airflow import DAG
from airflow.operators.bash import BashOperator
from airflow.providers.smtp.notifications.smtp import send_smtp_notification

with DAG(
dag_id="smtp_notifier",
schedule_interval=None,
start_date=datetime(2023, 1, 1),
catchup=False,
on_failure_callback=[
send_smtp_notification(
from_email="someone@mail.com",
to="someone@mail.com",
subject="[Error] The dag {{ dag.dag_id }} failed",
html_content="debug logs",
)
],
):
BashOperator(
task_id="mytask",
on_failure_callback=[
send_smtp_notification(
from_email="someone@mail.com",
to="someone@mail.com",
subject="[Error] The Task {{ ti.task_id }} failed",
html_content="debug logs",
)
],
bash_command="fail",
)
16 changes: 16 additions & 0 deletions tests/providers/smtp/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.
108 changes: 108 additions & 0 deletions tests/providers/smtp/notifications/test_smtp.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,108 @@
# 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.operators.empty import EmptyOperator
from airflow.providers.smtp.hooks.smtp import SmtpHook
from airflow.providers.smtp.notifications.smtp import (
SmtpNotifier,
send_smtp_notification,
)

SMTP_API_DEFAULT_CONN_ID = SmtpHook.default_conn_name


class TestPagerdutyNotifier:
@mock.patch("airflow.providers.smtp.notifications.smtp.SmtpHook")
def test_notifier(self, mock_smtphook_hook, dag_maker):
with dag_maker("test_notifier") as dag:
EmptyOperator(task_id="task1")
notifier = send_smtp_notification(
from_email="test_sender@test.com",
to="test_reciver@test.com",
subject="subject",
html_content="body",
)
notifier(context={"dag": dag})
mock_smtphook_hook.return_value.__enter__().send_email_smtp.assert_called_once_with(
from_email="test_sender@test.com",
to="test_reciver@test.com",
subject="subject",
html_content="body",
smtp_conn_id="smtp_default",
files=None,
cc=None,
bcc=None,
mime_subtype="mixed",
mime_charset="utf-8",
custom_headers=None,
)

@mock.patch("airflow.providers.smtp.notifications.smtp.SmtpHook")
def test_notifier_with_notifier_class(self, mock_smtphook_hook, dag_maker):
with dag_maker("test_notifier") as dag:
EmptyOperator(task_id="task1")
notifier = SmtpNotifier(
from_email="test_sender@test.com",
to="test_reciver@test.com",
subject="subject",
html_content="body",
)
notifier(context={"dag": dag})
mock_smtphook_hook.return_value.__enter__().send_email_smtp.assert_called_once_with(
from_email="test_sender@test.com",
to="test_reciver@test.com",
subject="subject",
html_content="body",
smtp_conn_id="smtp_default",
files=None,
cc=None,
bcc=None,
mime_subtype="mixed",
mime_charset="utf-8",
custom_headers=None,
)

@mock.patch("airflow.providers.smtp.notifications.smtp.SmtpHook")
def test_notifier_templated(self, mock_smtphook_hook, dag_maker):
with dag_maker("test_notifier") as dag:
EmptyOperator(task_id="task1")

notifier = SmtpNotifier(
from_email="test_sender@test.com {{dag.dag_id}}",
to="test_reciver@test.com {{dag.dag_id}}",
subject="subject {{dag.dag_id}}",
html_content="body {{dag.dag_id}}",
)
context = {"dag": dag}
notifier(context)
mock_smtphook_hook.return_value.__enter__().send_email_smtp.assert_called_once_with(
from_email="test_sender@test.com test_notifier",
to="test_reciver@test.com test_notifier",
subject="subject test_notifier",
html_content="body test_notifier",
smtp_conn_id="smtp_default",
files=None,
cc=None,
bcc=None,
mime_subtype="mixed",
mime_charset="utf-8",
custom_headers=None,
)