I’d personally find this very useful. There’s usually extra information I have 
about the DAG, and the current “docs_md” is usually not nearly sufficient 
enough as it’s poorly placed so if I start adding a lot of info it gets in the 
way of the regular UI. Also last I tested the markdown formatting didn’t work 
and neither did the other formatter options.

But I’m not sure how much other people have demand for this.

Thanks,
Damian

From: Collin McNulty <[email protected]>
Sent: Friday, August 20, 2021 16:36
To: [email protected]
Subject: Re: [DISCUSS] Adding better support for parametrized DAGs and dynamic 
DAGs using JSON/YAML dataformats

On the topic of pointing the code view to yaml, would we alternatively consider 
adding a view on the UI that would allow arbitrary text content? This could be 
accomplished by adding an optional parameter to the dag object that allowed you 
to pass text (or a filepath) that would then go through a renderer (e.g. 
markdown). It could be a readme, or yaml content or anything the author wanted.

Collin

On Fri, Aug 20, 2021 at 3:27 PM Shaw, Damian P. 
<[email protected]<mailto:[email protected]>> wrote:
FYI this is what I did on one of my past projects for Airflow.

The users wanted to write their DAGs as YAML files so my “DAG file” was a 
Python script that read the YAML files and converted them to DAGs. It was very 
easy to do and worked because of the flexibility of Airflow.

The one thing that would have been nice though is if I could of easily changed 
the “code view” in Airflow to point to the relevant YAML file instead of the 
less useful “DAG file”.

Damian

From: Jarek Potiuk <[email protected]<mailto:[email protected]>>
Sent: Friday, August 20, 2021 16:21
To: [email protected]<mailto:[email protected]>
Cc: [email protected]<mailto:[email protected]>
Subject: Re: [DISCUSS] Adding better support for parametrized DAGs and dynamic 
DAGs using JSON/YAML dataformats

Airflow DAGS are Python code.This is a very basic assumption - which is not 
likely to change. Ever.

And we are working on making it even more powerful. Writing DAGs in yaml/json 
makes them less powerful and less flexible. This is fine if you want to build 
on top of airflow and build a more declarative way of defining dags and use 
airflow to run it under the hood.
if you think there is a group of users who can benefit from that - cool. You 
can publish a code to convert those to Airflow DAGs and submit it to our 
Ecosystem page. There are plenty of tlike "CWL - Common Workflow Language" and 
others:
https://airflow.apache.org/ecosystem/#tools-integrating-with-airflow

J.

On Fri, Aug 20, 2021 at 2:48 PM Siddharth VP 
<[email protected]<mailto:[email protected]>> wrote:
Have we considered allowing dags in json/yaml formats before? I came up with a 
rather straightforward way to address parametrized and dynamic DAGs in Airflow, 
which I think makes dynamic dags work at scale.

Background / Current limitations:
1. Dynamic DAG generation using single-file 
methods<https://www.astronomer.io/guides/dynamically-generating-dags#single-file-methods>
 can cause scalability 
issues<https://www.astronomer.io/guides/dynamically-generating-dags#scalability>
 where there are too many active DAGs per file. The dag_file_processor_timeout 
is applied to the loader file, so all dynamically generated dags need to be 
processed in that time. Sure the timeout could be increased, but that may be 
undesirable (what if there are other static DAGs in the system on which we 
really want to enforce a small timeout?)
2. Parametrizing DAGs in Airflow is difficult. There is no good way to have 
multiple workflows that differ only by choices of some constants. Using 
TriggerDagRunOperator to trigger a generic DAG with conf doesn't give a 
native-ish experience as it creates DagRuns of the triggered dag rather than 
this dag - which also means a single scheduler log file.

Suggested approach:
1. User writes configuration files in JSON/YAML format. The schema can be 
arbitrary except for one condition that it must have a builder parameter with 
the path to a python file.
2. User writes the "builder" - a python file containing a make_dag method that 
receives the parsed json/yaml and returns a DAG object. (Just a sample 
strategy, we could instead say the file should contain a class that extends an 
abstract DagBuilder class.)
2. Airflow reads JSON/YAML files as well from the dags directory. It parses the 
file, imports the builder python file, and passes the parsed json/yaml to it 
and collects the generated DAG into the DagBag.

Sample implementation:
See 
https://github.com/siddharthvp/airflow/commit/47bad51fc4999737e9a300b134c04bbdbd04c88a;
 only major code change is in dagbag.py

Result:
Dag file processor logs show yaml/json file (instead of the builder python 
file). Each dynamically generated dag gets its own scheduler log file.
The configs dag_dir_list_interval, min_file_process_interval, 
file_parsing_sort_mode all directly apply to dag config files.
If the json/yaml fail to parse, it's registered as an import error.

Would like to know your thoughts on this. Thanks!
Siddharth VP


--
+48 660 796 129

==============================================================================
Please access the attached hyperlink for an important electronic communications 
disclaimer:
http://www.credit-suisse.com/legal/en/disclaimer_email_ib.html
==============================================================================

=============================================================================== 
Please access the attached hyperlink for an important electronic communications 
disclaimer: 
http://www.credit-suisse.com/legal/en/disclaimer_email_ib.html 
=============================================================================== 

Reply via email to