airflow-dev mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Maxime Beauchemin <maximebeauche...@gmail.com>
Subject Re: DAG Factory Issues
Date Thu, 07 Dec 2017 04:43:19 GMT
Side note: if I could redesign this I'd move away from the DagBag's
`os.walk` approach and have more explicit as in `dagbag.add(my_dag)`.
Meaning you'd have some central module where you'd import all of your DAGs
explicitely.

Instead of DAGS_FOLDER in config you'd have something like DAGBAG_OBJECT =
'some_module.some_dagbag' (referencing the object's python path instead of
the file system). I guess you could still have a
`DagBag.os_walk_load_dags(folder)` static method to emulate the current
default behavior if you prefer that.

Another tangent is around the challenge of `sys.modules` caching in Python
and how reloading a module doesn't re-evaluate the code, which makes
working in module scope a bad idea for anything dynamic. In those cases
it'd be better for the DagBag to keep a pointer to a DAG generator function
than a reference to a DAG object itself. If the DAG is dynamic you'd pass a
DAG generator function to the DagBag so it would know to re-evaluate it.

The current solutions for DAG re-evaluation aren't perfect: on the
scheduler - DAG are evaluated in a subprocess (multiprocessing queue) based
on the fileloc passed from the main process, and - on the web server - to
force reload modules (which has some caveats where modules imported by
those modules aren't reloaded) or to configure gunicorn to constantly
rotate workers.

Max

On Tue, Dec 5, 2017 at 2:34 PM, Alek Storm <alek.storm@gmail.com> wrote:

> Sorry I wasn’t clear; I just meant that it seems more useful for fileloc to
> reflect the file in the dags folder that the scheduler processed to yield
> the DAG
> <https://github.com/apache/incubator-airflow/blob/
> 1359d87352bda220f5d88613fd81904378624c7b/airflow/jobs.py#L1710>
> .
>
> Actually, the full_filepath attribute is already exactly this, so I suppose
> I’m advocating using full_filepath instead of fileloc in all cases.
>
> Alek
> ​
>
> On Tue, Dec 5, 2017 at 1:12 PM, Bolke de Bruin <bdbruin@gmail.com> wrote:
>
> > I dont see how the scheduler would know. DAGs can come from any location,
> > any module, and import is dependent on the python interpreter.
> >
> > If you know a way the suits every kind of structure please provide a pr.
> >
> > Cheers
> > Bolke
> >
> > Verstuurd vanaf mijn iPad
> >
> > > Op 5 dec. 2017 om 19:05 heeft Alek Storm <alek.storm@gmail.com> het
> > volgende geschreven:
> > >
> > > We solved this (hackily) by setting the fileloc field on the generated
> > DAG
> > > object to inspect.getsourcefile(inspect.stack()[1][0]), as the DAG
> > > constructor itself does. I agree a more general solution is needed;
> > > presumably the scheduler knows which Python file in the dags folder it
> > was
> > > processing when it found the DAG object.
> > >
> > > Alek
> > > ​
> > >
> > >> On Tue, Dec 5, 2017 at 12:00 PM, Michael Erdely <mjerdely@gmail.com>
> > wrote:
> > >>
> > >> Hi,
> > >>
> > >> In order to support multiple environments with different DAG settings
> > per
> > >> environment, we created a DAG factory to create the DAG operator where
> > each
> > >> DAG has different params (eg schedule, catchup, etc).
> > >>
> > >> Unfortunately, we noticed that the webserver code view shows the
> factory
> > >> code versus the actual Python code in the dagbag file. I also assume
> > that
> > >> the factory file modified date will be used versus the DAG file to
> > >> determine if the scheduler should reload.
> > >>
> > >> I see this issue has already been reported here
> > >> https://issues.apache.org/jira/browse/AIRFLOW-1108.
> > >>
> > >> Any ideas if this will be patched soon or have better suggestions on
> > >> handling this?
> > >>
> > >> -Michael
> > >>
> >
>

Mime
  • Unnamed multipart/alternative (inline, None, 0 bytes)
View raw message