Skip to content

Commit

Permalink
Bugfix: Task docs are not shown in the Task Instance Detail View (#15…
Browse files Browse the repository at this point in the history
…191)

closes apache/airflow#15178
closes apache/airflow#13761

This feature was added in 2015 in apache/airflow#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.

GitOrigin-RevId: e86f5ca8fa5ff22c1e1f48addc012919034c672f
  • Loading branch information
kaxil authored and Cloud Composer Team committed Sep 17, 2021
1 parent b0ebf14 commit 31dba3e
Show file tree
Hide file tree
Showing 8 changed files with 49 additions and 8 deletions.
1 change: 1 addition & 0 deletions airflow/example_dags/tutorial.py
Original file line number Diff line number Diff line change
Expand Up @@ -97,6 +97,7 @@
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)
"""
)
Expand Down
26 changes: 26 additions & 0 deletions airflow/models/baseoperator.py
Original file line number Diff line number Diff line change
Expand Up @@ -282,6 +282,21 @@ class derived from this one results in the creation of a task object,
: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
Expand Down Expand Up @@ -385,6 +400,11 @@ def __init__(
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
Expand Down Expand Up @@ -490,6 +510,12 @@ def __init__(
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()
Expand Down
7 changes: 6 additions & 1 deletion airflow/serialization/schema.json
Original file line number Diff line number Diff line change
Expand Up @@ -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
},
Expand Down
2 changes: 1 addition & 1 deletion airflow/www/utils.py
Original file line number Diff line number Diff line change
Expand Up @@ -338,7 +338,7 @@ def json_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
Expand Down
2 changes: 1 addition & 1 deletion airflow/www/views.py
Original file line number Diff line number Diff line change
Expand Up @@ -1247,7 +1247,7 @@ def task(self):
# 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)

Expand Down
6 changes: 3 additions & 3 deletions docs/apache-airflow/concepts.rst
Original file line number Diff line number Diff line change
Expand Up @@ -1415,8 +1415,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:

========== ================
Expand Down Expand Up @@ -1451,7 +1451,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:

Expand Down
9 changes: 9 additions & 0 deletions tests/serialization/test_dag_serialization.py
Original file line number Diff line number Diff line change
Expand Up @@ -79,6 +79,7 @@
},
"is_paused_upon_creation": False,
"_dag_id": "simple_dag",
"doc_md": "### DAG Tutorial Documentation",
"fileloc": None,
"tasks": [
{
Expand Down Expand Up @@ -110,6 +111,7 @@
}
},
},
"doc_md": "### Task Tutorial Documentation",
},
{
"task_id": "custom_task",
Expand Down Expand Up @@ -170,13 +172,15 @@ 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(
task_id='bash_task',
bash_command='echo {{ task.task_id }}',
owner='airflow',
executor_config={"pod_override": executor_config_pod},
doc_md="### Task Tutorial Documentation",
)
return {'simple_dag': dag}

Expand Down Expand Up @@ -853,6 +857,11 @@ def test_no_new_fields_added_to_base_operator(self):
'_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,
Expand Down
4 changes: 2 additions & 2 deletions tests/www/test_utils.py
Original file line number Diff line number Diff line change
Expand Up @@ -240,7 +240,7 @@ def test_wrapped_markdown_with_table(self):
)

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>'
Expand All @@ -255,4 +255,4 @@ def test_wrapped_markdown_with_indented_lines(self):
"""
)

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

0 comments on commit 31dba3e

Please sign in to comment.