From 1182a056a1aa5ff06504bbf9b3adddfafdc90b6c Mon Sep 17 00:00:00 2001 From: Pubudu Fernando Date: Fri, 3 Nov 2023 01:49:43 -0400 Subject: [PATCH 1/4] Add Jira Notifier implementation --- .../atlassian/jira/notifications/__init__.py | 16 ++++ .../atlassian/jira/notifications/jira.py | 83 +++++++++++++++++++ .../providers/atlassian/jira/provider.yaml | 3 + 3 files changed, 102 insertions(+) create mode 100644 airflow/providers/atlassian/jira/notifications/__init__.py create mode 100644 airflow/providers/atlassian/jira/notifications/jira.py diff --git a/airflow/providers/atlassian/jira/notifications/__init__.py b/airflow/providers/atlassian/jira/notifications/__init__.py new file mode 100644 index 0000000000000..13a83393a9124 --- /dev/null +++ b/airflow/providers/atlassian/jira/notifications/__init__.py @@ -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. diff --git a/airflow/providers/atlassian/jira/notifications/jira.py b/airflow/providers/atlassian/jira/notifications/jira.py new file mode 100644 index 0000000000000..74757e9c494ca --- /dev/null +++ b/airflow/providers/atlassian/jira/notifications/jira.py @@ -0,0 +1,83 @@ +# 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 + +from airflow.exceptions import AirflowOptionalProviderFeatureException +from airflow.providers.atlassian.jira.hooks.jira import JiraHook + +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 JiraNotifier(BaseNotifier): + """ + Jira notifier for creating Jira issues upon failures. + + :param jira_conn_id: The HTTP connection ID for the Jira instance. + :param description: The content for the body of the issue + :param summary: The title of the issue + :param project_id: The ID of the project under which the issue will be created + :param issue_type_id: The ID of the issue category + :param labels: The labels to be applied to the issue + """ + + template_fields = ("description", "summary", "project_id", "issue_type_id", "labels") + + def __init__( + self, + *, + jira_conn_id: str = JiraHook.default_conn_name, + proxies: Any | None = None, + description: str, + summary: str, + project_id: int, + issue_type_id: int, + labels: list[str] = [], + ): + super().__init__() + self.jira_conn_id = jira_conn_id + self.proxies = proxies + self.description = description + self.summary = summary + self.project_id = project_id + self.issue_type_id = issue_type_id + self.labels = labels + + @cached_property + def hook(self) -> JiraHook: + return JiraHook(jira_conn_id=self.jira_conn_id, proxies=self.proxies) + + def notify(self, context) -> None: + fields = dict( + description=self.description, + summary=self.summary, + project=dict(id=self.project_id), + issuetype=dict(id=self.issue_type_id), + labels=self.labels, + ) + self.hook.get_conn().create_issue(fields) + + +send_jira_notification = JiraNotifier diff --git a/airflow/providers/atlassian/jira/provider.yaml b/airflow/providers/atlassian/jira/provider.yaml index 997a6166e3098..6d3e1c9cc9f90 100644 --- a/airflow/providers/atlassian/jira/provider.yaml +++ b/airflow/providers/atlassian/jira/provider.yaml @@ -59,3 +59,6 @@ hooks: connection-types: - hook-class-name: airflow.providers.atlassian.jira.hooks.jira.JiraHook connection-type: jira + +notifications: + - airflow.providers.atlassian.jira.notifications.jira.JiraNotifier From 2fcb6ccba17b1c3a54c8af1b4dbe76610dfbe307 Mon Sep 17 00:00:00 2001 From: Pubudu Fernando Date: Sun, 5 Nov 2023 00:54:08 -0400 Subject: [PATCH 2/4] Add tests for Jira notifier --- .../atlassian/jira/notifications/__init__.py | 16 ++++ .../atlassian/jira/notifications/test_jira.py | 94 +++++++++++++++++++ 2 files changed, 110 insertions(+) create mode 100644 tests/providers/atlassian/jira/notifications/__init__.py create mode 100644 tests/providers/atlassian/jira/notifications/test_jira.py diff --git a/tests/providers/atlassian/jira/notifications/__init__.py b/tests/providers/atlassian/jira/notifications/__init__.py new file mode 100644 index 0000000000000..13a83393a9124 --- /dev/null +++ b/tests/providers/atlassian/jira/notifications/__init__.py @@ -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. diff --git a/tests/providers/atlassian/jira/notifications/test_jira.py b/tests/providers/atlassian/jira/notifications/test_jira.py new file mode 100644 index 0000000000000..438d9e228e058 --- /dev/null +++ b/tests/providers/atlassian/jira/notifications/test_jira.py @@ -0,0 +1,94 @@ +# 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 + +import pytest + +from airflow.operators.empty import EmptyOperator +from airflow.providers.atlassian.jira.hooks.jira import JiraHook +from airflow.providers.atlassian.jira.notifications.jira import JiraNotifier, send_jira_notification + +pytestmark = pytest.mark.db_test + +jira_create_issue_payload = dict( + description="Test operator failed", + summary="Test Jira issue", + project=dict(id=10000), + issuetype=dict(id=10003), + labels=["airflow-dag-failure"], +) + + +class TestJiraNotifier: + @mock.patch.object(JiraHook, "get_conn") + def test_jira_notifier(self, mock_jira_hook, dag_maker): + with dag_maker("test_jira_notifier") as dag: + EmptyOperator(task_id="task1") + + notifier = send_jira_notification( + jira_conn_id="jira_default", + project_id=10000, + description="Test operator failed", + summary="Test Jira issue", + issue_type_id=10003, + labels=["airflow-dag-failure"], + ) + notifier({"dag": dag}) + mock_jira_hook.return_value.create_issue.assert_called_once_with(jira_create_issue_payload) + + @mock.patch.object(JiraHook, "get_conn") + def test_jira_notifier_with_notifier_class(self, mock_jira_hook, dag_maker): + with dag_maker("test_jira_notifier") as dag: + EmptyOperator(task_id="task1") + + notifier = JiraNotifier( + jira_conn_id="jira_default", + project_id=10000, + description="Test operator failed", + summary="Test Jira issue", + issue_type_id=10003, + labels=["airflow-dag-failure"], + ) + notifier({"dag": dag}) + mock_jira_hook.return_value.create_issue.assert_called_once_with(jira_create_issue_payload) + + @mock.patch.object(JiraHook, "get_conn") + def test_jira_notifier_templated(self, mock_jira_hook, dag_maker): + with dag_maker("test_jira_notifier") as dag: + EmptyOperator(task_id="task1") + + notifier = send_jira_notification( + jira_conn_id="jira_default", + project_id=10000, + description="Test operator failed for dag: {{ dag.dag_id }}.", + summary="Test Jira issue", + issue_type_id=10003, + labels=["airflow-dag-failure"], + ) + notifier({"dag": dag}) + mock_jira_hook.return_value.create_issue.assert_called_once_with( + dict( + description="Test operator failed for dag: test_jira_notifier.", + summary="Test Jira issue", + project=dict(id=10000), + issuetype=dict(id=10003), + labels=["airflow-dag-failure"], + ) + ) From 09db14e52aa1f8f5b4b5ba6d33f8de1b9a143148 Mon Sep 17 00:00:00 2001 From: Pubudu Fernando Date: Sun, 5 Nov 2023 01:02:43 -0500 Subject: [PATCH 3/4] Add documentation for the Jira notifier --- .../index.rst | 7 ++ .../notifications/index.rst | 27 ++++++++ .../jira-notifier-howto-guide.rst | 65 +++++++++++++++++++ 3 files changed, 99 insertions(+) create mode 100644 docs/apache-airflow-providers-atlassian-jira/notifications/index.rst create mode 100644 docs/apache-airflow-providers-atlassian-jira/notifications/jira-notifier-howto-guide.rst diff --git a/docs/apache-airflow-providers-atlassian-jira/index.rst b/docs/apache-airflow-providers-atlassian-jira/index.rst index b41b6f4e261d8..88779be8b7ae5 100644 --- a/docs/apache-airflow-providers-atlassian-jira/index.rst +++ b/docs/apache-airflow-providers-atlassian-jira/index.rst @@ -29,6 +29,13 @@ Changelog Security +.. toctree:: + :hidden: + :maxdepth: 1 + :caption: Guides + + Notifications + .. toctree:: :hidden: :maxdepth: 1 diff --git a/docs/apache-airflow-providers-atlassian-jira/notifications/index.rst b/docs/apache-airflow-providers-atlassian-jira/notifications/index.rst new file mode 100644 index 0000000000000..56b4034759a03 --- /dev/null +++ b/docs/apache-airflow-providers-atlassian-jira/notifications/index.rst @@ -0,0 +1,27 @@ + + .. 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. + +Atlassian Jira Notifications +============================ + + +.. toctree:: + :maxdepth: 1 + :glob: + + * diff --git a/docs/apache-airflow-providers-atlassian-jira/notifications/jira-notifier-howto-guide.rst b/docs/apache-airflow-providers-atlassian-jira/notifications/jira-notifier-howto-guide.rst new file mode 100644 index 0000000000000..c242896809fd9 --- /dev/null +++ b/docs/apache-airflow-providers-atlassian-jira/notifications/jira-notifier-howto-guide.rst @@ -0,0 +1,65 @@ + + .. 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 Atlassian Jira notifications +============================================= + +Introduction +------------ +The Atlassian Jira notifier (:class:`airflow.providers.atlassian.jira.notifications.jira.JiraNotifier`) allows users to create +issues in a Jira instance using the various ``on_*_callbacks`` available 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.atlassian.jira.notifications.jira import send_jira_notification + + with DAG( + "test-dag", + start_date=datetime(2023, 11, 3), + on_failure_callback=[ + send_jira_notification( + jira_conn_id="my-jira-conn", + description="Failure in the DAG", + summary="Airflow DAG Issue", + project_id=10000, + issue_type_id=10003, + labels=["airflow-dag-failure"], + ) + ], + ): + BashOperator( + task_id="mytask", + on_failure_callback=[ + send_jira_notification( + jira_conn_id="my-jira-conn", + description="The task failed", + summary="Airflow Task Issue", + project_id=10000, + issue_type_id=10003, + labels=["airflow-task-failure"], + ) + ], + bash_command="fail", + retries=0, + ) From d7f6128215373ed3098210e3452b66c46d3483d4 Mon Sep 17 00:00:00 2001 From: Pubudu Fernando Date: Sun, 5 Nov 2023 21:56:01 +0530 Subject: [PATCH 4/4] Apply suggestions from code review Co-authored-by: Andrey Anshin --- airflow/providers/atlassian/jira/notifications/jira.py | 5 +++-- .../notifications/index.rst | 1 + .../notifications/jira-notifier-howto-guide.rst | 4 ++-- 3 files changed, 6 insertions(+), 4 deletions(-) diff --git a/airflow/providers/atlassian/jira/notifications/jira.py b/airflow/providers/atlassian/jira/notifications/jira.py index 74757e9c494ca..b88181e3a8f82 100644 --- a/airflow/providers/atlassian/jira/notifications/jira.py +++ b/airflow/providers/atlassian/jira/notifications/jira.py @@ -36,6 +36,7 @@ class JiraNotifier(BaseNotifier): Jira notifier for creating Jira issues upon failures. :param jira_conn_id: The HTTP connection ID for the Jira instance. + :param proxies: Proxies to make the Jira REST API call. Optional :param description: The content for the body of the issue :param summary: The title of the issue :param project_id: The ID of the project under which the issue will be created @@ -54,7 +55,7 @@ def __init__( summary: str, project_id: int, issue_type_id: int, - labels: list[str] = [], + labels: list[str] | None = None, ): super().__init__() self.jira_conn_id = jira_conn_id @@ -63,7 +64,7 @@ def __init__( self.summary = summary self.project_id = project_id self.issue_type_id = issue_type_id - self.labels = labels + self.labels = labels or [] @cached_property def hook(self) -> JiraHook: diff --git a/docs/apache-airflow-providers-atlassian-jira/notifications/index.rst b/docs/apache-airflow-providers-atlassian-jira/notifications/index.rst index 56b4034759a03..6d05c6a7db51f 100644 --- a/docs/apache-airflow-providers-atlassian-jira/notifications/index.rst +++ b/docs/apache-airflow-providers-atlassian-jira/notifications/index.rst @@ -19,6 +19,7 @@ Atlassian Jira Notifications ============================ +.. important:: This feature is only available in Airflow versions >= 2.6.0 .. toctree:: :maxdepth: 1 diff --git a/docs/apache-airflow-providers-atlassian-jira/notifications/jira-notifier-howto-guide.rst b/docs/apache-airflow-providers-atlassian-jira/notifications/jira-notifier-howto-guide.rst index c242896809fd9..e0ed12558145f 100644 --- a/docs/apache-airflow-providers-atlassian-jira/notifications/jira-notifier-howto-guide.rst +++ b/docs/apache-airflow-providers-atlassian-jira/notifications/jira-notifier-howto-guide.rst @@ -40,7 +40,7 @@ Example Code on_failure_callback=[ send_jira_notification( jira_conn_id="my-jira-conn", - description="Failure in the DAG", + description="Failure in the DAG {{ dag.dag_id }}", summary="Airflow DAG Issue", project_id=10000, issue_type_id=10003, @@ -53,7 +53,7 @@ Example Code on_failure_callback=[ send_jira_notification( jira_conn_id="my-jira-conn", - description="The task failed", + description="The task {{ ti.task_id }} failed", summary="Airflow Task Issue", project_id=10000, issue_type_id=10003,