You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@airflow.apache.org by Daniel Standish <da...@astronomer.io.INVALID> on 2023/03/23 06:57:38 UTC

[DISCUSS] AIP-52 updates - setup / teardown tasks

I’m part of a group working on the implementation of AIP-52.  We would like
to update the community on some changes to the implementation approach, the
planned roadmap, and give an opportunity to provide feedback.

First though, let’s recap briefly what are the main benefits of adding
setup and teardown as concepts in Airflow:

   -

   By separating setup and teardown from "work" tasks, after the failure of
   a work task, we can stop the dag from proceeding (i.e. onto subsequent work
   tasks) while still allowing needed teardown operations to proceed (e.g.
   deleting a cluster).
   -

   This separation also lets us optionally *not* fail a dag run when
   perhaps the important work was completed successfully but a cleanup or
   teardown operation failed.
   -

   By associating work and setup tasks, we can clear the setups (and their
   respective teardowns) when clearing the work tasks.


After experimenting with some different implementation approaches and
reviewing and writing a lot of example dags, we developed some conclusions
that caused us to change course somewhat, while still fulfilling the
principal goals of the AIP.

Perhaps most importantly, we believe it is essential that our design
choices leave room for multiple setup and teardown tasks in a given task
group or dag.  Dags don’t tend to do just one thing.  In a dag there could
be many tasks requiring their own “setup” and “teardown”.  Similarly, a
single “work” task may itself require multiple “setup” and teardown tasks.
For obvious reasons, combining the work of multiple operators into a single
task is not advisable.  And, requiring a new task group for each thing
requiring a setup also has pitfalls: it conflicts with the task group’s use
case as an arbitrary logical grouping of tasks, and as a task mapping
tool.  So we believe it will be necessary to be able to support multiple
setups within a group, and moreover we believe it will be necessary to be
able to set dependencies between them.

With that in mind, the main change we’d like to share is that we now
require that users must specify the relationship between setup/teardown
tasks and “normal” tasks.  *(In the original proposal, users were not
required to set relationships between setup/teardown tasks and the other
tasks in the group.)*

So in the original AIP you could do this:

with TaskGroup("group1") as tg1:

   setup1 = my_setup("g1_setup") # a setup task

   work1 = my_work("g1_work1")
  work2 = my_work("g1_work2")
  work1 >> work2

   teardown1 = my_teardown("g1_teardown") # a teardown task

Then in effect you’d get setup1 >> work1 >> work2 >> teardown1.

Now we require you to set those relationships explicitly.  Otherwise, if
you were to add a setup2 and a teardown2, it would not be clear what the
task sequencing should be.  Apart from this, we believe being explicit is
important for readability, because unless you are careful with object
naming in your dag it may not be obvious that setup1 and teardown1 are not
“normal” tasks, and therefore it might appear that they are free to run in
parallel as roots of the group.

Looking further ahead, while some of the design decisions are not yet
finalized, we’d like to give you a fuller preview of where we see this
going and how it should work.

At a high level, our approach is to make setup and teardown much more like
“normal” tasks, able to be organized and combined with all the flexibility
that Airflow users are accustomed to.  The behavior is mainly governed by a
few simple rules:

   -

   A teardown task will run if its setup has completed successfully and its
   upstreams are done.
   -

   The setup tasks “required by” a work task will be cleared when the work
   task is cleared.


When using multiple setups and teardowns, you will need to specify which
setup is for which teardown.  And the setup task “required by” a work task
can be inferred by its location between a setup and its teardown.

OK – any more detail would be too much for one email.  If you are
interested in reviewing our progress in greater detail and making comment,
you may review our working draft update here (
https://cwiki.apache.org/confluence/display/AIRFLOW/%5BWIP%5D+DRAFT+updates+to+AIP-52).
We’ve added lots of examples with graph screenshots to help illustrate the
behavior, and there’s some discussion of the ways it differs from the
original.


Thanks for your consideration.


Daniel

Re: [DISCUSS] AIP-52 updates - setup / teardown tasks

Posted by "Ferruzzi, Dennis" <fe...@amazon.com.INVALID>.
Feel free to ignore my last message, sorry about that.  I just came back from a vacation and it's Monday and I didn't realize I was missing an entire thread of discussion there.  Outlook does not handle the mailinglist message threading at all.  Apologies, it's not my intention to fork the conversation at this stage.

-ferruzzi

________________________________
From: Ferruzzi, Dennis <fe...@amazon.com.INVALID>
Sent: Monday, March 27, 2023 12:43 PM
To: dev@airflow.apache.org
Subject: RE: [EXTERNAL][DISCUSS] AIP-52 updates - setup / teardown tasks

CAUTION: This email originated from outside of the organization. Do not click links or open attachments unless you can confirm the sender and know the content is safe.



I really like the idea of this AIP and I'm looking forward to seeing how you implement it.   I know you've put lots of effort into it, and I'm not looking to derail your plans. It may be entirely too late at this point, but if you are doing a rework, how would you feel about the idea of the taskinstance accepting an optional list of tasks (callables maybe??) for setup and teardown?  Then when you define a task in a dag it might look something like (simplified)

====
@task
def setup1:
    create_thing()

@task(trigger_rule=TriggerRule.ALL_DONE)
def teardown1:
    delete_thing()

@task(trigger_rule=TriggerRule.ALL_DONE)
def teardown2:
    delete_logs()

with DAG() as dag:
    task1 = SomeRandomOperator(
        myVar = myVar,
        setup_tasks=[setup1],
        teardown_tasks=[teardown1, teardown2]
    )
====

One issue with that is that the @tasks would be added to the chain so there would need to be some way to stop that... maybe a new decorator called @setup and @teardown that inherits from @task but doesn't add to chain, or adding a parameter to @task like is_work_task which defaults to True and doesn't add to the chain if it's false or something?

-ferruzzi


________________________________
From: Daniel Standish <da...@astronomer.io.INVALID>
Sent: Wednesday, March 22, 2023 11:57 PM
To: dev@airflow.apache.org
Subject: [EXTERNAL] [DISCUSS] AIP-52 updates - setup / teardown tasks

CAUTION: This email originated from outside of the organization. Do not click links or open attachments unless you can confirm the sender and know the content is safe.



I’m part of a group working on the implementation of AIP-52.  We would like
to update the community on some changes to the implementation approach, the
planned roadmap, and give an opportunity to provide feedback.

First though, let’s recap briefly what are the main benefits of adding
setup and teardown as concepts in Airflow:

   -

   By separating setup and teardown from "work" tasks, after the failure of
   a work task, we can stop the dag from proceeding (i.e. onto subsequent work
   tasks) while still allowing needed teardown operations to proceed (e.g.
   deleting a cluster).
   -

   This separation also lets us optionally *not* fail a dag run when
   perhaps the important work was completed successfully but a cleanup or
   teardown operation failed.
   -

   By associating work and setup tasks, we can clear the setups (and their
   respective teardowns) when clearing the work tasks.


After experimenting with some different implementation approaches and
reviewing and writing a lot of example dags, we developed some conclusions
that caused us to change course somewhat, while still fulfilling the
principal goals of the AIP.

Perhaps most importantly, we believe it is essential that our design
choices leave room for multiple setup and teardown tasks in a given task
group or dag.  Dags don’t tend to do just one thing.  In a dag there could
be many tasks requiring their own “setup” and “teardown”.  Similarly, a
single “work” task may itself require multiple “setup” and teardown tasks.
For obvious reasons, combining the work of multiple operators into a single
task is not advisable.  And, requiring a new task group for each thing
requiring a setup also has pitfalls: it conflicts with the task group’s use
case as an arbitrary logical grouping of tasks, and as a task mapping
tool.  So we believe it will be necessary to be able to support multiple
setups within a group, and moreover we believe it will be necessary to be
able to set dependencies between them.

With that in mind, the main change we’d like to share is that we now
require that users must specify the relationship between setup/teardown
tasks and “normal” tasks.  *(In the original proposal, users were not
required to set relationships between setup/teardown tasks and the other
tasks in the group.)*

So in the original AIP you could do this:

with TaskGroup("group1") as tg1:

   setup1 = my_setup("g1_setup") # a setup task

   work1 = my_work("g1_work1")
  work2 = my_work("g1_work2")
  work1 >> work2

   teardown1 = my_teardown("g1_teardown") # a teardown task

Then in effect you’d get setup1 >> work1 >> work2 >> teardown1.

Now we require you to set those relationships explicitly.  Otherwise, if
you were to add a setup2 and a teardown2, it would not be clear what the
task sequencing should be.  Apart from this, we believe being explicit is
important for readability, because unless you are careful with object
naming in your dag it may not be obvious that setup1 and teardown1 are not
“normal” tasks, and therefore it might appear that they are free to run in
parallel as roots of the group.

Looking further ahead, while some of the design decisions are not yet
finalized, we’d like to give you a fuller preview of where we see this
going and how it should work.

At a high level, our approach is to make setup and teardown much more like
“normal” tasks, able to be organized and combined with all the flexibility
that Airflow users are accustomed to.  The behavior is mainly governed by a
few simple rules:

   -

   A teardown task will run if its setup has completed successfully and its
   upstreams are done.
   -

   The setup tasks “required by” a work task will be cleared when the work
   task is cleared.


When using multiple setups and teardowns, you will need to specify which
setup is for which teardown.  And the setup task “required by” a work task
can be inferred by its location between a setup and its teardown.

OK – any more detail would be too much for one email.  If you are
interested in reviewing our progress in greater detail and making comment,
you may review our working draft update here (
https://cwiki.apache.org/confluence/display/AIRFLOW/%5BWIP%5D+DRAFT+updates+to+AIP-52).
We’ve added lots of examples with graph screenshots to help illustrate the
behavior, and there’s some discussion of the ways it differs from the
original.


Thanks for your consideration.


Daniel

Re: [DISCUSS] AIP-52 updates - setup / teardown tasks

Posted by "Ferruzzi, Dennis" <fe...@amazon.com.INVALID>.
I really like the idea of this AIP and I'm looking forward to seeing how you implement it.   I know you've put lots of effort into it, and I'm not looking to derail your plans. It may be entirely too late at this point, but if you are doing a rework, how would you feel about the idea of the taskinstance accepting an optional list of tasks (callables maybe??) for setup and teardown?  Then when you define a task in a dag it might look something like (simplified)

====
@task
def setup1:
    create_thing()

@task(trigger_rule=TriggerRule.ALL_DONE)
def teardown1:
    delete_thing()

@task(trigger_rule=TriggerRule.ALL_DONE)
def teardown2:
    delete_logs()

with DAG() as dag:
    task1 = SomeRandomOperator(
        myVar = myVar,
        setup_tasks=[setup1],
        teardown_tasks=[teardown1, teardown2]
    )
====

One issue with that is that the @tasks would be added to the chain so there would need to be some way to stop that... maybe a new decorator called @setup and @teardown that inherits from @task but doesn't add to chain, or adding a parameter to @task like is_work_task which defaults to True and doesn't add to the chain if it's false or something?

-ferruzzi


________________________________
From: Daniel Standish <da...@astronomer.io.INVALID>
Sent: Wednesday, March 22, 2023 11:57 PM
To: dev@airflow.apache.org
Subject: [EXTERNAL] [DISCUSS] AIP-52 updates - setup / teardown tasks

CAUTION: This email originated from outside of the organization. Do not click links or open attachments unless you can confirm the sender and know the content is safe.



I’m part of a group working on the implementation of AIP-52.  We would like
to update the community on some changes to the implementation approach, the
planned roadmap, and give an opportunity to provide feedback.

First though, let’s recap briefly what are the main benefits of adding
setup and teardown as concepts in Airflow:

   -

   By separating setup and teardown from "work" tasks, after the failure of
   a work task, we can stop the dag from proceeding (i.e. onto subsequent work
   tasks) while still allowing needed teardown operations to proceed (e.g.
   deleting a cluster).
   -

   This separation also lets us optionally *not* fail a dag run when
   perhaps the important work was completed successfully but a cleanup or
   teardown operation failed.
   -

   By associating work and setup tasks, we can clear the setups (and their
   respective teardowns) when clearing the work tasks.


After experimenting with some different implementation approaches and
reviewing and writing a lot of example dags, we developed some conclusions
that caused us to change course somewhat, while still fulfilling the
principal goals of the AIP.

Perhaps most importantly, we believe it is essential that our design
choices leave room for multiple setup and teardown tasks in a given task
group or dag.  Dags don’t tend to do just one thing.  In a dag there could
be many tasks requiring their own “setup” and “teardown”.  Similarly, a
single “work” task may itself require multiple “setup” and teardown tasks.
For obvious reasons, combining the work of multiple operators into a single
task is not advisable.  And, requiring a new task group for each thing
requiring a setup also has pitfalls: it conflicts with the task group’s use
case as an arbitrary logical grouping of tasks, and as a task mapping
tool.  So we believe it will be necessary to be able to support multiple
setups within a group, and moreover we believe it will be necessary to be
able to set dependencies between them.

With that in mind, the main change we’d like to share is that we now
require that users must specify the relationship between setup/teardown
tasks and “normal” tasks.  *(In the original proposal, users were not
required to set relationships between setup/teardown tasks and the other
tasks in the group.)*

So in the original AIP you could do this:

with TaskGroup("group1") as tg1:

   setup1 = my_setup("g1_setup") # a setup task

   work1 = my_work("g1_work1")
  work2 = my_work("g1_work2")
  work1 >> work2

   teardown1 = my_teardown("g1_teardown") # a teardown task

Then in effect you’d get setup1 >> work1 >> work2 >> teardown1.

Now we require you to set those relationships explicitly.  Otherwise, if
you were to add a setup2 and a teardown2, it would not be clear what the
task sequencing should be.  Apart from this, we believe being explicit is
important for readability, because unless you are careful with object
naming in your dag it may not be obvious that setup1 and teardown1 are not
“normal” tasks, and therefore it might appear that they are free to run in
parallel as roots of the group.

Looking further ahead, while some of the design decisions are not yet
finalized, we’d like to give you a fuller preview of where we see this
going and how it should work.

At a high level, our approach is to make setup and teardown much more like
“normal” tasks, able to be organized and combined with all the flexibility
that Airflow users are accustomed to.  The behavior is mainly governed by a
few simple rules:

   -

   A teardown task will run if its setup has completed successfully and its
   upstreams are done.
   -

   The setup tasks “required by” a work task will be cleared when the work
   task is cleared.


When using multiple setups and teardowns, you will need to specify which
setup is for which teardown.  And the setup task “required by” a work task
can be inferred by its location between a setup and its teardown.

OK – any more detail would be too much for one email.  If you are
interested in reviewing our progress in greater detail and making comment,
you may review our working draft update here (
https://cwiki.apache.org/confluence/display/AIRFLOW/%5BWIP%5D+DRAFT+updates+to+AIP-52).
We’ve added lots of examples with graph screenshots to help illustrate the
behavior, and there’s some discussion of the ways it differs from the
original.


Thanks for your consideration.


Daniel

Re: [DISCUSS] AIP-52 updates - setup / teardown tasks

Posted by "Ferruzzi, Dennis" <fe...@amazon.com.INVALID>.
>  If the set-up ran then the short circuit shouldn't be able to skip it

>> I think this is overly and unnecessarily opinionated and limiting.  I would
not be concerned with having the default be one way or another, but to say
that "you should not be able to skip it", I disagree with the notion that
skipping a teardown with an operator specifically designed for the purpose
should be *forbidden,* as your comment suggests.

Another possible counterpoint would be one I have run into with system tests.  If the test passes, feel free to delete the infra and logs.  But if the test failed, it is very nice to have those things for troubleshooting.  So just because the setup task created them does not mean I always wan them to be torn down after.  Maybe this is a time to use ShortCuircuitOperator, but one way or another I think it is crucial that there is some form of escape hatch available there.


-ferruzzi

Re: [DISCUSS] AIP-52 updates - setup / teardown tasks

Posted by "Oliveira, Niko" <on...@amazon.com.INVALID>.
Chiming in on a few of the topics discussed so far:

- Context managers:
I found most of the context manager syntax proposals a little hard to understand, but some better than others. Ultimately if I put my DAG author hat on, I find this declaration the most straightforward, clear and it's easy to update existing code: chain(create_notification_channel.as_setup(), ... other tasks ... delete_notification_channel.teardown_for(create_notification_channel),...)

- Short Circuit Behaviour

Again putting my DAG author hat on, the example Daniel gave about skipping teardown steps in the event of a critical failure (so that we can debug files, clusters, etc left behind) is exactly something my team is interested in, and we're trying to build these things ourselves. Having the ShortCircuitOperator support this (although perhaps not by default) would be fantastic and as a user I would like to see this functionality maintained.

- Multiple setup/teardown

I also agree that whether we ship them now or later, we must ensure we don't walk through any one-way doors for future implementation. It's great that we're having these discussions early.

Cheers,
Niko



________________________________
From: Daniel Standish <da...@astronomer.io.INVALID>
Sent: Monday, March 27, 2023 9:43:07 AM
To: dev@airflow.apache.org
Subject: RE: [EXTERNAL][DISCUSS] AIP-52 updates - setup / teardown tasks

CAUTION: This email originated from outside of the organization. Do not click links or open attachments unless you can confirm the sender and know the content is safe.



>
> 1) I am not sure if we should make it private (I am not even sure what
> it would mean to be private:) ). But If it means that setting the rule
> type for non-teardown task should raise an error (and of course
> documenting this rule as only (and automatically) being applied to
> teardown task - I am good with it.


Yeah I have no concerns with throwing error if this rule is used "manually"
by user.  Like, at init time though, keeping it simple.  I.e. I think we
shouldn't boil the ocean to prevent a creative user from finding a way to
use it in some weird way.

2) Yes. The teardown must run in any case. I think documenting the
> behaviour in ShortCircuit as a special case and - especially - adding
> some examples showing that would be enough


Clarify Jarek, are you saying that we should allow ShortCircuit to "short
circuit" the teardowns [either by default or perhaps optionally with a
skip_teardowns_too param] and document with examples? Or are you saying
shortcircuit operator should be updated so it's not possible to use it to
skip a teardown?









On Mon, Mar 27, 2023 at 12:30 AM Jarek Potiuk <ja...@potiuk.com> wrote:

