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
<daniel.stand...@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 <a...@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
> > <daniel.stand...@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 <a...@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 <pierrejb...@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 <
> > jedcunning...@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-unsubscr...@airflow.apache.org
For additional commands, e-mail: dev-h...@airflow.apache.org

Reply via email to