You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@arrow.apache.org by ks...@apache.org on 2022/05/04 09:29:59 UTC

[arrow] branch master updated: ARROW-16357: [Archery][Dev] Add possibility to send nightly reports to Zulip/Slack

This is an automated email from the ASF dual-hosted git repository.

kszucs pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/arrow.git


The following commit(s) were added to refs/heads/master by this push:
     new 11890feb95 ARROW-16357: [Archery][Dev] Add possibility to send nightly reports to Zulip/Slack
11890feb95 is described below

commit 11890feb95378991c9438ec4773b3dd928599494
Author: Raúl Cumplido <ra...@gmail.com>
AuthorDate: Wed May 4 11:29:50 2022 +0200

    ARROW-16357: [Archery][Dev] Add possibility to send nightly reports to Zulip/Slack
    
    This PR adds the possibility to send the reports via a webhook to Zulip / Slack.
    
    Usage example:
    ```
    $ archery crossbow -t $GITHUB_TOKEN report-chat --webhook $SLACK_OR_ZULIP_WEBHOOK_URL --send --no-fetch $JOB_NAME
    ```
    The integration has been tested both in Zulip:
    ![image](https://user-images.githubusercontent.com/639755/165933376-4d15db5a-b506-497b-953d-2bb038a3e396.png)
    
    And Slack:
    ![image](https://user-images.githubusercontent.com/639755/165933320-33da69b8-bbbb-4e93-9bee-9ae548ca80d2.png)
    
    Closes #13031 from raulcd/ARROW-16357
    
    Authored-by: Raúl Cumplido <ra...@gmail.com>
    Signed-off-by: Krisztián Szűcs <sz...@gmail.com>
---
 dev/archery/archery/crossbow/cli.py                | 30 +++++++-
 dev/archery/archery/crossbow/core.py               |  2 +
 dev/archery/archery/crossbow/reports.py            | 90 +++++++++++++++-------
 .../crossbow/tests/fixtures/chat-report.txt        | 12 +++
 .../crossbow/tests/fixtures/crossbow-job.yaml      | 16 ++++
 dev/archery/archery/crossbow/tests/test_reports.py | 12 ++-
 .../chat_nightly_report.txt.j2}                    | 30 ++++----
 dev/archery/setup.py                               |  4 +-
 8 files changed, 146 insertions(+), 50 deletions(-)

diff --git a/dev/archery/archery/crossbow/cli.py b/dev/archery/archery/crossbow/cli.py
index cc4c648c6d..6005c22de9 100644
--- a/dev/archery/archery/crossbow/cli.py
+++ b/dev/archery/archery/crossbow/cli.py
@@ -21,7 +21,8 @@ import time
 import click
 
 from .core import Config, Repo, Queue, Target, Job, CrossbowError
-from .reports import EmailReport, ConsoleReport
+from .reports import (ChatReport, Report, ReportUtils, ConsoleReport,
+                      EmailReport)
 from ..utils.source import ArrowSources
 
 
@@ -302,6 +303,33 @@ def report(obj, job_name, sender_name, sender_email, recipient_email,
         report.show(output)
 
 
+@crossbow.command()
+@click.argument('job-name', required=True)
+@click.option('--send/--dry-run', default=False,
+              help='Just display the report, don\'t send it')
+@click.option('--webhook', '-w',
+              help='Zulip/Slack Webhook address to send the report to')
+@click.option('--fetch/--no-fetch', default=True,
+              help='Fetch references (branches and tags) from the remote')
+@click.pass_obj
+def report_chat(obj, job_name, send, webhook, fetch):
+    """
+    Send a chat report to a webhook showing success/failure
+    of tasks in a Crossbow run.
+    """
+    output = obj['output']
+    queue = obj['queue']
+    if fetch:
+        queue.fetch()
+
+    job = queue.get(job_name)
+    report_chat = ChatReport(report=Report(job))
+    if send:
+        ReportUtils.send_message(webhook, report_chat.render("text"))
+    else:
+        output.write(report_chat.render("text"))
+
+
 @crossbow.command()
 @click.argument('job-name', required=True)
 @click.option('-t', '--target-dir',
diff --git a/dev/archery/archery/crossbow/core.py b/dev/archery/archery/crossbow/core.py
index 1ad4763e29..9b09fa9d2c 100644
--- a/dev/archery/archery/crossbow/core.py
+++ b/dev/archery/archery/crossbow/core.py
@@ -1219,3 +1219,5 @@ yaml = YAML()
 yaml.register_class(Job)
 yaml.register_class(Task)
 yaml.register_class(Target)
+yaml.register_class(Queue)
+yaml.register_class(TaskStatus)
diff --git a/dev/archery/archery/crossbow/reports.py b/dev/archery/archery/crossbow/reports.py
index 79939a0e44..3858106a90 100644
--- a/dev/archery/archery/crossbow/reports.py
+++ b/dev/archery/archery/crossbow/reports.py
@@ -23,6 +23,10 @@ import functools
 from io import StringIO
 import textwrap
 
+import requests
+
+from archery.utils.report import JinjaReport
+
 
 # TODO(kszucs): use archery.report.JinjaReport instead
 class Report:
@@ -40,6 +44,35 @@ class Report:
 
         self._tasks = dict(tasks)
 
+    @property
+    def repo_url(self):
+        url = self.job.queue.remote_url
+        return url[:-4] if url.endswith('.git') else url
+
+    def url(self, query):
+        return '{}/branches/all?query={}'.format(self.repo_url, query)
+
+    def branch_url(self, branch):
+        return '{}/tree/{}'.format(self.repo_url, branch)
+
+    def task_url(self, task):
+        if task.status().build_links:
+            # show link to the actual build, some CI providers implement
+            # the statuses API others implement the checks API, retrieve any.
+            return task.status().build_links[0]
+        else:
+            # show link to the branch if no status build link was found.
+            return self.branch_url(task.branch)
+
+    @property
+    @functools.lru_cache(maxsize=1)
+    def tasks_by_state(self):
+        tasks_by_state = collections.defaultdict(dict)
+        for task_name, task in self.job.tasks.items():
+            state = task.status().combined_state
+            tasks_by_state[state][task_name] = task
+        return tasks_by_state
+
     @property
     def tasks(self):
         return self._tasks
@@ -134,6 +167,34 @@ class ConsoleReport(Report):
                                    asset))
 
 
+class ChatReport(JinjaReport):
+    templates = {
+        'text': 'chat_nightly_report.txt.j2',
+    }
+    fields = [
+        'report',
+    ]
+
+
+class ReportUtils:
+
+    @classmethod
+    def send_message(cls, webhook, message):
+        resp = requests.post(webhook, json={
+            "blocks": [
+                {
+                    "type": "section",
+                    "text": {
+                            "type": "mrkdwn",
+                            "text": message
+                    }
+                }
+            ]
+        }
+        )
+        return resp
+
+
 class EmailReport(Report):
 
     HEADER = textwrap.dedent("""
@@ -169,26 +230,6 @@ class EmailReport(Report):
         self.recipient_email = recipient_email
         super().__init__(job)
 
-    @property
-    def repo_url(self):
-        url = self.job.queue.remote_url
-        return url[:-4] if url.endswith('.git') else url
-
-    def url(self, query):
-        return '{}/branches/all?query={}'.format(self.repo_url, query)
-
-    def branch_url(self, branch):
-        return '{}/tree/{}'.format(self.repo_url, branch)
-
-    def task_url(self, task):
-        if task.status().build_links:
-            # show link to the actual build, some CI providers implement
-            # the statuses API others implement the checks API, retrieve any.
-            return task.status().build_links[0]
-        else:
-            # show link to the branch if no status build link was found.
-            return self.branch_url(task.branch)
-
     def listing(self, tasks):
         return '\n'.join(
             sorted(
@@ -213,15 +254,6 @@ class EmailReport(Report):
             f"{failures+errors} failed, {pending} pending"
         )
 
-    @property
-    @functools.lru_cache(maxsize=1)
-    def tasks_by_state(self):
-        tasks_by_state = collections.defaultdict(dict)
-        for task_name, task in self.job.tasks.items():
-            state = task.status().combined_state
-            tasks_by_state[state][task_name] = task
-        return tasks_by_state
-
     def body(self):
         buffer = StringIO()
         buffer.write(self.header())
diff --git a/dev/archery/archery/crossbow/tests/fixtures/chat-report.txt b/dev/archery/archery/crossbow/tests/fixtures/chat-report.txt
new file mode 100644
index 0000000000..3ee807d12a
--- /dev/null
+++ b/dev/archery/archery/crossbow/tests/fixtures/chat-report.txt
@@ -0,0 +1,12 @@
+
+*Archery crossbow report for <https://github.com/apache/crossbow/branches/all?query=ursabot-1|ursabot-1>*
+
+:x: *1 failed jobs*
+- <https://github.com/apache/crossbow/runs/2|wheel-osx-cp37m>
+
+:x: *1 errored jobs*
+- <https://github.com/apache/crossbow/runs/3|wheel-osx-cp36m>
+
+:warning: *1 pending jobs*
+
+:tada: *1 successful jobs*
\ No newline at end of file
diff --git a/dev/archery/archery/crossbow/tests/fixtures/crossbow-job.yaml b/dev/archery/archery/crossbow/tests/fixtures/crossbow-job.yaml
index c37c7b553a..90eab70498 100644
--- a/dev/archery/archery/crossbow/tests/fixtures/crossbow-job.yaml
+++ b/dev/archery/archery/crossbow/tests/fixtures/crossbow-job.yaml
@@ -18,6 +18,9 @@ tasks:
       - docker-compose run cpp-cmake32
     branch: ursabot-1-circle-docker-cpp-cmake32
     commit: a56b077c8d1b891a7935048e5672bf6fc07599ec
+    _status: !TaskStatus
+      combined_state: success
+      build_links: ["https://github.com/apache/crossbow/runs/1"]
   wheel-osx-cp37m: !Task
     ci: travis
     platform: osx
@@ -28,6 +31,9 @@ tasks:
       python_version: 3.7
     branch: ursabot-1-travis-wheel-osx-cp37m
     commit: a56b077c8d1b891a7935048e5672bf6fc07599ec
+    _status: !TaskStatus
+      combined_state: failure
+      build_links: ["https://github.com/apache/crossbow/runs/2"]
   wheel-osx-cp36m: !Task
     ci: travis
     platform: osx
@@ -38,6 +44,9 @@ tasks:
       python_version: 3.6
     branch: ursabot-1-travis-wheel-osx-cp36m
     commit: a56b077c8d1b891a7935048e5672bf6fc07599ec
+    _status: !TaskStatus
+      combined_state: error
+      build_links: ["https://github.com/apache/crossbow/runs/3"]
   wheel-win-cp36m: !Task
     ci: appveyor
     platform: win
@@ -48,4 +57,11 @@ tasks:
       python_version: 3.6
     branch: ursabot-1-appveyor-wheel-win-cp36m
     commit: a56b077c8d1b891a7935048e5672bf6fc07599ec
+    _status: !TaskStatus
+      combined_state: pending
+      build_links: ["https://github.com/apache/crossbow/runs/4"]
 branch: ursabot-1
+_queue: !Queue
+  path: the_path
+  github_token: xxxxxxxxx
+  _remote_url: https://github.com/apache/crossbow
\ No newline at end of file
diff --git a/dev/archery/archery/crossbow/tests/test_reports.py b/dev/archery/archery/crossbow/tests/test_reports.py
index 0df292bb55..deb90c47cf 100644
--- a/dev/archery/archery/crossbow/tests/test_reports.py
+++ b/dev/archery/archery/crossbow/tests/test_reports.py
@@ -18,7 +18,7 @@
 import textwrap
 
 from archery.crossbow.core import yaml
-from archery.crossbow.reports import CommentReport
+from archery.crossbow.reports import ChatReport, CommentReport, Report
 
 
 def test_crossbow_comment_formatter(load_fixture):
@@ -33,3 +33,13 @@ def test_crossbow_comment_formatter(load_fixture):
         status='has been succeeded.'
     )
     assert report.show() == textwrap.dedent(expected).strip()
+
+
+def test_crossbow_report(load_fixture):
+    expected_msg = load_fixture('chat-report.txt')
+    job = load_fixture('crossbow-job.yaml', decoder=yaml.load)
+    report = Report(job)
+    assert report.tasks_by_state is not None
+    report_chat = ChatReport(report=report)
+
+    assert report_chat.render("text") == textwrap.dedent(expected_msg)
diff --git a/dev/archery/archery/crossbow/tests/test_reports.py b/dev/archery/archery/templates/chat_nightly_report.txt.j2
similarity index 55%
copy from dev/archery/archery/crossbow/tests/test_reports.py
copy to dev/archery/archery/templates/chat_nightly_report.txt.j2
index 0df292bb55..150bf0d54f 100644
--- a/dev/archery/archery/crossbow/tests/test_reports.py
+++ b/dev/archery/archery/templates/chat_nightly_report.txt.j2
@@ -1,3 +1,4 @@
+{#
 # 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
@@ -14,22 +15,17 @@
 # KIND, either express or implied.  See the License for the
 # specific language governing permissions and limitations
 # under the License.
+#}
+*Archery crossbow report for <{{ report.url(report.job.branch) }}|{{ report.job.branch }}>*
 
-import textwrap
+:x: *{{ report.tasks_by_state["failure"] | length }} failed jobs*
+{% for task_name, task in report.tasks_by_state["failure"] | dictsort -%}
+- <{{ report.task_url(task) }}|{{ task_name }}>
+{% endfor %}
+:x: *{{ report.tasks_by_state["error"] | length }} errored jobs*
+{% for task_name, task in report.tasks_by_state["error"] | dictsort -%}
+- <{{ report.task_url(task) }}|{{ task_name }}>
+{% endfor %}
+:warning: *{{ report.tasks_by_state["pending"] | length }} pending jobs*
 
-from archery.crossbow.core import yaml
-from archery.crossbow.reports import CommentReport
-
-
-def test_crossbow_comment_formatter(load_fixture):
-    msg = load_fixture('crossbow-success-message.md')
-    job = load_fixture('crossbow-job.yaml', decoder=yaml.load)
-
-    report = CommentReport(job, crossbow_repo='ursa-labs/crossbow')
-    expected = msg.format(
-        repo='ursa-labs/crossbow',
-        branch='ursabot-1',
-        revision='f766a1d615dd1b7ee706d05102e579195951a61c',
-        status='has been succeeded.'
-    )
-    assert report.show() == textwrap.dedent(expected).strip()
+:tada: *{{ report.tasks_by_state["success"] | length }} successful jobs*
diff --git a/dev/archery/setup.py b/dev/archery/setup.py
index ce1b97e0ae..4b13608cf8 100755
--- a/dev/archery/setup.py
+++ b/dev/archery/setup.py
@@ -32,8 +32,8 @@ extras = {
     'benchmark': ['pandas'],
     'docker': ['ruamel.yaml', 'python-dotenv'],
     'release': [jinja_req, 'jira', 'semver', 'gitpython'],
-    'crossbow': ['github3.py', jinja_req, 'pygit2>=1.6.0', 'ruamel.yaml',
-                 'setuptools_scm'],
+    'crossbow': ['github3.py', jinja_req, 'pygit2>=1.6.0', 'requests',
+                 'ruamel.yaml', 'setuptools_scm'],
     'crossbow-upload': ['github3.py', jinja_req, 'ruamel.yaml',
                         'setuptools_scm'],
     'numpydoc': ['numpydoc==1.1.0']