What is your dynamic DAG doing? How long does it take to execute it just as
a python script (`time python mydag.py`)?

As an Airflow admin, people may want to lower the DAG parsing timeout
configuration key to force people to not do crazy thing in DAG module
scope. At some point at Airbnb we had someone running a Hive query in DAG
scope, clearly that needs to be prevented.

Loading DAGs by calling a database can bring all sorts of surprises that
can drive everyone crazy. As mentioned in a recent post, repo-contained,
deterministic "less dynamic" DAGs are great, because they are
self-contained and allow you to use source-control properly (revert a bad
change for instance). That may mean having a process or script that
compiles external things that are dynamic into things like yaml files
checked into the code repo. Things as simple as parsing duration become
more predictable (network latency and database load are not part of that
equation), but more importantly, all changes become tracked in the code
repo.

yaml parsing in python can be pretty slow too, and there are solutions /
alternatives there. Hocon is great. Also C-accelerated yaml is possible:
https://stackoverflow.com/questions/27743711/can-i-speedup-yaml

Max

On Wed, Aug 14, 2019 at 9:56 PM Bacal, Eugene <eba...@paypal.com.invalid>
wrote:

> Hello Airflow team,
>
> Please advise if you can. In our environment, we have noticed that dynamic
> tasks place quite of stress on scheduler, webserver and increase MySQL DB
> connections.
> We are run about 1000 Dynamic Tasks every 30 min and parsing time
> increases from 5 to 65 sec with Runtime from 2sec to 350+ . This happens at
> execution time then it drops to normal while still executing tasks.
> Webserver hangs for few minutes.
>
> Airflow 1.10.1.
> MySQL DB
>
> Example:
>
> Dynamic Tasks:
> Number of DAGs: 44
> Total task number: 950
> DagBag parsing time: 65.879642000000001
>
> Static Tasks:
> Number of DAGs: 73
> Total task number: 1351
> DagBag parsing time: 1.731088
>
> Is this something you aware of? Any advises on Dynamic tasks
> optimization/best practices?
>
> Thank you in advance,
> Eugene
>
>
>

Reply via email to