Gollum999 opened a new issue, #33007:
URL: https://github.com/apache/airflow/issues/33007

   ### Apache Airflow version
   
   2.6.3
   
   ### What happened
   
   If you hook up a `TaskGroup`-level dependency before all tasks have been 
added to the group, the DAG will appear correct on the Graph page, but the 
dependencies will actually be missing.  In other words, the Graph page lies to 
the user, so it appears that downstream tasks are running before their 
dependencies are complete, when in reality those dependencies don't even exist.
   
   ### What you think should happen instead
   
   Most importantly, the Graph page should accurately reflect the DAG's actual 
dependencies.  
   
   For the specific behavior around `TaskGroup` dependencies, I think any of 
these options is acceptable, in descending order of my personal preference:
   
   1. Group dependencies should work regardless of where the dependency is 
declared (obviously within the limits of Python)
   2. Keep the same behavior where adding tasks to the group "late" causes 
those dependencies to be missed, but update the Graph page to not show false 
dependencies
   3. Adding tasks "late" could result in a warning or error when the DAG is 
parsed
   
   ### How to reproduce
   
   ```
   #!/usr/bin/env python3
   import datetime
   import logging
   import time
   
   from airflow.decorators import dag, task
   from airflow.utils.task_group import TaskGroup
   
   
   logger = logging.getLogger(__name__)
   
   
   @dag(
       schedule=None,
       start_date=datetime.datetime(2023, 8, 1),
       default_args={
           'retries': 0,
       },
   )
   def test_group_deps():
       @task
       def upstream_task():
           time.sleep(30)
           raise RuntimeError('fail')
   
       @task
       def downstream_task():
           pass
   
       # hooking up group dependency outside of the group block
       with TaskGroup(group_id='group_dep_after_group'):
           # task -> group
           with TaskGroup(group_id='upstream1'):
               t = upstream_task()
           with TaskGroup(group_id='downstream1') as g:
               downstream_task()
           t >> g
   
           # group -> task
           with TaskGroup(group_id='downstream2'):
               t = downstream_task()
           with TaskGroup(group_id='upstream2') as g:
               upstream_task()
           g >> t
   
           # group -> group
           with TaskGroup(group_id='upstream3') as g_up:
               upstream_task()
           with TaskGroup(group_id='downstream3') as g_down:
               downstream_task()
           g_up >> g_down
   
       # hooking up group dependency inside of the group block, after tasks 
have been defined in the group
       with TaskGroup(group_id='group_dep_after_tasks_added'):
           # task -> group
           with TaskGroup(group_id='upstream1'):
               t = upstream_task()
           with TaskGroup(group_id='downstream1') as g:
               downstream_task()
               t >> g
   
           # group -> task
           with TaskGroup(group_id='downstream2'):
               t = downstream_task()
           with TaskGroup(group_id='upstream2') as g:
               upstream_task()
               g >> t
   
           # group -> group
           with TaskGroup(group_id='upstream3') as g_up:
               upstream_task()
           with TaskGroup(group_id='downstream3') as g_down:
               downstream_task()
               g_up >> g_down
   
       # hooking up group dependency inside of the group block, before tasks 
have been defined in the group
       with TaskGroup(group_id='group_dep_before_tasks_added'):
           # task -> group
           with TaskGroup(group_id='upstream1'):
               t = upstream_task()
           with TaskGroup(group_id='downstream1') as g:
               t >> g
               downstream_task()
   
           # group -> task
           with TaskGroup(group_id='downstream2'):
               t = downstream_task()
           with TaskGroup(group_id='upstream2') as g:
               g >> t
               upstream_task()
   
           # group -> group
           with TaskGroup(group_id='upstream3') as g_up:
               upstream_task()
           with TaskGroup(group_id='downstream3') as g_down:
               g_up >> g_down
               downstream_task()
   
   
   dag = test_group_deps()
   
   
   if __name__ == '__main__':
       dag.cli()
   ```
   
   If you run this DAG, you can see that all of the downstream tasks in the 
`group_dep_before_tasks_added` group start running before their dependencies 
are met:
   
   ![downstream tasks run and complete before their 
dependencies](https://i.imgur.com/ska05yS.png)
   
   The "Filter Upstream" option also reveals that there are issues:
   
   ![downstream task missing its upstream 
dependencies](https://i.imgur.com/Ofro2XY.png)
   
   ### Operating System
   
   CentOS Stream 8
   
   ### Versions of Apache Airflow Providers
   
   N/A
   
   ### Deployment
   
   Other
   
   ### Deployment details
   
   I reproduced this bug in our self-hosted Airflow cluster as well as in 
Standalone mode.
   
   ### Anything else
   
   This also seems to affect the new Graph tab on the Grid page:
   
   ![new graph tab also lies about the 
dependency](https://i.imgur.com/B2qcaoO.png)
   
   Some of the dependency traces on that graph do not draw correctly (there are 
lines "floating" off to the side), but I'm not sure if that is related to this 
bug or if that is a separate issue.
   
   ### Are you willing to submit PR?
   
   - [ ] Yes I am willing to submit a PR!
   
   ### Code of Conduct
   
   - [X] I agree to follow this project's [Code of 
Conduct](https://github.com/apache/airflow/blob/main/CODE_OF_CONDUCT.md)
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to