You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@airflow.apache.org by as...@apache.org on 2021/04/15 12:07:46 UTC

[airflow] 27/36: Bugfix: Task docs are not shown in the Task Instance Detail View (#15191)

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

ash pushed a commit to branch v2-0-test
in repository https://gitbox.apache.org/repos/asf/airflow.git

commit c8d1c670679995bab7cbfc0adb5c3a9969898f80
Author: Kaxil Naik <ka...@gmail.com>
AuthorDate: Mon Apr 5 03:46:41 2021 +0100

    Bugfix: Task docs are not shown in the Task Instance Detail View (#15191)
    
    closes https://github.com/apache/airflow/issues/15178
    closes https://github.com/apache/airflow/issues/13761
    
    This feature was added in 2015 in https://github.com/apache/airflow/pull/74 and it was expected to set `doc_md` (or `doc_rst` and other `doc_*`) via `task.doc_md` instead of passing via arg. However, this did not work with DAG Serialization as we only allowed a selected args to be stored in Serialized version of DAG.
    
    (cherry picked from commit e86f5ca8fa5ff22c1e1f48addc012919034c672f)
---
 airflow/example_dags/tutorial.py              |  1 +
 airflow/models/baseoperator.py                | 26 ++++++++++++++++++++++++++
 airflow/serialization/schema.json             |  7 ++++++-
 airflow/www/utils.py                          |  2 +-
 airflow/www/views.py                          |  2 +-
 docs/apache-airflow/concepts.rst              |  6 +++---
 tests/serialization/test_dag_serialization.py |  9 +++++++++
 tests/www/test_utils.py                       |  4 ++--
 8 files changed, 49 insertions(+), 8 deletions(-)

diff --git a/airflow/example_dags/tutorial.py b/airflow/example_dags/tutorial.py
index 518c801..09d6ca3 100644
--- a/airflow/example_dags/tutorial.py
+++ b/airflow/example_dags/tutorial.py
@@ -97,6 +97,7 @@ with DAG(
     You can document your task using the attributes `doc_md` (markdown),
     `doc` (plain text), `doc_rst`, `doc_json`, `doc_yaml` which gets
     rendered in the UI's Task Instance Details page.
+
     ![img](http://montcs.bloomu.edu/~bobmon/Semesters/2012-01/491/import%20soul.png)
     """
     )
diff --git a/airflow/models/baseoperator.py b/airflow/models/baseoperator.py
index 8bda785..eacea64 100644
--- a/airflow/models/baseoperator.py
+++ b/airflow/models/baseoperator.py
@@ -278,6 +278,21 @@ class BaseOperator(Operator, LoggingMixin, TaskMixin, metaclass=BaseOperatorMeta
     :param do_xcom_push: if True, an XCom is pushed containing the Operator's
         result
     :type do_xcom_push: bool
+    :param doc: Add documentation or notes to your Task objects that is visible in
+        Task Instance details View in the Webserver
+    :type doc: str
+    :param doc_md: Add documentation (in Markdown format) or notes to your Task objects
+        that is visible in Task Instance details View in the Webserver
+    :type doc_md: str
+    :param doc_rst: Add documentation (in RST format) or notes to your Task objects
+        that is visible in Task Instance details View in the Webserver
+    :type doc_rst: str
+    :param doc_json: Add documentation (in JSON format) or notes to your Task objects
+        that is visible in Task Instance details View in the Webserver
+    :type doc_json: str
+    :param doc_yaml: Add documentation (in YAML format) or notes to your Task objects
+        that is visible in Task Instance details View in the Webserver
+    :type doc_yaml: str
     """
 
     # For derived classes to define which fields will get jinjaified
@@ -381,6 +396,11 @@ class BaseOperator(Operator, LoggingMixin, TaskMixin, metaclass=BaseOperatorMeta
         inlets: Optional[Any] = None,
         outlets: Optional[Any] = None,
         task_group: Optional["TaskGroup"] = None,
+        doc: Optional[str] = None,
+        doc_md: Optional[str] = None,
+        doc_json: Optional[str] = None,
+        doc_yaml: Optional[str] = None,
+        doc_rst: Optional[str] = None,
         **kwargs,
     ):
         from airflow.models.dag import DagContext
@@ -486,6 +506,12 @@ class BaseOperator(Operator, LoggingMixin, TaskMixin, metaclass=BaseOperatorMeta
         self.executor_config = executor_config or {}
         self.do_xcom_push = do_xcom_push
 
+        self.doc_md = doc_md
+        self.doc_json = doc_json
+        self.doc_yaml = doc_yaml
+        self.doc_rst = doc_rst
+        self.doc = doc
+
         # Private attributes
         self._upstream_task_ids: Set[str] = set()
         self._downstream_task_ids: Set[str] = set()
diff --git a/airflow/serialization/schema.json b/airflow/serialization/schema.json
index 0fbe20f..3bc11ee 100644
--- a/airflow/serialization/schema.json
+++ b/airflow/serialization/schema.json
@@ -168,7 +168,12 @@
           "type": "array",
           "items": { "type": "string" },
           "uniqueItems": true
-        }
+        },
+        "doc":  { "type": "string" },
+        "doc_md":  { "type": "string" },
+        "doc_json":  { "type": "string" },
+        "doc_yaml":  { "type": "string" },
+        "doc_rst":  { "type": "string" }
       },
       "additionalProperties": true
     },
diff --git a/airflow/www/utils.py b/airflow/www/utils.py
index afd94c6..ad53436 100644
--- a/airflow/www/utils.py
+++ b/airflow/www/utils.py
@@ -321,7 +321,7 @@ def render(obj, lexer):
     return out
 
 
-def wrapped_markdown(s, css_class=None):
+def wrapped_markdown(s, css_class='rich_doc'):
     """Convert a Markdown string to HTML."""
     if s is None:
         return None
diff --git a/airflow/www/views.py b/airflow/www/views.py
index f0116b3..5f4c8c5 100644
--- a/airflow/www/views.py
+++ b/airflow/www/views.py
@@ -1220,7 +1220,7 @@ class Airflow(AirflowBaseView):  # noqa: D101  pylint: disable=too-many-public-m
         # Color coding the special attributes that are code
         special_attrs_rendered = {}
         for attr_name in wwwutils.get_attr_renderer():
-            if hasattr(task, attr_name):
+            if getattr(task, attr_name, None) is not None:
                 source = getattr(task, attr_name)
                 special_attrs_rendered[attr_name] = wwwutils.get_attr_renderer()[attr_name](source)
 
diff --git a/docs/apache-airflow/concepts.rst b/docs/apache-airflow/concepts.rst
index 2637b78..3de060b 100644
--- a/docs/apache-airflow/concepts.rst
+++ b/docs/apache-airflow/concepts.rst
@@ -1394,8 +1394,8 @@ Documentation & Notes
 =====================
 
 It's possible to add documentation or notes to your DAGs & task objects that
-become visible in the web interface ("Graph View" & "Tree View" for DAGs, "Task Details" for
-tasks). There are a set of special task attributes that get rendered as rich
+become visible in the web interface ("Graph View" & "Tree View" for DAGs, "Task Instance Details"
+for tasks). There are a set of special task attributes that get rendered as rich
 content if defined:
 
 ==========  ================
@@ -1430,7 +1430,7 @@ to the related tasks in Airflow.
     """
 
 This content will get rendered as markdown respectively in the "Graph View" and
-"Task Details" pages.
+"Task Instance Details" pages.
 
 .. _jinja-templating:
 
diff --git a/tests/serialization/test_dag_serialization.py b/tests/serialization/test_dag_serialization.py
index 55d2c5a..e447751 100644
--- a/tests/serialization/test_dag_serialization.py
+++ b/tests/serialization/test_dag_serialization.py
@@ -79,6 +79,7 @@ serialized_simple_dag_ground_truth = {
         },
         "is_paused_upon_creation": False,
         "_dag_id": "simple_dag",
+        "doc_md": "### DAG Tutorial Documentation",
         "fileloc": None,
         "tasks": [
             {
@@ -110,6 +111,7 @@ serialized_simple_dag_ground_truth = {
                         }
                     },
                 },
+                "doc_md": "### Task Tutorial Documentation",
             },
             {
                 "task_id": "custom_task",
@@ -170,6 +172,7 @@ def make_simple_dag():
         start_date=datetime(2019, 8, 1),
         is_paused_upon_creation=False,
         access_control={"test_role": {permissions.ACTION_CAN_READ, permissions.ACTION_CAN_EDIT}},
+        doc_md="### DAG Tutorial Documentation",
     ) as dag:
         CustomOperator(task_id='custom_task')
         BashOperator(
@@ -177,6 +180,7 @@ def make_simple_dag():
             bash_command='echo {{ task.task_id }}',
             owner='airflow',
             executor_config={"pod_override": executor_config_pod},
+            doc_md="### Task Tutorial Documentation",
         )
         return {'simple_dag': dag}
 
@@ -853,6 +857,11 @@ class TestStringifiedDAGs(unittest.TestCase):
             '_upstream_task_ids': set(),
             'depends_on_past': False,
             'do_xcom_push': True,
+            'doc': None,
+            'doc_json': None,
+            'doc_md': None,
+            'doc_rst': None,
+            'doc_yaml': None,
             'email': None,
             'email_on_failure': True,
             'email_on_retry': True,
diff --git a/tests/www/test_utils.py b/tests/www/test_utils.py
index 5ced73a..f4e50d9 100644
--- a/tests/www/test_utils.py
+++ b/tests/www/test_utils.py
@@ -240,7 +240,7 @@ class TestWrappedMarkdown(unittest.TestCase):
         )
 
         assert (
-            '<div class="None" ><table>\n<thead>\n<tr>\n<th>Job</th>\n'
+            '<div class="rich_doc" ><table>\n<thead>\n<tr>\n<th>Job</th>\n'
             '<th>Duration</th>\n</tr>\n</thead>\n<tbody>\n<tr>\n<td>ETL'
             '</td>\n<td>14m</td>\n</tr>\n</tbody>\n'
             '</table></div>'
@@ -255,4 +255,4 @@ class TestWrappedMarkdown(unittest.TestCase):
             """
         )
 
-        assert '<div class="None" ><h1>header</h1>\n<p>1st line\n2nd line</p></div>' == rendered
+        assert '<div class="rich_doc" ><h1>header</h1>\n<p>1st line\n2nd line</p></div>' == rendered