Improvement in task documentation capabilities

This commit is contained in:
Maxime Beauchemin 2015-06-24 09:44:47 -07:00
Родитель f16381922c
Коммит ec7422334b
3 изменённых файлов: 47 добавлений и 29 удалений

Просмотреть файл

@ -33,6 +33,14 @@ t1 = BashOperator(
bash_command='date',
dag=dag)
t1.doc_md = """\
#### Task Documentation
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 Details page.
![img](http://montcs.bloomu.edu/~bobmon/Semesters/2012-01/491/import%20soul.png)
"""
t2 = BashOperator(
task_id='sleep',
depends_on_past=False,

Просмотреть файл

@ -20,10 +20,7 @@ from wtforms import (
widgets,
Form, DateTimeField, SelectField, TextAreaField, PasswordField)
from pygments import highlight
from pygments.lexers import (
PythonLexer, SqlLexer, BashLexer,
IniLexer, YamlLexer, JsonLexer, TextLexer)
from pygments import highlight, lexers
from pygments.formatters import HtmlFormatter
import chartkick
@ -108,13 +105,26 @@ def data_profiling_required(f):
QUERY_LIMIT = 100000
CHART_LIMIT = 200000
special_attrs = {
'sql': SqlLexer,
'hql': SqlLexer,
'doc_yaml': YamlLexer,
'doc_json': JsonLexer,
'doc': TextLexer,
'bash_command': BashLexer,
def pygment_html_render(s, lexer=lexers.TextLexer):
return highlight(
s,
lexer(),
HtmlFormatter(linenos=True),
)
return s
def wrapped_markdown(s):
return '<div class="rich_doc">' + markdown.markdown(s) + "</div>"
attr_renderer = {
'bash_command': lambda x: pygment_html_render(x, lexers.BashLexer),
'hql': lambda x: pygment_html_render(x, lexers.SqlLexer),
'sql': lambda x: pygment_html_render(x, lexers.SqlLexer),
'doc': lambda x: pygment_html_render(x, lexers.TextLexer),
'doc_json': lambda x: pygment_html_render(x, lexers.JsonLexer),
'doc_rst': lambda x: pygment_html_render(x, lexers.RstLexer),
'doc_yaml': lambda x: pygment_html_render(x, lexers.YamlLexer),
'doc_md': wrapped_markdown,
}
@ -306,7 +316,7 @@ class Airflow(BaseView):
label = jinja2.Template(chart.label).render(**args)
payload['sql_html'] = Markup(highlight(
sql,
SqlLexer(), # Lexer call
lexers.SqlLexer(), # Lexer call
HtmlFormatter(noclasses=True))
)
payload['label'] = label
@ -561,7 +571,7 @@ class Airflow(BaseView):
if chart.show_sql:
sql = Markup(highlight(
chart.sql,
SqlLexer(), # Lexer call
lexers.SqlLexer(), # Lexer call
HtmlFormatter(noclasses=True))
)
return self.render(
@ -621,7 +631,7 @@ class Airflow(BaseView):
code = "".join(open(dag.full_filepath, 'r').readlines())
title = dag.filepath
html_code = highlight(
code, PythonLexer(), HtmlFormatter(linenos=True))
code, lexers.PythonLexer(), HtmlFormatter(linenos=True))
return self.render(
'airflow/dag_code.html', html_code=html_code, dag=dag, title=title,
demo_mode=conf.getboolean('webserver', 'demo_mode'))
@ -643,7 +653,7 @@ class Airflow(BaseView):
f.close()
code_html = Markup(highlight(
config,
IniLexer(), # Lexer call
lexers.IniLexer(), # Lexer call
HtmlFormatter(noclasses=True))
)
return self.render(
@ -694,12 +704,8 @@ class Airflow(BaseView):
html_dict = {}
for template_field in task.__class__.template_fields:
content = getattr(task, template_field)
if template_field in special_attrs:
html_dict[template_field] = highlight(
content,
special_attrs[template_field](), # Lexer call
HtmlFormatter(linenos=True),
)
if template_field in attr_renderer:
html_dict[template_field] = attr_renderer[template_field](content)
else:
html_dict[template_field] = (
"<pre><code>" + str(content) + "</pre></code>")
@ -787,20 +793,16 @@ class Airflow(BaseView):
if not attr_name.startswith('_'):
attr = getattr(task, attr_name)
if type(attr) != type(self.task) and \
attr_name not in special_attrs:
attr_name not in attr_renderer:
attributes.append((attr_name, str(attr)))
title = "Task Details"
# Color coding the special attributes that are code
special_attrs_rendered = {}
for attr_name in special_attrs:
for attr_name in attr_renderer:
if hasattr(task, attr_name):
source = getattr(task, attr_name)
special_attrs_rendered[attr_name] = highlight(
source,
special_attrs[attr_name](), # Lexer call
HtmlFormatter(linenos=True),
)
special_attrs_rendered[attr_name] = attr_renderer[attr_name](source)
return self.render(
'airflow/task.html',
@ -1545,7 +1547,7 @@ class ConfigurationView(wwwutils.SuperUserMixin, BaseView):
else:
code_html = Markup(highlight(
config,
IniLexer(), # Lexer call
lexers.IniLexer(), # Lexer call
HtmlFormatter(noclasses=True))
)
return self.render(

Просмотреть файл

@ -18,6 +18,14 @@ td>span.glyphicon{
button.btn {
border: 1px solid black;
}
div.rich_doc {
padding: 5px 10px;
border: 1px solid #dddddd;
background: white;
border-radius: 4px;
}
span.status_square {
width:10px;
height:10px;