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