> My view:
>
> 1) I am not sure if we should make it private (I am not even sure what
> it would mean to be private:) ). But If it means that setting the rule
> type for non-teardown task should raise an error (and of course
> documenting this rule as only (and automatically) being applied to
> teardown task - I am good with it.
>
> 2) Yes. The teardown must run in any case. I think documenting the
> behaviour in ShortCircuit as a special case and - especially - adding
> some examples showing that would be enough
>
> And yeah while it is indeed an implementation detail, it is somewhat
> "visible" as the public API is being concerned. So no wonder it came
> as a bit of a surprise while implementing setup/teardown (happens that
> something we consider as a detail for those deeply involved is a bit
> of a surprise for those that came to look at it at review time).  I
> guess this is somewhat a consequence of the way we operate in a
> distributed environment and some details are being discussed in
> smaller groups that are more focused on getting things done (we had a
> few of those for AIP-44 for one).
>
> But eventually we are discussing it now, so I think it is cool.
>
> J.
>
> On Mon, Mar 27, 2023 at 8:43 AM Ash Berlin-Taylor <as...@apache.org> wrote:
> >
> > If the set-up ran then the tear down _must_ run. No question.
> >
> > Nothing should be able to change this fact. If you can, then they don't
> fulfill the stated purpose of tear down tasks in the AIP: to tidy up
> resources created by a set up task.
> >
> > On 27 March 2023 06:22:47 BST, Daniel Standish
> <da...@astronomer.io.INVALID> wrote:
> > >>
> > >> When user set setup/teardown he has no idea unique trigger rule is set
> > >> under the hood. The user also has no idea that trigger rules are even
> > >> involved. That is not something he sees unless he checks the code of
> > >> teardown and setup decorators.
> > >
> > >This means that users of ShortCircuitOperator will not even know they
> need
> > >> to take action (until it wont work as expexted) and they will
> propbably
> > >> start as asking questions.
> > >
> > >
> > >Yeah, short circuit operator is a special operator that, if you're
> going to
> > >use it, you ought to know how it works.  And we can easily add a note in
> > >the docs that emphasizes its behavior on this point.  But I should point
> > >out, the same would be true if setup and teardown were not implemented
> with
> > >trigger rules; unless you investigate, read the docs, or look at the
> code,
> > >you would not know whether teardowns would be skipped by short
> circuit.  So
> > >either way it's something that we'd have to lean on docs for.  It's just
> > >not something that a prudent user would just make an assumption about.
> > >
> > >If the set-up ran then the short circuit shouldn't be able to skip it
> > >>
> > >
> > >I think this is overly and unnecessarily opinionated and limiting.  I
> would
> > >not be concerned with having the default be one way or another, but to
> say
> > >that "you should not be able to skip it", I disagree with the notion
> that
> > >skipping a teardown with an operator specifically designed for the
> purpose
> > >should be *forbidden,* as your comment suggests.
> > >
> > >take for example creating a cluster - you still want to delete it at the
> > >> end even if you skipped all the other tasks!
> > >>
> > >
> > >Yeah that's a perfectly fine example.  And, normally this is precisely
> how
> > >this feature works: if your task throws a skip exception, its downstream
> > >work tasks will be skipped but the teardown will run.  But what we're
> > >talking about is a special operator whose unique purpose is to
> > >short circuit the dag.  And as we know, every kind of dag you can
> imagine,
> > >there's a user who needs it.  So someone will want to be able to skip a
> > >teardown.  Teardowns aren't always going to be deleting a cluster.  They
> > >might be deleting *files*, let's say.  Well, what if you have this
> special
> > >operator in there to detect when something very bad happens and in that
> > >case you want to stop all processing and leave the current state alone
> so
> > >you can debug it.  Is there any good reason to forbid this?  I think no.
> > >As stewards of this pipeline writing language, we should have sensible
> > >defaults and maintain a good authoring interface, while allow users the
> > >power to write what they need.
> > >
> > >
> > >
> > >
> > >On Sun, Mar 26, 2023 at 9:19 PM Ash Berlin-Taylor <as...@apache.org>
> wrote:
> > >
> > >> If the set-up ran then the short circuit shouldn't be able to skip it:
> > >> take for example creating a cluster - you still want to delete it at
> the
> > >> end even if you skipped all the other tasks!
> > >>
> > >> This is precisely what I mean by set up and tear down tasks being
> special!
> > >>
> > >> On 27 March 2023 04:02:32 BST, Elad Kalif <el...@apache.org> wrote:
> > >> >Thanks Daniel,
> > >> >Let me clarify my concern.
> > >> >
> > >> >When user set setup/teardown he has no idea unique trigger rule is
> set
> > >> >under the hood. The user also has no idea that trigger rules are even
> > >> >involved. That is not something he sees unless he checks the code of
> > >> >teardown and setup decorators.
> > >> >
> > >> >This means that users of ShortCircuitOperator will not even know
> they need
> > >> >to take action (until it wont work as expexted) and they will
> propbably
> > >> >start as asking questions.
> > >> >
> > >> >I'm not saying this should modify the plan just raising it as a
> potential
> > >> >source for pitfall.
> > >> >
> > >> >בתאריך יום ב׳, 27 במרץ 2023, 05:50, מאת Daniel Standish
> > >> >‏<da...@astronomer.io.invalid>:
> > >> >
> > >> >> Thanks Elad for the feedback.
> > >> >>
> > >> >> re 1. i don't really see a problem with the trigger rule being
> public.
> > >> The
> > >> >> way I see it, it's another trigger rule like any other trigger
> rule.
> > >> Every
> > >> >> trigger rule behaves differently, that's true here too. This one
> > >> happens to
> > >> >> be relied upon for teardown tasks.  That said, I don't think I
> would
> > >> >> necessarily be opposed to making it private.
> > >> >>
> > >> >> re 2, personally I kindof think it's a good thing.  My
> understanding
> > >> from
> > >> >> your comments is that with ShortCircuitOperator you can set it to
> skip
> > >> all
> > >> >> downstream or just skip the direct relatives.  To me this seems
> great
> > >> cus
> > >> >> it provides a way to either skip everything (including teardowns)
> or
> > >> just
> > >> >> the next task (thus potentially allowing teardowns to run).  To me
> this
> > >> is
> > >> >> another way in which by staying within the existing dependency and
> > >> trigger
> > >> >> rule paradigm we have more consistent, predictable, and
> configurable
> > >> >> behavior.  E.g. if we were not using normal deps and trigger
> rules, then
> > >> >> surely someone would have the opposite concern: "i want to use
> short
> > >> >> circuit operator to just skip all tasks including teardowns" and we
> > >> might
> > >> >> not be able to grant that wish, or at least not without more
> > >> development.
> > >> >> When you use an operator like this, you simply need to know what
> it does
> > >> >> and configure it in a manner appropriate for your use case.
> > >> >>
> > >>
>
> ---------------------------------------------------------------------
> To unsubscribe, e-mail: dev-unsubscribe@airflow.apache.org
> For additional commands, e-mail: dev-help@airflow.apache.org
>
>

Re: [DISCUSS] AIP-52 updates - setup / teardown tasks

Posted by Daniel Standish <da...@astronomer.io.INVALID>.
You're not too late, and thanks for engaging with the issue.

I don't think anyone would dispute that users will sometimes want a setup
without a teardown.  But the question is should we require that users
explicitly make the scope of a setup well-defined.  Like Jarek, I have some
ambivalence on the question but lean toward requiring explicitness, though
I can accept the alternative.

Note currently we're not requiring an empty *setup* for unmatched teardowns
since it wouldn't make a difference anyway; i.e. the notion of "scope" is
only really a question for setups; for teardowns there's no material impact.

An aside concerning associations with unittests / pytest... I think it's
fair to say that this AIP took some inspiration from unittests / pytest.
But the goal of the AIP is not to replicate the API of unittests / pytest
in airflow.  Indeed airflow provides a much richer vocabulary for
expressing task pipelines, and this AIP adds to the richness of that
vocabulary.  And fundamentally, the goal is not to adhere to some aesthetic
ideal or perfect consistency with some other API but rather to provide
useful functionality for expressing real world pipelines and solving real
world problems in Airflow.  And along these lines I think that a 1-1
correspondence with unittests is neither something we should aspire to nor
be limited by (not that you're suggesting this, but it comes up).  Airflow
is something different from unittess, and the setup and teardown
functionality is something different from unittests / pytest,
notwithstanding the reasonable analogies we can observe.

Re: [DISCUSS] AIP-52 updates - setup / teardown tasks

Posted by utkarsh sharma <ut...@gmail.com>.
Hey everyone,

I hope I'm not late to the discussion. :)

IMO, there is a similarity between Airflow's Setup/Teardown and
Pytest's Setup/Teardown, and Pytest allows setup and teardown to exist
independently. There are cases where you might need one and not the other.
But I'm not sure of any airflow-specific concerns here.

Thanks,
Utkarsh

On Tue, Jul 4, 2023 at 3:45 PM Jarek Potiuk <ja...@potiuk.com> wrote:

> >  we require that a setup always have a teardown
>
> I'd personally prefer this one because "explicit is better than implicit".
> I see the cases where it could be useful (I was also contemplating no
> setup/no teardown before), but I think the wiggly behaviour is a recipe for
> a number of mistakes (and issues raised by users), especially when someone
> modifies existing DAG and adds teardown without realising the consequences.
>
> Not a strong preference, I am ok with both, but if I got to choose, I'd
> require teardown.
>
> J.
>
>
>
>
> On Fri, Jun 30, 2023 at 10:28 PM Daniel Standish
> <da...@astronomer.io.invalid> wrote:
>
> > Ok want to pick back up setup / teardown discussion as we get closer to
> > 2.7.
> >
> > For personal reasons I had to take some time off work just as we were
> > wrapping up work on 2.6, and at least partly due to that, we punted
> setup /
> > teardown to 2.7.
> >
> > But we've picked it back up and continued along the path outlined in this
> > thread and things look good for 2.7.
> >
> > In the time since the previous discussion, not a lot has changed except
> > that we've implemented in main some of the things discussed.
> >
> > For one we added a one-liner syntax.
> >
> > So now if you have an existing dag such as this:
> >
> > create_cluster >> run_query >> delete_cluster
> >
> > You can convert it to use setup / teardown with the following:
> >
> > create_cluster >> run_query >>
> > delete_cluster.as_teardown(setups=create_cluster)
> >
> > The context manager suggested by Ash has been added, so if you have setup
> > `s` and teardown `t`, you can do the following:
> >
> > with s >> t:
> >     some_work(other_work())
> >
> > Notably, just as with task group context manager, you must instantiate
> the
> > task inside the context in order for it to be registered.  Ephraim is
> > working on some improvements to the context manager to address this (so
> > that you can explicitly add tasks to the context in that scenario) and so
> > that we can throw an error on non-sensical usages such as `with a >> b >>
> > c:`
> >
> > There is a docs PR outlining the current state of the feature here
> > <https://github.com/apache/airflow/pull/32169> and you are all welcome
> to
> > comment.  There's still time to make adjustments if desired.
> >
> > But I wanted to raise one point for discussion here.  It's something I
> > could have gone either way on, and I want to tap into the wisdom of the
> > community.
> >
> > The question of whether a setup with no teardown, or a teardown with no
> > setup, should be added.
> >
> > There are two options as I see it.
> >
> > Option 1: if a setup has no teardown, then all the downstreams are
> assumed
> > to be "in scope" for the setup
> > Note: the notion of "scope" is about the clearing behavior aspect of
> setup
> > / teardown: you clear a task, it clears its setups too.  A work task is
> in
> > the scope of s and t if it's between s and t.
> > So then with s1 >> w1 >> w2, clearing work task w1 would clear w1 w2 and
> > s1.
> > And clearing w2 would clear w2 and s1.
> > This makes it easy to have a setup with no teardown. But it makes the
> > behavior slightly more... wiggly... for lack of a better term.  With this
> > example, s1 is a setup for w2.  But then suppose you add a teardown after
> > w1: s1 >> w1 >> [w2, t1.as_teardown(setups=s1)].  Now s1 is not a setup
> for
> > w2.
> > So the logic is ... "a work task is in scope of a setup if it is in
> between
> > the setup and its teardown, *or *if the setup has no teardowns".
> >
> > Option 1: we require that a setup always have a teardown
> > With this approach, the rule is simpler and it forces unambiguously
> defined
> > scopes.  If you just want to use a setup and don't need a teardown task,
> > you add an empty teardown.  The rule is then "a work task is in scope of
> a
> > setup if it is in between the setup and its teardown" -- i.e. the same
> rule
> > minus the `or` clause.
> > So the example would have to be s1 >> w1 >> w2 >>
> > EmptyOperator().as_teardown(setups=s1).
> >
> > Thanks in advance for your engagement.
> >
> > P.S. note that those of us in the U.S. will be off for holiday soon so
> may
> > be slower to respond but just wanted to get this conversation started so
> > that we can get this feature to a place that people generally feel good
> > enough about in 2.7.
> >
> > P.P.S. Random-ish note... we should consider marking some features as
> beta
> > I think.  There's a difference between beta and experimental, though
> > historically we've only employed the latter.  But I think beta can be
> good
> > to signal "we are committed to this feature we just might make a change
> or
> > two to the API".  Not saying we should do this with setup / teardown
> > necessarily but it's an option.  Sometimes I think it can be a little
> > overwhelming to put out features with the spectre of being locked in by
> > backcompat guarantees and perhaps something like a beta feature could be
> a
> > way to get things out there a little bit more freely and allow for a
> > feedback loop.
> >
> > Thanks again,
> >
> > Daniel
> >
>

Re: [DISCUSS] AIP-52 updates - setup / teardown tasks

Posted by Jarek Potiuk <ja...@potiuk.com>.
>  we require that a setup always have a teardown

I'd personally prefer this one because "explicit is better than implicit".
I see the cases where it could be useful (I was also contemplating no
setup/no teardown before), but I think the wiggly behaviour is a recipe for
a number of mistakes (and issues raised by users), especially when someone
modifies existing DAG and adds teardown without realising the consequences.

Not a strong preference, I am ok with both, but if I got to choose, I'd
require teardown.

J.




On Fri, Jun 30, 2023 at 10:28 PM Daniel Standish
<da...@astronomer.io.invalid> wrote:

> Ok want to pick back up setup / teardown discussion as we get closer to
> 2.7.
>
> For personal reasons I had to take some time off work just as we were
> wrapping up work on 2.6, and at least partly due to that, we punted setup /
> teardown to 2.7.
>
> But we've picked it back up and continued along the path outlined in this
> thread and things look good for 2.7.
>
> In the time since the previous discussion, not a lot has changed except
> that we've implemented in main some of the things discussed.
>
> For one we added a one-liner syntax.
>
> So now if you have an existing dag such as this:
>
> create_cluster >> run_query >> delete_cluster
>
> You can convert it to use setup / teardown with the following:
>
> create_cluster >> run_query >>
> delete_cluster.as_teardown(setups=create_cluster)
>
> The context manager suggested by Ash has been added, so if you have setup
> `s` and teardown `t`, you can do the following:
>
> with s >> t:
>     some_work(other_work())
>
> Notably, just as with task group context manager, you must instantiate the
> task inside the context in order for it to be registered.  Ephraim is
> working on some improvements to the context manager to address this (so
> that you can explicitly add tasks to the context in that scenario) and so
> that we can throw an error on non-sensical usages such as `with a >> b >>
> c:`
>
> There is a docs PR outlining the current state of the feature here
> <https://github.com/apache/airflow/pull/32169> and you are all welcome to
> comment.  There's still time to make adjustments if desired.
>
> But I wanted to raise one point for discussion here.  It's something I
> could have gone either way on, and I want to tap into the wisdom of the
> community.
>
> The question of whether a setup with no teardown, or a teardown with no
> setup, should be added.
>
> There are two options as I see it.
>
> Option 1: if a setup has no teardown, then all the downstreams are assumed
> to be "in scope" for the setup
> Note: the notion of "scope" is about the clearing behavior aspect of setup
> / teardown: you clear a task, it clears its setups too.  A work task is in
> the scope of s and t if it's between s and t.
> So then with s1 >> w1 >> w2, clearing work task w1 would clear w1 w2 and
> s1.
> And clearing w2 would clear w2 and s1.
> This makes it easy to have a setup with no teardown. But it makes the
> behavior slightly more... wiggly... for lack of a better term.  With this
> example, s1 is a setup for w2.  But then suppose you add a teardown after
> w1: s1 >> w1 >> [w2, t1.as_teardown(setups=s1)].  Now s1 is not a setup for
> w2.
> So the logic is ... "a work task is in scope of a setup if it is in between
> the setup and its teardown, *or *if the setup has no teardowns".
>
> Option 1: we require that a setup always have a teardown
> With this approach, the rule is simpler and it forces unambiguously defined
> scopes.  If you just want to use a setup and don't need a teardown task,
> you add an empty teardown.  The rule is then "a work task is in scope of a
> setup if it is in between the setup and its teardown" -- i.e. the same rule
> minus the `or` clause.
> So the example would have to be s1 >> w1 >> w2 >>
> EmptyOperator().as_teardown(setups=s1).
>
> Thanks in advance for your engagement.
>
> P.S. note that those of us in the U.S. will be off for holiday soon so may
> be slower to respond but just wanted to get this conversation started so
> that we can get this feature to a place that people generally feel good
> enough about in 2.7.
>
> P.P.S. Random-ish note... we should consider marking some features as beta
> I think.  There's a difference between beta and experimental, though
> historically we've only employed the latter.  But I think beta can be good
> to signal "we are committed to this feature we just might make a change or
> two to the API".  Not saying we should do this with setup / teardown
> necessarily but it's an option.  Sometimes I think it can be a little
> overwhelming to put out features with the spectre of being locked in by
> backcompat guarantees and perhaps something like a beta feature could be a
> way to get things out there a little bit more freely and allow for a
> feedback loop.
>
> Thanks again,
>
> Daniel
>

Re: [DISCUSS] AIP-52 updates - setup / teardown tasks

Posted by Daniel Standish <da...@astronomer.io.INVALID>.
Ok want to pick back up setup / teardown discussion as we get closer to 2.7.

For personal reasons I had to take some time off work just as we were
wrapping up work on 2.6, and at least partly due to that, we punted setup /
teardown to 2.7.

But we've picked it back up and continued along the path outlined in this
thread and things look good for 2.7.

In the time since the previous discussion, not a lot has changed except
that we've implemented in main some of the things discussed.

For one we added a one-liner syntax.

So now if you have an existing dag such as this:

create_cluster >> run_query >> delete_cluster

You can convert it to use setup / teardown with the following:

create_cluster >> run_query >>
delete_cluster.as_teardown(setups=create_cluster)

The context manager suggested by Ash has been added, so if you have setup
`s` and teardown `t`, you can do the following:

with s >> t:
    some_work(other_work())

Notably, just as with task group context manager, you must instantiate the
task inside the context in order for it to be registered.  Ephraim is
working on some improvements to the context manager to address this (so
that you can explicitly add tasks to the context in that scenario) and so
that we can throw an error on non-sensical usages such as `with a >> b >>
c:`

There is a docs PR outlining the current state of the feature here
<https://github.com/apache/airflow/pull/32169> and you are all welcome to
comment.  There's still time to make adjustments if desired.

But I wanted to raise one point for discussion here.  It's something I
could have gone either way on, and I want to tap into the wisdom of the
community.

The question of whether a setup with no teardown, or a teardown with no
setup, should be added.

There are two options as I see it.

Option 1: if a setup has no teardown, then all the downstreams are assumed
to be "in scope" for the setup
Note: the notion of "scope" is about the clearing behavior aspect of setup
/ teardown: you clear a task, it clears its setups too.  A work task is in
the scope of s and t if it's between s and t.
So then with s1 >> w1 >> w2, clearing work task w1 would clear w1 w2 and s1.
And clearing w2 would clear w2 and s1.
This makes it easy to have a setup with no teardown. But it makes the
behavior slightly more... wiggly... for lack of a better term.  With this
example, s1 is a setup for w2.  But then suppose you add a teardown after
w1: s1 >> w1 >> [w2, t1.as_teardown(setups=s1)].  Now s1 is not a setup for
w2.
So the logic is ... "a work task is in scope of a setup if it is in between
the setup and its teardown, *or *if the setup has no teardowns".

Option 1: we require that a setup always have a teardown
With this approach, the rule is simpler and it forces unambiguously defined
scopes.  If you just want to use a setup and don't need a teardown task,
you add an empty teardown.  The rule is then "a work task is in scope of a
setup if it is in between the setup and its teardown" -- i.e. the same rule
minus the `or` clause.
So the example would have to be s1 >> w1 >> w2 >>
EmptyOperator().as_teardown(setups=s1).

Thanks in advance for your engagement.

P.S. note that those of us in the U.S. will be off for holiday soon so may
be slower to respond but just wanted to get this conversation started so
that we can get this feature to a place that people generally feel good
enough about in 2.7.

P.P.S. Random-ish note... we should consider marking some features as beta
I think.  There's a difference between beta and experimental, though
historically we've only employed the latter.  But I think beta can be good
to signal "we are committed to this feature we just might make a change or
two to the API".  Not saying we should do this with setup / teardown
necessarily but it's an option.  Sometimes I think it can be a little
overwhelming to put out features with the spectre of being locked in by
backcompat guarantees and perhaps something like a beta feature could be a
way to get things out there a little bit more freely and allow for a
feedback loop.

Thanks again,

Daniel

Re: [DISCUSS] AIP-52 updates - setup / teardown tasks

Posted by Elad Kalif <el...@apache.org>.
Thank you everyone.
My concerns have been addressed.
I still think we should somehow advise users not to use this new trigger
rule explicitly but that is something we can talk in the PR itself.

בתאריך יום ג׳, 28 במרץ 2023, 09:17, מאת Jarek Potiuk ‏<ja...@potiuk.com>:

> Maybe just `NukeOperator` simply :)
>
> On Tue, Mar 28, 2023 at 7:57 AM Daniel Standish
> <da...@astronomer.io.invalid> wrote:
> >
> > Happy to see the engagement on this one.  Thanks to everyone for thinking
> > it through and contributing their thoughts.
> >
> > re niko
> >
> >
> > > - Context managers:
> > > I found most of the context manager syntax proposals a little hard to
> > > understand, but some better than others. Ultimately if I put my DAG
> author
> > > hat on, I find this declaration the most straightforward, clear and
> it's
> > > easy to update existing code:
> chain(create_notification_channel.as_setup(),
> > > ... other tasks ...
> > >
> delete_notification_channel.teardown_for(create_notification_channel),...)
> >
> >
> > +1 yeah i like this approach and, exactly -- it's the simplest syntax for
> > updating existing code to use the feature and requires essentially no
> > change in dag structure.
> >
> > re short circuit
> >
> > 1. it is a classic 80s movie 2. yeah no objections let's make short
> circuit
> > not skip teardowns, it seems like a reasonable default... that said
> > were someone down the line to create a PR with an option that let's you
> do
> > it, in a manner that is very clear about the consequences, or maybe
> instead
> > a distinct ShortCircuitNuclearOperator, it would not bother me. and if
> > either of those were to be rejected, it would not be hard to implement
> such
> > for your own project, or achieve same effect through other means.
>
> ---------------------------------------------------------------------
> To unsubscribe, e-mail: dev-unsubscribe@airflow.apache.org
> For additional commands, e-mail: dev-help@airflow.apache.org
>
>

