Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Applying labels to task groups shows a cycle in the graph view for the dag #19785

Closed
1 of 2 tasks
rustikk opened this issue Nov 23, 2021 · 12 comments · Fixed by #24847
Closed
1 of 2 tasks

Applying labels to task groups shows a cycle in the graph view for the dag #19785

rustikk opened this issue Nov 23, 2021 · 12 comments · Fixed by #24847
Labels
affected_version:2.2 Issues Reported for 2.2 area:core area:UI Related to UI/UX. For Frontend Developers. kind:bug This is a clearly a bug
Milestone

Comments

@rustikk
Copy link
Contributor

rustikk commented Nov 23, 2021

Apache Airflow version

2.2.2

Operating System

Docker (debian:buster)

Versions of Apache Airflow Providers

N/A

Deployment

Astronomer

Deployment details

run airflow with this dag

with DAG(
    dag_id="label_bug_without_chain"
) as dag:

    with TaskGroup(group_id="group1") as taskgroup1:
        t1 = DummyOperator(task_id="dummy1")
        t2 = DummyOperator(task_id="dummy2")
        t3 = DummyOperator(task_id="dummy3")
    
    t4 = DummyOperator(task_id="dummy4")

chain([Label("branch three")], taskgroup1, t4,)

What happened

expanded task views look like they have cycles

Screen Shot 2021-11-22 at 2 33 49 PM

What you expected to happen

The task group shouldn't display as if it has loops in it.

How to reproduce

View the dag shown in the deployment details.

Anything else

No response

Are you willing to submit PR?

  • Yes I am willing to submit a PR!

Code of Conduct

@rustikk rustikk added area:core kind:bug This is a clearly a bug labels Nov 23, 2021
@boring-cyborg
Copy link

boring-cyborg bot commented Nov 23, 2021

Thanks for opening your first issue here! Be sure to follow the issue template!

@eladkal eladkal added area:UI Related to UI/UX. For Frontend Developers. affected_version:2.2 Issues Reported for 2.2 labels Nov 23, 2021
@eladkal eladkal added this to the Airflow 2.2.3 milestone Nov 23, 2021
@eladkal
Copy link
Contributor

eladkal commented Nov 23, 2021

cc @bbovenzi

@bbovenzi
Copy link
Contributor

bbovenzi commented Dec 2, 2021

Related: #19899

I think we will remove the blue dots entirely as they seem to cause confusion and weird issues.

@eladkal
Copy link
Contributor

eladkal commented Jul 4, 2022

@bbovenzi I tried to reproduce this issue and It seems like now this DAG is breaking the Grid view:
Screen Shot 2022-07-04 at 18 35 18

Removing the line: chain([Label("branch three")], taskgroup1, t4,)
makes the Grid view works:
Screen Shot 2022-07-04 at 18 38 37

Code:

from datetime import datetime

from airflow.models.baseoperator import chain
from airflow.operators.dummy import DummyOperator
from airflow.utils.edgemodifier import Label
from airflow.utils.task_group import TaskGroup
from airflow import DAG


DEFAULT_ARGS = {
    'owner': 'admin',
    'depends_on_past': False,
    'start_date': datetime(2022, 5, 20, 0),
    'retries': 2
}

with DAG(
    dag_id="label_bug_without_chain",
    default_args=DEFAULT_ARGS,
) as dag:

    with TaskGroup(group_id="group1") as taskgroup1:
        t1 = DummyOperator(task_id="dummy1")
        t2 = DummyOperator(task_id="dummy2")
        t3 = DummyOperator(task_id="dummy3")

    t4 = DummyOperator(task_id="dummy4")

chain([Label("branch three")], taskgroup1, t4)

@ashb
Copy link
Member

ashb commented Jul 4, 2022

Is there a stack trace in the webserver logs? (It looks like the API call failed with a 500)

@ephraimbuddy
Copy link
Contributor

For the above dag, writing the chain as below works:

chain([Label("branch three")], [taskgroup1], t4)

or

chain(Label("branch three"), taskgroup1, t4)

@ephraimbuddy
Copy link
Contributor

Is there a stack trace in the webserver logs? (It looks like the API call failed with a 500)

Yes. Here's the stacktrace:

