Workon sub-dags is much needed, I'm excited to see how this progresses.

>   - *Unpack SubDags during dag parsing*: This rewrites the *DagBag.bag_dag*
>   method to unpack subdag while parsing, and it will give a flat
> structure at
>   the task level

The serialized_dag representation already does this I think. At least if
I've understood your idea here correctly.

-ash


On Jun 12 2020, at 9:51 am, Xinbin Huang <bin.huan...@gmail.com> wrote:

> Hi everyone,
> 
> Sending a message to everyone and collect feedback on the AIP-34 on
> rewriting SubDagOperator. This was previously briefly mentioned in the
> discussion about what needs to be done for Airflow 2.0, and one of the
> ideas is to make SubDagOperator attach tasks back to the root DAG.
> 
> This AIP-34 focuses on solving SubDagOperator related issues by reattaching
> all tasks back to the root dag while respecting dependencies during
> parsing. The original grouping effect on the UI will be achieved through
> grouping related tasks by metadata.
> 
> This also makes the dag_factory function more reusable because you don't
> need to have parent_dag_name and child_dag_name in the function signature
> anymore.
> 
> Changes proposed:
> 
>   - *Unpack SubDags during dag parsing*: This rewrites the *DagBag.bag_dag*
>   method to unpack subdag while parsing, and it will give a flat
> structure at
>   the task level
>   - *Simplify SubDagOperator*: The new SubDagOperator acts like a
>   container and most of the original methods are removed. The
> signature is
>   also changed to *subdag_factory *with *subdag_args *and *subdag_kwargs*.
>   This is similar to the PythonOperator signature.
>   - *Add a TaskGroup model and add current_group & parent_group attributes
>   to BaseOperator*: This metadata is used to group tasks for rendering at
>   UI level. It may potentially extend further to group arbitrary tasks
>   outside the context of subdag to allow group-level operations (i.e.
>   stop/trigger a group of task within the dag)
>   - *Webserver UI for SubDag*: Proposed UI modification to allow
>   (un)collapse a group of tasks for a flat structure to pair with the first
>   change instead of the original hierarchical structure.
> 
> 
> Please see related documents and PRs for details:
> AIP:
> https://cwiki.apache.org/confluence/display/AIRFLOW/AIP-34+Rewrite+SubDagOperator
> 
> Original Issue: https://github.com/apache/airflow/issues/8078
> Draft PR: https://github.com/apache/airflow/pull/9243
> 
> Please let me know if there are any aspects that you agree/disagree
> with or
> need more clarification (especially the third change regarding TaskGroup).
> Any comments are welcome and I am looking forward to it!
> 
> Cheers
> Bin
> 

Reply via email to