Re: [DISCUSS] AIP-52 updates - setup / teardown tasks

Posted by Jarek Potiuk <ja...@potiuk.com>.
Maybe just `NukeOperator` simply :)

On Tue, Mar 28, 2023 at 7:57 AM Daniel Standish
<da...@astronomer.io.invalid> wrote:
>
> Happy to see the engagement on this one.  Thanks to everyone for thinking
> it through and contributing their thoughts.
>
> re niko
>
>
> > - Context managers:
> > I found most of the context manager syntax proposals a little hard to
> > understand, but some better than others. Ultimately if I put my DAG author
> > hat on, I find this declaration the most straightforward, clear and it's
> > easy to update existing code: chain(create_notification_channel.as_setup(),
> > ... other tasks ...
> > delete_notification_channel.teardown_for(create_notification_channel),...)
>
>
> +1 yeah i like this approach and, exactly -- it's the simplest syntax for
> updating existing code to use the feature and requires essentially no
> change in dag structure.
>
> re short circuit
>
> 1. it is a classic 80s movie 2. yeah no objections let's make short circuit
> not skip teardowns, it seems like a reasonable default... that said
> were someone down the line to create a PR with an option that let's you do
> it, in a manner that is very clear about the consequences, or maybe instead
> a distinct ShortCircuitNuclearOperator, it would not bother me. and if
> either of those were to be rejected, it would not be hard to implement such
> for your own project, or achieve same effect through other means.

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@airflow.apache.org
For additional commands, e-mail: dev-help@airflow.apache.org


Re: [DISCUSS] AIP-52 updates - setup / teardown tasks

Posted by Daniel Standish <da...@astronomer.io.INVALID>.
Happy to see the engagement on this one.  Thanks to everyone for thinking
it through and contributing their thoughts.

re niko


> - Context managers:
> I found most of the context manager syntax proposals a little hard to
> understand, but some better than others. Ultimately if I put my DAG author
> hat on, I find this declaration the most straightforward, clear and it's
> easy to update existing code: chain(create_notification_channel.as_setup(),
> ... other tasks ...
> delete_notification_channel.teardown_for(create_notification_channel),...)


+1 yeah i like this approach and, exactly -- it's the simplest syntax for
updating existing code to use the feature and requires essentially no
change in dag structure.

re short circuit

1. it is a classic 80s movie 2. yeah no objections let's make short circuit
not skip teardowns, it seems like a reasonable default... that said
were someone down the line to create a PR with an option that let's you do
it, in a manner that is very clear about the consequences, or maybe instead
a distinct ShortCircuitNuclearOperator, it would not bother me. and if
either of those were to be rejected, it would not be hard to implement such
for your own project, or achieve same effect through other means.

Re: [DISCUSS] AIP-52 updates - setup / teardown tasks

Posted by Jarek Potiuk <ja...@potiuk.com>.
I think teardown should always happen no matter the short cricuit.
Short circuit IMHO should be updated so that it should not be able to
skip teardown.

Also I think Dennis later point about preserving the output /
intermediate resources is sometimes important, but this is a debugging
technique that might require some changes in the DAG. And here where I
think the new trigger is cool - simply comment the @teardown in your
dag and .... you get "don't remove resources on failure" effectively
:)

On Mon, Mar 27, 2023 at 6:43 PM Daniel Standish
<da...@astronomer.io.invalid> wrote:
>
> >
> > 1) I am not sure if we should make it private (I am not even sure what
> > it would mean to be private:) ). But If it means that setting the rule
> > type for non-teardown task should raise an error (and of course
> > documenting this rule as only (and automatically) being applied to
> > teardown task - I am good with it.
>
>
> Yeah I have no concerns with throwing error if this rule is used "manually"
> by user.  Like, at init time though, keeping it simple.  I.e. I think we
> shouldn't boil the ocean to prevent a creative user from finding a way to
> use it in some weird way.
>
> 2) Yes. The teardown must run in any case. I think documenting the
> > behaviour in ShortCircuit as a special case and - especially - adding
> > some examples showing that would be enough
>
>
> Clarify Jarek, are you saying that we should allow ShortCircuit to "short
> circuit" the teardowns [either by default or perhaps optionally with a
> skip_teardowns_too param] and document with examples? Or are you saying
> shortcircuit operator should be updated so it's not possible to use it to
> skip a teardown?
>
>
>
>
>
>
>
>
>
> On Mon, Mar 27, 2023 at 12:30 AM Jarek Potiuk <ja...@potiuk.com> wrote:
>
> > My view:
> >
> > 1) I am not sure if we should make it private (I am not even sure what
> > it would mean to be private:) ). But If it means that setting the rule
> > type for non-teardown task should raise an error (and of course
> > documenting this rule as only (and automatically) being applied to
> > teardown task - I am good with it.
> >
> > 2) Yes. The teardown must run in any case. I think documenting the
> > behaviour in ShortCircuit as a special case and - especially - adding
> > some examples showing that would be enough
> >
> > And yeah while it is indeed an implementation detail, it is somewhat
> > "visible" as the public API is being concerned. So no wonder it came
> > as a bit of a surprise while implementing setup/teardown (happens that
> > something we consider as a detail for those deeply involved is a bit
> > of a surprise for those that came to look at it at review time).  I
> > guess this is somewhat a consequence of the way we operate in a
> > distributed environment and some details are being discussed in
> > smaller groups that are more focused on getting things done (we had a
> > few of those for AIP-44 for one).
> >
> > But eventually we are discussing it now, so I think it is cool.
> >
> > J.
> >
> > On Mon, Mar 27, 2023 at 8:43 AM Ash Berlin-Taylor <as...@apache.org> wrote:
> > >
> > > If the set-up ran then the tear down _must_ run. No question.
> > >
> > > Nothing should be able to change this fact. If you can, then they don't
> > fulfill the stated purpose of tear down tasks in the AIP: to tidy up
> > resources created by a set up task.
> > >
> > > On 27 March 2023 06:22:47 BST, Daniel Standish
> > <da...@astronomer.io.INVALID> wrote:
> > > >>
> > > >> When user set setup/teardown he has no idea unique trigger rule is set
> > > >> under the hood. The user also has no idea that trigger rules are even
> > > >> involved. That is not something he sees unless he checks the code of
> > > >> teardown and setup decorators.
> > > >
> > > >This means that users of ShortCircuitOperator will not even know they
> > need
> > > >> to take action (until it wont work as expexted) and they will
> > propbably
> > > >> start as asking questions.
> > > >
> > > >
> > > >Yeah, short circuit operator is a special operator that, if you're
> > going to
> > > >use it, you ought to know how it works.  And we can easily add a note in
> > > >the docs that emphasizes its behavior on this point.  But I should point
> > > >out, the same would be true if setup and teardown were not implemented
> > with
> > > >trigger rules; unless you investigate, read the docs, or look at the
> > code,
> > > >you would not know whether teardowns would be skipped by short
> > circuit.  So
> > > >either way it's something that we'd have to lean on docs for.  It's just
> > > >not something that a prudent user would just make an assumption about.
> > > >
> > > >If the set-up ran then the short circuit shouldn't be able to skip it
> > > >>
> > > >
> > > >I think this is overly and unnecessarily opinionated and limiting.  I
> > would
> > > >not be concerned with having the default be one way or another, but to
> > say
> > > >that "you should not be able to skip it", I disagree with the notion
> > that
> > > >skipping a teardown with an operator specifically designed for the
> > purpose
> > > >should be *forbidden,* as your comment suggests.
> > > >
> > > >take for example creating a cluster - you still want to delete it at the
> > > >> end even if you skipped all the other tasks!
> > > >>
> > > >
> > > >Yeah that's a perfectly fine example.  And, normally this is precisely
> > how
> > > >this feature works: if your task throws a skip exception, its downstream
> > > >work tasks will be skipped but the teardown will run.  But what we're
> > > >talking about is a special operator whose unique purpose is to
> > > >short circuit the dag.  And as we know, every kind of dag you can
> > imagine,
> > > >there's a user who needs it.  So someone will want to be able to skip a
> > > >teardown.  Teardowns aren't always going to be deleting a cluster.  They
> > > >might be deleting *files*, let's say.  Well, what if you have this
> > special
> > > >operator in there to detect when something very bad happens and in that
> > > >case you want to stop all processing and leave the current state alone
> > so
> > > >you can debug it.  Is there any good reason to forbid this?  I think no.
> > > >As stewards of this pipeline writing language, we should have sensible
> > > >defaults and maintain a good authoring interface, while allow users the
> > > >power to write what they need.
> > > >
> > > >
> > > >
> > > >
> > > >On Sun, Mar 26, 2023 at 9:19 PM Ash Berlin-Taylor <as...@apache.org>
> > wrote:
> > > >
> > > >> If the set-up ran then the short circuit shouldn't be able to skip it:
> > > >> take for example creating a cluster - you still want to delete it at
> > the
> > > >> end even if you skipped all the other tasks!
> > > >>
> > > >> This is precisely what I mean by set up and tear down tasks being
> > special!
> > > >>
> > > >> On 27 March 2023 04:02:32 BST, Elad Kalif <el...@apache.org> wrote:
> > > >> >Thanks Daniel,
> > > >> >Let me clarify my concern.
> > > >> >
> > > >> >When user set setup/teardown he has no idea unique trigger rule is
> > set
> > > >> >under the hood. The user also has no idea that trigger rules are even
> > > >> >involved. That is not something he sees unless he checks the code of
> > > >> >teardown and setup decorators.
> > > >> >
> > > >> >This means that users of ShortCircuitOperator will not even know
> > they need
> > > >> >to take action (until it wont work as expexted) and they will
> > propbably
> > > >> >start as asking questions.
> > > >> >
> > > >> >I'm not saying this should modify the plan just raising it as a
> > potential
> > > >> >source for pitfall.
> > > >> >
> > > >> >בתאריך יום ב׳, 27 במרץ 2023, 05:50, מאת Daniel Standish
> > > >> >‏<da...@astronomer.io.invalid>:
> > > >> >
> > > >> >> Thanks Elad for the feedback.
> > > >> >>
> > > >> >> re 1. i don't really see a problem with the trigger rule being
> > public.
> > > >> The
> > > >> >> way I see it, it's another trigger rule like any other trigger
> > rule.
> > > >> Every
> > > >> >> trigger rule behaves differently, that's true here too. This one
> > > >> happens to
> > > >> >> be relied upon for teardown tasks.  That said, I don't think I
> > would
> > > >> >> necessarily be opposed to making it private.
> > > >> >>
> > > >> >> re 2, personally I kindof think it's a good thing.  My
> > understanding
> > > >> from
> > > >> >> your comments is that with ShortCircuitOperator you can set it to
> > skip
> > > >> all
> > > >> >> downstream or just skip the direct relatives.  To me this seems
> > great
> > > >> cus
> > > >> >> it provides a way to either skip everything (including teardowns)
> > or
> > > >> just
> > > >> >> the next task (thus potentially allowing teardowns to run).  To me
> > this
> > > >> is
> > > >> >> another way in which by staying within the existing dependency and
> > > >> trigger
> > > >> >> rule paradigm we have more consistent, predictable, and
> > configurable
> > > >> >> behavior.  E.g. if we were not using normal deps and trigger
> > rules, then
> > > >> >> surely someone would have the opposite concern: "i want to use
> > short
> > > >> >> circuit operator to just skip all tasks including teardowns" and we
> > > >> might
> > > >> >> not be able to grant that wish, or at least not without more
> > > >> development.
> > > >> >> When you use an operator like this, you simply need to know what
> > it does
> > > >> >> and configure it in a manner appropriate for your use case.
> > > >> >>
> > > >>
> >
> > ---------------------------------------------------------------------
> > To unsubscribe, e-mail: dev-unsubscribe@airflow.apache.org
> > For additional commands, e-mail: dev-help@airflow.apache.org
> >
> >

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@airflow.apache.org
For additional commands, e-mail: dev-help@airflow.apache.org


Re: [DISCUSS] AIP-52 updates - setup / teardown tasks

Posted by Daniel Standish <da...@astronomer.io.INVALID>.
>
> 1) I am not sure if we should make it private (I am not even sure what
> it would mean to be private:) ). But If it means that setting the rule
> type for non-teardown task should raise an error (and of course
> documenting this rule as only (and automatically) being applied to
> teardown task - I am good with it.


Yeah I have no concerns with throwing error if this rule is used "manually"
by user.  Like, at init time though, keeping it simple.  I.e. I think we
shouldn't boil the ocean to prevent a creative user from finding a way to
use it in some weird way.

2) Yes. The teardown must run in any case. I think documenting the
> behaviour in ShortCircuit as a special case and - especially - adding
> some examples showing that would be enough


Clarify Jarek, are you saying that we should allow ShortCircuit to "short
circuit" the teardowns [either by default or perhaps optionally with a
skip_teardowns_too param] and document with examples? Or are you saying
shortcircuit operator should be updated so it's not possible to use it to
skip a teardown?









On Mon, Mar 27, 2023 at 12:30 AM Jarek Potiuk <ja...@potiuk.com> wrote:

> My view:
>
> 1) I am not sure if we should make it private (I am not even sure what
> it would mean to be private:) ). But If it means that setting the rule
> type for non-teardown task should raise an error (and of course
> documenting this rule as only (and automatically) being applied to
> teardown task - I am good with it.
>
> 2) Yes. The teardown must run in any case. I think documenting the
> behaviour in ShortCircuit as a special case and - especially - adding
> some examples showing that would be enough
>
> And yeah while it is indeed an implementation detail, it is somewhat
> "visible" as the public API is being concerned. So no wonder it came
> as a bit of a surprise while implementing setup/teardown (happens that
> something we consider as a detail for those deeply involved is a bit
> of a surprise for those that came to look at it at review time).  I
> guess this is somewhat a consequence of the way we operate in a
> distributed environment and some details are being discussed in
> smaller groups that are more focused on getting things done (we had a
> few of those for AIP-44 for one).
>
> But eventually we are discussing it now, so I think it is cool.
>
> J.
>
> On Mon, Mar 27, 2023 at 8:43 AM Ash Berlin-Taylor <as...@apache.org> wrote:
> >
> > If the set-up ran then the tear down _must_ run. No question.
> >
> > Nothing should be able to change this fact. If you can, then they don't
> fulfill the stated purpose of tear down tasks in the AIP: to tidy up
> resources created by a set up task.
> >
> > On 27 March 2023 06:22:47 BST, Daniel Standish
> <da...@astronomer.io.INVALID> wrote:
> > >>
> > >> When user set setup/teardown he has no idea unique trigger rule is set
> > >> under the hood. The user also has no idea that trigger rules are even
> > >> involved. That is not something he sees unless he checks the code of
> > >> teardown and setup decorators.
> > >
> > >This means that users of ShortCircuitOperator will not even know they
> need
> > >> to take action (until it wont work as expexted) and they will
> propbably
> > >> start as asking questions.
> > >
> > >
> > >Yeah, short circuit operator is a special operator that, if you're
> going to
> > >use it, you ought to know how it works.  And we can easily add a note in
> > >the docs that emphasizes its behavior on this point.  But I should point
> > >out, the same would be true if setup and teardown were not implemented
> with
> > >trigger rules; unless you investigate, read the docs, or look at the
> code,
> > >you would not know whether teardowns would be skipped by short
> circuit.  So
> > >either way it's something that we'd have to lean on docs for.  It's just
> > >not something that a prudent user would just make an assumption about.
> > >
> > >If the set-up ran then the short circuit shouldn't be able to skip it
> > >>
> > >
> > >I think this is overly and unnecessarily opinionated and limiting.  I
> would
> > >not be concerned with having the default be one way or another, but to
> say
> > >that "you should not be able to skip it", I disagree with the notion
> that
> > >skipping a teardown with an operator specifically designed for the
> purpose
> > >should be *forbidden,* as your comment suggests.
> > >
> > >take for example creating a cluster - you still want to delete it at the
> > >> end even if you skipped all the other tasks!
> > >>
> > >
> > >Yeah that's a perfectly fine example.  And, normally this is precisely
> how
> > >this feature works: if your task throws a skip exception, its downstream
> > >work tasks will be skipped but the teardown will run.  But what we're
> > >talking about is a special operator whose unique purpose is to
> > >short circuit the dag.  And as we know, every kind of dag you can
> imagine,
> > >there's a user who needs it.  So someone will want to be able to skip a
> > >teardown.  Teardowns aren't always going to be deleting a cluster.  They
> > >might be deleting *files*, let's say.  Well, what if you have this
> special
> > >operator in there to detect when something very bad happens and in that
> > >case you want to stop all processing and leave the current state alone
> so
> > >you can debug it.  Is there any good reason to forbid this?  I think no.
> > >As stewards of this pipeline writing language, we should have sensible
> > >defaults and maintain a good authoring interface, while allow users the
> > >power to write what they need.
> > >
> > >
> > >
> > >
> > >On Sun, Mar 26, 2023 at 9:19 PM Ash Berlin-Taylor <as...@apache.org>
> wrote:
> > >
> > >> If the set-up ran then the short circuit shouldn't be able to skip it:
> > >> take for example creating a cluster - you still want to delete it at
> the
> > >> end even if you skipped all the other tasks!
> > >>
> > >> This is precisely what I mean by set up and tear down tasks being
> special!
> > >>
> > >> On 27 March 2023 04:02:32 BST, Elad Kalif <el...@apache.org> wrote:
> > >> >Thanks Daniel,
> > >> >Let me clarify my concern.
> > >> >
> > >> >When user set setup/teardown he has no idea unique trigger rule is
> set
> > >> >under the hood. The user also has no idea that trigger rules are even
> > >> >involved. That is not something he sees unless he checks the code of
> > >> >teardown and setup decorators.
> > >> >
> > >> >This means that users of ShortCircuitOperator will not even know
> they need
> > >> >to take action (until it wont work as expexted) and they will
> propbably
> > >> >start as asking questions.
> > >> >
> > >> >I'm not saying this should modify the plan just raising it as a
> potential
> > >> >source for pitfall.
> > >> >
> > >> >בתאריך יום ב׳, 27 במרץ 2023, 05:50, מאת Daniel Standish
> > >> >‏<da...@astronomer.io.invalid>:
> > >> >
> > >> >> Thanks Elad for the feedback.
> > >> >>
> > >> >> re 1. i don't really see a problem with the trigger rule being
> public.
> > >> The
> > >> >> way I see it, it's another trigger rule like any other trigger
> rule.
> > >> Every
> > >> >> trigger rule behaves differently, that's true here too. This one
> > >> happens to
> > >> >> be relied upon for teardown tasks.  That said, I don't think I
> would
> > >> >> necessarily be opposed to making it private.
> > >> >>
> > >> >> re 2, personally I kindof think it's a good thing.  My
> understanding
> > >> from
> > >> >> your comments is that with ShortCircuitOperator you can set it to
> skip
> > >> all
> > >> >> downstream or just skip the direct relatives.  To me this seems
> great
> > >> cus
> > >> >> it provides a way to either skip everything (including teardowns)
> or
> > >> just
> > >> >> the next task (thus potentially allowing teardowns to run).  To me
> this
> > >> is
> > >> >> another way in which by staying within the existing dependency and
> > >> trigger
> > >> >> rule paradigm we have more consistent, predictable, and
> configurable
> > >> >> behavior.  E.g. if we were not using normal deps and trigger
> rules, then
> > >> >> surely someone would have the opposite concern: "i want to use
> short
> > >> >> circuit operator to just skip all tasks including teardowns" and we
> > >> might
> > >> >> not be able to grant that wish, or at least not without more
> > >> development.
> > >> >> When you use an operator like this, you simply need to know what
> it does
> > >> >> and configure it in a manner appropriate for your use case.
> > >> >>
> > >>
>
> ---------------------------------------------------------------------
> To unsubscribe, e-mail: dev-unsubscribe@airflow.apache.org
> For additional commands, e-mail: dev-help@airflow.apache.org
>
>

Re: [DISCUSS] AIP-52 updates - setup / teardown tasks

Posted by Jarek Potiuk <ja...@potiuk.com>.
My view:

