Skip to content

Commit

Permalink
Tree view, repeated nodes are now collapsed by default
Browse files Browse the repository at this point in the history
  • Loading branch information
mistercrunch committed Oct 29, 2014
1 parent b8eb9fe commit 4d20e3a
Show file tree
Hide file tree
Showing 3 changed files with 18 additions and 4 deletions.
15 changes: 14 additions & 1 deletion airflow/www/app.py
Original file line number Diff line number Diff line change
Expand Up @@ -153,7 +153,20 @@ def tree(self):
for ti in dag.get_task_instances(from_date):
task_instances[(ti.task_id, ti.execution_date)] = ti

expanded = []

def recurse_nodes(task):
children = [recurse_nodes(t) for t in task.upstream_list]

# D3 tree uses children vs _children to define what is
# expanded or not. The following block makes it such that
# repeated nodes are collapsed by default.
children_key = 'children'
if task.task_id not in expanded:
expanded.append(task.task_id)
elif children:
children_key = "_children"

return {
'name': task.task_id,
'instances': [
Expand All @@ -163,7 +176,7 @@ def recurse_nodes(task):
'task_id': task.task_id
}
for d in dates],
'children': [recurse_nodes(t) for t in task.upstream_list],
children_key: children,
'num_dep': len(task.upstream_list),
'operator': task.task_type,
'retries': task.retries,
Expand Down
2 changes: 1 addition & 1 deletion airflow/www/static/graph.css
Original file line number Diff line number Diff line change
@@ -1,5 +1,5 @@
g.node rect {
stroke-width: 4;
stroke-width: 2;
stroke: #CCC;
cursor: pointer;
}
Expand Down
5 changes: 3 additions & 2 deletions dags/examples/example2.py
Original file line number Diff line number Diff line change
Expand Up @@ -20,15 +20,16 @@
for i in range(10):
i = str(i)
task = BashOperator(
task_id='runme_'+i,
bash_command='echo "'+str(i)+': {{ ti.execution_date }}"',
task_id='runme_'+i,
bash_command='echo "'+str(i)+': {{ ti.execution_date }}"',
**default_args)
task.set_downstream(run_this)
dag.add_task(task)

task = BashOperator(task_id='also_run_this', bash_command='ls -l', **default_args)
dag.add_task(task)
task.set_downstream(run_this_last)
task.set_upstream(run_this)

#dag.tree_view()
#dag.db_merge()
Expand Down

0 comments on commit 4d20e3a

Please sign in to comment.