Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
Show all changes
34 commits
Select commit Hold shift + click to select a range
f95b641
Create ChimeWebhook Hook to send messages to chime room from Airflow.
cjames23 Jun 15, 2023
b7ca461
Fix method name/Add connection docs
cjames23 Jun 15, 2023
0e66783
Fix chime conn doc formatting
cjames23 Jun 15, 2023
af36615
Merge branch 'apache:main' into chime-webhook-feature
cjames23 Jun 16, 2023
6cb89ed
Fix `BIGQUERY_JOB_DETAILS_LINK_FMT` in `BigQueryConsoleLink` (#31457)
bkossakowska Jun 15, 2023
34fe3b5
Revert "Fix `BIGQUERY_JOB_DETAILS_LINK_FMT` in `BigQueryConsoleLink` …
potiuk Jun 15, 2023
7ac8ebe
Fix Chime docs, add missing deps
cjames23 Jun 16, 2023
71fcf70
Handle missing chime extras
cjames23 Jun 16, 2023
c78c959
Merge branch 'apache:main' into chime-webhook-feature
cjames23 Jun 16, 2023
f904677
Merge branch 'main' into chime-webhook-feature
cjames23 Jun 16, 2023
e19b725
Add test, change endpoint to check none
cjames23 Jun 16, 2023
2c7e1db
Use custom fields for Chime conn
cjames23 Jun 19, 2023
e1a0209
Adjust method types
cjames23 Jun 19, 2023
50f334a
Fix implicit str concat
cjames23 Jun 20, 2023
d9d79f9
Fix Chime Conn docs
cjames23 Jun 20, 2023
8f30133
Merge branch 'main' into chime-webhook-feature
cjames23 Jun 20, 2023
b0786de
Fix Chime Hook tests
cjames23 Jun 23, 2023
0cdd5bc
Fix Chime Hook tests
cjames23 Jun 23, 2023
8309343
Merge branch 'main' into chime-webhook-feature
cjames23 Jun 23, 2023
f6bd762
Fix Chime Docs
cjames23 Jun 23, 2023
25da11c
Add chime logo
cjames23 Jun 23, 2023
0f56b90
Merge branch 'main' into chime-webhook-feature
cjames23 Jun 23, 2023
6bad327
Merge branch 'main' into chime-webhook-feature
cjames23 Jun 23, 2023
73d4f37
Merge branch 'apache:main' into chime-webhook-feature
cjames23 Jun 26, 2023
6adaaad
Add http to selective tests for amazon.
cjames23 Jun 26, 2023
f2c564e
Merge branch 'main' into chime-webhook-feature
cjames23 Jun 26, 2023
d75fa82
Fix amazon selective checks
cjames23 Jun 26, 2023
595dc04
Add amzn to affected providers http.
cjames23 Jun 26, 2023
5bd1d87
Merge branch 'main' into chime-webhook-feature
cjames23 Jun 26, 2023
3fdaa4d
Fix missing comma in selective checks
cjames23 Jun 26, 2023
0fa4525
Selective checks http change run-amazon-tests to true expected
cjames23 Jun 26, 2023
1f06a95
Fix Selective Checks for Amazon http dependency
cjames23 Jun 26, 2023
8a5bc35
Fix Selective Checks for Amazon http dep
cjames23 Jun 26, 2023
e874385
Merge branch 'main' into chime-webhook-feature
cjames23 Jun 27, 2023
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
116 changes: 116 additions & 0 deletions airflow/providers/amazon/aws/hooks/chime.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,116 @@
#
# 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 a web hook for Chime."""
from __future__ import annotations

import json
import re
from typing import Any

from airflow.exceptions import AirflowException
from airflow.providers.http.hooks.http import HttpHook


class ChimeWebhookHook(HttpHook):
"""Interact with Chime Web Hooks to create notifications.

.. warning:: This hook is only designed to work with web hooks and not chat bots.

:param chime_conn_id: Chime connection ID with Endpoint as "https://hooks.chime.aws" and
the webhook token in the form of ```{webhook.id}?token{webhook.token}```

"""

conn_name_attr = "chime_conn_id"
default_conn_name = "chime_default"
conn_type = "chime"
hook_name = "Chime Web Hook"

def __init__(
self,
chime_conn_id: str,
*args: Any,
**kwargs: Any,
) -> None:
super().__init__(*args, **kwargs)
self.webhook_endpoint = self._get_webhook_endpoint(chime_conn_id)

def _get_webhook_endpoint(self, conn_id: str) -> str:
"""
Given a Chime conn_id return the default webhook endpoint.

:param conn_id: The provided connection ID.
:return: Endpoint(str) for chime webhook.
"""
conn = self.get_connection(conn_id)
token = conn.get_password()
if token is None:
raise AirflowException("Webhook token field is missing and is required.")
url = conn.schema + "://" + conn.host
endpoint = url + token
# Check to make sure the endpoint matches what Chime expects
if not re.match(r"^[a-zA-Z0-9_-]+\?token=[a-zA-Z0-9_-]+$", token):
raise AirflowException(
"Expected Chime webhook token in the form of '{webhook.id}?token={webhook.token}'."
)

return endpoint

def _build_chime_payload(self, message: str) -> str:
"""
Builds payload for Chime and ensures messages do not exceed max length allowed.

:param message: The message you want to send to your Chime room.
(max 4096 characters)
"""
payload: dict[str, Any] = {}
# We need to make sure that the message does not exceed the max length for Chime
if len(message) > 4096:
raise AirflowException("Chime message must be 4096 characters or less.")

payload["Content"] = message
return json.dumps(payload)

def send_message(self, message: str) -> None:
"""Execute calling the Chime webhook endpoint.

:param message: The message you want to send to your Chime room.
(max 4096 characters)

"""
chime_payload = self._build_chime_payload(message)
self.run(
endpoint=self.webhook_endpoint, data=chime_payload, headers={"Content-type": "application/json"}
)

@classmethod
def get_ui_field_behaviour(cls) -> dict[str, Any]:
"""Returns custom field behaviour to only get what is needed for Chime webhooks to function."""
return {
"hidden_fields": ["login", "port", "extra"],
"relabeling": {
"host": "Chime Webhook Endpoint",
"password": "Webhook Token",
},
"placeholders": {
"schema": "https",
"host": "hooks.chime.aws/incomingwebhook/",
"password": "T00000000?token=XXXXXXXXXXXXXXXXXXXXXXXX",
},
}
11 changes: 11 additions & 0 deletions airflow/providers/amazon/provider.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -61,6 +61,7 @@ versions:
dependencies:
- apache-airflow>=2.4.0
- apache-airflow-providers-common-sql>=1.3.1
- apache-airflow-providers-http
- boto3>=1.24.0
- asgiref
# watchtower 3 has been released end Jan and introduced breaking change across the board that might
Expand All @@ -84,6 +85,10 @@ integrations:
how-to-guide:
- /docs/apache-airflow-providers-amazon/operators/athena.rst
tags: [aws]
- integration-name: Amazon Chime
external-doc-url: https://aws.amazon.com/chime/
logo: /integration-logos/aws/Amazon-Chime-light-bg.png
tags: [aws]
- integration-name: Amazon CloudFormation
external-doc-url: https://aws.amazon.com/cloudformation/
logo: /integration-logos/aws/[email protected]
Expand Down Expand Up @@ -407,6 +412,9 @@ hooks:
- integration-name: Amazon Athena
python-modules:
- airflow.providers.amazon.aws.hooks.athena
- integration-name: Amazon Chime
python-modules:
- airflow.providers.amazon.aws.hooks.chime
- integration-name: Amazon DynamoDB
python-modules:
- airflow.providers.amazon.aws.hooks.dynamodb
Expand Down Expand Up @@ -624,11 +632,14 @@ extra-links:
connection-types:
- hook-class-name: airflow.providers.amazon.aws.hooks.base_aws.AwsGenericHook
connection-type: aws
- hook-class-name: airflow.providers.amazon.aws.hooks.chime.ChimeWebhookHook
connection-type: chime
- hook-class-name: airflow.providers.amazon.aws.hooks.emr.EmrHook
connection-type: emr
- hook-class-name: airflow.providers.amazon.aws.hooks.redshift_sql.RedshiftSQLHook
connection-type: redshift


secrets-backends:
- airflow.providers.amazon.aws.secrets.secrets_manager.SecretsManagerBackend
- airflow.providers.amazon.aws.secrets.systems_manager.SystemsManagerParameterStoreBackend
Expand Down
14 changes: 7 additions & 7 deletions dev/breeze/tests/test_selective_checks.py
Original file line number Diff line number Diff line change
Expand Up @@ -191,7 +191,7 @@ def assert_outputs_are_printed(expected_outputs: dict[str, str], stderr: str):
"tests/providers/http/file.py",
),
{
"affected-providers-list-as-string": "airbyte apache.livy "
"affected-providers-list-as-string": "airbyte amazon apache.livy "
"dbt.cloud dingding discord http",
"all-python-versions": "['3.8']",
"all-python-versions-list-as-string": "3.8",
Expand All @@ -200,11 +200,11 @@ def assert_outputs_are_printed(expected_outputs: dict[str, str], stderr: str):
"image-build": "true",
"needs-helm-tests": "true",
"run-tests": "true",
"run-amazon-tests": "false",
"run-amazon-tests": "true",
"docs-build": "true",
"run-kubernetes-tests": "true",
"upgrade-to-newer-dependencies": "false",
"parallel-test-types-list-as-string": "Always "
"parallel-test-types-list-as-string": "Providers[amazon] Always "
"Providers[airbyte,apache.livy,dbt.cloud,dingding,discord,http]",
},
id="Helm tests, http and all relevant providers, kubernetes tests and "
Expand Down Expand Up @@ -311,7 +311,7 @@ def assert_outputs_are_printed(expected_outputs: dict[str, str], stderr: str):
("airflow/providers/amazon/__init__.py",),
{
"affected-providers-list-as-string": "amazon apache.hive cncf.kubernetes "
"common.sql exasol ftp google imap "
"common.sql exasol ftp google http imap "
"mongo mysql postgres salesforce ssh",
"all-python-versions": "['3.8']",
"all-python-versions-list-as-string": "3.8",
Expand All @@ -325,7 +325,7 @@ def assert_outputs_are_printed(expected_outputs: dict[str, str], stderr: str):
"upgrade-to-newer-dependencies": "false",
"run-amazon-tests": "true",
"parallel-test-types-list-as-string": "Providers[amazon] Always "
"Providers[apache.hive,cncf.kubernetes,common.sql,exasol,ftp,imap,"
"Providers[apache.hive,cncf.kubernetes,common.sql,exasol,ftp,http,imap,"
"mongo,mysql,postgres,salesforce,ssh] Providers[google]",
},
id="Providers tests run including amazon tests if amazon provider files changed",
Expand Down Expand Up @@ -353,7 +353,7 @@ def assert_outputs_are_printed(expected_outputs: dict[str, str], stderr: str):
("airflow/providers/amazon/file.py",),
{
"affected-providers-list-as-string": "amazon apache.hive cncf.kubernetes "
"common.sql exasol ftp google imap "
"common.sql exasol ftp google http imap "
"mongo mysql postgres salesforce ssh",
"all-python-versions": "['3.8']",
"all-python-versions-list-as-string": "3.8",
Expand All @@ -368,7 +368,7 @@ def assert_outputs_are_printed(expected_outputs: dict[str, str], stderr: str):
"upgrade-to-newer-dependencies": "false",
"parallel-test-types-list-as-string": "Providers[amazon] Always "
"Providers[apache.hive,cncf.kubernetes,common.sql,exasol,ftp,"
"imap,mongo,mysql,postgres,salesforce,ssh] Providers[google]",
"http,imap,mongo,mysql,postgres,salesforce,ssh] Providers[google]",
},
id="Providers tests run including amazon tests if amazon provider files changed",
),
Expand Down
60 changes: 60 additions & 0 deletions docs/apache-airflow-providers-amazon/connections/chime.rst
Original file line number Diff line number Diff line change
@@ -0,0 +1,60 @@
.. 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.

