You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@airflow.apache.org by ur...@apache.org on 2022/09/06 19:17:59 UTC

[airflow] branch main updated: Allow setting TaskGroup tooltip via function docstring (#26028)

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

uranusjr pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/airflow.git


The following commit(s) were added to refs/heads/main by this push:
     new da3dbff8d7 Allow setting TaskGroup tooltip via function docstring (#26028)
da3dbff8d7 is described below

commit da3dbff8d7b32aa5a266d8c3977dcfccf50a5f64
Author: Josh Fell <48...@users.noreply.github.com>
AuthorDate: Tue Sep 6 15:17:51 2022 -0400

    Allow setting TaskGroup tooltip via function docstring (#26028)
---
 airflow/decorators/task_group.py      |  3 +++
 docs/apache-airflow/concepts/dags.rst | 18 +++++++++++++----
 tests/decorators/test_task_group.py   | 38 ++++++++++++++++++++++++++++++++++-
 3 files changed, 54 insertions(+), 5 deletions(-)

diff --git a/airflow/decorators/task_group.py b/airflow/decorators/task_group.py
index 957317578e..df647d2885 100644
--- a/airflow/decorators/task_group.py
+++ b/airflow/decorators/task_group.py
@@ -62,6 +62,9 @@ class TaskGroupDecorator(Generic[R]):
 
     def __call__(self, *args, **kwargs) -> Union[R, TaskGroup]:
         with self._make_task_group(add_suffix_on_collision=True, **self.kwargs) as task_group:
+            if self.function.__doc__ and not task_group.tooltip:
+                task_group.tooltip = self.function.__doc__
+
             # Invoke function to run Tasks inside the TaskGroup
             retval = self.function(*args, **kwargs)
 
diff --git a/docs/apache-airflow/concepts/dags.rst b/docs/apache-airflow/concepts/dags.rst
index c3dfacfc85..4e52c1532a 100644
--- a/docs/apache-airflow/concepts/dags.rst
+++ b/docs/apache-airflow/concepts/dags.rst
@@ -481,18 +481,23 @@ Unlike :ref:`concepts:subdags`, TaskGroups are purely a UI grouping concept. Tas
 
 Dependency relationships can be applied across all tasks in a TaskGroup with the ``>>`` and ``<<`` operators. For example, the following code puts ``task1`` and ``task2`` in TaskGroup ``group1`` and then puts both tasks upstream of ``task3``::
 
-    with TaskGroup("group1") as group1:
+    from airflow.decorators import task_group
+
+    @task_group()
+    def group1():
         task1 = EmptyOperator(task_id="task1")
         task2 = EmptyOperator(task_id="task2")
 
     task3 = EmptyOperator(task_id="task3")
 
-    group1 >> task3
+    group1() >> task3
 
 TaskGroup also supports ``default_args`` like DAG, it will overwrite the ``default_args`` in DAG level::
 
     import pendulum
 
+    from airflow.decorators import task_group
+
     with DAG(
         dag_id='dag1',
         start_date=pendulum.datetime(2016, 1, 1, tz="UTC"),
@@ -500,13 +505,15 @@ TaskGroup also supports ``default_args`` like DAG, it will overwrite the ``defau
         catchup=False,
         default_args={'retries': 1},
     ):
-        with TaskGroup('group1', default_args={'retries': 3}):
+        @task_group(default_args={'retries': 3}):
+        def group1():
+            """This docstring will become the tooltip for the TaskGroup."
             task1 = EmptyOperator(task_id='task1')
             task2 = BashOperator(task_id='task2', bash_command='echo Hello World!', retries=2)
             print(task1.retries) # 3
             print(task2.retries) # 2
 
-If you want to see a more advanced use of TaskGroup, you can look at the ``example_task_group.py`` example DAG that comes with Airflow.
+If you want to see a more advanced use of TaskGroup, you can look at the ``example_task_group_decorator.py`` example DAG that comes with Airflow.
 
 .. note::
 
@@ -514,6 +521,9 @@ If you want to see a more advanced use of TaskGroup, you can look at the ``examp
 
     To disable the prefixing, pass ``prefix_group_id=False`` when creating the TaskGroup, but note that you will now be responsible for ensuring every single task and group has a unique ID of its own.
 
+.. note::
+
+    When using the ``@task_group`` decorator, the decorated-function's docstring will be used as the TaskGroups tooltip in the UI except when a ``tooltip`` value is explicitly supplied.
 
 .. _concepts:edge-labels:
 
diff --git a/tests/decorators/test_task_group.py b/tests/decorators/test_task_group.py
index 2d568749dc..ac1ca93249 100644
--- a/tests/decorators/test_task_group.py
+++ b/tests/decorators/test_task_group.py
@@ -15,8 +15,9 @@
 # KIND, either express or implied.  See the License for the
 # specific language governing permissions and limitations
 # under the License.
+import pendulum
 
-from airflow.decorators import task_group
+from airflow.decorators import dag, task_group
 
 
 def test_task_group_with_overridden_kwargs():
@@ -50,3 +51,38 @@ def test_task_group_with_overridden_kwargs():
         },
         'add_suffix_on_collision': True,
     }
+
+
+def test_tooltip_derived_from_function_docstring():
+    """Test that the tooltip for TaskGroup is the decorated-function's docstring."""
+
+    @dag(start_date=pendulum.datetime(2022, 1, 1))
+    def pipeline():
+        @task_group()
+        def tg():
+            """Function docstring."""
+
+        tg()
+
+    _ = pipeline()
+
+    assert _.task_group_dict["tg"].tooltip == "Function docstring."
+
+
+def test_tooltip_not_overriden_by_function_docstring():
+    """
+    Test that the tooltip for TaskGroup is the explicitly set value even if the decorated function has a
+    docstring.
+    """
+
+    @dag(start_date=pendulum.datetime(2022, 1, 1))
+    def pipeline():
+        @task_group(tooltip="tooltip for the TaskGroup")
+        def tg():
+            """Function docstring."""
+
+        tg()
+
+    _ = pipeline()
+
+    assert _.task_group_dict["tg"].tooltip == "tooltip for the TaskGroup"