You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@airflow.apache.org by Chao-Han Tsai <mi...@gmail.com> on 2019/05/16 05:40:53 UTC

Re: [2.0 spring cleaning] Deprecate subdags

Hi all,

I have a WIP PR <https://github.com/apache/airflow/pull/5279/files> that
aims to make SubDagOperator to use normal scheduler instead of the backfill
scheduler and would love to have some feedbacks from you with regard to the
implementation.

The basic idea is that we create a DagRun when executing the SubDagOperator
and wait until the DagRun to finish. Airflow scheduler picks up the DagRun
and executes the tasks.

Also I have another PR (https://github.com/apache/airflow/pull/5283) that
consistently set the is_paused state for parent DAGs and all of its
subdags, which I believe is required if we want to utilize normal scheduler
to schedule tasks in SubDagOperator.

Thanks,

Chao-Han



On Mon, Apr 15, 2019 at 4:39 AM Dan Davydov <dd...@twitter.com.invalid>
wrote:

> I don't think fixing subdags to run in the scheduler is enough, although
> it's a huge improvement over the current implementation (especially the
> part that lets Subdags specify custom executors). From my experience with
> Subdags, I think what makes more sense is adding various operators to allow
> combining regular DAGs.
>
> Here are a some other issues with Subdags off the top of my head:
> - Confusing/separate UI and clearing/running semantics (e.g. tasks in the
> Subdag will not get scheduled if you clear them but not the parent
> operator)
> - Nested Subdags are hard to work with in the UI (and IIRC don't behave
> correctly but I might be wrong on this).
> - The abstraction is confusing, e.g. looking at the log for the
> SubdagOperator task can be a bit confusing as
> - Tons of custom special-case logic in the Airflow code and schemas in the
> DB to handle Subdags which have led to a lot of complexity and a constant
> source of tricky bugs and upgrade issues
> - Additional abstraction that users have to learn
>
> An alternative would be allowing combining DAGs, e.g. something like:
> dag = DAG()
> dag_task1 = Op(dag = dag)
> dag_task2 = Op(dag = dag)
>
> subdag = DAG()
> subdag_task1 = Op(dag = subdag)
> subdag_task2 = Op(dag = subdag)
>
> dag_task1 >> subdag >> dag_task2
> # Results in the following topology:
> #                  ,-> subdag_task1 ---v
> # dag_task1                                 dag_task2
> #                  '-> subdag_task2----^
>
> This is also a lot more easily composable than subdags, and provides a more
> powerful abstraction, e.g. you don't need additional boilerplate to create
> subdags such as setting up an operator.
>
> On Sat, Apr 13, 2019 at 12:14 PM Felix Uellendall <felix.uellendall@gmx.de
> >
> wrote:
>
> > -1 on deprecating subdags, because of the extra level of abstraction
> > some of you already mentioned.
> >
> > We also use subdags in production.
> >
> > For example in cases where we get json data from an API but since we
> > mostly need it to be in csv format we have a subdag like
> > /specific_ap//i_specific_endpoint/_to_s3 that has two tasks one for
> > retrieving data from the API and loading it to s3 and one for
> > transforming it into csv.
> > This has the advantage that you don't need to think of how you transform
> > the json to csv. In our case Data Analyst don't want to think about
> > that. They want to work with tabular data.
> >
> > We also have subdags that are handling API cursoring/pagination (by
> > using xcom) and merging these multiple API response data into one file.
> > So you call one Task, a subdag operator with this subdag and get only
> > what you really need - the data.
> >
> > I really like subdags and I am for improving / maybe redesigning or
> > reimplementing of subdags.
> >
> > -feluelle
> >
> > Am 13/04/2019 um 07:52 schrieb Chao-Han Tsai:
> > > +1 on keeping it.
> > >
> > > I think we should keep the SubDags as it provides a good abstraction
> > layer.
> > > It just need some love from us to fix the underlying
> > > performance/reliability issues.
> > >
> > > On Fri, Apr 12, 2019 at 12:06 PM Ash Berlin-Taylor <as...@apache.org>
> > wrote:
> > >
> > >> This is what I was thinking - the dag collector in the scheduler
> should
> > >> "just" be able to collect the tasks for subdags up to the parent dag.
> > I'd
> > >> possibly go as far as saying no DagRun object for subdags too.
> > >>
> > >> (Yes, "just" will never be that simple).
> > >>
> > >> -a
> > >>
> > >> On 12 April 2019 18:37:24 BST, Bolke de Bruin <bd...@gmail.com>
> > wrote:
> > >>> +1
> > >>>
> > >>> Sub dags should be fixed within the scheduler and run normally.
> > >>>
> > >>>
> > >>>
> > >>>
> > >>> On 12 April 2019 at 19:36:27, Feng Lu (fenglu@google.com.invalid)
> > >>> wrote:
> > >>>
> > >>> Agree with others who think SubDag should stay, we should fix the
> > >>> SubDag
> > >>> implementation but not remove the abstraction itself.
> > >>>
> > >>> On Fri, Apr 12, 2019 at 8:42 AM Chen Tong <ci...@gmail.com> wrote:
> > >>>
> > >>>> Is it possible to re-implement it in the view-level, not in operator
> > >>> level?
> > >>>> And this operator is just define a different view in GUI, that these
> > >>> tasks
> > >>>> will be collapsed into another view.
> > >>>>
> > >>>> On Fri, Apr 12, 2019 at 11:31 AM James Meickle
> > >>>> <jm...@quantopian.com.invalid> wrote:
> > >>>>
> > >>>>> I have avoided using them because of outstanding issues like the
> > >>> open
> > >>>> JIRA
> > >>>>> issues I linked above, or similar issues that I've read about on
> > >>> blog
> > >>>>> posts. If it were just GUI or UX issues I'd use them, but many
> > >>> people
> > >>>> have
> > >>>>> reported issues which affect concurrency/stability, consistency, or
> > >>>>> correctness of results. I believe that it's working for you, but
> > >>> for
> > >>> me,
> > >>>>> it's not worth the risk to build using them in my environment (even
> > >>>> though
> > >>>>> they could be handy for many of our workflows).
> > >>>>>
> > >>>>> On Fri, Apr 12, 2019 at 11:18 AM Kaxil Naik <ka...@gmail.com>
> > >>> wrote:
> > >>>>>> I have been using SubDags in production and haven't had much
> > >>> problem
> > >>>> with
> > >>>>>> it.
> > >>>>>>
> > >>>>>> Can you list the issues you had?
> > >>>>>>
> > >>>>>> Regards,
> > >>>>>> Kaxil
> > >>>>>>
> > >>>>>>
> > >>>>>> On Fri, Apr 12, 2019, 16:16 James Meickle
> > >>> <jmeickle@quantopian.com
> > >>>>>> .invalid>
> > >>>>>> wrote:
> > >>>>>>
> > >>>>>>> Given their bad reputation, would it be appropriate to
> > >>> deprecate
> > >>>>> subDAGs
> > >>>>>>> now to advertise that they're no longer considered a suitable
> > >>>>>>> implementation? If a new and better implementation is created,
> > >>> would
> > >>>> it
> > >>>>>>> even be similar enough to subDAGs that we'd want to continue to
> > >>> call
> > >>>>> the
> > >>>>>>> feature that?
> > >>>>>>>
> > >>>>>>> They feel like a "new Airflow user trap" right now - I have had
> > >>> to
> > >>>> tell
> > >>>>>> my
> > >>>>>>> team never to use them, because they seem appealing and are in
> > >>> the
> > >>>>>> official
> > >>>>>>> docs.
> > >>>>>>>
> > >>>>>>> On Fri, Apr 12, 2019 at 10:51 AM Ash Berlin-Taylor
> > >>> <as...@apache.org>
> > >>>>>> wrote:
> > >>>>>>>> I'd like to find time to fix subdags as they do provide a
> > >>> useful
> > >>>>>>>> abstraction - but I agree right now they aren't great (I
> > >>> avoid
> > >>> them
> > >>>>>>> because
> > >>>>>>>> of this)
> > >>>>>>>>
> > >>>>>>>> I have half thoughts of how to it should work, I just need to
> > >>> look
> > >>>> at
> > >>>>>> the
> > >>>>>>>> code in depth to see if that makes sense. Now 1.10.3 is out I
> > >>> might
> > >>>>>> have
> > >>>>>>> a
> > >>>>>>>> bit more time to do this.
> > >>>>>>>>
> > >>>>>>>> -ash
> > >>>>>>>>
> > >>>>>>>>> On 12 Apr 2019, at 15:48, James Meickle
> > >>> <jmeickle@quantopian.com
> > >>>>>>> .INVALID>
> > >>>>>>>> wrote:
> > >>>>>>>>> I think we should deprecate SubDAGs given the complexity
> > >>> they
> > >>> add
> > >>>>> and
> > >>>>>>> the
> > >>>>>>>>> limited usage and use cases. Or, we should invest effort in
> > >>>>>> redesigning
> > >>>>>>>>> their API and implementation. I think that having to
> > >>> account
> > >>> for
> > >>>>>>>>> subdag-introduced complexity makes Airflow's code much
> > >>> harder
> > >>> to
> > >>>>>>> maintain
> > >>>>>>>>> and buggier, looking at how many open issues there are that
> > >>>>> reference
> > >>>>>>>>> subdags (and how unrelated in topic they are):
> > >>>>>>>>>
> > >>
> >
> https://issues.apache.org/jira/browse/AIRFLOW-3292?jql=project%20%3D%20AIRFLOW%20AND%20status%20%3D%20Open%20AND%20text%20~%20%22subdag%22
> > >>>>>>>>
> > >
> >
>


-- 

Chao-Han Tsai