Skip to content

Commit

Permalink
[AIRFLOW-7017] Respect default dag view in trigger dag origin (#7667)
Browse files Browse the repository at this point in the history
  • Loading branch information
kaxil committed Mar 31, 2020
1 parent 8f25d8d commit 4345868
Show file tree
Hide file tree
Showing 3 changed files with 67 additions and 10 deletions.
2 changes: 1 addition & 1 deletion airflow/www_rbac/templates/airflow/dag.html
Original file line number Diff line number Diff line change
Expand Up @@ -101,7 +101,7 @@ <h4 class="pull-right">
</a>
</li>
<li>
<a href="{{ url_for('Airflow.trigger', dag_id=dag.dag_id, origin=url_for('Airflow.tree', dag_id=dag.dag_id)) }}">
<a href="{{ url_for('Airflow.trigger', dag_id=dag.dag_id, origin=url_for('Airflow.' + dag.get_default_view(), dag_id=dag.dag_id)) }}">
<span class="glyphicon glyphicon-play-circle" aria-hidden="true"></span>
Trigger DAG
</a>
Expand Down
37 changes: 37 additions & 0 deletions tests/dags/test_default_views.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,37 @@
#
# Licensed to the Apache Software Foundation (ASF) under one
# or more contributor license agreements. See the NOTICE file
# distributed with this work for additional information
# regarding copyright ownership. The ASF licenses this file
# to you under the Apache License, Version 2.0 (the
# "License"); you may not use this file except in compliance
# with the License. You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing,
# software distributed under the License is distributed on an
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
# KIND, either express or implied. See the License for the
# specific language governing permissions and limitations
# under the License.
from airflow.models import DAG
from airflow.utils.dates import days_ago

args = {
'owner': 'airflow',
'retries': 3,
'start_date': days_ago(2)
}

tree_dag = DAG(
dag_id='test_tree_view', default_args=args,
schedule_interval='0 0 * * *',
default_view='tree',
)

graph_dag = DAG(
dag_id='test_graph_view', default_args=args,
schedule_interval='0 0 * * *',
default_view='graph',
)
38 changes: 29 additions & 9 deletions tests/www_rbac/test_views.py
Original file line number Diff line number Diff line change
Expand Up @@ -369,8 +369,8 @@ class TestAirflowBaseViews(TestBase):
@classmethod
def setUpClass(cls):
super(TestAirflowBaseViews, cls).setUpClass()
dagbag = models.DagBag(include_examples=True)
for dag in dagbag.dags.values():
cls.dagbag = models.DagBag(include_examples=True)
for dag in cls.dagbag.dags.values():
dag.sync_to_db()

def setUp(self):
Expand All @@ -393,13 +393,9 @@ def cleanup_dagruns(self):
self.session.commit()

def prepare_dagruns(self):
dagbag = models.DagBag(include_examples=True)
self.bash_dag = dagbag.dags['example_bash_operator']
self.bash_dag.sync_to_db()
self.sub_dag = dagbag.dags['example_subdag_operator']
self.sub_dag.sync_to_db()
self.xcom_dag = dagbag.dags['example_xcom']
self.xcom_dag.sync_to_db()
self.bash_dag = self.dagbag.dags['example_bash_operator']
self.sub_dag = self.dagbag.dags['example_subdag_operator']
self.xcom_dag = self.dagbag.dags['example_xcom']

self.bash_dagrun = self.bash_dag.create_dagrun(
run_id=self.run_id,
Expand Down Expand Up @@ -552,6 +548,30 @@ def test_dag_details(self):
resp = self.client.get(url, follow_redirects=True)
self.check_content_in_response('DAG details', resp)

def test_dag_details_trigger_origin_tree_view(self):
dag = self.dagbag.dags['test_tree_view']
dag.create_dagrun(
run_id=self.run_id,
execution_date=self.EXAMPLE_DAG_DEFAULT_DATE,
start_date=timezone.utcnow(),
state=State.RUNNING)

url = 'dag_details?dag_id=test_tree_view'
resp = self.client.get(url, follow_redirects=True)
self.check_content_in_response('origin=%2Ftree%3Fdag_id%3Dtest_tree_view', resp)

def test_dag_details_trigger_origin_graph_view(self):
dag = self.dagbag.dags['test_graph_view']
dag.create_dagrun(
run_id=self.run_id,
execution_date=self.EXAMPLE_DAG_DEFAULT_DATE,
start_date=timezone.utcnow(),
state=State.RUNNING)

url = 'dag_details?dag_id=test_graph_view'
resp = self.client.get(url, follow_redirects=True)
self.check_content_in_response('origin=%2Fgraph%3Fdag_id%3Dtest_graph_view', resp)

def test_dag_details_subdag(self):
url = 'dag_details?dag_id=example_subdag_operator.section-1'
resp = self.client.get(url, follow_redirects=True)
Expand Down

0 comments on commit 4345868

Please sign in to comment.