Love it. I think it solves quite a number of existing cases but - more
importantly - opens up the possibility of even more cases. The Setup and
Teardown tasks might  be also used to manage the lifecycle of Dags and
Tasks by DAG authors for a number of other cases. I think one such case
(taken straight from the CI world) is to manage local cache needed by the
task. This case will be only possible/useful in the future incarnations of
the AIP-52 - when multiple setup/teardown tasks will be possible, I think.

It looks like it is a very well thought design/proposal.

For the record and summary for those who would not go to see the AIP doc
but want to track it in the discussion here - I added few comments and
suggestions in the doc:

* worth considering if we need the "create_cluster()" and
"taardown_cluster()" explicit method calls at all (seems that just
declaring setup/teardown should be enough)
* proposed simpler (and more straightforward) approach for classic
operators, I think the one proposed in the doc is pretty ugly and
counter-intuitive. I think we could have "asSetupTask()" "asTeardownTask()"
method calls that would register the classic task as setup/teardown and the
resulting syntax will be IMHO much simpler.
* stressed that one of the huge wins of this approach versus what is
currently possible is much better visualisation with much less of a clutter
(look most of our system tests and you will know what I mean)
* I think we could solve  the multiple setup/teardown tasks in a simple way
(proposed it) - but I fully agree it can be done as a follow-up.

J.


On Fri, Dec 16, 2022 at 5:04 PM Ash Berlin-Taylor <[email protected]> wrote:

> Hi everyone,
>
> I'd like to start a discussion about a new feature we'd like to add to
> Airflow we call "setup and tear down tasks"
>
>
> https://cwiki.apache.org/confluence/display/AIRFLOW/AIP-52+Automatic+setup+and+teardown+tasks
>
> Lets start with a code example, that if you are familiar with
> unittest.TestCase I hope will be familiar and you can guess what the
> effect would be:
>
> ```
> from airflow import DAG, task, setup, teardown
>
>
> with DAG(dag_id='test'):
>     @setup
>     def create_cluster():
>         ...
>         return cluster_id
>
>     @task
>     def load(ti):
>         # Example:
>         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()
> ```
>
> (This has been an itch of mine that I've had since I first started using
> Airflow back in 2017!)
>
> We go in to quite a bit of detail about the semantics and behaviours of
> these new task types, but the tl;dr (copied from the AIP doc)
>
> Add a new syntax for marking tasks as setup/teardown that:
>
>  * Teardown tasks will still run even if the upstream tasks have
>    otherwise failed
>  * Teardown tasks failing don't always result in the DagRun being marked
>    as failed (up to DAG author to choose failure mode)
>  * Automatically clear setup/teardown tasks when clearing a dependent
>    task
>
> I'm not sure when we'll start a vote on this due to the holiday season,
> but we'd like to start working on this in January.
>
> -ash
>

Reply via email to