diff --git a/airflow/www/static/js/graph.js b/airflow/www/static/js/graph.js index 6e34cc2829fb1..715c46b3e2a8e 100644 --- a/airflow/www/static/js/graph.js +++ b/airflow/www/static/js/graph.js @@ -434,11 +434,10 @@ function handleRefresh() { // only refresh if the data has changed if (prevTis !== tis) { // eslint-disable-next-line no-global-assign - taskInstances = JSON.parse(tis); - updateNodesStates(taskInstances); + updateNodesStates(tis); // Only redraw the graph if labels have changed - const haveLabelsChanged = updateNodeLabels(nodes, taskInstances); + const haveLabelsChanged = updateNodeLabels(nodes, tis); if (haveLabelsChanged) draw(); // end refresh if all states are final diff --git a/tests/www/views/test_views_tasks.py b/tests/www/views/test_views_tasks.py index 2b6ca3c27136c..5ece885c67273 100644 --- a/tests/www/views/test_views_tasks.py +++ b/tests/www/views/test_views_tasks.py @@ -24,6 +24,7 @@ import urllib.parse from datetime import timedelta +import freezegun import pytest from airflow import settings @@ -60,30 +61,31 @@ def reset_dagruns(): @pytest.fixture(autouse=True) def init_dagruns(app, reset_dagruns): - app.dag_bag.get_dag("example_bash_operator").create_dagrun( - run_id=DEFAULT_DAGRUN, - run_type=DagRunType.SCHEDULED, - execution_date=DEFAULT_DATE, - data_interval=(DEFAULT_DATE, DEFAULT_DATE), - start_date=timezone.utcnow(), - state=State.RUNNING, - ) - app.dag_bag.get_dag("example_subdag_operator").create_dagrun( - run_id=DEFAULT_DAGRUN, - run_type=DagRunType.SCHEDULED, - execution_date=DEFAULT_DATE, - data_interval=(DEFAULT_DATE, DEFAULT_DATE), - start_date=timezone.utcnow(), - state=State.RUNNING, - ) - app.dag_bag.get_dag("example_xcom").create_dagrun( - run_id=DEFAULT_DAGRUN, - run_type=DagRunType.SCHEDULED, - execution_date=DEFAULT_DATE, - data_interval=(DEFAULT_DATE, DEFAULT_DATE), - start_date=timezone.utcnow(), - state=State.RUNNING, - ) + with freezegun.freeze_time(DEFAULT_DATE): + app.dag_bag.get_dag("example_bash_operator").create_dagrun( + run_id=DEFAULT_DAGRUN, + run_type=DagRunType.SCHEDULED, + execution_date=DEFAULT_DATE, + data_interval=(DEFAULT_DATE, DEFAULT_DATE), + start_date=timezone.utcnow(), + state=State.RUNNING, + ) + app.dag_bag.get_dag("example_subdag_operator").create_dagrun( + run_id=DEFAULT_DAGRUN, + run_type=DagRunType.SCHEDULED, + execution_date=DEFAULT_DATE, + data_interval=(DEFAULT_DATE, DEFAULT_DATE), + start_date=timezone.utcnow(), + state=State.RUNNING, + ) + app.dag_bag.get_dag("example_xcom").create_dagrun( + run_id=DEFAULT_DAGRUN, + run_type=DagRunType.SCHEDULED, + execution_date=DEFAULT_DATE, + data_interval=(DEFAULT_DATE, DEFAULT_DATE), + start_date=timezone.utcnow(), + state=State.RUNNING, + ) yield clear_db_runs() @@ -993,3 +995,217 @@ def test_graph_view_doesnt_fail_on_recursion_error(app, dag_maker, admin_client) url = f'/dags/{dag.dag_id}/graph' resp = admin_client.get(url, follow_redirects=True) assert resp.status_code == 200 + + +def test_task_instances(admin_client): + """Test task_instances view.""" + resp = admin_client.get( + f'/object/task_instances?dag_id=example_bash_operator&execution_date={DEFAULT_DATE}', + follow_redirects=True, + ) + assert resp.status_code == 200 + assert resp.json == { + 'also_run_this': { + 'dag_id': 'example_bash_operator', + 'duration': None, + 'end_date': None, + 'executor_config': {}, + 'external_executor_id': None, + 'hostname': '', + 'job_id': None, + 'map_index': -1, + 'max_tries': 0, + 'next_kwargs': None, + 'next_method': None, + 'operator': 'BashOperator', + 'pid': None, + 'pool': 'default_pool', + 'pool_slots': 1, + 'priority_weight': 2, + 'queue': 'default', + 'queued_by_job_id': None, + 'queued_dttm': None, + 'run_id': 'TEST_DAGRUN', + 'start_date': None, + 'state': None, + 'task_id': 'also_run_this', + 'trigger_id': None, + 'trigger_timeout': None, + 'try_number': 1, + 'unixname': 'root', + }, + 'run_after_loop': { + 'dag_id': 'example_bash_operator', + 'duration': None, + 'end_date': None, + 'executor_config': {}, + 'external_executor_id': None, + 'hostname': '', + 'job_id': None, + 'map_index': -1, + 'max_tries': 0, + 'next_kwargs': None, + 'next_method': None, + 'operator': 'BashOperator', + 'pid': None, + 'pool': 'default_pool', + 'pool_slots': 1, + 'priority_weight': 2, + 'queue': 'default', + 'queued_by_job_id': None, + 'queued_dttm': None, + 'run_id': 'TEST_DAGRUN', + 'start_date': None, + 'state': None, + 'task_id': 'run_after_loop', + 'trigger_id': None, + 'trigger_timeout': None, + 'try_number': 1, + 'unixname': 'root', + }, + 'run_this_last': { + 'dag_id': 'example_bash_operator', + 'duration': None, + 'end_date': None, + 'executor_config': {}, + 'external_executor_id': None, + 'hostname': '', + 'job_id': None, + 'map_index': -1, + 'max_tries': 0, + 'next_kwargs': None, + 'next_method': None, + 'operator': 'EmptyOperator', + 'pid': None, + 'pool': 'default_pool', + 'pool_slots': 1, + 'priority_weight': 1, + 'queue': 'default', + 'queued_by_job_id': None, + 'queued_dttm': None, + 'run_id': 'TEST_DAGRUN', + 'start_date': None, + 'state': None, + 'task_id': 'run_this_last', + 'trigger_id': None, + 'trigger_timeout': None, + 'try_number': 1, + 'unixname': 'root', + }, + 'runme_0': { + 'dag_id': 'example_bash_operator', + 'duration': None, + 'end_date': None, + 'executor_config': {}, + 'external_executor_id': None, + 'hostname': '', + 'job_id': None, + 'map_index': -1, + 'max_tries': 0, + 'next_kwargs': None, + 'next_method': None, + 'operator': 'BashOperator', + 'pid': None, + 'pool': 'default_pool', + 'pool_slots': 1, + 'priority_weight': 3, + 'queue': 'default', + 'queued_by_job_id': None, + 'queued_dttm': None, + 'run_id': 'TEST_DAGRUN', + 'start_date': None, + 'state': None, + 'task_id': 'runme_0', + 'trigger_id': None, + 'trigger_timeout': None, + 'try_number': 1, + 'unixname': 'root', + }, + 'runme_1': { + 'dag_id': 'example_bash_operator', + 'duration': None, + 'end_date': None, + 'executor_config': {}, + 'external_executor_id': None, + 'hostname': '', + 'job_id': None, + 'map_index': -1, + 'max_tries': 0, + 'next_kwargs': None, + 'next_method': None, + 'operator': 'BashOperator', + 'pid': None, + 'pool': 'default_pool', + 'pool_slots': 1, + 'priority_weight': 3, + 'queue': 'default', + 'queued_by_job_id': None, + 'queued_dttm': None, + 'run_id': 'TEST_DAGRUN', + 'start_date': None, + 'state': None, + 'task_id': 'runme_1', + 'trigger_id': None, + 'trigger_timeout': None, + 'try_number': 1, + 'unixname': 'root', + }, + 'runme_2': { + 'dag_id': 'example_bash_operator', + 'duration': None, + 'end_date': None, + 'executor_config': {}, + 'external_executor_id': None, + 'hostname': '', + 'job_id': None, + 'map_index': -1, + 'max_tries': 0, + 'next_kwargs': None, + 'next_method': None, + 'operator': 'BashOperator', + 'pid': None, + 'pool': 'default_pool', + 'pool_slots': 1, + 'priority_weight': 3, + 'queue': 'default', + 'queued_by_job_id': None, + 'queued_dttm': None, + 'run_id': 'TEST_DAGRUN', + 'start_date': None, + 'state': None, + 'task_id': 'runme_2', + 'trigger_id': None, + 'trigger_timeout': None, + 'try_number': 1, + 'unixname': 'root', + }, + 'this_will_skip': { + 'dag_id': 'example_bash_operator', + 'duration': None, + 'end_date': None, + 'executor_config': {}, + 'external_executor_id': None, + 'hostname': '', + 'job_id': None, + 'map_index': -1, + 'max_tries': 0, + 'next_kwargs': None, + 'next_method': None, + 'operator': 'BashOperator', + 'pid': None, + 'pool': 'default_pool', + 'pool_slots': 1, + 'priority_weight': 2, + 'queue': 'default', + 'queued_by_job_id': None, + 'queued_dttm': None, + 'run_id': 'TEST_DAGRUN', + 'start_date': None, + 'state': None, + 'task_id': 'this_will_skip', + 'trigger_id': None, + 'trigger_timeout': None, + 'try_number': 1, + 'unixname': 'root', + }, + }