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/atlassian/jira/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.
84 changes: 84 additions & 0 deletions airflow/providers/atlassian/jira/notifications/jira.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,84 @@
# 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 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
: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] | None = None,
):
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 or []

@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
3 changes: 3 additions & 0 deletions airflow/providers/atlassian/jira/provider.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -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
7 changes: 7 additions & 0 deletions docs/apache-airflow-providers-atlassian-jira/index.rst
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,13 @@
Changelog <changelog>
Security <security>

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

Notifications <notifications/index>

.. toctree::
:hidden:
:maxdepth: 1
Expand Down
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.

Atlassian Jira Notifications
============================

.. important:: This feature is only available in Airflow versions >= 2.6.0

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

*
Original file line number Diff line number Diff line change
@@ -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 {{ dag.dag_id }}",
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(
Copy link
Contributor

Choose a reason for hiding this comment

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

Shouodn't this be JiraNotifier(...) ?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

All the notifiers seem to have followed this pattern. e.g.,

send_chime_notification = ChimeNotifier

Copy link
Contributor

Choose a reason for hiding this comment

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

Yeah, this pattern exists from the very first notifier.

Copy link
Contributor

@eladkal eladkal Nov 10, 2023

Choose a reason for hiding this comment

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

Pattern aside.. this is an example that we publish in docs so we can change it if we wish.

Isn't on_failure_callback=ChimeNotifier(...) Works the same and is more intuitive?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Pattern aside.. this is an example that we publish in docs so we can change it if we wish.

Yes, can use either approach. But then it'll look inconsistent with the existing examples in other notifier docs though. e.g., https://airflow.apache.org/docs/apache-airflow-providers-amazon/8.6.0/notifications/chime_notifier_howto_guide.html

Isn't on_failure_callback=ChimeNotifier(...) Works the same and is more intuitive?

Functionally, exactly the same. Personally for me, having an action verb in the name reads better in this context though. Like, on failure, send this notification.

jira_conn_id="my-jira-conn",
description="The task {{ ti.task_id }} failed",
summary="Airflow Task Issue",
project_id=10000,
issue_type_id=10003,
labels=["airflow-task-failure"],
)
],
bash_command="fail",
retries=0,
)
16 changes: 16 additions & 0 deletions tests/providers/atlassian/jira/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.
94 changes: 94 additions & 0 deletions tests/providers/atlassian/jira/notifications/test_jira.py
Original file line number Diff line number Diff line change
@@ -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"],
)
)