[2022-07-04 21:18:55+0000 929ms] {app.py:1456} ERROR - Exception on /object/grid_data [GET]
Traceback (most recent call last):
  File "/usr/local/lib/python3.7/site-packages/flask/app.py", line 2077, in wsgi_app
    response = self.full_dispatch_request()
  File "/usr/local/lib/python3.7/site-packages/flask/app.py", line 1525, in full_dispatch_request
    rv = self.handle_user_exception(e)
  File "/usr/local/lib/python3.7/site-packages/flask/app.py", line 1523, in full_dispatch_request
    rv = self.dispatch_request()
  File "/usr/local/lib/python3.7/site-packages/flask/app.py", line 1509, in dispatch_request
    return self.ensure_sync(self.view_functions[rule.endpoint])(**req.view_args)
  File "/opt/airflow/airflow/www/auth.py", line 46, in decorated
    return func(*args, **kwargs)
  File "/opt/airflow/airflow/www/views.py", line 3628, in grid_data
    'groups': dag_to_grid(dag, dag_runs, session),
  File "/opt/airflow/airflow/www/views.py", line 409, in dag_to_grid
    return task_group_to_grid(dag.task_group, dag_runs, grouped_tis)
  File "/opt/airflow/airflow/www/views.py", line 361, in task_group_to_grid
    task_group_to_grid(child, dag_runs, grouped_tis) for child in task_group.topological_sort()
  File "/opt/airflow/airflow/utils/task_group.py", line 450, in topological_sort
    raise AirflowDagCycleException(f"A cyclic dependency occurred in dag: {self.dag_id}")
airflow.exceptions.AirflowDagCycleException: A cyclic dependency occurred in dag: label_bug_without_chain

@ashb
Copy link
Member

ashb commented Jul 5, 2022

I'm not sure this is a bug -- or at least not a bug in the UI.

On 2.2.5:

In [1]: import airflow

In [2]: from datetime import datetime
   ...: 
   ...: from airflow.models.baseoperator import chain
   ...: from airflow.operators.dummy import DummyOperator
   ...: from airflow.utils.edgemodifier import Label
   ...: from airflow.utils.task_group import TaskGroup
   ...: from airflow import DAG
   ...: 
   ...: 
   ...: DEFAULT_ARGS = {
   ...:     'owner': 'admin',
   ...:     'depends_on_past': False,
   ...:     'start_date': datetime(2022, 5, 20, 0),
   ...:     'retries': 2
   ...: }
   ...: 
   ...: with DAG(
   ...:     dag_id="label_bug_without_chain",
   ...:     default_args=DEFAULT_ARGS,
   ...: ) as dag:
   ...: 
   ...:     with TaskGroup(group_id="group1") as taskgroup1:
   ...:         t1 = DummyOperator(task_id="dummy1")
   ...:         t2 = DummyOperator(task_id="dummy2")
   ...:         t3 = DummyOperator(task_id="dummy3")
   ...: 
   ...:     t4 = DummyOperator(task_id="dummy4")
   ...: 
   ...: chain([Label("branch three")], taskgroup1, t4)

In [3]: taskgroup1.upstream_task_ids
Out[3]: {'group1.dummy1', 'group1.dummy2', 'group1.dummy3'}

The task group is marked as depending on itself! The bug is that we don't detect the cycle earlier during the DAG parsing process!

@ashb
Copy link
Member

ashb commented Jul 5, 2022

Not to mention this DAG is invalid -- a label must be between two nodes, it can't be at the start of one.

@eladkal
Copy link
Contributor

eladkal commented Jul 5, 2022

But the end result of Grid view crash is not desired.
If not valid state why it doesn't raise broken dag? maybe this is the issue to address if it results in broken dag then we won't have an issue with the grid view.

@ashb
Copy link
Member

ashb commented Jul 5, 2022

The example code given in the OP doesn't quite match the picture they showed (missing the start task)

@ashb
Copy link
Member

ashb commented Jul 5, 2022

from datetime import datetime

from airflow.models.baseoperator import chain
from airflow.operators.dummy import DummyOperator
from airflow.utils.edgemodifier import Label
from airflow.utils.task_group import TaskGroup
from airflow import DAG


DEFAULT_ARGS = {
    'depends_on_past': False,
    'retries': 2
}

with DAG(
    dag_id="label_bug_without_chain",
    start_date=datetime(2022, 5, 20, 0),
    default_args=DEFAULT_ARGS,
) as dag:

    start = DummyOperator(task_id="sleep_3_seconds")

    with TaskGroup(group_id="group1") as taskgroup1:
        t1 = DummyOperator(task_id="dummy1")
        t2 = DummyOperator(task_id="dummy2")
        t3 = DummyOperator(task_id="dummy3")

    t4 = DummyOperator(task_id="echo_done")

chain(start, [Label("branch three")], taskgroup1, t4)

The issue is still shows up with this dag though.

@eladkal eladkal removed this from the Airflow 2.3.4 milestone Jul 5, 2022
@ashb ashb added this to the Airflow 2.3.3 milestone Jul 6, 2022
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
affected_version:2.2 Issues Reported for 2.2 area:core area:UI Related to UI/UX. For Frontend Developers. kind:bug This is a clearly a bug
Projects
None yet
Development

Successfully merging a pull request may close this issue.

6 participants