Skip to content

Commit

Permalink
Fixed SlackAPIFileOperator to upload file and file content. (#17400)
Browse files Browse the repository at this point in the history
* [14168] 

* [14168] Fixed Lint errors

* [14168] Updated documentation for Slack API File Operator

* [14168] Fixed indendation in SlackAPIFileOperator docs

* Update airflow/providers/slack/operators/slack.py

Fixed indentation.

Co-authored-by: Tzu-ping Chung <[email protected]>

* Fixed docs comment in Slack.py

* Fixed flake8

* Fixed bug with SlackAPIFileOperator with uploading file.

* Added example DAG link for slack

* Fix doc error

* Fixed example DAG for slack and the comments

* Updated slack operator tests

* Fixed slack file tests

* Fixed tests

* Fixed PR review comments

* Fixed file variable in example_slack.py

Co-authored-by: Tzu-ping Chung <[email protected]>
  • Loading branch information
subkanthi and uranusjr authored Aug 16, 2021
1 parent c645d7a commit 2935be1
Show file tree
Hide file tree
Showing 6 changed files with 107 additions and 32 deletions.
16 changes: 16 additions & 0 deletions airflow/providers/slack/example_dags/__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.
50 changes: 50 additions & 0 deletions airflow/providers/slack/example_dags/example_slack.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,50 @@
# 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 airflow.models.dag import DAG
from airflow.providers.slack.operators.slack import SlackAPIFileOperator
from airflow.utils.dates import days_ago

with DAG(
dag_id='slack_example_dag',
schedule_interval=None,
start_date=days_ago(2),
max_active_runs=1,
tags=['example'],
) as dag:
# Send file with filename and filetype
slack_operator_file = SlackAPIFileOperator(
task_id="slack_file_upload_1",
dag=dag,
slack_conn_id="slack",
channel="#general",
initial_comment="Hello World!",
filename="/files/dags/test.txt",
filetype="txt",
)

# Send file content
slack_operator_file_content = SlackAPIFileOperator(
task_id="slack_file_upload_2",
dag=dag,
slack_conn_id="slack",
channel="#general",
initial_comment="Hello World!",
content="file content in txt",
)

slack_operator_file >> slack_operator_file_content
47 changes: 24 additions & 23 deletions airflow/providers/slack/operators/slack.py
Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,6 @@
# KIND, either express or implied. See the License for the
# specific language governing permissions and limitations
# under the License.

import json
from typing import Any, Dict, List, Optional

Expand Down Expand Up @@ -162,15 +161,16 @@ class SlackAPIFileOperator(SlackAPIOperator):
.. code-block:: python
# Send file with filename and filetype
slack = SlackAPIFileOperator(
task_id="slack_file_upload",
dag=dag,
slack_conn_id="slack",
channel="#general",
initial_comment="Hello World!",
filename="hello_world.csv",
filetype="csv",
)
with open("test.txt", "rb") as file:
slack = SlackAPIFileOperator(
task_id="slack_file_upload",
dag=dag,
slack_conn_id="slack",
channel="#general",
initial_comment="Hello World!",
file="/files/dags/test.txt",
filetype="txt",
)
# Send file content
slack = SlackAPIFileOperator(
Expand All @@ -195,7 +195,7 @@ class SlackAPIFileOperator(SlackAPIOperator):
:type content: str
"""

template_fields = ('channel', 'initial_comment', 'filename', 'filetype', 'content')
template_fields = ('channel', 'initial_comment', 'filetype', 'content')
ui_color = '#44BEDF'

def __init__(
Expand All @@ -216,28 +216,29 @@ def __init__(
self.file_params = {}
super().__init__(method=self.method, **kwargs)

def construct_api_call_params(self) -> Any:
def execute(self, **kwargs):
"""
The SlackAPIOperator calls will not fail even if the call is not unsuccessful.
It should not prevent a DAG from completing in success
"""
slack = SlackHook(token=self.token, slack_conn_id=self.slack_conn_id)

# If file content is passed.
if self.content is not None:
self.api_params = {
'channels': self.channel,
'content': self.content,
'initial_comment': self.initial_comment,
}
slack.call(self.method, data=self.api_params)
# If file name is passed.
elif self.filename is not None:
self.api_params = {
'channels': self.channel,
'filename': self.filename,
'filetype': self.filetype,
'initial_comment': self.initial_comment,
}
self.file_params = {'file': self.filename}

def execute(self, **kwargs):
"""
The SlackAPIOperator calls will not fail even if the call is not unsuccessful.
It should not prevent a DAG from completing in success
"""
if not self.api_params:
self.construct_api_call_params()
slack = SlackHook(token=self.token, slack_conn_id=self.slack_conn_id)
slack.call(self.method, data=self.api_params, files=self.file_params)
with open(self.filename, "rb") as file_handle:
slack.call(self.method, data=self.api_params, files={'file': file_handle})
file_handle.close()
1 change: 1 addition & 0 deletions docs/apache-airflow-providers-slack/index.rst
Original file line number Diff line number Diff line change
Expand Up @@ -28,6 +28,7 @@ Content

Connection Types <connections/slack>
Python API <_api/airflow/providers/slack/index>
Example DAGs <https://github.com/apache/airflow/tree/main/airflow/providers/slack/example_dags>

.. toctree::
:maxdepth: 1
Expand Down
1 change: 1 addition & 0 deletions tests/providers/slack/operators/test.csv
Original file line number Diff line number Diff line change
@@ -0,0 +1 @@
test
24 changes: 15 additions & 9 deletions tests/providers/slack/operators/test_slack.py
Original file line number Diff line number Diff line change
Expand Up @@ -140,7 +140,7 @@ def setUp(self):
self.test_username = 'test_username'
self.test_channel = '#test_slack_channel'
self.test_initial_comment = 'test text file test_filename.txt'
self.test_filename = 'test_filename.txt'
self.filename = 'test_filename.txt'
self.test_filetype = 'text'
self.test_content = 'This is a test text file!'

Expand All @@ -150,7 +150,7 @@ def setUp(self):
self.expected_api_params = {
'channel': self.test_channel,
'initial_comment': self.test_initial_comment,
'filename': self.test_filename,
'file': self.filename,
'filetype': self.test_filetype,
'content': self.test_content,
}
Expand All @@ -162,7 +162,7 @@ def __construct_operator(self, test_token, test_slack_conn_id, test_api_params=N
slack_conn_id=test_slack_conn_id,
channel=self.test_channel,
initial_comment=self.test_initial_comment,
filename=self.test_filename,
filename=self.filename,
filetype=self.test_filetype,
content=self.test_content,
api_params=test_api_params,
Expand All @@ -179,7 +179,7 @@ def test_init_with_valid_params(self):
assert slack_api_post_operator.initial_comment == self.test_initial_comment
assert slack_api_post_operator.channel == self.test_channel
assert slack_api_post_operator.api_params == self.test_api_params
assert slack_api_post_operator.filename == self.test_filename
assert slack_api_post_operator.filename == self.filename
assert slack_api_post_operator.filetype == self.test_filetype
assert slack_api_post_operator.content == self.test_content

Expand Down Expand Up @@ -208,19 +208,25 @@ def test_api_call_params_with_content_args(self, mock_hook):
def test_api_call_params_with_file_args(self, mock_hook):
test_slack_conn_id = 'test_slack_conn_id'

import os

# Look for your absolute directory path
absolute_path = os.path.dirname(os.path.abspath(__file__))
# Or: file_path = os.path.join(absolute_path, 'folder', 'my_file.py')
file_path = absolute_path + '/test.csv'

print(f"full path ${file_path}")

slack_api_post_operator = SlackAPIFileOperator(
task_id='slack', slack_conn_id=test_slack_conn_id, filename='test.csv', filetype='csv'
task_id='slack', slack_conn_id=test_slack_conn_id, filename=file_path, filetype='csv'
)

slack_api_post_operator.execute()

expected_api_params = {
'channels': '#general',
'initial_comment': 'No message has been set!',
'filename': 'test.csv',
'filename': file_path,
'filetype': 'csv',
}

expected_file_params = {'file': 'test.csv'}
assert expected_api_params == slack_api_post_operator.api_params
assert expected_file_params == slack_api_post_operator.file_params

0 comments on commit 2935be1

Please sign in to comment.