1) I am not sure if we should make it private (I am not even sure what
it would mean to be private:) ). But If it means that setting the rule
type for non-teardown task should raise an error (and of course
documenting this rule as only (and automatically) being applied to
teardown task - I am good with it.

2) Yes. The teardown must run in any case. I think documenting the
behaviour in ShortCircuit as a special case and - especially - adding
some examples showing that would be enough

And yeah while it is indeed an implementation detail, it is somewhat
"visible" as the public API is being concerned. So no wonder it came
as a bit of a surprise while implementing setup/teardown (happens that
something we consider as a detail for those deeply involved is a bit
of a surprise for those that came to look at it at review time).  I
guess this is somewhat a consequence of the way we operate in a
distributed environment and some details are being discussed in
smaller groups that are more focused on getting things done (we had a
few of those for AIP-44 for one).

But eventually we are discussing it now, so I think it is cool.

J.

On Mon, Mar 27, 2023 at 8:43 AM Ash Berlin-Taylor <as...@apache.org> wrote:
>
> If the set-up ran then the tear down _must_ run. No question.
>
> Nothing should be able to change this fact. If you can, then they don't fulfill the stated purpose of tear down tasks in the AIP: to tidy up resources created by a set up task.
>
> On 27 March 2023 06:22:47 BST, Daniel Standish <da...@astronomer.io.INVALID> wrote:
> >>
> >> When user set setup/teardown he has no idea unique trigger rule is set
> >> under the hood. The user also has no idea that trigger rules are even
> >> involved. That is not something he sees unless he checks the code of
> >> teardown and setup decorators.
> >
> >This means that users of ShortCircuitOperator will not even know they need
> >> to take action (until it wont work as expexted) and they will propbably
> >> start as asking questions.
> >
> >
> >Yeah, short circuit operator is a special operator that, if you're going to
> >use it, you ought to know how it works.  And we can easily add a note in
> >the docs that emphasizes its behavior on this point.  But I should point
> >out, the same would be true if setup and teardown were not implemented with
> >trigger rules; unless you investigate, read the docs, or look at the code,
> >you would not know whether teardowns would be skipped by short circuit.  So
> >either way it's something that we'd have to lean on docs for.  It's just
> >not something that a prudent user would just make an assumption about.
> >
> >If the set-up ran then the short circuit shouldn't be able to skip it
> >>
> >
> >I think this is overly and unnecessarily opinionated and limiting.  I would
> >not be concerned with having the default be one way or another, but to say
> >that "you should not be able to skip it", I disagree with the notion that
> >skipping a teardown with an operator specifically designed for the purpose
> >should be *forbidden,* as your comment suggests.
> >
> >take for example creating a cluster - you still want to delete it at the
> >> end even if you skipped all the other tasks!
> >>
> >
> >Yeah that's a perfectly fine example.  And, normally this is precisely how
> >this feature works: if your task throws a skip exception, its downstream
> >work tasks will be skipped but the teardown will run.  But what we're
> >talking about is a special operator whose unique purpose is to
> >short circuit the dag.  And as we know, every kind of dag you can imagine,
> >there's a user who needs it.  So someone will want to be able to skip a
> >teardown.  Teardowns aren't always going to be deleting a cluster.  They
> >might be deleting *files*, let's say.  Well, what if you have this special
> >operator in there to detect when something very bad happens and in that
> >case you want to stop all processing and leave the current state alone so
> >you can debug it.  Is there any good reason to forbid this?  I think no.
> >As stewards of this pipeline writing language, we should have sensible
> >defaults and maintain a good authoring interface, while allow users the
> >power to write what they need.
> >
> >
> >
> >
> >On Sun, Mar 26, 2023 at 9:19 PM Ash Berlin-Taylor <as...@apache.org> wrote:
> >
> >> If the set-up ran then the short circuit shouldn't be able to skip it:
> >> take for example creating a cluster - you still want to delete it at the
> >> end even if you skipped all the other tasks!
> >>
> >> This is precisely what I mean by set up and tear down tasks being special!
> >>
> >> On 27 March 2023 04:02:32 BST, Elad Kalif <el...@apache.org> wrote:
> >> >Thanks Daniel,
> >> >Let me clarify my concern.
> >> >
> >> >When user set setup/teardown he has no idea unique trigger rule is set
> >> >under the hood. The user also has no idea that trigger rules are even
> >> >involved. That is not something he sees unless he checks the code of
> >> >teardown and setup decorators.
> >> >
> >> >This means that users of ShortCircuitOperator will not even know they need
> >> >to take action (until it wont work as expexted) and they will propbably
> >> >start as asking questions.
> >> >
> >> >I'm not saying this should modify the plan just raising it as a potential
> >> >source for pitfall.
> >> >
> >> >בתאריך יום ב׳, 27 במרץ 2023, 05:50, מאת Daniel Standish
> >> >‏<da...@astronomer.io.invalid>:
> >> >
> >> >> Thanks Elad for the feedback.
> >> >>
> >> >> re 1. i don't really see a problem with the trigger rule being public.
> >> The
> >> >> way I see it, it's another trigger rule like any other trigger rule.
> >> Every
> >> >> trigger rule behaves differently, that's true here too. This one
> >> happens to
> >> >> be relied upon for teardown tasks.  That said, I don't think I would
> >> >> necessarily be opposed to making it private.
> >> >>
> >> >> re 2, personally I kindof think it's a good thing.  My understanding
> >> from
> >> >> your comments is that with ShortCircuitOperator you can set it to skip
> >> all
> >> >> downstream or just skip the direct relatives.  To me this seems great
> >> cus
> >> >> it provides a way to either skip everything (including teardowns) or
> >> just
> >> >> the next task (thus potentially allowing teardowns to run).  To me this
> >> is
> >> >> another way in which by staying within the existing dependency and
> >> trigger
> >> >> rule paradigm we have more consistent, predictable, and configurable
> >> >> behavior.  E.g. if we were not using normal deps and trigger rules, then
> >> >> surely someone would have the opposite concern: "i want to use short
> >> >> circuit operator to just skip all tasks including teardowns" and we
> >> might
> >> >> not be able to grant that wish, or at least not without more
> >> development.
> >> >> When you use an operator like this, you simply need to know what it does
> >> >> and configure it in a manner appropriate for your use case.
> >> >>
> >>

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@airflow.apache.org
For additional commands, e-mail: dev-help@airflow.apache.org


Re: [DISCUSS] AIP-52 updates - setup / teardown tasks

Posted by Ash Berlin-Taylor <as...@apache.org>.
If the set-up ran then the tear down _must_ run. No question.

Nothing should be able to change this fact. If you can, then they don't fulfill the stated purpose of tear down tasks in the AIP: to tidy up resources created by a set up task. 

On 27 March 2023 06:22:47 BST, Daniel Standish <da...@astronomer.io.INVALID> wrote:
>>
>> When user set setup/teardown he has no idea unique trigger rule is set
>> under the hood. The user also has no idea that trigger rules are even
>> involved. That is not something he sees unless he checks the code of
>> teardown and setup decorators.
>
>This means that users of ShortCircuitOperator will not even know they need
>> to take action (until it wont work as expexted) and they will propbably
>> start as asking questions.
>
>
>Yeah, short circuit operator is a special operator that, if you're going to
>use it, you ought to know how it works.  And we can easily add a note in
>the docs that emphasizes its behavior on this point.  But I should point
>out, the same would be true if setup and teardown were not implemented with
>trigger rules; unless you investigate, read the docs, or look at the code,
>you would not know whether teardowns would be skipped by short circuit.  So
>either way it's something that we'd have to lean on docs for.  It's just
>not something that a prudent user would just make an assumption about.
>
>If the set-up ran then the short circuit shouldn't be able to skip it
>>
>
>I think this is overly and unnecessarily opinionated and limiting.  I would
>not be concerned with having the default be one way or another, but to say
>that "you should not be able to skip it", I disagree with the notion that
>skipping a teardown with an operator specifically designed for the purpose
>should be *forbidden,* as your comment suggests.
>
>take for example creating a cluster - you still want to delete it at the
>> end even if you skipped all the other tasks!
>>
>
>Yeah that's a perfectly fine example.  And, normally this is precisely how
>this feature works: if your task throws a skip exception, its downstream
>work tasks will be skipped but the teardown will run.  But what we're
>talking about is a special operator whose unique purpose is to
>short circuit the dag.  And as we know, every kind of dag you can imagine,
>there's a user who needs it.  So someone will want to be able to skip a
>teardown.  Teardowns aren't always going to be deleting a cluster.  They
>might be deleting *files*, let's say.  Well, what if you have this special
>operator in there to detect when something very bad happens and in that
>case you want to stop all processing and leave the current state alone so
>you can debug it.  Is there any good reason to forbid this?  I think no.
>As stewards of this pipeline writing language, we should have sensible
>defaults and maintain a good authoring interface, while allow users the
>power to write what they need.
>
>
>
>
>On Sun, Mar 26, 2023 at 9:19 PM Ash Berlin-Taylor <as...@apache.org> wrote:
>
>> If the set-up ran then the short circuit shouldn't be able to skip it:
>> take for example creating a cluster - you still want to delete it at the
>> end even if you skipped all the other tasks!
>>
>> This is precisely what I mean by set up and tear down tasks being special!
>>
>> On 27 March 2023 04:02:32 BST, Elad Kalif <el...@apache.org> wrote:
>> >Thanks Daniel,
>> >Let me clarify my concern.
>> >
>> >When user set setup/teardown he has no idea unique trigger rule is set
>> >under the hood. The user also has no idea that trigger rules are even
>> >involved. That is not something he sees unless he checks the code of
>> >teardown and setup decorators.
>> >
>> >This means that users of ShortCircuitOperator will not even know they need
>> >to take action (until it wont work as expexted) and they will propbably
>> >start as asking questions.
>> >
>> >I'm not saying this should modify the plan just raising it as a potential
>> >source for pitfall.
>> >
>> >בתאריך יום ב׳, 27 במרץ 2023, 05:50, מאת Daniel Standish
>> >‏<da...@astronomer.io.invalid>:
>> >
>> >> Thanks Elad for the feedback.
>> >>
>> >> re 1. i don't really see a problem with the trigger rule being public.
>> The
>> >> way I see it, it's another trigger rule like any other trigger rule.
>> Every
>> >> trigger rule behaves differently, that's true here too. This one
>> happens to
>> >> be relied upon for teardown tasks.  That said, I don't think I would
>> >> necessarily be opposed to making it private.
>> >>
>> >> re 2, personally I kindof think it's a good thing.  My understanding
>> from
>> >> your comments is that with ShortCircuitOperator you can set it to skip
>> all
>> >> downstream or just skip the direct relatives.  To me this seems great
>> cus
>> >> it provides a way to either skip everything (including teardowns) or
>> just
>> >> the next task (thus potentially allowing teardowns to run).  To me this
>> is
>> >> another way in which by staying within the existing dependency and
>> trigger
>> >> rule paradigm we have more consistent, predictable, and configurable
>> >> behavior.  E.g. if we were not using normal deps and trigger rules, then
>> >> surely someone would have the opposite concern: "i want to use short
>> >> circuit operator to just skip all tasks including teardowns" and we
>> might
>> >> not be able to grant that wish, or at least not without more
>> development.
>> >> When you use an operator like this, you simply need to know what it does
>> >> and configure it in a manner appropriate for your use case.
>> >>
>>

Re: [DISCUSS] AIP-52 updates - setup / teardown tasks

Posted by Daniel Standish <da...@astronomer.io.INVALID>.
>
> When user set setup/teardown he has no idea unique trigger rule is set
> under the hood. The user also has no idea that trigger rules are even
> involved. That is not something he sees unless he checks the code of
> teardown and setup decorators.

This means that users of ShortCircuitOperator will not even know they need
> to take action (until it wont work as expexted) and they will propbably
> start as asking questions.


Yeah, short circuit operator is a special operator that, if you're going to
use it, you ought to know how it works.  And we can easily add a note in
the docs that emphasizes its behavior on this point.  But I should point
out, the same would be true if setup and teardown were not implemented with
trigger rules; unless you investigate, read the docs, or look at the code,
you would not know whether teardowns would be skipped by short circuit.  So
either way it's something that we'd have to lean on docs for.  It's just
not something that a prudent user would just make an assumption about.

If the set-up ran then the short circuit shouldn't be able to skip it
>

I think this is overly and unnecessarily opinionated and limiting.  I would
not be concerned with having the default be one way or another, but to say
that "you should not be able to skip it", I disagree with the notion that
skipping a teardown with an operator specifically designed for the purpose
should be *forbidden,* as your comment suggests.

take for example creating a cluster - you still want to delete it at the
> end even if you skipped all the other tasks!
>

Yeah that's a perfectly fine example.  And, normally this is precisely how
this feature works: if your task throws a skip exception, its downstream
work tasks will be skipped but the teardown will run.  But what we're
talking about is a special operator whose unique purpose is to
short circuit the dag.  And as we know, every kind of dag you can imagine,
there's a user who needs it.  So someone will want to be able to skip a
teardown.  Teardowns aren't always going to be deleting a cluster.  They
might be deleting *files*, let's say.  Well, what if you have this special
operator in there to detect when something very bad happens and in that
case you want to stop all processing and leave the current state alone so
you can debug it.  Is there any good reason to forbid this?  I think no.
As stewards of this pipeline writing language, we should have sensible
defaults and maintain a good authoring interface, while allow users the
power to write what they need.




On Sun, Mar 26, 2023 at 9:19 PM Ash Berlin-Taylor <as...@apache.org> wrote:

> If the set-up ran then the short circuit shouldn't be able to skip it:
> take for example creating a cluster - you still want to delete it at the
> end even if you skipped all the other tasks!
>
> This is precisely what I mean by set up and tear down tasks being special!
>
> On 27 March 2023 04:02:32 BST, Elad Kalif <el...@apache.org> wrote:
> >Thanks Daniel,
> >Let me clarify my concern.
> >
> >When user set setup/teardown he has no idea unique trigger rule is set
> >under the hood. The user also has no idea that trigger rules are even
> >involved. That is not something he sees unless he checks the code of
> >teardown and setup decorators.
> >
> >This means that users of ShortCircuitOperator will not even know they need
> >to take action (until it wont work as expexted) and they will propbably
> >start as asking questions.
> >
> >I'm not saying this should modify the plan just raising it as a potential
> >source for pitfall.
> >
> >בתאריך יום ב׳, 27 במרץ 2023, 05:50, מאת Daniel Standish
> >‏<da...@astronomer.io.invalid>:
> >
> >> Thanks Elad for the feedback.
> >>
> >> re 1. i don't really see a problem with the trigger rule being public.
> The
> >> way I see it, it's another trigger rule like any other trigger rule.
> Every
> >> trigger rule behaves differently, that's true here too. This one
> happens to
> >> be relied upon for teardown tasks.  That said, I don't think I would
> >> necessarily be opposed to making it private.
> >>
> >> re 2, personally I kindof think it's a good thing.  My understanding
> from
> >> your comments is that with ShortCircuitOperator you can set it to skip
> all
> >> downstream or just skip the direct relatives.  To me this seems great
> cus
> >> it provides a way to either skip everything (including teardowns) or
> just
> >> the next task (thus potentially allowing teardowns to run).  To me this
> is
> >> another way in which by staying within the existing dependency and
> trigger
> >> rule paradigm we have more consistent, predictable, and configurable
> >> behavior.  E.g. if we were not using normal deps and trigger rules, then
> >> surely someone would have the opposite concern: "i want to use short
> >> circuit operator to just skip all tasks including teardowns" and we
> might
> >> not be able to grant that wish, or at least not without more
> development.
> >> When you use an operator like this, you simply need to know what it does
> >> and configure it in a manner appropriate for your use case.
> >>
>

Re: [DISCUSS] AIP-52 updates - setup / teardown tasks

Posted by Ash Berlin-Taylor <as...@apache.org>.
If the set-up ran then the short circuit shouldn't be able to skip it: take for example creating a cluster - you still want to delete it at the end even if you skipped all the other tasks!

This is precisely what I mean by set up and tear down tasks being special!

On 27 March 2023 04:02:32 BST, Elad Kalif <el...@apache.org> wrote:
>Thanks Daniel,
>Let me clarify my concern.
>
>When user set setup/teardown he has no idea unique trigger rule is set
>under the hood. The user also has no idea that trigger rules are even
>involved. That is not something he sees unless he checks the code of
>teardown and setup decorators.
>
>This means that users of ShortCircuitOperator will not even know they need
>to take action (until it wont work as expexted) and they will propbably
>start as asking questions.
>
>I'm not saying this should modify the plan just raising it as a potential
>source for pitfall.
>
>בתאריך יום ב׳, 27 במרץ 2023, 05:50, מאת Daniel Standish
>‏<da...@astronomer.io.invalid>:
>
>> Thanks Elad for the feedback.
>>
>> re 1. i don't really see a problem with the trigger rule being public.  The
>> way I see it, it's another trigger rule like any other trigger rule.  Every
>> trigger rule behaves differently, that's true here too. This one happens to
>> be relied upon for teardown tasks.  That said, I don't think I would
>> necessarily be opposed to making it private.
>>
>> re 2, personally I kindof think it's a good thing.  My understanding from
>> your comments is that with ShortCircuitOperator you can set it to skip all
>> downstream or just skip the direct relatives.  To me this seems great cus
>> it provides a way to either skip everything (including teardowns) or just
>> the next task (thus potentially allowing teardowns to run).  To me this is
>> another way in which by staying within the existing dependency and trigger
>> rule paradigm we have more consistent, predictable, and configurable
>> behavior.  E.g. if we were not using normal deps and trigger rules, then
>> surely someone would have the opposite concern: "i want to use short
>> circuit operator to just skip all tasks including teardowns" and we might
>> not be able to grant that wish, or at least not without more development.
>> When you use an operator like this, you simply need to know what it does
>> and configure it in a manner appropriate for your use case.
>>

Re: [DISCUSS] AIP-52 updates - setup / teardown tasks

Posted by Elad Kalif <el...@apache.org>.
Thanks Daniel,
Let me clarify my concern.

When user set setup/teardown he has no idea unique trigger rule is set
under the hood. The user also has no idea that trigger rules are even
involved. That is not something he sees unless he checks the code of
teardown and setup decorators.

This means that users of ShortCircuitOperator will not even know they need
to take action (until it wont work as expexted) and they will propbably
start as asking questions.

I'm not saying this should modify the plan just raising it as a potential
source for pitfall.

בתאריך יום ב׳, 27 במרץ 2023, 05:50, מאת Daniel Standish
‏<da...@astronomer.io.invalid>:

> Thanks Elad for the feedback.
>
> re 1. i don't really see a problem with the trigger rule being public.  The
> way I see it, it's another trigger rule like any other trigger rule.  Every
> trigger rule behaves differently, that's true here too. This one happens to
> be relied upon for teardown tasks.  That said, I don't think I would
> necessarily be opposed to making it private.
>
> re 2, personally I kindof think it's a good thing.  My understanding from
> your comments is that with ShortCircuitOperator you can set it to skip all
> downstream or just skip the direct relatives.  To me this seems great cus
> it provides a way to either skip everything (including teardowns) or just
> the next task (thus potentially allowing teardowns to run).  To me this is
> another way in which by staying within the existing dependency and trigger
> rule paradigm we have more consistent, predictable, and configurable
> behavior.  E.g. if we were not using normal deps and trigger rules, then
> surely someone would have the opposite concern: "i want to use short
> circuit operator to just skip all tasks including teardowns" and we might
> not be able to grant that wish, or at least not without more development.
> When you use an operator like this, you simply need to know what it does
> and configure it in a manner appropriate for your use case.
>

Re: [DISCUSS] AIP-52 updates - setup / teardown tasks

Posted by Daniel Standish <da...@astronomer.io.INVALID>.
Thanks Elad for the feedback.

re 1. i don't really see a problem with the trigger rule being public.  The
way I see it, it's another trigger rule like any other trigger rule.  Every
trigger rule behaves differently, that's true here too. This one happens to
be relied upon for teardown tasks.  That said, I don't think I would
necessarily be opposed to making it private.

re 2, personally I kindof think it's a good thing.  My understanding from
your comments is that with ShortCircuitOperator you can set it to skip all
downstream or just skip the direct relatives.  To me this seems great cus
it provides a way to either skip everything (including teardowns) or just
the next task (thus potentially allowing teardowns to run).  To me this is
another way in which by staying within the existing dependency and trigger
rule paradigm we have more consistent, predictable, and configurable
behavior.  E.g. if we were not using normal deps and trigger rules, then
surely someone would have the opposite concern: "i want to use short
circuit operator to just skip all tasks including teardowns" and we might
not be able to grant that wish, or at least not without more development.
When you use an operator like this, you simply need to know what it does
and configure it in a manner appropriate for your use case.