.. _howto/connection:chime:

Amazon Chime Connection
==========================

The Chime connection works with calling Chime webhooks to send messages to a chime room.

Authenticating to Amazon Chime
---------------------------------
When a webhook is created in a Chime room a token will be included in the url for authentication.


Default Connection IDs
----------------------

The default connection ID is ``chime_default``.

Configuring the Connection
--------------------------
Chime Webhook Endpoint:
Specify the entire url or the base of the url for the service.


Chime Webhook token:
The token for authentication including the webhook ID.

Schema:
Whether or not the endpoint should be http or https


Examples
--------

**Connection**

* **Chime Webhook Endpoint**: hooks.chime.aws
* **Chime Webhook Token**:

.. code-block:: text

abceasd-3423-a1237-ffff-000cccccccc?token=somechimetoken

* **Schema**: https
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 @@ -134,6 +134,7 @@ Dependent package
`apache-airflow-providers-exasol <https://airflow.apache.org/docs/apache-airflow-providers-exasol>`_ ``exasol``
`apache-airflow-providers-ftp <https://airflow.apache.org/docs/apache-airflow-providers-ftp>`_ ``ftp``
`apache-airflow-providers-google <https://airflow.apache.org/docs/apache-airflow-providers-google>`_ ``google``
`apache-airflow-providers-http <https://airflow.apache.org/docs/apache-airflow-providers-http>`_ ``http``
`apache-airflow-providers-imap <https://airflow.apache.org/docs/apache-airflow-providers-imap>`_ ``imap``
`apache-airflow-providers-mongo <https://airflow.apache.org/docs/apache-airflow-providers-mongo>`_ ``mongo``
`apache-airflow-providers-salesforce <https://airflow.apache.org/docs/apache-airflow-providers-salesforce>`_ ``salesforce``
Expand Down
Loading
Sorry, something went wrong. Reload?
Sorry, we cannot display this file.
Sorry, this file is invalid so it cannot be displayed.
2 changes: 2 additions & 0 deletions generated/provider_dependencies.json
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@
"amazon": {
"deps": [
"apache-airflow-providers-common-sql>=1.3.1",
"apache-airflow-providers-http",
"apache-airflow>=2.4.0",
"asgiref",
"asgiref",
Expand All @@ -40,6 +41,7 @@
"exasol",
"ftp",
"google",
"http",
"imap",
"mongo",
"salesforce",
Expand Down
Loading