Lineage

Note

Lineage support is very experimental and subject to change.

Airflow can help track origins of data, what happens to it and where it moves over time. This can aid having audit trails and data governance, but also debugging of data flows.

Airflow tracks data by means of inlets and outlets of the tasks. Let’s work from an example and see how it works.

  1. import datetime
  2. import pendulum
  3. from airflow.lineage import AUTO
  4. from airflow.lineage.entities import File
  5. from airflow.models import DAG
  6. from airflow.operators.bash import BashOperator
  7. from airflow.operators.empty import EmptyOperator
  8. FILE_CATEGORIES = ["CAT1", "CAT2", "CAT3"]
  9. dag = DAG(
  10. dag_id="example_lineage",
  11. start_date=pendulum.datetime(2021, 1, 1, tz="UTC"),
  12. schedule="0 0 * * *",
  13. catchup=False,
  14. dagrun_timeout=datetime.timedelta(minutes=60),
  15. )
  16. f_final = File(url="/tmp/final")
  17. run_this_last = EmptyOperator(task_id="run_this_last", dag=dag, inlets=AUTO, outlets=f_final)
  18. f_in = File(url="/tmp/whole_directory/")
  19. outlets = []
  20. for file in FILE_CATEGORIES:
  21. f_out = File(url="/tmp/{}/{{{{ data_interval_start }}}}".format(file))
  22. outlets.append(f_out)
  23. run_this = BashOperator(task_id="run_me_first", bash_command="echo 1", dag=dag, inlets=f_in, outlets=outlets)
  24. run_this.set_downstream(run_this_last)

Inlets can be a (list of) upstream task ids or statically defined as an attr annotated object as is, for example, the File object. Outlets can only be attr annotated object. Both are rendered at run time. However, the outlets of a task in case they are inlets to another task will not be re-rendered for the downstream task.

Note

Operators can add inlets and outlets automatically if the operator supports it.

In the example DAG task run_this (task_id=run_me_first) is a BashOperator that takes 3 inlets: CAT1, CAT2, CAT3, that are generated from a list. Note that data_interval_start is a templated field and will be rendered when the task is running.

Note

Behind the scenes Airflow prepares the lineage metadata as part of the pre_execute method of a task. When the task has finished execution post_execute is called and lineage metadata is pushed into XCOM. Thus if you are creating your own operators that override this method make sure to decorate your method with prepare_lineage and apply_lineage respectively.

Shorthand notation

Shorthand notation is available as well, this works almost equal to unix command line pipes, inputs and outputs. Note that operator precedence still applies. Also the | operator will only work when the left hand side either has outlets defined (e.g. by using add_outlets(..) or has out of the box support of lineage operator.supports_lineage == True.

  1. f_in > run_this | (run_this_last > outlets)

Lineage Backend

It’s possible to push the lineage metrics to a custom backend by providing an instance of a LineageBackend in the config:

  1. [lineage]
  2. backend = my.lineage.CustomBackend

The backend should inherit from airflow.lineage.LineageBackend.

  1. from airflow.lineage.backend import LineageBackend
  2. class CustomBackend(LineageBackend):
  3. def send_lineage(self, operator, inlets=None, outlets=None, context=None):
  4. ...
  5. # Send the info to some external service