Re: [DISCUSS] AIP-52 updates - setup / teardown tasks

Posted by Elad Kalif <el...@apache.org>.
I like to raise a (minor?) point about the new trigger rule (see
https://github.com/apache/airflow/pull/30270 ) which was not part of the
original design (at least not explicitly?)
which has some drawbacks that we need to be mindful about:

   1. The trigger rule class is public API. Adding an internal trigger rule
   into this class may cause confusion. We should at least mark it very
   clearly that the new trigger rule is not part of the public API.
   2. ShortCircuitOperator is a very popular operator and while
   https://github.com/apache/airflow/pull/20044 added the ability for this
   operator to honor trigger rules, in its default (and common) behavior it
   does not. This means that any solution involved with trigger rules might
   not work for DAGs with ShortCircuitOperator as the teardown task might
   never be executed.


On Fri, Mar 24, 2023 at 11:08 PM Ash Berlin-Taylor <as...@apache.org> wrote:

> Yeah I am solely focusing on the DAG authors' API - I trust Daniel and
> team (in not working on the impl of this AIP) to do whatever is right there.
>
> > I personally think we should encourage the context manager use case
> > for regular use, but leave the direct manipulation of dependencies for
> > power users as well.
>
> My point is I think context manager should be the only way of using
> setup/teardown tasks.
>
> They behave differently to normal tasks. So you should not be able to use
> them like normal tasks.
>
> On 24 March 2023 19:31:24 GMT, Jarek Potiuk <ja...@potiuk.com> wrote:
> >Yep I think we are all converging.
> >
> >I **think** the context manager is good (and I saw it initially in the
> >doc from Daniel) and I tend to agree this (or similar) syntactic sugar
> >will be the way people will interact with setup/teardown.
> >
> >I personally believe there are two slightly independent streams here
> >
> >a) how dag authors will interact with setup/teardown to express them in
> the DAG
> >b) how it is modelled in our DAG structure/task relations
> >
> >I think Daniel's "modification" of the AIP is really about b) - in
> >terms of re-using a lot of what we currently have in terms of
> >triggering rules and task relations to model the setup/teardown
> >behaviour
> >where what Ash is concerned about (I am a little concerned too) is a)
> >-> how dag authors will write their dags (and yes I imagine they will
> >use the context managers in vast majority of cases).
> >But I think both approaches can be combined.
> >
> >I personally think we should encourage the context manager use case
> >for regular use, but leave the direct manipulation of dependencies for
> >power users as well.
> >I am also in favour of cutting short and just failing DAGs that have
> >dangling setup/teardown to cut some confusion and implicitness it
> >brings in case it happens in the power-user scenario.
> >
> >This - I think should be a versatile and future proof (but deliverable
> >in a rather short time) approach and I tend to agree with Daniel that
> >multi-setup/multi-teardown will make the feature much more useful and
> >we will need it anyway.
> >Also by piggybacking on the existing relations/trigger rule mechanism
> >we might get it implemented much faster.
> >
> >BTW. I also think that this is much more straightforward approach:
> >
> >with setup_teardown(my_setup, my_teardown):
> >    my_work >> my_other_work
> >
> >or even that:
> >
> >with setup_teardown([my_setup1, my_setup2], [my_teardown]):
> >    my_work >> my_other_work
> >
> >
> >
> >J.
> >
> >On Fri, Mar 24, 2023 at 5:28 PM Daniel Standish
> ><da...@astronomer.io.invalid> wrote:
> >>
> >> Just want to thank you Ash for seriously engaging with the proposal and
> >> trying to find a solution to your concerns.  I am optimistic that we can
> >> find common ground and get this feature out there.
> >>
> >> OK so with that dag example we looked at very similar examples.  I'd
> love
> >> it if we could do this (and it's mentioned in the... very long proposal)
> >>
> >> chain(
> >> create_notification_channel.as_setup(),
> >> enable_notification_channel.as_setup(),
> >> disable_notification_channel,
> >> list_notification_channel,
> >> create_alert_policy.as_setup(),
> >> enable_alert_policy.as_setup(),
> >> disable_alert_policy,
> >> list_alert_policies,
> >> delete_notification_channel.teardown_for(create_notification_channel),
> >> delete_notification_channel_2.teardown_for(enable_notification_channel),
> >> delete_alert_policy.teardown_for(create_alert_policy),
> >> delete_alert_policy_2.teardown_for(enable_alert_policy),
> >> )
> >>
> >> So basically you could take an existing dag, not have to refactor it at
> >> all, and just convert the relevant tasks to be setup / teardown.
> >>
> >> Re context managers, we also considered this and included it in the
> >> proposal ("other ideas under consideration") and I think the idea is
> >> definitely promising.
> >>
> >> The example in the doc is this:
> >>
> >> with setup_teardown(my_setup, my_teardown):
> >>     my_work >> my_other_work
> >>
> >> Now, I'm not sure it's practical to forbid users from wiring things up
> >> manually, though we could explore that.  I would be in favor of
> encouraging
> >> but not requiring.  But if requiring is a compromise that will get us
> over
> >> the hump then maybe that's good enough, because we could always consider
> >> removing the constraint at a future date.
> >>
> >> So basically what you're saying is let's disallow `setup1 >> work`
> directly
> >> and require it be defined with a context manager...  Would have to come
> up
> >> with something for unmatched setup / unmatched teardown.
> >>
> >> I'm not sure about this syntax though:
> >> with create_notification_channel >> [
> >> delete_notification_channel,
> >> delete_notification_channel_2,
> >> ]:
> >>
> >> Seems it might not always be obvious where to insert the "work"...
> >>
> >> Crucially I want us to not let perfect be the enemy of good, and all
> this
> >> > confusion and discussion is exactly why I had originally placed
> "multiple
> >> > setup/teardown" in future work. Having a single setup function and a
> single
> >> > task group gives our users so much more power than they have right
> now.
> >> > I want this released, and keeping it simpler means we get it out
> sooner.
> >> > The last thing I want is a repeat of the DAG versioning API that
> Kaxil and
> >> > I tried where we got blocked.
> >>
> >>
> >> I understand the goal of deferring multiple setup / teardown, i.e. being
> >> incremental and MVP and all that.  But the thing is, I think it's
> >> absolutely essential that our choices do not cause trouble for multiple
> >> setup / teardowns in the future.  So even if we postpone enablement of
> >> multiple, or put it behind an experimental flag, I do not think we can
> >> postpone *consideration* of multiple -- before we release this I think
> we
> >> need to know roughly how we're going to enable that or else the feature
> is
> >> doomed.  We can't just say we'll deal with it later and paint ourselves
> >> into a corner.  That's why I have laid out a solution with a vision for
> how
> >> we do multiple.  Maybe there's a better solution out there and if there
> is
> >> I am happy to support it
> >>
> >>
> >>
> >>
> >>
> >>
> >>
> >>
> >> On Fri, Mar 24, 2023 at 8:12 AM Ash Berlin-Taylor <as...@apache.org>
> wrote:
> >>
> >> > Okay, after chatting with TP a bit this morning (due to being easy
> for me
> >> > to grab on Slack in an overlapping timezone) I think I've realised our
> >> > disconnect.
> >> >
> >> > We both want explicit, but what we see as explicit is different!
> >> >
> >> > To me, the explicit was "you've entered a task group" (or a DAG,
> because
> >> > all DAGs have a root task group) -- and that was why I think setup and
> >> > teardown should "bookend" the TG. And I think scope is an important
> concept
> >> > to introduce for setup and teardown to not be footguns for users
> learning
> >> > and living with this feature.
> >> >
> >> > TP came up with an alternative suggestion idea that I'm exploring
> here:
> >> > Let's make the scope explicit by requring setup/teardown tasks to be
> used
> >> > in a context manager!
> >> >
> >> > My main issue with using a dependendy based approach is exactly the
> >> > complex system test dags -- it's not explicit in the dep chain which
> tasks
> >> > are special and which are normal. Picking a system test from the
> Google
> >> > provider at random:
> >> >
> >> >
> >> >
> https://github.com/apache/airflow/blob/main/tests/system/providers/google/cloud/stackdriver/example_stackdriver.py#L213-L226
> >> >
> >> > chain(
> >> > create_notification_channel,
> >> > enable_notification_channel,
> >> > disable_notification_channel,
> >> > list_notification_channel,
> >> > create_alert_policy,
> >> > enable_alert_policy,
> >> > disable_alert_policy,
> >> > list_alert_policies,
> >> > delete_notification_channel,
> >> > delete_notification_channel_2,
> >> > delete_alert_policy,
> >> > delete_alert_policy_2,
> >> > )
> >> > We can guess which of those have special treatment based on the name,
> but
> >> > that's implicit to me. I can't look at that and know which tasks have
> >> > special behaviour, nor which tasks actually need the "resources"
> created by
> >> > a setup tsk. I like having a distinct call out in the python code
> that a
> >> > task is special.
> >> >
> >> > I think this would be much clearer and explicit as something like
> this:
> >> >
> >> > with create_notification_channel >> [
> >> > delete_notification_channel,
> >> > delete_notification_channel_2,
> >> > ]:
> >> > (
> >> > enable_notification_channel
> >> > >> disable_notification_channel
> >> > >> list_notification_channel
> >> > )
> >> > with create_alert_policy >> [delete_alert_policy,
> delete_alert_policy_2]:
> >> > enable_alert_policy >> disable_alert_policy >> list_alert_policies
> >> >
> >> > I think that is what the tasks actually need for this dag. Even if
> it's
> >> > not, I think this is a useful illustrative example. Having the
> explicit
> >> > with scoping in the DAG file to match the runtime behaviour is a
> strong
> >> > property that I think is important such that users can understand the
> >> > "lifetime" of resources created by tasks.
> >> > Here's another example, lets say the tasks need both resources, but
> the
> >> > resources don't actually depend on each other:
> >> > with create_notification_channel >> [
> >> > delete_notification_channel,
> >> > delete_notification_channel_2,
> >> > ], create_alert_policy >> [delete_alert_policy,
> delete_alert_policy_2]:
> >> > (
> >> > enable_notification_channel
> >> > >> disable_notification_channel
> >> > >> list_notification_channel
> >> > )
> >> > enable_alert_policy >> disable_alert_policy >> list_alert_policies
> >> >
> >> > ***
> >> > The key thing for me: setup and teardown tasks are not normal tasks,
> and
> >> > shouldn't be used, nor appear, as such in DAG code.
> >> > ***
> >> >
> >> > Crucially I want us to not let perfect be the enemy of good, and all
> this
> >> > confusion and discussion is exactly why I had originally placed
> "multiple
> >> > setup/teardown" in future work. Having a single setup function and a
> single
> >> > task group gives our users so much more power than they have right
> now.
> >> > I want this released, and keeping it simpler means we get it out
> sooner.
> >> > The last thing I want is a repeat of the DAG versioning API that
> Kaxil and
> >> > I tried where we got blocked.
> >> > -ash
> >> > On Mar 24 2023, at 12:24 am, Daniel Standish
> >> > <da...@astronomer.io.INVALID> wrote:
> >> > > re
> >> > >
> >> > > > 2. `task1 >> task2 >> teardown_task` to me falsely implies that
> >> > teardown
> >> > > > depends on task2, But it doesn't. It only depends on the "scope
> being
> >> > > > exited".
> >> > >
> >> > >
> >> > > So that's not quite the case. With the proposed implementation,
> there's
> >> > no
> >> > > such scope concept. They're just normal tasks, with special
> features.
> >> > > With the above code, teardown_task does depend on task2. A teardown
> will
> >> > > run if its setup is successful and its non-setup upstreams are done.
> >> > >
> >> > > re this one:
> >> > > with TaskGroup("tg1"):
> >> > > > task1 >> teardown_task
> >> > > > task2 >> task3 >> task4 >> task5
> >> > >
> >> > >
> >> > > With the proposed implementation, teardown runs after task 1 and
> doesn't
> >> > > wait for task2 or its downstreams. And the same rule applies.
> Teardown
> >> > > will run if its setups are successful and non-setup upstreams are
> done.
> >> > In
> >> > > this case there's no setup so it is in effect all_done.
> >> > >
> >> > > And one might ask, what's the purpose / benefit of having a teardown
> >> > here?
> >> > > Well, it's just that task1 requires the teardown and the others
> don't.
> >> > And,
> >> > > as a teardown, (1) even if it is happens to be a dag leaf, it can be
> >> > > ignored for the purpose of dag run state and (2) teardowns are
> ignored
> >> > as a
> >> > > task group leaf when arrowing tg1 >> next_task so that its success
> is not
> >> > > required for the dag to continue running after task1 is done.
> >> > >
> >> > > Adding a setup to the example doesn't change too much:
> >> > > with TaskGroup("tg1"):
> >> > > setup1 >> task1 >> teardown_task
> >> > > setup1 >> teardown_task
> >> > > task2 >> task3 >> task4 >> task5
> >> > >
> >> > > So in this case we still just have two parallel sequences of tasks.
> >> > > teardown_task will run if setup1 is successful and task1 is done.
> >> > >
> >> > > task2 and downstreams don't care about task 1 and its setups /
> teardowns.
> >> > > They are very much normal tasks that operate in the same way but
> you get
> >> > > the power of the special behaviors re clearing, configurable dag run
> >> > state
> >> > > impact, and continuing downstream in spite of failure.
> >> > >
> >> > > If task2 etc require the setup then you just add the arrows.
> >> > >
> >> > >
> >> > >
> >> > >
> >> > >
> >> > >
> >> > >
> >> > > On Thu, Mar 23, 2023 at 4:12 PM Ash Berlin-Taylor <as...@apache.org>
> >> > wrote:
> >> > > > I'm obviously in favour of the way the AIP was written, and
> that's for
> >> > two
> >> > > > primary reasons.
> >> > > >
> >> > > > 1. It's analogous to setup and teardown in testing frameworks
> where you
> >> > > > don't ever explicitly call them - the framework handles it for
> you.
> >> > > > 2. `task1 >> task2 >> teardown_task` to me falsely implies that
> >> > teardown
> >> > > > depends on task2, But it doesn't. It only depends on the "scope
> being
> >> > > > exited".
> >> > > >
> >> > > > And as for the Zen of Python point: python itself doesn't even
> follow
> >> > them
> >> > > > well. There are three ways of formatting strings in python.
> >> > > >
> >> > > > On thinking a bit more about it, I think I have a counter point to
> >> > where I
> >> > > > think explicit dependencies lead to a false expectation:
> >> > > >
> >> > > > ```
> >> > > > with TaskGroup("tg1"):
> >> > > > task1 ≥≥ teardown_task
> >> > > >
> >> > > > task2 >> task3 >> task4 >> task5
> >> > > > ```
> >> > > >
> >> > > > Does teardown run as soon as task 1 is finished, or when all of
> task1
> >> > and
> >> > > > task5 are finished?
> >> > > >
> >> > > > I very strongly believe that teardown should only run at the end
> of a
> >> > > > TaskGroup - a hard rule on this makes it easier for users to
> reason
> >> > about
> >> > > > and understand it. If it's only as a result of it's explicit
> >> > dependencies
> >> > > > then it means users have to reason about when each teardown task
> is
> >> > run in
> >> > > > each situation as it might be different from dag to dag.
> >> > > >
> >> > > > In this case the teardown is akin to a "finally" block in python,
> and
> >> > the
> >> > > > TaskGroup is the "try" block, which I hope is a concept that
> almost
> >> > > > everyone writing DAGs will understand and be able to relate too.
> >> > > >
> >> > > > Teardown tasks are already special in a number of ways (clearing
> >> > > > behaviour, special failure rules for resulting dag run, different
> >> > trigger
> >> > > > rule) so users need to know how it works.
> >> > > >
> >> > > > So I vote for keeping it implicit only, but if we as a community
> favour
> >> > > > explicit only then we need to have an enforced requirement that
> there
> >> > only
> >> > > > leaves of a TG can be teardown (if they are in use) -- i.e.
> `[taak1,
> >> > task5]
> >> > > > >> teardown_task` would be required in this case.
> >> > > >
> >> > > > (And a similar role for set up. If there are any, the only root
> tasks
> >> > in a
> >> > > > TG must be set up)
> >> > > >
> >> > > > Ash
> >> > > >
> >> > > > On 23 March 2023 22:16:42 GMT, Pierre Jeambrun <
> pierrejbrun@gmail.com>
> >> > > > wrote:
> >> > > > >I am also in favor of explicit relationships only.
> >> > > > >
> >> > > > >From a person who didn't work on AIP-52, it seems easier to
> understand
> >> > > > what
> >> > > > >is going on without having to dive into the setup/teardown
> >> > documentation.
> >> > > > >
> >> > > > >Le jeu. 23 mars 2023 à 22:53, Jed Cunningham <
> >> > jedcunningham@apache.org> a
> >> > > > >écrit :
> >> > > > >
> >> > > > >> I've been working closely with Daniel on AIP-52 for a while
> now, but
> >> > > > I'll
> >> > > > >> still share my thoughts here.
> >> > > > >>
> >> > > > >> I'm also in favor of only supporting explicit relationships.
> >> > > > >>
> >> > > > >> In my opinion, even if multiple setup/teardown per scope never
> >> > > > materialize,
> >> > > > >> explicit relationships are still a better choice due to the
> clarity
> >> > it
> >> > > > >> brings today.
> >> > > > >>
> >> > > >
> >> > >
> >> >
> >> >
> >
> >---------------------------------------------------------------------
> >To unsubscribe, e-mail: dev-unsubscribe@airflow.apache.org
> >For additional commands, e-mail: dev-help@airflow.apache.org
> >
>

Re: [DISCUSS] AIP-52 updates - setup / teardown tasks

Posted by Ash Berlin-Taylor <as...@apache.org>.
Yeah I am solely focusing on the DAG authors' API - I trust Daniel and team (in not working on the impl of this AIP) to do whatever is right there.

> I personally think we should encourage the context manager use case
> for regular use, but leave the direct manipulation of dependencies for
> power users as well.

My point is I think context manager should be the only way of using setup/teardown tasks.

They behave differently to normal tasks. So you should not be able to use them like normal tasks. 

On 24 March 2023 19:31:24 GMT, Jarek Potiuk <ja...@potiuk.com> wrote:
>Yep I think we are all converging.
>
>I **think** the context manager is good (and I saw it initially in the
>doc from Daniel) and I tend to agree this (or similar) syntactic sugar
>will be the way people will interact with setup/teardown.
>
>I personally believe there are two slightly independent streams here
>
>a) how dag authors will interact with setup/teardown to express them in the DAG
>b) how it is modelled in our DAG structure/task relations
>
>I think Daniel's "modification" of the AIP is really about b) - in
>terms of re-using a lot of what we currently have in terms of
>triggering rules and task relations to model the setup/teardown
>behaviour
>where what Ash is concerned about (I am a little concerned too) is a)
>-> how dag authors will write their dags (and yes I imagine they will
>use the context managers in vast majority of cases).
>But I think both approaches can be combined.
>
>I personally think we should encourage the context manager use case
>for regular use, but leave the direct manipulation of dependencies for
>power users as well.
>I am also in favour of cutting short and just failing DAGs that have
>dangling setup/teardown to cut some confusion and implicitness it
>brings in case it happens in the power-user scenario.
>
>This - I think should be a versatile and future proof (but deliverable
>in a rather short time) approach and I tend to agree with Daniel that
>multi-setup/multi-teardown will make the feature much more useful and
>we will need it anyway.
>Also by piggybacking on the existing relations/trigger rule mechanism
>we might get it implemented much faster.
>
>BTW. I also think that this is much more straightforward approach:
>
>with setup_teardown(my_setup, my_teardown):
>    my_work >> my_other_work
>
>or even that:
>
>with setup_teardown([my_setup1, my_setup2], [my_teardown]):
>    my_work >> my_other_work
>
>
>
>J.
>
>On Fri, Mar 24, 2023 at 5:28 PM Daniel Standish
><da...@astronomer.io.invalid> wrote:
>>
>> Just want to thank you Ash for seriously engaging with the proposal and
>> trying to find a solution to your concerns.  I am optimistic that we can
>> find common ground and get this feature out there.
>>
>> OK so with that dag example we looked at very similar examples.  I'd love
>> it if we could do this (and it's mentioned in the... very long proposal)
>>
>> chain(
>> create_notification_channel.as_setup(),
>> enable_notification_channel.as_setup(),
>> disable_notification_channel,
>> list_notification_channel,
>> create_alert_policy.as_setup(),
>> enable_alert_policy.as_setup(),
>> disable_alert_policy,
>> list_alert_policies,
>> delete_notification_channel.teardown_for(create_notification_channel),
>> delete_notification_channel_2.teardown_for(enable_notification_channel),
>> delete_alert_policy.teardown_for(create_alert_policy),
>> delete_alert_policy_2.teardown_for(enable_alert_policy),
>> )
>>
>> So basically you could take an existing dag, not have to refactor it at
>> all, and just convert the relevant tasks to be setup / teardown.
>>
>> Re context managers, we also considered this and included it in the
>> proposal ("other ideas under consideration") and I think the idea is
>> definitely promising.
>>
>> The example in the doc is this:
>>
>> with setup_teardown(my_setup, my_teardown):
>>     my_work >> my_other_work
>>
>> Now, I'm not sure it's practical to forbid users from wiring things up
>> manually, though we could explore that.  I would be in favor of encouraging
>> but not requiring.  But if requiring is a compromise that will get us over
>> the hump then maybe that's good enough, because we could always consider
>> removing the constraint at a future date.
>>
>> So basically what you're saying is let's disallow `setup1 >> work` directly
>> and require it be defined with a context manager...  Would have to come up
>> with something for unmatched setup / unmatched teardown.
>>
>> I'm not sure about this syntax though:
>> with create_notification_channel >> [
>> delete_notification_channel,
>> delete_notification_channel_2,
>> ]:
>>
>> Seems it might not always be obvious where to insert the "work"...
>>
>> Crucially I want us to not let perfect be the enemy of good, and all this
>> > confusion and discussion is exactly why I had originally placed "multiple
>> > setup/teardown" in future work. Having a single setup function and a single
>> > task group gives our users so much more power than they have right now.
>> > I want this released, and keeping it simpler means we get it out sooner.
>> > The last thing I want is a repeat of the DAG versioning API that Kaxil and
>> > I tried where we got blocked.
>>
>>
>> I understand the goal of deferring multiple setup / teardown, i.e. being
>> incremental and MVP and all that.  But the thing is, I think it's
>> absolutely essential that our choices do not cause trouble for multiple
>> setup / teardowns in the future.  So even if we postpone enablement of
>> multiple, or put it behind an experimental flag, I do not think we can
>> postpone *consideration* of multiple -- before we release this I think we
>> need to know roughly how we're going to enable that or else the feature is
>> doomed.  We can't just say we'll deal with it later and paint ourselves
>> into a corner.  That's why I have laid out a solution with a vision for how
>> we do multiple.  Maybe there's a better solution out there and if there is
>> I am happy to support it
>>
>>
>>
>>
>>
>>
>>
>>
>> On Fri, Mar 24, 2023 at 8:12 AM Ash Berlin-Taylor <as...@apache.org> wrote:
>>
>> > Okay, after chatting with TP a bit this morning (due to being easy for me
>> > to grab on Slack in an overlapping timezone) I think I've realised our
>> > disconnect.
>> >
>> > We both want explicit, but what we see as explicit is different!
>> >
>> > To me, the explicit was "you've entered a task group" (or a DAG, because
>> > all DAGs have a root task group) -- and that was why I think setup and
>> > teardown should "bookend" the TG. And I think scope is an important concept
>> > to introduce for setup and teardown to not be footguns for users learning
>> > and living with this feature.
>> >
>> > TP came up with an alternative suggestion idea that I'm exploring here:
>> > Let's make the scope explicit by requring setup/teardown tasks to be used
>> > in a context manager!
>> >
>> > My main issue with using a dependendy based approach is exactly the
>> > complex system test dags -- it's not explicit in the dep chain which tasks
>> > are special and which are normal. Picking a system test from the Google
>> > provider at random:
>> >
>> >
>> > https://github.com/apache/airflow/blob/main/tests/system/providers/google/cloud/stackdriver/example_stackdriver.py#L213-L226
>> >
>> > chain(
>> > create_notification_channel,
>> > enable_notification_channel,
>> > disable_notification_channel,
>> > list_notification_channel,
>> > create_alert_policy,
>> > enable_alert_policy,
>> > disable_alert_policy,
>> > list_alert_policies,
>> > delete_notification_channel,
>> > delete_notification_channel_2,
>> > delete_alert_policy,
>> > delete_alert_policy_2,
>> > )
>> > We can guess which of those have special treatment based on the name, but
>> > that's implicit to me. I can't look at that and know which tasks have
>> > special behaviour, nor which tasks actually need the "resources" created by
>> > a setup tsk. I like having a distinct call out in the python code that a
>> > task is special.
>> >
>> > I think this would be much clearer and explicit as something like this:
>> >
>> > with create_notification_channel >> [
>> > delete_notification_channel,
>> > delete_notification_channel_2,
>> > ]:
>> > (
>> > enable_notification_channel
>> > >> disable_notification_channel
>> > >> list_notification_channel
>> > )
>> > with create_alert_policy >> [delete_alert_policy, delete_alert_policy_2]:
>> > enable_alert_policy >> disable_alert_policy >> list_alert_policies
>> >
>> > I think that is what the tasks actually need for this dag. Even if it's
>> > not, I think this is a useful illustrative example. Having the explicit
>> > with scoping in the DAG file to match the runtime behaviour is a strong
>> > property that I think is important such that users can understand the
>> > "lifetime" of resources created by tasks.
>> > Here's another example, lets say the tasks need both resources, but the
>> > resources don't actually depend on each other:
>> > with create_notification_channel >> [
>> > delete_notification_channel,
>> > delete_notification_channel_2,
>> > ], create_alert_policy >> [delete_alert_policy, delete_alert_policy_2]:
>> > (
>> > enable_notification_channel
>> > >> disable_notification_channel
>> > >> list_notification_channel
>> > )
>> > enable_alert_policy >> disable_alert_policy >> list_alert_policies
>> >
>> > ***
>> > The key thing for me: setup and teardown tasks are not normal tasks, and
>> > shouldn't be used, nor appear, as such in DAG code.
>> > ***
>> >
>> > Crucially I want us to not let perfect be the enemy of good, and all this
>> > confusion and discussion is exactly why I had originally placed "multiple
>> > setup/teardown" in future work. Having a single setup function and a single
>> > task group gives our users so much more power than they have right now.
>> > I want this released, and keeping it simpler means we get it out sooner.
>> > The last thing I want is a repeat of the DAG versioning API that Kaxil and
>> > I tried where we got blocked.
>> > -ash
>> > On Mar 24 2023, at 12:24 am, Daniel Standish
>> > <da...@astronomer.io.INVALID> wrote:
>> > > re
>> > >
>> > > > 2. `task1 >> task2 >> teardown_task` to me falsely implies that
>> > teardown
>> > > > depends on task2, But it doesn't. It only depends on the "scope being
>> > > > exited".
>> > >
>> > >
>> > > So that's not quite the case. With the proposed implementation, there's
>> > no
>> > > such scope concept. They're just normal tasks, with special features.
>> > > With the above code, teardown_task does depend on task2. A teardown will
>> > > run if its setup is successful and its non-setup upstreams are done.
>> > >
>> > > re this one:
>> > > with TaskGroup("tg1"):
>> > > > task1 >> teardown_task
>> > > > task2 >> task3 >> task4 >> task5
>> > >
>> > >
>> > > With the proposed implementation, teardown runs after task 1 and doesn't
>> > > wait for task2 or its downstreams. And the same rule applies. Teardown
>> > > will run if its setups are successful and non-setup upstreams are done.
>> > In
>> > > this case there's no setup so it is in effect all_done.
>> > >
>> > > And one might ask, what's the purpose / benefit of having a teardown
>> > here?
>> > > Well, it's just that task1 requires the teardown and the others don't.
>> > And,
>> > > as a teardown, (1) even if it is happens to be a dag leaf, it can be
>> > > ignored for the purpose of dag run state and (2) teardowns are ignored
>> > as a
>> > > task group leaf when arrowing tg1 >> next_task so that its success is not
>> > > required for the dag to continue running after task1 is done.
>> > >
>> > > Adding a setup to the example doesn't change too much:
>> > > with TaskGroup("tg1"):
>> > > setup1 >> task1 >> teardown_task
>> > > setup1 >> teardown_task
>> > > task2 >> task3 >> task4 >> task5
>> > >
>> > > So in this case we still just have two parallel sequences of tasks.
>> > > teardown_task will run if setup1 is successful and task1 is done.
>> > >
>> > > task2 and downstreams don't care about task 1 and its setups / teardowns.
>> > > They are very much normal tasks that operate in the same way but you get
>> > > the power of the special behaviors re clearing, configurable dag run
>> > state
>> > > impact, and continuing downstream in spite of failure.
>> > >
>> > > If task2 etc require the setup then you just add the arrows.
>> > >
>> > >
>> > >
>> > >
>> > >
>> > >
>> > >
>> > > On Thu, Mar 23, 2023 at 4:12 PM Ash Berlin-Taylor <as...@apache.org>
>> > wrote:
>> > > > I'm obviously in favour of the way the AIP was written, and that's for
>> > two
>> > > > primary reasons.
>> > > >
>> > > > 1. It's analogous to setup and teardown in testing frameworks where you
>> > > > don't ever explicitly call them - the framework handles it for you.
>> > > > 2. `task1 >> task2 >> teardown_task` to me falsely implies that
>> > teardown
>> > > > depends on task2, But it doesn't. It only depends on the "scope being
>> > > > exited".
>> > > >
>> > > > And as for the Zen of Python point: python itself doesn't even follow
>> > them
>> > > > well. There are three ways of formatting strings in python.
>> > > >
>> > > > On thinking a bit more about it, I think I have a counter point to
>> > where I
>> > > > think explicit dependencies lead to a false expectation:
>> > > >
>> > > > ```
>> > > > with TaskGroup("tg1"):
>> > > > task1 ≥≥ teardown_task
>> > > >
>> > > > task2 >> task3 >> task4 >> task5
>> > > > ```
>> > > >
>> > > > Does teardown run as soon as task 1 is finished, or when all of task1
>> > and
>> > > > task5 are finished?
>> > > >
>> > > > I very strongly believe that teardown should only run at the end of a
>> > > > TaskGroup - a hard rule on this makes it easier for users to reason
>> > about
>> > > > and understand it. If it's only as a result of it's explicit
>> > dependencies
>> > > > then it means users have to reason about when each teardown task is
>> > run in
>> > > > each situation as it might be different from dag to dag.
>> > > >
>> > > > In this case the teardown is akin to a "finally" block in python, and
>> > the
>> > > > TaskGroup is the "try" block, which I hope is a concept that almost
>> > > > everyone writing DAGs will understand and be able to relate too.
>> > > >
>> > > > Teardown tasks are already special in a number of ways (clearing
>> > > > behaviour, special failure rules for resulting dag run, different
>> > trigger
>> > > > rule) so users need to know how it works.
>> > > >
>> > > > So I vote for keeping it implicit only, but if we as a community favour
>> > > > explicit only then we need to have an enforced requirement that there
>> > only
>> > > > leaves of a TG can be teardown (if they are in use) -- i.e. `[taak1,
>> > task5]
>> > > > >> teardown_task` would be required in this case.
>> > > >
>> > > > (And a similar role for set up. If there are any, the only root tasks
>> > in a
>> > > > TG must be set up)
>> > > >
>> > > > Ash
>> > > >
>> > > > On 23 March 2023 22:16:42 GMT, Pierre Jeambrun <pi...@gmail.com>
>> > > > wrote:
>> > > > >I am also in favor of explicit relationships only.
>> > > > >
>> > > > >From a person who didn't work on AIP-52, it seems easier to understand
>> > > > what
>> > > > >is going on without having to dive into the setup/teardown
>> > documentation.
>> > > > >
>> > > > >Le jeu. 23 mars 2023 à 22:53, Jed Cunningham <
>> > jedcunningham@apache.org> a
>> > > > >écrit :
>> > > > >
>> > > > >> I've been working closely with Daniel on AIP-52 for a while now, but
>> > > > I'll
>> > > > >> still share my thoughts here.
>> > > > >>
>> > > > >> I'm also in favor of only supporting explicit relationships.
>> > > > >>
>> > > > >> In my opinion, even if multiple setup/teardown per scope never
>> > > > materialize,
>> > > > >> explicit relationships are still a better choice due to the clarity
>> > it
>> > > > >> brings today.
>> > > > >>
>> > > >
>> > >
>> >
>> >
>
>---------------------------------------------------------------------
>To unsubscribe, e-mail: dev-unsubscribe@airflow.apache.org
>For additional commands, e-mail: dev-help@airflow.apache.org
>

Re: [DISCUSS] AIP-52 updates - setup / teardown tasks

Posted by Jarek Potiuk <ja...@potiuk.com>.
Yep I think we are all converging.

I **think** the context manager is good (and I saw it initially in the
doc from Daniel) and I tend to agree this (or similar) syntactic sugar
will be the way people will interact with setup/teardown.

I personally believe there are two slightly independent streams here

a) how dag authors will interact with setup/teardown to express them in the DAG
b) how it is modelled in our DAG structure/task relations

