airflow-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "ASF subversion and git services (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (AIRFLOW-1614) Improve performance of DAG parsing when there are many subdags
Date Sat, 16 Sep 2017 15:23:02 GMT

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

ASF subversion and git services commented on AIRFLOW-1614:
----------------------------------------------------------

Commit a95adbb8cea92434405cfaa96c422605b6df6c98 in incubator-airflow's branch refs/heads/v1-9-test
from [~gbenison]
[ https://git-wip-us.apache.org/repos/asf?p=incubator-airflow.git;h=a95adbb ]

[AIRFLOW-1614] Replace inspect.stack() with sys._getframe()

inspect.stack() is really expensive, and slows
down processing of dags
having large numbers (100s, 1000s) of subdags.

Closes #2610 from gbenison/gcbenison2

(cherry picked from commit 558198e1b7a6f2839498987a9692b4c5d421fb8f)
Signed-off-by: Bolke de Bruin <bolke@xs4all.nl>


> Improve performance of DAG parsing when there are many subdags
> --------------------------------------------------------------
>
>                 Key: AIRFLOW-1614
>                 URL: https://issues.apache.org/jira/browse/AIRFLOW-1614
>             Project: Apache Airflow
>          Issue Type: Improvement
>            Reporter: Gregory Benison
>
> DAGs can be very slow to parse when they contain many (100s or 1000s) of subdags.  This
can be illustrated using the following DAG definition file:
> {code}from datetime import datetime, timedelta
> from airflow.models import DAG
> from airflow.operators.dummy_operator import DummyOperator
> from airflow.operators.subdag_operator import SubDagOperator
> dag = DAG(
>     'subdaggy-2',
>     schedule_interval=None,
>     start_date=datetime(2017,1,1)
> )
> def make_sub_dag(parent_dag, N):
>     dag = DAG(
>         '%s.task_%d' % (parent_dag.dag_id, N),
>         schedule_interval=parent_dag.schedule_interval,
>         start_date=parent_dag.start_date
>         )
>     DummyOperator(task_id='task1', dag=dag) >> DummyOperator(task_id='task2', dag=dag)
>     return dag
> downstream_task = DummyOperator(task_id='downstream', dag=dag)
> for N in range(20):
>     SubDagOperator(
>         dag=dag,
>         task_id='task_%d' % N,
>         subdag=make_sub_dag(dag, N)
>         ) >> downstream_task
> {code}
> When there are more than 50 or so subdags this file becomes slow enough to parse that
it fails to load in the web UI on a modest platform such as a laptop.
> It would be nice to support such DAGs, since there are useful workflows involving 100s
or 1000s of subdags.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

Mime
View raw message