[ 
https://issues.apache.org/jira/browse/AIRFLOW-5545?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17203997#comment-17203997
 ] 

ASF GitHub Bot commented on AIRFLOW-5545:
-----------------------------------------

dimberman commented on a change in pull request #6175:
URL: https://github.com/apache/airflow/pull/6175#discussion_r496782063



##########
File path: airflow/models/dag.py
##########
@@ -1467,22 +1467,26 @@ def _test_cycle_helper(self, visit_map, task_id):
         """
         from airflow.models.dagbag import DagBag  # Avoid circular imports
 
-        # print('Inspecting %s' % task_id)
-        if visit_map[task_id] == DagBag.CYCLE_DONE:
-            return False
-
-        visit_map[task_id] = DagBag.CYCLE_IN_PROGRESS
-
-        task = self.task_dict[task_id]
-        for descendant_id in task.get_direct_relative_ids():
-            if visit_map[descendant_id] == DagBag.CYCLE_IN_PROGRESS:
-                msg = "Cycle detected in DAG. Faulty task: {0} to {1}".format(
-                    task_id, descendant_id)
+        path, visited = [task_id], []
+
+        while path:
+            current_task_id = path.pop()
+            if visit_map[current_task_id] == DagBag.CYCLE_DONE:
+                return False
+            if visit_map[current_task_id] == DagBag.CYCLE_IN_PROGRESS:
+                msg = "Cycle detected in DAG. Faulty task: {}".format(
+                    current_task_id)
                 raise AirflowDagCycleException(msg)
-            else:
-                self._test_cycle_helper(visit_map, descendant_id)
-
-        visit_map[task_id] = DagBag.CYCLE_DONE
+            visit_map[current_task_id] = DagBag.CYCLE_IN_PROGRESS
+            task = self.task_dict[current_task_id]
+            for descendant_id in task.get_direct_relative_ids():
+                path.append(descendant_id)
+            if current_task_id in path:

Review comment:
       I ran this test on 100,000 tasks and it saved about 1/2 of a second. 
Don't really think it's worth it.




----------------------------------------------------------------
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.

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


> dag.test_cycle uses recursion, limiting DAG length.
> ---------------------------------------------------
>
>                 Key: AIRFLOW-5545
>                 URL: https://issues.apache.org/jira/browse/AIRFLOW-5545
>             Project: Apache Airflow
>          Issue Type: Bug
>          Components: DAG
>    Affects Versions: 1.10.3, 1.10.4, 1.10.5
>            Reporter: Daniel Imberman
>            Assignee: Daniel Imberman
>            Priority: Critical
>             Fix For: 2.0.0
>
>
> DAG Parsing is currently breaking for long DAGs because test_cycle uses 
> recursive DFS
> Example breaking DAG
>  
> {code:java}
> dag = DAG( 'dag', start_date=DEFAULT_DATE, default_args={'owner': 'owner1'})
> with dag: 
>     last = dag 
>      for i in range(1,10000): 
>          op = DummyOperator(task_id='{}'.format(i)) 
>          last >> op 
>          last = op
> {code}



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to