I think Daniel's "modification" of the AIP is really about b) - in
terms of re-using a lot of what we currently have in terms of
triggering rules and task relations to model the setup/teardown
behaviour
where what Ash is concerned about (I am a little concerned too) is a)
-> how dag authors will write their dags (and yes I imagine they will
use the context managers in vast majority of cases).
But I think both approaches can be combined.

I personally think we should encourage the context manager use case
for regular use, but leave the direct manipulation of dependencies for
power users as well.
I am also in favour of cutting short and just failing DAGs that have
dangling setup/teardown to cut some confusion and implicitness it
brings in case it happens in the power-user scenario.

This - I think should be a versatile and future proof (but deliverable
in a rather short time) approach and I tend to agree with Daniel that
multi-setup/multi-teardown will make the feature much more useful and
we will need it anyway.
Also by piggybacking on the existing relations/trigger rule mechanism
we might get it implemented much faster.

BTW. I also think that this is much more straightforward approach:

with setup_teardown(my_setup, my_teardown):
    my_work >> my_other_work

or even that:

with setup_teardown([my_setup1, my_setup2], [my_teardown]):
    my_work >> my_other_work



J.

On Fri, Mar 24, 2023 at 5:28 PM Daniel Standish
<da...@astronomer.io.invalid> wrote:
>
> Just want to thank you Ash for seriously engaging with the proposal and
> trying to find a solution to your concerns.  I am optimistic that we can
> find common ground and get this feature out there.
>
> OK so with that dag example we looked at very similar examples.  I'd love
> it if we could do this (and it's mentioned in the... very long proposal)
>
> chain(
> create_notification_channel.as_setup(),
> enable_notification_channel.as_setup(),
> disable_notification_channel,
> list_notification_channel,
> create_alert_policy.as_setup(),
> enable_alert_policy.as_setup(),
> disable_alert_policy,
> list_alert_policies,
> delete_notification_channel.teardown_for(create_notification_channel),
> delete_notification_channel_2.teardown_for(enable_notification_channel),
> delete_alert_policy.teardown_for(create_alert_policy),
> delete_alert_policy_2.teardown_for(enable_alert_policy),
> )
>
> So basically you could take an existing dag, not have to refactor it at
> all, and just convert the relevant tasks to be setup / teardown.
>
> Re context managers, we also considered this and included it in the
> proposal ("other ideas under consideration") and I think the idea is
> definitely promising.
>
> The example in the doc is this:
>
> with setup_teardown(my_setup, my_teardown):
>     my_work >> my_other_work
>
> Now, I'm not sure it's practical to forbid users from wiring things up
> manually, though we could explore that.  I would be in favor of encouraging
> but not requiring.  But if requiring is a compromise that will get us over
> the hump then maybe that's good enough, because we could always consider
> removing the constraint at a future date.
>
> So basically what you're saying is let's disallow `setup1 >> work` directly
> and require it be defined with a context manager...  Would have to come up
> with something for unmatched setup / unmatched teardown.
>
> I'm not sure about this syntax though:
> with create_notification_channel >> [
> delete_notification_channel,
> delete_notification_channel_2,
> ]:
>
> Seems it might not always be obvious where to insert the "work"...
>
> Crucially I want us to not let perfect be the enemy of good, and all this
> > confusion and discussion is exactly why I had originally placed "multiple
> > setup/teardown" in future work. Having a single setup function and a single
> > task group gives our users so much more power than they have right now.
> > I want this released, and keeping it simpler means we get it out sooner.
> > The last thing I want is a repeat of the DAG versioning API that Kaxil and
> > I tried where we got blocked.
>
>
> I understand the goal of deferring multiple setup / teardown, i.e. being
> incremental and MVP and all that.  But the thing is, I think it's
> absolutely essential that our choices do not cause trouble for multiple
> setup / teardowns in the future.  So even if we postpone enablement of
> multiple, or put it behind an experimental flag, I do not think we can
> postpone *consideration* of multiple -- before we release this I think we
> need to know roughly how we're going to enable that or else the feature is
> doomed.  We can't just say we'll deal with it later and paint ourselves
> into a corner.  That's why I have laid out a solution with a vision for how
> we do multiple.  Maybe there's a better solution out there and if there is
> I am happy to support it
>
>
>
>
>
>
>
>
> On Fri, Mar 24, 2023 at 8:12 AM Ash Berlin-Taylor <as...@apache.org> wrote:
>
> > Okay, after chatting with TP a bit this morning (due to being easy for me
> > to grab on Slack in an overlapping timezone) I think I've realised our
> > disconnect.
> >
> > We both want explicit, but what we see as explicit is different!
> >
> > To me, the explicit was "you've entered a task group" (or a DAG, because
> > all DAGs have a root task group) -- and that was why I think setup and
> > teardown should "bookend" the TG. And I think scope is an important concept
> > to introduce for setup and teardown to not be footguns for users learning
> > and living with this feature.
> >
> > TP came up with an alternative suggestion idea that I'm exploring here:
> > Let's make the scope explicit by requring setup/teardown tasks to be used
> > in a context manager!
> >
> > My main issue with using a dependendy based approach is exactly the
> > complex system test dags -- it's not explicit in the dep chain which tasks
> > are special and which are normal. Picking a system test from the Google
> > provider at random:
> >
> >
> > https://github.com/apache/airflow/blob/main/tests/system/providers/google/cloud/stackdriver/example_stackdriver.py#L213-L226
> >
> > chain(
> > create_notification_channel,
> > enable_notification_channel,
> > disable_notification_channel,
> > list_notification_channel,
> > create_alert_policy,
> > enable_alert_policy,
> > disable_alert_policy,
> > list_alert_policies,
> > delete_notification_channel,
> > delete_notification_channel_2,
> > delete_alert_policy,
> > delete_alert_policy_2,
> > )
> > We can guess which of those have special treatment based on the name, but
> > that's implicit to me. I can't look at that and know which tasks have
> > special behaviour, nor which tasks actually need the "resources" created by
> > a setup tsk. I like having a distinct call out in the python code that a
> > task is special.
> >
> > I think this would be much clearer and explicit as something like this:
> >
> > with create_notification_channel >> [
> > delete_notification_channel,
> > delete_notification_channel_2,
> > ]:
> > (
> > enable_notification_channel
> > >> disable_notification_channel
> > >> list_notification_channel
> > )
> > with create_alert_policy >> [delete_alert_policy, delete_alert_policy_2]:
> > enable_alert_policy >> disable_alert_policy >> list_alert_policies
> >
> > I think that is what the tasks actually need for this dag. Even if it's
> > not, I think this is a useful illustrative example. Having the explicit
> > with scoping in the DAG file to match the runtime behaviour is a strong
> > property that I think is important such that users can understand the
> > "lifetime" of resources created by tasks.
> > Here's another example, lets say the tasks need both resources, but the
> > resources don't actually depend on each other:
> > with create_notification_channel >> [
> > delete_notification_channel,
> > delete_notification_channel_2,
> > ], create_alert_policy >> [delete_alert_policy, delete_alert_policy_2]:
> > (
> > enable_notification_channel
> > >> disable_notification_channel
> > >> list_notification_channel
> > )
> > enable_alert_policy >> disable_alert_policy >> list_alert_policies
> >
> > ***
> > The key thing for me: setup and teardown tasks are not normal tasks, and
> > shouldn't be used, nor appear, as such in DAG code.
> > ***
> >
> > Crucially I want us to not let perfect be the enemy of good, and all this
> > confusion and discussion is exactly why I had originally placed "multiple
> > setup/teardown" in future work. Having a single setup function and a single
> > task group gives our users so much more power than they have right now.
> > I want this released, and keeping it simpler means we get it out sooner.
> > The last thing I want is a repeat of the DAG versioning API that Kaxil and
> > I tried where we got blocked.
> > -ash
> > On Mar 24 2023, at 12:24 am, Daniel Standish
> > <da...@astronomer.io.INVALID> wrote:
> > > re
> > >
> > > > 2. `task1 >> task2 >> teardown_task` to me falsely implies that
> > teardown
> > > > depends on task2, But it doesn't. It only depends on the "scope being
> > > > exited".
> > >
> > >
> > > So that's not quite the case. With the proposed implementation, there's
> > no
> > > such scope concept. They're just normal tasks, with special features.
> > > With the above code, teardown_task does depend on task2. A teardown will
> > > run if its setup is successful and its non-setup upstreams are done.
> > >
> > > re this one:
> > > with TaskGroup("tg1"):
> > > > task1 >> teardown_task
> > > > task2 >> task3 >> task4 >> task5
> > >
> > >
> > > With the proposed implementation, teardown runs after task 1 and doesn't
> > > wait for task2 or its downstreams. And the same rule applies. Teardown
> > > will run if its setups are successful and non-setup upstreams are done.
> > In
> > > this case there's no setup so it is in effect all_done.
> > >
> > > And one might ask, what's the purpose / benefit of having a teardown
> > here?
> > > Well, it's just that task1 requires the teardown and the others don't.
> > And,
> > > as a teardown, (1) even if it is happens to be a dag leaf, it can be
> > > ignored for the purpose of dag run state and (2) teardowns are ignored
> > as a
> > > task group leaf when arrowing tg1 >> next_task so that its success is not
> > > required for the dag to continue running after task1 is done.
> > >
> > > Adding a setup to the example doesn't change too much:
> > > with TaskGroup("tg1"):
> > > setup1 >> task1 >> teardown_task
> > > setup1 >> teardown_task
> > > task2 >> task3 >> task4 >> task5
> > >
> > > So in this case we still just have two parallel sequences of tasks.
> > > teardown_task will run if setup1 is successful and task1 is done.
> > >
> > > task2 and downstreams don't care about task 1 and its setups / teardowns.
> > > They are very much normal tasks that operate in the same way but you get
> > > the power of the special behaviors re clearing, configurable dag run
> > state
> > > impact, and continuing downstream in spite of failure.
> > >
> > > If task2 etc require the setup then you just add the arrows.
> > >
> > >
> > >
> > >
> > >
> > >
> > >
> > > On Thu, Mar 23, 2023 at 4:12 PM Ash Berlin-Taylor <as...@apache.org>
> > wrote:
> > > > I'm obviously in favour of the way the AIP was written, and that's for
> > two
> > > > primary reasons.
> > > >
> > > > 1. It's analogous to setup and teardown in testing frameworks where you
> > > > don't ever explicitly call them - the framework handles it for you.
> > > > 2. `task1 >> task2 >> teardown_task` to me falsely implies that
> > teardown
> > > > depends on task2, But it doesn't. It only depends on the "scope being
> > > > exited".
> > > >
> > > > And as for the Zen of Python point: python itself doesn't even follow
> > them
> > > > well. There are three ways of formatting strings in python.
> > > >
> > > > On thinking a bit more about it, I think I have a counter point to
> > where I
> > > > think explicit dependencies lead to a false expectation:
> > > >
> > > > ```
> > > > with TaskGroup("tg1"):
> > > > task1 ≥≥ teardown_task
> > > >
> > > > task2 >> task3 >> task4 >> task5
> > > > ```
> > > >
> > > > Does teardown run as soon as task 1 is finished, or when all of task1
> > and
> > > > task5 are finished?
> > > >
> > > > I very strongly believe that teardown should only run at the end of a
> > > > TaskGroup - a hard rule on this makes it easier for users to reason
> > about
> > > > and understand it. If it's only as a result of it's explicit
> > dependencies
> > > > then it means users have to reason about when each teardown task is
> > run in
> > > > each situation as it might be different from dag to dag.
> > > >
> > > > In this case the teardown is akin to a "finally" block in python, and
> > the
> > > > TaskGroup is the "try" block, which I hope is a concept that almost
> > > > everyone writing DAGs will understand and be able to relate too.
> > > >
> > > > Teardown tasks are already special in a number of ways (clearing
> > > > behaviour, special failure rules for resulting dag run, different
> > trigger
> > > > rule) so users need to know how it works.
> > > >
> > > > So I vote for keeping it implicit only, but if we as a community favour
> > > > explicit only then we need to have an enforced requirement that there
> > only
> > > > leaves of a TG can be teardown (if they are in use) -- i.e. `[taak1,
> > task5]
> > > > >> teardown_task` would be required in this case.
> > > >
> > > > (And a similar role for set up. If there are any, the only root tasks
> > in a
> > > > TG must be set up)
> > > >
> > > > Ash
> > > >
> > > > On 23 March 2023 22:16:42 GMT, Pierre Jeambrun <pi...@gmail.com>
> > > > wrote:
> > > > >I am also in favor of explicit relationships only.
> > > > >
> > > > >From a person who didn't work on AIP-52, it seems easier to understand
> > > > what
> > > > >is going on without having to dive into the setup/teardown
> > documentation.
> > > > >
> > > > >Le jeu. 23 mars 2023 à 22:53, Jed Cunningham <
> > jedcunningham@apache.org> a
> > > > >écrit :
> > > > >
> > > > >> I've been working closely with Daniel on AIP-52 for a while now, but
> > > > I'll
> > > > >> still share my thoughts here.
> > > > >>
> > > > >> I'm also in favor of only supporting explicit relationships.
> > > > >>
> > > > >> In my opinion, even if multiple setup/teardown per scope never
> > > > materialize,
> > > > >> explicit relationships are still a better choice due to the clarity
> > it
> > > > >> brings today.
> > > > >>
> > > >
> > >
> >
> >

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@airflow.apache.org
For additional commands, e-mail: dev-help@airflow.apache.org


Re: [DISCUSS] AIP-52 updates - setup / teardown tasks

Posted by Daniel Standish <da...@astronomer.io.INVALID>.
Just want to thank you Ash for seriously engaging with the proposal and
trying to find a solution to your concerns.  I am optimistic that we can
find common ground and get this feature out there.

OK so with that dag example we looked at very similar examples.  I'd love
it if we could do this (and it's mentioned in the... very long proposal)

chain(
create_notification_channel.as_setup(),
enable_notification_channel.as_setup(),
disable_notification_channel,
list_notification_channel,
create_alert_policy.as_setup(),
enable_alert_policy.as_setup(),
disable_alert_policy,
list_alert_policies,
delete_notification_channel.teardown_for(create_notification_channel),
delete_notification_channel_2.teardown_for(enable_notification_channel),
delete_alert_policy.teardown_for(create_alert_policy),
delete_alert_policy_2.teardown_for(enable_alert_policy),
)

So basically you could take an existing dag, not have to refactor it at
all, and just convert the relevant tasks to be setup / teardown.

Re context managers, we also considered this and included it in the
proposal ("other ideas under consideration") and I think the idea is
definitely promising.

The example in the doc is this:

with setup_teardown(my_setup, my_teardown):
    my_work >> my_other_work

Now, I'm not sure it's practical to forbid users from wiring things up
manually, though we could explore that.  I would be in favor of encouraging
but not requiring.  But if requiring is a compromise that will get us over
the hump then maybe that's good enough, because we could always consider
removing the constraint at a future date.

So basically what you're saying is let's disallow `setup1 >> work` directly
and require it be defined with a context manager...  Would have to come up
with something for unmatched setup / unmatched teardown.

I'm not sure about this syntax though:
with create_notification_channel >> [
delete_notification_channel,
delete_notification_channel_2,
]:

Seems it might not always be obvious where to insert the "work"...

Crucially I want us to not let perfect be the enemy of good, and all this
> confusion and discussion is exactly why I had originally placed "multiple
> setup/teardown" in future work. Having a single setup function and a single
> task group gives our users so much more power than they have right now.
> I want this released, and keeping it simpler means we get it out sooner.
> The last thing I want is a repeat of the DAG versioning API that Kaxil and
> I tried where we got blocked.


I understand the goal of deferring multiple setup / teardown, i.e. being
incremental and MVP and all that.  But the thing is, I think it's
absolutely essential that our choices do not cause trouble for multiple
setup / teardowns in the future.  So even if we postpone enablement of
multiple, or put it behind an experimental flag, I do not think we can
postpone *consideration* of multiple -- before we release this I think we
need to know roughly how we're going to enable that or else the feature is
doomed.  We can't just say we'll deal with it later and paint ourselves
into a corner.  That's why I have laid out a solution with a vision for how
we do multiple.  Maybe there's a better solution out there and if there is
I am happy to support it








On Fri, Mar 24, 2023 at 8:12 AM Ash Berlin-Taylor <as...@apache.org> wrote:

> Okay, after chatting with TP a bit this morning (due to being easy for me
> to grab on Slack in an overlapping timezone) I think I've realised our
> disconnect.
>
> We both want explicit, but what we see as explicit is different!
>
> To me, the explicit was "you've entered a task group" (or a DAG, because
> all DAGs have a root task group) -- and that was why I think setup and
> teardown should "bookend" the TG. And I think scope is an important concept
> to introduce for setup and teardown to not be footguns for users learning
> and living with this feature.
>
> TP came up with an alternative suggestion idea that I'm exploring here:
> Let's make the scope explicit by requring setup/teardown tasks to be used
> in a context manager!
>
> My main issue with using a dependendy based approach is exactly the
> complex system test dags -- it's not explicit in the dep chain which tasks
> are special and which are normal. Picking a system test from the Google
> provider at random:
>
>
> https://github.com/apache/airflow/blob/main/tests/system/providers/google/cloud/stackdriver/example_stackdriver.py#L213-L226
>
> chain(
> create_notification_channel,
> enable_notification_channel,
> disable_notification_channel,
> list_notification_channel,
> create_alert_policy,
> enable_alert_policy,
> disable_alert_policy,
> list_alert_policies,
> delete_notification_channel,
> delete_notification_channel_2,
> delete_alert_policy,
> delete_alert_policy_2,
> )
> We can guess which of those have special treatment based on the name, but
> that's implicit to me. I can't look at that and know which tasks have
> special behaviour, nor which tasks actually need the "resources" created by
> a setup tsk. I like having a distinct call out in the python code that a
> task is special.
>
> I think this would be much clearer and explicit as something like this:
>
> with create_notification_channel >> [
> delete_notification_channel,
> delete_notification_channel_2,
> ]:
> (
> enable_notification_channel
> >> disable_notification_channel
> >> list_notification_channel
> )
> with create_alert_policy >> [delete_alert_policy, delete_alert_policy_2]:
> enable_alert_policy >> disable_alert_policy >> list_alert_policies
>
> I think that is what the tasks actually need for this dag. Even if it's
> not, I think this is a useful illustrative example. Having the explicit
> with scoping in the DAG file to match the runtime behaviour is a strong
> property that I think is important such that users can understand the
> "lifetime" of resources created by tasks.
> Here's another example, lets say the tasks need both resources, but the
> resources don't actually depend on each other:
> with create_notification_channel >> [
> delete_notification_channel,
> delete_notification_channel_2,
> ], create_alert_policy >> [delete_alert_policy, delete_alert_policy_2]:
> (
> enable_notification_channel
> >> disable_notification_channel
> >> list_notification_channel
> )
> enable_alert_policy >> disable_alert_policy >> list_alert_policies
>
> ***
> The key thing for me: setup and teardown tasks are not normal tasks, and
> shouldn't be used, nor appear, as such in DAG code.
> ***
>
> Crucially I want us to not let perfect be the enemy of good, and all this
> confusion and discussion is exactly why I had originally placed "multiple
> setup/teardown" in future work. Having a single setup function and a single
> task group gives our users so much more power than they have right now.
> I want this released, and keeping it simpler means we get it out sooner.
> The last thing I want is a repeat of the DAG versioning API that Kaxil and
> I tried where we got blocked.
> -ash
> On Mar 24 2023, at 12:24 am, Daniel Standish
> <da...@astronomer.io.INVALID> wrote:
> > re
> >
> > > 2. `task1 >> task2 >> teardown_task` to me falsely implies that
> teardown
> > > depends on task2, But it doesn't. It only depends on the "scope being
> > > exited".
> >
> >
> > So that's not quite the case. With the proposed implementation, there's
> no
> > such scope concept. They're just normal tasks, with special features.
> > With the above code, teardown_task does depend on task2. A teardown will
> > run if its setup is successful and its non-setup upstreams are done.
> >
> > re this one:
> > with TaskGroup("tg1"):
> > > task1 >> teardown_task
> > > task2 >> task3 >> task4 >> task5
> >
> >
> > With the proposed implementation, teardown runs after task 1 and doesn't
> > wait for task2 or its downstreams. And the same rule applies. Teardown
> > will run if its setups are successful and non-setup upstreams are done.
> In
> > this case there's no setup so it is in effect all_done.
> >
> > And one might ask, what's the purpose / benefit of having a teardown
> here?
> > Well, it's just that task1 requires the teardown and the others don't.
> And,
> > as a teardown, (1) even if it is happens to be a dag leaf, it can be
> > ignored for the purpose of dag run state and (2) teardowns are ignored
> as a
> > task group leaf when arrowing tg1 >> next_task so that its success is not
> > required for the dag to continue running after task1 is done.
> >
> > Adding a setup to the example doesn't change too much:
> > with TaskGroup("tg1"):
> > setup1 >> task1 >> teardown_task
> > setup1 >> teardown_task
> > task2 >> task3 >> task4 >> task5
> >
> > So in this case we still just have two parallel sequences of tasks.
> > teardown_task will run if setup1 is successful and task1 is done.
> >
> > task2 and downstreams don't care about task 1 and its setups / teardowns.
> > They are very much normal tasks that operate in the same way but you get
> > the power of the special behaviors re clearing, configurable dag run
> state
> > impact, and continuing downstream in spite of failure.
> >
> > If task2 etc require the setup then you just add the arrows.
> >
> >
> >
> >
> >
> >
> >
> > On Thu, Mar 23, 2023 at 4:12 PM Ash Berlin-Taylor <as...@apache.org>
> wrote:
> > > I'm obviously in favour of the way the AIP was written, and that's for
> two
> > > primary reasons.
> > >
> > > 1. It's analogous to setup and teardown in testing frameworks where you
> > > don't ever explicitly call them - the framework handles it for you.
> > > 2. `task1 >> task2 >> teardown_task` to me falsely implies that
> teardown
> > > depends on task2, But it doesn't. It only depends on the "scope being
> > > exited".
> > >
> > > And as for the Zen of Python point: python itself doesn't even follow
> them
> > > well. There are three ways of formatting strings in python.
> > >
> > > On thinking a bit more about it, I think I have a counter point to
> where I
> > > think explicit dependencies lead to a false expectation:
> > >
> > > ```
> > > with TaskGroup("tg1"):
> > > task1 ≥≥ teardown_task
> > >
> > > task2 >> task3 >> task4 >> task5
> > > ```
> > >
> > > Does teardown run as soon as task 1 is finished, or when all of task1
> and
> > > task5 are finished?
> > >
> > > I very strongly believe that teardown should only run at the end of a
> > > TaskGroup - a hard rule on this makes it easier for users to reason
> about
> > > and understand it. If it's only as a result of it's explicit
> dependencies
> > > then it means users have to reason about when each teardown task is
> run in
> > > each situation as it might be different from dag to dag.
> > >
> > > In this case the teardown is akin to a "finally" block in python, and
> the
> > > TaskGroup is the "try" block, which I hope is a concept that almost
> > > everyone writing DAGs will understand and be able to relate too.
> > >
> > > Teardown tasks are already special in a number of ways (clearing
> > > behaviour, special failure rules for resulting dag run, different
> trigger
> > > rule) so users need to know how it works.
> > >
> > > So I vote for keeping it implicit only, but if we as a community favour
> > > explicit only then we need to have an enforced requirement that there
> only
> > > leaves of a TG can be teardown (if they are in use) -- i.e. `[taak1,
> task5]
> > > >> teardown_task` would be required in this case.
> > >
> > > (And a similar role for set up. If there are any, the only root tasks
> in a
> > > TG must be set up)
> > >
> > > Ash
> > >
> > > On 23 March 2023 22:16:42 GMT, Pierre Jeambrun <pi...@gmail.com>
> > > wrote:
> > > >I am also in favor of explicit relationships only.
> > > >
> > > >From a person who didn't work on AIP-52, it seems easier to understand
> > > what
> > > >is going on without having to dive into the setup/teardown
> documentation.
> > > >
> > > >Le jeu. 23 mars 2023 à 22:53, Jed Cunningham <
> jedcunningham@apache.org> a
> > > >écrit :
> > > >
> > > >> I've been working closely with Daniel on AIP-52 for a while now, but
> > > I'll
> > > >> still share my thoughts here.
> > > >>
> > > >> I'm also in favor of only supporting explicit relationships.
> > > >>
> > > >> In my opinion, even if multiple setup/teardown per scope never
> > > materialize,
> > > >> explicit relationships are still a better choice due to the clarity
> it
> > > >> brings today.
> > > >>
> > >
> >
>
>

Re: [DISCUSS] AIP-52 updates - setup / teardown tasks

Posted by Ash Berlin-Taylor <as...@apache.org>.
Okay, after chatting with TP a bit this morning (due to being easy for me to grab on Slack in an overlapping timezone) I think I've realised our disconnect.

We both want explicit, but what we see as explicit is different!

To me, the explicit was "you've entered a task group" (or a DAG, because all DAGs have a root task group) -- and that was why I think setup and teardown should "bookend" the TG. And I think scope is an important concept to introduce for setup and teardown to not be footguns for users learning and living with this feature.

TP came up with an alternative suggestion idea that I'm exploring here: Let's make the scope explicit by requring setup/teardown tasks to be used in a context manager!

My main issue with using a dependendy based approach is exactly the complex system test dags -- it's not explicit in the dep chain which tasks are special and which are normal. Picking a system test from the Google provider at random:

https://github.com/apache/airflow/blob/main/tests/system/providers/google/cloud/stackdriver/example_stackdriver.py#L213-L226

chain(
create_notification_channel,
enable_notification_channel,
disable_notification_channel,
list_notification_channel,
create_alert_policy,
enable_alert_policy,
disable_alert_policy,
list_alert_policies,
delete_notification_channel,
delete_notification_channel_2,
delete_alert_policy,
delete_alert_policy_2,
)
We can guess which of those have special treatment based on the name, but that's implicit to me. I can't look at that and know which tasks have special behaviour, nor which tasks actually need the "resources" created by a setup tsk. I like having a distinct call out in the python code that a task is special.

I think this would be much clearer and explicit as something like this:

with create_notification_channel >> [
delete_notification_channel,
delete_notification_channel_2,
]:
(
enable_notification_channel
>> disable_notification_channel
>> list_notification_channel
)
with create_alert_policy >> [delete_alert_policy, delete_alert_policy_2]:
enable_alert_policy >> disable_alert_policy >> list_alert_policies

I think that is what the tasks actually need for this dag. Even if it's not, I think this is a useful illustrative example. Having the explicit with scoping in the DAG file to match the runtime behaviour is a strong property that I think is important such that users can understand the "lifetime" of resources created by tasks.
Here's another example, lets say the tasks need both resources, but the resources don't actually depend on each other:
with create_notification_channel >> [
delete_notification_channel,
delete_notification_channel_2,
], create_alert_policy >> [delete_alert_policy, delete_alert_policy_2]:
(
enable_notification_channel
>> disable_notification_channel
>> list_notification_channel
)
enable_alert_policy >> disable_alert_policy >> list_alert_policies

***
The key thing for me: setup and teardown tasks are not normal tasks, and shouldn't be used, nor appear, as such in DAG code.
***

Crucially I want us to not let perfect be the enemy of good, and all this confusion and discussion is exactly why I had originally placed "multiple setup/teardown" in future work. Having a single setup function and a single task group gives our users so much more power than they have right now.
I want this released, and keeping it simpler means we get it out sooner. The last thing I want is a repeat of the DAG versioning API that Kaxil and I tried where we got blocked.
-ash
On Mar 24 2023, at 12:24 am, Daniel Standish <da...@astronomer.io.INVALID> wrote:
> re
>
> > 2. `task1 >> task2 >> teardown_task` to me falsely implies that teardown
> > depends on task2, But it doesn't. It only depends on the "scope being
> > exited".
>
>
> So that's not quite the case. With the proposed implementation, there's no
> such scope concept. They're just normal tasks, with special features.
> With the above code, teardown_task does depend on task2. A teardown will
> run if its setup is successful and its non-setup upstreams are done.
>
> re this one:
> with TaskGroup("tg1"):
> > task1 >> teardown_task
> > task2 >> task3 >> task4 >> task5
>
>
> With the proposed implementation, teardown runs after task 1 and doesn't
> wait for task2 or its downstreams. And the same rule applies. Teardown
> will run if its setups are successful and non-setup upstreams are done. In
> this case there's no setup so it is in effect all_done.
>
> And one might ask, what's the purpose / benefit of having a teardown here?
> Well, it's just that task1 requires the teardown and the others don't. And,
> as a teardown, (1) even if it is happens to be a dag leaf, it can be
> ignored for the purpose of dag run state and (2) teardowns are ignored as a
> task group leaf when arrowing tg1 >> next_task so that its success is not
> required for the dag to continue running after task1 is done.
>
> Adding a setup to the example doesn't change too much:
> with TaskGroup("tg1"):
> setup1 >> task1 >> teardown_task
> setup1 >> teardown_task
> task2 >> task3 >> task4 >> task5
>
> So in this case we still just have two parallel sequences of tasks.
> teardown_task will run if setup1 is successful and task1 is done.
>
> task2 and downstreams don't care about task 1 and its setups / teardowns.
> They are very much normal tasks that operate in the same way but you get
> the power of the special behaviors re clearing, configurable dag run state
> impact, and continuing downstream in spite of failure.
>
> If task2 etc require the setup then you just add the arrows.
>
>
>
>
>
>
>
> On Thu, Mar 23, 2023 at 4:12 PM Ash Berlin-Taylor <as...@apache.org> wrote:
> > I'm obviously in favour of the way the AIP was written, and that's for two
> > primary reasons.
> >
> > 1. It's analogous to setup and teardown in testing frameworks where you
> > don't ever explicitly call them - the framework handles it for you.
> > 2. `task1 >> task2 >> teardown_task` to me falsely implies that teardown
> > depends on task2, But it doesn't. It only depends on the "scope being
> > exited".
> >
> > And as for the Zen of Python point: python itself doesn't even follow them
> > well. There are three ways of formatting strings in python.
> >
> > On thinking a bit more about it, I think I have a counter point to where I
> > think explicit dependencies lead to a false expectation:
> >
> > ```
> > with TaskGroup("tg1"):
> > task1 ≥≥ teardown_task
> >
> > task2 >> task3 >> task4 >> task5
> > ```
> >
> > Does teardown run as soon as task 1 is finished, or when all of task1 and
> > task5 are finished?
> >
> > I very strongly believe that teardown should only run at the end of a
> > TaskGroup - a hard rule on this makes it easier for users to reason about
> > and understand it. If it's only as a result of it's explicit dependencies
> > then it means users have to reason about when each teardown task is run in
> > each situation as it might be different from dag to dag.
> >
> > In this case the teardown is akin to a "finally" block in python, and the
> > TaskGroup is the "try" block, which I hope is a concept that almost
> > everyone writing DAGs will understand and be able to relate too.
> >
> > Teardown tasks are already special in a number of ways (clearing
> > behaviour, special failure rules for resulting dag run, different trigger
> > rule) so users need to know how it works.
> >
> > So I vote for keeping it implicit only, but if we as a community favour
> > explicit only then we need to have an enforced requirement that there only
> > leaves of a TG can be teardown (if they are in use) -- i.e. `[taak1, task5]
> > >> teardown_task` would be required in this case.
> >
> > (And a similar role for set up. If there are any, the only root tasks in a
> > TG must be set up)
> >
> > Ash
> >
> > On 23 March 2023 22:16:42 GMT, Pierre Jeambrun <pi...@gmail.com>
> > wrote:
> > >I am also in favor of explicit relationships only.
> > >
> > >From a person who didn't work on AIP-52, it seems easier to understand
> > what
> > >is going on without having to dive into the setup/teardown documentation.
> > >
> > >Le jeu. 23 mars 2023 à 22:53, Jed Cunningham <je...@apache.org> a
> > >écrit :
> > >
> > >> I've been working closely with Daniel on AIP-52 for a while now, but
> > I'll
> > >> still share my thoughts here.
> > >>
> > >> I'm also in favor of only supporting explicit relationships.
> > >>
> > >> In my opinion, even if multiple setup/teardown per scope never
> > materialize,
> > >> explicit relationships are still a better choice due to the clarity it
> > >> brings today.
> > >>
> >
>


Re: [DISCUSS] AIP-52 updates - setup / teardown tasks

Posted by Daniel Standish <da...@astronomer.io.INVALID>.
re

> 2. `task1 >> task2 >> teardown_task` to me falsely implies that teardown
> depends on task2, But it doesn't. It only depends on the "scope being
> exited".


So that's not quite the case.  With the proposed implementation, there's no
such scope concept.  They're just normal tasks, with special features.
With the above code, teardown_task does depend on task2.  A teardown will
run if its setup is successful and its non-setup upstreams are done.

re this one:

with TaskGroup("tg1"):
>     task1 >> teardown_task
>     task2 >> task3 >> task4 >> task5


With the proposed implementation, teardown runs after task 1 and doesn't
wait for task2 or its downstreams.  And the same rule applies.  Teardown
will run if its setups are successful and non-setup upstreams are done.  In
this case there's no setup so it is in effect all_done.

And one might ask, what's the purpose / benefit of having a teardown here?
Well, it's just that task1 requires the teardown and the others don't. And,
as a teardown, (1) even if it is happens to be a dag leaf, it can be
ignored for the purpose of dag run state and (2) teardowns are ignored as a
task group leaf when arrowing tg1 >> next_task so that its success is not
required for the dag to continue running after task1 is done.

Adding a setup to the example doesn't change too much:

with TaskGroup("tg1"):
    setup1 >> task1 >> teardown_task
    setup1 >> teardown_task
    task2 >> task3 >> task4 >> task5

So in this case we still just have two parallel sequences of tasks.
teardown_task will run if setup1 is successful and task1 is done.

task2 and downstreams don't care about task 1 and its setups / teardowns.
They are very much normal tasks that operate in the same way but you get
the power of the special behaviors re clearing, configurable dag run state
impact, and continuing downstream in spite of failure.

If task2 etc require the setup then you just add the arrows.








On Thu, Mar 23, 2023 at 4:12 PM Ash Berlin-Taylor <as...@apache.org> wrote:

> I'm obviously in favour of the way the AIP was written, and that's for two
> primary reasons.
>
> 1. It's analogous to setup and teardown in testing frameworks where you
> don't ever explicitly call them - the framework handles it for you.
> 2. `task1 >> task2 >> teardown_task` to me falsely implies that teardown
> depends on task2, But it doesn't. It only depends on the "scope being
> exited".
>
> And as for the Zen of Python point: python itself doesn't even follow them
> well. There are three ways of formatting strings in python.
>
> On thinking a bit more about it, I think I have a counter point to where I
> think explicit dependencies lead to a false expectation:
>
> ```
> with TaskGroup("tg1"):
>     task1 ≥≥ teardown_task
>
>     task2 >> task3 >> task4 >> task5
> ```
>
> Does teardown run as soon as task 1 is finished, or when all of task1 and
> task5 are finished?
>
> I very strongly believe that teardown should only run at the end of a
> TaskGroup - a hard rule on this makes it easier for users to reason about
> and understand it. If it's only as a result of it's explicit dependencies
> then it means users have to reason about when each teardown task is run in
> each situation as it might be different from dag to dag.
>
> In this case the teardown is akin to a "finally" block in python, and the
> TaskGroup is the "try" block, which I hope is a concept that almost
> everyone writing DAGs will understand and be able to relate too.
>
> Teardown tasks are already special in a number of ways (clearing
> behaviour, special failure rules for resulting dag run, different trigger
> rule) so users need to know how it works.
>
> So I vote for keeping it implicit only, but if we as a community favour
> explicit only then we need to have an enforced requirement that there only
> leaves of a TG can be teardown (if they are in use) -- i.e. `[taak1, task5]
> >> teardown_task` would be required in this case.
>
> (And a similar role for set up. If there are any, the only root tasks in a
> TG must be set up)
>
> Ash
>
> On 23 March 2023 22:16:42 GMT, Pierre Jeambrun <pi...@gmail.com>
> wrote:
> >I am also in favor of explicit relationships only.
> >
> >From a person who didn't work on AIP-52, it seems easier to understand
> what
> >is going on without having to dive into the setup/teardown documentation.
> >
> >Le jeu. 23 mars 2023 à 22:53, Jed Cunningham <je...@apache.org> a
> >écrit :
> >
> >> I've been working closely with Daniel on AIP-52 for a while now, but
> I'll
> >> still share my thoughts here.
> >>
> >> I'm also in favor of only supporting explicit relationships.
> >>
> >> In my opinion, even if multiple setup/teardown per scope never
> materialize,
> >> explicit relationships are still a better choice due to the clarity it
> >> brings today.
> >>
>

Re: [DISCUSS] AIP-52 updates - setup / teardown tasks

Posted by Ash Berlin-Taylor <as...@apache.org>.
I'm obviously in favour of the way the AIP was written, and that's for two primary reasons.

1. It's analogous to setup and teardown in testing frameworks where you don't ever explicitly call them - the framework handles it for you.
2. `task1 >> task2 >> teardown_task` to me falsely implies that teardown depends on task2, But it doesn't. It only depends on the "scope being exited".

And as for the Zen of Python point: python itself doesn't even follow them well. There are three ways of formatting strings in python.

On thinking a bit more about it, I think I have a counter point to where I think explicit dependencies lead to a false expectation:

```
with TaskGroup("tg1"):
    task1 ≥≥ teardown_task

    task2 >> task3 >> task4 >> task5
```

Does teardown run as soon as task 1 is finished, or when all of task1 and task5 are finished?

I very strongly believe that teardown should only run at the end of a TaskGroup - a hard rule on this makes it easier for users to reason about and understand it. If it's only as a result of it's explicit dependencies then it means users have to reason about when each teardown task is run in each situation as it might be different from dag to dag.

In this case the teardown is akin to a "finally" block in python, and the TaskGroup is the "try" block, which I hope is a concept that almost everyone writing DAGs will understand and be able to relate too.

Teardown tasks are already special in a number of ways (clearing behaviour, special failure rules for resulting dag run, different trigger rule) so users need to know how it works.

So I vote for keeping it implicit only, but if we as a community favour explicit only then we need to have an enforced requirement that there only leaves of a TG can be teardown (if they are in use) -- i.e. `[taak1, task5] >> teardown_task` would be required in this case.

(And a similar role for set up. If there are any, the only root tasks in a TG must be set up)

Ash 

On 23 March 2023 22:16:42 GMT, Pierre Jeambrun <pi...@gmail.com> wrote:
>I am also in favor of explicit relationships only.
>
>From a person who didn't work on AIP-52, it seems easier to understand what
>is going on without having to dive into the setup/teardown documentation.
>
>Le jeu. 23 mars 2023 à 22:53, Jed Cunningham <je...@apache.org> a
>écrit :
>
>> I've been working closely with Daniel on AIP-52 for a while now, but I'll
>> still share my thoughts here.
>>
>> I'm also in favor of only supporting explicit relationships.
>>
>> In my opinion, even if multiple setup/teardown per scope never materialize,
>> explicit relationships are still a better choice due to the clarity it
>> brings today.
>>

Re: [DISCUSS] AIP-52 updates - setup / teardown tasks

Posted by Pierre Jeambrun <pi...@gmail.com>.
I am also in favor of explicit relationships only.

From a person who didn't work on AIP-52, it seems easier to understand what
is going on without having to dive into the setup/teardown documentation.

Le jeu. 23 mars 2023 à 22:53, Jed Cunningham <je...@apache.org> a
écrit :

> I've been working closely with Daniel on AIP-52 for a while now, but I'll
> still share my thoughts here.
>
> I'm also in favor of only supporting explicit relationships.
>
> In my opinion, even if multiple setup/teardown per scope never materialize,
> explicit relationships are still a better choice due to the clarity it
> brings today.
>

Re: [DISCUSS] AIP-52 updates - setup / teardown tasks

Posted by Jed Cunningham <je...@apache.org>.
I've been working closely with Daniel on AIP-52 for a while now, but I'll
still share my thoughts here.

I'm also in favor of only supporting explicit relationships.

In my opinion, even if multiple setup/teardown per scope never materialize,
explicit relationships are still a better choice due to the clarity it
brings today.

Re: [DISCUSS] AIP-52 updates - setup / teardown tasks

Posted by Jarek Potiuk <ja...@potiuk.com>.
Disclaimer - I've spent some time with Daniel discussing the options
and brainstorming some consequences of the change over the last few
days (er... evenings)  and that was quite a brain-teaser. So I
perfectly understand if it takes time and effort to digest.

But here is the digest of my thoughts about it.

I initially thought it is to complex and convoluted to be useful. But
I got to realise that it is much more "Airlfow-y" this way. Here is
why:

* I really, really like the explicitness of the proposal. The
"explicit is better than implicit" from the Zen of Python
https://peps.python.org/pep-0020/ PEP-0020 is in full effect here. I
think the @setup and @teardown were a bit too magical
* the ability to have multiple setup/teardowns and ability to use them
independently from DAG or "task group" are IMHO a key to success of
setup/teardown - we attempted to apply the patterns to existing system
test (not perfect but some of them are complex to show all the complex
cases to handle here were really helpful and ability to mark existing
operators as setup - each of them individually and ability to perform
a single teardown for multiple setups are the gist of the proposal
* I think while it seems complex at the definition time, it is
important to add visualisation for the setup/teardown scopes (on mouse
hover etc.) - this will prevent some potential problems, where DAG
authors might make mistakes with explicit dependencies
* knowing that we could - relatively easy - implement multiple
setup/teardown, being tied by a single setup/teardown for DAG/group
seems very limiting

I think it's a good improvement for the original AIP-52

J.

On Thu, Mar 23, 2023 at 6:13 PM Daniel Standish
<da...@astronomer.io.invalid> wrote:
>
> Hi, would like to clarify, in this thread we're specifically hoping to get
> community feedback on the proposal to drop the "implicit" logic.
>
> In the original AIP, if you instantiate a setup task in a group, in effect
> it's made the setup task for all tasks in the group.  And the proposal up
> for discussion here is that we don't make that assumption.
>
> So for example in the original AIP there's this example:
>
> from airflow import DAG, task, setup, teardown
>
>
> with DAG(dag_id='test'):
>     @setup
>     def create_cluster():
>         ...
>         return cluster_id
>
>     @task
>     def load(ti):
>         cluster_id = ti.xcom_pull(task_id="create_cluster")
>
>     def summarize():
>         ...
>
>     @teardown(on_failure_fail_dagrun=False)
>     def teardown_cluster():
>         ...
>        cluster_id = ti.xcom_pull(task_id="create_cluster")
>
>     create_cluster()
>     load() >> summarize()
>     teardown_cluster()
>
> *Proposal: require to set deps always*
>
> The proposal for discussion is that we require you to set the dependency :
>
> create_cluster() >> load() >> summarize() >> teardown_cluster()
>
> And the reason is both for compatiibility with multiple setup and teardown
> tasks, as well as greater flexibility even in the case of one setup and
> teardown pair. So for example it might be that your "setup" task isn't the
> first thing in the group.  E.g.
>
> with TaskGroup() as tg:
>     send_some_email >> create_cluster >> run_query >> delete_cluster
>     create_cluster >> delete_cluster
>
> Here create cluster is the setup (for the query to run) and delete cluster
> is the teardown, and sending the email just happens first.
>
> And then when thinking ahead to a world of multiple setup and teardown
> tasks, it becomes more important to specify deps.  Maybe some steps need to
> run in sequence, maybe others need to run in parallel.
>
> *Alternative 1: allow implicit in the simple case*
>
> We can try and have it both ways. So, continue to allow users to have
> "implicit" setups and teardowns, but only in the *simple* case, i.e. when
> there's only one setup and teardown in the group, and when no upstream /
> downstream has been set to it.
>
> So for example here we'd allow you to not arrow the setup:
>
> with TaskGroup() as tg:
>     my_setup()
>     my_task1() >> my_task2()
>     my_teardown()
>
> And Airflow would automatically add the relationship my_setup() >>
> my_task1() >> my_task2() >> my_teardown().
>
> But if there's more than one setup we'd force you to wire up the deps.  So
> this would either be strictly disallowed or would simply not work as
> expected:
>
> with TaskGroup() as tg:
>     my_setup1()
>     my_setup2()
>     my_task1() >> my_task2()
>     my_teardown1()
>     my_teardown2()
>
> *Alternative 2: only implicit*
>
> This alternative is essentially a complete rejection of the proposal.  You
> can't set deps between /  among setup tasks / teardown tasks / normal
> tasks.
>
> So this would be permitted and work as expected:
>
> with TaskGroup() as tg:
>     my_setup1()
>     my_task1() >> my_task2()
>     my_teardown2()
>
> This would not be permitted:
>
> with TaskGroup() as tg:
>     my_setup1() >> my_task1() >> my_task2() >> my_teardown2()
>
> And then if multiple setups / teardowns were ever to be supported, either
> you wouldn't be able to sequence the setups and teardowns, or we'd have to
> come up with some other way to define their sequencing aside from `>>`.
> And there would be no possibility of having setup tasks or teardown tasks
> anywhere but at the start or end of the group.
>
> We welcome your thoughts.  Thanks

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@airflow.apache.org
For additional commands, e-mail: dev-help@airflow.apache.org


Re: [DISCUSS] AIP-52 updates - setup / teardown tasks

Posted by Daniel Standish <da...@astronomer.io.INVALID>.
Hi, would like to clarify, in this thread we're specifically hoping to get
community feedback on the proposal to drop the "implicit" logic.

In the original AIP, if you instantiate a setup task in a group, in effect
it's made the setup task for all tasks in the group.  And the proposal up
for discussion here is that we don't make that assumption.

So for example in the original AIP there's this example:

from airflow import DAG, task, setup, teardown


with DAG(dag_id='test'):
    @setup
    def create_cluster():
        ...
        return cluster_id

    @task
    def load(ti):
        cluster_id = ti.xcom_pull(task_id="create_cluster")

    def summarize():
        ...

    @teardown(on_failure_fail_dagrun=False)
    def teardown_cluster():
        ...
       cluster_id = ti.xcom_pull(task_id="create_cluster")

    create_cluster()
    load() >> summarize()
    teardown_cluster()

*Proposal: require to set deps always*

The proposal for discussion is that we require you to set the dependency :

create_cluster() >> load() >> summarize() >> teardown_cluster()

And the reason is both for compatiibility with multiple setup and teardown
tasks, as well as greater flexibility even in the case of one setup and
teardown pair. So for example it might be that your "setup" task isn't the
first thing in the group.  E.g.

with TaskGroup() as tg:
    send_some_email >> create_cluster >> run_query >> delete_cluster
    create_cluster >> delete_cluster

Here create cluster is the setup (for the query to run) and delete cluster
is the teardown, and sending the email just happens first.

And then when thinking ahead to a world of multiple setup and teardown
tasks, it becomes more important to specify deps.  Maybe some steps need to
run in sequence, maybe others need to run in parallel.

*Alternative 1: allow implicit in the simple case*

We can try and have it both ways. So, continue to allow users to have
"implicit" setups and teardowns, but only in the *simple* case, i.e. when
there's only one setup and teardown in the group, and when no upstream /
downstream has been set to it.

So for example here we'd allow you to not arrow the setup:

with TaskGroup() as tg:
    my_setup()
    my_task1() >> my_task2()
    my_teardown()

And Airflow would automatically add the relationship my_setup() >>
my_task1() >> my_task2() >> my_teardown().

But if there's more than one setup we'd force you to wire up the deps.  So
this would either be strictly disallowed or would simply not work as
expected:

with TaskGroup() as tg:
    my_setup1()
    my_setup2()
    my_task1() >> my_task2()
    my_teardown1()
    my_teardown2()

*Alternative 2: only implicit*

This alternative is essentially a complete rejection of the proposal.  You
can't set deps between /  among setup tasks / teardown tasks / normal
tasks.

So this would be permitted and work as expected:

with TaskGroup() as tg:
    my_setup1()
    my_task1() >> my_task2()
    my_teardown2()

This would not be permitted:

with TaskGroup() as tg:
    my_setup1() >> my_task1() >> my_task2() >> my_teardown2()

And then if multiple setups / teardowns were ever to be supported, either
you wouldn't be able to sequence the setups and teardowns, or we'd have to
come up with some other way to define their sequencing aside from `>>`.
And there would be no possibility of having setup tasks or teardown tasks
anywhere but at the start or end of the group.

We welcome your thoughts.  Thanks